You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pig.apache.org by xu...@apache.org on 2016/11/03 15:58:20 UTC
svn commit: r1767913 -
/pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkLauncher.java
Author: xuefu
Date: Thu Nov 3 15:58:19 2016
New Revision: 1767913
URL: http://svn.apache.org/viewvc?rev=1767913&view=rev
Log:
PIG-5052: Initialize MRConfiguration.JOB_ID in spark mode correctly (Liyun via Xuefu)
Modified:
pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkLauncher.java
Modified: pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkLauncher.java
URL: http://svn.apache.org/viewvc/pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkLauncher.java?rev=1767913&r1=1767912&r2=1767913&view=diff
==============================================================================
--- pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkLauncher.java (original)
+++ pig/branches/spark/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkLauncher.java Thu Nov 3 15:58:19 2016
@@ -45,6 +45,7 @@ import org.apache.pig.backend.BackendExc
import org.apache.pig.backend.executionengine.ExecException;
import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
import org.apache.pig.backend.hadoop.executionengine.Launcher;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRConfiguration;
import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapReduce;
import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
@@ -162,10 +163,8 @@ public class SparkLauncher extends Launc
startSparkIfNeeded(pigContext);
- // Set a unique group id for this query, so we can lookup all Spark job
- // ids
- // related to this query.
- jobGroupID = UUID.randomUUID().toString();
+ jobGroupID = sparkContext.getConf().getAppId();
+ jobConf.set(MRConfiguration.JOB_ID, jobGroupID);
sparkContext.setJobGroup(jobGroupID, "Pig query to Spark cluster",
false);
jobMetricsListener.reset();