public void run() { try { TaskAttemptID mapId = new TaskAttemptID(new TaskID(jobId, true, taskId), 0); LOG.info("Starting task: " + mapId); mapIds.add(mapId); MapTask map = new MapTask(systemJobFile.toString(), mapId, taskId, info.getSplitIndex(), 1); map.setUser(UserGroupInformation.getCurrentUser().getShortUserName()); TaskRunner.setupChildMapredLocalDirs(map, localConf); MapOutputFile mapOutput = new MapOutputFile(); mapOutput.setConf(localConf); mapOutputFiles.put(mapId, mapOutput); map.setJobFile(localJobFile.toString()); localConf.setUser(map.getUser()); map.localizeConfiguration(localConf); map.setConf(localConf); try { map_tasks.getAndIncrement(); myMetrics.launchMap(mapId); map.run(localConf, Job.this); myMetrics.completeMap(mapId); } finally { map_tasks.getAndDecrement(); } LOG.info("Finishing task: " + mapId); } catch (Throwable e) { this.storedException = e; } }
/** * Creates the configuration for mumak simulation. This is kept modular mostly for testing * purposes. so that the standard configuration can be modified before passing it to the init() * function. * * @return JobConf: the configuration for the SimulatorJobTracker */ JobConf createMumakConf() { JobConf jobConf = new JobConf(getConf()); jobConf.setClass( "topology.node.switch.mapping.impl", StaticMapping.class, DNSToSwitchMapping.class); jobConf.set("fs.default.name", "file:///"); jobConf.set("mapred.job.tracker", "localhost:8012"); jobConf.setInt("mapred.jobtracker.job.history.block.size", 512); jobConf.setInt("mapred.jobtracker.job.history.buffer.size", 512); jobConf.setLong("mapred.tasktracker.expiry.interval", 5000); jobConf.setInt("mapred.reduce.copy.backoff", 4); jobConf.setLong("mapred.job.reuse.jvm.num.tasks", -1); jobConf.setUser("mumak"); jobConf.set( "mapred.system.dir", jobConf.get("hadoop.log.dir", "/tmp/hadoop-" + jobConf.getUser()) + "/mapred/system"); return jobConf; }
@SuppressWarnings("unchecked") @Override public void run() { JobID jobId = profile.getJobID(); JobContext jContext = new JobContextImpl(conf, jobId); org.apache.hadoop.mapreduce.OutputCommitter outputCommitter = null; try { outputCommitter = createOutputCommitter(conf.getUseNewMapper(), jobId, conf); } catch (Exception e) { LOG.info("Failed to createOutputCommitter", e); return; } try { TaskSplitMetaInfo[] taskSplitMetaInfos = SplitMetaInfoReader.readSplitMetaInfo(jobId, localFs, conf, systemJobDir); int numReduceTasks = job.getNumReduceTasks(); if (numReduceTasks > 1 || numReduceTasks < 0) { // we only allow 0 or 1 reducer in local mode numReduceTasks = 1; job.setNumReduceTasks(1); } outputCommitter.setupJob(jContext); status.setSetupProgress(1.0f); Map<TaskAttemptID, MapOutputFile> mapOutputFiles = Collections.synchronizedMap(new HashMap<TaskAttemptID, MapOutputFile>()); List<MapTaskRunnable> taskRunnables = getMapTaskRunnables(taskSplitMetaInfos, jobId, mapOutputFiles); ExecutorService mapService = createMapExecutor(taskRunnables.size()); // Start populating the executor with work units. // They may begin running immediately (in other threads). for (Runnable r : taskRunnables) { mapService.submit(r); } try { mapService.shutdown(); // Instructs queue to drain. // Wait for tasks to finish; do not use a time-based timeout. // (See http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6179024) LOG.info("Waiting for map tasks"); mapService.awaitTermination(Long.MAX_VALUE, TimeUnit.NANOSECONDS); } catch (InterruptedException ie) { // Cancel all threads. mapService.shutdownNow(); throw ie; } LOG.info("Map task executor complete."); // After waiting for the map tasks to complete, if any of these // have thrown an exception, rethrow it now in the main thread context. for (MapTaskRunnable r : taskRunnables) { if (r.storedException != null) { throw new Exception(r.storedException); } } TaskAttemptID reduceId = new TaskAttemptID(new TaskID(jobId, false, 0), 0); try { if (numReduceTasks > 0) { ReduceTask reduce = new ReduceTask(systemJobFile.toString(), reduceId, 0, mapIds.size(), 1); reduce.setUser(UserGroupInformation.getCurrentUser().getShortUserName()); JobConf localConf = new JobConf(job); localConf.set("mapreduce.jobtracker.address", "local"); TaskRunner.setupChildMapredLocalDirs(reduce, localConf); // move map output to reduce input for (int i = 0; i < mapIds.size(); i++) { if (!this.isInterrupted()) { TaskAttemptID mapId = mapIds.get(i); Path mapOut = mapOutputFiles.get(mapId).getOutputFile(); MapOutputFile localOutputFile = new MapOutputFile(); localOutputFile.setConf(localConf); Path reduceIn = localOutputFile.getInputFileForWrite( mapId.getTaskID(), localFs.getFileStatus(mapOut).getLen()); if (!localFs.mkdirs(reduceIn.getParent())) { throw new IOException( "Mkdirs failed to create " + reduceIn.getParent().toString()); } if (!localFs.rename(mapOut, reduceIn)) throw new IOException("Couldn't rename " + mapOut); } else { throw new InterruptedException(); } } if (!this.isInterrupted()) { reduce.setJobFile(localJobFile.toString()); localConf.setUser(reduce.getUser()); reduce.localizeConfiguration(localConf); reduce.setConf(localConf); reduce_tasks += 1; myMetrics.launchReduce(reduce.getTaskID()); reduce.run(localConf, this); myMetrics.completeReduce(reduce.getTaskID()); reduce_tasks -= 1; } else { throw new InterruptedException(); } } } finally { for (MapOutputFile output : mapOutputFiles.values()) { output.removeAll(); } } // delete the temporary directory in output directory outputCommitter.commitJob(jContext); status.setCleanupProgress(1.0f); if (killed) { this.status.setRunState(JobStatus.KILLED); } else { this.status.setRunState(JobStatus.SUCCEEDED); } JobEndNotifier.localRunnerNotification(job, status); } catch (Throwable t) { try { outputCommitter.abortJob(jContext, org.apache.hadoop.mapreduce.JobStatus.State.FAILED); } catch (IOException ioe) { LOG.info("Error cleaning up job:" + id); } status.setCleanupProgress(1.0f); if (killed) { this.status.setRunState(JobStatus.KILLED); } else { this.status.setRunState(JobStatus.FAILED); } LOG.warn(id, t); JobEndNotifier.localRunnerNotification(job, status); } finally { try { fs.delete(systemJobFile.getParent(), true); // delete submit dir localFs.delete(localJobFile, true); // delete local copy // Cleanup distributed cache taskDistributedCacheManager.release(); trackerDistributedCacheManager.purgeCache(); } catch (IOException e) { LOG.warn("Error cleaning up " + id + ": " + e); } } }