private Job getJob()

in src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/JobControlCompiler.java [505:1044]


    private Job getJob(MROperPlan plan, MapReduceOper mro, Configuration config, PigContext pigContext) throws JobCreationException{
        org.apache.hadoop.mapreduce.Job nwJob = null;

        try{
            nwJob = new org.apache.hadoop.mapreduce.Job(config);
        }catch(Exception e) {
            throw new JobCreationException(e);
        }

        Configuration conf = nwJob.getConfiguration();

        ArrayList<POLoad> inp = new ArrayList<POLoad>();
        ArrayList<List<OperatorKey>> inpTargets = new ArrayList<List<OperatorKey>>();
        ArrayList<String> inpSignatureLists = new ArrayList<String>();
        ArrayList<Long> inpLimits = new ArrayList<Long>();
        ArrayList<POStore> storeLocations = new ArrayList<POStore>();
        Path tmpLocation = null;

        // add settings for pig statistics
        String setScriptProp = conf.get(PigConfiguration.PIG_SCRIPT_INFO_ENABLED, "true");
        if (setScriptProp.equalsIgnoreCase("true")) {
            MRScriptState ss = MRScriptState.get();
            ss.addSettingsToConf(mro, conf);
        }

        conf.set(MRConfiguration.MAPPER_NEW_API, "true");
        conf.set(MRConfiguration.REDUCER_NEW_API, "true");

        String buffPercent = conf.get(MRConfiguration.JOB_REDUCE_MARKRESET_BUFFER_PERCENT);
        if (buffPercent == null || Double.parseDouble(buffPercent) <= 0) {
            log.info(MRConfiguration.JOB_REDUCE_MARKRESET_BUFFER_PERCENT + " is not set, set to default 0.3");
            conf.set(MRConfiguration.JOB_REDUCE_MARKRESET_BUFFER_PERCENT, "0.3");
        }else{
            log.info(MRConfiguration.JOB_REDUCE_MARKRESET_BUFFER_PERCENT + " is set to " +
                     conf.get(MRConfiguration.JOB_REDUCE_MARKRESET_BUFFER_PERCENT));
        }


        DateTimeWritable.setupAvailableZoneIds();

        configureCompression(conf);

        try{
            //Process the POLoads
            List<POLoad> lds = PlanHelper.getPhysicalOperators(mro.mapPlan, POLoad.class);

            if(lds!=null && lds.size()>0){
                for (POLoad ld : lds) {
                    LoadFunc lf = ld.getLoadFunc();
                    lf.setLocation(ld.getLFile().getFileName(), nwJob);
                    lf.addCredentials(nwJob.getCredentials(), conf);
                    ld.setParentPlan(null);
                    //Store the inp filespecs
                    inp.add(ld);
                }
            }

            //Process the POUserFunc
            List<POUserFunc> userFuncs = PlanHelper.getPhysicalOperators(mro.mapPlan, POUserFunc.class);
            userFuncs.addAll(PlanHelper.getPhysicalOperators(mro.reducePlan, POUserFunc.class));
            for (POUserFunc userFunc : userFuncs) {
                userFunc.getFunc().addCredentials(nwJob.getCredentials(), conf);
            }

            if(!mro.reducePlan.isEmpty()){
                log.info("Reduce phase detected, estimating # of required reducers.");
                adjustNumReducers(plan, mro, nwJob);
            } else {
                nwJob.setNumReduceTasks(0);
            }

            if (!pigContext.inIllustrator && ! pigContext.getExecType().isLocal())
            {
                if (okToRunLocal(nwJob, mro, lds)) {
                    log.info(SMALL_JOB_LOG_MSG);
                    // override with the default conf to run in local mode
                    for (Entry<String, String> entry : defaultConf) {
                        String key = entry.getKey();
                        if (key.equals(MRConfiguration.REDUCE_TASKS) || key.equals(MRConfiguration.JOB_REDUCES)) {
                            // this must not be set back to the default in case it has been set to 0 for example.
                            continue;
                        }
                        if (key.startsWith("fs.")) {
                            // we don't want to change fs settings back
                            continue;
                        }
                        if (key.startsWith("io.")) {
                            // we don't want to change io settings back
                            continue;
                        }
                        String value = entry.getValue();
                        if (conf.get(key) == null || !conf.get(key).equals(value)) {
                            conf.set(key, value);
                        }
                    }

                    conf.setBoolean(PigImplConstants.CONVERTED_TO_LOCAL, true);
                } else {
                    log.info(BIG_JOB_LOG_MSG);
                    // Search to see if we have any UDF/LoadFunc/StoreFunc that need to pack things into the
                    // distributed cache.
                    List<String> cacheFiles = new ArrayList<String>();
                    List<String> shipFiles = new ArrayList<String>();
                    UdfCacheShipFilesVisitor mapUdfCacheFileVisitor = new UdfCacheShipFilesVisitor(mro.mapPlan);
                    mapUdfCacheFileVisitor.visit();
                    cacheFiles.addAll(mapUdfCacheFileVisitor.getCacheFiles());
                    shipFiles.addAll(mapUdfCacheFileVisitor.getShipFiles());

                    UdfCacheShipFilesVisitor reduceUdfCacheFileVisitor = new UdfCacheShipFilesVisitor(mro.reducePlan);
                    reduceUdfCacheFileVisitor.visit();
                    cacheFiles.addAll(reduceUdfCacheFileVisitor.getCacheFiles());
                    shipFiles.addAll(reduceUdfCacheFileVisitor.getShipFiles());

                    setupDistributedCache(pigContext, conf, cacheFiles.toArray(new String[]{}), false);

                    // Setup the DistributedCache for this job
                    List<URL> allJars = new ArrayList<URL>();

                    for (URL extraJar : pigContext.extraJars) {
                        if (!allJars.contains(extraJar)) {
                            allJars.add(extraJar);
                        }
                    }

                    for (String udf : mro.UDFs) {
                        Class clazz = pigContext.getClassForAlias(udf);
                        if (clazz != null) {
                            String jar = JarManager.findContainingJar(clazz);
                            if (jar!=null) {
                                URL jarURL = new File(jar).toURI().toURL();
                                if (!allJars.contains(jarURL)) {
                                    allJars.add(jarURL);
                                }
                            }
                        }
                    }

                    for (String scriptJar : pigContext.scriptJars) {
                        URL jar = new File(scriptJar).toURI().toURL();
                        if (!allJars.contains(jar)) {
                            allJars.add(jar);
                        }
                    }

                    for (String shipFile : shipFiles) {
                        URL jar = new File(shipFile).toURI().toURL();
                        if (!allJars.contains(jar)) {
                            allJars.add(jar);
                        }
                    }

                    for (String defaultJar : JarManager.getDefaultJars()) {
                        URL jar = new File(defaultJar).toURI().toURL();
                        if (!allJars.contains(jar)) {
                            allJars.add(jar);
                        }
                    }

                    for (URL jar : allJars) {
                        boolean predeployed = false;
                        for (String predeployedJar : pigContext.predeployedJars) {
                            if (predeployedJar.contains(new File(jar.toURI()).getName())) {
                                predeployed = true;
                            }
                        }
                        if (!predeployed) {
                            if (jar.getFile().toLowerCase().endsWith(".jar")) {
                                putJarOnClassPathThroughDistributedCache(pigContext, conf, jar);
                            } else {
                                setupDistributedCache(pigContext, conf, new String[] {jar.getPath()}, true);
                            }
                        }
                    }

                    File scriptUDFJarFile = JarManager.createPigScriptUDFJar(pigContext);
                    if (scriptUDFJarFile != null) {
                        putJarOnClassPathThroughDistributedCache(pigContext, conf, scriptUDFJarFile.toURI().toURL());
                    }
                }
            }

            for (String udf : mro.UDFs) {
                if (udf.contains("GFCross")) {
                    Object func = PigContext.instantiateFuncFromSpec(new FuncSpec(udf));
                    if (func instanceof GFCross) {
                        String crossKey = ((GFCross)func).getCrossKey();
                        conf.set(PigImplConstants.PIG_CROSS_PARALLELISM + "." + crossKey,
                                Integer.toString(mro.getRequestedParallelism()));
                    }
                }
            }

            if(lds!=null && lds.size()>0){
                for (POLoad ld : lds) {
                    //Store the target operators for tuples read
                    //from this input
                    List<PhysicalOperator> ldSucs = mro.mapPlan.getSuccessors(ld);
                    List<OperatorKey> ldSucKeys = new ArrayList<OperatorKey>();
                    if(ldSucs!=null){
                        for (PhysicalOperator operator2 : ldSucs) {
                            ldSucKeys.add(operator2.getOperatorKey());
                        }
                    }
                    inpTargets.add(ldSucKeys);
                    inpSignatureLists.add(ld.getSignature());
                    inpLimits.add(ld.getLimit());
                    //Remove the POLoad from the plan
                    if (!pigContext.inIllustrator)
                        mro.mapPlan.remove(ld);
                }
            }

            if(Utils.isLocal(pigContext, conf)) {
                ConfigurationUtil.replaceConfigForLocalMode(conf);
            }
            conf.set(PigInputFormat.PIG_LOADS, ObjectSerializer.serialize(inp));
            conf.set(PigInputFormat.PIG_INPUT_TARGETS, ObjectSerializer.serialize(inpTargets));
            conf.set(PigInputFormat.PIG_INPUT_SIGNATURES, ObjectSerializer.serialize(inpSignatureLists));
            conf.set(PigInputFormat.PIG_INPUT_LIMITS, ObjectSerializer.serialize(inpLimits));

            // Removing job credential entry before serializing pigcontext into jobconf
            // since this path would be invalid for the new job being created
            pigContext.getProperties().remove("mapreduce.job.credentials.binary");

            conf.setBoolean(PigImplConstants.PIG_EXECTYPE_MODE_LOCAL, pigContext.getExecType().isLocal());
            conf.set(PigImplConstants.PIG_LOG4J_PROPERTIES, ObjectSerializer.serialize(pigContext.getLog4jProperties()));
            conf.set("udf.import.list", ObjectSerializer.serialize(PigContext.getPackageImportList()));
            // this is for unit tests since some don't create PigServer

            // if user specified the job name using -D switch, Pig won't reset the name then.
            if (System.getProperty(MRConfiguration.JOB_NAME) == null &&
                    pigContext.getProperties().getProperty(PigContext.JOB_NAME) != null){
                nwJob.setJobName(pigContext.getProperties().getProperty(PigContext.JOB_NAME));
            }

            if (pigContext.getProperties().getProperty(PigContext.JOB_PRIORITY) != null) {
                // If the job priority was set, attempt to get the corresponding enum value
                // and set the hadoop job priority.
                String jobPriority = pigContext.getProperties().getProperty(PigContext.JOB_PRIORITY).toUpperCase();
                try {
                    // Allow arbitrary case; the Hadoop job priorities are all upper case.
                    conf.set(MRConfiguration.JOB_PRIORITY, JobPriority.valueOf(jobPriority).toString());

                } catch (IllegalArgumentException e) {
                    StringBuffer sb = new StringBuffer("The job priority must be one of [");
                    JobPriority[] priorities = JobPriority.values();
                    for (int i = 0; i < priorities.length; ++i) {
                        if (i > 0)  sb.append(", ");
                        sb.append(priorities[i]);
                    }
                    sb.append("].  You specified [" + jobPriority + "]");
                    throw new JobCreationException(sb.toString());
                }
            }

            setupDistributedCache(pigContext, conf, pigContext.getProperties(),
                    "pig.streaming.ship.files", true);
            setupDistributedCache(pigContext, conf, pigContext.getProperties(),
                    "pig.streaming.cache.files", false);

            nwJob.setInputFormatClass(PigInputFormat.class);

            // tmp file compression setups
            // PIG-3741 This must be done before setStoreLocation on POStores
            Utils.setTmpFileCompressionOnConf(pigContext, conf);

            //Process POStore and remove it from the plan
            LinkedList<POStore> mapStores = PlanHelper.getPhysicalOperators(mro.mapPlan, POStore.class);
            LinkedList<POStore> reduceStores = PlanHelper.getPhysicalOperators(mro.reducePlan, POStore.class);

            for (POStore st : mapStores) {
                storeLocations.add(st);
                StoreFuncInterface sFunc = st.getStoreFunc();
                sFunc.setStoreLocation(st.getSFile().getFileName(), nwJob);
                if (sFunc instanceof OverwritableStoreFunc) {
                    OverwritableStoreFunc osf = (OverwritableStoreFunc) sFunc;
                    if (osf.shouldOverwrite()) {
                        osf.cleanupOutput(st, nwJob);
                    }
                }
                sFunc.addCredentials(nwJob.getCredentials(), conf);
            }

            for (POStore st : reduceStores) {
                storeLocations.add(st);
                StoreFuncInterface sFunc = st.getStoreFunc();
                sFunc.setStoreLocation(st.getSFile().getFileName(), nwJob);
                if (sFunc instanceof OverwritableStoreFunc) {
                    OverwritableStoreFunc osf = (OverwritableStoreFunc) sFunc;
                    if (osf.shouldOverwrite()) {
                        osf.cleanupOutput(st, nwJob);
                    }
                }
                sFunc.addCredentials(nwJob.getCredentials(), conf);
            }

            setOutputFormat(nwJob);

            if (mapStores.size() + reduceStores.size() == 1) { // single store case
                log.info("Setting up single store job");

                POStore st;
                if (reduceStores.isEmpty()) {
                    st = mapStores.get(0);
                    if(!pigContext.inIllustrator)
                        mro.mapPlan.remove(st);
                }
                else {
                    st = reduceStores.get(0);
                    if(!pigContext.inIllustrator)
                        mro.reducePlan.remove(st);
                }

                MapRedUtil.setupStreamingDirsConfSingle(st, pigContext, conf);
            }
            else if (mapStores.size() + reduceStores.size() > 0) { // multi store case
                log.info("Setting up multi store job");
                MapRedUtil.setupStreamingDirsConfMulti(pigContext, conf);

                boolean disableCounter = conf.getBoolean("pig.disable.counter", false);
                if (disableCounter) {
                    log.info("Disable Pig custom output counters");
                }
                int idx = 0;
                for (POStore sto: storeLocations) {
                    sto.setDisableCounter(disableCounter);
                    sto.setMultiStore(true);
                    sto.setIndex(idx++);
                }
            }

            // store map key type
            // this is needed when the key is null to create
            // an appropriate NullableXXXWritable object
            conf.set("pig.map.keytype", ObjectSerializer.serialize(new byte[] { mro.mapKeyType }));

            // set parent plan in all operators in map and reduce plans
            // currently the parent plan is really used only when POStream is present in the plan
            new PhyPlanSetter(mro.mapPlan).visit();
            new PhyPlanSetter(mro.combinePlan).visit();
            new PhyPlanSetter(mro.reducePlan).visit();

            // this call modifies the ReplFiles names of POFRJoin operators
            // within the MR plans, must be called before the plans are
            // serialized
            setupDistributedCacheForJoin(mro, pigContext, conf);

            SchemaTupleFrontend.copyAllGeneratedToDistributedCache(pigContext, conf);

            POPackage pack = null;
            if(mro.reducePlan.isEmpty()){
                //MapOnly Job
                nwJob.setMapperClass(PigMapOnly.Map.class);
                if(!pigContext.inIllustrator)
                    conf.set("pig.mapPlan", ObjectSerializer.serialize(mro.mapPlan));
                if(mro.isEndOfAllInputSetInMap()) {
                    // this is used in Map.close() to decide whether the
                    // pipeline needs to be rerun one more time in the close()
                    // The pipeline is rerun if there either was a stream or POMergeJoin
                    conf.set(END_OF_INP_IN_MAP, "true");
                }
            }
            else{
                //Map Reduce Job
                //Process the POPackage operator and remove it from the reduce plan
                if(!mro.combinePlan.isEmpty()){
                    POPackage combPack = (POPackage)mro.combinePlan.getRoots().get(0);
                    mro.combinePlan.remove(combPack);
                    nwJob.setCombinerClass(PigCombiner.Combine.class);
                    conf.set("pig.combinePlan", ObjectSerializer.serialize(mro.combinePlan));
                    conf.set("pig.combine.package", ObjectSerializer.serialize(combPack));
                } else if (mro.needsDistinctCombiner()) {
                    nwJob.setCombinerClass(DistinctCombiner.Combine.class);
                    log.info("Setting identity combiner class.");
                }
                pack = (POPackage)mro.reducePlan.getRoots().get(0);

                if (!pigContext.inIllustrator) {
                    mro.reducePlan.remove(pack);
                }
                nwJob.setMapperClass(PigMapReduce.Map.class);
                nwJob.setReducerClass(PigMapReduce.Reduce.class);

                if (mro.customPartitioner != null)
                    nwJob.setPartitionerClass(PigContext.resolveClassName(mro.customPartitioner));

                if(!pigContext.inIllustrator)
                    conf.set("pig.mapPlan", ObjectSerializer.serialize(mro.mapPlan));
                if(mro.isEndOfAllInputSetInMap()) {
                    // this is used in Map.close() to decide whether the
                    // pipeline needs to be rerun one more time in the close()
                    // The pipeline is rerun only if there was a stream or merge-join.
                    conf.set(END_OF_INP_IN_MAP, "true");
                }
                if(!pigContext.inIllustrator)
                    conf.set("pig.reducePlan", ObjectSerializer.serialize(mro.reducePlan));
                if(mro.isEndOfAllInputSetInReduce()) {
                    // this is used in Map.close() to decide whether the
                    // pipeline needs to be rerun one more time in the close()
                    // The pipeline is rerun only if there was a stream
                    conf.set("pig.stream.in.reduce", "true");
                }
                if (!pigContext.inIllustrator)
                    conf.set("pig.reduce.package", ObjectSerializer.serialize(pack));
                conf.set("pig.reduce.key.type",
                        Byte.toString(pack.getPkgr().getKeyType()));

                if (mro.getUseSecondaryKey()) {
                    nwJob.setGroupingComparatorClass(PigSecondaryKeyGroupComparator.class);
                    nwJob.setPartitionerClass(SecondaryKeyPartitioner.class);
                    nwJob.setSortComparatorClass(PigSecondaryKeyComparator.class);
                    nwJob.setOutputKeyClass(NullableTuple.class);
                    conf.set("pig.secondarySortOrder",
                            ObjectSerializer.serialize(mro.getSecondarySortOrder()));

                }
                else
                {
                    Class<? extends WritableComparable> keyClass = HDataType
                            .getWritableComparableTypes(
                                    pack.getPkgr().getKeyType()).getClass();
                    nwJob.setOutputKeyClass(keyClass);
                    selectComparator(mro, pack.getPkgr().getKeyType(), nwJob);
                }
                nwJob.setOutputValueClass(NullableTuple.class);
            }

            if(mro.isGlobalSort() || mro.isLimitAfterSort()){
                if (mro.isGlobalSort()) {
                    String symlink = addSingleFileToDistributedCache(
                            pigContext, conf, mro.getQuantFile(), "pigsample");
                    conf.set("pig.quantilesFile", symlink);
                    nwJob.setPartitionerClass(WeightedRangePartitioner.class);
                }

                if (mro.isUDFComparatorUsed) {
                    boolean usercomparator = false;
                    for (String compFuncSpec : mro.UDFs) {
                        Class comparator = PigContext.resolveClassName(compFuncSpec);
                        if(ComparisonFunc.class.isAssignableFrom(comparator)) {
                            nwJob.setMapperClass(PigMapReduce.MapWithComparator.class);
                            nwJob.setReducerClass(PigMapReduce.ReduceWithComparator.class);
                            conf.set("pig.reduce.package", ObjectSerializer.serialize(pack));
                            conf.set("pig.usercomparator", "true");
                            nwJob.setOutputKeyClass(NullableTuple.class);
                            nwJob.setSortComparatorClass(comparator);
                            usercomparator = true;
                            break;
                        }
                    }
                    if (!usercomparator) {
                        String msg = "Internal error. Can't find the UDF comparator";
                        throw new IOException (msg);
                    }

                } else {
                    conf.set("pig.sortOrder",
                            ObjectSerializer.serialize(mro.getSortOrder()));
                }
            }

            if (mro.isSkewedJoin()) {
                String symlink = addSingleFileToDistributedCache(pigContext,
                        conf, mro.getSkewedJoinPartitionFile(), "pigdistkey");
                conf.set("pig.keyDistFile", symlink);
                nwJob.setPartitionerClass(SkewedPartitioner.class);
                nwJob.setMapperClass(PigMapReduce.MapWithPartitionIndex.class);
                nwJob.setMapOutputKeyClass(NullablePartitionWritable.class);
                nwJob.setGroupingComparatorClass(PigGroupingPartitionWritableComparator.class);
            }

            if (mro.isCounterOperation()) {
                if (mro.isRowNumber()) {
                    nwJob.setMapperClass(PigMapReduceCounter.PigMapCounter.class);
                } else {
                    nwJob.setReducerClass(PigMapReduceCounter.PigReduceCounter.class);
                }
            }

            if(mro.isRankOperation()) {
                Iterator<String> operationIDs = mro.getRankOperationId().iterator();

                while(operationIDs.hasNext()) {
                    String operationID = operationIDs.next();
                    Iterator<Pair<String, Long>> itPairs = globalCounters.get(operationID).iterator();
                    Pair<String,Long> pair = null;
                    while(itPairs.hasNext()) {
                        pair = itPairs.next();
                        conf.setLong(pair.first, pair.second);
                    }
                }
            }

            if (!pigContext.inIllustrator)
            {
                // unset inputs for POStore, otherwise, map/reduce plan will be unnecessarily deserialized
                for (POStore st: mapStores) { st.setInputs(null); st.setParentPlan(null);}
                for (POStore st: reduceStores) { st.setInputs(null); st.setParentPlan(null);}
                conf.set(PIG_MAP_STORES, ObjectSerializer.serialize(mapStores));
                conf.set(PIG_REDUCE_STORES, ObjectSerializer.serialize(reduceStores));
            }

            String tmp;
            long maxCombinedSplitSize = 0;
            if (!mro.combineSmallSplits() || pigContext.getProperties().getProperty("pig.splitCombination", "true").equals("false"))
                conf.setBoolean("pig.noSplitCombination", true);
            else if ((tmp = pigContext.getProperties().getProperty("pig.maxCombinedSplitSize", null)) != null) {
                try {
                    maxCombinedSplitSize = Long.parseLong(tmp);
                } catch (NumberFormatException e) {
                    log.warn("Invalid numeric format for pig.maxCombinedSplitSize; use the default maximum combined split size");
                }
            }
            if (maxCombinedSplitSize > 0)
                conf.setLong("pig.maxCombinedSplitSize", maxCombinedSplitSize);

            // It's a hack to set distributed cache file for hadoop 23. Once MiniMRCluster do not require local
            // jar on fixed location, this can be removed
            if (pigContext.getExecType() == ExecType.MAPREDUCE) {
                String newfiles = conf.get("alternative.mapreduce.job.cache.files");
                if (newfiles!=null) {
                    String files = conf.get(MRConfiguration.JOB_CACHE_FILES);
                    conf.set(MRConfiguration.JOB_CACHE_FILES,
                            files == null ? newfiles.toString() : files + "," + newfiles);
                }
            }
            // Serialize the UDF specific context info.
            UDFContext.getUDFContext().serialize(conf);
            Job cjob = new Job(new JobConf(conf), new ArrayList<Job>());
            jobStoreMap.put(cjob,new Pair<List<POStore>, Path>(storeLocations, tmpLocation));
            return cjob;

        } catch (JobCreationException jce) {
            throw jce;
        } catch(Exception e) {
            int errCode = 2017;
            String msg = "Internal error creating job configuration.";
            throw new JobCreationException(msg, errCode, PigException.BUG, e);
        }
    }