public void dumpProfile(JobletProfile jProfile) { Map<String, Long> counters = jProfile.getCounters(); counterMap.forEach((key, value) -> counters.put(key, value.get())); for (Task task : taskMap.values()) { TaskProfile taskProfile = new TaskProfile(task.getTaskAttemptId(), new Hashtable<>(task.getPartitionSendProfile()), new StatsCollector()); task.dumpProfile(taskProfile); jProfile.getTaskProfiles().put(task.getTaskAttemptId(), taskProfile); } }
@Override public void run() { TaskProfile taskProfile = new TaskProfile(task.getTaskAttemptId(), task.getPartitionSendProfile(), task.getStatsCollector()); try { ncs.getClusterController(task.getJobletContext().getJobId().getCcId()).notifyTaskComplete( task.getJobletContext().getJobId(), task.getTaskAttemptId(), ncs.getId(), taskProfile); } catch (Exception e) { LOGGER.log(Level.ERROR, "Failed notifying task complete for " + task.getTaskAttemptId(), e); } task.getJoblet().removeTask(task); }
private void logPendingTasks() { for (Task task : runningTasks) { final List<Thread> pendingThreads = task.getPendingThreads(); LOGGER.error("task {} was stuck. Stuck thread count = {}", task.getTaskAttemptId(), pendingThreads.size()); for (Thread thread : pendingThreads) { LOGGER.error("Stuck thread trace", ExceptionUtils.fromThreadStack(thread)); } } } }
if (!addPendingThread(ct)) { exceptions.add(HyracksDataException.create(TASK_ABORTED, getTaskAttemptId())); ExceptionUtils.setNodeIds(exceptions, ncs.getId()); ncs.getWorkQueue() try { Thread thread = Thread.currentThread(); if (!addPendingThread(thread)) { return; thread.setPriority(Thread.MIN_PRIORITY); try { pushFrames(collector, inputChannelsFromConnectors.get(cIdx), writer); } catch (HyracksDataException e) { synchronized (Task.this) { removePendingThread(thread); pushFrames(collectors[0], inputChannelsFromConnectors.get(0), operator.getInputFrameWriter(0)); } finally { sem.acquireUninterruptibly(collectors.length - 1); exceptions.add(HyracksDataException.create(e)); } finally { close(); removePendingThread(ct); completed = true;
List<IConnectorDescriptor> inputs = ac.getActivityInputMap().get(aid); task = null; task = new Task(joblet, flags, taId, han.getClass().getName(), ncs.getExecutor(), ncs, createInputChannels(td, inputs)); IOperatorNodePushable operator = han.createPushRuntime(task, rdp, partition, td.getPartitionCount()); task.setTaskRuntime(collectors.toArray(new IPartitionCollector[collectors.size()]), operator); joblet.addTask(task); task.start(); taskIndex++;
@Override public String toString() { return getName() + ": [" + ncs.getId() + "[" + task.getJoblet().getJobId() + ":" + task.getTaskAttemptId() + "]"; } }
@Override public void sendApplicationMessageToCC(byte[] message, DeploymentId deploymentId) throws Exception { this.ncs.sendApplicationMessageToCC(getJobletContext().getJobId().getCcId(), message, deploymentId); }
@Override public void run() { if (LOGGER.isInfoEnabled()) { LOGGER.info("Aborting Tasks: " + jobId + ":" + tasks); } IResultPartitionManager resultPartitionManager = ncs.getResultPartitionManager(); if (resultPartitionManager != null) { ncs.getResultPartitionManager().abortReader(jobId); } Joblet ji = ncs.getJobletMap().get(jobId); if (ji != null) { Map<TaskAttemptId, Task> taskMap = ji.getTaskMap(); for (TaskAttemptId taId : tasks) { Task task = taskMap.get(taId); if (task != null) { task.abort(); } } } else { LOGGER.log(Level.WARN, "Joblet couldn't be found. Tasks of job " + jobId + " have all either completed or failed"); } } }
@Override public void run() { Exception ex = exceptions.get(0); LOGGER.log(ExceptionUtils.causedByInterrupt(ex) ? Level.DEBUG : Level.WARN, "task " + taskId + " has failed", ex); try { IResultPartitionManager resultPartitionManager = ncs.getResultPartitionManager(); if (resultPartitionManager != null) { resultPartitionManager.abortReader(jobId); } ncs.getClusterController(jobId.getCcId()).notifyTaskFailure(jobId, taskId, ncs.getId(), exceptions); } catch (Exception e) { LOGGER.log(Level.ERROR, "Failure reporting task failure to cluster controller", e); } if (task != null) { task.getJoblet().removeTask(task); } }
@Override public void sendApplicationMessageToCC(Serializable message, DeploymentId deploymentId) throws Exception { this.ncs.sendApplicationMessageToCC(getJobletContext().getJobId().getCcId(), JavaSerializationUtils.serialize(message), deploymentId); }
@Override protected void doRun() throws Exception { LOGGER.info("Aborting all tasks for controller {}", ccId); IResultPartitionManager resultPartitionManager = ncs.getResultPartitionManager(); if (resultPartitionManager == null) { LOGGER.log(Level.WARN, "ResultPartitionManager is null on " + ncs.getId()); } Deque<Task> abortedTasks = new ArrayDeque<>(); Collection<Joblet> joblets = ncs.getJobletMap().values(); // TODO(mblow): should we have one jobletmap per cc? joblets.stream().filter(joblet -> joblet.getJobId().getCcId().equals(ccId)).forEach(joblet -> { joblet.getTaskMap().values().forEach(task -> { task.abort(); abortedTasks.add(task); }); final JobId jobId = joblet.getJobId(); if (resultPartitionManager != null) { resultPartitionManager.abortReader(jobId); resultPartitionManager.sweep(jobId); } ncs.getWorkQueue().schedule(new CleanupJobletWork(ncs, jobId, JobStatus.FAILURE)); }); ncs.getExecutor().submit(new EnsureAllCcTasksCompleted(ncs, ccId, abortedTasks)); } }