LOG.info("Input " + rootInputContext.getInputName() + " asking for " + numTasks
+ " tasks. Headroom: " + totalResource + " Task Resource: "
+ taskResource + " waves: " + waves);
InputSplitInfoMem inputSplitInfo = null;
String realInputFormatName = userPayloadProto.getInputFormatName();
if ( realInputFormatName != null && !realInputFormatName.isEmpty()) {
// split grouping on the AM
JobConf jobConf = new JobConf(conf);
if (jobConf.getUseNewMapper()) {
LOG.info("Grouping mapreduce api input splits");
Job job = Job.getInstance(conf);
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, conf);
} 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, conf);
}
} else {
inputSplitInfo = MRHelpers.generateInputSplitsToMem(conf);
}
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());
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);
}
}
}