/** * 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. */ }
@Override public void run() { // if a cancel thread has already entered this executor, we have not reason to continue. if (!hasCloseoutThread.compareAndSet(false, true)) { return; } final Thread myThread = Thread.currentThread(); myThreadRef.set(myThread); final String originalThreadName = myThread.getName(); final FragmentHandle fragmentHandle = fragmentContext.getHandle(); final DrillbitContext drillbitContext = fragmentContext.getDrillbitContext(); final ClusterCoordinator clusterCoordinator = drillbitContext.getClusterCoordinator(); final DrillbitStatusListener drillbitStatusListener = new FragmentDrillbitStatusListener(); final String newThreadName = QueryIdHelper.getExecutorThreadName(fragmentHandle); try { myThread.setName(newThreadName); // if we didn't get the root operator when the executor was created, create it now. final FragmentRoot rootOperator = this.rootOperator != null ? this.rootOperator : drillbitContext.getPlanReader().readFragmentOperator(fragment.getFragmentJson()); root = ImplCreator.getExec(fragmentContext, rootOperator); if (root == null) { return; } clusterCoordinator.addDrillbitStatusListener(drillbitStatusListener); updateState(FragmentState.RUNNING); acceptExternalEvents.countDown(); injector.injectPause(fragmentContext.getExecutionControls(), "fragment-running", logger); final DrillbitEndpoint endpoint = drillbitContext.getEndpoint(); logger.debug( "Starting fragment {}:{} on {}:{}", fragmentHandle.getMajorFragmentId(), fragmentHandle.getMinorFragmentId(), endpoint.getAddress(), endpoint.getUserPort()); final UserGroupInformation queryUserUgi = fragmentContext.isImpersonationEnabled() ? ImpersonationUtil.createProxyUgi(fragmentContext.getQueryUserName()) : ImpersonationUtil.getProcessUserUGI(); queryUserUgi.doAs( new PrivilegedExceptionAction<Void>() { public Void run() throws Exception { injector.injectChecked( fragmentContext.getExecutionControls(), "fragment-execution", IOException.class); /* * Run the query until root.next returns false OR we no longer need to continue. */ while (shouldContinue() && root.next()) { // loop } return null; } }); } catch (OutOfMemoryError | OutOfMemoryException e) { if (!(e instanceof OutOfMemoryError) || "Direct buffer memory".equals(e.getMessage())) { fail(UserException.memoryError(e).build(logger)); } else { // we have a heap out of memory error. The JVM in unstable, exit. CatastrophicFailure.exit( e, "Unable to handle out of memory condition in FragmentExecutor.", -2); } } catch (AssertionError | Exception e) { fail(e); } finally { // no longer allow this thread to be interrupted. We synchronize here to make sure that cancel // can't set an // interruption after we have moved beyond this block. synchronized (myThreadRef) { myThreadRef.set(null); Thread.interrupted(); } // We need to sure we countDown at least once. We'll do it here to guarantee that. acceptExternalEvents.countDown(); // here we could be in FAILED, RUNNING, or CANCELLATION_REQUESTED cleanup(FragmentState.FINISHED); clusterCoordinator.removeDrillbitStatusListener(drillbitStatusListener); myThread.setName(originalThreadName); } }