From efb99da8fb505aaeeb0d95fff99c245bd3c0a0b8 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 1 Nov 2018 13:26:47 -0700 Subject: [PATCH 1/2] [SPARK-25918][SQL] LOAD DATA LOCAL INPATH should handle a relative path --- .../org/apache/spark/sql/execution/command/tables.scala | 2 +- .../apache/spark/sql/hive/execution/SQLQuerySuite.scala | 9 ++++++++- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 871eba49dfbd0..a46e000927b1a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -393,7 +393,7 @@ object LoadDataCommand { throw new IllegalArgumentException(e) } } else { - path + new Path(pathUri) } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index dfcde8cc0d39f..b7d0a905b8011 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -24,7 +24,7 @@ import java.sql.{Date, Timestamp} import java.util.{Locale, Set} import com.google.common.io.Files -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.{FileContext, FileSystem, FsConstants, Path} import org.apache.spark.TestUtils import org.apache.spark.sql._ @@ -1987,6 +1987,13 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } + test("SPARK-25918: LOAD DATA LOCAL INPATH should handle a relative path") { + val localFS = FileContext.getLocalFSFileContext() + val r = LoadDataCommand.makeQualified( + FsConstants.LOCAL_FS_URI, localFS.getWorkingDirectory, new Path("kv1.txt")) + assert(r === new Path(s"${localFS.getWorkingDirectory}/kv1.txt")) + } + test("SPARK-25738: defaultFs can have a port") { val defaultURI = new URI("hdfs://fizz.buzz.com:8020") val r = LoadDataCommand.makeQualified(defaultURI, new Path("/foo/bar"), new Path("/flim/flam")) From 85a5864a5b6a910f3cc702d0407a5e015de2efcc Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 1 Nov 2018 17:59:34 -0700 Subject: [PATCH 2/2] Address comments --- .../org/apache/spark/sql/execution/command/tables.scala | 5 +++-- .../spark/sql/hive/execution/HiveCommandSuite.scala | 9 +++++++++ .../apache/spark/sql/hive/execution/SQLQuerySuite.scala | 9 +-------- 3 files changed, 13 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index a46e000927b1a..823dc0d5ed387 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -376,7 +376,8 @@ object LoadDataCommand { * @return qualified path object */ private[sql] def makeQualified(defaultUri: URI, workingDir: Path, path: Path): Path = { - val pathUri = if (path.isAbsolute()) path.toUri() else new Path(workingDir, path).toUri() + val newPath = new Path(workingDir, path) + val pathUri = if (path.isAbsolute()) path.toUri() else newPath.toUri() if (pathUri.getScheme == null || pathUri.getAuthority == null && defaultUri.getAuthority != null) { val scheme = if (pathUri.getScheme == null) defaultUri.getScheme else pathUri.getScheme @@ -393,7 +394,7 @@ object LoadDataCommand { throw new IllegalArgumentException(e) } } else { - new Path(pathUri) + newPath } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala index 6937e97a47dc6..9147a98c94457 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala @@ -20,11 +20,13 @@ package org.apache.spark.sql.hive.execution import java.io.File import com.google.common.io.Files +import org.apache.hadoop.fs.{FileContext, FsConstants, Path} import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} +import org.apache.spark.sql.execution.command.LoadDataCommand import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types.StructType @@ -439,4 +441,11 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto } } + test("SPARK-25918: LOAD DATA LOCAL INPATH should handle a relative path") { + val localFS = FileContext.getLocalFSFileContext() + val workingDir = localFS.getWorkingDirectory + val r = LoadDataCommand.makeQualified( + FsConstants.LOCAL_FS_URI, workingDir, new Path("kv1.txt")) + assert(r === new Path(s"$workingDir/kv1.txt")) + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index b7d0a905b8011..dfcde8cc0d39f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -24,7 +24,7 @@ import java.sql.{Date, Timestamp} import java.util.{Locale, Set} import com.google.common.io.Files -import org.apache.hadoop.fs.{FileContext, FileSystem, FsConstants, Path} +import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.TestUtils import org.apache.spark.sql._ @@ -1987,13 +1987,6 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } - test("SPARK-25918: LOAD DATA LOCAL INPATH should handle a relative path") { - val localFS = FileContext.getLocalFSFileContext() - val r = LoadDataCommand.makeQualified( - FsConstants.LOCAL_FS_URI, localFS.getWorkingDirectory, new Path("kv1.txt")) - assert(r === new Path(s"${localFS.getWorkingDirectory}/kv1.txt")) - } - test("SPARK-25738: defaultFs can have a port") { val defaultURI = new URI("hdfs://fizz.buzz.com:8020") val r = LoadDataCommand.makeQualified(defaultURI, new Path("/foo/bar"), new Path("/flim/flam"))