Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
33 changes: 5 additions & 28 deletions docs/sql-programming-guide.md
Original file line number Diff line number Diff line change
Expand Up @@ -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>
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Here I am just fixing some duplication of docs that relate to this change. There were 3 identical stanzas


<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>

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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";
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Here I've just updated the example's location to match the new default.

SparkSession spark = SparkSession
.builder()
.appName("Java Spark Hive Example")
Expand Down
2 changes: 1 addition & 1 deletion examples/src/main/python/sql/hive.py
Original file line number Diff line number Diff line change
Expand Up @@ -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 \
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand All @@ -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()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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()
}
}
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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`")
Expand All @@ -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,
"",
Expand All @@ -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,
"",
Expand All @@ -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)
Expand All @@ -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`")
Expand All @@ -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,
"",
Expand Down Expand Up @@ -476,15 +473,15 @@ 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`")

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")

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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._
Expand Down Expand Up @@ -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 + "/")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

could "/" change on Windows?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think it will be a forward-slash because this will have to be a URI in any event in order to work on Windows. (The reason it's there is because the URI of a local directory will resolve with a trailing slash if the JDK knows the directory exists, otherwise it will be left as-is. The default URI won't have a trailing slash because it doesn't find the directory; it will exist at this point in the test though. Both are equally fine for Spark's purposes but that's why this exists in the test, for the record.)

} finally {
sql(s"set ${SQLConf.WAREHOUSE_PATH}=$original")
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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}

Expand Down Expand Up @@ -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)")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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)

Expand Down