pig-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From da...@apache.org
Subject svn commit: r1717331 - in /pig/trunk: CHANGES.txt src/org/apache/pig/backend/hadoop/executionengine/tez/TezLauncher.java src/org/apache/pig/backend/hadoop/executionengine/tez/TezResourceManager.java
Date Mon, 30 Nov 2015 21:11:13 GMT
Author: daijy
Date: Mon Nov 30 21:11:13 2015
New Revision: 1717331

URL: http://svn.apache.org/viewvc?rev=1717331&view=rev
Log:
PIG-4744: Honor tez.staging-dir setting in tez-site.xml

Modified:
    pig/trunk/CHANGES.txt
    pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/TezLauncher.java
    pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/TezResourceManager.java

Modified: pig/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/pig/trunk/CHANGES.txt?rev=1717331&r1=1717330&r2=1717331&view=diff
==============================================================================
--- pig/trunk/CHANGES.txt (original)
+++ pig/trunk/CHANGES.txt Mon Nov 30 21:11:13 2015
@@ -24,6 +24,8 @@ INCOMPATIBLE CHANGES
 
 IMPROVEMENTS
 
+PIG-4744: Honor tez.staging-dir setting in tez-site.xml (rohini via daijy)
+
 PIG-4742: Document Pig's Register Artifact Command added in PIG-4417 (akshayrai09 via daijy)
 
 PIG-4417: Pig's register command should support automatic fetching of jars from repo (akshayrai09
via daijy)

Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/TezLauncher.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/TezLauncher.java?rev=1717331&r1=1717330&r2=1717331&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/TezLauncher.java (original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/TezLauncher.java Mon Nov
30 21:11:13 2015
@@ -33,7 +33,6 @@ import java.util.concurrent.ThreadFactor
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.counters.Limits;
 import org.apache.hadoop.util.StringUtils;
@@ -139,9 +138,16 @@ public class TezLauncher extends Launche
         TezResourceManager tezResourceManager = TezResourceManager.getInstance();
         tezResourceManager.init(pc, conf);
 
-        Path stagingDir = tezResourceManager.getStagingDir();
-        log.info("Tez staging directory is " + stagingDir.toString());
-        conf.set(TezConfiguration.TEZ_AM_STAGING_DIR, stagingDir.toString());
+        String stagingDir = conf.get(TezConfiguration.TEZ_AM_STAGING_DIR);
+        String resourcesDir = tezResourceManager.getResourcesDir().toString();
+        if (stagingDir == null) {
+            // If not set in tez-site.xml, use Pig's tez resources directory as staging directory
+            // instead of TezConfiguration.TEZ_AM_STAGING_DIR_DEFAULT
+            stagingDir = resourcesDir;
+            conf.set(TezConfiguration.TEZ_AM_STAGING_DIR, resourcesDir);
+        }
+        log.info("Tez staging directory is " + stagingDir + " and resources directory is
" + resourcesDir);
+
 
         List<TezOperPlan> processedPlans = new ArrayList<TezOperPlan>();
 

Modified: pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/TezResourceManager.java
URL: http://svn.apache.org/viewvc/pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/TezResourceManager.java?rev=1717331&r1=1717330&r2=1717331&view=diff
==============================================================================
--- pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/TezResourceManager.java
(original)
+++ pig/trunk/src/org/apache/pig/backend/hadoop/executionengine/tez/TezResourceManager.java
Mon Nov 30 21:11:13 2015
@@ -17,6 +17,8 @@
  */
 package org.apache.pig.backend.hadoop.executionengine.tez;
 
+import static org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.JobControlCompiler.getFromCache;
+
 import java.io.IOException;
 import java.net.URI;
 import java.util.HashMap;
@@ -33,16 +35,14 @@ import org.apache.hadoop.yarn.api.record
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.util.ConverterUtils;
+import org.apache.pig.PigConfiguration;
 import org.apache.pig.impl.PigContext;
 import org.apache.pig.impl.io.FileLocalizer;
-import org.apache.pig.PigConfiguration;
-import static org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.JobControlCompiler.getFromCache;
-import static org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.JobControlCompiler.getCacheStagingDir;
 
 public class TezResourceManager {
     private static TezResourceManager instance = null;
     private boolean inited = false;
-    private Path stagingDir;
+    private Path resourcesDir;
     private FileSystem remoteFs;
     private Configuration conf;
     private PigContext pigContext;
@@ -57,7 +57,7 @@ public class TezResourceManager {
 
     public void init(PigContext pigContext, Configuration conf) throws IOException {
         if (!inited) {
-            this.stagingDir = FileLocalizer.getTemporaryResourcePath(pigContext);
+            this.resourcesDir = FileLocalizer.getTemporaryResourcePath(pigContext);
             this.remoteFs = FileSystem.get(conf);
             this.conf = conf;
             this.pigContext = pigContext;
@@ -65,8 +65,8 @@ public class TezResourceManager {
         }
     }
 
-    public Path getStagingDir() {
-        return stagingDir;
+    public Path getResourcesDir() {
+        return resourcesDir;
     }
 
     // Add files from the source FS as local resources. The resource name will
@@ -94,7 +94,7 @@ public class TezResourceManager {
 
                 }
 
-                Path remoteFsPath = remoteFs.makeQualified(new Path(stagingDir, resourceName));
+                Path remoteFsPath = remoteFs.makeQualified(new Path(resourcesDir, resourceName));
                 remoteFs.copyFromLocalFile(resourcePath, remoteFsPath);
                 remoteFs.setReplication(remoteFsPath, (short)conf.getInt(Job.SUBMIT_REPLICATION,
3));
                 resources.put(resourceName, remoteFsPath);



Mime
View raw message