public ExternalSortBatch(ExternalSort popConfig, FragmentContext context, RecordBatch incoming) throws OutOfMemoryException { super(popConfig, context, true); this.incoming = incoming; DrillConfig config = context.getConfig(); Configuration conf = new Configuration(); conf.set("fs.default.name", config.getString(ExecConstants.EXTERNAL_SORT_SPILL_FILESYSTEM)); try { this.fs = FileSystem.get(conf); } catch (IOException e) { throw new RuntimeException(e); } SPILL_BATCH_GROUP_SIZE = config.getInt(ExecConstants.EXTERNAL_SORT_SPILL_GROUP_SIZE); SPILL_THRESHOLD = config.getInt(ExecConstants.EXTERNAL_SORT_SPILL_THRESHOLD); dirs = Iterators.cycle(config.getStringList(ExecConstants.EXTERNAL_SORT_SPILL_DIRS)); oAllocator = oContext.getAllocator(); copierAllocator = oAllocator.newChildAllocator( oAllocator.getName() + ":copier", PriorityQueueCopier.INITIAL_ALLOCATION, PriorityQueueCopier.MAX_ALLOCATION); FragmentHandle handle = context.getHandle(); fileName = String.format( "%s_majorfragment%s_minorfragment%s_operator%s", QueryIdHelper.getQueryId(handle.getQueryId()), handle.getMajorFragmentId(), handle.getMinorFragmentId(), popConfig.getOperatorId()); }
/** * Constructor. Sets up the Foreman, but does not initiate any execution. * * @param bee used to submit additional work * @param drillbitContext * @param connection * @param queryId the id for the query * @param queryRequest the query to execute */ public Foreman( final WorkerBee bee, final DrillbitContext drillbitContext, final UserClientConnection connection, final QueryId queryId, final RunQuery queryRequest) { this.bee = bee; this.queryId = queryId; queryIdString = QueryIdHelper.getQueryId(queryId); this.queryRequest = queryRequest; this.drillbitContext = drillbitContext; initiatingClient = connection; this.closeFuture = initiatingClient.getChannel().closeFuture(); closeFuture.addListener(closeListener); queryContext = new QueryContext(connection.getSession(), drillbitContext); queryManager = new QueryManager( queryId, queryRequest, drillbitContext.getPersistentStoreProvider(), stateListener, this); // TODO reference escapes before ctor is complete via stateListener, this recordNewState(QueryState.PENDING); }
private void logQuerySummary() { try { LoggedQuery q = new LoggedQuery( QueryIdHelper.getQueryId(queryId), queryContext.getQueryContextInfo().getDefaultSchemaName(), queryText, new Date(queryContext.getQueryContextInfo().getQueryStartTime()), new Date(System.currentTimeMillis()), state, queryContext.getSession().getCredentials().getUserName()); queryLogger.info(MAPPER.writeValueAsString(q)); } catch (Exception e) { logger.error("Failure while recording query information to query log.", e); } }
public QueryContext( final UserSession session, final DrillbitContext drillbitContext, QueryId queryId) { this.drillbitContext = drillbitContext; this.session = session; queryOptions = new QueryOptionManager(session.getOptions()); executionControls = new ExecutionControls(queryOptions, drillbitContext.getEndpoint()); plannerSettings = new PlannerSettings(queryOptions, getFunctionRegistry()); plannerSettings.setNumEndPoints(drillbitContext.getBits().size()); table = new DrillOperatorTable(getFunctionRegistry(), drillbitContext.getOptionManager()); queryContextInfo = Utilities.createQueryContextInfo(session.getDefaultSchemaName()); contextInformation = new ContextInformation(session.getCredentials(), queryContextInfo); allocator = drillbitContext .getAllocator() .newChildAllocator( "query:" + QueryIdHelper.getQueryId(queryId), PlannerSettings.getInitialPlanningMemorySize(), plannerSettings.getPlanningMemoryLimit()); bufferManager = new BufferManagerImpl(this.allocator); viewExpansionContext = new ViewExpansionContext(this); schemaTreeProvider = new SchemaTreeProvider(drillbitContext); }
/** * Called by execution pool to do query setup, and kick off remote execution. * * <p>Note that completion of this function is not the end of the Foreman's role in the query's * lifecycle. */ @Override public void run() { // rename the thread we're using for debugging purposes final Thread currentThread = Thread.currentThread(); final String originalName = currentThread.getName(); currentThread.setName(QueryIdHelper.getQueryId(queryId) + ":foreman"); // track how long the query takes queryManager.markStartTime(); try { injector.injectChecked( queryContext.getExecutionControls(), "run-try-beginning", ForemanException.class); queryText = queryRequest.getPlan(); // convert a run query request into action switch (queryRequest.getType()) { case LOGICAL: parseAndRunLogicalPlan(queryRequest.getPlan()); break; case PHYSICAL: parseAndRunPhysicalPlan(queryRequest.getPlan()); break; case SQL: runSQL(queryRequest.getPlan()); break; default: throw new IllegalStateException(); } injector.injectChecked( queryContext.getExecutionControls(), "run-try-end", ForemanException.class); } catch (final OutOfMemoryException | OutOfMemoryRuntimeException e) { moveToState(QueryState.FAILED, UserException.memoryError(e).build(logger)); } catch (final ForemanException e) { moveToState(QueryState.FAILED, e); } catch (AssertionError | Exception ex) { moveToState( QueryState.FAILED, new ForemanException( "Unexpected exception during fragment initialization: " + ex.getMessage(), ex)); } catch (final OutOfMemoryError e) { if ("Direct buffer memory".equals(e.getMessage())) { moveToState( QueryState.FAILED, UserException.resourceError(e) .message("One or more nodes ran out of memory while executing the query.") .build(logger)); } else { /* * FragmentExecutors use a DrillbitStatusListener to watch out for the death of their query's Foreman. So, if we * die here, they should get notified about that, and cancel themselves; we don't have to attempt to notify * them, which might not work under these conditions. */ System.out.println("Node ran out of Heap memory, exiting."); e.printStackTrace(); System.out.flush(); System.exit(-1); } } finally { /* * Begin accepting external events. * * Doing this here in the finally clause will guarantee that it occurs. Otherwise, if there * is an exception anywhere during setup, it wouldn't occur, and any events that are generated * as a result of any partial setup that was done (such as the FragmentSubmitListener, * the ResponseSendListener, or an external call to cancel()), will hang the thread that makes the * event delivery call. * * If we do throw an exception during setup, and have already moved to QueryState.FAILED, we just need to * make sure that we can't make things any worse as those events are delivered, but allow * any necessary remaining cleanup to proceed. * * Note that cancellations cannot be simulated before this point, i.e. pauses can be injected, because Foreman * would wait on the cancelling thread to signal a resume and the cancelling thread would wait on the Foreman * to accept events. */ acceptExternalEvents.countDown(); // If we received the resume signal before fragments are setup, the first call does not // actually resume the // fragments. Since setup is done, all fragments must have been delivered to remote nodes. Now // we can resume. if (resume) { resume(); } injector.injectPause(queryContext.getExecutionControls(), "foreman-ready", logger); // restore the thread's original name currentThread.setName(originalName); } /* * Note that despite the run() completing, the Foreman continues to exist, and receives * events (indirectly, through the QueryManager's use of stateListener), about fragment * completions. It won't go away until everything is completed, failed, or cancelled. */ }