From 69f8f0544847439782a02e6ae335533932d2d628 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 6 Oct 2016 20:49:41 +0100 Subject: [PATCH 1/3] Resolve all paths without scheme as file: using standard Utils.resolveURI; a little standardization of related tests --- docs/sql-programming-guide.md | 33 +++----------- .../sql/hive/JavaSparkHiveExample.java | 2 +- examples/src/main/python/sql/hive.py | 2 +- .../examples/sql/hive/SparkHiveExample.scala | 2 +- .../apache/spark/sql/internal/SQLConf.scala | 6 +-- .../sql/execution/command/DDLSuite.scala | 45 ++++++++++--------- .../spark/sql/internal/SQLConfSuite.scala | 5 +-- 7 files changed, 36 insertions(+), 59 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index d334a86bc73d7..064af41965b70 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -904,50 +904,27 @@ access data stored in Hive. Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` (for security configuration), and `hdfs-site.xml` (for HDFS configuration) file in `conf/`. -
- -
- When working with Hive, one must instantiate `SparkSession` with Hive support, including connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined functions. Users who do not have an existing Hive deployment can still enable Hive support. When not configured by the `hive-site.xml`, the context automatically creates `metastore_db` in the current directory and creates a directory configured by `spark.sql.warehouse.dir`, which defaults to the directory -`spark-warehouse` in the current directory that the spark application is started. Note that +`spark-warehouse` in the current directory that the Spark application is started. Note that the `hive.metastore.warehouse.dir` property in `hive-site.xml` is deprecated since Spark 2.0.0. Instead, use `spark.sql.warehouse.dir` to specify the default location of database in warehouse. -You may need to grant write privilege to the user who starts the spark application. +You may need to grant write privilege to the user who starts the Spark application. +
+ +
{% include_example spark_hive scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala %}
- -When working with Hive, one must instantiate `SparkSession` with Hive support, including -connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined functions. -Users who do not have an existing Hive deployment can still enable Hive support. When not configured -by the `hive-site.xml`, the context automatically creates `metastore_db` in the current directory and -creates a directory configured by `spark.sql.warehouse.dir`, which defaults to the directory -`spark-warehouse` in the current directory that the spark application is started. Note that -the `hive.metastore.warehouse.dir` property in `hive-site.xml` is deprecated since Spark 2.0.0. -Instead, use `spark.sql.warehouse.dir` to specify the default location of database in warehouse. -You may need to grant write privilege to the user who starts the spark application. - {% include_example spark_hive java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java %}
- -When working with Hive, one must instantiate `SparkSession` with Hive support, including -connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined functions. -Users who do not have an existing Hive deployment can still enable Hive support. When not configured -by the `hive-site.xml`, the context automatically creates `metastore_db` in the current directory and -creates a directory configured by `spark.sql.warehouse.dir`, which defaults to the directory -`spark-warehouse` in the current directory that the spark application is started. Note that -the `hive.metastore.warehouse.dir` property in `hive-site.xml` is deprecated since Spark 2.0.0. -Instead, use `spark.sql.warehouse.dir` to specify the default location of database in warehouse. -You may need to grant write privilege to the user who starts the spark application. - {% include_example spark_hive python/sql/hive.py %}
diff --git a/examples/src/main/java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java b/examples/src/main/java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java index 76dd160d5568b..052153c9e9736 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java @@ -56,7 +56,7 @@ public void setValue(String value) { public static void main(String[] args) { // $example on:spark_hive$ // warehouseLocation points to the default location for managed databases and tables - String warehouseLocation = "file:" + System.getProperty("user.dir") + "spark-warehouse"; + String warehouseLocation = "spark-warehouse"; SparkSession spark = SparkSession .builder() .appName("Java Spark Hive Example") diff --git a/examples/src/main/python/sql/hive.py b/examples/src/main/python/sql/hive.py index 98b48908b5a12..ad83fe1cf14b5 100644 --- a/examples/src/main/python/sql/hive.py +++ b/examples/src/main/python/sql/hive.py @@ -34,7 +34,7 @@ if __name__ == "__main__": # $example on:spark_hive$ # warehouse_location points to the default location for managed databases and tables - warehouse_location = 'file:${system:user.dir}/spark-warehouse' + warehouse_location = 'spark-warehouse' spark = SparkSession \ .builder \ diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala index 11e84c0e45632..ded18dacf1fe3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala @@ -38,7 +38,7 @@ object SparkHiveExample { // $example on:spark_hive$ // warehouseLocation points to the default location for managed databases and tables - val warehouseLocation = "file:${system:user.dir}/spark-warehouse" + val warehouseLocation = "spark-warehouse" val spark = SparkSession .builder() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 8afd39d657865..3ba3270d5155c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -23,13 +23,13 @@ import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ import scala.collection.immutable -import org.apache.hadoop.fs.Path import org.apache.parquet.hadoop.ParquetOutputCommitter import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.network.util.ByteUnit import org.apache.spark.sql.catalyst.CatalystConf +import org.apache.spark.util.Utils //////////////////////////////////////////////////////////////////////////////////////////////////// // This file defines the configuration options for Spark SQL. @@ -56,7 +56,7 @@ object SQLConf { val WAREHOUSE_PATH = SQLConfigBuilder("spark.sql.warehouse.dir") .doc("The default location for managed databases and tables.") .stringConf - .createWithDefault("${system:user.dir}/spark-warehouse") + .createWithDefault("spark-warehouse") val OPTIMIZER_MAX_ITERATIONS = SQLConfigBuilder("spark.sql.optimizer.maxIterations") .internal() @@ -755,7 +755,7 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def variableSubstituteDepth: Int = getConf(VARIABLE_SUBSTITUTE_DEPTH) - def warehousePath: String = new Path(getConf(WAREHOUSE_PATH)).toString + def warehousePath: String = Utils.resolveURI(getConf(WAREHOUSE_PATH)).toString def ignoreCorruptFiles: Boolean = getConf(IGNORE_CORRUPT_FILES) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index c8b8e9ebabc75..23c87c3beb3c3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.command import java.io.File +import java.net.URI import org.apache.hadoop.fs.Path import org.scalatest.BeforeAndAfterEach @@ -43,8 +44,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { // drop all databases, tables and functions after each test spark.sessionState.catalog.reset() } finally { - val path = System.getProperty("user.dir") + "/spark-warehouse" - Utils.deleteRecursively(new File(path)) + Utils.deleteRecursively(new File("spark-warehouse")) super.afterEach() } } @@ -116,21 +116,21 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { val catalog = spark.sessionState.catalog withTempDir { tmpDir => - val path = tmpDir.toString + val path = tmpDir.getCanonicalPath // The generated temp path is not qualified. assert(!path.startsWith("file:/")) val uri = tmpDir.toURI sql(s"CREATE DATABASE db1 LOCATION '$uri'") - val pathInCatalog = new Path(catalog.getDatabaseMetadata("db1").locationUri).toUri + val pathInCatalog = URI.create(catalog.getDatabaseMetadata("db1").locationUri) assert("file" === pathInCatalog.getScheme) - val expectedPath = new Path(path).toUri + val expectedPath = URI.create(path) assert(expectedPath.getPath === pathInCatalog.getPath) withSQLConf(SQLConf.WAREHOUSE_PATH.key -> path) { sql(s"CREATE DATABASE db2") - val pathInCatalog2 = new Path(catalog.getDatabaseMetadata("db2").locationUri).toUri + val pathInCatalog2 = URI.create(catalog.getDatabaseMetadata("db2").locationUri) assert("file" === pathInCatalog2.getScheme) - val expectedPath2 = new Path(spark.sessionState.conf.warehousePath + "/" + "db2.db").toUri + val expectedPath2 = URI.create(spark.sessionState.conf.warehousePath + "db2.db") assert(expectedPath2.getPath === pathInCatalog2.getPath) } @@ -139,16 +139,16 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } } - private def makeQualifiedPath(path: String): String = { - // copy-paste from SessionCatalog - val hadoopPath = new Path(path) - val fs = hadoopPath.getFileSystem(sparkContext.hadoopConfiguration) - fs.makeQualified(hadoopPath).toString + private def stripTrailingSeparator(path: String): String = { + // Resolved local directory URIs may or may not end with a backslash depending on if + // the directory exists or not when resolved. In several instances, normalize by + // removing the separators + if (path.endsWith("/")) path.init else path } test("Create/Drop Database") { withTempDir { tmpDir => - val path = tmpDir.toString + val path = tmpDir.getCanonicalPath withSQLConf(SQLConf.WAREHOUSE_PATH.key -> path) { val catalog = spark.sessionState.catalog val databaseNames = Seq("db1", "`database`") @@ -159,7 +159,8 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { sql(s"CREATE DATABASE $dbName") val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) - val expectedLocation = makeQualifiedPath(path + "/" + s"$dbNameWithoutBackTicks.db") + val expectedLocation = + stripTrailingSeparator(Utils.resolveURI(s"$path/$dbNameWithoutBackTicks.db").toString) assert(db1 == CatalogDatabase( dbNameWithoutBackTicks, "", @@ -185,8 +186,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { sql(s"CREATE DATABASE $dbName") val db1 = catalog.getDatabaseMetadata(dbName) val expectedLocation = - makeQualifiedPath(s"${System.getProperty("user.dir")}/spark-warehouse" + - "/" + s"$dbName.db") + stripTrailingSeparator(Utils.resolveURI(s"spark-warehouse/$dbName.db").toString) assert(db1 == CatalogDatabase( dbName, "", @@ -204,13 +204,13 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { val catalog = spark.sessionState.catalog val databaseNames = Seq("db1", "`database`") withTempDir { tmpDir => - val path = new Path(tmpDir.toString).toUri.toString + val path = Utils.resolveURI(tmpDir.getCanonicalPath).toString databaseNames.foreach { dbName => try { val dbNameWithoutBackTicks = cleanIdentifier(dbName) sql(s"CREATE DATABASE $dbName Location '$path'") val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) - val expPath = makeQualifiedPath(tmpDir.toString) + val expPath = stripTrailingSeparator(path) assert(db1 == CatalogDatabase( dbNameWithoutBackTicks, "", @@ -227,7 +227,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { test("Create Database - database already exists") { withTempDir { tmpDir => - val path = tmpDir.toString + val path = tmpDir.getCanonicalPath withSQLConf(SQLConf.WAREHOUSE_PATH.key -> path) { val catalog = spark.sessionState.catalog val databaseNames = Seq("db1", "`database`") @@ -237,7 +237,8 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { val dbNameWithoutBackTicks = cleanIdentifier(dbName) sql(s"CREATE DATABASE $dbName") val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) - val expectedLocation = makeQualifiedPath(path + "/" + s"$dbNameWithoutBackTicks.db") + val expectedLocation = + stripTrailingSeparator(Utils.resolveURI(s"$path/$dbNameWithoutBackTicks.db").toString) assert(db1 == CatalogDatabase( dbNameWithoutBackTicks, "", @@ -476,7 +477,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { test("Alter/Describe Database") { withTempDir { tmpDir => - val path = tmpDir.toString + val path = tmpDir.getCanonicalPath withSQLConf(SQLConf.WAREHOUSE_PATH.key -> path) { val catalog = spark.sessionState.catalog val databaseNames = Seq("db1", "`database`") @@ -484,7 +485,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { databaseNames.foreach { dbName => try { val dbNameWithoutBackTicks = cleanIdentifier(dbName) - val location = makeQualifiedPath(path + "/" + s"$dbNameWithoutBackTicks.db") + val location = Utils.resolveURI(s"$path/$dbNameWithoutBackTicks.db").toString sql(s"CREATE DATABASE $dbName") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index df640ffab91de..4f4ef3577c841 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.internal -import org.apache.hadoop.fs.Path +import java.io.File import org.apache.spark.SparkContext import org.apache.spark.sql._ @@ -219,8 +219,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { try { // to get the default value, always unset it spark.conf.unset(SQLConf.WAREHOUSE_PATH.key) - assert(spark.sessionState.conf.warehousePath - === new Path(s"${System.getProperty("user.dir")}/spark-warehouse").toString) + assert(new File("spark-warehouse").toURI.toString === spark.sessionState.conf.warehousePath) } finally { sql(s"set ${SQLConf.WAREHOUSE_PATH}=$original") } From 5e6350ca367369c26becb1b0875e90a78e15d660 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 12 Oct 2016 11:38:31 +0100 Subject: [PATCH 2/3] Directly handle warehouse path as URI in tests too --- .../org/apache/spark/sql/hive/execution/HiveQuerySuite.scala | 4 +++- .../org/apache/spark/sql/sources/BucketedReadSuite.scala | 5 +++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 2b945dbbe03dd..6fbbed1d47e04 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive.execution import java.io.File +import java.net.URI import java.sql.Timestamp import java.util.{Locale, TimeZone} @@ -954,7 +955,8 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd .mkString("/") // Loads partition data to a temporary table to verify contents - val path = s"${sparkSession.getWarehousePath}/dynamic_part_table/$partFolder/part-00000" + val warehousePathFile = new URI(sparkSession.getWarehousePath()).getPath + val path = s"$warehousePathFile/dynamic_part_table/$partFolder/part-00000" sql("DROP TABLE IF EXISTS dp_verify") sql("CREATE TABLE dp_verify(intcol INT)") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index 3ff85176de10e..155b19991417c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.sources import java.io.File +import java.net.URI import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.catalog.BucketSpec @@ -411,8 +412,8 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet test("error if there exists any malformed bucket files") { withTable("bucketed_table") { df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") - val tableDir = new File(hiveContext - .sparkSession.getWarehousePath, "bucketed_table") + val warehouseFilePath = new URI(hiveContext.sparkSession.getWarehousePath).getPath + val tableDir = new File(warehouseFilePath, "bucketed_table") Utils.deleteRecursively(tableDir) df1.write.parquet(tableDir.getAbsolutePath) From 69c538333ba02fc806fe7e8f794f269936ebdc28 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 19 Oct 2016 12:03:44 +0100 Subject: [PATCH 3/3] Back out some of last change; only pass default path of "spark-warehouse" through Utils.resolveURI --- .../apache/spark/sql/internal/SQLConf.scala | 5 +-- .../sql/execution/command/DDLSuite.scala | 34 ++++++++----------- .../spark/sql/internal/SQLConfSuite.scala | 7 ++-- 3 files changed, 22 insertions(+), 24 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 3ba3270d5155c..4c08ddcaa6f71 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -23,6 +23,7 @@ import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ import scala.collection.immutable +import org.apache.hadoop.fs.Path import org.apache.parquet.hadoop.ParquetOutputCommitter import org.apache.spark.internal.Logging @@ -56,7 +57,7 @@ object SQLConf { val WAREHOUSE_PATH = SQLConfigBuilder("spark.sql.warehouse.dir") .doc("The default location for managed databases and tables.") .stringConf - .createWithDefault("spark-warehouse") + .createWithDefault(Utils.resolveURI("spark-warehouse").toString) val OPTIMIZER_MAX_ITERATIONS = SQLConfigBuilder("spark.sql.optimizer.maxIterations") .internal() @@ -755,7 +756,7 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def variableSubstituteDepth: Int = getConf(VARIABLE_SUBSTITUTE_DEPTH) - def warehousePath: String = Utils.resolveURI(getConf(WAREHOUSE_PATH)).toString + def warehousePath: String = new Path(getConf(WAREHOUSE_PATH)).toString def ignoreCorruptFiles: Boolean = getConf(IGNORE_CORRUPT_FILES) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 23c87c3beb3c3..a02fcaccd51f3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.command import java.io.File -import java.net.URI import org.apache.hadoop.fs.Path import org.scalatest.BeforeAndAfterEach @@ -121,16 +120,16 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { assert(!path.startsWith("file:/")) val uri = tmpDir.toURI sql(s"CREATE DATABASE db1 LOCATION '$uri'") - val pathInCatalog = URI.create(catalog.getDatabaseMetadata("db1").locationUri) + val pathInCatalog = new Path(catalog.getDatabaseMetadata("db1").locationUri).toUri assert("file" === pathInCatalog.getScheme) - val expectedPath = URI.create(path) + val expectedPath = new Path(path).toUri assert(expectedPath.getPath === pathInCatalog.getPath) withSQLConf(SQLConf.WAREHOUSE_PATH.key -> path) { sql(s"CREATE DATABASE db2") - val pathInCatalog2 = URI.create(catalog.getDatabaseMetadata("db2").locationUri) + val pathInCatalog2 = new Path(catalog.getDatabaseMetadata("db2").locationUri).toUri assert("file" === pathInCatalog2.getScheme) - val expectedPath2 = URI.create(spark.sessionState.conf.warehousePath + "db2.db") + val expectedPath2 = new Path(spark.sessionState.conf.warehousePath + "/" + "db2.db").toUri assert(expectedPath2.getPath === pathInCatalog2.getPath) } @@ -139,11 +138,11 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } } - private def stripTrailingSeparator(path: String): String = { - // Resolved local directory URIs may or may not end with a backslash depending on if - // the directory exists or not when resolved. In several instances, normalize by - // removing the separators - if (path.endsWith("/")) path.init else path + private def makeQualifiedPath(path: String): String = { + // copy-paste from SessionCatalog + val hadoopPath = new Path(path) + val fs = hadoopPath.getFileSystem(sparkContext.hadoopConfiguration) + fs.makeQualified(hadoopPath).toString } test("Create/Drop Database") { @@ -159,8 +158,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { sql(s"CREATE DATABASE $dbName") val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) - val expectedLocation = - stripTrailingSeparator(Utils.resolveURI(s"$path/$dbNameWithoutBackTicks.db").toString) + val expectedLocation = makeQualifiedPath(s"$path/$dbNameWithoutBackTicks.db") assert(db1 == CatalogDatabase( dbNameWithoutBackTicks, "", @@ -185,8 +183,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { try { sql(s"CREATE DATABASE $dbName") val db1 = catalog.getDatabaseMetadata(dbName) - val expectedLocation = - stripTrailingSeparator(Utils.resolveURI(s"spark-warehouse/$dbName.db").toString) + val expectedLocation = makeQualifiedPath(s"spark-warehouse/$dbName.db") assert(db1 == CatalogDatabase( dbName, "", @@ -204,13 +201,13 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { val catalog = spark.sessionState.catalog val databaseNames = Seq("db1", "`database`") withTempDir { tmpDir => - val path = Utils.resolveURI(tmpDir.getCanonicalPath).toString + val path = new Path(tmpDir.getCanonicalPath).toUri databaseNames.foreach { dbName => try { val dbNameWithoutBackTicks = cleanIdentifier(dbName) sql(s"CREATE DATABASE $dbName Location '$path'") val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) - val expPath = stripTrailingSeparator(path) + val expPath = makeQualifiedPath(tmpDir.toString) assert(db1 == CatalogDatabase( dbNameWithoutBackTicks, "", @@ -237,8 +234,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { val dbNameWithoutBackTicks = cleanIdentifier(dbName) sql(s"CREATE DATABASE $dbName") val db1 = catalog.getDatabaseMetadata(dbNameWithoutBackTicks) - val expectedLocation = - stripTrailingSeparator(Utils.resolveURI(s"$path/$dbNameWithoutBackTicks.db").toString) + val expectedLocation = makeQualifiedPath(s"$path/$dbNameWithoutBackTicks.db") assert(db1 == CatalogDatabase( dbNameWithoutBackTicks, "", @@ -485,7 +481,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { databaseNames.foreach { dbName => try { val dbNameWithoutBackTicks = cleanIdentifier(dbName) - val location = Utils.resolveURI(s"$path/$dbNameWithoutBackTicks.db").toString + val location = makeQualifiedPath(s"$path/$dbNameWithoutBackTicks.db") sql(s"CREATE DATABASE $dbName") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 4f4ef3577c841..a89a43fa1e777 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -17,13 +17,13 @@ package org.apache.spark.sql.internal -import java.io.File +import org.apache.hadoop.fs.Path -import org.apache.spark.SparkContext import org.apache.spark.sql._ import org.apache.spark.sql.execution.WholeStageCodegenExec import org.apache.spark.sql.internal.StaticSQLConf._ import org.apache.spark.sql.test.{SharedSQLContext, TestSQLContext} +import org.apache.spark.util.Utils class SQLConfSuite extends QueryTest with SharedSQLContext { import testImplicits._ @@ -219,7 +219,8 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { try { // to get the default value, always unset it spark.conf.unset(SQLConf.WAREHOUSE_PATH.key) - assert(new File("spark-warehouse").toURI.toString === spark.sessionState.conf.warehousePath) + assert(new Path(Utils.resolveURI("spark-warehouse")).toString === + spark.sessionState.conf.warehousePath + "/") } finally { sql(s"set ${SQLConf.WAREHOUSE_PATH}=$original") }