public List getGroupedSplits()

in tez-mapreduce/src/main/java/org/apache/hadoop/mapreduce/split/TezMapReduceSplitsGrouper.java [74:418]


  public List<InputSplit> getGroupedSplits(Configuration conf,
      List<InputSplit> originalSplits, int desiredNumSplits,
      String wrappedInputFormatName) throws IOException, InterruptedException {
    LOG.info("Grouping splits in Tez");

    int configNumSplits = conf.getInt(TezConfiguration.TEZ_AM_GROUPING_SPLIT_COUNT, 0);
    if (configNumSplits > 0) {
      // always use config override if specified
      desiredNumSplits = configNumSplits;
      LOG.info("Desired numSplits overridden by config to: " + desiredNumSplits);
    }
    
    if (! (configNumSplits > 0 || 
          originalSplits == null || 
          originalSplits.size() == 0)) {
      // numSplits has not been overridden by config
      // numSplits has been set at runtime
      // there are splits generated
      // desired splits is less than number of splits generated
      // Do sanity checks
      long totalLength = 0;
      for (InputSplit split : originalSplits) {
        totalLength += split.getLength();
      }
  
      int splitCount = desiredNumSplits>0?desiredNumSplits:originalSplits.size();
      long lengthPerGroup = totalLength/splitCount;

      long maxLengthPerGroup = conf.getLong(
          TezConfiguration.TEZ_AM_GROUPING_SPLIT_MAX_SIZE,
          TezConfiguration.TEZ_AM_GROUPING_SPLIT_MAX_SIZE_DEFAULT);
      long minLengthPerGroup = conf.getLong(
          TezConfiguration.TEZ_AM_GROUPING_SPLIT_MIN_SIZE,
          TezConfiguration.TEZ_AM_GROUPING_SPLIT_MIN_SIZE_DEFAULT);
      if (maxLengthPerGroup < minLengthPerGroup || 
          minLengthPerGroup <=0) {
        throw new TezUncheckedException(
          "Invalid max/min group lengths. Required min>0, max>=min. " +
          " max: " + maxLengthPerGroup + " min: " + minLengthPerGroup);
      }
      if (lengthPerGroup > maxLengthPerGroup) {
        // splits too big to work. Need to override with max size.
        int newDesiredNumSplits = (int)(totalLength/maxLengthPerGroup) + 1;
        LOG.info("Desired splits: " + desiredNumSplits + " too small. " + 
            " Desired splitLength: " + lengthPerGroup + 
            " Max splitLength: " + maxLengthPerGroup +
            " New desired splits: " + newDesiredNumSplits + 
            " Total length: " + totalLength +
            " Original splits: " + originalSplits.size());
        
        desiredNumSplits = newDesiredNumSplits;
      } else if (lengthPerGroup < minLengthPerGroup) {
        // splits too small to work. Need to override with size.
        int newDesiredNumSplits = (int)(totalLength/minLengthPerGroup) + 1;
        LOG.info("Desired splits: " + desiredNumSplits + " too large. " + 
            " Desired splitLength: " + lengthPerGroup + 
            " Min splitLength: " + minLengthPerGroup +
            " New desired splits: " + newDesiredNumSplits + 
            " Total length: " + totalLength +
            " Original splits: " + originalSplits.size());
        
        desiredNumSplits = newDesiredNumSplits;
      }
    }
     
    List<InputSplit> groupedSplits = null;
    
    if (desiredNumSplits == 0 ||
        originalSplits.size() == 0 ||
        desiredNumSplits >= originalSplits.size()) {
      // nothing set. so return all the splits as is
      LOG.info("Using original number of splits: " + originalSplits.size() +
          " desired splits: " + desiredNumSplits);
      groupedSplits = new ArrayList<InputSplit>(originalSplits.size());
      for (InputSplit split : originalSplits) {
        TezGroupedSplit newSplit = 
            new TezGroupedSplit(1, wrappedInputFormatName, split.getLocations());
        newSplit.addSplit(split);
        groupedSplits.add(newSplit);
      }
      return groupedSplits;
    }
    
    String emptyLocation = "EmptyLocation";
    String[] emptyLocations = {emptyLocation};
    groupedSplits = new ArrayList<InputSplit>(desiredNumSplits);
    
    long totalLength = 0;
    Map<String, LocationHolder> distinctLocations = new HashMap<String, LocationHolder>();
    // go through splits and add them to locations
    for (InputSplit split : originalSplits) {
      totalLength += split.getLength();
      String[] locations = split.getLocations();
      if (locations == null || locations.length == 0) {
        locations = emptyLocations;
      }
      for (String location : locations ) {
        if (location == null) {
          location = emptyLocation;
        }
        distinctLocations.put(location, null);
      }
    }
    
    long lengthPerGroup = totalLength/desiredNumSplits;
    int numNodeLocations = distinctLocations.size();
    int numSplitsPerLocation = originalSplits.size()/numNodeLocations;
    int numSplitsInGroup = originalSplits.size()/desiredNumSplits;
    
    // allocation loop here so that we have a good initial size for the lists
    for (String location : distinctLocations.keySet()) {
      distinctLocations.put(location, new LocationHolder(numSplitsPerLocation+1));
    }
    
    Set<String> locSet = new HashSet<String>();
    for (InputSplit split : originalSplits) {
      locSet.clear();
      SplitHolder splitHolder = new SplitHolder(split);
      String[] locations = split.getLocations();
      if (locations == null || locations.length == 0) {
        locations = emptyLocations;
      }
      for (String location : locations) {
        if (location == null) {
          location = emptyLocation;
        }
        locSet.add(location);
      }
      for (String location : locSet) {
        LocationHolder holder = distinctLocations.get(location);
        holder.splits.add(splitHolder);
      }
    }
    
    boolean groupByLength = conf.getBoolean(
        TezConfiguration.TEZ_AM_GROUPING_SPLIT_BY_LENGTH,
        TezConfiguration.TEZ_AM_GROUPING_SPLIT_BY_LENGTH_DEFAULT);
    boolean groupByCount = conf.getBoolean(
        TezConfiguration.TEZ_AM_GROUPING_SPLIT_BY_COUNT,
        TezConfiguration.TEZ_AM_GROUPING_SPLIT_BY_COUNT_DEFAULT);
    if (!(groupByLength || groupByCount)) {
      throw new TezUncheckedException(
          "None of the grouping parameters are true: "
              + TezConfiguration.TEZ_AM_GROUPING_SPLIT_BY_LENGTH + ", "
              + TezConfiguration.TEZ_AM_GROUPING_SPLIT_BY_COUNT);
    }
    LOG.info("Desired numSplits: " + desiredNumSplits +
        " lengthPerGroup: " + lengthPerGroup +
        " numLocations: " + numNodeLocations +
        " numSplitsPerLocation: " + numSplitsPerLocation +
        " numSplitsInGroup: " + numSplitsInGroup + 
        " totalLength: " + totalLength +
        " numOriginalSplits: " + originalSplits.size() +
        " . Grouping by length: " + groupByLength + " count: " + groupByCount);
    
    // go through locations and group splits
    int splitsProcessed = 0;
    List<SplitHolder> group = new ArrayList<SplitHolder>(numSplitsInGroup);
    Set<String> groupLocationSet = new HashSet<String>(10);
    boolean allowSmallGroups = false;
    boolean doingRackLocal = false;
    int iterations = 0;
    while (splitsProcessed < originalSplits.size()) {
      iterations++;
      int numFullGroupsCreated = 0;
      for (Map.Entry<String, LocationHolder> entry : distinctLocations.entrySet()) {
        group.clear();
        groupLocationSet.clear();
        String location = entry.getKey();
        LocationHolder holder = entry.getValue();
        SplitHolder splitHolder = holder.getUnprocessedHeadSplit();
        if (splitHolder == null) {
          // all splits on node processed
          continue;
        }
        int oldHeadIndex = holder.headIndex;
        long groupLength = 0;
        int groupNumSplits = 0;
        do {
          group.add(splitHolder);
          groupLength += splitHolder.split.getLength();
          groupNumSplits++;
          holder.incrementHeadIndex();
          splitHolder = holder.getUnprocessedHeadSplit();
        } while(splitHolder != null  
            && (!groupByLength || 
                (groupLength + splitHolder.split.getLength() <= lengthPerGroup))
            && (!groupByCount || 
                (groupNumSplits + 1 <= numSplitsInGroup)));

        if (holder.isEmpty() 
            && !allowSmallGroups
            && (!groupByLength || groupLength < lengthPerGroup/2)
            && (!groupByCount || groupNumSplits < numSplitsInGroup/2)) {
          // group too small, reset it
          holder.headIndex = oldHeadIndex;
          continue;
        }
        
        numFullGroupsCreated++;

        // One split group created
        String[] groupLocation = {location};
        if (location == emptyLocation) {
          groupLocation = null;
        } else if (doingRackLocal) {
          for (SplitHolder splitH : group) {
            String[] locations = splitH.split.getLocations();
            if (locations != null) {
              for (String loc : locations) {
                if (loc != null) {
                  groupLocationSet.add(loc);
                }
              }
            }
          }
          groupLocation = groupLocationSet.toArray(groupLocation);
        }
        TezGroupedSplit groupedSplit = 
            new TezGroupedSplit(group.size(), wrappedInputFormatName, 
                groupLocation,
                // pass rack local hint directly to AM
                ((doingRackLocal && location != emptyLocation)?location:null));
        for (SplitHolder groupedSplitHolder : group) {
          groupedSplit.addSplit(groupedSplitHolder.split);
          Preconditions.checkState(groupedSplitHolder.isProcessed == false,
              "Duplicates in grouping at location: " + location);
          groupedSplitHolder.isProcessed = true;
          splitsProcessed++;
        }
        if (LOG.isDebugEnabled()) {
          LOG.debug("Grouped " + group.size()
              + " length: " + groupedSplit.getLength()
              + " split at: " + location);
        }
        groupedSplits.add(groupedSplit);
      }
      
      if (!doingRackLocal && numFullGroupsCreated < 1) {
        // no node could create a node-local group. go rack-local
        doingRackLocal = true;
        // re-create locations
        int numRemainingSplits = originalSplits.size() - splitsProcessed;
        Set<InputSplit> remainingSplits = new HashSet<InputSplit>(numRemainingSplits);
        // gather remaining splits.
        for (Map.Entry<String, LocationHolder> entry : distinctLocations.entrySet()) {
          LocationHolder locHolder = entry.getValue();
          while (!locHolder.isEmpty()) {
            SplitHolder splitHolder = locHolder.getUnprocessedHeadSplit();
            if (splitHolder != null) {
              remainingSplits.add(splitHolder.split);
              locHolder.incrementHeadIndex();
            }
          }
        }
        if (remainingSplits.size() != numRemainingSplits) {
          throw new TezUncheckedException("Expected: " + numRemainingSplits 
              + " got: " + remainingSplits.size());
        }
        
        // doing all this now instead of up front because the number of remaining
        // splits is expected to be much smaller
        RackResolver.init(conf);
        Map<String, String> locToRackMap = new HashMap<String, String>(distinctLocations.size());
        Map<String, LocationHolder> rackLocations = new HashMap<String, LocationHolder>();
        for (String location : distinctLocations.keySet()) {
          String rack = emptyLocation;
          if (location != emptyLocation) {
            rack = RackResolver.resolve(location).getNetworkLocation();
          }
          locToRackMap.put(location, rack);
          if (rackLocations.get(rack) == null) {
            // splits will probably be located in all racks
            rackLocations.put(rack, new LocationHolder(numRemainingSplits));
          }
        }
        HashSet<String> rackSet = new HashSet<String>(rackLocations.size());
        for (InputSplit split : remainingSplits) {
          rackSet.clear();
          SplitHolder splitHolder = new SplitHolder(split);
          String[] locations = split.getLocations();
          if (locations == null || locations.length == 0) {
            locations = emptyLocations;
          }
          for (String location : locations ) {
            if (location == null) {
              location = emptyLocation;
            }
            rackSet.add(locToRackMap.get(location));
          }
          for (String rack : rackSet) {
            rackLocations.get(rack).splits.add(splitHolder);
          }
        }
        
        distinctLocations.clear();
        distinctLocations = rackLocations;
        // adjust split length to be smaller because the data is non local
        float rackSplitReduction = conf.getFloat(
            TezConfiguration.TEZ_AM_GROUPING_RACK_SPLIT_SIZE_REDUCTION,
            TezConfiguration.TEZ_AM_GROUPING_RACK_SPLIT_SIZE_REDUCTION_DEFAULT);
        if (rackSplitReduction > 0) {
          long newLengthPerGroup = (long)(lengthPerGroup*rackSplitReduction);
          int newNumSplitsInGroup = (int) (numSplitsInGroup*rackSplitReduction);
          if (newLengthPerGroup > 0) {
            lengthPerGroup = newLengthPerGroup;
          }
          if (newNumSplitsInGroup > 0) {
            numSplitsInGroup = newNumSplitsInGroup;
          }
        }
        
        LOG.info("Doing rack local after iteration: " + iterations +
            " splitsProcessed: " + splitsProcessed + 
            " numFullGroupsInRound: " + numFullGroupsCreated +
            " totalGroups: " + groupedSplits.size() +
            " lengthPerGroup: " + lengthPerGroup +
            " numSplitsInGroup: " + numSplitsInGroup);
        
        // dont do smallGroups for the first pass
        continue;
      }
      
      if (!allowSmallGroups && numFullGroupsCreated <= numNodeLocations/10) {
        // a few nodes have a lot of data or data is thinly spread across nodes
        // so allow small groups now        
        allowSmallGroups = true;
        LOG.info("Allowing small groups after iteration: " + iterations +
            " splitsProcessed: " + splitsProcessed + 
            " numFullGroupsInRound: " + numFullGroupsCreated +
            " totalGroups: " + groupedSplits.size());
      }
      
      if (LOG.isDebugEnabled()) {
        LOG.debug("Iteration: " + iterations +
            " splitsProcessed: " + splitsProcessed + 
            " numFullGroupsInRound: " + numFullGroupsCreated +
            " totalGroups: " + groupedSplits.size());
      }
    }
    LOG.info("Number of splits desired: " + desiredNumSplits + 
        " created: " + groupedSplits.size() + 
        " splitsProcessed: " + splitsProcessed);
    return groupedSplits;
  }