@Override public String toString() { return getName() + ": nodeId: " + nodeId; } }
@Override public String toString() { return getName() + ": [" + ncs.getId() + "[" + jobId + ":" + taskId + "]"; } }
@Override public void run() { try { LOGGER.info("Ensuring all tasks of CC {} have completed", ccId); final Span maxWaitTime = Span.start(2, TimeUnit.MINUTES); while (!maxWaitTime.elapsed()) { removeCompleted(); if (runningTasks.isEmpty()) { break; } LOGGER.info("{} tasks are still running", runningTasks.size()); TimeUnit.SECONDS.sleep(1); // Check once a second } if (runningTasks.isEmpty()) { LOGGER.info("All tasks of CC {} have completed", ccId); ncs.notifyTasksCompleted(ccId); } else { LOGGER.error("{} tasks associated with CC {} failed to complete after {}ms. Giving up", runningTasks.size(), ccId, TIMEOUT); logPendingTasks(); ExitUtil.halt(ExitUtil.EC_NC_FAILED_TO_ABORT_ALL_PREVIOUS_TASKS); } } catch (Throwable th) { LOGGER.error("Failed to abort all previous tasks associated with CC {}", ccId, th); ExitUtil.halt(ExitUtil.EC_NC_FAILED_TO_ABORT_ALL_PREVIOUS_TASKS); } }
CCNCFunctions.SendApplicationMessageFunction amf = (CCNCFunctions.SendApplicationMessageFunction) fn; ncs.getWorkQueue().schedule( new ApplicationMessageWork(ncs, amf.getMessage(), amf.getDeploymentId(), amf.getNodeId())); return; case START_TASKS: CCNCFunctions.StartTasksFunction stf = (CCNCFunctions.StartTasksFunction) fn; ncs.getWorkQueue() .schedule(new StartTasksWork(ncs, stf.getDeploymentId(), stf.getJobId(), stf.getPlanBytes(), stf.getTaskDescriptors(), stf.getConnectorPolicies(), stf.getFlags(), stf.getJobParameters(), stf.getDeployedJobSpecId(), stf.getJobStartTime())); case ABORT_TASKS: CCNCFunctions.AbortTasksFunction atf = (CCNCFunctions.AbortTasksFunction) fn; ncs.getWorkQueue().schedule(new AbortTasksWork(ncs, atf.getJobId(), atf.getTasks())); return; case ABORT_ALL_JOBS: CCNCFunctions.AbortCCJobsFunction aajf = (CCNCFunctions.AbortCCJobsFunction) fn; ncs.getWorkQueue().schedule(new AbortAllJobsWork(ncs, aajf.getCcId())); return; case CLEANUP_JOBLET: CCNCFunctions.CleanupJobletFunction cjf = (CCNCFunctions.CleanupJobletFunction) fn; ncs.getWorkQueue().schedule(new CleanupJobletWork(ncs, cjf.getJobId(), cjf.getStatus())); return; case REPORT_PARTITION_AVAILABILITY: (CCNCFunctions.ReportPartitionAvailabilityFunction) fn; ncs.getWorkQueue().schedule( new ReportPartitionAvailabilityWork(ncs, rpaf.getPartitionId(), rpaf.getNetworkAddress())); return;
ncs.updateMaxJobId(jobId); NCServiceContext serviceCtx = ncs.getContext(); Joblet joblet = getOrCreateLocalJoblet(deploymentId, serviceCtx, acgBytes); if (ncs.getNodeStatus() != NodeStatus.ACTIVE) { throw HyracksException.create(ErrorCode.NODE_IS_NOT_ACTIVE, ncs.getId()); 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()); List<IPartitionCollector> collectors = new ArrayList<>(); RecordDescriptor recordDesc = ac.getConnectorRecordDescriptorMap().get(conn.getConnectorId()); IPartitionCollector collector = createPartitionCollector(td, partition, task, i, conn, recordDesc, cPolicy); collectors.add(collector); createPartitionWriterFactory(task, cPolicy, jobId, conn, partition, taId, flags); LOGGER.trace("input: {}: {}", i, conn.getConnectorId()); IFrameWriter writer = conn.createPartitioner(task, recordDesc, pwFactory, partition, ExceptionUtils.setNodeIds(exceptions, ncs.getId()); TaskAttemptId taskId = taskDescriptors.get(taskIndex).getTaskAttemptId(); ncs.getWorkQueue().schedule(new NotifyTaskFailureWork(ncs, task, exceptions, jobId, taskId));
@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)); } }
ExceptionUtils.setNodeIds(exceptions, ncs.getId()); ncs.getWorkQueue() .schedule(new NotifyTaskFailureWork(ncs, this, exceptions, joblet.getJobId(), taskAttemptId)); return; throw operatorException; ncs.getWorkQueue().schedule(new NotifyTaskCompleteWork(ncs, this)); } catch (Throwable e) { // NOSONAR: Catch all failures exceptions.add(HyracksDataException.create(e)); .schedule(new NotifyTaskFailureWork(ncs, this, exceptions, joblet.getJobId(), taskAttemptId));
@Override public String toString() { return getName() + ": [" + ncs.getId() + "[" + task.getJoblet().getJobId() + ":" + task.getTaskAttemptId() + "]"; } }
@Override public void run() { try { FutureValue<List<JobProfile>> fv = new FutureValue<>(); BuildJobProfilesWork bjpw = new BuildJobProfilesWork(NodeControllerService.this, ccId, fv); workQueue.scheduleAndSync(bjpw); List<JobProfile> profiles = fv.get(); if (!profiles.isEmpty()) { cc.reportProfile(id, profiles); } } catch (Exception e) { LOGGER.log(Level.WARN, "Exception reporting profile", e); } } }
() -> String.valueOf(e)); getWorkQueue().scheduleAndSync(new AbortAllJobsWork(this, ccId)); HeartbeatManager hbMgr = heartbeatManagers.remove(ccId); if (hbMgr != null) {