[
https://issues.apache.org/jira/browse/SPARK-9600?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Cheng Lian updated SPARK-9600:
------------------------------
Description:
Get a clean Spark 1.4.1 build:
{noformat}
$ git checkout v1.4.1
$ ./build/sbt -Phive -Phive-thriftserver -Phadoop-1 -Dhadoop.version=1.2.1
clean assembly/assembly
{noformat}
Stop any running local Hadoop instance and unset all Hadoop environment
variables, so that we force Spark run with local file system only:
{noformat}
$ unset HADOOP_CONF_DIR
$ unset HADOOP_PREFIX
$ unset HADOOP_LIBEXEC_DIR
$ unset HADOOP_CLASSPATH
{noformat}
In this way we also ensure that the default Hive warehouse location points to
local file system {{file:///user/hive/warehouse}}. Now we create warehouse
directories for testing:
{noformat}
$ sudo rm -rf /user # !! WARNING: IT'S /user RATHER THAN /usr !!
$ sudo mkdir -p /user/hive/{warehouse,warehouse_hive13}
$ sudo chown -R lian:staff /user
$ tree /user
/user
└── hive
├── warehouse
└── warehouse_hive13
{noformat}
Create a minimal {{hive-site.xml}}, only override the warehouse location, put
it under {{$SPARK_HOME/conf}}:
{noformat}
<?xml version="1.0"?>
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
<configuration>
<property>
<name>hive.metastore.warehouse.dir</name>
<value>file:///user/hive/warehouse_hive13</value>
</property>
</configuration>
{noformat}
Now run our test snippets with {{pyspark}}:
{noformat}
$ ./bin/pyspark
In [1]: sqlContext.range(10).coalesce(1).write.saveAsTable("ds")
{noformat}
Check warehouse directories:
{noformat}
$ tree /user
/user
└── hive
├── warehouse
│ └── ds
│ ├── _SUCCESS
│ ├── _common_metadata
│ ├── _metadata
│ └── part-r-00000-46e4b32a-5c4d-4dba-b8d6-8d30ae910dc9.gz.parquet
└── warehouse_hive13
└── ds
{noformat}
Here you may notice the weird part: we have {{ds}} under both {{warehouse}} and
{{warehouse_hive13}}, but data are only written into the former.
Now let's try HiveQl:
{noformat}
In [2]: sqlContext.range(10).coalesce(1).registerTempTable("t")
In [3]: sqlContext.sql("CREATE TABLE ds_ctas AS SELECT * FROM t")
{noformat}
Check the directories again:
{noformat}
$ tree /user
/user
└── hive
├── warehouse
│ └── ds
│ ├── _SUCCESS
│ ├── _common_metadata
│ ├── _metadata
│ └── part-r-00000-46e4b32a-5c4d-4dba-b8d6-8d30ae910dc9.gz.parquet
└── warehouse_hive13
├── ds
└── ds_ctas
├── _SUCCESS
└── part-00000
{noformat}
So HiveQl works fine. (Hive never writes Parquet summary files, so
{{_common_metadata}} and {{_metadata}} are missing in {{ds_ctas}}).
was:
Having a {{hive-site.xml}} with a non-default {{hive.metastore.warehouse.dir}}
value, Spark SQL still writes to the default warehouse location
{{/user/hive/warehouse}} when saving data source tables using
{{DataFrameWriter.saveAsTable()}}:
{noformat}
<?xml version="1.0"?>
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
<configuration>
<property>
<name>javax.jdo.option.ConnectionURL</name>
<value>jdbc:mysql://localhost/metastore_hive13_hadoop2</value>
</property>
<property>
<name>javax.jdo.option.ConnectionDriverName</name>
<value>com.mysql.jdbc.Driver</value>
</property>
<property>
<name>javax.jdo.option.ConnectionUserName</name>
<value>hive</value>
</property>
<property>
<name>javax.jdo.option.ConnectionPassword</name>
<value>password</value>
</property>
<property>
<name>hive.metastore.warehouse.dir</name>
<value>hdfs://localhost:9000/user/hive/warehouse_hive13</value>
</property>
</configuration>
{noformat}
Spark shell snippet to reproduce:
{noformat}
sqlContext.range(10).write.saveAsTable("xxx")
{noformat}
Running {{DESC EXTENDED xxx}} in Hive to check SerDe propertyies:
{noformat}
...
location:hdfs://localhost:9000/user/hive/warehouse_hive13/xxx
...
parameters:{path=hdfs://localhost:9000/user/hive/warehouse/xxx,
serialization.format=1})
...
{noformat}
We are probably using execution Hive configuration when calling
{{HiveMetastoreCatalog.hiveDefaultTableFilePath()}}.
> DataFrameWriter.saveAsTable always writes data to "/user/hive/warehouse"
> ------------------------------------------------------------------------
>
> Key: SPARK-9600
> URL: https://issues.apache.org/jira/browse/SPARK-9600
> Project: Spark
> Issue Type: Bug
> Components: SQL
> Affects Versions: 1.4.1, 1.5.0
> Reporter: Cheng Lian
> Assignee: Sudhakar Thota
> Priority: Critical
> Attachments: SPARK-9600-fl1.txt
>
>
> Get a clean Spark 1.4.1 build:
> {noformat}
> $ git checkout v1.4.1
> $ ./build/sbt -Phive -Phive-thriftserver -Phadoop-1 -Dhadoop.version=1.2.1
> clean assembly/assembly
> {noformat}
> Stop any running local Hadoop instance and unset all Hadoop environment
> variables, so that we force Spark run with local file system only:
> {noformat}
> $ unset HADOOP_CONF_DIR
> $ unset HADOOP_PREFIX
> $ unset HADOOP_LIBEXEC_DIR
> $ unset HADOOP_CLASSPATH
> {noformat}
> In this way we also ensure that the default Hive warehouse location points to
> local file system {{file:///user/hive/warehouse}}. Now we create warehouse
> directories for testing:
> {noformat}
> $ sudo rm -rf /user # !! WARNING: IT'S /user RATHER THAN /usr !!
> $ sudo mkdir -p /user/hive/{warehouse,warehouse_hive13}
> $ sudo chown -R lian:staff /user
> $ tree /user
> /user
> └── hive
> ├── warehouse
> └── warehouse_hive13
> {noformat}
> Create a minimal {{hive-site.xml}}, only override the warehouse location, put
> it under {{$SPARK_HOME/conf}}:
> {noformat}
> <?xml version="1.0"?>
> <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
> <configuration>
> <property>
> <name>hive.metastore.warehouse.dir</name>
> <value>file:///user/hive/warehouse_hive13</value>
> </property>
> </configuration>
> {noformat}
> Now run our test snippets with {{pyspark}}:
> {noformat}
> $ ./bin/pyspark
> In [1]: sqlContext.range(10).coalesce(1).write.saveAsTable("ds")
> {noformat}
> Check warehouse directories:
> {noformat}
> $ tree /user
> /user
> └── hive
> ├── warehouse
> │ └── ds
> │ ├── _SUCCESS
> │ ├── _common_metadata
> │ ├── _metadata
> │ └── part-r-00000-46e4b32a-5c4d-4dba-b8d6-8d30ae910dc9.gz.parquet
> └── warehouse_hive13
> └── ds
> {noformat}
> Here you may notice the weird part: we have {{ds}} under both {{warehouse}}
> and {{warehouse_hive13}}, but data are only written into the former.
> Now let's try HiveQl:
> {noformat}
> In [2]: sqlContext.range(10).coalesce(1).registerTempTable("t")
> In [3]: sqlContext.sql("CREATE TABLE ds_ctas AS SELECT * FROM t")
> {noformat}
> Check the directories again:
> {noformat}
> $ tree /user
> /user
> └── hive
> ├── warehouse
> │ └── ds
> │ ├── _SUCCESS
> │ ├── _common_metadata
> │ ├── _metadata
> │ └── part-r-00000-46e4b32a-5c4d-4dba-b8d6-8d30ae910dc9.gz.parquet
> └── warehouse_hive13
> ├── ds
> └── ds_ctas
> ├── _SUCCESS
> └── part-00000
> {noformat}
> So HiveQl works fine. (Hive never writes Parquet summary files, so
> {{_common_metadata}} and {{_metadata}} are missing in {{ds_ctas}}).
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]