Author: szita
Date: Tue Nov 28 15:25:15 2017
New Revision: 1816554

URL: http://svn.apache.org/viewvc?rev=1816554&view=rev
Log:
PIG-5316: Initialize mapred.task.id property for PoS jobs (fix)

Modified:
    
pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkLauncher.java

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=1816554&r1=1816553&r2=1816554&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 15:25:15 2017
@@ -42,7 +42,6 @@ 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;
@@ -75,6 +74,7 @@ import org.apache.pig.backend.hadoop.exe
 import 
org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStore;
 import 
org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStream;
 import 
org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POUnion;
+import org.apache.pig.backend.hadoop.executionengine.shims.HadoopShims;
 import 
org.apache.pig.backend.hadoop.executionengine.spark.converter.BroadcastConverter;
 import 
org.apache.pig.backend.hadoop.executionengine.spark.converter.CollectedGroupConverter;
 import 
org.apache.pig.backend.hadoop.executionengine.spark.converter.CounterConverter;
@@ -183,7 +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());
+        jobConf.set(MRConfiguration.TASK_ID, 
HadoopShims.getNewTaskAttemptID().toString());
 
         sparkContext.setJobGroup(jobGroupID, "Pig query to Spark cluster",
                 false);


Reply via email to