private void updatePreemptionListAndNotify(EndReason reason) { // if this task was added to pre-emption list, remove it if (enablePreemption) { String state = reason == null ? "FAILED" : reason.name(); boolean removed = preemptionQueue.remove(taskWrapper); if (removed && isInfoEnabled) { LOG.info( TaskRunnerCallable.getTaskIdentifierString( taskWrapper.getTaskRunnerCallable().getRequest()) + " request " + state + "! Removed from preemption list."); } } numSlotsAvailable.incrementAndGet(); if (isDebugEnabled) { LOG.debug( "Task {} complete. WaitQueueSize={}, numSlotsAvailable={}, preemptionQueueSize={}", taskWrapper.getRequestId(), waitQueue.size(), numSlotsAvailable.get(), preemptionQueue.size()); } synchronized (lock) { if (!waitQueue.isEmpty()) { lock.notify(); } } }
private void trySchedule(final TaskWrapper taskWrapper) throws RejectedExecutionException { synchronized (lock) { boolean canFinish = taskWrapper.getTaskRunnerCallable().canFinish(); LOG.info("Attempting to execute {}", taskWrapper); ListenableFuture<TaskRunner2Result> future = executorService.submit(taskWrapper.getTaskRunnerCallable()); taskWrapper.setIsInWaitQueue(false); FutureCallback<TaskRunner2Result> wrappedCallback = createInternalCompletionListener(taskWrapper); // Callback on a separate thread so that when a task completes, the thread in the main queue // is actually available for execution and will not potentially result in a RejectedExecution Futures.addCallback(future, wrappedCallback, executionCompletionExecutorService); if (isDebugEnabled) { LOG.debug( "{} scheduled for execution. canFinish={}", taskWrapper.getRequestId(), canFinish); } // only tasks that cannot finish immediately are pre-emptable. In other words, if all inputs // to the tasks are not ready yet, the task is eligible for pre-emptable. if (enablePreemption) { if (!canFinish) { if (isInfoEnabled) { LOG.info( "{} is not finishable. Adding it to pre-emption queue", taskWrapper.getRequestId()); } addToPreemptionQueue(taskWrapper); } } } numSlotsAvailable.decrementAndGet(); }
@Override public void killFragment(String fragmentId) { synchronized (lock) { TaskWrapper taskWrapper = knownTasks.remove(fragmentId); // Can be null since the task may have completed meanwhile. if (taskWrapper != null) { if (taskWrapper.inWaitQueue) { if (isDebugEnabled) { LOG.debug("Removing {} from waitQueue", fragmentId); } taskWrapper.setIsInWaitQueue(false); waitQueue.remove(taskWrapper); } if (taskWrapper.inPreemptionQueue) { if (isDebugEnabled) { LOG.debug("Removing {} from preemptionQueue", fragmentId); } taskWrapper.setIsInPreemptableQueue(false); preemptionQueue.remove(taskWrapper); } taskWrapper.getTaskRunnerCallable().killTask(); } else { LOG.info("Ignoring killFragment request for {} since it isn't known", fragmentId); } lock.notify(); } }
@Override public void onSuccess(TaskRunner2Result result) { knownTasks.remove(taskWrapper.getRequestId()); taskWrapper.setIsInPreemptableQueue(false); taskWrapper.maybeUnregisterForFinishedStateNotifications(); taskWrapper.getTaskRunnerCallable().getCallback().onSuccess(result); updatePreemptionListAndNotify(result.getEndReason()); }
@Override public void onFailure(Throwable t) { knownTasks.remove(taskWrapper.getRequestId()); taskWrapper.setIsInPreemptableQueue(false); taskWrapper.maybeUnregisterForFinishedStateNotifications(); taskWrapper.getTaskRunnerCallable().getCallback().onFailure(t); updatePreemptionListAndNotify(null); LOG.error("Failed notification received: Stacktrace: " + ExceptionUtils.getStackTrace(t)); }
private TaskWrapper removeAndGetFromPreemptionQueue() { TaskWrapper taskWrapper; synchronized (lock) { taskWrapper = preemptionQueue.remove(); if (taskWrapper != null) { taskWrapper.setIsInPreemptableQueue(false); } } return taskWrapper; }
@Override public int compare(TaskWrapper t1, TaskWrapper t2) { TaskRunnerCallable o1 = t1.getTaskRunnerCallable(); TaskRunnerCallable o2 = t2.getTaskRunnerCallable(); FragmentRuntimeInfo fri1 = o1.getFragmentRuntimeInfo(); FragmentRuntimeInfo fri2 = o2.getFragmentRuntimeInfo(); if (fri1.getNumSelfAndUpstreamTasks() > fri2.getNumSelfAndUpstreamTasks()) { return 1; } else if (fri1.getNumSelfAndUpstreamTasks() < fri2.getNumSelfAndUpstreamTasks()) { return -1; } return 0; }
private void handleScheduleAttemptedRejection(TaskWrapper taskWrapper) { if (enablePreemption && taskWrapper.getTaskRunnerCallable().canFinish() && !preemptionQueue.isEmpty()) { if (isDebugEnabled) { LOG.debug("Preemption Queue: " + preemptionQueue); } TaskWrapper pRequest = removeAndGetFromPreemptionQueue(); // Avoid preempting tasks which are finishable - callback still to be processed. if (pRequest != null) { if (pRequest.getTaskRunnerCallable().canFinish()) { LOG.info( "Removed {} from preemption queue, but not preempting since it's now finishable", pRequest.getRequestId()); } else { if (isInfoEnabled) { LOG.info( "Invoking kill task for {} due to pre-emption to run {}", pRequest.getRequestId(), taskWrapper.getRequestId()); } // The task will either be killed or is already in the process of completing, which will // trigger the next scheduling run, or result in available slots being higher than 0, // which will cause the scheduler loop to continue. pRequest.getTaskRunnerCallable().killTask(); } } } }
@Override public Set<String> getExecutorsStatus() { Set<String> result = new HashSet<>(); StringBuilder value = new StringBuilder(); for (Map.Entry<String, TaskWrapper> e : knownTasks.entrySet()) { value.setLength(0); value.append(e.getKey()); TaskWrapper task = e.getValue(); boolean isFirst = true; TaskRunnerCallable c = task.getTaskRunnerCallable(); if (c != null && c.getRequest() != null && c.getRequest().getFragmentSpec() != null) { FragmentSpecProto fs = c.getRequest().getFragmentSpec(); value .append(isFirst ? " (" : ", ") .append(fs.getDagName()) .append("/") .append(fs.getVertexName()); isFirst = false; } value.append(isFirst ? " (" : ", "); if (task.isInWaitQueue()) { value.append("in queue"); } else if (c != null) { long startTime = c.getStartTime(); if (startTime != 0) { value.append("started at ").append(sdf.get().format(new Date(startTime))); } else { value.append("not started"); } } else { value.append("has no callable"); } if (task.isInPreemptionQueue()) { value.append(", ").append("preemptable"); } value.append(")"); result.add(value.toString()); } return result; }
private void finishableStateUpdated(TaskWrapper taskWrapper, boolean newFinishableState) { synchronized (lock) { if (taskWrapper.isInWaitQueue()) { // Re-order the wait queue LOG.debug( "Re-ordering the wait queue since {} finishable state moved to {}", taskWrapper.getRequestId(), newFinishableState); if (waitQueue.remove(taskWrapper)) { // Put element back only if it existed. waitQueue.offer(taskWrapper); } else { LOG.warn( "Failed to remove {} from waitQueue", taskWrapper.getTaskRunnerCallable().getRequestId()); } } if (newFinishableState == true && taskWrapper.isInPreemptionQueue()) { LOG.debug( "Removing {} from preemption queue because it's state changed to {}", taskWrapper.getRequestId(), newFinishableState); preemptionQueue.remove(taskWrapper.getTaskRunnerCallable()); } else if (newFinishableState == false && !taskWrapper.isInPreemptionQueue() && !taskWrapper.isInWaitQueue()) { LOG.debug( "Adding {} to preemption queue since finishable state changed to {}", taskWrapper.getRequestId(), newFinishableState); preemptionQueue.offer(taskWrapper); } lock.notify(); } }
private void addToPreemptionQueue(TaskWrapper taskWrapper) { synchronized (lock) { preemptionQueue.add(taskWrapper); taskWrapper.setIsInPreemptableQueue(true); } }
@Override public void schedule(TaskRunnerCallable task) throws RejectedExecutionException { TaskWrapper taskWrapper = new TaskWrapper(task, this); TaskWrapper evictedTask; synchronized (lock) { // If the queue does not have capacity, it does not throw a Rejection. Instead it will // return the task with the lowest priority, which could be the task which is currently being // processed. // TODO HIVE-11687 It's possible for a bunch of tasks to come in around the same time, without // the // actual executor threads picking up any work. This will lead to unnecessary rejection of // tasks. // The wait queue should be able to fit at least (waitQueue + currentFreeExecutor slots) evictedTask = waitQueue.offer(taskWrapper); if (evictedTask != taskWrapper) { knownTasks.put(taskWrapper.getRequestId(), taskWrapper); taskWrapper.setIsInWaitQueue(true); if (isDebugEnabled) { LOG.debug( "{} added to wait queue. Current wait queue size={}", task.getRequestId(), waitQueue.size()); } } else { if (isInfoEnabled) { LOG.info("wait queue full, size={}. {} not added", waitQueue.size(), task.getRequestId()); } evictedTask.getTaskRunnerCallable().killTask(); throw new RejectedExecutionException("Wait queue full"); } } // At this point, the task has been added into the queue. It may have caused an eviction for // some other task. // This registration has to be done after knownTasks has been populated. // Register for state change notifications so that the waitQueue can be re-ordered correctly // if the fragment moves in or out of the finishable state. boolean canFinish = taskWrapper.getTaskRunnerCallable().canFinish(); // It's safe to register outside of the lock since the stateChangeTracker ensures that updates // and registrations are mutually exclusive. taskWrapper.maybeRegisterForFinishedStateNotifications(canFinish); if (isDebugEnabled) { LOG.debug("Wait Queue: {}", waitQueue); } if (evictedTask != null) { knownTasks.remove(evictedTask.getRequestId()); evictedTask.maybeUnregisterForFinishedStateNotifications(); evictedTask.setIsInWaitQueue(false); evictedTask.getTaskRunnerCallable().killTask(); if (isInfoEnabled) { LOG.info( "{} evicted from wait queue in favor of {} because of lower priority", evictedTask.getRequestId(), task.getRequestId()); } } synchronized (lock) { lock.notify(); } }
@Override public void run() { try { while (!isShutdown.get()) { RejectedExecutionException rejectedException = null; synchronized (lock) { // Since schedule() can be called from multiple threads, we peek the wait queue, // try scheduling the task and then remove the task if scheduling is successful. // This will make sure the task's place in the wait queue is held until it gets // scheduled. task = waitQueue.peek(); if (task == null) { if (!isShutdown.get()) { lock.wait(); } continue; } // if the task cannot finish and if no slots are available then don't schedule it. boolean shouldWait = false; if (task.getTaskRunnerCallable().canFinish()) { if (isDebugEnabled) { LOG.debug( "Attempting to schedule task {}, canFinish={}. Current state: preemptionQueueSize={}, numSlotsAvailable={}, waitQueueSize={}", task.getRequestId(), task.getTaskRunnerCallable().canFinish(), preemptionQueue.size(), numSlotsAvailable.get(), waitQueue.size()); } if (numSlotsAvailable.get() == 0 && preemptionQueue.isEmpty()) { shouldWait = true; } } else { if (numSlotsAvailable.get() == 0) { shouldWait = true; } } if (shouldWait) { if (!isShutdown.get()) { lock.wait(); } // Another task at a higher priority may have come in during the wait. Lookup the // queue again to pick up the task at the highest priority. continue; } try { trySchedule(task); // wait queue could have been re-ordered in the mean time because of concurrent task // submission. So remove the specific task instead of the head task. waitQueue.remove(task); } catch (RejectedExecutionException e) { rejectedException = e; } } // Handle the rejection outside of the lock if (rejectedException != null) { handleScheduleAttemptedRejection(task); } synchronized (lock) { while (waitQueue.isEmpty()) { if (!isShutdown.get()) { lock.wait(); } } } } } catch (InterruptedException e) { if (isShutdown.get()) { LOG.info( WAIT_QUEUE_SCHEDULER_THREAD_NAME_FORMAT + " thread has been interrupted after shutdown."); } else { LOG.warn(WAIT_QUEUE_SCHEDULER_THREAD_NAME_FORMAT + " interrupted without shutdown", e); throw new RuntimeException(e); } } }