http://git-wip-us.apache.org/repos/asf/ignite/blob/facedf50/modules/hadoop-impl/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java ---------------------------------------------------------------------- diff --git a/modules/hadoop-impl/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java b/modules/hadoop-impl/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java new file mode 100644 index 0000000..bffb82b --- /dev/null +++ b/modules/hadoop-impl/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java @@ -0,0 +1,1699 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.hadoop.jobtracker; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; +import javax.cache.event.CacheEntryEvent; +import javax.cache.event.CacheEntryUpdatedListener; +import javax.cache.expiry.Duration; +import javax.cache.expiry.ExpiryPolicy; +import javax.cache.expiry.ModifiedExpiryPolicy; +import javax.cache.processor.EntryProcessor; +import javax.cache.processor.MutableEntry; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.events.DiscoveryEvent; +import org.apache.ignite.events.Event; +import org.apache.ignite.events.EventType; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; +import org.apache.ignite.internal.processors.cache.GridCacheAdapter; +import org.apache.ignite.internal.processors.cache.IgniteInternalCache; +import org.apache.ignite.internal.processors.hadoop.HadoopClassLoader; +import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils; +import org.apache.ignite.internal.processors.hadoop.HadoopComponent; +import org.apache.ignite.internal.processors.hadoop.HadoopContext; +import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit; +import org.apache.ignite.internal.processors.hadoop.HadoopJob; +import org.apache.ignite.internal.processors.hadoop.HadoopJobId; +import org.apache.ignite.internal.processors.hadoop.HadoopJobInfo; +import org.apache.ignite.internal.processors.hadoop.HadoopJobPhase; +import org.apache.ignite.internal.processors.hadoop.HadoopJobStatus; +import org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlan; +import org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlanner; +import org.apache.ignite.internal.processors.hadoop.HadoopTaskCancelledException; +import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo; +import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounterWriter; +import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters; +import org.apache.ignite.internal.processors.hadoop.counter.HadoopCountersImpl; +import org.apache.ignite.internal.processors.hadoop.counter.HadoopPerformanceCounter; +import org.apache.ignite.internal.processors.hadoop.taskexecutor.HadoopTaskStatus; +import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.HadoopProcessDescriptor; +import org.apache.ignite.internal.util.GridMutex; +import org.apache.ignite.internal.util.GridSpinReadWriteLock; +import org.apache.ignite.internal.util.future.GridFinishedFuture; +import org.apache.ignite.internal.util.future.GridFutureAdapter; +import org.apache.ignite.internal.util.typedef.CI1; +import org.apache.ignite.internal.util.typedef.CIX1; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.SB; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteInClosure; +import org.jetbrains.annotations.Nullable; +import org.jsr166.ConcurrentHashMap8; + +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.apache.ignite.internal.processors.hadoop.HadoopJobPhase.PHASE_CANCELLING; +import static org.apache.ignite.internal.processors.hadoop.HadoopJobPhase.PHASE_COMPLETE; +import static org.apache.ignite.internal.processors.hadoop.HadoopJobPhase.PHASE_MAP; +import static org.apache.ignite.internal.processors.hadoop.HadoopJobPhase.PHASE_REDUCE; +import static org.apache.ignite.internal.processors.hadoop.HadoopJobPhase.PHASE_SETUP; +import static org.apache.ignite.internal.processors.hadoop.HadoopTaskType.ABORT; +import static org.apache.ignite.internal.processors.hadoop.HadoopTaskType.COMMIT; +import static org.apache.ignite.internal.processors.hadoop.HadoopTaskType.MAP; +import static org.apache.ignite.internal.processors.hadoop.HadoopTaskType.REDUCE; +import static org.apache.ignite.internal.processors.hadoop.HadoopTaskType.SETUP; +import static org.apache.ignite.internal.processors.hadoop.taskexecutor.HadoopTaskState.COMPLETED; +import static org.apache.ignite.internal.processors.hadoop.taskexecutor.HadoopTaskState.CRASHED; +import static org.apache.ignite.internal.processors.hadoop.taskexecutor.HadoopTaskState.FAILED; +import static org.apache.ignite.internal.processors.hadoop.taskexecutor.HadoopTaskState.RUNNING; + +/** + * Hadoop job tracker. + */ +public class HadoopJobTracker extends HadoopComponent { + /** */ + private final GridMutex mux = new GridMutex(); + + /** */ + private volatile IgniteInternalCache<HadoopJobId, HadoopJobMetadata> jobMetaPrj; + + /** Projection with expiry policy for finished job updates. */ + private volatile IgniteInternalCache<HadoopJobId, HadoopJobMetadata> finishedJobMetaPrj; + + /** Map-reduce execution planner. */ + @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized") + private HadoopMapReducePlanner mrPlanner; + + /** All the known jobs. */ + private final ConcurrentMap<HadoopJobId, GridFutureAdapter<HadoopJob>> jobs = new ConcurrentHashMap8<>(); + + /** Locally active jobs. */ + private final ConcurrentMap<HadoopJobId, JobLocalState> activeJobs = new ConcurrentHashMap8<>(); + + /** Locally requested finish futures. */ + private final ConcurrentMap<HadoopJobId, GridFutureAdapter<HadoopJobId>> activeFinishFuts = + new ConcurrentHashMap8<>(); + + /** Event processing service. */ + private ExecutorService evtProcSvc; + + /** Component busy lock. */ + private GridSpinReadWriteLock busyLock; + + /** Class to create HadoopJob instances from. */ + private Class<? extends HadoopJob> jobCls; + + /** Closure to check result of async transform of system cache. */ + private final IgniteInClosure<IgniteInternalFuture<?>> failsLog = new CI1<IgniteInternalFuture<?>>() { + @Override public void apply(IgniteInternalFuture<?> gridFut) { + try { + gridFut.get(); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to transform system cache.", e); + } + } + }; + + /** {@inheritDoc} */ + @SuppressWarnings("unchecked") + @Override public void start(final HadoopContext ctx) throws IgniteCheckedException { + super.start(ctx); + + busyLock = new GridSpinReadWriteLock(); + + evtProcSvc = Executors.newFixedThreadPool(1); + + assert jobCls == null; + + HadoopClassLoader ldr = ctx.kernalContext().hadoopHelper().commonClassLoader(); + + try { + jobCls = (Class<HadoopJob>)ldr.loadClass(HadoopCommonUtils.JOB_CLS_NAME); + } + catch (Exception ioe) { + throw new IgniteCheckedException("Failed to load job class [class=" + + HadoopCommonUtils.JOB_CLS_NAME + ']', ioe); + } + } + + /** + * @return Job meta projection. + */ + @SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext") + private IgniteInternalCache<HadoopJobId, HadoopJobMetadata> jobMetaCache() { + IgniteInternalCache<HadoopJobId, HadoopJobMetadata> prj = jobMetaPrj; + + if (prj == null) { + synchronized (mux) { + if ((prj = jobMetaPrj) == null) { + GridCacheAdapter<HadoopJobId, HadoopJobMetadata> sysCache = ctx.kernalContext().cache() + .internalCache(CU.SYS_CACHE_HADOOP_MR); + + assert sysCache != null; + + mrPlanner = ctx.planner(); + + try { + ctx.kernalContext().resource().injectGeneric(mrPlanner); + } + catch (IgniteCheckedException e) { // Must not happen. + U.error(log, "Failed to inject resources.", e); + + throw new IllegalStateException(e); + } + + jobMetaPrj = prj = sysCache; + + if (ctx.configuration().getFinishedJobInfoTtl() > 0) { + ExpiryPolicy finishedJobPlc = new ModifiedExpiryPolicy( + new Duration(MILLISECONDS, ctx.configuration().getFinishedJobInfoTtl())); + + finishedJobMetaPrj = prj.withExpiryPolicy(finishedJobPlc); + } + else + finishedJobMetaPrj = jobMetaPrj; + } + } + } + + return prj; + } + + /** + * @return Projection with expiry policy for finished job updates. + */ + private IgniteInternalCache<HadoopJobId, HadoopJobMetadata> finishedJobMetaCache() { + IgniteInternalCache<HadoopJobId, HadoopJobMetadata> prj = finishedJobMetaPrj; + + if (prj == null) { + jobMetaCache(); + + prj = finishedJobMetaPrj; + + assert prj != null; + } + + return prj; + } + + /** {@inheritDoc} */ + @SuppressWarnings("deprecation") + @Override public void onKernalStart() throws IgniteCheckedException { + super.onKernalStart(); + + jobMetaCache().context().continuousQueries().executeInternalQuery( + new CacheEntryUpdatedListener<HadoopJobId, HadoopJobMetadata>() { + @Override public void onUpdated(final Iterable<CacheEntryEvent<? extends HadoopJobId, + ? extends HadoopJobMetadata>> evts) { + if (!busyLock.tryReadLock()) + return; + + try { + // Must process query callback in a separate thread to avoid deadlocks. + evtProcSvc.submit(new EventHandler() { + @Override protected void body() throws IgniteCheckedException { + processJobMetadataUpdates(evts); + } + }); + } + finally { + busyLock.readUnlock(); + } + } + }, + null, + true, + true, + false + ); + + ctx.kernalContext().event().addLocalEventListener(new GridLocalEventListener() { + @Override public void onEvent(final Event evt) { + if (!busyLock.tryReadLock()) + return; + + try { + // Must process discovery callback in a separate thread to avoid deadlock. + evtProcSvc.submit(new EventHandler() { + @Override protected void body() { + processNodeLeft((DiscoveryEvent)evt); + } + }); + } + finally { + busyLock.readUnlock(); + } + } + }, EventType.EVT_NODE_FAILED, EventType.EVT_NODE_LEFT); + } + + /** {@inheritDoc} */ + @Override public void onKernalStop(boolean cancel) { + super.onKernalStop(cancel); + + busyLock.writeLock(); + + evtProcSvc.shutdown(); + + // Fail all pending futures. + for (GridFutureAdapter<HadoopJobId> fut : activeFinishFuts.values()) + fut.onDone(new IgniteCheckedException("Failed to execute Hadoop map-reduce job (grid is stopping).")); + } + + /** + * Submits execution of Hadoop job to grid. + * + * @param jobId Job ID. + * @param info Job info. + * @return Job completion future. + */ + @SuppressWarnings("unchecked") + public IgniteInternalFuture<HadoopJobId> submit(HadoopJobId jobId, HadoopJobInfo info) { + if (!busyLock.tryReadLock()) { + return new GridFinishedFuture<>(new IgniteCheckedException("Failed to execute map-reduce job " + + "(grid is stopping): " + info)); + } + + try { + long jobPrepare = U.currentTimeMillis(); + + if (jobs.containsKey(jobId) || jobMetaCache().containsKey(jobId)) + throw new IgniteCheckedException("Failed to submit job. Job with the same ID already exists: " + jobId); + + HadoopJob job = job(jobId, info); + + HadoopMapReducePlan mrPlan = mrPlanner.preparePlan(job, ctx.nodes(), null); + + HadoopJobMetadata meta = new HadoopJobMetadata(ctx.localNodeId(), jobId, info); + + meta.mapReducePlan(mrPlan); + + meta.pendingSplits(allSplits(mrPlan)); + meta.pendingReducers(allReducers(mrPlan)); + + GridFutureAdapter<HadoopJobId> completeFut = new GridFutureAdapter<>(); + + GridFutureAdapter<HadoopJobId> old = activeFinishFuts.put(jobId, completeFut); + + assert old == null : "Duplicate completion future [jobId=" + jobId + ", old=" + old + ']'; + + if (log.isDebugEnabled()) + log.debug("Submitting job metadata [jobId=" + jobId + ", meta=" + meta + ']'); + + long jobStart = U.currentTimeMillis(); + + HadoopPerformanceCounter perfCntr = HadoopPerformanceCounter.getCounter(meta.counters(), + ctx.localNodeId()); + + perfCntr.clientSubmissionEvents(info); + perfCntr.onJobPrepare(jobPrepare); + perfCntr.onJobStart(jobStart); + + if (jobMetaCache().getAndPutIfAbsent(jobId, meta) != null) + throw new IgniteCheckedException("Failed to submit job. Job with the same ID already exists: " + jobId); + + return completeFut; + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to submit job: " + jobId, e); + + return new GridFinishedFuture<>(e); + } + finally { + busyLock.readUnlock(); + } + } + + /** + * Convert Hadoop job metadata to job status. + * + * @param meta Metadata. + * @return Status. + */ + @SuppressWarnings("ThrowableResultOfMethodCallIgnored") + public static HadoopJobStatus status(HadoopJobMetadata meta) { + HadoopJobInfo jobInfo = meta.jobInfo(); + + return new HadoopJobStatus( + meta.jobId(), + jobInfo.jobName(), + jobInfo.user(), + meta.pendingSplits() != null ? meta.pendingSplits().size() : 0, + meta.pendingReducers() != null ? meta.pendingReducers().size() : 0, + meta.mapReducePlan().mappers(), + meta.mapReducePlan().reducers(), + meta.phase(), + meta.failCause() != null, + meta.version() + ); + } + + /** + * Gets hadoop job status for given job ID. + * + * @param jobId Job ID to get status for. + * @return Job status for given job ID or {@code null} if job was not found. + */ + @Nullable public HadoopJobStatus status(HadoopJobId jobId) throws IgniteCheckedException { + if (!busyLock.tryReadLock()) + return null; // Grid is stopping. + + try { + HadoopJobMetadata meta = jobMetaCache().get(jobId); + + return meta != null ? status(meta) : null; + } + finally { + busyLock.readUnlock(); + } + } + + /** + * Gets job finish future. + * + * @param jobId Job ID. + * @return Finish future or {@code null}. + * @throws IgniteCheckedException If failed. + */ + @Nullable public IgniteInternalFuture<?> finishFuture(HadoopJobId jobId) throws IgniteCheckedException { + if (!busyLock.tryReadLock()) + return null; // Grid is stopping. + + try { + HadoopJobMetadata meta = jobMetaCache().get(jobId); + + if (meta == null) + return null; + + if (log.isTraceEnabled()) + log.trace("Got job metadata for status check [locNodeId=" + ctx.localNodeId() + ", meta=" + meta + ']'); + + if (meta.phase() == PHASE_COMPLETE) { + if (log.isTraceEnabled()) + log.trace("Job is complete, returning finished future: " + jobId); + + return new GridFinishedFuture<>(jobId); + } + + GridFutureAdapter<HadoopJobId> fut = F.addIfAbsent(activeFinishFuts, jobId, + new GridFutureAdapter<HadoopJobId>()); + + // Get meta from cache one more time to close the window. + meta = jobMetaCache().get(jobId); + + if (log.isTraceEnabled()) + log.trace("Re-checking job metadata [locNodeId=" + ctx.localNodeId() + ", meta=" + meta + ']'); + + if (meta == null) { + fut.onDone(); + + activeFinishFuts.remove(jobId , fut); + } + else if (meta.phase() == PHASE_COMPLETE) { + fut.onDone(jobId, meta.failCause()); + + activeFinishFuts.remove(jobId , fut); + } + + return fut; + } + finally { + busyLock.readUnlock(); + } + } + + /** + * Gets job plan by job ID. + * + * @param jobId Job ID. + * @return Job plan. + * @throws IgniteCheckedException If failed. + */ + public HadoopMapReducePlan plan(HadoopJobId jobId) throws IgniteCheckedException { + if (!busyLock.tryReadLock()) + return null; + + try { + HadoopJobMetadata meta = jobMetaCache().get(jobId); + + if (meta != null) + return meta.mapReducePlan(); + + return null; + } + finally { + busyLock.readUnlock(); + } + } + + /** + * Callback from task executor invoked when a task has been finished. + * + * @param info Task info. + * @param status Task status. + */ + @SuppressWarnings({"ConstantConditions", "ThrowableResultOfMethodCallIgnored"}) + public void onTaskFinished(HadoopTaskInfo info, HadoopTaskStatus status) { + if (!busyLock.tryReadLock()) + return; + + try { + assert status.state() != RUNNING; + + if (log.isDebugEnabled()) + log.debug("Received task finished callback [info=" + info + ", status=" + status + ']'); + + JobLocalState state = activeJobs.get(info.jobId()); + + // Task CRASHes with null fail cause. + assert (status.state() != FAILED) || status.failCause() != null : + "Invalid task status [info=" + info + ", status=" + status + ']'; + + assert state != null || (ctx.jobUpdateLeader() && (info.type() == COMMIT || info.type() == ABORT)): + "Missing local state for finished task [info=" + info + ", status=" + status + ']'; + + StackedProcessor incrCntrs = null; + + if (status.state() == COMPLETED) + incrCntrs = new IncrementCountersProcessor(null, status.counters()); + + switch (info.type()) { + case SETUP: { + state.onSetupFinished(info, status, incrCntrs); + + break; + } + + case MAP: { + state.onMapFinished(info, status, incrCntrs); + + break; + } + + case REDUCE: { + state.onReduceFinished(info, status, incrCntrs); + + break; + } + + case COMBINE: { + state.onCombineFinished(info, status, incrCntrs); + + break; + } + + case COMMIT: + case ABORT: { + IgniteInternalCache<HadoopJobId, HadoopJobMetadata> cache = finishedJobMetaCache(); + + cache.invokeAsync(info.jobId(), new UpdatePhaseProcessor(incrCntrs, PHASE_COMPLETE)). + listen(failsLog); + + break; + } + } + } + finally { + busyLock.readUnlock(); + } + } + + /** + * @param jobId Job id. + * @param c Closure of operation. + */ + private void transform(HadoopJobId jobId, EntryProcessor<HadoopJobId, HadoopJobMetadata, Void> c) { + jobMetaCache().invokeAsync(jobId, c).listen(failsLog); + } + + /** + * Callback from task executor called when process is ready to received shuffle messages. + * + * @param jobId Job ID. + * @param reducers Reducers. + * @param desc Process descriptor. + */ + public void onExternalMappersInitialized(HadoopJobId jobId, Collection<Integer> reducers, + HadoopProcessDescriptor desc) { + transform(jobId, new InitializeReducersProcessor(null, reducers, desc)); + } + + /** + * Gets all input splits for given hadoop map-reduce plan. + * + * @param plan Map-reduce plan. + * @return Collection of all input splits that should be processed. + */ + @SuppressWarnings("ConstantConditions") + private Map<HadoopInputSplit, Integer> allSplits(HadoopMapReducePlan plan) { + Map<HadoopInputSplit, Integer> res = new HashMap<>(); + + int taskNum = 0; + + for (UUID nodeId : plan.mapperNodeIds()) { + for (HadoopInputSplit split : plan.mappers(nodeId)) { + if (res.put(split, taskNum++) != null) + throw new IllegalStateException("Split duplicate."); + } + } + + return res; + } + + /** + * Gets all reducers for this job. + * + * @param plan Map-reduce plan. + * @return Collection of reducers. + */ + private Collection<Integer> allReducers(HadoopMapReducePlan plan) { + Collection<Integer> res = new HashSet<>(); + + for (int i = 0; i < plan.reducers(); i++) + res.add(i); + + return res; + } + + /** + * Processes node leave (or fail) event. + * + * @param evt Discovery event. + */ + @SuppressWarnings("ConstantConditions") + private void processNodeLeft(DiscoveryEvent evt) { + if (log.isDebugEnabled()) + log.debug("Processing discovery event [locNodeId=" + ctx.localNodeId() + ", evt=" + evt + ']'); + + // Check only if this node is responsible for job status updates. + if (ctx.jobUpdateLeader()) { + boolean checkSetup = evt.eventNode().order() < ctx.localNodeOrder(); + + // Iteration over all local entries is correct since system cache is REPLICATED. + for (Object metaObj : jobMetaCache().values()) { + HadoopJobMetadata meta = (HadoopJobMetadata)metaObj; + + HadoopJobId jobId = meta.jobId(); + + HadoopMapReducePlan plan = meta.mapReducePlan(); + + HadoopJobPhase phase = meta.phase(); + + try { + if (checkSetup && phase == PHASE_SETUP && !activeJobs.containsKey(jobId)) { + // Failover setup task. + HadoopJob job = job(jobId, meta.jobInfo()); + + Collection<HadoopTaskInfo> setupTask = setupTask(jobId); + + assert setupTask != null; + + ctx.taskExecutor().run(job, setupTask); + } + else if (phase == PHASE_MAP || phase == PHASE_REDUCE) { + // Must check all nodes, even that are not event node ID due to + // multiple node failure possibility. + Collection<HadoopInputSplit> cancelSplits = null; + + for (UUID nodeId : plan.mapperNodeIds()) { + if (ctx.kernalContext().discovery().node(nodeId) == null) { + // Node has left the grid. + Collection<HadoopInputSplit> mappers = plan.mappers(nodeId); + + if (cancelSplits == null) + cancelSplits = new HashSet<>(); + + cancelSplits.addAll(mappers); + } + } + + Collection<Integer> cancelReducers = null; + + for (UUID nodeId : plan.reducerNodeIds()) { + if (ctx.kernalContext().discovery().node(nodeId) == null) { + // Node has left the grid. + int[] reducers = plan.reducers(nodeId); + + if (cancelReducers == null) + cancelReducers = new HashSet<>(); + + for (int rdc : reducers) + cancelReducers.add(rdc); + } + } + + if (cancelSplits != null || cancelReducers != null) + jobMetaCache().invoke(meta.jobId(), new CancelJobProcessor(null, new IgniteCheckedException( + "One or more nodes participating in map-reduce job execution failed."), cancelSplits, + cancelReducers)); + } + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to cancel job: " + meta, e); + } + } + } + } + + /** + * @param updated Updated cache entries. + * @throws IgniteCheckedException If failed. + */ + private void processJobMetadataUpdates( + Iterable<CacheEntryEvent<? extends HadoopJobId, ? extends HadoopJobMetadata>> updated) + throws IgniteCheckedException { + UUID locNodeId = ctx.localNodeId(); + + for (CacheEntryEvent<? extends HadoopJobId, ? extends HadoopJobMetadata> entry : updated) { + HadoopJobId jobId = entry.getKey(); + HadoopJobMetadata meta = entry.getValue(); + + if (meta == null || !ctx.isParticipating(meta)) + continue; + + if (log.isDebugEnabled()) + log.debug("Processing job metadata update callback [locNodeId=" + locNodeId + + ", meta=" + meta + ']'); + + try { + ctx.taskExecutor().onJobStateChanged(meta); + } + catch (IgniteCheckedException e) { + U.error(log, "Failed to process job state changed callback (will fail the job) " + + "[locNodeId=" + locNodeId + ", jobId=" + jobId + ", meta=" + meta + ']', e); + + transform(jobId, new CancelJobProcessor(null, e)); + + continue; + } + + processJobMetaUpdate(jobId, meta, locNodeId); + } + } + + /** + * @param jobId Job ID. + * @param plan Map-reduce plan. + */ + @SuppressWarnings({"unused", "ConstantConditions" }) + private void printPlan(HadoopJobId jobId, HadoopMapReducePlan plan) { + log.info("Plan for " + jobId); + + SB b = new SB(); + + b.a(" Map: "); + + for (UUID nodeId : plan.mapperNodeIds()) + b.a(nodeId).a("=").a(plan.mappers(nodeId).size()).a(' '); + + log.info(b.toString()); + + b = new SB(); + + b.a(" Reduce: "); + + for (UUID nodeId : plan.reducerNodeIds()) + b.a(nodeId).a("=").a(Arrays.toString(plan.reducers(nodeId))).a(' '); + + log.info(b.toString()); + } + + /** + * @param jobId Job ID. + * @param meta Job metadata. + * @param locNodeId Local node ID. + * @throws IgniteCheckedException If failed. + */ + private void processJobMetaUpdate(HadoopJobId jobId, HadoopJobMetadata meta, UUID locNodeId) + throws IgniteCheckedException { + JobLocalState state = activeJobs.get(jobId); + + HadoopJob job = job(jobId, meta.jobInfo()); + + HadoopMapReducePlan plan = meta.mapReducePlan(); + + switch (meta.phase()) { + case PHASE_SETUP: { + if (ctx.jobUpdateLeader()) { + Collection<HadoopTaskInfo> setupTask = setupTask(jobId); + + if (setupTask != null) + ctx.taskExecutor().run(job, setupTask); + } + + break; + } + + case PHASE_MAP: { + // Check if we should initiate new task on local node. + Collection<HadoopTaskInfo> tasks = mapperTasks(plan.mappers(locNodeId), meta); + + if (tasks != null) + ctx.taskExecutor().run(job, tasks); + + break; + } + + case PHASE_REDUCE: { + if (meta.pendingReducers().isEmpty() && ctx.jobUpdateLeader()) { + HadoopTaskInfo info = new HadoopTaskInfo(COMMIT, jobId, 0, 0, null); + + if (log.isDebugEnabled()) + log.debug("Submitting COMMIT task for execution [locNodeId=" + locNodeId + + ", jobId=" + jobId + ']'); + + ctx.taskExecutor().run(job, Collections.singletonList(info)); + + break; + } + + Collection<HadoopTaskInfo> tasks = reducerTasks(plan.reducers(locNodeId), job); + + if (tasks != null) + ctx.taskExecutor().run(job, tasks); + + break; + } + + case PHASE_CANCELLING: { + // Prevent multiple task executor notification. + if (state != null && state.onCancel()) { + if (log.isDebugEnabled()) + log.debug("Cancelling local task execution for job: " + meta); + + ctx.taskExecutor().cancelTasks(jobId); + } + + if (meta.pendingSplits().isEmpty() && meta.pendingReducers().isEmpty()) { + if (ctx.jobUpdateLeader()) { + if (state == null) + state = initState(jobId); + + // Prevent running multiple abort tasks. + if (state.onAborted()) { + HadoopTaskInfo info = new HadoopTaskInfo(ABORT, jobId, 0, 0, null); + + if (log.isDebugEnabled()) + log.debug("Submitting ABORT task for execution [locNodeId=" + locNodeId + + ", jobId=" + jobId + ']'); + + ctx.taskExecutor().run(job, Collections.singletonList(info)); + } + } + + break; + } + else { + // Check if there are unscheduled mappers or reducers. + Collection<HadoopInputSplit> cancelMappers = new ArrayList<>(); + Collection<Integer> cancelReducers = new ArrayList<>(); + + Collection<HadoopInputSplit> mappers = plan.mappers(ctx.localNodeId()); + + if (mappers != null) { + for (HadoopInputSplit b : mappers) { + if (state == null || !state.mapperScheduled(b)) + cancelMappers.add(b); + } + } + + int[] rdc = plan.reducers(ctx.localNodeId()); + + if (rdc != null) { + for (int r : rdc) { + if (state == null || !state.reducerScheduled(r)) + cancelReducers.add(r); + } + } + + if (!cancelMappers.isEmpty() || !cancelReducers.isEmpty()) + transform(jobId, new CancelJobProcessor(null, cancelMappers, cancelReducers)); + } + + break; + } + + case PHASE_COMPLETE: { + if (log.isDebugEnabled()) + log.debug("Job execution is complete, will remove local state from active jobs " + + "[jobId=" + jobId + ", meta=" + meta + ']'); + + if (state != null) { + state = activeJobs.remove(jobId); + + assert state != null; + + ctx.shuffle().jobFinished(jobId); + } + + GridFutureAdapter<HadoopJobId> finishFut = activeFinishFuts.remove(jobId); + + if (finishFut != null) { + if (log.isDebugEnabled()) + log.debug("Completing job future [locNodeId=" + locNodeId + ", meta=" + meta + ']'); + + finishFut.onDone(jobId, meta.failCause()); + } + + assert job != null; + + if (ctx.jobUpdateLeader()) + job.cleanupStagingDirectory(); + + jobs.remove(jobId); + + if (ctx.jobUpdateLeader()) { + ClassLoader ldr = job.getClass().getClassLoader(); + + try { + String statWriterClsName = job.info().property(HadoopCommonUtils.JOB_COUNTER_WRITER_PROPERTY); + + if (statWriterClsName != null) { + Class<?> cls = ldr.loadClass(statWriterClsName); + + HadoopCounterWriter writer = (HadoopCounterWriter)cls.newInstance(); + + HadoopCounters cntrs = meta.counters(); + + writer.write(job, cntrs); + } + } + catch (Exception e) { + log.error("Can't write statistic due to: ", e); + } + } + + job.dispose(false); + + break; + } + + default: + throw new IllegalStateException("Unknown phase: " + meta.phase()); + } + } + + /** + * Creates setup task based on job information. + * + * @param jobId Job ID. + * @return Setup task wrapped in collection. + */ + @Nullable private Collection<HadoopTaskInfo> setupTask(HadoopJobId jobId) { + if (activeJobs.containsKey(jobId)) + return null; + else { + initState(jobId); + + return Collections.singleton(new HadoopTaskInfo(SETUP, jobId, 0, 0, null)); + } + } + + /** + * Creates mapper tasks based on job information. + * + * @param mappers Mapper blocks. + * @param meta Job metadata. + * @return Collection of created task infos or {@code null} if no mapper tasks scheduled for local node. + */ + private Collection<HadoopTaskInfo> mapperTasks(Iterable<HadoopInputSplit> mappers, HadoopJobMetadata meta) { + UUID locNodeId = ctx.localNodeId(); + HadoopJobId jobId = meta.jobId(); + + JobLocalState state = activeJobs.get(jobId); + + Collection<HadoopTaskInfo> tasks = null; + + if (mappers != null) { + if (state == null) + state = initState(jobId); + + for (HadoopInputSplit split : mappers) { + if (state.addMapper(split)) { + if (log.isDebugEnabled()) + log.debug("Submitting MAP task for execution [locNodeId=" + locNodeId + + ", split=" + split + ']'); + + HadoopTaskInfo taskInfo = new HadoopTaskInfo(MAP, jobId, meta.taskNumber(split), 0, split); + + if (tasks == null) + tasks = new ArrayList<>(); + + tasks.add(taskInfo); + } + } + } + + return tasks; + } + + /** + * Creates reducer tasks based on job information. + * + * @param reducers Reducers (may be {@code null}). + * @param job Job instance. + * @return Collection of task infos. + */ + private Collection<HadoopTaskInfo> reducerTasks(int[] reducers, HadoopJob job) { + UUID locNodeId = ctx.localNodeId(); + HadoopJobId jobId = job.id(); + + JobLocalState state = activeJobs.get(jobId); + + Collection<HadoopTaskInfo> tasks = null; + + if (reducers != null) { + if (state == null) + state = initState(job.id()); + + for (int rdc : reducers) { + if (state.addReducer(rdc)) { + if (log.isDebugEnabled()) + log.debug("Submitting REDUCE task for execution [locNodeId=" + locNodeId + + ", rdc=" + rdc + ']'); + + HadoopTaskInfo taskInfo = new HadoopTaskInfo(REDUCE, jobId, rdc, 0, null); + + if (tasks == null) + tasks = new ArrayList<>(); + + tasks.add(taskInfo); + } + } + } + + return tasks; + } + + /** + * Initializes local state for given job metadata. + * + * @param jobId Job ID. + * @return Local state. + */ + private JobLocalState initState(HadoopJobId jobId) { + return F.addIfAbsent(activeJobs, jobId, new JobLocalState()); + } + + /** + * Gets or creates job instance. + * + * @param jobId Job ID. + * @param jobInfo Job info. + * @return Job. + * @throws IgniteCheckedException If failed. + */ + @Nullable public HadoopJob job(HadoopJobId jobId, @Nullable HadoopJobInfo jobInfo) throws IgniteCheckedException { + GridFutureAdapter<HadoopJob> fut = jobs.get(jobId); + + if (fut != null || (fut = jobs.putIfAbsent(jobId, new GridFutureAdapter<HadoopJob>())) != null) + return fut.get(); + + fut = jobs.get(jobId); + + HadoopJob job = null; + + try { + if (jobInfo == null) { + HadoopJobMetadata meta = jobMetaCache().get(jobId); + + if (meta == null) + throw new IgniteCheckedException("Failed to find job metadata for ID: " + jobId); + + jobInfo = meta.jobInfo(); + } + + job = jobInfo.createJob(jobCls, jobId, log, ctx.configuration().getNativeLibraryNames(), + ctx.kernalContext().hadoopHelper()); + + job.initialize(false, ctx.localNodeId()); + + fut.onDone(job); + + return job; + } + catch (IgniteCheckedException e) { + fut.onDone(e); + + jobs.remove(jobId, fut); + + if (job != null) { + try { + job.dispose(false); + } + catch (IgniteCheckedException e0) { + U.error(log, "Failed to dispose job: " + jobId, e0); + } + } + + throw e; + } + } + + /** + * Kills job. + * + * @param jobId Job ID. + * @return {@code True} if job was killed. + * @throws IgniteCheckedException If failed. + */ + public boolean killJob(HadoopJobId jobId) throws IgniteCheckedException { + if (!busyLock.tryReadLock()) + return false; // Grid is stopping. + + try { + HadoopJobMetadata meta = jobMetaCache().get(jobId); + + if (meta != null && meta.phase() != PHASE_COMPLETE && meta.phase() != PHASE_CANCELLING) { + HadoopTaskCancelledException err = new HadoopTaskCancelledException("Job cancelled."); + + jobMetaCache().invoke(jobId, new CancelJobProcessor(null, err)); + } + } + finally { + busyLock.readUnlock(); + } + + IgniteInternalFuture<?> fut = finishFuture(jobId); + + if (fut != null) { + try { + fut.get(); + } + catch (Exception e) { + if (e.getCause() instanceof HadoopTaskCancelledException) + return true; + } + } + + return false; + } + + /** + * Returns job counters. + * + * @param jobId Job identifier. + * @return Job counters or {@code null} if job cannot be found. + * @throws IgniteCheckedException If failed. + */ + @Nullable public HadoopCounters jobCounters(HadoopJobId jobId) throws IgniteCheckedException { + if (!busyLock.tryReadLock()) + return null; + + try { + final HadoopJobMetadata meta = jobMetaCache().get(jobId); + + return meta != null ? meta.counters() : null; + } + finally { + busyLock.readUnlock(); + } + } + + /** + * Event handler protected by busy lock. + */ + private abstract class EventHandler implements Runnable { + /** {@inheritDoc} */ + @Override public void run() { + if (!busyLock.tryReadLock()) + return; + + try { + body(); + } + catch (Throwable e) { + U.error(log, "Unhandled exception while processing event.", e); + + if (e instanceof Error) + throw (Error)e; + } + finally { + busyLock.readUnlock(); + } + } + + /** + * Handler body. + */ + protected abstract void body() throws Exception; + } + + /** + * + */ + private class JobLocalState { + /** Mappers. */ + private final Collection<HadoopInputSplit> currMappers = new HashSet<>(); + + /** Reducers. */ + private final Collection<Integer> currReducers = new HashSet<>(); + + /** Number of completed mappers. */ + private final AtomicInteger completedMappersCnt = new AtomicInteger(); + + /** Cancelled flag. */ + private boolean cancelled; + + /** Aborted flag. */ + private boolean aborted; + + /** + * @param mapSplit Map split to add. + * @return {@code True} if mapper was added. + */ + private boolean addMapper(HadoopInputSplit mapSplit) { + return currMappers.add(mapSplit); + } + + /** + * @param rdc Reducer number to add. + * @return {@code True} if reducer was added. + */ + private boolean addReducer(int rdc) { + return currReducers.add(rdc); + } + + /** + * Checks whether this split was scheduled for given attempt. + * + * @param mapSplit Map split to check. + * @return {@code True} if mapper was scheduled. + */ + public boolean mapperScheduled(HadoopInputSplit mapSplit) { + return currMappers.contains(mapSplit); + } + + /** + * Checks whether this split was scheduled for given attempt. + * + * @param rdc Reducer number to check. + * @return {@code True} if reducer was scheduled. + */ + public boolean reducerScheduled(int rdc) { + return currReducers.contains(rdc); + } + + /** + * @param taskInfo Task info. + * @param status Task status. + * @param prev Previous closure. + */ + private void onSetupFinished(final HadoopTaskInfo taskInfo, HadoopTaskStatus status, StackedProcessor prev) { + final HadoopJobId jobId = taskInfo.jobId(); + + if (status.state() == FAILED || status.state() == CRASHED) + transform(jobId, new CancelJobProcessor(prev, status.failCause())); + else + transform(jobId, new UpdatePhaseProcessor(prev, PHASE_MAP)); + } + + /** + * @param taskInfo Task info. + * @param status Task status. + * @param prev Previous closure. + */ + private void onMapFinished(final HadoopTaskInfo taskInfo, HadoopTaskStatus status, + final StackedProcessor prev) { + final HadoopJobId jobId = taskInfo.jobId(); + + boolean lastMapperFinished = completedMappersCnt.incrementAndGet() == currMappers.size(); + + if (status.state() == FAILED || status.state() == CRASHED) { + // Fail the whole job. + transform(jobId, new RemoveMappersProcessor(prev, taskInfo.inputSplit(), status.failCause())); + + return; + } + + IgniteInClosure<IgniteInternalFuture<?>> cacheUpdater = new CIX1<IgniteInternalFuture<?>>() { + @Override public void applyx(IgniteInternalFuture<?> f) { + Throwable err = null; + + if (f != null) { + try { + f.get(); + } + catch (IgniteCheckedException e) { + err = e; + } + } + + transform(jobId, new RemoveMappersProcessor(prev, taskInfo.inputSplit(), err)); + } + }; + + if (lastMapperFinished) + ctx.shuffle().flush(jobId).listen(cacheUpdater); + else + cacheUpdater.apply(null); + } + + /** + * @param taskInfo Task info. + * @param status Task status. + * @param prev Previous closure. + */ + private void onReduceFinished(HadoopTaskInfo taskInfo, HadoopTaskStatus status, StackedProcessor prev) { + HadoopJobId jobId = taskInfo.jobId(); + if (status.state() == FAILED || status.state() == CRASHED) + // Fail the whole job. + transform(jobId, new RemoveReducerProcessor(prev, taskInfo.taskNumber(), status.failCause())); + else + transform(jobId, new RemoveReducerProcessor(prev, taskInfo.taskNumber())); + } + + /** + * @param taskInfo Task info. + * @param status Task status. + * @param prev Previous closure. + */ + private void onCombineFinished(HadoopTaskInfo taskInfo, HadoopTaskStatus status, + final StackedProcessor prev) { + final HadoopJobId jobId = taskInfo.jobId(); + + if (status.state() == FAILED || status.state() == CRASHED) + // Fail the whole job. + transform(jobId, new RemoveMappersProcessor(prev, currMappers, status.failCause())); + else { + ctx.shuffle().flush(jobId).listen(new CIX1<IgniteInternalFuture<?>>() { + @Override public void applyx(IgniteInternalFuture<?> f) { + Throwable err = null; + + if (f != null) { + try { + f.get(); + } + catch (IgniteCheckedException e) { + err = e; + } + } + + transform(jobId, new RemoveMappersProcessor(prev, currMappers, err)); + } + }); + } + } + + /** + * @return {@code True} if job was cancelled by this (first) call. + */ + public boolean onCancel() { + if (!cancelled && !aborted) { + cancelled = true; + + return true; + } + + return false; + } + + /** + * @return {@code True} if job was aborted this (first) call. + */ + public boolean onAborted() { + if (!aborted) { + aborted = true; + + return true; + } + + return false; + } + } + + /** + * Update job phase transform closure. + */ + private static class UpdatePhaseProcessor extends StackedProcessor { + /** */ + private static final long serialVersionUID = 0L; + + /** Phase to update. */ + private final HadoopJobPhase phase; + + /** + * @param prev Previous closure. + * @param phase Phase to update. + */ + private UpdatePhaseProcessor(@Nullable StackedProcessor prev, HadoopJobPhase phase) { + super(prev); + + this.phase = phase; + } + + /** {@inheritDoc} */ + @Override protected void update(HadoopJobMetadata meta, HadoopJobMetadata cp) { + cp.phase(phase); + } + } + + /** + * Remove mapper transform closure. + */ + private static class RemoveMappersProcessor extends StackedProcessor { + /** */ + private static final long serialVersionUID = 0L; + + /** Mapper split to remove. */ + private final Collection<HadoopInputSplit> splits; + + /** Error. */ + private final Throwable err; + + /** + * @param prev Previous closure. + * @param split Mapper split to remove. + * @param err Error. + */ + private RemoveMappersProcessor(@Nullable StackedProcessor prev, HadoopInputSplit split, Throwable err) { + this(prev, Collections.singletonList(split), err); + } + + /** + * @param prev Previous closure. + * @param splits Mapper splits to remove. + * @param err Error. + */ + private RemoveMappersProcessor(@Nullable StackedProcessor prev, Collection<HadoopInputSplit> splits, + Throwable err) { + super(prev); + + this.splits = splits; + this.err = err; + } + + /** {@inheritDoc} */ + @Override protected void update(HadoopJobMetadata meta, HadoopJobMetadata cp) { + Map<HadoopInputSplit, Integer> splitsCp = new HashMap<>(cp.pendingSplits()); + + for (HadoopInputSplit s : splits) + splitsCp.remove(s); + + cp.pendingSplits(splitsCp); + + if (cp.phase() != PHASE_CANCELLING && err != null) + cp.failCause(err); + + if (err != null) + cp.phase(PHASE_CANCELLING); + + if (splitsCp.isEmpty()) { + if (cp.phase() != PHASE_CANCELLING) + cp.phase(PHASE_REDUCE); + } + } + } + + /** + * Remove reducer transform closure. + */ + private static class RemoveReducerProcessor extends StackedProcessor { + /** */ + private static final long serialVersionUID = 0L; + + /** Mapper split to remove. */ + private final int rdc; + + /** Error. */ + private Throwable err; + + /** + * @param prev Previous closure. + * @param rdc Reducer to remove. + */ + private RemoveReducerProcessor(@Nullable StackedProcessor prev, int rdc) { + super(prev); + + this.rdc = rdc; + } + + /** + * @param prev Previous closure. + * @param rdc Reducer to remove. + * @param err Error. + */ + private RemoveReducerProcessor(@Nullable StackedProcessor prev, int rdc, Throwable err) { + super(prev); + + this.rdc = rdc; + this.err = err; + } + + /** {@inheritDoc} */ + @Override protected void update(HadoopJobMetadata meta, HadoopJobMetadata cp) { + Collection<Integer> rdcCp = new HashSet<>(cp.pendingReducers()); + + rdcCp.remove(rdc); + + cp.pendingReducers(rdcCp); + + if (err != null) { + cp.phase(PHASE_CANCELLING); + cp.failCause(err); + } + } + } + + /** + * Initialize reducers. + */ + private static class InitializeReducersProcessor extends StackedProcessor { + /** */ + private static final long serialVersionUID = 0L; + + /** Reducers. */ + private final Collection<Integer> rdc; + + /** Process descriptor for reducers. */ + private final HadoopProcessDescriptor desc; + + /** + * @param prev Previous closure. + * @param rdc Reducers to initialize. + * @param desc External process descriptor. + */ + private InitializeReducersProcessor(@Nullable StackedProcessor prev, + Collection<Integer> rdc, + HadoopProcessDescriptor desc) { + super(prev); + + assert !F.isEmpty(rdc); + assert desc != null; + + this.rdc = rdc; + this.desc = desc; + } + + /** {@inheritDoc} */ + @Override protected void update(HadoopJobMetadata meta, HadoopJobMetadata cp) { + Map<Integer, HadoopProcessDescriptor> oldMap = meta.reducersAddresses(); + + Map<Integer, HadoopProcessDescriptor> rdcMap = oldMap == null ? + new HashMap<Integer, HadoopProcessDescriptor>() : new HashMap<>(oldMap); + + for (Integer r : rdc) + rdcMap.put(r, desc); + + cp.reducersAddresses(rdcMap); + } + } + + /** + * Remove reducer transform closure. + */ + private static class CancelJobProcessor extends StackedProcessor { + /** */ + private static final long serialVersionUID = 0L; + + /** Mapper split to remove. */ + private final Collection<HadoopInputSplit> splits; + + /** Reducers to remove. */ + private final Collection<Integer> rdc; + + /** Error. */ + private final Throwable err; + + /** + * @param prev Previous closure. + * @param err Fail cause. + */ + private CancelJobProcessor(@Nullable StackedProcessor prev, Throwable err) { + this(prev, err, null, null); + } + + /** + * @param prev Previous closure. + * @param splits Splits to remove. + * @param rdc Reducers to remove. + */ + private CancelJobProcessor(@Nullable StackedProcessor prev, + Collection<HadoopInputSplit> splits, + Collection<Integer> rdc) { + this(prev, null, splits, rdc); + } + + /** + * @param prev Previous closure. + * @param err Error. + * @param splits Splits to remove. + * @param rdc Reducers to remove. + */ + private CancelJobProcessor(@Nullable StackedProcessor prev, + Throwable err, + Collection<HadoopInputSplit> splits, + Collection<Integer> rdc) { + super(prev); + + this.splits = splits; + this.rdc = rdc; + this.err = err; + } + + /** {@inheritDoc} */ + @Override protected void update(HadoopJobMetadata meta, HadoopJobMetadata cp) { + final HadoopJobPhase currPhase = meta.phase(); + + assert currPhase == PHASE_CANCELLING || currPhase == PHASE_COMPLETE + || err != null: "Invalid phase for cancel: " + currPhase; + + Collection<Integer> rdcCp = new HashSet<>(cp.pendingReducers()); + + if (rdc != null) + rdcCp.removeAll(rdc); + + cp.pendingReducers(rdcCp); + + Map<HadoopInputSplit, Integer> splitsCp = new HashMap<>(cp.pendingSplits()); + + if (splits != null) { + for (HadoopInputSplit s : splits) + splitsCp.remove(s); + } + + cp.pendingSplits(splitsCp); + + if (currPhase != PHASE_COMPLETE && currPhase != PHASE_CANCELLING) + cp.phase(PHASE_CANCELLING); + + if (err != null) + cp.failCause(err); + } + } + + /** + * Increment counter values closure. + */ + private static class IncrementCountersProcessor extends StackedProcessor { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private final HadoopCounters counters; + + /** + * @param prev Previous closure. + * @param counters Task counters to add into job counters. + */ + private IncrementCountersProcessor(@Nullable StackedProcessor prev, HadoopCounters counters) { + super(prev); + + assert counters != null; + + this.counters = counters; + } + + /** {@inheritDoc} */ + @Override protected void update(HadoopJobMetadata meta, HadoopJobMetadata cp) { + HadoopCounters cntrs = new HadoopCountersImpl(cp.counters()); + + cntrs.merge(counters); + + cp.counters(cntrs); + } + } + + /** + * Abstract stacked closure. + */ + private abstract static class StackedProcessor implements + EntryProcessor<HadoopJobId, HadoopJobMetadata, Void>, Serializable { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private final StackedProcessor prev; + + /** + * @param prev Previous closure. + */ + private StackedProcessor(@Nullable StackedProcessor prev) { + this.prev = prev; + } + + /** {@inheritDoc} */ + @Override public Void process(MutableEntry<HadoopJobId, HadoopJobMetadata> e, Object... args) { + HadoopJobMetadata val = apply(e.getValue()); + + if (val != null) + e.setValue(val); + else + e.remove(); + + return null; + } + + /** + * @param meta Old value. + * @return New value. + */ + private HadoopJobMetadata apply(HadoopJobMetadata meta) { + if (meta == null) + return null; + + HadoopJobMetadata cp = prev != null ? prev.apply(meta) : new HadoopJobMetadata(meta); + + update(meta, cp); + + return cp; + } + + /** + * Update given job metadata object. + * + * @param meta Initial job metadata. + * @param cp Copy. + */ + protected abstract void update(HadoopJobMetadata meta, HadoopJobMetadata cp); + } +}
http://git-wip-us.apache.org/repos/asf/ignite/blob/facedf50/modules/hadoop-impl/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopEmbeddedTaskExecutor.java ---------------------------------------------------------------------- diff --git a/modules/hadoop-impl/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopEmbeddedTaskExecutor.java b/modules/hadoop-impl/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopEmbeddedTaskExecutor.java new file mode 100644 index 0000000..5ede18e --- /dev/null +++ b/modules/hadoop-impl/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopEmbeddedTaskExecutor.java @@ -0,0 +1,153 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.hadoop.taskexecutor; + +import java.util.Collection; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.processors.hadoop.HadoopJob; +import org.apache.ignite.internal.processors.hadoop.HadoopJobId; +import org.apache.ignite.internal.processors.hadoop.HadoopJobPhase; +import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext; +import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo; +import org.apache.ignite.internal.processors.hadoop.HadoopTaskInput; +import org.apache.ignite.internal.processors.hadoop.HadoopTaskOutput; +import org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobMetadata; +import org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobTracker; +import org.apache.ignite.internal.util.GridConcurrentHashSet; +import org.apache.ignite.internal.util.typedef.internal.U; + + +/** + * Task executor. + */ +public class HadoopEmbeddedTaskExecutor extends HadoopTaskExecutorAdapter { + /** Job tracker. */ + private HadoopJobTracker jobTracker; + + /** */ + private final ConcurrentMap<HadoopJobId, Collection<HadoopRunnableTask>> jobs = new ConcurrentHashMap<>(); + + /** Executor service to run tasks. */ + private HadoopExecutorService exec; + + /** {@inheritDoc} */ + @Override public void onKernalStart() throws IgniteCheckedException { + super.onKernalStart(); + + jobTracker = ctx.jobTracker(); + + exec = new HadoopExecutorService(log, ctx.kernalContext().gridName(), + ctx.configuration().getMaxParallelTasks(), ctx.configuration().getMaxTaskQueueSize()); + } + + /** {@inheritDoc} */ + @Override public void onKernalStop(boolean cancel) { + if (exec != null) { + exec.shutdown(3000); + + if (cancel) { + for (HadoopJobId jobId : jobs.keySet()) + cancelTasks(jobId); + } + } + } + + /** {@inheritDoc} */ + @Override public void stop(boolean cancel) { + if (exec != null && !exec.shutdown(30000)) + U.warn(log, "Failed to finish running tasks in 30 sec."); + } + + /** {@inheritDoc} */ + @Override public void run(final HadoopJob job, Collection<HadoopTaskInfo> tasks) throws IgniteCheckedException { + if (log.isDebugEnabled()) + log.debug("Submitting tasks for local execution [locNodeId=" + ctx.localNodeId() + + ", tasksCnt=" + tasks.size() + ']'); + + Collection<HadoopRunnableTask> executedTasks = jobs.get(job.id()); + + if (executedTasks == null) { + executedTasks = new GridConcurrentHashSet<>(); + + Collection<HadoopRunnableTask> extractedCol = jobs.put(job.id(), executedTasks); + + assert extractedCol == null; + } + + final Collection<HadoopRunnableTask> finalExecutedTasks = executedTasks; + + for (final HadoopTaskInfo info : tasks) { + assert info != null; + + HadoopRunnableTask task = new HadoopRunnableTask(log, job, ctx.shuffle().memory(), info, + ctx.localNodeId()) { + @Override protected void onTaskFinished(HadoopTaskStatus status) { + if (log.isDebugEnabled()) + log.debug("Finished task execution [jobId=" + job.id() + ", taskInfo=" + info + ", " + + "waitTime=" + waitTime() + ", execTime=" + executionTime() + ']'); + + finalExecutedTasks.remove(this); + + jobTracker.onTaskFinished(info, status); + } + + @Override protected HadoopTaskInput createInput(HadoopTaskContext taskCtx) throws IgniteCheckedException { + return ctx.shuffle().input(taskCtx); + } + + @Override protected HadoopTaskOutput createOutput(HadoopTaskContext taskCtx) throws IgniteCheckedException { + return ctx.shuffle().output(taskCtx); + } + }; + + executedTasks.add(task); + + exec.submit(task); + } + } + + /** + * Cancels all currently running tasks for given job ID and cancels scheduled execution of tasks + * for this job ID. + * <p> + * It is guaranteed that this method will not be called concurrently with + * {@link #run(org.apache.ignite.internal.processors.hadoop.HadoopJob, Collection)} method. No more job submissions will be performed via + * {@link #run(org.apache.ignite.internal.processors.hadoop.HadoopJob, Collection)} method for given job ID after this method is called. + * + * @param jobId Job ID to cancel. + */ + @Override public void cancelTasks(HadoopJobId jobId) { + Collection<HadoopRunnableTask> executedTasks = jobs.get(jobId); + + if (executedTasks != null) { + for (HadoopRunnableTask task : executedTasks) + task.cancel(); + } + } + + /** {@inheritDoc} */ + @Override public void onJobStateChanged(HadoopJobMetadata meta) throws IgniteCheckedException { + if (meta.phase() == HadoopJobPhase.PHASE_COMPLETE) { + Collection<HadoopRunnableTask> executedTasks = jobs.remove(meta.jobId()); + + assert executedTasks == null || executedTasks.isEmpty(); + } + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/facedf50/modules/hadoop-impl/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopExecutorService.java ---------------------------------------------------------------------- diff --git a/modules/hadoop-impl/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopExecutorService.java b/modules/hadoop-impl/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopExecutorService.java new file mode 100644 index 0000000..993ecc9 --- /dev/null +++ b/modules/hadoop-impl/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopExecutorService.java @@ -0,0 +1,234 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.hadoop.taskexecutor; + + +import java.util.Collection; +import java.util.concurrent.Callable; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo; +import org.apache.ignite.internal.util.worker.GridWorker; +import org.apache.ignite.internal.util.worker.GridWorkerListener; +import org.apache.ignite.internal.util.worker.GridWorkerListenerAdapter; +import org.apache.ignite.thread.IgniteThread; +import org.jsr166.ConcurrentHashMap8; + +import static java.util.Collections.newSetFromMap; + +/** + * Executor service without thread pooling. + */ +public class HadoopExecutorService { + /** */ + private final LinkedBlockingQueue<Callable<?>> queue; + + /** */ + private final Collection<GridWorker> workers = newSetFromMap(new ConcurrentHashMap8<GridWorker, Boolean>()); + + /** */ + private final AtomicInteger active = new AtomicInteger(); + + /** */ + private final int maxTasks; + + /** */ + private final String gridName; + + /** */ + private final IgniteLogger log; + + /** */ + private volatile boolean shutdown; + + /** */ + private final GridWorkerListener lsnr = new GridWorkerListenerAdapter() { + @Override public void onStopped(GridWorker w) { + workers.remove(w); + + if (shutdown) { + active.decrementAndGet(); + + return; + } + + Callable<?> task = queue.poll(); + + if (task != null) + startThread(task); + else { + active.decrementAndGet(); + + if (!queue.isEmpty()) + startFromQueue(); + } + } + }; + + /** + * @param log Logger. + * @param gridName Grid name. + * @param maxTasks Max number of tasks. + * @param maxQueue Max queue length. + */ + public HadoopExecutorService(IgniteLogger log, String gridName, int maxTasks, int maxQueue) { + assert maxTasks > 0 : maxTasks; + assert maxQueue > 0 : maxQueue; + + this.maxTasks = maxTasks; + this.queue = new LinkedBlockingQueue<>(maxQueue); + this.gridName = gridName; + this.log = log.getLogger(HadoopExecutorService.class); + } + + /** + * @return Number of active workers. + */ + public int active() { + return workers.size(); + } + + /** + * Submit task. + * + * @param task Task. + */ + public void submit(Callable<?> task) { + while (queue.isEmpty()) { + int active0 = active.get(); + + if (active0 == maxTasks) + break; + + if (active.compareAndSet(active0, active0 + 1)) { + startThread(task); + + return; // Started in new thread bypassing queue. + } + } + + try { + while (!queue.offer(task, 100, TimeUnit.MILLISECONDS)) { + if (shutdown) + return; // Rejected due to shutdown. + } + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + + return; + } + + startFromQueue(); + } + + /** + * Attempts to start task from queue. + */ + private void startFromQueue() { + do { + int active0 = active.get(); + + if (active0 == maxTasks) + break; + + if (active.compareAndSet(active0, active0 + 1)) { + Callable<?> task = queue.poll(); + + if (task == null) { + int res = active.decrementAndGet(); + + assert res >= 0 : res; + + break; + } + + startThread(task); + } + } + while (!queue.isEmpty()); + } + + /** + * @param task Task. + */ + private void startThread(final Callable<?> task) { + String workerName; + + if (task instanceof HadoopRunnableTask) { + final HadoopTaskInfo i = ((HadoopRunnableTask)task).taskInfo(); + + workerName = "Hadoop-task-" + i.jobId() + "-" + i.type() + "-" + i.taskNumber() + "-" + i.attempt(); + } + else + workerName = task.toString(); + + GridWorker w = new GridWorker(gridName, workerName, log, lsnr) { + @Override protected void body() { + try { + task.call(); + } + catch (Exception e) { + log.error("Failed to execute task: " + task, e); + } + } + }; + + workers.add(w); + + if (shutdown) + w.cancel(); + + new IgniteThread(w).start(); + } + + /** + * Shuts down this executor service. + * + * @param awaitTimeMillis Time in milliseconds to wait for tasks completion. + * @return {@code true} If all tasks completed. + */ + public boolean shutdown(long awaitTimeMillis) { + shutdown = true; + + for (GridWorker w : workers) + w.cancel(); + + while (awaitTimeMillis > 0 && !workers.isEmpty()) { + try { + Thread.sleep(100); + + awaitTimeMillis -= 100; + } + catch (InterruptedException e) { + break; + } + } + + return workers.isEmpty(); + } + + /** + * @return {@code true} If method {@linkplain #shutdown(long)} was already called. + */ + public boolean isShutdown() { + return shutdown; + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/facedf50/modules/hadoop-impl/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopRunnableTask.java ---------------------------------------------------------------------- diff --git a/modules/hadoop-impl/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopRunnableTask.java b/modules/hadoop-impl/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopRunnableTask.java new file mode 100644 index 0000000..a57efe6 --- /dev/null +++ b/modules/hadoop-impl/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopRunnableTask.java @@ -0,0 +1,293 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.hadoop.taskexecutor; + +import java.util.UUID; +import java.util.concurrent.Callable; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.processors.hadoop.HadoopJob; +import org.apache.ignite.internal.processors.hadoop.HadoopTaskCancelledException; +import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext; +import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo; +import org.apache.ignite.internal.processors.hadoop.HadoopTaskInput; +import org.apache.ignite.internal.processors.hadoop.HadoopTaskOutput; +import org.apache.ignite.internal.processors.hadoop.counter.HadoopPerformanceCounter; +import org.apache.ignite.internal.processors.hadoop.shuffle.collections.HadoopHashMultimap; +import org.apache.ignite.internal.processors.hadoop.shuffle.collections.HadoopMultimap; +import org.apache.ignite.internal.processors.hadoop.shuffle.collections.HadoopSkipList; +import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory; +import org.apache.ignite.internal.util.typedef.internal.U; + +import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.COMBINER_HASHMAP_SIZE; +import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.SHUFFLE_COMBINER_NO_SORTING; +import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.get; +import static org.apache.ignite.internal.processors.hadoop.HadoopTaskType.COMBINE; +import static org.apache.ignite.internal.processors.hadoop.HadoopTaskType.MAP; + +/** + * Runnable task. + */ +public abstract class HadoopRunnableTask implements Callable<Void> { + /** */ + private final GridUnsafeMemory mem; + + /** */ + private final IgniteLogger log; + + /** */ + private final HadoopJob job; + + /** Task to run. */ + private final HadoopTaskInfo info; + + /** Submit time. */ + private final long submitTs = U.currentTimeMillis(); + + /** Execution start timestamp. */ + private long execStartTs; + + /** Execution end timestamp. */ + private long execEndTs; + + /** */ + private HadoopMultimap combinerInput; + + /** */ + private volatile HadoopTaskContext ctx; + + /** Set if task is to cancelling. */ + private volatile boolean cancelled; + + /** Node id. */ + private UUID nodeId; + + /** + * @param log Log. + * @param job Job. + * @param mem Memory. + * @param info Task info. + * @param nodeId Node id. + */ + protected HadoopRunnableTask(IgniteLogger log, HadoopJob job, GridUnsafeMemory mem, HadoopTaskInfo info, + UUID nodeId) { + this.nodeId = nodeId; + this.log = log.getLogger(HadoopRunnableTask.class); + this.job = job; + this.mem = mem; + this.info = info; + } + + /** + * @return Wait time. + */ + public long waitTime() { + return execStartTs - submitTs; + } + + /** + * @return Execution time. + */ + public long executionTime() { + return execEndTs - execStartTs; + } + + /** {@inheritDoc} */ + @Override public Void call() throws IgniteCheckedException { + ctx = job.getTaskContext(info); + + return ctx.runAsJobOwner(new Callable<Void>() { + @Override public Void call() throws Exception { + call0(); + + return null; + } + }); + } + + /** + * Implements actual task running. + * @throws IgniteCheckedException + */ + void call0() throws IgniteCheckedException { + execStartTs = U.currentTimeMillis(); + + Throwable err = null; + + HadoopTaskState state = HadoopTaskState.COMPLETED; + + HadoopPerformanceCounter perfCntr = null; + + try { + perfCntr = HadoopPerformanceCounter.getCounter(ctx.counters(), nodeId); + + perfCntr.onTaskSubmit(info, submitTs); + perfCntr.onTaskPrepare(info, execStartTs); + + ctx.prepareTaskEnvironment(); + + runTask(perfCntr); + + if (info.type() == MAP && job.info().hasCombiner()) { + ctx.taskInfo(new HadoopTaskInfo(COMBINE, info.jobId(), info.taskNumber(), info.attempt(), null)); + + try { + runTask(perfCntr); + } + finally { + ctx.taskInfo(info); + } + } + } + catch (HadoopTaskCancelledException ignored) { + state = HadoopTaskState.CANCELED; + } + catch (Throwable e) { + state = HadoopTaskState.FAILED; + err = e; + + U.error(log, "Task execution failed.", e); + + if (e instanceof Error) + throw e; + } + finally { + execEndTs = U.currentTimeMillis(); + + if (perfCntr != null) + perfCntr.onTaskFinish(info, execEndTs); + + onTaskFinished(new HadoopTaskStatus(state, err, ctx==null ? null : ctx.counters())); + + if (combinerInput != null) + combinerInput.close(); + + if (ctx != null) + ctx.cleanupTaskEnvironment(); + } + } + + /** + * @param perfCntr Performance counter. + * @throws IgniteCheckedException If failed. + */ + private void runTask(HadoopPerformanceCounter perfCntr) throws IgniteCheckedException { + if (cancelled) + throw new HadoopTaskCancelledException("Task cancelled."); + + try (HadoopTaskOutput out = createOutputInternal(ctx); + HadoopTaskInput in = createInputInternal(ctx)) { + + ctx.input(in); + ctx.output(out); + + perfCntr.onTaskStart(ctx.taskInfo(), U.currentTimeMillis()); + + ctx.run(); + } + } + + /** + * Cancel the executed task. + */ + public void cancel() { + cancelled = true; + + if (ctx != null) + ctx.cancel(); + } + + /** + * @param status Task status. + */ + protected abstract void onTaskFinished(HadoopTaskStatus status); + + /** + * @param ctx Task context. + * @return Task input. + * @throws IgniteCheckedException If failed. + */ + @SuppressWarnings("unchecked") + private HadoopTaskInput createInputInternal(HadoopTaskContext ctx) throws IgniteCheckedException { + switch (ctx.taskInfo().type()) { + case SETUP: + case MAP: + case COMMIT: + case ABORT: + return null; + + case COMBINE: + assert combinerInput != null; + + return combinerInput.input(ctx); + + default: + return createInput(ctx); + } + } + + /** + * @param ctx Task context. + * @return Input. + * @throws IgniteCheckedException If failed. + */ + protected abstract HadoopTaskInput createInput(HadoopTaskContext ctx) throws IgniteCheckedException; + + /** + * @param ctx Task info. + * @return Output. + * @throws IgniteCheckedException If failed. + */ + protected abstract HadoopTaskOutput createOutput(HadoopTaskContext ctx) throws IgniteCheckedException; + + /** + * @param ctx Task info. + * @return Task output. + * @throws IgniteCheckedException If failed. + */ + private HadoopTaskOutput createOutputInternal(HadoopTaskContext ctx) throws IgniteCheckedException { + switch (ctx.taskInfo().type()) { + case SETUP: + case REDUCE: + case COMMIT: + case ABORT: + return null; + + case MAP: + if (job.info().hasCombiner()) { + assert combinerInput == null; + + combinerInput = get(job.info(), SHUFFLE_COMBINER_NO_SORTING, false) ? + new HadoopHashMultimap(job.info(), mem, get(job.info(), COMBINER_HASHMAP_SIZE, 8 * 1024)): + new HadoopSkipList(job.info(), mem); // TODO replace with red-black tree + + return combinerInput.startAdding(ctx); + } + + default: + return createOutput(ctx); + } + } + + /** + * @return Task info. + */ + public HadoopTaskInfo taskInfo() { + return info; + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/facedf50/modules/hadoop-impl/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopTaskExecutorAdapter.java ---------------------------------------------------------------------- diff --git a/modules/hadoop-impl/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopTaskExecutorAdapter.java b/modules/hadoop-impl/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopTaskExecutorAdapter.java new file mode 100644 index 0000000..f13c76a --- /dev/null +++ b/modules/hadoop-impl/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopTaskExecutorAdapter.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.hadoop.taskexecutor; + +import java.util.Collection; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.processors.hadoop.HadoopComponent; +import org.apache.ignite.internal.processors.hadoop.HadoopJob; +import org.apache.ignite.internal.processors.hadoop.HadoopJobId; +import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo; +import org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobMetadata; + +/** + * Common superclass for task executor. + */ +public abstract class HadoopTaskExecutorAdapter extends HadoopComponent { + /** + * Runs tasks. + * + * @param job Job. + * @param tasks Tasks. + * @throws IgniteCheckedException If failed. + */ + public abstract void run(final HadoopJob job, Collection<HadoopTaskInfo> tasks) throws IgniteCheckedException; + + /** + * Cancels all currently running tasks for given job ID and cancels scheduled execution of tasks + * for this job ID. + * <p> + * It is guaranteed that this method will not be called concurrently with + * {@link #run(org.apache.ignite.internal.processors.hadoop.HadoopJob, Collection)} method. No more job submissions will be performed via + * {@link #run(org.apache.ignite.internal.processors.hadoop.HadoopJob, Collection)} method for given job ID after this method is called. + * + * @param jobId Job ID to cancel. + */ + public abstract void cancelTasks(HadoopJobId jobId) throws IgniteCheckedException; + + /** + * On job state change callback; + * + * @param meta Job metadata. + */ + public abstract void onJobStateChanged(HadoopJobMetadata meta) throws IgniteCheckedException; +} \ No newline at end of file