Repository: hive Updated Branches: refs/heads/master 94ed34fa4 -> 4d9810afb
HIVE-16292: SparkUtilities upload to HDFS doesn't work with viewfs (Jimmy Xiang, reviewed by Rui Li) Project: http://git-wip-us.apache.org/repos/asf/hive/repo Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/4d9810af Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/4d9810af Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/4d9810af Branch: refs/heads/master Commit: 4d9810afba996eadb506864ef0bd0d5e3617186d Parents: 94ed34f Author: Jimmy Xiang <jxi...@apache.org> Authored: Fri Mar 24 09:24:59 2017 -0700 Committer: Jimmy Xiang <jxi...@apache.org> Committed: Mon Mar 27 09:31:32 2017 -0700 ---------------------------------------------------------------------- .../org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java | 4 ++-- .../java/org/apache/hive/spark/client/SparkClientUtilities.java | 3 ++- 2 files changed, 4 insertions(+), 3 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hive/blob/4d9810af/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java ---------------------------------------------------------------------- diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java index 7d18c0a..eb9883a 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java @@ -78,7 +78,7 @@ public class SparkUtilities { Path localFile = new Path(source.getPath()); Path remoteFile = new Path(SessionState.get().getSparkSession().getHDFSSessionDir(), getFileName(source)); - FileSystem fileSystem = FileSystem.get(conf); + FileSystem fileSystem = FileSystem.get(remoteFile.toUri(), conf); // Overwrite if the remote file already exists. Whether the file can be added // on executor is up to spark, i.e. spark.files.overwrite fileSystem.copyFromLocalFile(false, true, localFile, remoteFile); @@ -92,7 +92,7 @@ public class SparkUtilities { String deployMode = sparkConf.contains("spark.submit.deployMode") ? sparkConf.get("spark.submit.deployMode") : null; return SparkClientUtilities.isYarnClusterMode(master, deployMode) && - !source.getScheme().equals("hdfs"); + !(source.getScheme().equals("hdfs") || source.getScheme().equals("viewfs")); } private static String getFileName(URI uri) { http://git-wip-us.apache.org/repos/asf/hive/blob/4d9810af/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientUtilities.java ---------------------------------------------------------------------- diff --git a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientUtilities.java b/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientUtilities.java index 9ef3f38..210da2a 100644 --- a/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientUtilities.java +++ b/spark-client/src/main/java/org/apache/hive/spark/client/SparkClientUtilities.java @@ -83,7 +83,8 @@ public class SparkClientUtilities { try { if (StringUtils.indexOf(path, "file:/") == 0) { url = new URL(path); - } else if (StringUtils.indexOf(path, "hdfs:/") == 0) { + } else if (StringUtils.indexOf(path, "hdfs:/") == 0 + || StringUtils.indexOf(path, "viewfs:/") == 0) { Path remoteFile = new Path(path); Path localFile = new Path(localTmpDir.getAbsolutePath() + File.separator + remoteFile.getName());