hive-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From xu...@apache.org
Subject svn commit: r1660304 - in /hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark: RemoteHiveSparkClient.java SparkUtilities.java
Date Tue, 17 Feb 2015 06:59:50 GMT
Author: xuefu
Date: Tue Feb 17 06:59:49 2015
New Revision: 1660304

URL: http://svn.apache.org/r1660304
Log:
HIVE-9696: Address RB comments for HIVE-9425 [Spark Branch] (Rui via Xuefu)

Modified:
    hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java
    hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java?rev=1660304&r1=1660303&r2=1660304&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java
(original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/RemoteHiveSparkClient.java
Tue Feb 17 06:59:49 2015
@@ -165,7 +165,9 @@ public class RemoteHiveSparkClient imple
       try {
         URI fileUri = SparkUtilities.getURI(addedFile);
         if (fileUri != null && !localFiles.contains(fileUri)) {
-          fileUri = SparkUtilities.uploadToHDFS(fileUri, hiveConf);
+          if (SparkUtilities.needUploadToHDFS(fileUri, sparkConf)) {
+            fileUri = SparkUtilities.uploadToHDFS(fileUri, hiveConf);
+          }
           localFiles.add(fileUri);
           remoteClient.addFile(fileUri);
         }
@@ -180,7 +182,9 @@ public class RemoteHiveSparkClient imple
       try {
         URI jarUri = SparkUtilities.getURI(addedJar);
         if (jarUri != null && !localJars.contains(jarUri)) {
-          jarUri = SparkUtilities.uploadToHDFS(jarUri, hiveConf);
+          if (SparkUtilities.needUploadToHDFS(jarUri, sparkConf)) {
+            jarUri = SparkUtilities.uploadToHDFS(jarUri, hiveConf);
+          }
           localJars.add(jarUri);
           remoteClient.addJar(jarUri);
         }

Modified: hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java
URL: http://svn.apache.org/viewvc/hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java?rev=1660304&r1=1660303&r2=1660304&view=diff
==============================================================================
--- hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java
(original)
+++ hive/branches/spark/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java
Tue Feb 17 06:59:49 2015
@@ -34,6 +34,7 @@ import org.apache.hadoop.hive.ql.io.Hive
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.io.BytesWritable;
+import org.apache.spark.SparkConf;
 
 /**
  * Contains utilities methods used as part of Spark tasks.
@@ -69,7 +70,7 @@ public class SparkUtilities {
   }
 
   /**
-   * Copies local file to HDFS in yarn-cluster mode.
+   * Uploads a local file to HDFS
    *
    * @param source
    * @param conf
@@ -77,18 +78,18 @@ public class SparkUtilities {
    * @throws IOException
    */
   public static URI uploadToHDFS(URI source, HiveConf conf) throws IOException {
-    URI result = source;
-    if (conf.get("spark.master").equals("yarn-cluster")) {
-      if (!source.getScheme().equals("hdfs")) {
-        Path tmpDir = SessionState.getHDFSSessionPath(conf);
-        FileSystem fileSystem = FileSystem.get(conf);
-        fileSystem.copyFromLocalFile(new Path(source.getPath()), tmpDir);
-        String filePath = tmpDir + File.separator + getFileName(source);
-        Path fullPath = fileSystem.getFileStatus(new Path(filePath)).getPath();
-        result = fullPath.toUri();
-      }
-    }
-    return result;
+    Path tmpDir = SessionState.getHDFSSessionPath(conf);
+    FileSystem fileSystem = FileSystem.get(conf);
+    fileSystem.copyFromLocalFile(new Path(source.getPath()), tmpDir);
+    String filePath = tmpDir + File.separator + getFileName(source);
+    Path fullPath = fileSystem.getFileStatus(new Path(filePath)).getPath();
+    return fullPath.toUri();
+  }
+
+  // checks if a resource has to be uploaded to HDFS for yarn-cluster mode
+  public static boolean needUploadToHDFS(URI source, SparkConf sparkConf) {
+    return sparkConf.get("spark.master").equals("yarn-cluster") &&
+        !source.getScheme().equals("hdfs");
   }
 
   private static String getFileName(URI uri) {



Mime
View raw message