Commit graph

1229 commits

Author SHA1 Message Date
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