aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--docs/sql-programming-guide.md33
-rw-r--r--examples/src/main/java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java2
-rw-r--r--examples/src/main/python/sql/hive.py2
-rw-r--r--examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala2
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala3
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala23
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala6
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala4
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala5
9 files changed, 29 insertions, 51 deletions
diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md
index d334a86bc7..064af41965 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/`.
-<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>
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 76dd160d55..052153c9e9 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")
diff --git a/examples/src/main/python/sql/hive.py b/examples/src/main/python/sql/hive.py
index 98b48908b5..ad83fe1cf1 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 \
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 11e84c0e45..ded18dacf1 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 a6e2fa26cb..f47ec7f396 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()
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 a6da8a86c1..d593bfb4ce 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
@@ -43,8 +43,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,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
@@ -148,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`")
@@ -159,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,
"",
@@ -184,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,
"",
@@ -204,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)
@@ -227,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`")
@@ -237,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,
"",
@@ -476,7 +473,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 +481,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")
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 df640ffab9..a89a43fa1e 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
@@ -19,11 +19,11 @@ package org.apache.spark.sql.internal
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,8 +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(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")
}
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 2b945dbbe0..6fbbed1d47 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 9ed454e578..d9ddcbd57c 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
@@ -489,8 +490,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)