/** * This limits the number of "small" and "large" queries that a Drill cluster will run * simultaneously, if queueing is enabled. If the query is unable to run, this will block until it * can. Beware that this is called under run(), and so will consume a Thread while it waits for * the required distributed semaphore. * * @param plan the query plan * @throws ForemanSetupException */ private void acquireQuerySemaphore(final PhysicalPlan plan) throws ForemanSetupException { final OptionManager optionManager = queryContext.getOptions(); final boolean queuingEnabled = optionManager.getOption(ExecConstants.ENABLE_QUEUE); if (queuingEnabled) { final long queueThreshold = optionManager.getOption(ExecConstants.QUEUE_THRESHOLD_SIZE); double totalCost = 0; for (final PhysicalOperator ops : plan.getSortedOperators()) { totalCost += ops.getCost(); } final long queueTimeout = optionManager.getOption(ExecConstants.QUEUE_TIMEOUT); final String queueName; try { @SuppressWarnings("resource") final ClusterCoordinator clusterCoordinator = drillbitContext.getClusterCoordinator(); final DistributedSemaphore distributedSemaphore; // get the appropriate semaphore if (totalCost > queueThreshold) { final int largeQueue = (int) optionManager.getOption(ExecConstants.LARGE_QUEUE_SIZE); distributedSemaphore = clusterCoordinator.getSemaphore("query.large", largeQueue); queueName = "large"; } else { final int smallQueue = (int) optionManager.getOption(ExecConstants.SMALL_QUEUE_SIZE); distributedSemaphore = clusterCoordinator.getSemaphore("query.small", smallQueue); queueName = "small"; } lease = distributedSemaphore.acquire(queueTimeout, TimeUnit.MILLISECONDS); } catch (final Exception e) { throw new ForemanSetupException("Unable to acquire slot for query.", e); } if (lease == null) { throw UserException.resourceError() .message( "Unable to acquire queue resources for query within timeout. Timeout for %s queue was set at %d seconds.", queueName, queueTimeout / 1000) .build(logger); } } }
/** * 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. */ }
public BatchGroup mergeAndSpill(LinkedList<BatchGroup> batchGroups) throws SchemaChangeException { logger.debug("Copier allocator current allocation {}", copierAllocator.getAllocatedMemory()); logger.debug( "mergeAndSpill: starting total size in memory = {}", oAllocator.getAllocatedMemory()); VectorContainer outputContainer = new VectorContainer(); List<BatchGroup> batchGroupList = Lists.newArrayList(); int batchCount = batchGroups.size(); for (int i = 0; i < batchCount / 2; i++) { if (batchGroups.size() == 0) { break; } BatchGroup batch = batchGroups.pollLast(); assert batch != null : "Encountered a null batch during merge and spill operation"; batchGroupList.add(batch); } if (batchGroupList.size() == 0) { return null; } int estimatedRecordSize = 0; for (VectorWrapper<?> w : batchGroupList.get(0)) { try { estimatedRecordSize += TypeHelper.getSize(w.getField().getType()); } catch (UnsupportedOperationException e) { estimatedRecordSize += 50; } } int targetRecordCount = Math.max(1, COPIER_BATCH_MEM_LIMIT / estimatedRecordSize); VectorContainer hyperBatch = constructHyperBatch(batchGroupList); createCopier(hyperBatch, batchGroupList, outputContainer, true); int count = copier.next(targetRecordCount); assert count > 0; logger.debug( "mergeAndSpill: estimated record size = {}, target record count = {}", estimatedRecordSize, targetRecordCount); // 1 output container is kept in memory, so we want to hold on to it and transferClone // allows keeping ownership VectorContainer c1 = VectorContainer.getTransferClone(outputContainer, oContext); c1.buildSchema(BatchSchema.SelectionVectorMode.NONE); c1.setRecordCount(count); String spillDir = dirs.next(); Path currSpillPath = new Path(Joiner.on("/").join(spillDir, fileName)); currSpillDirs.add(currSpillPath); String outputFile = Joiner.on("/").join(currSpillPath, spillCount++); try { fs.deleteOnExit(currSpillPath); } catch (IOException e) { // since this is meant to be used in a batches's spilling, we don't propagate the exception logger.warn("Unable to mark spill directory " + currSpillPath + " for deleting on exit", e); } stats.setLongStat(Metric.SPILL_COUNT, spillCount); BatchGroup newGroup = new BatchGroup(c1, fs, outputFile, oContext); try (AutoCloseable a = AutoCloseables.all(batchGroupList)) { logger.info("Merging and spilling to {}", outputFile); while ((count = copier.next(targetRecordCount)) > 0) { outputContainer.buildSchema(BatchSchema.SelectionVectorMode.NONE); outputContainer.setRecordCount(count); // note that addBatch also clears the outputContainer newGroup.addBatch(outputContainer); } injector.injectChecked( context.getExecutionControls(), INTERRUPTION_WHILE_SPILLING, IOException.class); newGroup.closeOutputStream(); } catch (Throwable e) { // we only need to cleanup newGroup if spill failed try { AutoCloseables.close(e, newGroup); } catch (Throwable t) { /* close() may hit the same IO issue; just ignore */ } throw UserException.resourceError(e) .message("External Sort encountered an error while spilling to disk") .addContext(e.getMessage() /* more detail */) .build(logger); } finally { hyperBatch.clear(); } logger.debug("mergeAndSpill: final total size in memory = {}", oAllocator.getAllocatedMemory()); logger.info("Completed spilling to {}", outputFile); return newGroup; }