Commit graph

1446 commits

Author SHA1 Message Date
Dongjoon Hyun d0d28507ca [SPARK-16286][SQL] Implement stack table generating function
## What changes were proposed in this pull request?

This PR implements `stack` table generating function.

## How was this patch tested?

Pass the Jenkins tests including new testcases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14033 from dongjoon-hyun/SPARK-16286.
2016-07-06 10:54:43 +08:00
Dongjoon Hyun 4db63fd2b4 [SPARK-16383][SQL] Remove SessionState.executeSql
## What changes were proposed in this pull request?

This PR removes `SessionState.executeSql` in favor of `SparkSession.sql`. We can remove this safely since the visibility `SessionState` is `private[sql]` and `executeSql` is only used in one **ignored** test, `test("Multiple Hive Instances")`.

## How was this patch tested?

Pass the Jenkins tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14055 from dongjoon-hyun/SPARK-16383.
2016-07-05 16:47:32 -07:00
Reynold Xin 5b7a1770ac [HOTFIX] Fix build break. 2016-07-05 12:06:41 -07:00
Reynold Xin 16a2a7d714 [SPARK-16311][SQL] Metadata refresh should work on temporary views
## What changes were proposed in this pull request?
This patch fixes the bug that the refresh command does not work on temporary views. This patch is based on https://github.com/apache/spark/pull/13989, but removes the public Dataset.refresh() API as well as improved test coverage.

Note that I actually think the public refresh() API is very useful. We can in the future implement it by also invalidating the lazy vals in QueryExecution (or alternatively just create a new QueryExecution).

## How was this patch tested?
Re-enabled a previously ignored test, and added a new test suite for Hive testing behavior of temporary views against MetastoreRelation.

Author: Reynold Xin <rxin@databricks.com>
Author: petermaxlee <petermaxlee@gmail.com>

Closes #14009 from rxin/SPARK-16311.
2016-07-05 11:36:05 -07:00
hyukjinkwon 7742d9f158 [SPARK-15198][SQL] Support for pushing down filters for boolean types in ORC data source
## What changes were proposed in this pull request?

It seems ORC supports all the types in  ([`PredicateLeaf.Type`](e085b7e9bd/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/PredicateLeaf.java (L50-L56))) which includes boolean types. So, this was tested first.

This PR adds the support for pushing filters down for `BooleanType` in ORC data source.

This PR also removes `OrcTableScan` class and the companion object, which is not used anymore.

## How was this patch tested?

Unittest in `OrcFilterSuite` and `OrcQuerySuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #12972 from HyukjinKwon/SPARK-15198.
2016-07-05 13:59:13 +08:00
Michael Allman 8f6cf00c69 [SPARK-15968][SQL] Nonempty partitioned metastore tables are not cached
(Please note this is a revision of PR #13686, which has been closed in favor of this PR.)

This PR addresses [SPARK-15968](https://issues.apache.org/jira/browse/SPARK-15968).

## What changes were proposed in this pull request?

The `getCached` method of [HiveMetastoreCatalog](https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala) computes `pathsInMetastore` from the metastore relation's catalog table. This only returns the table base path, which is incomplete/inaccurate for a nonempty partitioned table. As a result, cached lookups on nonempty partitioned tables always miss.

Rather than get `pathsInMetastore` from

    metastoreRelation.catalogTable.storage.locationUri.toSeq

I modified the `getCached` method to take a `pathsInMetastore` argument. Calls to this method pass in the paths computed from calls to the Hive metastore. This is how `getCached` was implemented in Spark 1.5:

e0c3212a9b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala (L444).

I also added a call in `InsertIntoHiveTable.scala` to invalidate the table from the SQL session catalog.

## How was this patch tested?

I've added a new unit test to `parquetSuites.scala`:

    SPARK-15968: nonempty partitioned metastore Parquet table lookup should use cached relation

Note that the only difference between this new test and the one above it in the file is that the new test populates its partitioned table with a single value, while the existing test leaves the table empty. This reveals a subtle, unexpected hole in test coverage present before this patch.

Note I also modified a different but related unit test in `parquetSuites.scala`:

    SPARK-15248: explicitly added partitions should be readable

This unit test asserts that Spark SQL should return data from a table partition which has been placed there outside a metastore query immediately after it is added. I changed the test so that, instead of adding the data as a parquet file saved in the partition's location, the data is added through a SQL `INSERT` query. I made this change because I could find no way to efficiently support partitioned table caching without failing that test.

In addition to my primary motivation, I can offer a few reasons I believe this is an acceptable weakening of that test. First, it still validates a fix for [SPARK-15248](https://issues.apache.org/jira/browse/SPARK-15248), the issue for which it was written. Second, the assertion made is stronger than that required for non-partitioned tables. If you write data to the storage location of a non-partitioned metastore table without using a proper SQL DML query, a subsequent call to show that data will not return it. I believe this is an intentional limitation put in place to make table caching feasible, but I'm only speculating.

Building a large `HadoopFsRelation` requires `stat`-ing all of its data files. In our environment, where we have tables with 10's of thousands of partitions, the difference between using a cached relation versus a new one is a matter of seconds versus minutes. Caching partitioned table metadata vastly improves the usability of Spark SQL for these cases.

Thanks.

Author: Michael Allman <michael@videoamp.com>

Closes #13818 from mallman/spark-15968.
2016-07-05 09:49:25 +08:00
gatorsmile 2628333978 [SPARK-16358][SQL] Remove InsertIntoHiveTable From Logical Plan
#### What changes were proposed in this pull request?
LogicalPlan `InsertIntoHiveTable` is useless. Thus, we can remove it from the code base.

#### How was this patch tested?
The existing test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14037 from gatorsmile/InsertIntoHiveTable.
2016-07-04 13:45:07 +08:00
Dongjoon Hyun 88134e7368 [SPARK-16288][SQL] Implement inline table generating function
## What changes were proposed in this pull request?

This PR implements `inline` table generating function.

## How was this patch tested?

Pass the Jenkins tests with new testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13976 from dongjoon-hyun/SPARK-16288.
2016-07-04 01:57:45 +08:00
Dongjoon Hyun 54b27c1797 [SPARK-16278][SPARK-16279][SQL] Implement map_keys/map_values SQL functions
## What changes were proposed in this pull request?

This PR adds `map_keys` and `map_values` SQL functions in order to remove Hive fallback.

## How was this patch tested?

Pass the Jenkins tests including new testcases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13967 from dongjoon-hyun/SPARK-16278.
2016-07-03 16:59:40 +08:00
Reynold Xin 38f4d6f44e [SPARK-15954][SQL] Disable loading test tables in Python tests
## What changes were proposed in this pull request?
This patch introduces a flag to disable loading test tables in TestHiveSparkSession and disables that in Python. This fixes an issue in which python/run-tests would fail due to failure to load test tables.

Note that these test tables are not used outside of HiveCompatibilitySuite. In the long run we should probably decouple the loading of test tables from the test Hive setup.

## How was this patch tested?
This is a test only change.

Author: Reynold Xin <rxin@databricks.com>

Closes #14005 from rxin/SPARK-15954.
2016-06-30 19:02:35 -07:00
petermaxlee 85f2303eca [SPARK-16276][SQL] Implement elt SQL function
## What changes were proposed in this pull request?
This patch implements the elt function, as it is implemented in Hive.

## How was this patch tested?
Added expression unit test in StringExpressionsSuite and end-to-end test in StringFunctionsSuite.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #13966 from petermaxlee/SPARK-16276.
2016-07-01 07:57:48 +08:00
Dongjoon Hyun 46395db80e [SPARK-16289][SQL] Implement posexplode table generating function
## What changes were proposed in this pull request?

This PR implements `posexplode` table generating function. Currently, master branch raises the following exception for `map` argument. It's different from Hive.

**Before**
```scala
scala> sql("select posexplode(map('a', 1, 'b', 2))").show
org.apache.spark.sql.AnalysisException: No handler for Hive UDF ... posexplode() takes an array as a parameter; line 1 pos 7
```

**After**
```scala
scala> sql("select posexplode(map('a', 1, 'b', 2))").show
+---+---+-----+
|pos|key|value|
+---+---+-----+
|  0|  a|    1|
|  1|  b|    2|
+---+---+-----+
```

For `array` argument, `after` is the same with `before`.
```
scala> sql("select posexplode(array(1, 2, 3))").show
+---+---+
|pos|col|
+---+---+
|  0|  1|
|  1|  2|
|  2|  3|
+---+---+
```

## How was this patch tested?

Pass the Jenkins tests with newly added testcases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13971 from dongjoon-hyun/SPARK-16289.
2016-06-30 12:03:54 -07:00
Sital Kedia 07f46afc73 [SPARK-13850] Force the sorter to Spill when number of elements in th…
## What changes were proposed in this pull request?

Force the sorter to Spill when number of elements in the pointer array reach a certain size. This is to workaround the issue of timSort failing on large buffer size.

## How was this patch tested?

Tested by running a job which was failing without this change due to TimSort bug.

Author: Sital Kedia <skedia@fb.com>

Closes #13107 from sitalkedia/fix_TimSort.
2016-06-30 10:53:18 -07:00
petermaxlee d3af6731fa [SPARK-16274][SQL] Implement xpath_boolean
## What changes were proposed in this pull request?
This patch implements xpath_boolean expression for Spark SQL, a xpath function that returns true or false. The implementation is modelled after Hive's xpath_boolean, except that how the expression handles null inputs. Hive throws a NullPointerException at runtime if either of the input is null. This implementation returns null if either of the input is null.

## How was this patch tested?
Created two new test suites. One for unit tests covering the expression, and the other for end-to-end test in SQL.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #13964 from petermaxlee/SPARK-16274.
2016-06-30 09:27:48 +08:00
Dongjoon Hyun 831a04f5d1 [SPARK-16267][TEST] Replace deprecated CREATE TEMPORARY TABLE ... USING from testsuites.
## What changes were proposed in this pull request?

After SPARK-15674, `DDLStrategy` prints out the following deprecation messages in the testsuites.

```
12:10:53.284 WARN org.apache.spark.sql.execution.SparkStrategies$DDLStrategy:
CREATE TEMPORARY TABLE normal_orc_source USING... is deprecated,
please use CREATE TEMPORARY VIEW viewName USING... instead
```

Total : 40
- JDBCWriteSuite: 14
- DDLSuite: 6
- TableScanSuite: 6
- ParquetSourceSuite: 5
- OrcSourceSuite: 2
- SQLQuerySuite: 2
- HiveCommandSuite: 2
- JsonSuite: 1
- PrunedScanSuite: 1
- FilteredScanSuite  1

This PR replaces `CREATE TEMPORARY TABLE` with `CREATE TEMPORARY VIEW` in order to remove the deprecation messages in the above testsuites except `DDLSuite`, `SQLQuerySuite`, `HiveCommandSuite`.

The Jenkins results shows only remaining 10 messages.

https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/61422/consoleFull

## How was this patch tested?

This is a testsuite-only change.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13956 from dongjoon-hyun/SPARK-16267.
2016-06-29 17:29:17 -07:00
Dongjoon Hyun 2eaabfa414 [SPARK-16228][SQL] HiveSessionCatalog should return double-param functions for decimal param lookups
## What changes were proposed in this pull request?

This PR supports a fallback lookup by casting `DecimalType` into `DoubleType` for the external functions with `double`-type parameter.

**Reported Error Scenarios**
```scala
scala> sql("select percentile(value, 0.5) from values 1,2,3 T(value)")
org.apache.spark.sql.AnalysisException: ... No matching method for class org.apache.hadoop.hive.ql.udf.UDAFPercentile with (int, decimal(38,18)). Possible choices: _FUNC_(bigint, array<double>)  _FUNC_(bigint, double)  ; line 1 pos 7

scala> sql("select percentile_approx(value, 0.5) from values 1.0,2.0,3.0 T(value)")
org.apache.spark.sql.AnalysisException: ... Only a float/double or float/double array argument is accepted as parameter 2, but decimal(38,18) was passed instead.; line 1 pos 7
```

## How was this patch tested?

Pass the Jenkins tests (including a new testcase).

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13930 from dongjoon-hyun/SPARK-16228.
2016-06-29 16:08:10 -07:00
gatorsmile 7ee9e39cb4 [SPARK-16157][SQL] Add New Methods for comments in StructField and StructType
#### What changes were proposed in this pull request?
Based on the previous discussion with cloud-fan hvanhovell in another related PR https://github.com/apache/spark/pull/13764#discussion_r67994276, it looks reasonable to add convenience methods for users to add `comment` when defining `StructField`.

Currently, the column-related `comment` attribute is stored in `Metadata` of `StructField`. For example, users can add the `comment` attribute using the following way:
```Scala
StructType(
  StructField(
    "cl1",
    IntegerType,
    nullable = false,
    new MetadataBuilder().putString("comment", "test").build()) :: Nil)
```
This PR is to add more user friendly methods for the `comment` attribute when defining a `StructField`. After the changes, users are provided three different ways to do it:
```Scala
val struct = (new StructType)
  .add("a", "int", true, "test1")

val struct = (new StructType)
  .add("c", StringType, true, "test3")

val struct = (new StructType)
  .add(StructField("d", StringType).withComment("test4"))
```

#### How was this patch tested?
Added test cases:
- `DataTypeSuite` is for testing three types of API changes,
- `DataFrameReaderWriterSuite` is for parquet, json and csv formats - using in-memory catalog
- `OrcQuerySuite.scala` is for orc format using Hive-metastore

Author: gatorsmile <gatorsmile@gmail.com>

Closes #13860 from gatorsmile/newMethodForComment.
2016-06-29 19:36:21 +08:00
Reynold Xin 363bcedeea [SPARK-16248][SQL] Whitelist the list of Hive fallback functions
## What changes were proposed in this pull request?
This patch removes the blind fallback into Hive for functions. Instead, it creates a whitelist and adds only a small number of functions to the whitelist, i.e. the ones we intend to support in the long run in Spark.

## How was this patch tested?
Updated tests to reflect the change.

Author: Reynold Xin <rxin@databricks.com>

Closes #13939 from rxin/hive-whitelist.
2016-06-28 19:36:53 -07:00
Bill Chambers c48c8ebc0a [SPARK-16220][SQL] Revert Change to Bring Back SHOW FUNCTIONS Functionality
## What changes were proposed in this pull request?

- Fix tests regarding show functions functionality
- Revert `catalog.ListFunctions` and `SHOW FUNCTIONS` to return to `Spark 1.X` functionality.

Cherry picked changes from this PR: https://github.com/apache/spark/pull/13413/files

## How was this patch tested?

Unit tests.

Author: Bill Chambers <bill@databricks.com>
Author: Bill Chambers <wchambers@ischool.berkeley.edu>

Closes #13916 from anabranch/master.
2016-06-27 11:50:34 -07:00
Cheng Lian 2d2f607bfa [SPARK-13709][SQL] Initialize deserializer with both table and partition properties when reading partitioned tables
## What changes were proposed in this pull request?

When reading partitions of a partitioned Hive SerDe table, we only initializes the deserializer using partition properties. However, for SerDes like `AvroSerDe`, essential properties (e.g. Avro schema information) may be defined in table properties. We should merge both table properties and partition properties before initializing the deserializer.

Note that an individual partition may have different properties than the one defined in the table properties (e.g. partitions within a table can have different SerDes). Thus, for any property key defined in both partition and table properties, the value set in partition properties wins.

## How was this patch tested?

New test case added in `QueryPartitionSuite`.

Author: Cheng Lian <lian@databricks.com>

Closes #13865 from liancheng/spark-13709-partitioned-avro-table.
2016-06-23 23:11:46 -07:00
Brian Cho 4374a46bfc [SPARK-16162] Remove dead code OrcTableScan.
## What changes were proposed in this pull request?

SPARK-14535 removed all calls to class OrcTableScan. This removes the dead code.

## How was this patch tested?

Existing unit tests.

Author: Brian Cho <bcho@fb.com>

Closes #13869 from dafrista/clean-up-orctablescan.
2016-06-22 22:37:50 -07:00
gatorsmile 9f990fa3f9 [SPARK-16024][SQL][TEST] Verify Column Comment for Data Source Tables
#### What changes were proposed in this pull request?
This PR is to improve test coverage. It verifies whether `Comment` of `Column` can be appropriate handled.

The test cases verify the related parts in Parser, both SQL and DataFrameWriter interface, and both Hive Metastore catalog and In-memory catalog.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #13764 from gatorsmile/dataSourceComment.
2016-06-23 09:12:20 +08:00
Brian Cho 4f869f88ee [SPARK-15956][SQL] When unwrapping ORC avoid pattern matching at runtime
## What changes were proposed in this pull request?

Extend the returning of unwrapper functions from primitive types to all types.

This PR is based on https://github.com/apache/spark/pull/13676. It only fixes a bug with scala-2.10 compilation. All credit should go to dafrista.

## How was this patch tested?

The patch should pass all unit tests. Reading ORC files with non-primitive types with this change reduced the read time by ~15%.

Author: Brian Cho <bcho@fb.com>
Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #13854 from hvanhovell/SPARK-15956-scala210.
2016-06-22 16:56:55 -07:00
Herman van Hovell 472d611a70 [SPARK-15956][SQL] Revert "[] When unwrapping ORC avoid pattern matching…
This reverts commit 0a9c027595. It breaks the 2.10 build, I'll fix this in a different PR.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #13853 from hvanhovell/SPARK-15956-revert.
2016-06-22 11:36:32 -07:00
Brian Cho 0a9c027595 [SPARK-15956][SQL] When unwrapping ORC avoid pattern matching at runtime
## What changes were proposed in this pull request?

Extend the returning of unwrapper functions from primitive types to all types.

## How was this patch tested?

The patch should pass all unit tests. Reading ORC files with non-primitive types with this change reduced the read time by ~15%.

===

The github diff is very noisy. Attaching the screenshots below for improved readability:

![screen shot 2016-06-14 at 5 33 16 pm](https://cloud.githubusercontent.com/assets/1514239/16064580/4d6f7a98-3257-11e6-9172-65e4baff948b.png)

![screen shot 2016-06-14 at 5 33 28 pm](https://cloud.githubusercontent.com/assets/1514239/16064587/5ae6c244-3257-11e6-8460-69eee70de219.png)

Author: Brian Cho <bcho@fb.com>

Closes #13676 from dafrista/improve-orc-master.
2016-06-22 10:38:42 -07:00
Cheng Lian f4a3d45e38 [SPARK-16037][SQL] Follow-up: add DataFrameWriter.insertInto() test cases for by position resolution
## What changes were proposed in this pull request?

This PR migrates some test cases introduced in #12313 as a follow-up of #13754 and #13766. These test cases cover `DataFrameWriter.insertInto()`, while the former two only cover SQL `INSERT` statements.

Note that the `testPartitionedTable` utility method tests both Hive SerDe tables and data source tables.

## How was this patch tested?

N/A

Author: Cheng Lian <lian@databricks.com>

Closes #13810 from liancheng/spark-16037-follow-up-tests.
2016-06-21 11:58:33 -07:00
Yin Huai 905f774b71 [SPARK-16030][SQL] Allow specifying static partitions when inserting to data source tables
## What changes were proposed in this pull request?
This PR adds the static partition support to INSERT statement when the target table is a data source table.

## How was this patch tested?
New tests in InsertIntoHiveTableSuite and DataSourceAnalysisSuite.

**Note: This PR is based on https://github.com/apache/spark/pull/13766. The last commit is the actual change.**

Author: Yin Huai <yhuai@databricks.com>

Closes #13769 from yhuai/SPARK-16030-1.
2016-06-20 20:17:47 +08:00
Yin Huai 6d0f921aed [SPARK-16036][SPARK-16037][SPARK-16034][SQL] Follow up code clean up and improvement
## What changes were proposed in this pull request?
This PR is the follow-up PR for https://github.com/apache/spark/pull/13754/files and https://github.com/apache/spark/pull/13749. I will comment inline to explain my changes.

## How was this patch tested?
Existing tests.

Author: Yin Huai <yhuai@databricks.com>

Closes #13766 from yhuai/caseSensitivity.
2016-06-19 21:45:53 -07:00
Wenchen Fan 3d010c8375 [SPARK-16036][SPARK-16037][SQL] fix various table insertion problems
## What changes were proposed in this pull request?

The current table insertion has some weird behaviours:

1. inserting into a partitioned table with mismatch columns has confusing error message for hive table, and wrong result for datasource table
2. inserting into a partitioned table without partition list has wrong result for hive table.

This PR fixes these 2 problems.

## How was this patch tested?

new test in hive `SQLQuerySuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #13754 from cloud-fan/insert2.
2016-06-18 10:32:27 -07:00
Cheng Lian 10b671447b [SPARK-16033][SQL] insertInto() can't be used together with partitionBy()
## What changes were proposed in this pull request?

When inserting into an existing partitioned table, partitioning columns should always be determined by catalog metadata of the existing table to be inserted. Extra `partitionBy()` calls don't make sense, and mess up existing data because newly inserted data may have wrong partitioning directory layout.

## How was this patch tested?

New test case added in `InsertIntoHiveTableSuite`.

Author: Cheng Lian <lian@databricks.com>

Closes #13747 from liancheng/spark-16033-insert-into-without-partition-by.
2016-06-17 20:13:04 -07:00
gatorsmile e5d703bca8 [SPARK-15706][SQL] Fix Wrong Answer when using IF NOT EXISTS in INSERT OVERWRITE for DYNAMIC PARTITION
#### What changes were proposed in this pull request?
`IF NOT EXISTS` in `INSERT OVERWRITE` should not support dynamic partitions. If we specify `IF NOT EXISTS`, the inserted statement is not shown in the table.

This PR is to issue an exception in this case, just like what Hive does. Also issue an exception if users specify `IF NOT EXISTS` if users do not specify any `PARTITION` specification.

#### How was this patch tested?
Added test cases into `PlanParserSuite` and `InsertIntoHiveTableSuite`

Author: gatorsmile <gatorsmile@gmail.com>

Closes #13447 from gatorsmile/insertIfNotExist.
2016-06-16 22:54:02 -07:00
Yin Huai d9c6628c47 [SPARK-15991] SparkContext.hadoopConfiguration should be always the base of hadoop conf created by SessionState
## What changes were proposed in this pull request?
Before this patch, after a SparkSession has been created, hadoop conf set directly to SparkContext.hadoopConfiguration will not affect the hadoop conf created by SessionState. This patch makes the change to always use SparkContext.hadoopConfiguration  as the base.

This patch also changes the behavior of hive-site.xml support added in https://github.com/apache/spark/pull/12689/. With this patch, we will load hive-site.xml to SparkContext.hadoopConfiguration.

## How was this patch tested?
New test in SparkSessionBuilderSuite.

Author: Yin Huai <yhuai@databricks.com>

Closes #13711 from yhuai/SPARK-15991.
2016-06-16 17:06:24 -07:00
gatorsmile 796429d711 [SPARK-15998][SQL] Verification of SQLConf HIVE_METASTORE_PARTITION_PRUNING
#### What changes were proposed in this pull request?
`HIVE_METASTORE_PARTITION_PRUNING` is a public `SQLConf`. When `true`, some predicates will be pushed down into the Hive metastore so that unmatching partitions can be eliminated earlier. The current default value is `false`. For performance improvement, users might turn this parameter on.

So far, the code base does not have such a test case to verify whether this `SQLConf` properly works. This PR is to improve the test case coverage for avoiding future regression.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #13716 from gatorsmile/addTestMetastorePartitionPruning.
2016-06-16 14:23:17 -07:00
gatorsmile 6451cf9270 [SPARK-15862][SQL] Better Error Message When Having Database Name in CACHE TABLE AS SELECT
#### What changes were proposed in this pull request?
~~If the temp table already exists, we should not silently replace it when doing `CACHE TABLE AS SELECT`. This is inconsistent with the behavior of `CREAT VIEW` or `CREATE TABLE`. This PR is to fix this silent drop.~~

~~Maybe, we also can introduce new syntax for replacing the existing one. For example, in Hive, to replace a view, the syntax should be like `ALTER VIEW AS SELECT` or `CREATE OR REPLACE VIEW AS SELECT`~~

The table name in `CACHE TABLE AS SELECT` should NOT contain database prefix like "database.table". Thus, this PR captures this in Parser and outputs a better error message, instead of reporting the view already exists.

In addition, refactoring the `Parser` to generate table identifiers instead of returning the table name string.

#### How was this patch tested?
- Added a test case for caching and uncaching qualified table names
- Fixed a few test cases that do not drop temp table at the end
- Added the related test case for the issue resolved in this PR

Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

Closes #13572 from gatorsmile/cacheTableAsSelect.
2016-06-16 10:01:59 -07:00
Herman van Hovell b75f454f94 [SPARK-15824][SQL] Execute WITH .... INSERT ... statements immediately
## What changes were proposed in this pull request?
We currently immediately execute `INSERT` commands when they are issued. This is not the case as soon as we use a `WITH` to define common table expressions, for example:
```sql
WITH
tbl AS (SELECT * FROM x WHERE id = 10)
INSERT INTO y
SELECT *
FROM   tbl
```

This PR fixes this problem. This PR closes https://github.com/apache/spark/pull/13561 (which fixes the a instance of this problem in the ThriftSever).

## How was this patch tested?
Added a test to `InsertSuite`

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #13678 from hvanhovell/SPARK-15824.
2016-06-15 21:33:26 -07:00
gatorsmile 09925735b5 [SPARK-15901][SQL][TEST] Verification of CONVERT_METASTORE_ORC and CONVERT_METASTORE_PARQUET
#### What changes were proposed in this pull request?
So far, we do not have test cases for verifying whether the external parameters `HiveUtils .CONVERT_METASTORE_ORC` and `HiveUtils.CONVERT_METASTORE_PARQUET` properly works when users use non-default values. This PR is to add such test cases for avoiding potential regression.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #13622 from gatorsmile/addTestCase4parquetOrcConversion.
2016-06-15 14:08:55 -07:00
Yin Huai e1585cc748 [SPARK-15959][SQL] Add the support of hive.metastore.warehouse.dir back
## What changes were proposed in this pull request?
This PR adds the support of conf `hive.metastore.warehouse.dir` back. With this patch, the way of setting the warehouse dir is described as follows:
* If `spark.sql.warehouse.dir` is set, `hive.metastore.warehouse.dir` will be automatically set to the value of `spark.sql.warehouse.dir`. The warehouse dir is effectively set to the value of `spark.sql.warehouse.dir`.
* If `spark.sql.warehouse.dir` is not set but `hive.metastore.warehouse.dir` is set, `spark.sql.warehouse.dir` will be automatically set to the value of `hive.metastore.warehouse.dir`. The warehouse dir is effectively set to the value of `hive.metastore.warehouse.dir`.
* If neither `spark.sql.warehouse.dir` nor `hive.metastore.warehouse.dir` is set, `hive.metastore.warehouse.dir` will be automatically set to the default value of `spark.sql.warehouse.dir`. The warehouse dir is effectively set to the default value of `spark.sql.warehouse.dir`.

## How was this patch tested?
`set hive.metastore.warehouse.dir` in `HiveSparkSubmitSuite`.

JIRA: https://issues.apache.org/jira/browse/SPARK-15959

Author: Yin Huai <yhuai@databricks.com>

Closes #13679 from yhuai/hiveWarehouseDir.
2016-06-15 11:50:54 -07:00
Herman van Hovell 0bd86c0fe4 [SPARK-15011][SQL] Re-enable 'analyze MetastoreRelations' in hive StatisticsSuite
## What changes were proposed in this pull request?
This test re-enables the `analyze MetastoreRelations` in `org.apache.spark.sql.hive.StatisticsSuite`.

The flakiness of this test was traced back to a shared configuration option, `hive.exec.compress.output`, in `TestHive`. This property was set to `true` by the `HiveCompatibilitySuite`. I have added configuration resetting logic to `HiveComparisonTest`, in order to prevent such a thing from happening again.

## How was this patch tested?
Is a test.

Author: Herman van Hovell <hvanhovell@databricks.com>
Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #13498 from hvanhovell/SPARK-15011.
2016-06-14 18:24:59 -07:00
gatorsmile df4ea6614d [SPARK-15864][SQL] Fix Inconsistent Behaviors when Uncaching Non-cached Tables
#### What changes were proposed in this pull request?
To uncache a table, we have three different ways:
- _SQL interface_: `UNCACHE TABLE`
- _DataSet API_: `sparkSession.catalog.uncacheTable`
- _DataSet API_: `sparkSession.table(tableName).unpersist()`

When the table is not cached,
- _SQL interface_: `UNCACHE TABLE non-cachedTable` -> **no error message**
- _Dataset API_: `sparkSession.catalog.uncacheTable("non-cachedTable")` -> **report a strange error message:**
```requirement failed: Table [a: int] is not cached```
- _Dataset API_: `sparkSession.table("non-cachedTable").unpersist()` -> **no error message**

This PR will make them consistent. No operation if the table has already been uncached.

In addition, this PR also removes `uncacheQuery` and renames `tryUncacheQuery` to `uncacheQuery`, and documents it that it's noop if the table has already been uncached

#### How was this patch tested?
Improved the existing test case for verifying the cases when the table has not been cached.
Also added test cases for verifying the cases when the table does not exist

Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

Closes #13593 from gatorsmile/uncacheNonCachedTable.
2016-06-14 11:44:37 -07:00
Takuya UESHIN c5b7355819 [SPARK-15915][SQL] Logical plans should use canonicalized plan when override sameResult.
## What changes were proposed in this pull request?

`DataFrame` with plan overriding `sameResult` but not using canonicalized plan to compare can't cacheTable.

The example is like:

```
    val localRelation = Seq(1, 2, 3).toDF()
    localRelation.createOrReplaceTempView("localRelation")

    spark.catalog.cacheTable("localRelation")
    assert(
      localRelation.queryExecution.withCachedData.collect {
        case i: InMemoryRelation => i
      }.size == 1)
```

and this will fail as:

```
ArrayBuffer() had size 0 instead of expected size 1
```

The reason is that when do `spark.catalog.cacheTable("localRelation")`, `CacheManager` tries to cache for the plan wrapped by `SubqueryAlias` but when planning for the DataFrame `localRelation`, `CacheManager` tries to find cached table for the not-wrapped plan because the plan for DataFrame `localRelation` is not wrapped.
Some plans like `LocalRelation`, `LogicalRDD`, etc. override `sameResult` method, but not use canonicalized plan to compare so the `CacheManager` can't detect the plans are the same.

This pr modifies them to use canonicalized plan when override `sameResult` method.

## How was this patch tested?

Added a test to check if DataFrame with plan overriding sameResult but not using canonicalized plan to compare can cacheTable.

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #13638 from ueshin/issues/SPARK-15915.
2016-06-14 10:52:13 -07:00
gatorsmile bc02d01129 [SPARK-15655][SQL] Fix Wrong Partition Column Order when Fetching Partitioned Tables
#### What changes were proposed in this pull request?
When fetching the partitioned table, the output contains wrong results. The order of partition key values do not match the order of partition key columns in output schema. For example,

```SQL
CREATE TABLE table_with_partition(c1 string) PARTITIONED BY (p1 string,p2 string,p3 string,p4 string,p5 string)

INSERT OVERWRITE TABLE table_with_partition PARTITION (p1='a',p2='b',p3='c',p4='d',p5='e') SELECT 'blarr'

SELECT p1, p2, p3, p4, p5, c1 FROM table_with_partition
```
```
+---+---+---+---+---+-----+
| p1| p2| p3| p4| p5|   c1|
+---+---+---+---+---+-----+
|  d|  e|  c|  b|  a|blarr|
+---+---+---+---+---+-----+
```

The expected result should be
```
+---+---+---+---+---+-----+
| p1| p2| p3| p4| p5|   c1|
+---+---+---+---+---+-----+
|  a|  b|  c|  d|  e|blarr|
+---+---+---+---+---+-----+
```
This PR is to fix this by enforcing the order matches the table partition definition.

#### How was this patch tested?
Added a test case into `SQLQuerySuite`

Author: gatorsmile <gatorsmile@gmail.com>

Closes #13400 from gatorsmile/partitionedTableFetch.
2016-06-14 09:58:06 -07:00
Sandeep Singh 1842cdd4ee [SPARK-15663][SQL] SparkSession.catalog.listFunctions shouldn't include the list of built-in functions
## What changes were proposed in this pull request?
SparkSession.catalog.listFunctions currently returns all functions, including the list of built-in functions. This makes the method not as useful because anytime it is run the result set contains over 100 built-in functions.

## How was this patch tested?
CatalogSuite

Author: Sandeep Singh <sandeep@techaddict.me>

Closes #13413 from techaddict/SPARK-15663.
2016-06-13 21:58:52 -07:00
gatorsmile 5827b65e28 [SPARK-15808][SQL] File Format Checking When Appending Data
#### What changes were proposed in this pull request?
**Issue:** Got wrong results or strange errors when append data to a table with mismatched file format.

_Example 1: PARQUET -> CSV_
```Scala
createDF(0, 9).write.format("parquet").saveAsTable("appendParquetToOrc")
createDF(10, 19).write.mode(SaveMode.Append).format("orc").saveAsTable("appendParquetToOrc")
```

Error we got:
```
Job aborted due to stage failure: Task 0 in stage 2.0 failed 1 times, most recent failure: Lost task 0.0 in stage 2.0 (TID 2, localhost): java.lang.RuntimeException: file:/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/warehouse-bc8fedf2-aa6a-4002-a18b-524c6ac859d4/appendorctoparquet/part-r-00000-c0e3f365-1d46-4df5-a82c-b47d7af9feb9.snappy.orc is not a Parquet file. expected magic number at tail [80, 65, 82, 49] but found [79, 82, 67, 23]
```

_Example 2: Json -> CSV_
```Scala
createDF(0, 9).write.format("json").saveAsTable("appendJsonToCSV")
createDF(10, 19).write.mode(SaveMode.Append).format("parquet").saveAsTable("appendJsonToCSV")
```

No exception, but wrong results:
```
+----+----+
|  c1|  c2|
+----+----+
|null|null|
|null|null|
|null|null|
|null|null|
|   0|str0|
|   1|str1|
|   2|str2|
|   3|str3|
|   4|str4|
|   5|str5|
|   6|str6|
|   7|str7|
|   8|str8|
|   9|str9|
+----+----+
```
_Example 3: Json -> Text_
```Scala
createDF(0, 9).write.format("json").saveAsTable("appendJsonToText")
createDF(10, 19).write.mode(SaveMode.Append).format("text").saveAsTable("appendJsonToText")
```

Error we got:
```
Text data source supports only a single column, and you have 2 columns.
```

This PR is to issue an exception with appropriate error messages.

#### How was this patch tested?
Added test cases.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #13546 from gatorsmile/fileFormatCheck.
2016-06-13 19:31:40 -07:00
Wenchen Fan c4b1ad0209 [SPARK-15887][SQL] Bring back the hive-site.xml support for Spark 2.0
## What changes were proposed in this pull request?

Right now, Spark 2.0 does not load hive-site.xml. Based on users' feedback, it seems make sense to still load this conf file.

This PR adds a `hadoopConf` API in `SharedState`, which is `sparkContext.hadoopConfiguration` by default. When users are under hive context, `SharedState.hadoopConf` will load hive-site.xml and append its configs to `sparkContext.hadoopConfiguration`.

When we need to read hadoop config in spark sql, we should call `SessionState.newHadoopConf`, which contains `sparkContext.hadoopConfiguration`, hive-site.xml and sql configs.

## How was this patch tested?

new test in `HiveDataFrameSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #13611 from cloud-fan/hive-site.
2016-06-13 14:57:35 -07:00
Wenchen Fan f5d38c3925 Revert "[SPARK-15753][SQL] Move Analyzer stuff to Analyzer from DataFrameWriter"
This reverts commit 0ec279ffdf.
2016-06-12 16:52:15 -07:00
hyukjinkwon 9e204c62c6 [SPARK-15840][SQL] Add two missing options in documentation and some option related changes
## What changes were proposed in this pull request?

This PR

1. Adds the documentations for some missing options, `inferSchema` and `mergeSchema` for Python and Scala.

2. Fiixes `[[DataFrame]]` to ```:class:`DataFrame` ``` so that this can be shown

  - from
    ![2016-06-09 9 31 16](https://cloud.githubusercontent.com/assets/6477701/15929721/8b864734-2e89-11e6-83f6-207527de4ac9.png)

  - to (with class link)
    ![2016-06-09 9 31 00](https://cloud.githubusercontent.com/assets/6477701/15929717/8a03d728-2e89-11e6-8a3f-08294964db22.png)

  (Please refer [the latest documentation](https://people.apache.org/~pwendell/spark-nightly/spark-master-docs/latest/api/python/pyspark.sql.html))

3. Moves `mergeSchema` option to `ParquetOptions` with removing unused options, `metastoreSchema` and `metastoreTableName`.

  They are not used anymore. They were removed in e720dda42e and there are no use cases as below:

  ```bash
  grep -r -e METASTORE_SCHEMA -e \"metastoreSchema\" -e \"metastoreTableName\" -e METASTORE_TABLE_NAME .
  ```

  ```
  ./sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala:  private[sql] val METASTORE_SCHEMA = "metastoreSchema"
  ./sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala:  private[sql] val METASTORE_TABLE_NAME = "metastoreTableName"
  ./sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala:        ParquetFileFormat.METASTORE_TABLE_NAME -> TableIdentifier(
```

  It only sets `metastoreTableName` in the last case but does not use the table name.

4. Sets the correct default values (in the documentation) for `compression` option for ORC(`snappy`, see [OrcOptions.scala#L33-L42](3ded5bc4db/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcOptions.scala (L33-L42))) and Parquet(`the value specified in SQLConf`, see [ParquetOptions.scala#L38-L47](3ded5bc4db/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala (L38-L47))) and `columnNameOfCorruptRecord` for JSON(`the value specified in SQLConf`, see [JsonFileFormat.scala#L53-L55](4538443e27/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala (L53-L55)) and [JsonFileFormat.scala#L105-L106](4538443e27/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala (L105-L106))).

## How was this patch tested?

Existing tests should cover this.

Author: hyukjinkwon <gurwls223@gmail.com>
Author: Hyukjin Kwon <gurwls223@gmail.com>

Closes #13576 from HyukjinKwon/SPARK-15840.
2016-06-11 23:20:40 -07:00
Sameer Agarwal 468da03e23 [SPARK-15678] Add support to REFRESH data source paths
## What changes were proposed in this pull request?

Spark currently incorrectly continues to use cached data even if the underlying data is overwritten.

Current behavior:
```scala
val dir = "/tmp/test"
sqlContext.range(1000).write.mode("overwrite").parquet(dir)
val df = sqlContext.read.parquet(dir).cache()
df.count() // outputs 1000
sqlContext.range(10).write.mode("overwrite").parquet(dir)
sqlContext.read.parquet(dir).count() // outputs 1000 <---- We are still using the cached dataset
```

This patch fixes this bug by adding support for `REFRESH path` that invalidates and refreshes all the cached data (and the associated metadata) for any dataframe that contains the given data source path.

Expected behavior:
```scala
val dir = "/tmp/test"
sqlContext.range(1000).write.mode("overwrite").parquet(dir)
val df = sqlContext.read.parquet(dir).cache()
df.count() // outputs 1000
sqlContext.range(10).write.mode("overwrite").parquet(dir)
spark.catalog.refreshResource(dir)
sqlContext.read.parquet(dir).count() // outputs 10 <---- We are not using the cached dataset
```

## How was this patch tested?

Unit tests for overwrites and appends in `ParquetQuerySuite` and `CachedTableSuite`.

Author: Sameer Agarwal <sameer@databricks.com>

Closes #13566 from sameeragarwal/refresh-path-2.
2016-06-10 20:43:18 -07:00
Davies Liu aec502d911 [SPARK-15654] [SQL] fix non-splitable files for text based file formats
## What changes were proposed in this pull request?

Currently, we always split the files when it's bigger than maxSplitBytes, but Hadoop LineRecordReader does not respect the splits for compressed files correctly, we should have a API for FileFormat to check whether the file could be splitted or not.

This PR is based on #13442, closes #13442

## How was this patch tested?

add regression tests.

Author: Davies Liu <davies@databricks.com>

Closes #13531 from davies/fix_split.
2016-06-10 14:32:43 -07:00
Sandeep Singh 865ec32dd9 [MINOR][X][X] Replace all occurrences of None: Option with Option.empty
## What changes were proposed in this pull request?
Replace all occurrences of `None: Option[X]` with `Option.empty[X]`

## How was this patch tested?
Exisiting Tests

Author: Sandeep Singh <sandeep@techaddict.me>

Closes #13591 from techaddict/minor-7.
2016-06-10 13:06:51 -07:00
Liwei Lin fb219029dd [SPARK-15871][SQL] Add assertNotPartitioned check in DataFrameWriter
## What changes were proposed in this pull request?

It doesn't make sense to specify partitioning parameters, when we write data out from Datasets/DataFrames into `jdbc` tables or streaming `ForeachWriter`s.

This patch adds `assertNotPartitioned` check in `DataFrameWriter`.

<table>
<tr>
	<td align="center"><strong>operation</strong></td>
	<td align="center"><strong>should check not partitioned?</strong></td>
</tr>
<tr>
	<td align="center">mode</td>
	<td align="center"></td>
</tr>
<tr>
	<td align="center">outputMode</td>
	<td align="center"></td>
</tr>
<tr>
	<td align="center">trigger</td>
	<td align="center"></td>
</tr>
<tr>
	<td align="center">format</td>
	<td align="center"></td>
</tr>
<tr>
	<td align="center">option/options</td>
	<td align="center"></td>
</tr>
<tr>
	<td align="center">partitionBy</td>
	<td align="center"></td>
</tr>
<tr>
	<td align="center">bucketBy</td>
	<td align="center"></td>
</tr>
<tr>
	<td align="center">sortBy</td>
	<td align="center"></td>
</tr>
<tr>
	<td align="center">save</td>
	<td align="center"></td>
</tr>
<tr>
	<td align="center">queryName</td>
	<td align="center"></td>
</tr>
<tr>
	<td align="center">startStream</td>
	<td align="center"></td>
</tr>
<tr>
	<td align="center">foreach</td>
	<td align="center">yes</td>
</tr>
<tr>
	<td align="center">insertInto</td>
	<td align="center"></td>
</tr>
<tr>
	<td align="center">saveAsTable</td>
	<td align="center"></td>
</tr>
<tr>
	<td align="center">jdbc</td>
	<td align="center">yes</td>
</tr>
<tr>
	<td align="center">json</td>
	<td align="center"></td>
</tr>
<tr>
	<td align="center">parquet</td>
	<td align="center"></td>
</tr>
<tr>
	<td align="center">orc</td>
	<td align="center"></td>
</tr>
<tr>
	<td align="center">text</td>
	<td align="center"></td>
</tr>
<tr>
	<td align="center">csv</td>
	<td align="center"></td>
</tr>
</table>

## How was this patch tested?

New dedicated tests.

Author: Liwei Lin <lwlin7@gmail.com>

Closes #13597 from lw-lin/add-assertNotPartitioned.
2016-06-10 13:01:29 -07:00
Liang-Chi Hsieh 0ec279ffdf [SPARK-15753][SQL] Move Analyzer stuff to Analyzer from DataFrameWriter
## What changes were proposed in this pull request?

This patch moves some codes in `DataFrameWriter.insertInto` that belongs to `Analyzer`.

## How was this patch tested?
Existing tests.

Author: Liang-Chi Hsieh <simonh@tw.ibm.com>

Closes #13496 from viirya/move-analyzer-stuff.
2016-06-10 11:05:04 -07:00
Shixiong Zhu 00c310133d [SPARK-15593][SQL] Add DataFrameWriter.foreach to allow the user consuming data in ContinuousQuery
## What changes were proposed in this pull request?

* Add DataFrameWriter.foreach to allow the user consuming data in ContinuousQuery
  * ForeachWriter is the interface for the user to consume partitions of data
* Add a type parameter T to DataFrameWriter

Usage
```Scala
val ds = spark.read....stream().as[String]
ds.....write
         .queryName(...)
        .option("checkpointLocation", ...)
        .foreach(new ForeachWriter[Int] {
          def open(partitionId: Long, version: Long): Boolean = {
             // prepare some resources for a partition
             // check `version` if possible and return `false` if this is a duplicated data to skip the data processing.
          }

          override def process(value: Int): Unit = {
              // process data
          }

          def close(errorOrNull: Throwable): Unit = {
             // release resources for a partition
             // check `errorOrNull` and handle the error if necessary.
          }
        })
```

## How was this patch tested?

New unit tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #13342 from zsxwing/foreach.
2016-06-10 00:11:46 -07:00
Zheng RuiFeng fd8af39713 [MINOR] Fix Typos 'an -> a'
## What changes were proposed in this pull request?

`an -> a`

Use cmds like `find . -name '*.R' | xargs -i sh -c "grep -in ' an [^aeiou]' {} && echo {}"` to generate candidates, and review them one by one.

## How was this patch tested?
manual tests

Author: Zheng RuiFeng <ruifengz@foxmail.com>

Closes #13515 from zhengruifeng/an_a.
2016-06-06 09:35:47 +01:00
Andrew Or b1cc7da3e3 [SPARK-15722][SQL] Disallow specifying schema in CTAS statement
## What changes were proposed in this pull request?

As of this patch, the following throws an exception because the schemas may not match:
```
CREATE TABLE students (age INT, name STRING) AS SELECT * FROM boxes
```
but this is OK:
```
CREATE TABLE students AS SELECT * FROM boxes
```

## How was this patch tested?

SQLQuerySuite, HiveDDLCommandSuite

Author: Andrew Or <andrew@databricks.com>

Closes #13490 from andrewor14/ctas-no-column.
2016-06-03 14:39:41 -07:00
Dongjoon Hyun b9fcfb3bd1 [SPARK-15744][SQL] Rename two TungstenAggregation*Suites and update codgen/error messages/comments
## What changes were proposed in this pull request?

For consistency, this PR updates some remaining `TungstenAggregation/SortBasedAggregate` after SPARK-15728.
- Update a comment in codegen in `VectorizedHashMapGenerator.scala`.
- `TungstenAggregationQuerySuite` --> `HashAggregationQuerySuite`
- `TungstenAggregationQueryWithControlledFallbackSuite` --> `HashAggregationQueryWithControlledFallbackSuite`
- Update two error messages in `SQLQuerySuite.scala` and `AggregationQuerySuite.scala`.
- Update several comments.

## How was this patch tested?

Manual (Only comment changes and test suite renamings).

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13487 from dongjoon-hyun/SPARK-15744.
2016-06-03 00:36:06 -07:00
Sameer Agarwal f7288e166c [SPARK-15745][SQL] Use classloader's getResource() for reading resource files in HiveTests
## What changes were proposed in this pull request?

This is a cleaner approach in general but my motivation behind this change in particular is to be able to run these tests from anywhere without relying on system properties.

## How was this patch tested?

Test only change

Author: Sameer Agarwal <sameer@databricks.com>

Closes #13489 from sameeragarwal/resourcepath.
2016-06-03 00:13:43 -07:00
Andrew Or d1c1fbc345 [SPARK-15715][SQL] Fix alter partition with storage information in Hive
## What changes were proposed in this pull request?

This command didn't work for Hive tables. Now it does:
```
ALTER TABLE boxes PARTITION (width=3)
    SET SERDE 'com.sparkbricks.serde.ColumnarSerDe'
    WITH SERDEPROPERTIES ('compress'='true')
```

## How was this patch tested?

`HiveExternalCatalogSuite`

Author: Andrew Or <andrew@databricks.com>

Closes #13453 from andrewor14/alter-partition-storage.
2016-06-02 17:44:48 -07:00
Wenchen Fan f34aadc54c [SPARK-15718][SQL] better error message for writing bucketed data
## What changes were proposed in this pull request?

Currently we don't support bucketing for `save` and `insertInto`.

For `save`, we just write the data out into a directory users specified, and it's not a table, we don't keep its metadata. When we read it back, we have no idea if the data is bucketed or not, so it doesn't make sense to use `save` to write bucketed data, as we can't use the bucket information anyway.

We can support it in the future, once we have features like bucket discovery, or we save bucket information in the data directory too, so that we don't need to rely on a metastore.

For `insertInto`, it inserts data into an existing table, so it doesn't make sense to specify bucket information, as we should get the bucket information from the existing table.

This PR improves the error message for the above 2  cases.
## How was this patch tested?

new test in `BukctedWriteSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #13452 from cloud-fan/error-msg.
2016-06-02 17:39:56 -07:00
Cheng Lian 4315427657 [SPARK-15719][SQL] Disables writing Parquet summary files by default
## What changes were proposed in this pull request?

This PR disables writing Parquet summary files by default (i.e., when Hadoop configuration "parquet.enable.summary-metadata" is not set).

Please refer to [SPARK-15719][1] for more details.

## How was this patch tested?

New test case added in `ParquetQuerySuite` to check no summary files are written by default.

[1]: https://issues.apache.org/jira/browse/SPARK-15719

Author: Cheng Lian <lian@databricks.com>

Closes #13455 from liancheng/spark-15719-disable-parquet-summary-files.
2016-06-02 16:16:27 -07:00
Sean Zhong d109a1beee [SPARK-15711][SQL] Ban CREATE TEMPORARY TABLE USING AS SELECT
## What changes were proposed in this pull request?

This PR bans syntax like `CREATE TEMPORARY TABLE USING AS SELECT`

`CREATE TEMPORARY TABLE ... USING ... AS ...` is not properly implemented, the temporary data is not cleaned up when the session exits. Before a full fix, we probably should ban this syntax.

This PR only impact syntax like `CREATE TEMPORARY TABLE ... USING ... AS ...`.
Other syntax like `CREATE TEMPORARY TABLE .. USING ...` and `CREATE TABLE ... USING ...` are not impacted.

## How was this patch tested?

Unit test.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #13451 from clockfly/ban_create_temp_table_using_as.
2016-06-02 14:11:01 -07:00
gatorsmile 9aff6f3b19 [SPARK-15515][SQL] Error Handling in Running SQL Directly On Files
#### What changes were proposed in this pull request?
This PR is to address the following issues:

- **ISSUE 1:** For ORC source format, we are reporting the strange error message when we did not enable Hive support:
```SQL
SQL Example:
  select id from `org.apache.spark.sql.hive.orc`.`file_path`
Error Message:
  Table or view not found: `org.apache.spark.sql.hive.orc`.`file_path`
```
Instead, we should issue the error message like:
```
Expected Error Message:
   The ORC data source must be used with Hive support enabled
```
- **ISSUE 2:** For the Avro format, we report the strange error message like:

The example query is like
  ```SQL
SQL Example:
  select id from `avro`.`file_path`
  select id from `com.databricks.spark.avro`.`file_path`
Error Message:
  Table or view not found: `com.databricks.spark.avro`.`file_path`
   ```
The desired message should be like:
```
Expected Error Message:
  Failed to find data source: avro. Please use Spark package http://spark-packages.org/package/databricks/spark-avro"
```

- ~~**ISSUE 3:** Unable to detect incompatibility libraries for Spark 2.0 in Data Source Resolution. We report a strange error message:~~

**Update**: The latest code changes contains
- For JDBC format, we added an extra checking in the rule `ResolveRelations` of `Analyzer`. Without the PR, Spark will return the error message like: `Option 'url' not specified`. Now, we are reporting `Unsupported data source type for direct query on files: jdbc`
- Make data source format name case incensitive so that error handling behaves consistent with the normal cases.
- Added the test cases for all the supported formats.

#### How was this patch tested?
Added test cases to cover all the above issues

Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

Closes #13283 from gatorsmile/runSQLAgainstFile.
2016-06-02 13:22:43 -07:00
Yin Huai 6dddb70c38 [SPARK-15646][SQL] When spark.sql.hive.convertCTAS is true, the conversion rule needs to respect TEXTFILE/SEQUENCEFILE format and the user-defined location
## What changes were proposed in this pull request?
When `spark.sql.hive.convertCTAS` is true, for a CTAS statement, we will create a data source table using the default source (i.e. parquet) if the CTAS does not specify any Hive storage format. However, there are two issues with this conversion logic.
1. First, we determine if a CTAS statement defines storage format by checking the serde. However, TEXTFILE/SEQUENCEFILE does not have a default serde. When we do the check, we have not set the default serde. So, a query like `CREATE TABLE abc STORED AS TEXTFILE AS SELECT ...` actually creates a data source parquet table.
2. In the conversion logic, we are ignoring the user-specified location.

This PR fixes the above two issues.

Also, this PR makes the parser throws an exception when a CTAS statement has a PARTITIONED BY clause. This change is made because Hive's syntax does not allow it and our current implementation actually does not work for this case (the insert operation always throws an exception because the insertion does not pick up the partitioning info).

## How was this patch tested?
I am adding new tests in SQLQuerySuite and HiveDDLCommandSuite.

Author: Yin Huai <yhuai@databricks.com>

Closes #13386 from yhuai/SPARK-14507.
2016-06-01 17:55:37 -07:00
Cheng Lian 7bb64aae27 [SPARK-15269][SQL] Removes unexpected empty table directories created while creating external Spark SQL data sourcet tables.
This PR is an alternative to #13120 authored by xwu0226.

## What changes were proposed in this pull request?

When creating an external Spark SQL data source table and persisting its metadata to Hive metastore, we don't use the standard Hive `Table.dataLocation` field because Hive only allows directory paths as data locations while Spark SQL also allows file paths. However, if we don't set `Table.dataLocation`, Hive always creates an unexpected empty table directory under database location, but doesn't remove it while dropping the table (because the table is external).

This PR works around this issue by explicitly setting `Table.dataLocation` and then manullay removing the created directory after creating the external table.

Please refer to [this JIRA comment][1] for more details about why we chose this approach as a workaround.

[1]: https://issues.apache.org/jira/browse/SPARK-15269?focusedCommentId=15297408&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15297408

## How was this patch tested?

1. A new test case is added in `HiveQuerySuite` for this case
2. Updated `ShowCreateTableSuite` to use the same table name in all test cases. (This is how I hit this issue at the first place.)

Author: Cheng Lian <lian@databricks.com>

Closes #13270 from liancheng/spark-15269-unpleasant-fix.
2016-06-01 16:02:27 -07:00
Dongjoon Hyun 85d6b0db9f [SPARK-15618][SQL][MLLIB] Use SparkSession.builder.sparkContext if applicable.
## What changes were proposed in this pull request?

This PR changes function `SparkSession.builder.sparkContext(..)` from **private[sql]** into **private[spark]**, and uses it if applicable like the followings.
```
- val spark = SparkSession.builder().config(sc.getConf).getOrCreate()
+ val spark = SparkSession.builder().sparkContext(sc).getOrCreate()
```

## How was this patch tested?

Pass the existing Jenkins tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13365 from dongjoon-hyun/SPARK-15618.
2016-05-31 17:40:44 -07:00
Dilip Biswal dfe2cbeb43 [SPARK-15557] [SQL] cast the string into DoubleType when it's used together with decimal
In this case, the result type of the expression becomes DECIMAL(38, 36) as we promote the individual string literals to DECIMAL(38, 18) when we handle string promotions for `BinaryArthmaticExpression`.

I think we need to cast the string literals to Double type instead. I looked at the history and found that  this was changed to use decimal instead of double to avoid potential loss of precision when we cast decimal to double.

To double check i ran the query against hive, mysql. This query returns non NULL result for both the databases and both promote the expression to use double.
Here is the output.

- Hive
```SQL
hive> create table l2 as select (cast(99 as decimal(19,6)) + '2') from l1;
OK
hive> describe l2;
OK
_c0                 	double
```
- MySQL
```SQL
mysql> create table foo2 as select (cast(99 as decimal(19,6)) + '2') from test;
Query OK, 1 row affected (0.01 sec)
Records: 1  Duplicates: 0  Warnings: 0

mysql> describe foo2;
+-----------------------------------+--------+------+-----+---------+-------+
| Field                             | Type   | Null | Key | Default | Extra |
+-----------------------------------+--------+------+-----+---------+-------+
| (cast(99 as decimal(19,6)) + '2') | double | NO   |     | 0       |       |
+-----------------------------------+--------+------+-----+---------+-------+
```

## How was this patch tested?
Added a new test in SQLQuerySuite

Author: Dilip Biswal <dbiswal@us.ibm.com>

Closes #13368 from dilipbiswal/spark-15557.
2016-05-31 15:49:45 -07:00
Lianhui Wang 2bfc4f1521 [SPARK-15649][SQL] Avoid to serialize MetastoreRelation in HiveTableScanExec
## What changes were proposed in this pull request?
in HiveTableScanExec, schema is lazy and is related with relation.attributeMap. So it needs to serialize MetastoreRelation when serializing task binary bytes.It can avoid to serialize MetastoreRelation.

## How was this patch tested?

Author: Lianhui Wang <lianhuiwang09@gmail.com>

Closes #13397 from lianhuiwang/avoid-serialize.
2016-05-31 09:21:51 -07:00
Reynold Xin 675921040e [SPARK-15638][SQL] Audit Dataset, SparkSession, and SQLContext
## What changes were proposed in this pull request?
This patch contains a list of changes as a result of my auditing Dataset, SparkSession, and SQLContext. The patch audits the categorization of experimental APIs, function groups, and deprecations. For the detailed list of changes, please see the diff.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #13370 from rxin/SPARK-15638.
2016-05-30 22:47:58 -07:00
Yadong Qi b4c32c4952 [SPARK-15549][SQL] Disable bucketing when the output doesn't contain all bucketing columns
## What changes were proposed in this pull request?
I create a bucketed table bucketed_table with bucket column i,
```scala
case class Data(i: Int, j: Int, k: Int)
sc.makeRDD(Array((1, 2, 3))).map(x => Data(x._1, x._2, x._3)).toDF.write.bucketBy(2, "i").saveAsTable("bucketed_table")
```

and I run the following SQLs:
```sql
SELECT j FROM bucketed_table;
Error in query: bucket column i not found in existing columns (j);

SELECT j, MAX(k) FROM bucketed_table GROUP BY j;
Error in query: bucket column i not found in existing columns (j, k);
```

I think we should add a check that, we only enable bucketing when it satisfies all conditions below:
1. the conf is enabled
2. the relation is bucketed
3. the output contains all bucketing columns

## How was this patch tested?
Updated test cases to reflect the changes.

Author: Yadong Qi <qiyadong2010@gmail.com>

Closes #13321 from watermen/SPARK-15549.
2016-05-28 10:19:29 -07:00
Tejas Patil a96e4151a9 [SPARK-14400][SQL] ScriptTransformation does not fail the job for bad user command
## What changes were proposed in this pull request?

- Refer to the Jira for the problem: jira : https://issues.apache.org/jira/browse/SPARK-14400
- The fix is to check if the process has exited with a non-zero exit code in `hasNext()`. I have moved this and checking of writer thread exception to a separate method.

## How was this patch tested?

- Ran a job which had incorrect transform script command and saw that the job fails
- Existing unit tests for `ScriptTransformationSuite`. Added a new unit test

Author: Tejas Patil <tejasp@fb.com>

Closes #12194 from tejasapatil/script_transform.
2016-05-27 12:05:11 -07:00
Dongjoon Hyun 4538443e27 [SPARK-15584][SQL] Abstract duplicate code: spark.sql.sources. properties
## What changes were proposed in this pull request?

This PR replaces `spark.sql.sources.` strings with `CreateDataSourceTableUtils.*` constant variables.

## How was this patch tested?

Pass the existing Jenkins tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13349 from dongjoon-hyun/SPARK-15584.
2016-05-27 11:10:31 -07:00
gatorsmile d5911d1173 [SPARK-15529][SQL] Replace SQLContext and HiveContext with SparkSession in Test
#### What changes were proposed in this pull request?
This PR is to use the new entrance `Sparksession` to replace the existing `SQLContext` and `HiveContext` in SQL test suites.

No change is made in the following suites:
- `ListTablesSuite` is to test the APIs of `SQLContext`.
- `SQLContextSuite` is to test `SQLContext`
- `HiveContextCompatibilitySuite` is to test `HiveContext`

**Update**: Move tests in `ListTableSuite` to `SQLContextSuite`

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

Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

Closes #13337 from gatorsmile/sparkSessionTest.
2016-05-26 22:40:57 -07:00
Zheng RuiFeng 6b1a6180e7 [MINOR] Fix Typos 'a -> an'
## What changes were proposed in this pull request?

`a` -> `an`

I use regex to generate potential error lines:
`grep -in ' a [aeiou]' mllib/src/main/scala/org/apache/spark/ml/*/*scala`
and review them line by line.

## How was this patch tested?

local build
`lint-java` checking

Author: Zheng RuiFeng <ruifengz@foxmail.com>

Closes #13317 from zhengruifeng/a_an.
2016-05-26 22:39:14 -07:00
Andrew Or 3fca635b4e [SPARK-15583][SQL] Disallow altering datasource properties
## What changes were proposed in this pull request?

Certain table properties (and SerDe properties) are in the protected namespace `spark.sql.sources.`, which we use internally for datasource tables. The user should not be allowed to

(1) Create a Hive table setting these properties
(2) Alter these properties in an existing table

Previously, we threw an exception if the user tried to alter the properties of an existing datasource table. However, this is overly restrictive for datasource tables and does not do anything for Hive tables.

## How was this patch tested?

DDLSuite

Author: Andrew Or <andrew@databricks.com>

Closes #13341 from andrewor14/alter-table-props.
2016-05-26 20:11:09 -07:00
Yin Huai 3ac2363d75 [SPARK-15532][SQL] SQLContext/HiveContext's public constructors should use SparkSession.build.getOrCreate
## What changes were proposed in this pull request?
This PR changes SQLContext/HiveContext's public constructor to use SparkSession.build.getOrCreate and removes isRootContext from SQLContext.

## How was this patch tested?
Existing tests.

Author: Yin Huai <yhuai@databricks.com>

Closes #13310 from yhuai/SPARK-15532.
2016-05-26 16:53:31 -07:00
Reynold Xin 0f61d6efb4 [SPARK-15552][SQL] Remove unnecessary private[sql] methods in SparkSession
## What changes were proposed in this pull request?
SparkSession has a list of unnecessary private[sql] methods. These methods cause some trouble because private[sql] doesn't apply in Java. In the cases that they are easy to remove, we can simply remove them. This patch does that.

As part of this pull request, I also replaced a bunch of protected[sql] with private[sql], to tighten up visibility.

## How was this patch tested?
Updated test cases to reflect the changes.

Author: Reynold Xin <rxin@databricks.com>

Closes #13319 from rxin/SPARK-15552.
2016-05-26 13:03:07 -07:00
Andrew Or 2b1ac6cea8 [SPARK-15539][SQL] DROP TABLE throw exception if table doesn't exist
## What changes were proposed in this pull request?

Same as #13302, but for DROP TABLE.

## How was this patch tested?

`DDLSuite`

Author: Andrew Or <andrew@databricks.com>

Closes #13307 from andrewor14/drop-table.
2016-05-26 12:04:18 -07:00
Bo Meng 53d4abe9e9 [SPARK-15537][SQL] fix dir delete issue
## What changes were proposed in this pull request?

For some of the test cases, e.g. `OrcSourceSuite`, it will create temp folders and temp files inside them. But after tests finish, the folders are not removed. This will cause lots of temp files created and space occupied, if we keep running the test cases.

The reason is dir.delete() won't work if dir is not empty. We need to recursively delete the content before deleting the folder.

## How was this patch tested?

Manually checked the temp folder to make sure the temp files were deleted.

Author: Bo Meng <mengbo@hotmail.com>

Closes #13304 from bomeng/SPARK-15537.
2016-05-26 00:22:47 -07:00
Reynold Xin 361ebc282b [SPARK-15543][SQL] Rename DefaultSources to make them more self-describing
## What changes were proposed in this pull request?
This patch renames various DefaultSources to make their names more self-describing. The choice of "DefaultSource" was from the days when we did not have a good way to specify short names.

They are now named:
- LibSVMFileFormat
- CSVFileFormat
- JdbcRelationProvider
- JsonFileFormat
- ParquetFileFormat
- TextFileFormat

Backward compatibility is maintained through aliasing.

## How was this patch tested?
Updated relevant test cases too.

Author: Reynold Xin <rxin@databricks.com>

Closes #13311 from rxin/SPARK-15543.
2016-05-25 23:54:24 -07:00
Andrew Or ee682fe293 [SPARK-15534][SPARK-15535][SQL] Truncate table fixes
## What changes were proposed in this pull request?

Two changes:
- When things fail, `TRUNCATE TABLE` just returns nothing. Instead, we should throw exceptions.
- Remove `TRUNCATE TABLE ... COLUMN`, which was never supported by either Spark or Hive.

## How was this patch tested?
Jenkins.

Author: Andrew Or <andrew@databricks.com>

Closes #13302 from andrewor14/truncate-table.
2016-05-25 15:08:39 -07:00
lfzCarlosC 02c8072eea [MINOR][MLLIB][STREAMING][SQL] Fix typos
fixed typos for source code for components [mllib] [streaming] and [SQL]

None and obvious.

Author: lfzCarlosC <lfz.carlos@gmail.com>

Closes #13298 from lfzCarlosC/master.
2016-05-25 10:53:57 -07:00
Reynold Xin 4f27b8dd58 [SPARK-15436][SQL] Remove DescribeFunction and ShowFunctions
## What changes were proposed in this pull request?
This patch removes the last two commands defined in the catalyst module: DescribeFunction and ShowFunctions. They were unnecessary since the parser could just generate DescribeFunctionCommand and ShowFunctionsCommand directly.

## How was this patch tested?
Created a new SparkSqlParserSuite.

Author: Reynold Xin <rxin@databricks.com>

Closes #13292 from rxin/SPARK-15436.
2016-05-25 19:17:53 +02:00
Parth Brahmbhatt 4acababcab [SPARK-15365][SQL] When table size statistics are not available from metastore, we should fallback to HDFS
## What changes were proposed in this pull request?
Currently if a table is used in join operation we rely on Metastore returned size to calculate if we can convert the operation to Broadcast join. This optimization only kicks in for table's that have the statistics available in metastore. Hive generally rolls over to HDFS if the statistics are not available directly from metastore and this seems like a reasonable choice to adopt given the optimization benefit of using broadcast joins.

## How was this patch tested?
I have executed queries locally to test.

Author: Parth Brahmbhatt <pbrahmbhatt@netflix.com>

Closes #13150 from Parth-Brahmbhatt/SPARK-15365.
2016-05-24 20:58:20 -07:00
wangyang 784cc07d16 [SPARK-15388][SQL] Fix spark sql CREATE FUNCTION with hive 1.2.1
## What changes were proposed in this pull request?

spark.sql("CREATE FUNCTION myfunc AS 'com.haizhi.bdp.udf.UDFGetGeoCode'") throws "org.apache.hadoop.hive.ql.metadata.HiveException:MetaException(message:NoSuchObjectException(message:Function default.myfunc does not exist))" with hive 1.2.1.

I think it is introduced by pr #12853. Fixing it by catching Exception (not NoSuchObjectException) and string matching.

## How was this patch tested?

added a unit test and also tested it manually

Author: wangyang <wangyang@haizhi.com>

Closes #13177 from wangyang1992/fixCreateFunc2.
2016-05-24 11:03:12 -07:00
Xin Wu 01659bc50c [SPARK-15431][SQL] Support LIST FILE(s)|JAR(s) command natively
## What changes were proposed in this pull request?
Currently command `ADD FILE|JAR <filepath | jarpath>` is supported natively in SparkSQL. However, when this command is run, the file/jar is added to the resources that can not be looked up by `LIST FILE(s)|JAR(s)` command because the `LIST` command is passed to Hive command processor in Spark-SQL or simply not supported in Spark-shell. There is no way users can find out what files/jars are added to the spark context.
Refer to [Hive commands](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Cli)

This PR is to support following commands:
`LIST (FILE[s] [filepath ...] | JAR[s] [jarfile ...])`

### For example:
##### LIST FILE(s)
```
scala> spark.sql("add file hdfs://bdavm009.svl.ibm.com:8020/tmp/test.txt")
res1: org.apache.spark.sql.DataFrame = []
scala> spark.sql("add file hdfs://bdavm009.svl.ibm.com:8020/tmp/test1.txt")
res2: org.apache.spark.sql.DataFrame = []

scala> spark.sql("list file hdfs://bdavm009.svl.ibm.com:8020/tmp/test1.txt").show(false)
+----------------------------------------------+
|result                                        |
+----------------------------------------------+
|hdfs://bdavm009.svl.ibm.com:8020/tmp/test1.txt|
+----------------------------------------------+

scala> spark.sql("list files").show(false)
+----------------------------------------------+
|result                                        |
+----------------------------------------------+
|hdfs://bdavm009.svl.ibm.com:8020/tmp/test1.txt|
|hdfs://bdavm009.svl.ibm.com:8020/tmp/test.txt |
+----------------------------------------------+
```

##### LIST JAR(s)
```
scala> spark.sql("add jar /Users/xinwu/spark/core/src/test/resources/TestUDTF.jar")
res9: org.apache.spark.sql.DataFrame = [result: int]

scala> spark.sql("list jar TestUDTF.jar").show(false)
+---------------------------------------------+
|result                                       |
+---------------------------------------------+
|spark://192.168.1.234:50131/jars/TestUDTF.jar|
+---------------------------------------------+

scala> spark.sql("list jars").show(false)
+---------------------------------------------+
|result                                       |
+---------------------------------------------+
|spark://192.168.1.234:50131/jars/TestUDTF.jar|
+---------------------------------------------+
```
## How was this patch tested?
New test cases are added for Spark-SQL, Spark-Shell and SparkContext API code path.

Author: Xin Wu <xinwu@us.ibm.com>
Author: xin Wu <xinwu@us.ibm.com>

Closes #13212 from xwu0226/list_command.
2016-05-23 17:32:01 -07:00
Andrew Or 2585d2b322 [SPARK-15279][SQL] Catch conflicting SerDe when creating table
## What changes were proposed in this pull request?

The user may do something like:
```
CREATE TABLE my_tab ROW FORMAT SERDE 'anything' STORED AS PARQUET
CREATE TABLE my_tab ROW FORMAT SERDE 'anything' STORED AS ... SERDE 'myserde'
CREATE TABLE my_tab ROW FORMAT DELIMITED ... STORED AS ORC
CREATE TABLE my_tab ROW FORMAT DELIMITED ... STORED AS ... SERDE 'myserde'
```
None of these should be allowed because the SerDe's conflict. As of this patch:
- `ROW FORMAT DELIMITED` is only compatible with `TEXTFILE`
- `ROW FORMAT SERDE` is only compatible with `TEXTFILE`, `RCFILE` and `SEQUENCEFILE`

## How was this patch tested?

New tests in `DDLCommandSuite`.

Author: Andrew Or <andrew@databricks.com>

Closes #13068 from andrewor14/row-format-conflict.
2016-05-23 11:55:03 -07:00
Sameer Agarwal dafcb05c2e [SPARK-15425][SQL] Disallow cross joins by default
## What changes were proposed in this pull request?

In order to prevent users from inadvertently writing queries with cartesian joins, this patch introduces a new conf `spark.sql.crossJoin.enabled` (set to `false` by default) that if not set, results in a `SparkException` if the query contains one or more cartesian products.

## How was this patch tested?

Added a test to verify the new behavior in `JoinSuite`. Additionally, `SQLQuerySuite` and `SQLMetricsSuite` were modified to explicitly enable cartesian products.

Author: Sameer Agarwal <sameer@databricks.com>

Closes #13209 from sameeragarwal/disallow-cartesian.
2016-05-22 23:32:39 -07:00
xin Wu df9adb5ec9 [SPARK-15206][SQL] add testcases for distinct aggregate in having clause
## What changes were proposed in this pull request?
Add new test cases for including distinct aggregate in having clause in 2.0 branch.
This is a followup PR for [#12974](https://github.com/apache/spark/pull/12974), which is for 1.6 branch.

Author: xin Wu <xinwu@us.ibm.com>

Closes #12984 from xwu0226/SPARK-15206.
2016-05-21 21:41:12 -07:00
Ergin Seyfe c18fa464f4 [SPARK-15280] Input/Output] Refactored OrcOutputWriter and moved serialization to a new class.
## What changes were proposed in this pull request?
Refactoring: Separated ORC serialization logic from OrcOutputWriter and moved to a new class called OrcSerializer.

## How was this patch tested?
Manual tests & existing tests.

Author: Ergin Seyfe <eseyfe@fb.com>

Closes #13066 from seyfe/orc_serializer.
2016-05-21 16:08:31 -07:00
Reynold Xin 45b7557e61 [SPARK-15424][SPARK-15437][SPARK-14807][SQL] Revert Create a hivecontext-compatibility module
## What changes were proposed in this pull request?
I initially asked to create a hivecontext-compatibility module to put the HiveContext there. But we are so close to Spark 2.0 release and there is only a single class in it. It seems overkill to have an entire package, which makes it more inconvenient, for a single class.

## How was this patch tested?
Tests were moved.

Author: Reynold Xin <rxin@databricks.com>

Closes #13207 from rxin/SPARK-15424.
2016-05-20 22:01:55 -07:00
Michael Armbrust 2ba3ff0449 [SPARK-10216][SQL] Revert "[] Avoid creating empty files during overwrit…
This reverts commit 8d05a7a from #12855, which seems to have caused regressions when working with empty DataFrames.

Author: Michael Armbrust <michael@databricks.com>

Closes #13181 from marmbrus/revert12855.
2016-05-20 13:00:29 -07:00
Reynold Xin e8adc552df [SPARK-15435][SQL] Append Command to all commands
## What changes were proposed in this pull request?
We started this convention to append Command suffix to all SQL commands. However, not all commands follow that convention. This patch adds Command suffix to all RunnableCommands.

## How was this patch tested?
Updated test cases to reflect the renames.

Author: Reynold Xin <rxin@databricks.com>

Closes #13215 from rxin/SPARK-15435.
2016-05-20 09:36:14 -07:00
gatorsmile 39fd469078 [SPARK-15367][SQL] Add refreshTable back
#### What changes were proposed in this pull request?
`refreshTable` was a method in `HiveContext`. It was deleted accidentally while we were migrating the APIs. This PR is to add it back to `HiveContext`.

In addition, in `SparkSession`, we put it under the catalog namespace (`SparkSession.catalog.refreshTable`).

#### How was this patch tested?
Changed the existing test cases to use the function `refreshTable`. Also added a test case for refreshTable in `hivecontext-compatibility`

Author: gatorsmile <gatorsmile@gmail.com>

Closes #13156 from gatorsmile/refreshTable.
2016-05-20 14:38:25 +08:00
Lianhui Wang 09a00510c4 [SPARK-15335][SQL] Implement TRUNCATE TABLE Command
## What changes were proposed in this pull request?

Like TRUNCATE TABLE Command in Hive, TRUNCATE TABLE is also supported by Hive. See the link: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL
Below is the related Hive JIRA: https://issues.apache.org/jira/browse/HIVE-446
This PR is to implement such a command for truncate table excluded column truncation(HIVE-4005).

## How was this patch tested?
Added a test case.

Author: Lianhui Wang <lianhuiwang09@gmail.com>

Closes #13170 from lianhuiwang/truncate.
2016-05-19 23:03:59 -07:00
Oleg Danilov e384c7fbb9 [SPARK-14261][SQL] Memory leak in Spark Thrift Server
Fixed memory leak (HiveConf in the CommandProcessorFactory)

Author: Oleg Danilov <oleg.danilov@wandisco.com>

Closes #12932 from dosoft/SPARK-14261.
2016-05-19 22:23:28 -07:00
Reynold Xin f2ee0ed4b7 [SPARK-15075][SPARK-15345][SQL] Clean up SparkSession builder and propagate config options to existing sessions if specified
## What changes were proposed in this pull request?
Currently SparkSession.Builder use SQLContext.getOrCreate. It should probably the the other way around, i.e. all the core logic goes in SparkSession, and SQLContext just calls that. This patch does that.

This patch also makes sure config options specified in the builder are propagated to the existing (and of course the new) SparkSession.

## How was this patch tested?
Updated tests to reflect the change, and also introduced a new SparkSessionBuilderSuite that should cover all the branches.

Author: Reynold Xin <rxin@databricks.com>

Closes #13200 from rxin/SPARK-15075.
2016-05-19 21:53:26 -07:00
Cheng Lian 6ac1c3a040 [SPARK-14346][SQL] Lists unsupported Hive features in SHOW CREATE TABLE output
## What changes were proposed in this pull request?

This PR is a follow-up of #13079. It replaces `hasUnsupportedFeatures: Boolean` in `CatalogTable` with `unsupportedFeatures: Seq[String]`, which contains unsupported Hive features of the underlying Hive table. In this way, we can accurately report all unsupported Hive features in the exception message.

## How was this patch tested?

Updated existing test case to check exception message.

Author: Cheng Lian <lian@databricks.com>

Closes #13173 from liancheng/spark-14346-follow-up.
2016-05-19 12:02:41 -07:00
Cheng Lian c4a45fd855 [SPARK-15334][SQL][HOTFIX] Fixes compilation error for Scala 2.10
## What changes were proposed in this pull request?

This PR fixes a Scala 2.10 compilation failure introduced in PR #13127.

## How was this patch tested?

Jenkins build.

Author: Cheng Lian <lian@databricks.com>

Closes #13166 from liancheng/hotfix-for-scala-2.10.
2016-05-18 18:58:24 +08:00
Sean Zhong 6e02aec44b [SPARK-15334][SQL] HiveClient facade not compatible with Hive 0.12
## What changes were proposed in this pull request?

HiveClient facade is not compatible with Hive 0.12.

This PR Fixes the following compatibility issues:
1. `org.apache.spark.sql.hive.client.HiveClientImpl` use `AddPartitionDesc(db, table, ignoreIfExists)` to create partitions, however, Hive 0.12 doesn't have this constructor for `AddPartitionDesc`.
2. `HiveClientImpl` uses `PartitionDropOptions` when dropping partition, however, class `PartitionDropOptions` doesn't exist in Hive 0.12.
3. Hive 0.12 doesn't support adding permanent functions. It is not valid to call `org.apache.hadoop.hive.ql.metadata.Hive.createFunction`, `org.apache.hadoop.hive.ql.metadata.Hive.alterFunction`, and `org.apache.hadoop.hive.ql.metadata.Hive.alterFunction`
4. `org.apache.spark.sql.hive.client.VersionsSuite` doesn't have enough test coverage for different hive versions 0.12, 0.13, 0.14, 1.0.0, 1.1.0, 1.2.0.

## How was this patch tested?

Unit test.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #13127 from clockfly/versionSuite.
2016-05-18 16:00:02 +08:00
Sean Zhong 25b315e6ca [SPARK-15171][SQL] Remove the references to deprecated method dataset.registerTempTable
## What changes were proposed in this pull request?

Update the unit test code, examples, and documents to remove calls to deprecated method `dataset.registerTempTable`.

## How was this patch tested?

This PR only changes the unit test code, examples, and comments. It should be safe.
This is a follow up of PR https://github.com/apache/spark/pull/12945 which was merged.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #13098 from clockfly/spark-15171-remove-deprecation.
2016-05-18 09:01:59 +08:00
Cheng Lian b674e67c22 [SPARK-14346][SQL] Native SHOW CREATE TABLE for Hive tables/views
## What changes were proposed in this pull request?

This is a follow-up of #12781. It adds native `SHOW CREATE TABLE` support for Hive tables and views. A new field `hasUnsupportedFeatures` is added to `CatalogTable` to indicate whether all table metadata retrieved from the concrete underlying external catalog (i.e. Hive metastore in this case) can be mapped to fields in `CatalogTable`. This flag is useful when the target Hive table contains structures that can't be handled by Spark SQL, e.g., skewed columns and storage handler, etc..

## How was this patch tested?

New test cases are added in `ShowCreateTableSuite` to do round-trip tests.

Author: Cheng Lian <lian@databricks.com>

Closes #13079 from liancheng/spark-14346-show-create-table-for-hive-tables.
2016-05-17 15:56:44 -07:00
hyukjinkwon 8d05a7a98b [SPARK-10216][SQL] Avoid creating empty files during overwriting with group by query
## What changes were proposed in this pull request?

Currently, `INSERT INTO` with `GROUP BY` query tries to make at least 200 files (default value of `spark.sql.shuffle.partition`), which results in lots of empty files.

This PR makes it avoid creating empty files during overwriting into Hive table and in internal data sources  with group by query.

This checks whether the given partition has data in it or not and creates/writes file only when it actually has data.

## How was this patch tested?

Unittests in `InsertIntoHiveTableSuite` and `HadoopFsRelationTest`.

Closes #8411

Author: hyukjinkwon <gurwls223@gmail.com>
Author: Keuntae Park <sirpkt@apache.org>

Closes #12855 from HyukjinKwon/pr/8411.
2016-05-17 11:18:51 -07:00
Sean Owen 122302cbf5 [SPARK-15290][BUILD] Move annotations, like @Since / @DeveloperApi, into spark-tags
## What changes were proposed in this pull request?

(See https://github.com/apache/spark/pull/12416 where most of this was already reviewed and committed; this is just the module structure and move part. This change does not move the annotations into test scope, which was the apparently problem last time.)

Rename `spark-test-tags` -> `spark-tags`; move common annotations like `Since` to `spark-tags`

## How was this patch tested?

Jenkins tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #13074 from srowen/SPARK-15290.
2016-05-17 09:55:53 +01:00
Sean Zhong 4a5ee1954a [SPARK-15253][SQL] Support old table schema config key "spark.sql.sources.schema" for DESCRIBE TABLE
## What changes were proposed in this pull request?

"DESCRIBE table" is broken when table schema is stored at key "spark.sql.sources.schema".

Originally, we used spark.sql.sources.schema to store the schema of a data source table.
After SPARK-6024, we removed this flag. Although we are not using spark.sql.sources.schema any more, we need to still support it.

## How was this patch tested?

Unit test.

When using spark2.0 to load a table generated by spark 1.2.
Before change:
`DESCRIBE table` => Schema of this table is inferred at runtime,,

After change:
`DESCRIBE table` => correct output.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #13073 from clockfly/spark-15253.
2016-05-16 10:41:20 +08:00
hyukjinkwon 3ded5bc4db [SPARK-15267][SQL] Refactor options for JDBC and ORC data sources and change default compression for ORC
## What changes were proposed in this pull request?

Currently, Parquet, JSON and CSV data sources have a class for thier options, (`ParquetOptions`, `JSONOptions` and `CSVOptions`).

It is convenient to manage options for sources to gather options into a class. Currently, `JDBC`, `Text`, `libsvm` and `ORC` datasources do not have this class. This might be nicer if these options are in a unified format so that options can be added and

This PR refactors the options in Spark internal data sources adding new classes, `OrcOptions`, `TextOptions`, `JDBCOptions` and `LibSVMOptions`.

Also, this PR change the default compression codec for ORC from `NONE` to `SNAPPY`.

## How was this patch tested?

Existing tests should cover this for refactoring and unittests in `OrcHadoopFsRelationSuite` for changing the default compression codec for ORC.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #13048 from HyukjinKwon/SPARK-15267.
2016-05-13 09:04:37 -07:00
Reynold Xin eda2800d44 [SPARK-14541][SQL] Support IFNULL, NULLIF, NVL and NVL2
## What changes were proposed in this pull request?
This patch adds support for a few SQL functions to improve compatibility with other databases: IFNULL, NULLIF, NVL and NVL2. In order to do this, this patch introduced a RuntimeReplaceable expression trait that allows replacing an unevaluable expression in the optimizer before evaluation.

Note that the semantics are not completely identical to other databases in esoteric cases.

## How was this patch tested?
Added a new test suite SQLCompatibilityFunctionSuite.

Closes #12373.

Author: Reynold Xin <rxin@databricks.com>

Closes #13084 from rxin/SPARK-14541.
2016-05-12 22:18:39 -07:00
Herman van Hovell bb1362eb3b [SPARK-10605][SQL] Create native collect_list/collect_set aggregates
## What changes were proposed in this pull request?
We currently use the Hive implementations for the collect_list/collect_set aggregate functions. This has a few major drawbacks: the use of HiveUDAF (which has quite a bit of overhead) and the lack of support for struct datatypes. This PR adds native implementation of these functions to Spark.

The size of the collected list/set may vary, this means we cannot use the fast, Tungsten, aggregation path to perform the aggregation, and that we fallback to the slower sort based path. Another big issue with these operators is that when the size of the collected list/set grows too large, we can start experiencing large GC pauzes and OOMEs.

This `collect*` aggregates implemented in this PR rely on the sort based aggregate path for correctness. They maintain their own internal buffer which holds the rows for one group at a time. The sortbased aggregation path is triggered by disabling `partialAggregation` for these aggregates (which is kinda funny); this technique is also employed in `org.apache.spark.sql.hiveHiveUDAFFunction`.

I have done some performance testing:
```scala
import org.apache.spark.sql.{Dataset, Row}

sql("create function collect_list2 as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFCollectList'")

val df = range(0, 10000000).select($"id", (rand(213123L) * 100000).cast("int").as("grp"))
df.select(countDistinct($"grp")).show

def benchmark(name: String, plan: Dataset[Row], maxItr: Int = 5): Unit = {
   // Do not measure planning.
   plan1.queryExecution.executedPlan

   // Execute the plan a number of times and average the result.
   val start = System.nanoTime
   var i = 0
   while (i < maxItr) {
     plan.rdd.foreach(row => Unit)
     i += 1
   }
   val time = (System.nanoTime - start) / (maxItr * 1000000L)
   println(s"[$name] $maxItr iterations completed in an average time of $time ms.")
}

val plan1 = df.groupBy($"grp").agg(collect_list($"id"))
val plan2 = df.groupBy($"grp").agg(callUDF("collect_list2", $"id"))

benchmark("Spark collect_list", plan1)
...
> [Spark collect_list] 5 iterations completed in an average time of 3371 ms.

benchmark("Hive collect_list", plan2)
...
> [Hive collect_list] 5 iterations completed in an average time of 9109 ms.
```
Performance is improved by a factor 2-3.

## How was this patch tested?
Added tests to `DataFrameAggregateSuite`.

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #12874 from hvanhovell/implode.
2016-05-12 13:56:00 -07:00
gatorsmile be617f3d06 [SPARK-14684][SPARK-15277][SQL] Partition Spec Validation in SessionCatalog and Checking Partition Spec Existence Before Dropping
#### What changes were proposed in this pull request?
~~Currently, multiple partitions are allowed to drop by using a single DDL command: Alter Table Drop Partition. However, the internal implementation could break atomicity. That means, we could just drop a subset of qualified partitions, if hitting an exception when dropping one of qualified partitions~~

~~This PR contains the following behavior changes:~~
~~- disallow dropping multiple partitions by a single command ~~
~~- allow users to input predicates in partition specification and issue a nicer error message if the predicate's comparison operator is not `=`.~~
~~- verify the partition spec in SessionCatalog. This can ensure each partition spec in `Drop Partition` does not correspond to multiple partitions.~~

This PR has two major parts:
- Verify the partition spec in SessionCatalog for fixing the following issue:
  ```scala
  sql(s"ALTER TABLE $externalTab DROP PARTITION (ds='2008-04-09', unknownCol='12')")
  ```
  Above example uses an invalid partition spec. Without this PR, we will drop all the partitions. The reason is Hive megastores getPartitions API returns all the partitions if we provide an invalid spec.

- Re-implemented the `dropPartitions` in `HiveClientImpl`. Now, we always check if all the user-specified partition specs exist before attempting to drop the partitions. Previously, we start drop the partition before completing checking the existence of all the partition specs. If any failure happened after we start to drop the partitions, we will log an error message to indicate which partitions have been dropped and which partitions have not been dropped.

#### How was this patch tested?
Modified the existing test cases and added new test cases.

Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

Closes #12801 from gatorsmile/banDropMultiPart.
2016-05-12 11:14:40 -07:00
Sean Zhong 33c6eb5218 [SPARK-15171][SQL] Deprecate registerTempTable and add dataset.createTempView
## What changes were proposed in this pull request?

Deprecates registerTempTable and add dataset.createTempView, dataset.createOrReplaceTempView.

## How was this patch tested?

Unit tests.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #12945 from clockfly/spark-15171.
2016-05-12 15:51:53 +08:00
Wenchen Fan 46991448aa [SPARK-15160][SQL] support data source table in InMemoryCatalog
## What changes were proposed in this pull request?

This PR adds a new rule to convert `SimpleCatalogRelation` to data source table if its table property contains data source information.

## How was this patch tested?

new test in SQLQuerySuite

Author: Wenchen Fan <wenchen@databricks.com>

Closes #12935 from cloud-fan/ds-table.
2016-05-11 23:55:42 -07:00
Cheng Lian f036dd7ce7 [SPARK-14346] SHOW CREATE TABLE for data source tables
## What changes were proposed in this pull request?

This PR adds native `SHOW CREATE TABLE` DDL command for data source tables. Support for Hive tables will be added in follow-up PR(s).

To show table creation DDL for data source tables created by CTAS statements, this PR also added partitioning and bucketing support for normal `CREATE TABLE ... USING ...` syntax.

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)

A new test suite `ShowCreateTableSuite` is added in sql/hive package to test the new feature.

Author: Cheng Lian <lian@databricks.com>

Closes #12781 from liancheng/spark-14346-show-create-table.
2016-05-11 20:44:04 -07:00
Andrew Or f14c4ba001 [SPARK-15276][SQL] CREATE TABLE with LOCATION should imply EXTERNAL
## What changes were proposed in this pull request?

Before:
```sql
-- uses that location but issues a warning
CREATE TABLE my_tab LOCATION /some/path
-- deletes any existing data in the specified location
DROP TABLE my_tab
```

After:
```sql
-- uses that location but creates an EXTERNAL table instead
CREATE TABLE my_tab LOCATION /some/path
-- does not delete the data at /some/path
DROP TABLE my_tab
```

This patch essentially makes the `EXTERNAL` field optional. This is related to #13032.

## How was this patch tested?

New test in `DDLCommandSuite`.

Author: Andrew Or <andrew@databricks.com>

Closes #13060 from andrewor14/location-implies-external.
2016-05-11 17:29:58 -07:00
Andrew Or 8881765ac7 [SPARK-15257][SQL] Require CREATE EXTERNAL TABLE to specify LOCATION
## What changes were proposed in this pull request?

Before:
```sql
-- uses warehouse dir anyway
CREATE EXTERNAL TABLE my_tab
-- doesn't actually delete the data
DROP TABLE my_tab
```
After:
```sql
-- no location is provided, throws exception
CREATE EXTERNAL TABLE my_tab
-- creates an external table using that location
CREATE EXTERNAL TABLE my_tab LOCATION '/path/to/something'
-- doesn't delete the data, which is expected
DROP TABLE my_tab
```

## How was this patch tested?

New test in `DDLCommandSuite`

Author: Andrew Or <andrew@databricks.com>

Closes #13032 from andrewor14/create-external-table-location.
2016-05-11 15:30:53 -07:00
Tathagata Das 81c68eceba [SPARK-15248][SQL] Make MetastoreFileCatalog consider directories from partition specs of a partitioned metastore table
Table partitions can be added with locations different from default warehouse location of a hive table.
`CREATE TABLE parquetTable (a int) PARTITIONED BY (b int) STORED AS parquet `
`ALTER TABLE parquetTable ADD PARTITION (b=1) LOCATION '/partition'`
Querying such a table throws error as the MetastoreFileCatalog does not list the added partition directory, it only lists the default base location.

```
[info] - SPARK-15248: explicitly added partitions should be readable *** FAILED *** (1 second, 8 milliseconds)
[info]   java.util.NoSuchElementException: key not found: file:/Users/tdas/Projects/Spark/spark2/target/tmp/spark-b39ad224-c5d1-4966-8981-fb45a2066d61/partition
[info]   at scala.collection.MapLike$class.default(MapLike.scala:228)
[info]   at scala.collection.AbstractMap.default(Map.scala:59)
[info]   at scala.collection.MapLike$class.apply(MapLike.scala:141)
[info]   at scala.collection.AbstractMap.apply(Map.scala:59)
[info]   at org.apache.spark.sql.execution.datasources.PartitioningAwareFileCatalog$$anonfun$listFiles$1.apply(PartitioningAwareFileCatalog.scala:59)
[info]   at org.apache.spark.sql.execution.datasources.PartitioningAwareFileCatalog$$anonfun$listFiles$1.apply(PartitioningAwareFileCatalog.scala:55)
[info]   at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
[info]   at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
[info]   at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
[info]   at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
[info]   at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
[info]   at scala.collection.AbstractTraversable.map(Traversable.scala:104)
[info]   at org.apache.spark.sql.execution.datasources.PartitioningAwareFileCatalog.listFiles(PartitioningAwareFileCatalog.scala:55)
[info]   at org.apache.spark.sql.execution.datasources.FileSourceStrategy$.apply(FileSourceStrategy.scala:93)
[info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:59)
[info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:59)
[info]   at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
[info]   at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440)
[info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:60)
[info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner.planLater(QueryPlanner.scala:55)
[info]   at org.apache.spark.sql.execution.SparkStrategies$SpecialLimits$.apply(SparkStrategies.scala:55)
[info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:59)
[info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:59)
[info]   at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
[info]   at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440)
[info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:60)
[info]   at org.apache.spark.sql.execution.QueryExecution.sparkPlan$lzycompute(QueryExecution.scala:77)
[info]   at org.apache.spark.sql.execution.QueryExecution.sparkPlan(QueryExecution.scala:75)
[info]   at org.apache.spark.sql.execution.QueryExecution.executedPlan$lzycompute(QueryExecution.scala:82)
[info]   at org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:82)
[info]   at org.apache.spark.sql.QueryTest.assertEmptyMissingInput(QueryTest.scala:330)
[info]   at org.apache.spark.sql.QueryTest.checkAnswer(QueryTest.scala:146)
[info]   at org.apache.spark.sql.QueryTest.checkAnswer(QueryTest.scala:159)
[info]   at org.apache.spark.sql.hive.ParquetMetastoreSuite$$anonfun$12$$anonfun$apply$mcV$sp$7$$anonfun$apply$mcV$sp$25.apply(parquetSuites.scala:554)
[info]   at org.apache.spark.sql.hive.ParquetMetastoreSuite$$anonfun$12$$anonfun$apply$mcV$sp$7$$anonfun$apply$mcV$sp$25.apply(parquetSuites.scala:535)
[info]   at org.apache.spark.sql.test.SQLTestUtils$class.withTempDir(SQLTestUtils.scala:125)
[info]   at org.apache.spark.sql.hive.ParquetPartitioningTest.withTempDir(parquetSuites.scala:726)
[info]   at org.apache.spark.sql.hive.ParquetMetastoreSuite$$anonfun$12$$anonfun$apply$mcV$sp$7.apply$mcV$sp(parquetSuites.scala:535)
[info]   at org.apache.spark.sql.test.SQLTestUtils$class.withTable(SQLTestUtils.scala:166)
[info]   at org.apache.spark.sql.hive.ParquetPartitioningTest.withTable(parquetSuites.scala:726)
[info]   at org.apache.spark.sql.hive.ParquetMetastoreSuite$$anonfun$12.apply$mcV$sp(parquetSuites.scala:534)
[info]   at org.apache.spark.sql.hive.ParquetMetastoreSuite$$anonfun$12.apply(parquetSuites.scala:534)
[info]   at org.apache.spark.sql.hive.ParquetMetastoreSuite$$anonfun$12.apply(parquetSuites.scala:534)
```

The solution in this PR to get the paths to list from the partition spec and not rely on the default table path alone.

unit tests.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #13022 from tdas/SPARK-15248.
2016-05-11 12:36:25 -07:00
Dongjoon Hyun e1576478bd [SPARK-14933][HOTFIX] Replace sqlContext with spark.
## What changes were proposed in this pull request?

This fixes compile errors.

## How was this patch tested?

Pass the Jenkins tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13053 from dongjoon-hyun/hotfix_sqlquerysuite.
2016-05-11 10:03:51 -07:00
xin Wu 427c20dd6d [SPARK-14933][SQL] Failed to create view out of a parquet or orc table
## What changes were proposed in this pull request?
#### Symptom
 If a table is created as parquet or ORC table with hive syntaxt DDL, such as
```SQL
create table t1 (c1 int, c2 string) stored as parquet
```
The following command will fail
```SQL
create view v1 as select * from t1
```
#### Root Cause
Currently, `HiveMetaStoreCatalog` converts Paruqet/Orc tables to `LogicalRelation` without giving any `tableIdentifier`. `SQLBuilder` expects the `LogicalRelation` to have an associated `tableIdentifier`. However, the `LogicalRelation` created earlier does not have such a `tableIdentifier`. Thus, `SQLBuilder.toSQL` can not recognize this logical plan and issue an exception.

This PR is to assign a `TableIdentifier` to the `LogicalRelation` when resolving parquet or orc tables in `HiveMetaStoreCatalog`.

## How was this patch tested?
testcases created and dev/run-tests is run.

Author: xin Wu <xinwu@us.ibm.com>

Closes #12716 from xwu0226/SPARK_14933.
2016-05-11 22:17:59 +08:00
Shixiong Zhu 875ef76428 [SPARK-15231][SQL] Document the semantic of saveAsTable and insertInto and don't drop columns silently
## What changes were proposed in this pull request?

This PR adds documents about the different behaviors between `insertInto` and `saveAsTable`, and throws an exception when the user try to add too man columns using `saveAsTable with append`.

## How was this patch tested?

Unit tests added in this PR.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #13013 from zsxwing/SPARK-15231.
2016-05-10 23:53:55 -07:00
Sandeep Singh da02d006bb [SPARK-15249][SQL] Use FunctionResource instead of (String, String) in CreateFunction and CatalogFunction for resource
Use FunctionResource instead of (String, String) in CreateFunction and CatalogFunction for resource
see: TODO's here
https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala#L36
https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/command/functions.scala#L42

Existing tests

Author: Sandeep Singh <sandeep@techaddict.me>

Closes #13024 from techaddict/SPARK-15249.
2016-05-10 14:22:03 -07:00
Dongjoon Hyun db3b4a2015 [SPARK-15037][HOTFIX] Replace sqlContext and sparkSession with spark.
This replaces `sparkSession` with `spark` in CatalogSuite.scala.

Pass the Jenkins tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13030 from dongjoon-hyun/hotfix_sparkSession.
2016-05-10 11:53:44 -07:00
gatorsmile 5c6b085578 [SPARK-14603][SQL] Verification of Metadata Operations by Session Catalog
Since we cannot really trust if the underlying external catalog can throw exceptions when there is an invalid metadata operation, let's do it in SessionCatalog.

- [X] The first step is to unify the error messages issued in Hive-specific Session Catalog and general Session Catalog.
- [X] The second step is to verify the inputs of metadata operations for partitioning-related operations. This is moved to a separate PR: https://github.com/apache/spark/pull/12801
- [X] The third step is to add database existence verification in `SessionCatalog`
- [X] The fourth step is to add table existence verification in `SessionCatalog`
- [X] The fifth step is to add function existence verification in `SessionCatalog`

Add test cases and verify the error messages we issued

Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

Closes #12385 from gatorsmile/verifySessionAPIs.
2016-05-10 11:25:55 -07:00
Sandeep Singh ed0b4070fb [SPARK-15037][SQL][MLLIB] Use SparkSession instead of SQLContext in Scala/Java TestSuites
## What changes were proposed in this pull request?
Use SparkSession instead of SQLContext in Scala/Java TestSuites
as this PR already very big working Python TestSuites in a diff PR.

## How was this patch tested?
Existing tests

Author: Sandeep Singh <sandeep@techaddict.me>

Closes #12907 from techaddict/SPARK-15037.
2016-05-10 11:17:47 -07:00
Herman van Hovell 2646265368 [SPARK-14773] [SPARK-15179] [SQL] Fix SQL building and enable Hive tests
## What changes were proposed in this pull request?
This PR fixes SQL building for predicate subqueries and correlated scalar subqueries. It also enables most Hive subquery tests.

## How was this patch tested?
Enabled new tests in HiveComparisionSuite.

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #12988 from hvanhovell/SPARK-14773.
2016-05-10 09:56:07 -07:00
Cheng Lian 8a12580d25 [SPARK-14127][SQL] "DESC <table>": Extracts schema information from table properties for data source tables
## What changes were proposed in this pull request?

This is a follow-up of #12934 and #12844. This PR adds a set of utility methods in `DDLUtils` to help extract schema information (user-defined schema, partition columns, and bucketing information) from data source table properties. These utility methods are then used in `DescribeTableCommand` to refine output for data source tables. Before this PR, the aforementioned schema information are only shown as table properties, which are hard to read.

Sample output:

```
+----------------------------+---------------------------------------------------------+-------+
|col_name                    |data_type                                                |comment|
+----------------------------+---------------------------------------------------------+-------+
|a                           |bigint                                                   |       |
|b                           |bigint                                                   |       |
|c                           |bigint                                                   |       |
|d                           |bigint                                                   |       |
|# Partition Information     |                                                         |       |
|# col_name                  |                                                         |       |
|d                           |                                                         |       |
|                            |                                                         |       |
|# Detailed Table Information|                                                         |       |
|Database:                   |default                                                  |       |
|Owner:                      |lian                                                     |       |
|Create Time:                |Tue May 10 03:20:34 PDT 2016                             |       |
|Last Access Time:           |Wed Dec 31 16:00:00 PST 1969                             |       |
|Location:                   |file:/Users/lian/local/src/spark/workspace-a/target/...  |       |
|Table Type:                 |MANAGED                                                  |       |
|Table Parameters:           |                                                         |       |
|  rawDataSize               |-1                                                       |       |
|  numFiles                  |1                                                        |       |
|  transient_lastDdlTime     |1462875634                                               |       |
|  totalSize                 |684                                                      |       |
|  spark.sql.sources.provider|parquet                                                  |       |
|  EXTERNAL                  |FALSE                                                    |       |
|  COLUMN_STATS_ACCURATE     |false                                                    |       |
|  numRows                   |-1                                                       |       |
|                            |                                                         |       |
|# Storage Information       |                                                         |       |
|SerDe Library:              |org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe       |       |
|InputFormat:                |org.apache.hadoop.mapred.SequenceFileInputFormat         |       |
|OutputFormat:               |org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat|       |
|Compressed:                 |No                                                       |       |
|Num Buckets:                |2                                                        |       |
|Bucket Columns:             |[b]                                                      |       |
|Sort Columns:               |[c]                                                      |       |
|Storage Desc Parameters:    |                                                         |       |
|  path                      |file:/Users/lian/local/src/spark/workspace-a/target/...  |       |
|  serialization.format      |1                                                        |       |
+----------------------------+---------------------------------------------------------+-------+
```

## How was this patch tested?

Test cases are added in `HiveDDLSuite` to check command output.

Author: Cheng Lian <lian@databricks.com>

Closes #13025 from liancheng/spark-14127-extract-schema-info.
2016-05-10 09:00:53 -07:00
gatorsmile 5706472670 [SPARK-15215][SQL] Fix Explain Parsing and Output
#### What changes were proposed in this pull request?
This PR is to address a few existing issues in `EXPLAIN`:
- The `EXPLAIN` options `LOGICAL | FORMATTED | EXTENDED | CODEGEN` should not be 0 or more match. It should 0 or one match. Parser does not allow users to use more than one option in a single command.
- The option `LOGICAL` is not supported. Issue an exception when users specify this option in the command.
- The output of `EXPLAIN ` contains a weird empty line when the output of analyzed plan is empty. We should remove it. For example:
  ```
  == Parsed Logical Plan ==
  CreateTable CatalogTable(`t`,CatalogTableType(MANAGED),CatalogStorageFormat(None,Some(org.apache.hadoop.mapred.TextInputFormat),Some(org.apache.hadoop.hive.ql.io.  HiveIgnoreKeyTextOutputFormat),None,false,Map()),List(CatalogColumn(col,int,true,None)),List(),List(),List(),-1,,1462725171656,-1,Map(),None,None,None), false

  == Analyzed Logical Plan ==

  CreateTable CatalogTable(`t`,CatalogTableType(MANAGED),CatalogStorageFormat(None,Some(org.apache.hadoop.mapred.TextInputFormat),Some(org.apache.hadoop.hive.ql.io.  HiveIgnoreKeyTextOutputFormat),None,false,Map()),List(CatalogColumn(col,int,true,None)),List(),List(),List(),-1,,1462725171656,-1,Map(),None,None,None), false

  == Optimized Logical Plan ==
  CreateTable CatalogTable(`t`,CatalogTableType(MANAGED),CatalogStorageFormat(None,Some(org.apache.hadoop.mapred.TextInputFormat),Some(org.apache.hadoop.hive.ql.io.  HiveIgnoreKeyTextOutputFormat),None,false,Map()),List(CatalogColumn(col,int,true,None)),List(),List(),List(),-1,,1462725171656,-1,Map(),None,None,None), false
  ...
  ```

#### How was this patch tested?
Added and modified a few test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #12991 from gatorsmile/explainCreateTable.
2016-05-10 11:53:37 +02:00
gatorsmile f45379173b [SPARK-15187][SQL] Disallow Dropping Default Database
#### What changes were proposed in this pull request?
In Hive Metastore, dropping default database is not allowed. However, in `InMemoryCatalog`, this is allowed.

This PR is to disallow users to drop default database.

#### How was this patch tested?
Previously, we already have a test case in HiveDDLSuite. Now, we also add the same one in DDLSuite

Author: gatorsmile <gatorsmile@gmail.com>

Closes #12962 from gatorsmile/dropDefaultDB.
2016-05-10 11:57:01 +08:00
xin Wu 980bba0dcf [SPARK-15025][SQL] fix duplicate of PATH key in datasource table options
## What changes were proposed in this pull request?
The issue is that when the user provides the path option with uppercase "PATH" key, `options` contains `PATH` key and will get into the non-external case in the following code in `createDataSourceTables.scala`, where a new key "path" is created with a default path.
```
val optionsWithPath =
      if (!options.contains("path")) {
        isExternal = false
        options + ("path" -> sessionState.catalog.defaultTablePath(tableIdent))
      } else {
        options
      }
```
So before creating hive table, serdeInfo.parameters will contain both "PATH" and "path" keys and different directories. and Hive table's dataLocation contains the value of "path".

The fix in this PR is to convert `options` in the code above to `CaseInsensitiveMap` before checking for containing "path" key.

## How was this patch tested?
A testcase is added

Author: xin Wu <xinwu@us.ibm.com>

Closes #12804 from xwu0226/SPARK-15025.
2016-05-09 17:18:48 -07:00
Wenchen Fan 2adb11f6db [SPARK-15173][SQL] DataFrameWriter.insertInto should work with datasource table stored in hive
When we parse `CREATE TABLE USING`, we should build a `CreateTableUsing` plan with the `managedIfNoPath` set to true. Then we will add default table path to options when write it to hive.

new test in `SQLQuerySuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #12949 from cloud-fan/bug.
2016-05-09 12:58:27 -07:00
Andrew Or 7bf9b12019 [SPARK-15166][SQL] Move some hive-specific code from SparkSession
## What changes were proposed in this pull request?

This also simplifies the code being moved.

## How was this patch tested?

Existing tests.

Author: Andrew Or <andrew@databricks.com>

Closes #12941 from andrewor14/move-code.
2016-05-09 11:24:58 -07:00
Cheng Lian 671b382a80 [SPARK-14127][SQL] Makes 'DESC [EXTENDED|FORMATTED] <table>' support data source tables
## What changes were proposed in this pull request?

This is a follow-up of PR #12844. It makes the newly updated `DescribeTableCommand` to support data sources tables.

## How was this patch tested?

A test case is added to check `DESC [EXTENDED | FORMATTED] <table>` output.

Author: Cheng Lian <lian@databricks.com>

Closes #12934 from liancheng/spark-14127-desc-table-follow-up.
2016-05-09 10:53:32 -07:00
Ryan Blue 652bbb1bf6 [SPARK-14459][SQL] Detect relation partitioning and adjust the logical plan
## What changes were proposed in this pull request?

This detects a relation's partitioning and adds checks to the analyzer.
If an InsertIntoTable node has no partitioning, it is replaced by the
relation's partition scheme and input columns are correctly adjusted,
placing the partition columns at the end in partition order. If an
InsertIntoTable node has partitioning, it is checked against the table's
reported partitions.

These changes required adding a PartitionedRelation trait to the catalog
interface because Hive's MetastoreRelation doesn't extend
CatalogRelation.

This commit also includes a fix to InsertIntoTable's resolved logic,
which now detects that all expected columns are present, including
dynamic partition columns. Previously, the number of expected columns
was not checked and resolved was true if there were missing columns.

## How was this patch tested?

This adds new tests to the InsertIntoTableSuite that are fixed by this PR.

Author: Ryan Blue <blue@apache.org>

Closes #12239 from rdblue/SPARK-14459-detect-hive-partitioning.
2016-05-09 17:01:23 +08:00
Tathagata Das f7b7ef4166 [SPARK-14997][SQL] Fixed FileCatalog to return correct set of files when there is no partitioning scheme in the given paths
## What changes were proposed in this pull request?
Lets says there are json files in the following directories structure
```
xyz/file0.json
xyz/subdir1/file1.json
xyz/subdir2/file2.json
xyz/subdir1/subsubdir1/file3.json
```
`sqlContext.read.json("xyz")` should read only file0.json according to behavior in Spark 1.6.1. However in current master, all the 4 files are read.

The fix is to make FileCatalog return only the children files of the given path if there is not partitioning detected (instead of all the recursive list of files).

Closes #12774

## How was this patch tested?

unit tests

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #12856 from tdas/SPARK-14997.
2016-05-06 15:04:16 -07:00
gatorsmile 5c8fad7b9b [SPARK-15108][SQL] Describe Permanent UDTF
#### What changes were proposed in this pull request?
When Describe a UDTF, the command returns a wrong result. The command is unable to find the function, which has been created and cataloged in the catalog but not in the functionRegistry.

This PR is to correct it. If the function is not in the functionRegistry, we will check the catalog for collecting the information of the UDTF function.

#### How was this patch tested?
Added test cases to verify the results

Author: gatorsmile <gatorsmile@gmail.com>

Closes #12885 from gatorsmile/showFunction.
2016-05-06 11:43:07 -07:00
hyukjinkwon fa928ff9a3 [SPARK-14962][SQL] Do not push down isnotnull/isnull on unsuportted types in ORC
## What changes were proposed in this pull request?

https://issues.apache.org/jira/browse/SPARK-14962

ORC filters were being pushed down for all types for both `IsNull` and `IsNotNull`.

This is apparently OK because both `IsNull` and `IsNotNull` do not take a type as an argument (Hive 1.2.x) during building filters (`SearchArgument`) in Spark-side but they do not filter correctly because stored statistics always produces `null` for not supported types (eg `ArrayType`) in ORC-side. So, it is always `true` for `IsNull` which ends up with always `false` for `IsNotNull`. (Please see [RecordReaderImpl.java#L296-L318](https://github.com/apache/hive/blob/branch-1.2/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java#L296-L318)  and [RecordReaderImpl.java#L359-L365](https://github.com/apache/hive/blob/branch-1.2/ql/src/java/org/apache/hadoop/hive/ql/io/orc/RecordReaderImpl.java#L359-L365) in Hive 1.2)

This looks prevented in Hive 1.3.x >= by forcing to give a type ([`PredicateLeaf.Type`](e085b7e9bd/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/PredicateLeaf.java (L50-L56))) when building a filter ([`SearchArgument`](26b5c7b56a/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgument.java (L260))) but Hive 1.2.x seems not doing this.

This PR prevents ORC filter creation for `IsNull` and `IsNotNull` on unsupported types. `OrcFilters` resembles `ParquetFilters`.

## How was this patch tested?

Unittests in `OrcQuerySuite` and `OrcFilterSuite` and `sbt scalastyle`.

Author: hyukjinkwon <gurwls223@gmail.com>
Author: Hyukjin Kwon <gurwls223@gmail.com>

Closes #12777 from HyukjinKwon/SPARK-14962.
2016-05-07 01:46:45 +08:00
Dilip Biswal 02c07e8999 [SPARK-14893][SQL] Re-enable HiveSparkSubmitSuite SPARK-8489 test after HiveContext is removed
## What changes were proposed in this pull request?

Enable the test that was disabled when HiveContext was removed.

## How was this patch tested?

Made sure the enabled test passes with the new jar.

Author: Dilip Biswal <dbiswal@us.ibm.com>

Closes #12924 from dilipbiswal/spark-14893.
2016-05-05 14:44:45 -07:00
Shixiong Zhu bb9991dec5 [SPARK-15135][SQL] Make sure SparkSession thread safe
## What changes were proposed in this pull request?

Went through SparkSession and its members and fixed non-thread-safe classes used by SparkSession

## How was this patch tested?

Existing unit tests

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #12915 from zsxwing/spark-session-thread-safe.
2016-05-05 14:36:47 -07:00
Sandeep Singh ed6f3f8a5f [SPARK-15072][SQL][REPL][EXAMPLES] Remove SparkSession.withHiveSupport
## What changes were proposed in this pull request?
Removing the `withHiveSupport` method of `SparkSession`, instead use `enableHiveSupport`

## How was this patch tested?
ran tests locally

Author: Sandeep Singh <sandeep@techaddict.me>

Closes #12851 from techaddict/SPARK-15072.
2016-05-05 14:35:15 -07:00
gatorsmile 8cba57a75c [SPARK-14124][SQL][FOLLOWUP] Implement Database-related DDL Commands
#### What changes were proposed in this pull request?

First, a few test cases failed in mac OS X  because the property value of `java.io.tmpdir` does not include a trailing slash on some platform. Hive always removes the last trailing slash. For example, what I got in the web:
```
Win NT  --> C:\TEMP\
Win XP  --> C:\TEMP
Solaris --> /var/tmp/
Linux   --> /var/tmp
```
Second, a couple of test cases are added to verify if the commands work properly.

#### How was this patch tested?
Added a test case for it and correct the previous test cases.

Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

Closes #12081 from gatorsmile/mkdir.
2016-05-05 14:34:24 -07:00
Sean Zhong 8fb1463d6a [SPARK-6339][SQL] Supports CREATE TEMPORARY VIEW tableIdentifier AS query
## What changes were proposed in this pull request?

This PR support new SQL syntax CREATE TEMPORARY VIEW.
Like:
```
CREATE TEMPORARY VIEW viewName AS SELECT * from xx
CREATE OR REPLACE TEMPORARY VIEW viewName AS SELECT * from xx
CREATE TEMPORARY VIEW viewName (c1 COMMENT 'blabla', c2 COMMENT 'blabla') AS SELECT * FROM xx
```

## How was this patch tested?

Unit tests.

Author: Sean Zhong <clockfly@gmail.com>

Closes #12872 from clockfly/spark-6399.
2016-05-04 18:27:25 -07:00
Tathagata Das 0fd3a47484 [SPARK-15103][SQL] Refactored FileCatalog class to allow StreamFileCatalog to infer partitioning
## What changes were proposed in this pull request?

File Stream Sink writes the list of written files in a metadata log. StreamFileCatalog reads the list of the files for processing. However StreamFileCatalog does not infer partitioning like HDFSFileCatalog.

This PR enables that by refactoring HDFSFileCatalog to create an abstract class PartitioningAwareFileCatalog, that has all the functionality to infer partitions from a list of leaf files.
- HDFSFileCatalog has been renamed to ListingFileCatalog and it extends PartitioningAwareFileCatalog by providing a list of leaf files from recursive directory scanning.
- StreamFileCatalog has been renamed to MetadataLogFileCatalog and it extends PartitioningAwareFileCatalog by providing a list of leaf files from the metadata log.
- The above two classes has been moved into their own files as they are not interfaces that should be in fileSourceInterfaces.scala.

## How was this patch tested?
- FileStreamSinkSuite was update to see if partitioning gets inferred, and on reading whether the partitions get pruned correctly based on the query.
- Other unit tests are unchanged and pass as expected.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #12879 from tdas/SPARK-15103.
2016-05-04 11:02:48 -07:00
Cheng Lian f152fae306 [SPARK-14127][SQL] Native "DESC [EXTENDED | FORMATTED] <table>" DDL command
## What changes were proposed in this pull request?

This PR implements native `DESC [EXTENDED | FORMATTED] <table>` DDL command. Sample output:

```
scala> spark.sql("desc extended src").show(100, truncate = false)
+----------------------------+---------------------------------+-------+
|col_name                    |data_type                        |comment|
+----------------------------+---------------------------------+-------+
|key                         |int                              |       |
|value                       |string                           |       |
|                            |                                 |       |
|# Detailed Table Information|CatalogTable(`default`.`src`, ...|       |
+----------------------------+---------------------------------+-------+

scala> spark.sql("desc formatted src").show(100, truncate = false)
+----------------------------+----------------------------------------------------------+-------+
|col_name                    |data_type                                                 |comment|
+----------------------------+----------------------------------------------------------+-------+
|key                         |int                                                       |       |
|value                       |string                                                    |       |
|                            |                                                          |       |
|# Detailed Table Information|                                                          |       |
|Database:                   |default                                                   |       |
|Owner:                      |lian                                                      |       |
|Create Time:                |Mon Jan 04 17:06:00 CST 2016                              |       |
|Last Access Time:           |Thu Jan 01 08:00:00 CST 1970                              |       |
|Location:                   |hdfs://localhost:9000/user/hive/warehouse_hive121/src     |       |
|Table Type:                 |MANAGED                                                   |       |
|Table Parameters:           |                                                          |       |
|  transient_lastDdlTime     |1451898360                                                |       |
|                            |                                                          |       |
|# Storage Information       |                                                          |       |
|SerDe Library:              |org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe        |       |
|InputFormat:                |org.apache.hadoop.mapred.TextInputFormat                  |       |
|OutputFormat:               |org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat|       |
|Num Buckets:                |-1                                                        |       |
|Bucket Columns:             |[]                                                        |       |
|Sort Columns:               |[]                                                        |       |
|Storage Desc Parameters:    |                                                          |       |
|  serialization.format      |1                                                         |       |
+----------------------------+----------------------------------------------------------+-------+
```

## How was this patch tested?

A test case is added to `HiveDDLSuite` to check command output.

Author: Cheng Lian <lian@databricks.com>

Closes #12844 from liancheng/spark-14127-desc-table.
2016-05-04 16:44:09 +08:00
Wenchen Fan 6c12e801e8 [SPARK-15029] improve error message for Generate
## What changes were proposed in this pull request?

This PR improve the error message for `Generate` in 3 cases:

1. generator is nested in expressions, e.g. `SELECT explode(list) + 1 FROM tbl`
2. generator appears more than one time in SELECT, e.g. `SELECT explode(list), explode(list) FROM tbl`
3. generator appears in other operator which is not project, e.g. `SELECT * FROM tbl SORT BY explode(list)`

## How was this patch tested?

new tests in `AnalysisErrorSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #12810 from cloud-fan/bug.
2016-05-04 00:10:20 -07:00
Cheng Lian bc3760d405 [SPARK-14237][SQL] De-duplicate partition value appending logic in various buildReader() implementations
## What changes were proposed in this pull request?

Currently, various `FileFormat` data sources share approximately the same code for partition value appending. This PR tries to eliminate this duplication.

A new method `buildReaderWithPartitionValues()` is added to `FileFormat` with a default implementation that appends partition values to `InternalRow`s produced by the reader function returned by `buildReader()`.

Special data sources like Parquet, which implements partition value appending inside `buildReader()` because of the vectorized reader, and the Text data source, which doesn't support partitioning, override `buildReaderWithPartitionValues()` and simply delegate to `buildReader()`.

This PR brings two benefits:

1. Apparently, it de-duplicates partition value appending logic

2. Now the reader function returned by `buildReader()` is only required to produce `InternalRow`s rather than `UnsafeRow`s if the data source doesn't override `buildReaderWithPartitionValues()`.

   Because the safe-to-unsafe conversion is also performed while appending partition values. This makes 3rd-party data sources (e.g. spark-avro) easier to implement since they no longer need to access private APIs involving `UnsafeRow`.

## How was this patch tested?

Existing tests should do the work.

Author: Cheng Lian <lian@databricks.com>

Closes #12866 from liancheng/spark-14237-simplify-partition-values-appending.
2016-05-04 14:16:57 +08:00
Andrew Or 6ba17cd147 [SPARK-14414][SQL] Make DDL exceptions more consistent
## What changes were proposed in this pull request?

Just a bunch of small tweaks on DDL exception messages.

## How was this patch tested?

`DDLCommandSuite` et al.

Author: Andrew Or <andrew@databricks.com>

Closes #12853 from andrewor14/make-exceptions-consistent.
2016-05-03 18:07:53 -07:00
Herman van Hovell 1c19c2769e [SPARK-15047][SQL] Cleanup SQL Parser
## What changes were proposed in this pull request?
This PR addresses a few minor issues in SQL parser:

- Removes some unused rules and keywords in the grammar.
- Removes code path for fallback SQL parsing (was needed for Hive native parsing).
- Use `UnresolvedGenerator` instead of hard-coding `Explode` & `JsonTuple`.
- Adds a more generic way of creating error messages for unsupported Hive features.
- Use `visitFunctionName` as much as possible.
- Interpret a `CatalogColumn`'s `DataType` directly instead of parsing it again.

## How was this patch tested?
Existing tests.

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #12826 from hvanhovell/SPARK-15047.
2016-05-02 18:12:31 -07:00
Yin Huai 0182d9599d [SPARK-15034][SPARK-15035][SPARK-15036][SQL] Use spark.sql.warehouse.dir as the warehouse location
This PR contains three changes:
1. We will use spark.sql.warehouse.dir set warehouse location. We will not use hive.metastore.warehouse.dir.
2. SessionCatalog needs to set the location to default db. Otherwise, when creating a table in SparkSession without hive support, the default db's path will be an empty string.
3. When we create a database, we need to make the path qualified.

Existing tests and new tests

Author: Yin Huai <yhuai@databricks.com>

Closes #12812 from yhuai/warehouse.
2016-04-30 18:04:42 -07:00
Reynold Xin 8dc3987d09 [SPARK-15028][SQL] Remove HiveSessionState.setDefaultOverrideConfs
## What changes were proposed in this pull request?
This patch removes some code that are no longer relevant -- mainly HiveSessionState.setDefaultOverrideConfs.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #12806 from rxin/SPARK-15028.
2016-04-30 01:32:00 -07:00
Yin Huai ac41fc648d [SPARK-14591][SQL] Remove DataTypeParser and add more keywords to the nonReserved list.
## What changes were proposed in this pull request?
CatalystSqlParser can parse data types. So, we do not need to have an individual DataTypeParser.

## How was this patch tested?
Existing tests

Author: Yin Huai <yhuai@databricks.com>

Closes #12796 from yhuai/removeDataTypeParser.
2016-04-29 22:49:12 -07:00
hyukjinkwon d7755cfd07 [SPARK-14917][SQL] Enable some ORC compressions tests for writing
## What changes were proposed in this pull request?

https://issues.apache.org/jira/browse/SPARK-14917

As it is described in the JIRA, it seems Hive 1.2.1 which Spark uses now supports snappy and none.

So, this PR enables some tests for writing ORC files with compression codes, `SNAPPY` and `NONE`.

## How was this patch tested?

Unittests in `OrcQuerySuite` and `sbt scalastyle`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #12699 from HyukjinKwon/SPARK-14917.
2016-04-29 21:14:24 -07:00
Andrew Or 66773eb8a5 [SPARK-15012][SQL] Simplify configuration API further
## What changes were proposed in this pull request?

1. Remove all the `spark.setConf` etc. Just expose `spark.conf`
2. Make `spark.conf` take in things set in the core `SparkConf` as well, otherwise users may get confused

This was done for both the Python and Scala APIs.

## How was this patch tested?
`SQLConfSuite`, python tests.

This one fixes the failed tests in #12787

Closes #12787

Author: Andrew Or <andrew@databricks.com>
Author: Yin Huai <yhuai@databricks.com>

Closes #12798 from yhuai/conf-api.
2016-04-29 20:46:07 -07:00
Yin Huai b33d6b7288 [SPARK-15019][SQL] Propagate all Spark Confs to HiveConf created in HiveClientImpl
## What changes were proposed in this pull request?
This PR makes two changes:
1. We will propagate Spark Confs to HiveConf created in HiveClientImpl. So, users can also use spark conf to set warehouse location and metastore url.
2. In sql/hive, HiveClientImpl will be the only place where we create a new HiveConf.

## How was this patch tested?
Existing tests.

Author: Yin Huai <yhuai@databricks.com>

Closes #12791 from yhuai/onlyUseHiveConfInHiveClientImpl.
2016-04-29 17:07:15 -07:00
Yin Huai af32f4aed6 [SPARK-15013][SQL] Remove hiveConf from HiveSessionState
## What changes were proposed in this pull request?
The hiveConf in HiveSessionState is not actually used anymore. Let's remove it.

## How was this patch tested?
Existing tests

Author: Yin Huai <yhuai@databricks.com>

Closes #12786 from yhuai/removeHiveConf.
2016-04-29 14:54:40 -07:00
Cheng Lian a04b1de5fa [SPARK-14981][SQL] Throws exception if DESC is specified for sorting columns
## What changes were proposed in this pull request?

Currently Spark SQL doesn't support sorting columns in descending order. However, the parser accepts the syntax and silently drops sorting directions. This PR fixes this by throwing an exception if `DESC` is specified as sorting direction of a sorting column.

## How was this patch tested?

A test case is added to test the invalid sorting order by checking exception message.

Author: Cheng Lian <lian@databricks.com>

Closes #12759 from liancheng/spark-14981.
2016-04-29 14:52:32 -07:00
Yin Huai ac115f6628 [SPARK-15011][SQL][TEST] Ignore org.apache.spark.sql.hive.StatisticsSuite.analyze MetastoreRelation
This test always fail with sbt's hadoop 2.3 and 2.4 tests. Let'e disable it for now and investigate the problem.

Author: Yin Huai <yhuai@databricks.com>

Closes #12783 from yhuai/SPARK-15011-ignore.
2016-04-29 12:14:28 -07:00
Reynold Xin 054f991c43 [SPARK-14994][SQL] Remove execution hive from HiveSessionState
## What changes were proposed in this pull request?
This patch removes executionHive from HiveSessionState and HiveSharedState.

## How was this patch tested?
Updated test cases.

Author: Reynold Xin <rxin@databricks.com>
Author: Yin Huai <yhuai@databricks.com>

Closes #12770 from rxin/SPARK-14994.
2016-04-29 01:14:02 -07:00
Reynold Xin e249e6f8b5 [HOTFIX] Disable flaky test StatisticsSuite.analyze MetastoreRelations 2016-04-29 00:23:59 -07:00
Reynold Xin 4607f6e7f7 [SPARK-14991][SQL] Remove HiveNativeCommand
## What changes were proposed in this pull request?
This patch removes HiveNativeCommand, so we can continue to remove the dependency on Hive. This pull request also removes the ability to generate golden result file using Hive.

## How was this patch tested?
Updated tests to reflect this.

Author: Reynold Xin <rxin@databricks.com>

Closes #12769 from rxin/SPARK-14991.
2016-04-28 21:58:48 -07:00
Yin Huai 9c7c42bc6a Revert "[SPARK-14613][ML] Add @Since into the matrix and vector classes in spark-mllib-local"
This reverts commit dae538a4d7.
2016-04-28 19:57:41 -07:00
Pravin Gadakh dae538a4d7 [SPARK-14613][ML] Add @Since into the matrix and vector classes in spark-mllib-local
## What changes were proposed in this pull request?

This PR adds `since` tag into the matrix and vector classes in spark-mllib-local.

## How was this patch tested?

Scala-style checks passed.

Author: Pravin Gadakh <prgadakh@in.ibm.com>

Closes #12416 from pravingadakh/SPARK-14613.
2016-04-28 15:59:18 -07:00
Wenchen Fan bf5496dbda [SPARK-14654][CORE] New accumulator API
## What changes were proposed in this pull request?

This PR introduces a new accumulator API  which is much simpler than before:

1. the type hierarchy is simplified, now we only have an `Accumulator` class
2. Combine `initialValue` and `zeroValue` concepts into just one concept: `zeroValue`
3. there in only one `register` method, the accumulator registration and cleanup registration are combined.
4. the `id`,`name` and `countFailedValues` are combined into an `AccumulatorMetadata`, and is provided during registration.

`SQLMetric` is a good example to show the simplicity of this new API.

What we break:

1. no `setValue` anymore. In the new API, the intermedia type can be different from the result type, it's very hard to implement a general `setValue`
2. accumulator can't be serialized before registered.

Problems need to be addressed in follow-ups:

1. with this new API, `AccumulatorInfo` doesn't make a lot of sense, the partial output is not partial updates, we need to expose the intermediate value.
2. `ExceptionFailure` should not carry the accumulator updates. Why do users care about accumulator updates for failed cases? It looks like we only use this feature to update the internal metrics, how about we sending a heartbeat to update internal metrics after the failure event?
3. the public event `SparkListenerTaskEnd` carries a `TaskMetrics`. Ideally this `TaskMetrics` don't need to carry external accumulators, as the only method of `TaskMetrics` that can access external accumulators is `private[spark]`. However, `SQLListener` use it to retrieve sql metrics.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #12612 from cloud-fan/acc.
2016-04-28 00:26:39 -07:00
Andrew Or 37575115b9 [SPARK-14940][SQL] Move ExternalCatalog to own file
## What changes were proposed in this pull request?

`interfaces.scala` was getting big. This just moves the biggest class in there to a new file for cleanliness.

## How was this patch tested?

Just moving things around.

Author: Andrew Or <andrew@databricks.com>

Closes #12721 from andrewor14/move-external-catalog.
2016-04-27 14:17:36 -07:00
Cheng Lian 24bea00047 [SPARK-14954] [SQL] Add PARTITION BY and BUCKET BY clause for data source CTAS syntax
Currently, we can only create persisted partitioned and/or bucketed data source tables using the Dataset API but not using SQL DDL. This PR implements the following syntax to add partitioning and bucketing support to the SQL DDL:

```
CREATE TABLE <table-name>
USING <provider> [OPTIONS (<key1> <value1>, <key2> <value2>, ...)]
[PARTITIONED BY (col1, col2, ...)]
[CLUSTERED BY (col1, col2, ...) [SORTED BY (col1, col2, ...)] INTO <n> BUCKETS]
AS SELECT ...
```

Test cases are added in `MetastoreDataSourcesSuite` to check the newly added syntax.

Author: Cheng Lian <lian@databricks.com>
Author: Yin Huai <yhuai@databricks.com>

Closes #12734 from liancheng/spark-14954.
2016-04-27 13:55:13 -07:00
Reynold Xin ea017b5574 [SPARK-14949][SQL] Remove HiveConf dependency from InsertIntoHiveTable
## What changes were proposed in this pull request?
This patch removes the use of HiveConf from InsertIntoHiveTable. I think this is the last major use of HiveConf and after this we can try to remove the execution HiveConf.

## How was this patch tested?
Internal refactoring and should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #12728 from rxin/SPARK-14949.
2016-04-27 09:30:57 -07:00
Yin Huai 54a3eb8312 [SPARK-14130][SQL] Throw exceptions for ALTER TABLE ADD/REPLACE/CHANGE COLUMN, ALTER TABLE SET FILEFORMAT, DFS, and transaction related commands
## What changes were proposed in this pull request?
This PR will make Spark SQL not allow ALTER TABLE ADD/REPLACE/CHANGE COLUMN, ALTER TABLE SET FILEFORMAT, DFS, and transaction related commands.

## How was this patch tested?
Existing tests. For those tests that I put in the blacklist, I am adding the useful parts back to SQLQuerySuite.

Author: Yin Huai <yhuai@databricks.com>

Closes #12714 from yhuai/banNativeCommand.
2016-04-27 00:30:54 -07:00
Reynold Xin d73d67f623 [SPARK-14944][SPARK-14943][SQL] Remove HiveConf from HiveTableScanExec, HiveTableReader, and ScriptTransformation
## What changes were proposed in this pull request?
This patch removes HiveConf from HiveTableScanExec and HiveTableReader and instead just uses our own configuration system. I'm splitting the large change of removing HiveConf into multiple independent pull requests because it is very difficult to debug test failures when they are all combined in one giant one.

## How was this patch tested?
Should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #12727 from rxin/SPARK-14944.
2016-04-26 23:42:42 -07:00
Reynold Xin 8fda5a73dc [SPARK-14913][SQL] Simplify configuration API
## What changes were proposed in this pull request?
We currently expose both Hadoop configuration and Spark SQL configuration in RuntimeConfig. I think we can remove the Hadoop configuration part, and simply generate Hadoop Configuration on the fly by passing all the SQL configurations into it. This way, there is a single interface (in Java/Scala/Python/SQL) for end-users.

As part of this patch, I also removed some config options deprecated in Spark 1.x.

## How was this patch tested?
Updated relevant tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #12689 from rxin/SPARK-14913.
2016-04-26 22:02:28 -07:00
Andrew Or d8a83a564f [SPARK-13477][SQL] Expose new user-facing Catalog interface
## What changes were proposed in this pull request?

#12625 exposed a new user-facing conf interface in `SparkSession`. This patch adds a catalog interface.

## How was this patch tested?

See `CatalogSuite`.

Author: Andrew Or <andrew@databricks.com>

Closes #12713 from andrewor14/user-facing-catalog.
2016-04-26 21:29:25 -07:00
Dilip Biswal d93976d866 [SPARK-14445][SQL] Support native execution of SHOW COLUMNS and SHOW PARTITIONS
## What changes were proposed in this pull request?
This PR adds Native execution of SHOW COLUMNS and SHOW PARTITION commands.

Command Syntax:
``` SQL
SHOW COLUMNS (FROM | IN) table_identifier [(FROM | IN) database]
```
``` SQL
SHOW PARTITIONS [db_name.]table_name [PARTITION(partition_spec)]
```

## How was this patch tested?

Added test cases in HiveCommandSuite to verify execution and DDLCommandSuite
to verify plans.

Author: Dilip Biswal <dbiswal@us.ibm.com>

Closes #12222 from dilipbiswal/dkb_show_columns.
2016-04-27 09:28:24 +08:00
Sameer Agarwal 9797cc20c0 [SPARK-14929] [SQL] Disable vectorized map for wide schemas & high-precision decimals
## What changes were proposed in this pull request?

While the vectorized hash map in `TungstenAggregate` is currently supported for all primitive data types during partial aggregation, this patch only enables the hash map for a subset of cases that've been verified to show performance improvements on our benchmarks subject to an internal conf that sets an upper limit on the maximum length of the aggregate key/value schema. This list of supported use-cases should be expanded over time.

## How was this patch tested?

This is no new change in functionality so existing tests should suffice. Performance tests were done on TPCDS benchmarks.

Author: Sameer Agarwal <sameer@databricks.com>

Closes #12710 from sameeragarwal/vectorized-enable.
2016-04-26 14:51:14 -07:00
Davies Liu 7131b03bcf [SPARK-14853] [SQL] Support LeftSemi/LeftAnti in SortMergeJoinExec
## What changes were proposed in this pull request?

This PR update SortMergeJoinExec to support LeftSemi/LeftAnti, so it could support all the join types, same as other three join implementations: BroadcastHashJoinExec, ShuffledHashJoinExec,and BroadcastNestedLoopJoinExec.

This PR also simplify the join selection in SparkStrategy.

## How was this patch tested?

Added new tests.

Author: Davies Liu <davies@databricks.com>

Closes #12668 from davies/smj_semi.
2016-04-26 12:43:47 -07:00
Reynold Xin 5cb03220a0 [SPARK-14912][SQL] Propagate data source options to Hadoop configuration
## What changes were proposed in this pull request?
We currently have no way for users to propagate options to the underlying library that rely in Hadoop configurations to work. For example, there are various options in parquet-mr that users might want to set, but the data source API does not expose a per-job way to set it. This patch propagates the user-specified options also into Hadoop Configuration.

## How was this patch tested?
Used a mock data source implementation to test both the read path and the write path.

Author: Reynold Xin <rxin@databricks.com>

Closes #12688 from rxin/SPARK-14912.
2016-04-26 10:58:56 -07:00
Andrew Or 18c2c92580 [SPARK-14861][SQL] Replace internal usages of SQLContext with SparkSession
## What changes were proposed in this pull request?

In Spark 2.0, `SparkSession` is the new thing. Internally we should stop using `SQLContext` everywhere since that's supposed to be not the main user-facing API anymore.

In this patch I took care to not break any public APIs. The one place that's suspect is `o.a.s.ml.source.libsvm.DefaultSource`, but according to mengxr it's not supposed to be public so it's OK to change the underlying `FileFormat` trait.

**Reviewers**: This is a big patch that may be difficult to review but the changes are actually really straightforward. If you prefer I can break it up into a few smaller patches, but it will delay the progress of this issue a little.

## How was this patch tested?

No change in functionality intended.

Author: Andrew Or <andrew@databricks.com>

Closes #12625 from andrewor14/spark-session-refactor.
2016-04-25 20:54:31 -07:00
Andrew Or cfa64882fc [SPARK-14902][SQL] Expose RuntimeConfig in SparkSession
## What changes were proposed in this pull request?

`RuntimeConfig` is the new user-facing API in 2.0 added in #11378. Until now, however, it's been dead code. This patch uses `RuntimeConfig` in `SessionState` and exposes that through the `SparkSession`.

## How was this patch tested?

New test in `SQLContextSuite`.

Author: Andrew Or <andrew@databricks.com>

Closes #12669 from andrewor14/use-runtime-conf.
2016-04-25 17:52:25 -07:00
Reynold Xin f36c9c8379 [SPARK-14888][SQL] UnresolvedFunction should use FunctionIdentifier
## What changes were proposed in this pull request?
This patch changes UnresolvedFunction and UnresolvedGenerator to use a FunctionIdentifier rather than just a String for function name. Also changed SessionCatalog to accept FunctionIdentifier in lookupFunction.

## How was this patch tested?
Updated related unit tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #12659 from rxin/SPARK-14888.
2016-04-25 16:20:57 -07:00
Andrew Or 3c5e65c339 [SPARK-14721][SQL] Remove HiveContext (part 2)
## What changes were proposed in this pull request?

This removes the class `HiveContext` itself along with all code usages associated with it. The bulk of the work was already done in #12485. This is mainly just code cleanup and actually removing the class.

Note: A couple of things will break after this patch. These will be fixed separately.
- the python HiveContext
- all the documentation / comments referencing HiveContext
- there will be no more HiveContext in the REPL (fixed by #12589)

## How was this patch tested?

No change in functionality.

Author: Andrew Or <andrew@databricks.com>

Closes #12585 from andrewor14/delete-hive-context.
2016-04-25 13:23:05 -07:00
gatorsmile 88e54218d5 [SPARK-14892][SQL][TEST] Disable the HiveCompatibilitySuite test case for INPUTDRIVER and OUTPUTDRIVER.
#### What changes were proposed in this pull request?
Disable the test case involving INPUTDRIVER and OUTPUTDRIVER, which are not supported

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #12662 from gatorsmile/disableInOutDriver.
2016-04-25 12:04:16 -07:00
Cheng Lian e66afd5c66 [SPARK-14875][SQL] Makes OutputWriterFactory.newInstance public
## What changes were proposed in this pull request?

This method was accidentally made `private[sql]` in Spark 2.0. This PR makes it public again, since 3rd party data sources like spark-avro depend on it.

## How was this patch tested?

N/A

Author: Cheng Lian <lian@databricks.com>

Closes #12652 from liancheng/spark-14875.
2016-04-25 20:42:49 +08:00
Reynold Xin d0ca5797a8 [SPARK-14876][SQL] SparkSession should be case insensitive by default
## What changes were proposed in this pull request?
This patch changes SparkSession to be case insensitive by default, in order to match other database systems.

## How was this patch tested?
N/A - I'm sure some tests will fail and I will need to fix those.

Author: Reynold Xin <rxin@databricks.com>

Closes #12643 from rxin/SPARK-14876.
2016-04-24 19:38:21 -07:00
Reynold Xin 0c8e5332ff Disable flaky script transformation test 2016-04-24 12:54:56 -07:00
gatorsmile 337289d712 [SPARK-14691][SQL] Simplify and Unify Error Generation for Unsupported Alter Table DDL
#### What changes were proposed in this pull request?
So far, we are capturing each unsupported Alter Table in separate visit functions. They should be unified and issue the same ParseException instead.

This PR is to refactor the existing implementation and make error message consistent for Alter Table DDL.

#### How was this patch tested?
Updated the existing test cases and also added new test cases to ensure all the unsupported statements are covered.

Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

Closes #12459 from gatorsmile/cleanAlterTable.
2016-04-24 18:53:27 +02:00
Yin Huai 1672149c26 [SPARK-14879][SQL] Move CreateMetastoreDataSource and CreateMetastoreDataSourceAsSelect to sql/core
## What changes were proposed in this pull request?

CreateMetastoreDataSource and CreateMetastoreDataSourceAsSelect are not Hive-specific. So, this PR moves them from sql/hive to sql/core. Also, I am adding `Command` suffix to these two classes.

## How was this patch tested?
Existing tests.

Author: Yin Huai <yhuai@databricks.com>

Closes #12645 from yhuai/moveCreateDataSource.
2016-04-23 22:29:31 -07:00
Reynold Xin 162e12b085 [SPARK-14877][SQL] Remove HiveMetastoreTypes class
## What changes were proposed in this pull request?
It is unnecessary as DataType.catalogString largely replaces the need for this class.

## How was this patch tested?
Mostly removing dead code and should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #12644 from rxin/SPARK-14877.
2016-04-23 15:41:17 -07:00
Reynold Xin e3c1366bbc [SPARK-14865][SQL] Better error handling for view creation.
## What changes were proposed in this pull request?
This patch improves error handling in view creation. CreateViewCommand itself will analyze the view SQL query first, and if it cannot successfully analyze it, throw an AnalysisException.

In addition, I also added the following two conservative guards for easier identification of Spark bugs:

1. If there is a bug and the generated view SQL cannot be analyzed, throw an exception at runtime. Note that this is not an AnalysisException because it is not caused by the user and more likely indicate a bug in Spark.
2. SQLBuilder when it gets an unresolved plan, it will also show the plan in the error message.

I also took the chance to simplify the internal implementation of CreateViewCommand, and *removed* a fallback path that would've masked an exception from before.

## How was this patch tested?
1. Added a unit test for the user facing error handling.
2. Manually introduced some bugs in Spark to test the internal defensive error handling.
3. Also added a test case to test nested views (not super relevant).

Author: Reynold Xin <rxin@databricks.com>

Closes #12633 from rxin/SPARK-14865.
2016-04-23 13:19:57 -07:00
Reynold Xin 890abd1279 [SPARK-14869][SQL] Don't mask exceptions in ResolveRelations
## What changes were proposed in this pull request?
In order to support running SQL directly on files, we added some code in ResolveRelations to catch the exception thrown by catalog.lookupRelation and ignore it. This unfortunately masks all the exceptions. This patch changes the logic to simply test the table's existence.

## How was this patch tested?
I manually hacked some bugs into Spark and made sure the exceptions were being propagated up.

Author: Reynold Xin <rxin@databricks.com>

Closes #12634 from rxin/SPARK-14869.
2016-04-23 12:49:36 -07:00
Reynold Xin f0bba7447f Turn script transformation back on.
## What changes were proposed in this pull request?

(Please fill in changes proposed in this fix)

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)

(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

Author: Reynold Xin <rxin@databricks.com>

Closes #12565 from rxin/test-flaky.
2016-04-23 11:11:48 -07:00
Rajesh Balamohan e5226e3007 [SPARK-14551][SQL] Reduce number of NameNode calls in OrcRelation
## What changes were proposed in this pull request?
When FileSourceStrategy is used, record reader is created which incurs a NN call internally. Later in OrcRelation.unwrapOrcStructs, it ends ups reading the file information to get the ObjectInspector. This incurs additional NN call. It would be good to avoid this additional NN call (specifically for partitioned datasets).

Added OrcRecordReader which is very similar to OrcNewInputFormat.OrcRecordReader with an option of exposing the ObjectInspector. This eliminates the need to look up the file later for generating the object inspector. This would be specifically be useful for partitioned tables/datasets.

## How was this patch tested?
Ran tpc-ds queries manually and also verified by running org.apache.spark.sql.hive.orc.OrcSuite,org.apache.spark.sql.hive.orc.OrcQuerySuite,org.apache.spark.sql.hive.orc.OrcPartitionDiscoverySuite,OrcPartitionDiscoverySuite.OrcHadoopFsRelationSuite,org.apache.spark.sql.hive.execution.HiveCompatibilitySuite

…SourceStrategy mode

Author: Rajesh Balamohan <rbalamohan@apache.org>

Closes #12319 from rajeshbalamohan/SPARK-14551.
2016-04-22 22:51:40 -07:00
Reynold Xin 95faa731c1 [SPARK-14866][SQL] Break SQLQuerySuite out into smaller test suites
## What changes were proposed in this pull request?
This patch breaks SQLQuerySuite out into smaller test suites. It was a little bit too large for debugging.

## How was this patch tested?
This is a test only change.

Author: Reynold Xin <rxin@databricks.com>

Closes #12630 from rxin/SPARK-14866.
2016-04-22 22:50:32 -07:00
Reynold Xin c06110187b [SPARK-14842][SQL] Implement view creation in sql/core
## What changes were proposed in this pull request?
This patch re-implements view creation command in sql/core, based on the pre-existing view creation command in the Hive module. This consolidates the view creation logical command and physical command into a single one, called CreateViewCommand.

## How was this patch tested?
All the code should've been tested by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #12615 from rxin/SPARK-14842-2.
2016-04-22 20:30:51 -07:00
Reynold Xin d7d0cad0ad [SPARK-14855][SQL] Add "Exec" suffix to physical operators
## What changes were proposed in this pull request?
This patch adds "Exec" suffix to all physical operators. Before this patch, Spark's physical operators and logical operators are named the same (e.g. Project could be logical.Project or execution.Project), which caused small issues in code review and bigger issues in code refactoring.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #12617 from rxin/exec-node.
2016-04-22 17:43:56 -07:00
Reynold Xin aeb52bea56 [SPARK-14841][SQL] Move SQLBuilder into sql/core
## What changes were proposed in this pull request?
This patch moves SQLBuilder into sql/core so we can in the future move view generation also into sql/core.

## How was this patch tested?
Also moved unit tests.

Author: Reynold Xin <rxin@databricks.com>
Author: Wenchen Fan <wenchen@databricks.com>

Closes #12602 from rxin/SPARK-14841.
2016-04-22 11:10:31 -07:00
Liang-Chi Hsieh e09ab5da8b [SPARK-14609][SQL] Native support for LOAD DATA DDL command
## What changes were proposed in this pull request?

Add the native support for LOAD DATA DDL command that loads data into Hive table/partition.

## How was this patch tested?

`HiveDDLCommandSuite` and `HiveQuerySuite`. Besides, few Hive tests (`WindowQuerySuite`, `HiveTableScanSuite` and `HiveSerDeSuite`) also use `LOAD DATA` command.

Author: Liang-Chi Hsieh <simonh@tw.ibm.com>

Closes #12412 from viirya/ddl-load-data.
2016-04-22 18:26:28 +08:00
Reynold Xin 284b15d2fb [SPARK-14826][SQL] Remove HiveQueryExecution
## What changes were proposed in this pull request?
This patch removes HiveQueryExecution. As part of this, I consolidated all the describe commands into DescribeTableCommand.

## How was this patch tested?
Should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #12588 from rxin/SPARK-14826.
2016-04-22 01:31:13 -07:00
Reynold Xin 3405cc7758 [SPARK-14835][SQL] Remove MetastoreRelation dependency from SQLBuilder
## What changes were proposed in this pull request?
This patch removes SQLBuilder's dependency on MetastoreRelation. We should be able to move SQLBuilder into the sql/core package after this change.

## How was this patch tested?
N/A - covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #12594 from rxin/SPARK-14835.
2016-04-21 21:48:48 -07:00
Cheng Lian 145433f1aa [SPARK-14369] [SQL] Locality support for FileScanRDD
(This PR is a rebased version of PR #12153.)

## What changes were proposed in this pull request?

This PR adds preliminary locality support for `FileFormat` data sources by overriding `FileScanRDD.preferredLocations()`. The strategy can be divided into two parts:

1.  Block location lookup

    Unlike `HadoopRDD` or `NewHadoopRDD`, `FileScanRDD` doesn't have access to the underlying `InputFormat` or `InputSplit`, and thus can't rely on `InputSplit.getLocations()` to gather locality information. Instead, this PR queries block locations using `FileSystem.getBlockLocations()` after listing all `FileStatus`es in `HDFSFileCatalog` and convert all `FileStatus`es into `LocatedFileStatus`es.

    Note that although S3/S3A/S3N file systems don't provide valid locality information, their `getLocatedStatus()` implementations don't actually issue remote calls either. So there's no need to special case these file systems.

2.  Selecting preferred locations

    For each `FilePartition`, we pick up top 3 locations that containing the most data to be retrieved. This isn't necessarily the best algorithm out there. Further improvements may be brought up in follow-up PRs.

## How was this patch tested?

Tested by overriding default `FileSystem` implementation for `file:///` with a mocked one, which returns mocked block locations.

Author: Cheng Lian <lian@databricks.com>

Closes #12527 from liancheng/spark-14369-locality-rebased.
2016-04-21 21:48:09 -07:00
Andrew Or df1953f0df [SPARK-14824][SQL] Rename HiveContext object to HiveUtils
## What changes were proposed in this pull request?

Just a rename so we can get rid of `HiveContext.scala`. Note that this will conflict with #12585.

## How was this patch tested?

No change in functionality.

Author: Andrew Or <andrew@databricks.com>

Closes #12586 from andrewor14/rename-hc-object.
2016-04-21 17:57:59 -07:00
Reynold Xin f181aee07c [SPARK-14821][SQL] Implement AnalyzeTable in sql/core and remove HiveSqlAstBuilder
## What changes were proposed in this pull request?
This patch moves analyze table parsing into SparkSqlAstBuilder and removes HiveSqlAstBuilder.

In order to avoid extensive refactoring, I created a common trait for CatalogRelation and MetastoreRelation, and match on that. In the future we should probably just consolidate the two into a single thing so we don't need this common trait.

## How was this patch tested?
Updated unit tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #12584 from rxin/SPARK-14821.
2016-04-21 17:41:29 -07:00
Reynold Xin 1a95397bb6 [SPARK-14798][SQL] Move native command and script transformation parsing into SparkSqlAstBuilder
## What changes were proposed in this pull request?
This patch moves native command and script transformation into SparkSqlAstBuilder. This builds on #12561. See the last commit for diff.

## How was this patch tested?
Updated test cases to reflect this.

Author: Reynold Xin <rxin@databricks.com>

Closes #12564 from rxin/SPARK-14798.
2016-04-21 15:59:37 -07:00
Reynold Xin 8e1bb0456d [SPARK-14801][SQL] Move MetastoreRelation to its own file
## What changes were proposed in this pull request?
This class is currently in HiveMetastoreCatalog.scala, which is a large file that makes refactoring and searching of usage difficult. Moving it out so I can then do SPARK-14799 and make the review of that simpler.

## How was this patch tested?
N/A - this is a straightforward move and should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #12567 from rxin/SPARK-14801.
2016-04-21 11:54:10 -07:00
Reynold Xin 3a21e8d5ed [SPARK-14795][SQL] Remove the use of Hive's variable substitution
## What changes were proposed in this pull request?
This patch builds on #12556 and completely removes the use of Hive's variable substitution.

## How was this patch tested?
Covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #12561 from rxin/SPARK-14795.
2016-04-21 11:42:25 -07:00
Reynold Xin 79008e6cfd [SPARK-14799][SQL] Remove MetastoreRelation dependency from AnalyzeTable - part 1
## What changes were proposed in this pull request?
This patch isolates AnalyzeTable's dependency on MetastoreRelation into a single line. After this we can work on converging MetastoreRelation and CatalogTable.

## How was this patch tested?
Covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #12566 from rxin/SPARK-14799.
2016-04-21 10:57:16 -07:00
Josh Rosen a70d40314c [SPARK-14783] Preserve full exception stacktrace in IsolatedClientLoader
In IsolatedClientLoader, we have a`catch` block which throws an exception without wrapping the original exception, causing the full exception stacktrace and any nested exceptions to be lost. This patch fixes this, improving the usefulness of classloading error messages.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #12548 from JoshRosen/improve-logging-for-hive-classloader-issues.
2016-04-21 10:43:22 -07:00
Reynold Xin 228128ce25 [SPARK-14794][SQL] Don't pass analyze command into Hive
## What changes were proposed in this pull request?
We shouldn't pass analyze command to Hive because some of those would require running MapReduce jobs. For now, let's just always run the no scan analyze.

## How was this patch tested?
Updated test case to reflect this change.

Author: Reynold Xin <rxin@databricks.com>

Closes #12558 from rxin/parser-analyze.
2016-04-21 00:31:06 -07:00