/* * Creates the configuration object necessary to run a specific vertex from * map work. This includes input formats, input processor, etc. */ private JobConf initializeVertexConf(JobConf baseConf, MapWork mapWork) { JobConf conf = new JobConf(baseConf); if (mapWork.getNumMapTasks() != null) { conf.setInt(MRJobConfig.NUM_MAPS, mapWork.getNumMapTasks().intValue()); } if (mapWork.getMaxSplitSize() != null) { HiveConf.setLongVar( conf, HiveConf.ConfVars.MAPREDMAXSPLITSIZE, mapWork.getMaxSplitSize().longValue()); } if (mapWork.getMinSplitSize() != null) { HiveConf.setLongVar( conf, HiveConf.ConfVars.MAPREDMINSPLITSIZE, mapWork.getMinSplitSize().longValue()); } if (mapWork.getMinSplitSizePerNode() != null) { HiveConf.setLongVar( conf, HiveConf.ConfVars.MAPREDMINSPLITSIZEPERNODE, mapWork.getMinSplitSizePerNode().longValue()); } if (mapWork.getMinSplitSizePerRack() != null) { HiveConf.setLongVar( conf, HiveConf.ConfVars.MAPREDMINSPLITSIZEPERRACK, mapWork.getMinSplitSizePerRack().longValue()); } Utilities.setInputAttributes(conf, mapWork); String inpFormat = HiveConf.getVar(conf, HiveConf.ConfVars.HIVETEZINPUTFORMAT); if ((inpFormat == null) || (!StringUtils.isNotBlank(inpFormat))) { inpFormat = ShimLoader.getHadoopShims().getInputFormatClassName(); } if (mapWork.isUseBucketizedHiveInputFormat()) { inpFormat = BucketizedHiveInputFormat.class.getName(); } conf.set("mapred.mapper.class", ExecMapper.class.getName()); conf.set("mapred.input.format.class", inpFormat); return conf; }
/** Execute a query plan using Hadoop. */ @SuppressWarnings({"deprecation", "unchecked"}) @Override public int execute(DriverContext driverContext) { IOPrepareCache ioPrepareCache = IOPrepareCache.get(); ioPrepareCache.clear(); boolean success = true; Context ctx = driverContext.getCtx(); boolean ctxCreated = false; Path emptyScratchDir; MapWork mWork = work.getMapWork(); ReduceWork rWork = work.getReduceWork(); try { if (ctx == null) { ctx = new Context(job); ctxCreated = true; } emptyScratchDir = ctx.getMRTmpPath(); FileSystem fs = emptyScratchDir.getFileSystem(job); fs.mkdirs(emptyScratchDir); } catch (IOException e) { e.printStackTrace(); console.printError( "Error launching map-reduce job", "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e)); return 5; } HiveFileFormatUtils.prepareJobOutput(job); // See the javadoc on HiveOutputFormatImpl and HadoopShims.prepareJobOutput() job.setOutputFormat(HiveOutputFormatImpl.class); job.setMapperClass(ExecMapper.class); job.setMapOutputKeyClass(HiveKey.class); job.setMapOutputValueClass(BytesWritable.class); try { String partitioner = HiveConf.getVar(job, ConfVars.HIVEPARTITIONER); job.setPartitionerClass(JavaUtils.loadClass(partitioner)); } catch (ClassNotFoundException e) { throw new RuntimeException(e.getMessage(), e); } if (mWork.getNumMapTasks() != null) { job.setNumMapTasks(mWork.getNumMapTasks().intValue()); } if (mWork.getMaxSplitSize() != null) { HiveConf.setLongVar( job, HiveConf.ConfVars.MAPREDMAXSPLITSIZE, mWork.getMaxSplitSize().longValue()); } if (mWork.getMinSplitSize() != null) { HiveConf.setLongVar( job, HiveConf.ConfVars.MAPREDMINSPLITSIZE, mWork.getMinSplitSize().longValue()); } if (mWork.getMinSplitSizePerNode() != null) { HiveConf.setLongVar( job, HiveConf.ConfVars.MAPREDMINSPLITSIZEPERNODE, mWork.getMinSplitSizePerNode().longValue()); } if (mWork.getMinSplitSizePerRack() != null) { HiveConf.setLongVar( job, HiveConf.ConfVars.MAPREDMINSPLITSIZEPERRACK, mWork.getMinSplitSizePerRack().longValue()); } job.setNumReduceTasks(rWork != null ? rWork.getNumReduceTasks().intValue() : 0); job.setReducerClass(ExecReducer.class); // set input format information if necessary setInputAttributes(job); // Turn on speculative execution for reducers boolean useSpeculativeExecReducers = HiveConf.getBoolVar(job, HiveConf.ConfVars.HIVESPECULATIVEEXECREDUCERS); HiveConf.setBoolVar( job, HiveConf.ConfVars.HADOOPSPECULATIVEEXECREDUCERS, useSpeculativeExecReducers); String inpFormat = HiveConf.getVar(job, HiveConf.ConfVars.HIVEINPUTFORMAT); if (mWork.isUseBucketizedHiveInputFormat()) { inpFormat = BucketizedHiveInputFormat.class.getName(); } LOG.info("Using " + inpFormat); try { job.setInputFormat(JavaUtils.loadClass(inpFormat)); } catch (ClassNotFoundException e) { throw new RuntimeException(e.getMessage(), e); } // No-Op - we don't really write anything here .. job.setOutputKeyClass(Text.class); job.setOutputValueClass(Text.class); // Transfer HIVEAUXJARS and HIVEADDEDJARS to "tmpjars" so hadoop understands // it String auxJars = HiveConf.getVar(job, HiveConf.ConfVars.HIVEAUXJARS); String addedJars = HiveConf.getVar(job, HiveConf.ConfVars.HIVEADDEDJARS); if (StringUtils.isNotBlank(auxJars) || StringUtils.isNotBlank(addedJars)) { String allJars = StringUtils.isNotBlank(auxJars) ? (StringUtils.isNotBlank(addedJars) ? addedJars + "," + auxJars : auxJars) : addedJars; LOG.info("adding libjars: " + allJars); initializeFiles("tmpjars", allJars); } // Transfer HIVEADDEDFILES to "tmpfiles" so hadoop understands it String addedFiles = HiveConf.getVar(job, HiveConf.ConfVars.HIVEADDEDFILES); if (StringUtils.isNotBlank(addedFiles)) { initializeFiles("tmpfiles", addedFiles); } int returnVal = 0; boolean noName = StringUtils.isEmpty(HiveConf.getVar(job, HiveConf.ConfVars.HADOOPJOBNAME)); if (noName) { // This is for a special case to ensure unit tests pass HiveConf.setVar(job, HiveConf.ConfVars.HADOOPJOBNAME, "JOB" + Utilities.randGen.nextInt()); } String addedArchives = HiveConf.getVar(job, HiveConf.ConfVars.HIVEADDEDARCHIVES); // Transfer HIVEADDEDARCHIVES to "tmparchives" so hadoop understands it if (StringUtils.isNotBlank(addedArchives)) { initializeFiles("tmparchives", addedArchives); } try { MapredLocalWork localwork = mWork.getMapRedLocalWork(); if (localwork != null && localwork.hasStagedAlias()) { if (!ShimLoader.getHadoopShims().isLocalMode(job)) { Path localPath = localwork.getTmpPath(); Path hdfsPath = mWork.getTmpHDFSPath(); FileSystem hdfs = hdfsPath.getFileSystem(job); FileSystem localFS = localPath.getFileSystem(job); FileStatus[] hashtableFiles = localFS.listStatus(localPath); int fileNumber = hashtableFiles.length; String[] fileNames = new String[fileNumber]; for (int i = 0; i < fileNumber; i++) { fileNames[i] = hashtableFiles[i].getPath().getName(); } // package and compress all the hashtable files to an archive file String stageId = this.getId(); String archiveFileName = Utilities.generateTarFileName(stageId); localwork.setStageID(stageId); CompressionUtils.tar(localPath.toUri().getPath(), fileNames, archiveFileName); Path archivePath = Utilities.generateTarPath(localPath, stageId); LOG.info("Archive " + hashtableFiles.length + " hash table files to " + archivePath); // upload archive file to hdfs Path hdfsFilePath = Utilities.generateTarPath(hdfsPath, stageId); short replication = (short) job.getInt("mapred.submit.replication", 10); hdfs.copyFromLocalFile(archivePath, hdfsFilePath); hdfs.setReplication(hdfsFilePath, replication); LOG.info("Upload 1 archive file from" + archivePath + " to: " + hdfsFilePath); // add the archive file to distributed cache DistributedCache.createSymlink(job); DistributedCache.addCacheArchive(hdfsFilePath.toUri(), job); LOG.info( "Add 1 archive file to distributed cache. Archive file: " + hdfsFilePath.toUri()); } } work.configureJobConf(job); List<Path> inputPaths = Utilities.getInputPaths(job, mWork, emptyScratchDir, ctx, false); Utilities.setInputPaths(job, inputPaths); Utilities.setMapRedWork(job, work, ctx.getMRTmpPath()); if (mWork.getSamplingType() > 0 && rWork != null && job.getNumReduceTasks() > 1) { try { handleSampling(ctx, mWork, job); job.setPartitionerClass(HiveTotalOrderPartitioner.class); } catch (IllegalStateException e) { console.printInfo("Not enough sampling data.. Rolling back to single reducer task"); rWork.setNumReduceTasks(1); job.setNumReduceTasks(1); } catch (Exception e) { LOG.error("Sampling error", e); console.printError( e.toString(), "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e)); rWork.setNumReduceTasks(1); job.setNumReduceTasks(1); } } // remove the pwd from conf file so that job tracker doesn't show this // logs String pwd = HiveConf.getVar(job, HiveConf.ConfVars.METASTOREPWD); if (pwd != null) { HiveConf.setVar(job, HiveConf.ConfVars.METASTOREPWD, "HIVE"); } JobClient jc = new JobClient(job); // make this client wait if job tracker is not behaving well. Throttle.checkJobTracker(job, LOG); if (mWork.isGatheringStats() || (rWork != null && rWork.isGatheringStats())) { // initialize stats publishing table StatsPublisher statsPublisher; StatsFactory factory = StatsFactory.newFactory(job); if (factory != null) { statsPublisher = factory.getStatsPublisher(); List<String> statsTmpDir = Utilities.getStatsTmpDirs(mWork, job); if (rWork != null) { statsTmpDir.addAll(Utilities.getStatsTmpDirs(rWork, job)); } StatsCollectionContext sc = new StatsCollectionContext(job); sc.setStatsTmpDirs(statsTmpDir); if (!statsPublisher.init(sc)) { // creating stats table if not exists if (HiveConf.getBoolVar(job, HiveConf.ConfVars.HIVE_STATS_RELIABLE)) { throw new HiveException( ErrorMsg.STATSPUBLISHER_INITIALIZATION_ERROR.getErrorCodedMsg()); } } } } Utilities.createTmpDirs(job, mWork); Utilities.createTmpDirs(job, rWork); SessionState ss = SessionState.get(); if (HiveConf.getVar(job, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE).equals("tez") && ss != null) { TezSessionState session = ss.getTezSession(); TezSessionPoolManager.getInstance().close(session, true); } // Finally SUBMIT the JOB! rj = jc.submitJob(job); // replace it back if (pwd != null) { HiveConf.setVar(job, HiveConf.ConfVars.METASTOREPWD, pwd); } returnVal = jobExecHelper.progress(rj, jc, ctx.getHiveTxnManager()); success = (returnVal == 0); } catch (Exception e) { e.printStackTrace(); String mesg = " with exception '" + Utilities.getNameMessage(e) + "'"; if (rj != null) { mesg = "Ended Job = " + rj.getJobID() + mesg; } else { mesg = "Job Submission failed" + mesg; } // Has to use full name to make sure it does not conflict with // org.apache.commons.lang.StringUtils console.printError(mesg, "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e)); success = false; returnVal = 1; } finally { Utilities.clearWork(job); try { if (ctxCreated) { ctx.clear(); } if (rj != null) { if (returnVal != 0) { rj.killJob(); } jobID = rj.getID().toString(); } } catch (Exception e) { LOG.warn("Failed while cleaning up ", e); } finally { HadoopJobExecHelper.runningJobs.remove(rj); } } // get the list of Dynamic partition paths try { if (rj != null) { if (mWork.getAliasToWork() != null) { for (Operator<? extends OperatorDesc> op : mWork.getAliasToWork().values()) { op.jobClose(job, success); } } if (rWork != null) { rWork.getReducer().jobClose(job, success); } } } catch (Exception e) { // jobClose needs to execute successfully otherwise fail task if (success) { success = false; returnVal = 3; String mesg = "Job Commit failed with exception '" + Utilities.getNameMessage(e) + "'"; console.printError(mesg, "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e)); } } return (returnVal); }
@Override /** start a new map-reduce job to do the merge, almost the same as ExecDriver. */ public int execute(DriverContext driverContext) { HiveConf.setVar(job, HiveConf.ConfVars.HIVEINPUTFORMAT, CombineHiveInputFormat.class.getName()); success = true; ShimLoader.getHadoopShims().setNullOutputFormat(job); job.setMapperClass(work.getMapperClass()); Context ctx = driverContext.getCtx(); boolean ctxCreated = false; try { if (ctx == null) { ctx = new Context(job); ctxCreated = true; } } catch (IOException e) { e.printStackTrace(); console.printError( "Error launching map-reduce job", "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e)); return 5; } job.setMapOutputKeyClass(NullWritable.class); job.setMapOutputValueClass(NullWritable.class); if (work.getNumMapTasks() != null) { job.setNumMapTasks(work.getNumMapTasks()); } // zero reducers job.setNumReduceTasks(0); if (work.getMinSplitSize() != null) { HiveConf.setLongVar( job, HiveConf.ConfVars.MAPREDMINSPLITSIZE, work.getMinSplitSize().longValue()); } if (work.getInputformat() != null) { HiveConf.setVar(job, HiveConf.ConfVars.HIVEINPUTFORMAT, work.getInputformat()); } String inpFormat = HiveConf.getVar(job, HiveConf.ConfVars.HIVEINPUTFORMAT); if ((inpFormat == null) || (!StringUtils.isNotBlank(inpFormat))) { inpFormat = ShimLoader.getHadoopShims().getInputFormatClassName(); } LOG.info("Using " + inpFormat); try { job.setInputFormat((Class<? extends InputFormat>) (Class.forName(inpFormat))); } catch (ClassNotFoundException e) { throw new RuntimeException(e.getMessage()); } String outputPath = this.work.getOutputDir(); Path tempOutPath = Utilities.toTempPath(new Path(outputPath)); try { FileSystem fs = tempOutPath.getFileSystem(job); if (!fs.exists(tempOutPath)) { fs.mkdirs(tempOutPath); } } catch (IOException e) { console.printError("Can't make path " + outputPath + " : " + e.getMessage()); return 6; } RCFileBlockMergeOutputFormat.setMergeOutputPath(job, new Path(outputPath)); job.setOutputKeyClass(NullWritable.class); job.setOutputValueClass(NullWritable.class); HiveConf.setBoolVar( job, HiveConf.ConfVars.HIVEMERGECURRENTJOBHASDYNAMICPARTITIONS, work.hasDynamicPartitions()); int returnVal = 0; RunningJob rj = null; boolean noName = StringUtils.isEmpty(HiveConf.getVar(job, HiveConf.ConfVars.HADOOPJOBNAME)); String jobName = null; if (noName && this.getQueryPlan() != null) { int maxlen = conf.getIntVar(HiveConf.ConfVars.HIVEJOBNAMELENGTH); jobName = Utilities.abbreviate(this.getQueryPlan().getQueryStr(), maxlen - 6); } if (noName) { // This is for a special case to ensure unit tests pass HiveConf.setVar( job, HiveConf.ConfVars.HADOOPJOBNAME, jobName != null ? jobName : "JOB" + Utilities.randGen.nextInt()); } try { addInputPaths(job, work); Utilities.setMapRedWork(job, work, ctx.getMRTmpFileURI()); // remove the pwd from conf file so that job tracker doesn't show this // logs String pwd = HiveConf.getVar(job, HiveConf.ConfVars.METASTOREPWD); if (pwd != null) { HiveConf.setVar(job, HiveConf.ConfVars.METASTOREPWD, "HIVE"); } JobClient jc = new JobClient(job); String addedJars = Utilities.getResourceFiles(job, SessionState.ResourceType.JAR); if (!addedJars.isEmpty()) { job.set("tmpjars", addedJars); } // make this client wait if job trcker is not behaving well. Throttle.checkJobTracker(job, LOG); // Finally SUBMIT the JOB! rj = jc.submitJob(job); returnVal = jobExecHelper.progress(rj, jc); success = (returnVal == 0); } catch (Exception e) { e.printStackTrace(); String mesg = " with exception '" + Utilities.getNameMessage(e) + "'"; if (rj != null) { mesg = "Ended Job = " + rj.getJobID() + mesg; } else { mesg = "Job Submission failed" + mesg; } // Has to use full name to make sure it does not conflict with // org.apache.commons.lang.StringUtils console.printError(mesg, "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e)); success = false; returnVal = 1; } finally { try { if (ctxCreated) { ctx.clear(); } if (rj != null) { if (returnVal != 0) { rj.killJob(); } HadoopJobExecHelper.runningJobKillURIs.remove(rj.getJobID()); jobID = rj.getID().toString(); } RCFileMergeMapper.jobClose(outputPath, success, job, console); } catch (Exception e) { } } return (returnVal); }