public List initialize()

in tez-mapreduce/src/main/java/org/apache/tez/mapreduce/common/MRInputAMSplitGenerator.java [62:203]


  public List<Event> initialize(TezRootInputInitializerContext rootInputContext)
      throws Exception {
    Stopwatch sw = null;
    if (LOG.isDebugEnabled()) {
      sw = new Stopwatch().start();
    }
    MRInputUserPayloadProto userPayloadProto = MRHelpers
        .parseMRInputPayload(rootInputContext.getUserPayload());
    if (LOG.isDebugEnabled()) {
      sw.stop();
      LOG.debug("Time to parse MRInput payload into prot: "
          + sw.elapsedMillis());
    }
    if (LOG.isDebugEnabled()) {
      sw.reset().start();
    }
    Configuration conf = MRHelpers.createConfFromByteString(userPayloadProto
        .getConfigurationBytes());
    
    sendSerializedEvents = conf.getBoolean(
        MRJobConfig.MR_TEZ_INPUT_INITIALIZER_SERIALIZE_EVENT_PAYLOAD,
        MRJobConfig.MR_TEZ_INPUT_INITIALIZER_SERIALIZE_EVENT_PAYLOAD_DEFAULT);
    LOG.info("Emitting serialized splits: " + sendSerializedEvents);
    if (LOG.isDebugEnabled()) {
      sw.stop();
      LOG.debug("Time converting ByteString to configuration: " + sw.elapsedMillis());
    }

    if (LOG.isDebugEnabled()) {
      sw.reset().start();
    }

    int totalResource = rootInputContext.getTotalAvailableResource().getMemory();
    int taskResource = rootInputContext.getVertexTaskResource().getMemory();
    float waves = conf.getFloat(
        TezConfiguration.TEZ_AM_GROUPING_SPLIT_WAVES,
        TezConfiguration.TEZ_AM_GROUPING_SPLIT_WAVES_DEFAULT);

    int numTasks = (int)((totalResource*waves)/taskResource);

    LOG.info("Input " + rootInputContext.getInputName() + " asking for " + numTasks
        + " tasks. Headroom: " + totalResource + " Task Resource: "
        + taskResource + " waves: " + waves);

    // Read all credentials into the credentials instance stored in JobConf.
    JobConf jobConf = new JobConf(conf);
    jobConf.getCredentials().mergeAll(UserGroupInformation.getCurrentUser().getCredentials());

    InputSplitInfoMem inputSplitInfo = null;
    String realInputFormatName = userPayloadProto.getInputFormatName(); 
    if ( realInputFormatName != null && !realInputFormatName.isEmpty()) {
      // split grouping on the AM
      if (jobConf.getUseNewMapper()) {
        LOG.info("Grouping mapreduce api input splits");
        Job job = Job.getInstance(jobConf);
        org.apache.hadoop.mapreduce.InputSplit[] splits = MRHelpers
            .generateNewSplits(job, realInputFormatName, numTasks);

        // Move all this into a function
        List<TaskLocationHint> locationHints = Lists
            .newArrayListWithCapacity(splits.length);
        for (org.apache.hadoop.mapreduce.InputSplit split : splits) {
          String rack = 
              ((org.apache.hadoop.mapreduce.split.TezGroupedSplit) split).getRack();
          if (rack == null) {
            if (split.getLocations() != null) {
              locationHints.add(new TaskLocationHint(new HashSet<String>(Arrays
                  .asList(split.getLocations())), null));
            } else {
              locationHints.add(new TaskLocationHint(null, null));
            }
          } else {
            locationHints.add(new TaskLocationHint(null, 
                Collections.singleton(rack)));
          }
        }
        inputSplitInfo = new InputSplitInfoMem(splits, locationHints, splits.length, null, jobConf);
      } else {
        LOG.info("Grouping mapred api input splits");
        org.apache.hadoop.mapred.InputSplit[] splits = MRHelpers
            .generateOldSplits(jobConf, realInputFormatName, numTasks);
        List<TaskLocationHint> locationHints = Lists
            .newArrayListWithCapacity(splits.length);
        for (org.apache.hadoop.mapred.InputSplit split : splits) {
          String rack = 
              ((org.apache.hadoop.mapred.split.TezGroupedSplit) split).getRack();
          if (rack == null) {
            if (split.getLocations() != null) {
              locationHints.add(new TaskLocationHint(new HashSet<String>(Arrays
                  .asList(split.getLocations())), null));
            } else {
              locationHints.add(new TaskLocationHint(null, null));
            }
          } else {
            locationHints.add(new TaskLocationHint(null, 
                Collections.singleton(rack)));
          }
        }
        inputSplitInfo = new InputSplitInfoMem(splits, locationHints, splits.length, null, jobConf);
      }
    } else {
      inputSplitInfo = MRHelpers.generateInputSplitsToMem(jobConf);
    }
    if (LOG.isDebugEnabled()) {
      sw.stop();
      LOG.debug("Time to create splits to mem: " + sw.elapsedMillis());
    }

    List<Event> events = Lists.newArrayListWithCapacity(inputSplitInfo
        .getNumTasks() + 1);
    
    RootInputConfigureVertexTasksEvent configureVertexEvent = new RootInputConfigureVertexTasksEvent(
        inputSplitInfo.getNumTasks(), inputSplitInfo.getTaskLocationHints(),
        RootInputSpecUpdate.getDefaultSinglePhysicalInputSpecUpdate());
    events.add(configureVertexEvent);

    if (sendSerializedEvents) {
      MRSplitsProto splitsProto = inputSplitInfo.getSplitsProto();
      int count = 0;
      for (MRSplitProto mrSplit : splitsProto.getSplitsList()) {
        // Unnecessary array copy, can be avoided by using ByteBuffer instead of a raw array.
        RootInputDataInformationEvent diEvent = new RootInputDataInformationEvent(count++,
            mrSplit.toByteArray());
        events.add(diEvent);
      }
    } else {
      int count = 0;
      if (inputSplitInfo.holdsNewFormatSplits()) {
        for (org.apache.hadoop.mapreduce.InputSplit split : inputSplitInfo.getNewFormatSplits()) {
          RootInputDataInformationEvent diEvent = new RootInputDataInformationEvent(count++, split);
          events.add(diEvent);
        }
      } else {
        for (org.apache.hadoop.mapred.InputSplit split : inputSplitInfo.getOldFormatSplits()) {
          RootInputDataInformationEvent diEvent = new RootInputDataInformationEvent(count++, split);
          events.add(diEvent);
        }
      }
    }
    
    return events;
  }