Repository: hive
Updated Branches:
  refs/heads/master ffdbee050 -> dab8cc012


HIVE-18871: hive on tez execution error due to set hive.aux.jars.path to 
hdfs:// (zhuwei reviewed by Prasanth Jayachandran)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/dab8cc01
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/dab8cc01
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/dab8cc01

Branch: refs/heads/master
Commit: dab8cc01297bc82e17d3e666a49e3b6392f878b4
Parents: ffdbee0
Author: zhuwei <zhuwei8...@gmail.com>
Authored: Mon Sep 24 10:24:27 2018 -0700
Committer: Prasanth Jayachandran <prasan...@apache.org>
Committed: Tue Sep 25 10:09:41 2018 -0700

----------------------------------------------------------------------
 ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java | 8 +++++++-
 1 file changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/dab8cc01/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java 
b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java
index de0abd1..1a88b77 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java
@@ -62,6 +62,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
@@ -1263,7 +1264,12 @@ public class DagUtils {
         return createLocalResource(destFS, dest, type, 
LocalResourceVisibility.PRIVATE);
       }
       try {
-        destFS.copyFromLocalFile(false, false, src, dest);
+        if (src.toUri().getScheme()!=null) {
+          FileUtil.copy(src.getFileSystem(conf), src, destFS, dest, false, 
false, conf);
+        }
+        else {
+          destFS.copyFromLocalFile(false, false, src, dest);
+        }
         synchronized (notifier) {
           notifier.notifyAll(); // Notify if we have successfully copied the 
file.
         }

Reply via email to