Repository: spark
Updated Branches:
  refs/heads/branch-2.0 00a2e01e8 -> 064db176d


[SPARK-17810][SQL] Default spark.sql.warehouse.dir is relative to local FS but 
can resolve as HDFS path

Always resolve spark.sql.warehouse.dir as a local path, and as relative to 
working dir not home dir

Existing tests.

Author: Sean Owen <[email protected]>

Closes #15382 from srowen/SPARK-17810.

(cherry picked from commit 4ecbe1b92f4c4c5b2d734895c09d8ded0ed48d4d)
Signed-off-by: Sean Owen <[email protected]>


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

Branch: refs/heads/branch-2.0
Commit: 064db176d2f7e69fd280b14c5ef13a905180051e
Parents: 00a2e01
Author: Sean Owen <[email protected]>
Authored: Mon Oct 24 10:44:45 2016 +0100
Committer: Sean Owen <[email protected]>
Committed: Mon Oct 24 11:19:52 2016 +0100

----------------------------------------------------------------------
 docs/sql-programming-guide.md                   | 33 +++-----------------
 .../examples/sql/hive/JavaSparkHiveExample.java |  2 +-
 examples/src/main/python/sql/hive.py            |  2 +-
 .../examples/sql/hive/SparkHiveExample.scala    |  2 +-
 .../org/apache/spark/sql/internal/SQLConf.scala |  3 +-
 .../spark/sql/execution/command/DDLSuite.scala  | 20 ++++++------
 .../spark/sql/internal/SQLConfSuite.scala       |  5 +--
 7 files changed, 22 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/064db176/docs/sql-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md
index 3a90323..bac0e81 100644
--- a/docs/sql-programming-guide.md
+++ b/docs/sql-programming-guide.md
@@ -857,50 +857,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/`.
 
-<div class="codetabs">
-
-<div data-lang="scala"  markdown="1">
-
 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.
 
+<div class="codetabs">
+
+<div data-lang="scala"  markdown="1">
 {% include_example spark_hive 
scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala %}
 </div>
 
 <div data-lang="java"  markdown="1">
-
-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 %}
 </div>
 
 <div data-lang="python"  markdown="1">
-
-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 %}
 </div>
 

http://git-wip-us.apache.org/repos/asf/spark/blob/064db176/examples/src/main/java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java
----------------------------------------------------------------------
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 76dd160..052153c 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 class JavaSparkHiveExample {
   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")

http://git-wip-us.apache.org/repos/asf/spark/blob/064db176/examples/src/main/python/sql/hive.py
----------------------------------------------------------------------
diff --git a/examples/src/main/python/sql/hive.py 
b/examples/src/main/python/sql/hive.py
index 9b2a2c4..142213d 100644
--- a/examples/src/main/python/sql/hive.py
+++ b/examples/src/main/python/sql/hive.py
@@ -34,7 +34,7 @@ Run with:
 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 \

http://git-wip-us.apache.org/repos/asf/spark/blob/064db176/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala
----------------------------------------------------------------------
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 11e84c0..ded18da 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()

http://git-wip-us.apache.org/repos/asf/spark/blob/064db176/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
----------------------------------------------------------------------
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 8ba87c8..7598d47 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
@@ -30,6 +30,7 @@ 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 +57,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(Utils.resolveURI("spark-warehouse").toString)
 
   val OPTIMIZER_MAX_ITERATIONS = 
SQLConfigBuilder("spark.sql.optimizer.maxIterations")
     .internal()

http://git-wip-us.apache.org/repos/asf/spark/blob/064db176/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
----------------------------------------------------------------------
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 1f5492e..caa2fca 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
@@ -115,7 +115,7 @@ 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
@@ -147,7 +147,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with 
BeforeAndAfterEach {
 
   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`")
@@ -158,7 +158,7 @@ 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 = 
makeQualifiedPath(s"$path/$dbNameWithoutBackTicks.db")
             assert(db1 == CatalogDatabase(
               dbNameWithoutBackTicks,
               "",
@@ -183,9 +183,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with 
BeforeAndAfterEach {
       try {
         sql(s"CREATE DATABASE $dbName")
         val db1 = catalog.getDatabaseMetadata(dbName)
-        val expectedLocation =
-          
makeQualifiedPath(s"${System.getProperty("user.dir")}/spark-warehouse" +
-            "/" + s"$dbName.db")
+        val expectedLocation = makeQualifiedPath(s"spark-warehouse/$dbName.db")
         assert(db1 == CatalogDatabase(
           dbName,
           "",
@@ -203,7 +201,7 @@ 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 = new Path(tmpDir.getCanonicalPath).toUri
       databaseNames.foreach { dbName =>
         try {
           val dbNameWithoutBackTicks = cleanIdentifier(dbName)
@@ -226,7 +224,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`")
@@ -236,7 +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 = makeQualifiedPath(path + "/" + 
s"$dbNameWithoutBackTicks.db")
+            val expectedLocation = 
makeQualifiedPath(s"$path/$dbNameWithoutBackTicks.db")
             assert(db1 == CatalogDatabase(
               dbNameWithoutBackTicks,
               "",
@@ -269,7 +267,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`")
@@ -277,7 +275,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 = 
makeQualifiedPath(s"$path/$dbNameWithoutBackTicks.db")
 
             sql(s"CREATE DATABASE $dbName")
 

http://git-wip-us.apache.org/repos/asf/spark/blob/064db176/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala
----------------------------------------------------------------------
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 761bbe3..a230344 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
@@ -22,6 +22,7 @@ import org.apache.hadoop.fs.Path
 import org.apache.spark.sql.{QueryTest, Row, SparkSession, SQLContext}
 import org.apache.spark.sql.execution.WholeStageCodegenExec
 import org.apache.spark.sql.test.{SharedSQLContext, TestSQLContext}
+import org.apache.spark.util.Utils
 
 class SQLConfSuite extends QueryTest with SharedSQLContext {
   private val testKey = "test.key.0"
@@ -215,8 +216,8 @@ 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 Path(Utils.resolveURI("spark-warehouse")).toString ===
+        spark.sessionState.conf.warehousePath + "/")
     } finally {
       sql(s"set ${SQLConf.WAREHOUSE_PATH}=$original")
     }


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to