PIG-5316: Initialize mapred.task.id property for PoS jobs (fix)

git-svn-id: https://svn.apache.org/repos/asf/pig/trunk@1816554 13f79535-47bb-0310-9956-ffa450edef68
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkLauncher.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkLauncher.java
index 6f579e9..be0719f 100644
--- a/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkLauncher.java
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkLauncher.java
@@ -42,7 +42,6 @@
 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.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 @@
         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);