Repository: tez Updated Branches: refs/heads/master b4c949c9c -> eb6fb67c4
TEZ-3477. MRInputHelpers generateInputSplitsToMem public API modified (jeagles) Project: http://git-wip-us.apache.org/repos/asf/tez/repo Commit: http://git-wip-us.apache.org/repos/asf/tez/commit/eb6fb67c Tree: http://git-wip-us.apache.org/repos/asf/tez/tree/eb6fb67c Diff: http://git-wip-us.apache.org/repos/asf/tez/diff/eb6fb67c Branch: refs/heads/master Commit: eb6fb67c4220ea868fee6d73cf9e9b76af3696af Parents: b4c949c Author: Jonathan Eagles <[email protected]> Authored: Tue Nov 8 12:59:28 2016 -0600 Committer: Jonathan Eagles <[email protected]> Committed: Tue Nov 8 12:59:28 2016 -0600 ---------------------------------------------------------------------- CHANGES.txt | 1 + .../tez/mapreduce/hadoop/MRInputHelpers.java | 61 ++++++++++++++++++++ 2 files changed, 62 insertions(+) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/tez/blob/eb6fb67c/CHANGES.txt ---------------------------------------------------------------------- diff --git a/CHANGES.txt b/CHANGES.txt index ecfe935..88767ec 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -7,6 +7,7 @@ INCOMPATIBLE CHANGES ALL CHANGES: + TEZ-3477. MRInputHelpers generateInputSplitsToMem public API modified TEZ-3465. Support broadcast edge into cartesian product vertex and forbid other edges. TEZ-3493. DAG submit timeout cannot be set to a month TEZ-3505. Move license to the file header for TezBytesWritableSerialization http://git-wip-us.apache.org/repos/asf/tez/blob/eb6fb67c/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java ---------------------------------------------------------------------- diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java index 97e1677..b0a76fa 100644 --- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java +++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java @@ -102,6 +102,7 @@ public class MRInputHelpers { * as a data source to a {@link org.apache.tez.dag.api.Vertex} */ @InterfaceStability.Unstable + @InterfaceAudience.LimitedPrivate({"hive, pig"}) public static DataSourceDescriptor configureMRInputWithLegacySplitGeneration(Configuration conf, Path splitsDir, boolean useLegacyInput) { @@ -140,6 +141,7 @@ public class MRInputHelpers { * @throws IOException */ @InterfaceStability.Evolving + @InterfaceAudience.LimitedPrivate({"hive, pig"}) public static MRRuntimeProtos.MRInputUserPayloadProto parseMRInputPayload(UserPayload payload) throws IOException { return MRRuntimeProtos.MRInputUserPayloadProto.parseFrom(ByteString.copyFrom(payload.getPayload())); @@ -157,6 +159,7 @@ public class MRInputHelpers { */ @SuppressWarnings("unchecked") @InterfaceStability.Evolving + @InterfaceAudience.LimitedPrivate({"hive, pig"}) public static InputSplit createOldFormatSplitFromUserPayload( MRRuntimeProtos.MRSplitProto splitProto, SerializationFactory serializationFactory) throws IOException { @@ -239,6 +242,7 @@ public class MRInputHelpers { } @InterfaceStability.Evolving + @InterfaceAudience.LimitedPrivate({"hive, pig"}) public static MRRuntimeProtos.MRSplitProto createSplitProto( org.apache.hadoop.mapred.InputSplit oldSplit) throws IOException { MRRuntimeProtos.MRSplitProto.Builder builder = MRRuntimeProtos.MRSplitProto.newBuilder(); @@ -278,6 +282,38 @@ public class MRInputHelpers { * @throws InterruptedException */ @InterfaceStability.Unstable + @InterfaceAudience.LimitedPrivate({"hive, pig"}) + public static InputSplitInfoMem generateInputSplitsToMem(Configuration conf, + boolean groupSplits, int targetTasks) + throws IOException, ClassNotFoundException, InterruptedException { + return generateInputSplitsToMem(conf, groupSplits, true, targetTasks); + } + + /** + * Generates Input splits and stores them in a {@link org.apache.tez.mapreduce.protos.MRRuntimeProtos.MRSplitsProto} instance. + * + * Returns an instance of {@link InputSplitInfoMem} + * + * With grouping enabled, the eventual configuration used by the tasks, will have + * the user-specified InputFormat replaced by either {@link org.apache.hadoop.mapred.split.TezGroupedSplitsInputFormat} + * or {@link org.apache.hadoop.mapreduce.split.TezGroupedSplitsInputFormat} + * + * @param conf + * an instance of Configuration which is used to determine whether + * the mapred of mapreduce API is being used. This Configuration + * instance should also contain adequate information to be able to + * generate splits - like the InputFormat being used and related + * configuration. + * @param groupSplits whether to group the splits or not + * @param sortSplits whether to sort the splits or not + * @param targetTasks the number of target tasks if grouping is enabled. Specify as 0 otherwise. + * @return an instance of {@link InputSplitInfoMem} which supports a subset of + * the APIs defined on {@link InputSplitInfo} + * @throws IOException + * @throws ClassNotFoundException + * @throws InterruptedException + */ + @InterfaceStability.Unstable public static InputSplitInfoMem generateInputSplitsToMem(Configuration conf, boolean groupSplits, boolean sortSplits, int targetTasks) throws IOException, ClassNotFoundException, InterruptedException { @@ -669,6 +705,31 @@ public class MRInputHelpers { } /** + * Called to specify that grouping of input splits be performed by Tez + * The conf should have the input format class configuration + * set to the TezGroupedSplitsInputFormat. The real input format class name + * should be passed as an argument to this method. + * <p/> + * With grouping enabled, the eventual configuration used by the tasks, will have + * the user-specified InputFormat replaced by either {@link org.apache.hadoop.mapred.split.TezGroupedSplitsInputFormat} + * or {@link org.apache.hadoop.mapreduce.split.TezGroupedSplitsInputFormat} + */ + @InterfaceAudience.Private + protected static UserPayload createMRInputPayloadWithGrouping(Configuration conf) throws IOException { + Preconditions + .checkArgument(conf != null, "Configuration must be specified"); + return createMRInputPayload(TezUtils.createByteStringFromConf(conf), + null, true, true); + } + + @InterfaceAudience.Private + protected static UserPayload createMRInputPayload(Configuration conf, + MRRuntimeProtos.MRSplitsProto mrSplitsProto) throws + IOException { + return createMRInputPayload(conf, mrSplitsProto, false, true); + } + + /** * When isGrouped is true, it specifies that grouping of input splits be * performed by Tez The conf should have the input format class configuration * set to the TezGroupedSplitsInputFormat. The real input format class name
