private static void setupBucketMapJoinInfo(
      MapredWork plan,
      AbstractMapJoinOperator<? extends MapJoinDesc> currMapJoinOp,
      boolean createLocalPlan) {
    if (currMapJoinOp != null) {
      LinkedHashMap<String, LinkedHashMap<String, ArrayList<String>>> aliasBucketFileNameMapping =
          currMapJoinOp.getConf().getAliasBucketFileNameMapping();
      if (aliasBucketFileNameMapping != null) {
        MapredLocalWork localPlan = plan.getMapLocalWork();
        if (localPlan == null) {
          if (currMapJoinOp instanceof SMBMapJoinOperator) {
            localPlan = ((SMBMapJoinOperator) currMapJoinOp).getConf().getLocalWork();
          }
          if (localPlan == null && createLocalPlan) {
            localPlan =
                new MapredLocalWork(
                    new LinkedHashMap<String, Operator<? extends Serializable>>(),
                    new LinkedHashMap<String, FetchWork>());
          }
        } else {
          // local plan is not null, we want to merge it into SMBMapJoinOperator's local work
          if (currMapJoinOp instanceof SMBMapJoinOperator) {
            MapredLocalWork smbLocalWork =
                ((SMBMapJoinOperator) currMapJoinOp).getConf().getLocalWork();
            if (smbLocalWork != null) {
              localPlan.getAliasToFetchWork().putAll(smbLocalWork.getAliasToFetchWork());
              localPlan.getAliasToWork().putAll(smbLocalWork.getAliasToWork());
            }
          }
        }

        if (localPlan == null) {
          return;
        }

        if (currMapJoinOp instanceof SMBMapJoinOperator) {
          plan.setMapLocalWork(null);
          ((SMBMapJoinOperator) currMapJoinOp).getConf().setLocalWork(localPlan);
        } else {
          plan.setMapLocalWork(localPlan);
        }
        BucketMapJoinContext bucketMJCxt = new BucketMapJoinContext();
        localPlan.setBucketMapjoinContext(bucketMJCxt);
        bucketMJCxt.setAliasBucketFileNameMapping(aliasBucketFileNameMapping);
        bucketMJCxt.setBucketFileNameMapping(currMapJoinOp.getConf().getBucketFileNameMapping());
        localPlan.setInputFileChangeSensitive(true);
        bucketMJCxt.setMapJoinBigTableAlias(currMapJoinOp.getConf().getBigTableAlias());
        bucketMJCxt.setBucketMatcherClass(
            org.apache.hadoop.hive.ql.exec.DefaultBucketMatcher.class);
      }
    }
  }
  /**
   * set the current task in the mapredWork.
   *
   * @param alias current alias
   * @param topOp the top operator of the stack
   * @param plan current plan
   * @param local whether you need to add to map-reduce or local work
   * @param tt_desc table descriptor
   */
  public static void setTaskPlan(
      String path,
      String alias,
      Operator<? extends Serializable> topOp,
      MapredWork plan,
      boolean local,
      TableDesc tt_desc)
      throws SemanticException {

    if (path == null || alias == null) {
      return;
    }

    if (!local) {
      if (plan.getPathToAliases().get(path) == null) {
        plan.getPathToAliases().put(path, new ArrayList<String>());
      }
      plan.getPathToAliases().get(path).add(alias);
      plan.getPathToPartitionInfo().put(path, new PartitionDesc(tt_desc, null));
      plan.getAliasToWork().put(alias, topOp);
    } else {
      // populate local work if needed
      MapredLocalWork localPlan = plan.getMapLocalWork();
      if (localPlan == null) {
        localPlan =
            new MapredLocalWork(
                new LinkedHashMap<String, Operator<? extends Serializable>>(),
                new LinkedHashMap<String, FetchWork>());
      }

      assert localPlan.getAliasToWork().get(alias) == null;
      assert localPlan.getAliasToFetchWork().get(alias) == null;
      localPlan.getAliasToWork().put(alias, topOp);
      localPlan.getAliasToFetchWork().put(alias, new FetchWork(alias, tt_desc));
      plan.setMapLocalWork(localPlan);
    }
  }
  @Override
  public void close() {

    // No row was processed
    if (oc == null) {
      LOG.trace("Close called without any rows processed");
    }

    try {
      if (groupKey != null) {
        // If a operator wants to do some work at the end of a group
        LOG.trace("End Group");
        reducer.endGroup();
      }
      if (isLogInfoEnabled) {
        logCloseInfo();
      }

      reducer.close(abort);

      if (localWork != null) {
        for (Operator<? extends OperatorDesc> dummyOp : localWork.getDummyParentOp()) {
          dummyOp.close(abort);
        }
      }

      ReportStats rps = new ReportStats(rp, jc);
      reducer.preorderMap(rps);

    } catch (Exception e) {
      if (!abort) {
        // signal new failure to map-reduce
        LOG.error("Hit error while closing operators - failing tree");
        throw new RuntimeException(
            "Hive Runtime Error while closing operators: " + e.getMessage(), e);
      }
    } finally {
      MapredContext.close();
      Utilities.clearWorkMap();
    }
  }
  /**
   * set the current task in the mapredWork.
   *
   * @param alias_id current alias
   * @param topOp the top operator of the stack
   * @param plan current plan
   * @param local whether you need to add to map-reduce or local work
   * @param opProcCtx processing context
   * @param pList pruned partition list. If it is null it will be computed on-the-fly.
   */
  public static void setTaskPlan(
      String alias_id,
      Operator<? extends Serializable> topOp,
      MapredWork plan,
      boolean local,
      GenMRProcContext opProcCtx,
      PrunedPartitionList pList)
      throws SemanticException {
    ParseContext parseCtx = opProcCtx.getParseCtx();
    Set<ReadEntity> inputs = opProcCtx.getInputs();

    ArrayList<Path> partDir = new ArrayList<Path>();
    ArrayList<PartitionDesc> partDesc = new ArrayList<PartitionDesc>();

    Path tblDir = null;
    TableDesc tblDesc = null;

    PrunedPartitionList partsList = pList;

    if (partsList == null) {
      try {
        partsList = parseCtx.getOpToPartList().get((TableScanOperator) topOp);
        if (partsList == null) {
          partsList =
              PartitionPruner.prune(
                  parseCtx.getTopToTable().get(topOp),
                  parseCtx.getOpToPartPruner().get(topOp),
                  opProcCtx.getConf(),
                  alias_id,
                  parseCtx.getPrunedPartitions());
          parseCtx.getOpToPartList().put((TableScanOperator) topOp, partsList);
        }
      } catch (SemanticException e) {
        throw e;
      } catch (HiveException e) {
        LOG.error(org.apache.hadoop.util.StringUtils.stringifyException(e));
        throw new SemanticException(e.getMessage(), e);
      }
    }

    // Generate the map work for this alias_id
    Set<Partition> parts = null;
    // pass both confirmed and unknown partitions through the map-reduce
    // framework

    parts = partsList.getConfirmedPartns();
    parts.addAll(partsList.getUnknownPartns());
    PartitionDesc aliasPartnDesc = null;
    try {
      if (!parts.isEmpty()) {
        aliasPartnDesc = Utilities.getPartitionDesc(parts.iterator().next());
      }
    } catch (HiveException e) {
      LOG.error(org.apache.hadoop.util.StringUtils.stringifyException(e));
      throw new SemanticException(e.getMessage(), e);
    }

    // The table does not have any partitions
    if (aliasPartnDesc == null) {
      aliasPartnDesc =
          new PartitionDesc(Utilities.getTableDesc(parseCtx.getTopToTable().get(topOp)), null);
    }

    plan.getAliasToPartnInfo().put(alias_id, aliasPartnDesc);

    for (Partition part : parts) {
      if (part.getTable().isPartitioned()) {
        inputs.add(new ReadEntity(part));
      } else {
        inputs.add(new ReadEntity(part.getTable()));
      }

      // Later the properties have to come from the partition as opposed
      // to from the table in order to support versioning.
      Path[] paths;
      sampleDesc sampleDescr = parseCtx.getOpToSamplePruner().get(topOp);

      if (sampleDescr != null) {
        paths = SamplePruner.prune(part, sampleDescr);
      } else {
        paths = part.getPath();
      }

      // is it a partitioned table ?
      if (!part.getTable().isPartitioned()) {
        assert ((tblDir == null) && (tblDesc == null));

        tblDir = paths[0];
        tblDesc = Utilities.getTableDesc(part.getTable());
      }

      for (Path p : paths) {
        if (p == null) {
          continue;
        }
        String path = p.toString();
        if (LOG.isDebugEnabled()) {
          LOG.debug("Adding " + path + " of table" + alias_id);
        }

        partDir.add(p);
        try {
          partDesc.add(Utilities.getPartitionDesc(part));
        } catch (HiveException e) {
          LOG.error(org.apache.hadoop.util.StringUtils.stringifyException(e));
          throw new SemanticException(e.getMessage(), e);
        }
      }
    }

    Iterator<Path> iterPath = partDir.iterator();
    Iterator<PartitionDesc> iterPartnDesc = partDesc.iterator();

    if (!local) {
      while (iterPath.hasNext()) {
        assert iterPartnDesc.hasNext();
        String path = iterPath.next().toString();

        PartitionDesc prtDesc = iterPartnDesc.next();

        // Add the path to alias mapping
        if (plan.getPathToAliases().get(path) == null) {
          plan.getPathToAliases().put(path, new ArrayList<String>());
        }
        plan.getPathToAliases().get(path).add(alias_id);
        plan.getPathToPartitionInfo().put(path, prtDesc);
        if (LOG.isDebugEnabled()) {
          LOG.debug("Information added for path " + path);
        }
      }

      assert plan.getAliasToWork().get(alias_id) == null;
      plan.getAliasToWork().put(alias_id, topOp);
    } else {
      // populate local work if needed
      MapredLocalWork localPlan = plan.getMapLocalWork();
      if (localPlan == null) {
        localPlan =
            new MapredLocalWork(
                new LinkedHashMap<String, Operator<? extends Serializable>>(),
                new LinkedHashMap<String, FetchWork>());
      }

      assert localPlan.getAliasToWork().get(alias_id) == null;
      assert localPlan.getAliasToFetchWork().get(alias_id) == null;
      localPlan.getAliasToWork().put(alias_id, topOp);
      if (tblDir == null) {
        localPlan
            .getAliasToFetchWork()
            .put(alias_id, new FetchWork(FetchWork.convertPathToStringArray(partDir), partDesc));
      } else {
        localPlan.getAliasToFetchWork().put(alias_id, new FetchWork(tblDir.toString(), tblDesc));
      }
      plan.setMapLocalWork(localPlan);
    }
  }
Example #5
0
  /** 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
  @SuppressWarnings("unchecked")
  public void init(JobConf job, OutputCollector output, Reporter reporter) throws Exception {
    perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.SPARK_INIT_OPERATORS);
    super.init(job, output, reporter);

    rowObjectInspector = new ObjectInspector[Byte.MAX_VALUE];
    ObjectInspector[] valueObjectInspector = new ObjectInspector[Byte.MAX_VALUE];
    ObjectInspector keyObjectInspector;

    ReduceWork gWork = Utilities.getReduceWork(job);

    reducer = gWork.getReducer();
    vectorized = gWork.getVectorMode();
    reducer.setParentOperators(null); // clear out any parents as reducer is the
    // root
    isTagged = gWork.getNeedsTagging();
    try {
      keyTableDesc = gWork.getKeyDesc();
      inputKeyDeserializer = ReflectionUtils.newInstance(keyTableDesc.getDeserializerClass(), null);
      SerDeUtils.initializeSerDe(inputKeyDeserializer, null, keyTableDesc.getProperties(), null);
      keyObjectInspector = inputKeyDeserializer.getObjectInspector();
      valueTableDesc = new TableDesc[gWork.getTagToValueDesc().size()];

      if (vectorized) {
        final int maxTags = gWork.getTagToValueDesc().size();
        keyStructInspector = (StructObjectInspector) keyObjectInspector;
        batches = new VectorizedRowBatch[maxTags];
        valueStructInspectors = new StructObjectInspector[maxTags];
        valueStringWriters = new List[maxTags];
        keysColumnOffset = keyStructInspector.getAllStructFieldRefs().size();
        buffer = new DataOutputBuffer();
      }

      for (int tag = 0; tag < gWork.getTagToValueDesc().size(); tag++) {
        // We should initialize the SerDe with the TypeInfo when available.
        valueTableDesc[tag] = gWork.getTagToValueDesc().get(tag);
        inputValueDeserializer[tag] =
            ReflectionUtils.newInstance(valueTableDesc[tag].getDeserializerClass(), null);
        SerDeUtils.initializeSerDe(
            inputValueDeserializer[tag], null, valueTableDesc[tag].getProperties(), null);
        valueObjectInspector[tag] = inputValueDeserializer[tag].getObjectInspector();

        ArrayList<ObjectInspector> ois = new ArrayList<ObjectInspector>();

        if (vectorized) {
          /* vectorization only works with struct object inspectors */
          valueStructInspectors[tag] = (StructObjectInspector) valueObjectInspector[tag];

          ObjectPair<VectorizedRowBatch, StandardStructObjectInspector> pair =
              VectorizedBatchUtil.constructVectorizedRowBatch(
                  keyStructInspector,
                  valueStructInspectors[tag],
                  gWork.getVectorScratchColumnTypeMap());
          batches[tag] = pair.getFirst();
          final int totalColumns =
              keysColumnOffset + valueStructInspectors[tag].getAllStructFieldRefs().size();
          valueStringWriters[tag] = new ArrayList<VectorExpressionWriter>(totalColumns);
          valueStringWriters[tag].addAll(
              Arrays.asList(
                  VectorExpressionWriterFactory.genVectorStructExpressionWritables(
                      keyStructInspector)));
          valueStringWriters[tag].addAll(
              Arrays.asList(
                  VectorExpressionWriterFactory.genVectorStructExpressionWritables(
                      valueStructInspectors[tag])));

          rowObjectInspector[tag] = pair.getSecond();
        } else {
          ois.add(keyObjectInspector);
          ois.add(valueObjectInspector[tag]);
          // reducer.setGroupKeyObjectInspector(keyObjectInspector);
          rowObjectInspector[tag] =
              ObjectInspectorFactory.getStandardStructObjectInspector(
                  Utilities.reduceFieldNameList, ois);
        }
      }
    } catch (Exception e) {
      throw new RuntimeException(e);
    }

    ExecMapperContext execContext = new ExecMapperContext(job);
    localWork = gWork.getMapRedLocalWork();
    execContext.setJc(jc);
    execContext.setLocalWork(localWork);
    reducer.passExecContext(execContext);

    reducer.setReporter(rp);
    OperatorUtils.setChildrenCollector(
        Arrays.<Operator<? extends OperatorDesc>>asList(reducer), output);

    // initialize reduce operator tree
    try {
      LOG.info(reducer.dump(0));
      reducer.initialize(jc, rowObjectInspector);

      if (localWork != null) {
        for (Operator<? extends OperatorDesc> dummyOp : localWork.getDummyParentOp()) {
          dummyOp.setExecContext(execContext);
          dummyOp.initialize(jc, null);
        }
      }

    } catch (Throwable e) {
      abort = true;
      if (e instanceof OutOfMemoryError) {
        // Don't create a new object if we are already out of memory
        throw (OutOfMemoryError) e;
      } else {
        throw new RuntimeException("Reduce operator initialization failed", e);
      }
    }
    perfLogger.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_INIT_OPERATORS);
  }