Author: rohini Date: Fri Oct 12 17:01:44 2018 New Revision: 1843691 URL: http://svn.apache.org/viewvc?rev=1843691&view=rev Log: Fix test failure for PIG-5359
Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/util/TezJobSplitWriter.java Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/util/TezJobSplitWriter.java URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/util/TezJobSplitWriter.java?rev=1843691&r1=1843690&r2=1843691&view=diff ============================================================================== --- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/util/TezJobSplitWriter.java (original) +++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/util/TezJobSplitWriter.java Fri Oct 12 17:01:44 2018 @@ -73,8 +73,8 @@ public class TezJobSplitWriter { * @throws IOException * @throws InterruptedException */ - public static <T extends InputSplit> void createSplitFiles(Path jobSubmitDir, Configuration conf, FileSystem fs, - T[] splits, MRSplitsProto splitsProto) throws IOException, InterruptedException { + public static void createSplitFiles(Path jobSubmitDir, Configuration conf, FileSystem fs, + InputSplit[] splits, MRSplitsProto splitsProto) throws IOException, InterruptedException { FSDataOutputStream out = createFile(fs, JobSubmissionFiles.getJobSplitFile(jobSubmitDir), conf); SplitMetaInfo[] info = writeSplits(conf, splits, out, splitsProto); out.close(); @@ -108,11 +108,11 @@ public class TezJobSplitWriter { * @throws InterruptedException */ @SuppressWarnings("unchecked") - private static <T extends InputSplit> SplitMetaInfo[] writeSplits(Configuration conf, T[] array, + private static SplitMetaInfo[] writeSplits(Configuration conf, InputSplit[] inputSplits, FSDataOutputStream out, MRSplitsProto splitsProto) throws IOException, InterruptedException { SplitMetaInfo[] info = null; - if (array.length != 0) { - info = new SplitMetaInfo[array.length]; + if (inputSplits.length != 0) { + info = new SplitMetaInfo[inputSplits.length]; SerializationFactory factory = new SerializationFactory(conf); int maxBlockLocations = conf.getInt(MRConfig.MAX_BLOCK_LOCATIONS_KEY, MRConfig.MAX_BLOCK_LOCATIONS_DEFAULT); long offset = out.getPos(); @@ -121,16 +121,18 @@ public class TezJobSplitWriter { long prevCount = out.getPos(); Text.writeString(out, splitProto.getSplitClassName()); splitProto.getSplitBytes().writeTo(out); - info[i++] = createSplitMetaInfo(array[i], offset, maxBlockLocations); + info[i] = createSplitMetaInfo(inputSplits[i], offset, maxBlockLocations); + i++; offset += out.getPos() - prevCount; } - while(i < array.length) { + while(i < inputSplits.length) { long prevCount = out.getPos(); - Text.writeString(out, array[i].getClass().getName()); - Serializer<T> serializer = factory.getSerializer((Class<T>) array[i].getClass()); + Text.writeString(out, inputSplits[i].getClass().getName()); + Serializer<InputSplit> serializer = factory.getSerializer((Class<InputSplit>) inputSplits[i].getClass()); serializer.open(out); - serializer.serialize(array[i]); - info[i++] = createSplitMetaInfo(array[i], offset, maxBlockLocations); + serializer.serialize(inputSplits[i]); + info[i] = createSplitMetaInfo(inputSplits[i], offset, maxBlockLocations); + i++; offset += out.getPos() - prevCount; } }