Skip to content
Snippets Groups Projects
Commit 6cb8f836 authored by gatorsmile's avatar gatorsmile Committed by Reynold Xin
Browse files

[SPARK-15396][SQL][DOC] It can't connect hive metastore database

#### What changes were proposed in this pull request?
The `hive.metastore.warehouse.dir` property in hive-site.xml is deprecated since Spark 2.0.0. Users might not be able to connect to the existing metastore if they do not use the new conf parameter `spark.sql.warehouse.dir`.

This PR is to update the document and example for explaining the latest changes in the configuration of default location of database.

Below is the screenshot of the latest generated docs:

<img width="681" alt="screenshot 2016-05-20 08 38 10" src="https://cloud.githubusercontent.com/assets/11567269/15433296/a05c4ace-1e66-11e6-8d2b-73682b32e9c2.png">

<img width="789" alt="screenshot 2016-05-20 08 53 26" src="https://cloud.githubusercontent.com/assets/11567269/15433734/645dc42e-1e68-11e6-9476-effc9f8721bb.png">

<img width="789" alt="screenshot 2016-05-20 08 53 37" src="https://cloud.githubusercontent.com/assets/11567269/15433738/68569f92-1e68-11e6-83d3-ef5bb221a8d8.png">

No change is made in the R's example.

<img width="860" alt="screenshot 2016-05-20 08 54 38" src="https://cloud.githubusercontent.com/assets/11567269/15433779/965b8312-1e68-11e6-8bc4-53c88ceacde2.png">

#### How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #13225 from gatorsmile/document.
parent 223f6339
No related branches found
No related tags found
No related merge requests found
...@@ -1663,43 +1663,50 @@ Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` ( ...@@ -1663,43 +1663,50 @@ Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` (
<div data-lang="scala" markdown="1"> <div data-lang="scala" markdown="1">
When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and When working with Hive, one must instantiate `SparkSession` with Hive support, including
adds support for finding tables in the MetaStore and writing queries using HiveQL. Users who do connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined functions.
not have an existing Hive deployment can still create a `HiveContext`. When not configured by the Users who do not have an existing Hive deployment can still enable Hive support. When not configured
hive-site.xml, the context automatically creates `metastore_db` in the current directory and by the `hive-site.xml`, the context automatically creates `metastore_db` in the current directory and
creates `warehouse` directory indicated by HiveConf, which defaults to `/user/hive/warehouse`. creates a directory configured by `spark.sql.warehouse.dir`, which defaults to the directory
Note that you may need to grant write privilege on `/user/hive/warehouse` to the user who starts `spark-warehouse` in the current directory that the spark application is started. Note that
the spark application. 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.
{% highlight scala %} {% highlight scala %}
// sc is an existing SparkContext. // warehouse_location points to the default location for managed databases and tables
val sqlContext = new org.apache.spark.sql.hive.HiveContext(sc) val conf = new SparkConf().setAppName("HiveFromSpark").set("spark.sql.warehouse.dir", warehouse_location)
val spark = SparkSession.builder.config(conf).enableHiveSupport().getOrCreate()
sqlContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") spark.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)")
sqlContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") spark.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src")
// Queries are expressed in HiveQL // Queries are expressed in HiveQL
sqlContext.sql("FROM src SELECT key, value").collect().foreach(println) spark.sql("FROM src SELECT key, value").collect().foreach(println)
{% endhighlight %} {% endhighlight %}
</div> </div>
<div data-lang="java" markdown="1"> <div data-lang="java" markdown="1">
When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and When working with Hive, one must instantiate `SparkSession` with Hive support, including
adds support for finding tables in the MetaStore and writing queries using HiveQL. In addition to connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined functions.
the `sql` method a `HiveContext` also provides an `hql` method, which allows queries to be Users who do not have an existing Hive deployment can still enable Hive support. When not configured
expressed in HiveQL. 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.
{% highlight java %} {% highlight java %}
// sc is an existing JavaSparkContext. SparkSession spark = SparkSession.builder().appName("JavaSparkSQL").getOrCreate();
HiveContext sqlContext = new org.apache.spark.sql.hive.HiveContext(sc.sc);
sqlContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)"); spark.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)");
sqlContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src"); spark.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src");
// Queries are expressed in HiveQL. // Queries are expressed in HiveQL.
Row[] results = sqlContext.sql("FROM src SELECT key, value").collect(); Row[] results = spark.sql("FROM src SELECT key, value").collect();
{% endhighlight %} {% endhighlight %}
...@@ -1707,18 +1714,25 @@ Row[] results = sqlContext.sql("FROM src SELECT key, value").collect(); ...@@ -1707,18 +1714,25 @@ Row[] results = sqlContext.sql("FROM src SELECT key, value").collect();
<div data-lang="python" markdown="1"> <div data-lang="python" markdown="1">
When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and When working with Hive, one must instantiate `SparkSession` with Hive support, including
adds support for finding tables in the MetaStore and writing queries using HiveQL. 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.
{% highlight python %} {% highlight python %}
# sc is an existing SparkContext. from pyspark.sql import SparkSession
from pyspark.sql import HiveContext spark = SparkSession.builder.enableHiveSupport().getOrCreate()
sqlContext = HiveContext(sc)
sqlContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") spark.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)")
sqlContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") spark.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src")
# Queries can be expressed in HiveQL. # Queries can be expressed in HiveQL.
results = sqlContext.sql("FROM src SELECT key, value").collect() results = spark.sql("FROM src SELECT key, value").collect()
{% endhighlight %} {% endhighlight %}
......
...@@ -37,10 +37,13 @@ object HiveFromSpark { ...@@ -37,10 +37,13 @@ object HiveFromSpark {
def main(args: Array[String]) { def main(args: Array[String]) {
val sparkConf = new SparkConf().setAppName("HiveFromSpark") val sparkConf = new SparkConf().setAppName("HiveFromSpark")
// A hive context adds support for finding tables in the MetaStore and writing queries // When working with Hive, one must instantiate `SparkSession` with Hive support, including
// using HiveQL. Users who do not have an existing Hive deployment can still create a // connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined
// HiveContext. When not configured by the hive-site.xml, the context automatically // functions. Users who do not have an existing Hive deployment can still enable Hive support.
// creates metastore_db and warehouse in the current directory. // 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.
val spark = SparkSession.builder val spark = SparkSession.builder
.config(sparkConf) .config(sparkConf)
.enableHiveSupport() .enableHiveSupport()
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment