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..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 { - path + 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")) + } }