Author: szita Date: Tue Nov 28 13:14:51 2017 New Revision: 1816542 URL: http://svn.apache.org/viewvc?rev=1816542&view=rev Log: PIG-5316: Initialize mapred.task.id property for PoS jobs (nkollar via szita)
Modified: pig/trunk/CHANGES.txt pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkLauncher.java Modified: pig/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1816542&r1=1816541&r2=1816542&view=diff ============================================================================== --- pig/trunk/CHANGES.txt (original) +++ pig/trunk/CHANGES.txt Tue Nov 28 13:14:51 2017 @@ -26,6 +26,8 @@ PIG-5282: Upgade to Java 8 (satishsaley IMPROVEMENTS +PIG-5316: Initialize mapred.task.id property for PoS jobs (nkollar via szita) + PIG-5302: Remove HttpClient dependency (nkollar via szita) PIG-5305: Enable yarn-client mode execution of tests in Spark (1) mode (szita) Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkLauncher.java URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkLauncher.java?rev=1816542&r1=1816541&r2=1816542&view=diff ============================================================================== --- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkLauncher.java (original) +++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkLauncher.java Tue Nov 28 13:14:51 2017 @@ -42,6 +42,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.TaskAttemptID; import org.apache.pig.PigConfiguration; import org.apache.pig.PigException; import org.apache.pig.PigWarning; @@ -182,6 +183,7 @@ public class SparkLauncher extends Launc jobGroupID = String.format("%s-%s",sparkContext.getConf().getAppId(), UUID.randomUUID().toString()); jobConf.set(MRConfiguration.JOB_ID,jobGroupID); + jobConf.set(MRConfiguration.TASK_ID, new TaskAttemptID().toString()); sparkContext.setJobGroup(jobGroupID, "Pig query to Spark cluster", false);