Commit graph

1943 commits

Author SHA1 Message Date
Reynold Xin b7277e03d1 [SPARK-19495][SQL] Make SQLConf slightly more extensible
## What changes were proposed in this pull request?
This pull request makes SQLConf slightly more extensible by removing the visibility limitations on the build* functions.

## How was this patch tested?
N/A - there are no logic changes and everything should be covered by existing unit tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #16835 from rxin/SPARK-19495.
2017-02-07 18:55:19 +01:00
anabranch 7a7ce272fe [SPARK-16609] Add to_date/to_timestamp with format functions
## What changes were proposed in this pull request?

This pull request adds two new user facing functions:
- `to_date` which accepts an expression and a format and returns a date.
- `to_timestamp` which accepts an expression and a format and returns a timestamp.

For example, Given a date in format: `2016-21-05`. (YYYY-dd-MM)

### Date Function
*Previously*
```
to_date(unix_timestamp(lit("2016-21-05"), "yyyy-dd-MM").cast("timestamp"))
```
*Current*
```
to_date(lit("2016-21-05"), "yyyy-dd-MM")
```

### Timestamp Function
*Previously*
```
unix_timestamp(lit("2016-21-05"), "yyyy-dd-MM").cast("timestamp")
```
*Current*
```
to_timestamp(lit("2016-21-05"), "yyyy-dd-MM")
```
### Tasks

- [X] Add `to_date` to Scala Functions
- [x] Add `to_date` to Python Functions
- [x] Add `to_date` to SQL Functions
- [X] Add `to_timestamp` to Scala Functions
- [x] Add `to_timestamp` to Python Functions
- [x] Add `to_timestamp` to SQL Functions
- [x] Add function to R

## How was this patch tested?

- [x] Add Functions to `DateFunctionsSuite`
- Test new `ParseToTimestamp` Expression (*not necessary*)
- Test new `ParseToDate` Expression (*not necessary*)
- [x] Add test for R
- [x] Add test for Python in test.py

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: anabranch <wac.chambers@gmail.com>
Author: Bill Chambers <bill@databricks.com>
Author: anabranch <bill@databricks.com>

Closes #16138 from anabranch/SPARK-16609.
2017-02-07 15:50:30 +01:00
Ala Luszczak 6ed285c68f [SPARK-19447] Fixing input metrics for range operator.
## What changes were proposed in this pull request?

This change introduces a new metric "number of generated rows". It is used exclusively for Range, which is a leaf in the query tree, yet doesn't read any input data, and therefore cannot report "recordsRead".

Additionally the way in which the metrics are reported by the JIT-compiled version of Range was changed. Previously, it was immediately reported that all the records were produced. This could be confusing for a user monitoring execution progress in the UI. Now, the metric is updated gradually.

In order to avoid negative impact on Range performance, the code generation was reworked. The values are now produced in batches in the tighter inner loop, while the metrics are updated in the outer loop.

The change also contains a number of unit tests, which should help ensure the correctness of metrics for various input sources.

## How was this patch tested?

Unit tests.

Author: Ala Luszczak <ala@databricks.com>

Closes #16829 from ala/SPARK-19447.
2017-02-07 14:21:30 +01:00
Wenchen Fan aff53021cf [SPARK-19080][SQL] simplify data source analysis
## What changes were proposed in this pull request?

The current way of resolving `InsertIntoTable` and `CreateTable` is convoluted: sometimes we replace them with concrete implementation commands during analysis, sometimes during planning phase.

And the error checking logic is also a mess: we may put it in extended analyzer rules, or extended checking rules, or `CheckAnalysis`.

This PR simplifies the data source analysis:

1.  `InsertIntoTable` and `CreateTable` are always unresolved and need to be replaced by concrete implementation commands during analysis.
2. The error checking logic is mainly in 2 rules: `PreprocessTableCreation` and `PreprocessTableInsertion`.

## How was this patch tested?

existing test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16269 from cloud-fan/ddl.
2017-02-07 00:36:57 +08:00
gatorsmile 65b10ffb38 [SPARK-19279][SQL] Infer Schema for Hive Serde Tables and Block Creating a Hive Table With an Empty Schema
### What changes were proposed in this pull request?
So far, we allow users to create a table with an empty schema: `CREATE TABLE tab1`. This could break many code paths if we enable it. Thus, we should follow Hive to block it.

For Hive serde tables, some serde libraries require the specified schema and record it in the metastore. To get the list, we need to check `hive.serdes.using.metastore.for.schema,` which contains a list of serdes that require user-specified schema. The default values are

- org.apache.hadoop.hive.ql.io.orc.OrcSerde
- org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
- org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe
- org.apache.hadoop.hive.serde2.dynamic_type.DynamicSerDe
- org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe
- org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe
- org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe
- org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe

### How was this patch tested?
Added test cases for both Hive and data source tables

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16636 from gatorsmile/fixEmptyTableSchema.
2017-02-06 13:30:07 +08:00
hyukjinkwon f1a1f2607d
[SPARK-19402][DOCS] Support LaTex inline formula correctly and fix warnings in Scala/Java APIs generation
## What changes were proposed in this pull request?

This PR proposes three things as below:

- Support LaTex inline-formula, `\( ... \)` in Scala API documentation
  It seems currently,

  ```
  \( ... \)
  ```

  are rendered as they are, for example,

  <img width="345" alt="2017-01-30 10 01 13" src="https://cloud.githubusercontent.com/assets/6477701/22423960/ab37d54a-e737-11e6-9196-4f6229c0189c.png">

  It seems mistakenly more backslashes were added.

- Fix warnings Scaladoc/Javadoc generation
  This PR fixes t two types of warnings as below:

  ```
  [warn] .../spark/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala:335: Could not find any member to link for "UnsupportedOperationException".
  [warn]   /**
  [warn]   ^
  ```

  ```
  [warn] .../spark/sql/core/src/main/scala/org/apache/spark/sql/internal/VariableSubstitution.scala:24: Variable var undefined in comment for class VariableSubstitution in class VariableSubstitution
  [warn]  * `${var}`, `${system:var}` and `${env:var}`.
  [warn]      ^
  ```

- Fix Javadoc8 break
  ```
  [error] .../spark/mllib/target/java/org/apache/spark/ml/PredictionModel.java:7: error: reference not found
  [error]  *                       E.g., {link VectorUDT} for vector features.
  [error]                                       ^
  [error] .../spark/mllib/target/java/org/apache/spark/ml/PredictorParams.java:12: error: reference not found
  [error]    *                          E.g., {link VectorUDT} for vector features.
  [error]                                            ^
  [error] .../spark/mllib/target/java/org/apache/spark/ml/Predictor.java:10: error: reference not found
  [error]  *                       E.g., {link VectorUDT} for vector features.
  [error]                                       ^
  [error] .../spark/sql/hive/target/java/org/apache/spark/sql/hive/HiveAnalysis.java:5: error: reference not found
  [error]  * Note that, this rule must be run after {link PreprocessTableInsertion}.
  [error]                                                  ^
  ```

## How was this patch tested?

Manually via `sbt unidoc` and `jeykil build`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16741 from HyukjinKwon/warn-and-break.
2017-02-01 13:26:16 +00:00
Wenchen Fan f7c07db852 [SPARK-19152][SQL][FOLLOWUP] simplify CreateHiveTableAsSelectCommand
## What changes were proposed in this pull request?

After https://github.com/apache/spark/pull/16552 , `CreateHiveTableAsSelectCommand` becomes very similar to `CreateDataSourceTableAsSelectCommand`, and we can further simplify it by only creating table in the table-not-exist branch.

This PR also adds hive provider checking in DataStream reader/writer, which is missed in #16552

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16693 from cloud-fan/minor.
2017-01-28 20:38:03 -08:00
gatorsmile cfcfc92f7b [SPARK-19359][SQL] Revert Clear useless path after rename a partition with upper-case by HiveExternalCatalog
### What changes were proposed in this pull request?

This PR is to revert the changes made in https://github.com/apache/spark/pull/16700. It could cause the data loss after partition rename, because we have a bug in the file renaming.

Not all the OSs have the same behaviors. For example, on mac OS, if we renaming a path from `.../tbl/a=5/b=6` to `.../tbl/A=5/B=6`. The result is `.../tbl/a=5/B=6`. The expected result is `.../tbl/A=5/B=6`. Thus, renaming on mac OS is not recursive. However, the systems used in Jenkin does not have such an issue. Although this PR is not the root cause, it exposes an existing issue on the code `tablePath.getFileSystem(hadoopConf).rename(wrongPath, rightPath)`

---

Hive metastore is not case preserving and keep partition columns with lower case names.

If SparkSQL create a table with upper-case partion name use HiveExternalCatalog, when we rename partition, it first call the HiveClient to renamePartition, which will create a new lower case partition path, then SparkSql rename the lower case path to the upper-case.

while if the renamed partition contains more than one depth partition ,e.g. A=1/B=2, hive renamePartition change to a=1/b=2, then SparkSql rename it to A=1/B=2, but the a=1 still exists in the filesystem, we should also delete it.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16728 from gatorsmile/revert-pr-16700.
2017-01-28 13:32:30 -08:00
windpiger 1b5ee2003c [SPARK-19359][SQL] clear useless path after rename a partition with upper-case by HiveExternalCatalog
## What changes were proposed in this pull request?

Hive metastore is not case preserving and keep partition columns with lower case names.

If SparkSQL create a table with upper-case partion name use HiveExternalCatalog, when we rename partition, it first call the HiveClient to renamePartition, which will create a new lower case partition path, then SparkSql rename the lower case path to the upper-case.

while if the renamed partition contains more than one depth partition ,e.g. A=1/B=2, hive renamePartition change to a=1/b=2, then SparkSql rename it to A=1/B=2, but the a=1 still exists in the filesystem, we should also delete it.

## How was this patch tested?
unit test added

Author: windpiger <songjun@outlook.com>

Closes #16700 from windpiger/clearUselessPathAfterRenamPartition.
2017-01-27 17:17:17 -08:00
hyukjinkwon 4e35c5a3d3
[SPARK-12970][DOCS] Fix the example in SturctType APIs for Scala and Java
## What changes were proposed in this pull request?

This PR fixes both,

javadoc8 break

```
[error] .../spark/sql/hive/target/java/org/apache/spark/sql/hive/FindHiveSerdeTable.java:3: error: reference not found
[error]  * Replaces {link SimpleCatalogRelation} with {link MetastoreRelation} if its table provider is hive.
```

and the example in `StructType` as a self-contained example as below:

```scala
import org.apache.spark.sql._
import org.apache.spark.sql.types._

val struct =
  StructType(
    StructField("a", IntegerType, true) ::
    StructField("b", LongType, false) ::
    StructField("c", BooleanType, false) :: Nil)

// Extract a single StructField.
val singleField = struct("b")
// singleField: StructField = StructField(b,LongType,false)

// If this struct does not have a field called "d", it throws an exception.
struct("d")
// java.lang.IllegalArgumentException: Field "d" does not exist.
//   ...

// Extract multiple StructFields. Field names are provided in a set.
// A StructType object will be returned.
val twoFields = struct(Set("b", "c"))
// twoFields: StructType =
//   StructType(StructField(b,LongType,false), StructField(c,BooleanType,false))

// Any names without matching fields will throw an exception.
// For the case shown below, an exception is thrown due to "d".
struct(Set("b", "c", "d"))
// java.lang.IllegalArgumentException: Field "d" does not exist.
//    ...
```

```scala
import org.apache.spark.sql._
import org.apache.spark.sql.types._

val innerStruct =
  StructType(
    StructField("f1", IntegerType, true) ::
    StructField("f2", LongType, false) ::
    StructField("f3", BooleanType, false) :: Nil)

val struct = StructType(
  StructField("a", innerStruct, true) :: Nil)

// Create a Row with the schema defined by struct
val row = Row(Row(1, 2, true))
```

Also, now when the column is missing, it throws an exception rather than ignoring.

## How was this patch tested?

Manually via `sbt unidoc`.

- Scaladoc

  <img width="665" alt="2017-01-26 12 54 13" src="https://cloud.githubusercontent.com/assets/6477701/22297905/1245620e-e362-11e6-9e22-43bb8d9871af.png">

- Javadoc

  <img width="722" alt="2017-01-26 12 54 27" src="https://cloud.githubusercontent.com/assets/6477701/22297899/0fd87e0c-e362-11e6-9033-7590bda1aea6.png">

  <img width="702" alt="2017-01-26 12 54 32" src="https://cloud.githubusercontent.com/assets/6477701/22297900/0fe14154-e362-11e6-9882-768381c53163.png">

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16703 from HyukjinKwon/SPARK-12970.
2017-01-27 10:06:54 +00:00
Takuya UESHIN 2969fb4370 [SPARK-18936][SQL] Infrastructure for session local timezone support.
## What changes were proposed in this pull request?

As of Spark 2.1, Spark SQL assumes the machine timezone for datetime manipulation, which is bad if users are not in the same timezones as the machines, or if different users have different timezones.

We should introduce a session local timezone setting that is used for execution.

An explicit non-goal is locale handling.

### Semantics

Setting the session local timezone means that the timezone-aware expressions listed below should use the timezone to evaluate values, and also it should be used to convert (cast) between string and timestamp or between timestamp and date.

- `CurrentDate`
- `CurrentBatchTimestamp`
- `Hour`
- `Minute`
- `Second`
- `DateFormatClass`
- `ToUnixTimestamp`
- `UnixTimestamp`
- `FromUnixTime`

and below are implicitly timezone-aware through cast from timestamp to date:

- `DayOfYear`
- `Year`
- `Quarter`
- `Month`
- `DayOfMonth`
- `WeekOfYear`
- `LastDay`
- `NextDay`
- `TruncDate`

For example, if you have timestamp `"2016-01-01 00:00:00"` in `GMT`, the values evaluated by some of timezone-aware expressions are:

```scala
scala> val df = Seq(new java.sql.Timestamp(1451606400000L)).toDF("ts")
df: org.apache.spark.sql.DataFrame = [ts: timestamp]

scala> df.selectExpr("cast(ts as string)", "year(ts)", "month(ts)", "dayofmonth(ts)", "hour(ts)", "minute(ts)", "second(ts)").show(truncate = false)
+-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+
|ts                 |year(CAST(ts AS DATE))|month(CAST(ts AS DATE))|dayofmonth(CAST(ts AS DATE))|hour(ts)|minute(ts)|second(ts)|
+-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+
|2016-01-01 00:00:00|2016                  |1                      |1                           |0       |0         |0         |
+-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+
```

whereas setting the session local timezone to `"PST"`, they are:

```scala
scala> spark.conf.set("spark.sql.session.timeZone", "PST")

scala> df.selectExpr("cast(ts as string)", "year(ts)", "month(ts)", "dayofmonth(ts)", "hour(ts)", "minute(ts)", "second(ts)").show(truncate = false)
+-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+
|ts                 |year(CAST(ts AS DATE))|month(CAST(ts AS DATE))|dayofmonth(CAST(ts AS DATE))|hour(ts)|minute(ts)|second(ts)|
+-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+
|2015-12-31 16:00:00|2015                  |12                     |31                          |16      |0         |0         |
+-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+
```

Notice that even if you set the session local timezone, it affects only in `DataFrame` operations, neither in `Dataset` operations, `RDD` operations nor in `ScalaUDF`s. You need to properly handle timezone by yourself.

### Design of the fix

I introduced an analyzer to pass session local timezone to timezone-aware expressions and modified DateTimeUtils to take the timezone argument.

## How was this patch tested?

Existing tests and added tests for timezone aware expressions.

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

Closes #16308 from ueshin/issues/SPARK-18350.
2017-01-26 11:51:05 +01:00
Wenchen Fan 59c184e028 [SPARK-17913][SQL] compare atomic and string type column may return confusing result
## What changes were proposed in this pull request?

Spark SQL follows MySQL to do the implicit type conversion for binary comparison: http://dev.mysql.com/doc/refman/5.7/en/type-conversion.html

However, this may return confusing result, e.g. `1 = 'true'` will return true, `19157170390056973L = '19157170390056971'` will return true.

I think it's more reasonable to follow postgres in this case, i.e. cast string to the type of the other side, but return null if the string is not castable to keep hive compatibility.

## How was this patch tested?

newly added tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15880 from cloud-fan/compare.
2017-01-24 10:18:25 -08:00
windpiger 3c86fdddf4 [SPARK-19152][SQL] DataFrameWriter.saveAsTable support hive append
## What changes were proposed in this pull request?

After [SPARK-19107](https://issues.apache.org/jira/browse/SPARK-19107), we now can treat hive as a data source and create hive tables with DataFrameWriter and Catalog. However, the support is not completed, there are still some cases we do not support.

This PR implement:
DataFrameWriter.saveAsTable work with hive format with append mode

## How was this patch tested?
unit test added

Author: windpiger <songjun@outlook.com>

Closes #16552 from windpiger/saveAsTableWithHiveAppend.
2017-01-24 20:40:27 +08:00
jiangxingbo 3bdf3ee860 [SPARK-19272][SQL] Remove the param viewOriginalText from CatalogTable
## What changes were proposed in this pull request?

Hive will expand the view text, so it needs 2 fields: originalText and viewText. Since we don't expand the view text, but only add table properties, perhaps only a single field `viewText` is enough in CatalogTable.

This PR brought in the following changes:
1. Remove the param `viewOriginalText` from `CatalogTable`;
2. Update the output of command `DescribeTableCommand`.

## How was this patch tested?

Tested by exsiting test cases, also updated the failed test cases.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #16679 from jiangxb1987/catalogTable.
2017-01-24 12:37:30 +08:00
Wenchen Fan fcfd5d0bba [SPARK-19290][SQL] add a new extending interface in Analyzer for post-hoc resolution
## What changes were proposed in this pull request?

To implement DDL commands, we added several analyzer rules in sql/hive module to analyze DDL related plans. However, our `Analyzer` currently only have one extending interface: `extendedResolutionRules`, which defines extra rules that will be run together with other rules in the resolution batch, and doesn't fit DDL rules well, because:

1. DDL rules may do some checking and normalization, but we may do it many times as the resolution batch will run rules again and again, until fixed point, and it's hard to tell if a DDL rule has already done its checking and normalization. It's fine because DDL rules are idempotent, but it's bad for analysis performance
2. some DDL rules may depend on others, and it's pretty hard to write `if` conditions to guarantee the dependencies. It will be good if we have a batch which run rules in one pass, so that we can guarantee the dependencies by rules order.

This PR adds a new extending interface in `Analyzer`: `postHocResolutionRules`, which defines rules that will be run only once in a batch runs right after the resolution batch.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16645 from cloud-fan/analyzer.
2017-01-23 20:01:10 -08:00
gatorsmile 772035e771 [SPARK-19229][SQL] Disallow Creating Hive Source Tables when Hive Support is Not Enabled
### What changes were proposed in this pull request?
It is weird to create Hive source tables when using InMemoryCatalog. We are unable to operate it. This PR is to block users to create Hive source tables.

### How was this patch tested?
Fixed the test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16587 from gatorsmile/blockHiveTable.
2017-01-22 20:37:37 -08:00
windpiger aa014eb74b [SPARK-19153][SQL] DataFrameWriter.saveAsTable work with create partitioned table
## What changes were proposed in this pull request?

After [SPARK-19107](https://issues.apache.org/jira/browse/SPARK-19153), we now can treat hive as a data source and create hive tables with DataFrameWriter and Catalog. However, the support is not completed, there are still some cases we do not support.

this PR provide DataFrameWriter.saveAsTable work with hive format to create partitioned table.

## How was this patch tested?
unit test added

Author: windpiger <songjun@outlook.com>

Closes #16593 from windpiger/saveAsTableWithPartitionedTable.
2017-01-22 11:41:27 +08:00
hyukjinkwon 6113fe78a5
[SPARK-19117][SPARK-18922][TESTS] Fix the rest of flaky, newly introduced and missed test failures on Windows
## What changes were proposed in this pull request?

**Failed tests**

```
org.apache.spark.sql.hive.execution.HiveQuerySuite:
 - transform with SerDe3 *** FAILED ***
 - transform with SerDe4 *** FAILED ***
```

```
org.apache.spark.sql.hive.execution.HiveDDLSuite:
 - create hive serde table with new syntax *** FAILED ***
 - add/drop partition with location - managed table *** FAILED ***
```

```
org.apache.spark.sql.hive.ParquetMetastoreSuite:
 - Explicitly added partitions should be readable after load *** FAILED ***
 - Non-partitioned table readable after load *** FAILED ***
```

**Aborted tests**

```
Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.execution.HiveSerDeSuite *** ABORTED *** (157 milliseconds)
   org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: C:projectssparksqlhive   argetscala-2.11   est-classesdatafilessales.txt;
```

**Flaky tests(failed 9ish out of 10)**

```
org.apache.spark.scheduler.SparkListenerSuite:
 - local metrics *** FAILED ***
```

## How was this patch tested?

Manually tested via AppVeyor.

**Failed tests**

```
org.apache.spark.sql.hive.execution.HiveQuerySuite:
 - transform with SerDe3 !!! CANCELED !!! (0 milliseconds)
 - transform with SerDe4 !!! CANCELED !!! (0 milliseconds)
```

```
org.apache.spark.sql.hive.execution.HiveDDLSuite:
 - create hive serde table with new syntax (1 second, 672 milliseconds)
 - add/drop partition with location - managed table (2 seconds, 391 milliseconds)
```

```
org.apache.spark.sql.hive.ParquetMetastoreSuite:
 - Explicitly added partitions should be readable after load (609 milliseconds)
 - Non-partitioned table readable after load (344 milliseconds)
```

**Aborted tests**

```
spark.sql.hive.execution.HiveSerDeSuite:
 - Read with RegexSerDe (2 seconds, 142 milliseconds)
 - Read and write with LazySimpleSerDe (tab separated) (2 seconds)
 - Read with AvroSerDe (1 second, 47 milliseconds)
 - Read Partitioned with AvroSerDe (1 second, 422 milliseconds)
```

**Flaky tests (failed 9ish out of 10)**

```
org.apache.spark.scheduler.SparkListenerSuite:
 - local metrics (4 seconds, 562 milliseconds)
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16586 from HyukjinKwon/set-path-appveyor.
2017-01-21 14:08:01 +00:00
Wenchen Fan 3c2ba9fcc4 [SPARK-19305][SQL] partitioned table should always put partition columns at the end of table schema
## What changes were proposed in this pull request?

For data source tables, we will always reorder the specified table schema, or the query in CTAS, to put partition columns at the end. e.g. `CREATE TABLE t(a int, b int, c int, d int) USING parquet PARTITIONED BY (d, b)` will create a table with schema `<a, c, d, b>`

Hive serde tables don't have this problem before, because its CREATE TABLE syntax specifies data schema and partition schema individually.

However, after we unifed the CREATE TABLE syntax, Hive serde table also need to do the reorder. This PR puts the reorder logic in a analyzer rule,  which works with both data source tables and Hive serde tables.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16655 from cloud-fan/schema.
2017-01-21 13:57:50 +08:00
Wenchen Fan 0bf605c2c6 [SPARK-19292][SQL] filter with partition columns should be case-insensitive on Hive tables
## What changes were proposed in this pull request?

When we query a table with a filter on partitioned columns, we will push the partition filter to the metastore to get matched partitions directly.

In `HiveExternalCatalog.listPartitionsByFilter`, we assume the column names in partition filter are already normalized and we don't need to consider case sensitivity. However, `HiveTableScanExec` doesn't follow this assumption. This PR fixes it.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16647 from cloud-fan/bug.
2017-01-19 20:09:48 -08:00
Yin Huai 63d839028a [SPARK-19295][SQL] IsolatedClientLoader's downloadVersion should log the location of downloaded metastore client jars
## What changes were proposed in this pull request?
This will help the users to know the location of those downloaded jars when `spark.sql.hive.metastore.jars` is set to `maven`.

## How was this patch tested?
jenkins

Author: Yin Huai <yhuai@databricks.com>

Closes #16649 from yhuai/SPARK-19295.
2017-01-19 14:23:36 -08:00
Wenchen Fan 2e62560024 [SPARK-19265][SQL] make table relation cache general and does not depend on hive
## What changes were proposed in this pull request?

We have a table relation plan cache in `HiveMetastoreCatalog`, which caches a lot of things: file status, resolved data source, inferred schema, etc.

However, it doesn't make sense to limit this cache with hive support, we should move it to SQL core module so that users can use this cache without hive support.

It can also reduce the size of `HiveMetastoreCatalog`, so that it's easier to remove it eventually.

main changes:
1. move the table relation cache to `SessionCatalog`
2. `SessionCatalog.lookupRelation` will return `SimpleCatalogRelation` and the analyzer will convert it to `LogicalRelation` or `MetastoreRelation` later, then `HiveSessionCatalog` doesn't need to override `lookupRelation` anymore
3. `FindDataSourceTable` will read/write the table relation cache.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16621 from cloud-fan/plan-cache.
2017-01-19 00:07:48 -08:00
jiangxingbo f85f29608d [SPARK-19024][SQL] Implement new approach to write a permanent view
## What changes were proposed in this pull request?

On CREATE/ALTER a view, it's no longer needed to generate a SQL text string from the LogicalPlan, instead we store the SQL query text、the output column names of the query plan, and current database to CatalogTable. Permanent views created by this approach can be resolved by current view resolution approach.

The main advantage includes:
1. If you update an underlying view, the current view also gets updated;
2. That gives us a change to get ride of SQL generation for operators.

Major changes of this PR:
1. Generate the view-specific properties(e.g. view default database, view query output column names) during permanent view creation and store them as properties in the CatalogTable;
2. Update the commands `CreateViewCommand` and `AlterViewAsCommand`, get rid of SQL generation from them.

## How was this patch tested?
Existing tests.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #16613 from jiangxb1987/view-write-path.
2017-01-18 19:13:01 +08:00
uncleGen eefdf9f9dd
[SPARK-19227][SPARK-19251] remove unused imports and outdated comments
## What changes were proposed in this pull request?
remove ununsed imports and outdated comments, and fix some minor code style issue.

## How was this patch tested?
existing ut

Author: uncleGen <hustyugm@gmail.com>

Closes #16591 from uncleGen/SPARK-19227.
2017-01-18 09:44:32 +00:00
Wenchen Fan 4494cd9716 [SPARK-18243][SQL] Port Hive writing to use FileFormat interface
## What changes were proposed in this pull request?

Inserting data into Hive tables has its own implementation that is distinct from data sources: `InsertIntoHiveTable`, `SparkHiveWriterContainer` and `SparkHiveDynamicPartitionWriterContainer`.

Note that one other major difference is that data source tables write directly to the final destination without using some staging directory, and then Spark itself adds the partitions/tables to the catalog. Hive tables actually write to some staging directory, and then call Hive metastore's loadPartition/loadTable function to load those data in. So we still need to keep `InsertIntoHiveTable` to put this special logic. In the future, we should think of writing to the hive table location directly, so that we don't need to call `loadTable`/`loadPartition` at the end and remove `InsertIntoHiveTable`.

This PR removes `SparkHiveWriterContainer` and `SparkHiveDynamicPartitionWriterContainer`, and create a `HiveFileFormat` to implement the write logic. In the future, we should also implement the read logic in `HiveFileFormat`.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16517 from cloud-fan/insert-hive.
2017-01-17 23:37:59 -08:00
Bogdan Raducanu 2992a0e79e [SPARK-13721][SQL] Support outer generators in DataFrame API
## What changes were proposed in this pull request?

Added outer_explode, outer_posexplode, outer_inline functions and expressions.
Some bug fixing in GenerateExec.scala for CollectionGenerator. Previously it was not correctly handling the case of outer with empty collections, only with nulls.

## How was this patch tested?

New tests added to GeneratorFunctionSuite

Author: Bogdan Raducanu <bogdan.rdc@gmail.com>

Closes #16608 from bogdanrdc/SPARK-13721.
2017-01-17 15:39:24 -08:00
gatorsmile a23debd7bc [SPARK-19129][SQL] SessionCatalog: Disallow empty part col values in partition spec
### What changes were proposed in this pull request?
Empty partition column values are not valid for partition specification. Before this PR, we accept users to do it; however, Hive metastore does not detect and disallow it too. Thus, users hit the following strange error.

```Scala
val df = spark.createDataFrame(Seq((0, "a"), (1, "b"))).toDF("partCol1", "name")
df.write.mode("overwrite").partitionBy("partCol1").saveAsTable("partitionedTable")
spark.sql("alter table partitionedTable drop partition(partCol1='')")
spark.table("partitionedTable").show()
```

In the above example, the WHOLE table is DROPPED when users specify a partition spec containing only one partition column with empty values.

When the partition columns contains more than one, Hive metastore APIs simply ignore the columns with empty values and treat it as partial spec. This is also not expected. This does not follow the actual Hive behaviors. This PR is to disallow users to specify such an invalid partition spec in the `SessionCatalog` APIs.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16583 from gatorsmile/disallowEmptyPartColValue.
2017-01-18 02:01:30 +08:00
Nick Lavers 0019005a2d
[SPARK-19219][SQL] Fix Parquet log output defaults
## What changes were proposed in this pull request?

Changing the default parquet logging levels to reflect the changes made in PR [#15538](https://github.com/apache/spark/pull/15538), in order to prevent the flood of log messages by default.

## How was this patch tested?

Default log output when reading from parquet 1.6 files was compared with and without this change. The change eliminates the extraneous logging and makes the output readable.

Author: Nick Lavers <nick.lavers@videoamp.com>

Closes #16580 from nicklavers/spark-19219-set_default_parquet_log_level.
2017-01-17 12:14:38 +00:00
jiangxingbo e635cbb6e6 [SPARK-18801][SQL][FOLLOWUP] Alias the view with its child
## What changes were proposed in this pull request?

This PR is a follow-up to address the comments https://github.com/apache/spark/pull/16233/files#r95669988 and https://github.com/apache/spark/pull/16233/files#r95662299.

We try to wrap the child by:
1. Generate the `queryOutput` by:
    1.1. If the query column names are defined, map the column names to attributes in the child output by name;
    1.2. Else set the child output attributes to `queryOutput`.
2. Map the `queryQutput` to view output by index, if the corresponding attributes don't match, try to up cast and alias the attribute in `queryOutput` to the attribute in the view output.
3. Add a Project over the child, with the new output generated by the previous steps.
If the view output doesn't have the same number of columns neither with the child output, nor with the query column names, throw an AnalysisException.

## How was this patch tested?

Add new test cases in `SQLViewSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #16561 from jiangxb1987/alias-view.
2017-01-16 19:11:21 +08:00
gatorsmile de62ddf7ff [SPARK-19120] Refresh Metadata Cache After Loading Hive Tables
### What changes were proposed in this pull request?
```Scala
        sql("CREATE TABLE tab (a STRING) STORED AS PARQUET")

        // This table fetch is to fill the cache with zero leaf files
        spark.table("tab").show()

        sql(
          s"""
             |LOAD DATA LOCAL INPATH '$newPartitionDir' OVERWRITE
             |INTO TABLE tab
           """.stripMargin)

        spark.table("tab").show()
```

In the above example, the returned result is empty after table loading. The metadata cache could be out of dated after loading new data into the table, because loading/inserting does not update the cache. So far, the metadata cache is only used for data source tables. Thus, for Hive serde tables, only `parquet` and `orc` formats are facing such issues, because the Hive serde tables in the format of  parquet/orc could be converted to data source tables when `spark.sql.hive.convertMetastoreParquet`/`spark.sql.hive.convertMetastoreOrc` is on.

This PR is to refresh the metadata cache after processing the `LOAD DATA` command.

In addition, Spark SQL does not convert **partitioned** Hive tables (orc/parquet) to data source tables in the write path, but the read path is using the metadata cache for both **partitioned** and non-partitioned Hive tables (orc/parquet). That means, writing the partitioned parquet/orc tables still use `InsertIntoHiveTable`, instead of `InsertIntoHadoopFsRelationCommand`. To avoid reading the out-of-dated cache, `InsertIntoHiveTable` needs to refresh the metadata cache for partitioned tables. Note, it does not need to refresh the cache for non-partitioned parquet/orc tables, because it does not call `InsertIntoHiveTable` at all. Based on the comments, this PR will keep the existing logics unchanged. That means, we always refresh the table no matter whether the table is partitioned or not.

### How was this patch tested?
Added test cases in parquetSuites.scala

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16500 from gatorsmile/refreshInsertIntoHiveTable.
2017-01-15 20:40:44 +08:00
windpiger 8942353905 [SPARK-19151][SQL] DataFrameWriter.saveAsTable support hive overwrite
## What changes were proposed in this pull request?

After [SPARK-19107](https://issues.apache.org/jira/browse/SPARK-19107), we now can treat hive as a data source and create hive tables with DataFrameWriter and Catalog. However, the support is not completed, there are still some cases we do not support.

This PR implement:
DataFrameWriter.saveAsTable work with hive format with overwrite mode

## How was this patch tested?
unit test added

Author: windpiger <songjun@outlook.com>

Closes #16549 from windpiger/saveAsTableWithHiveOverwrite.
2017-01-14 10:53:33 -08:00
gatorsmile 3356b8b6a9 [SPARK-19092][SQL] Save() API of DataFrameWriter should not scan all the saved files
### What changes were proposed in this pull request?
`DataFrameWriter`'s [save() API](5d38f09f47/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala (L207)) is performing a unnecessary full filesystem scan for the saved files. The save() API is the most basic/core API in `DataFrameWriter`. We should avoid it.

The related PR: https://github.com/apache/spark/pull/16090

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16481 from gatorsmile/saveFileScan.
2017-01-13 13:05:53 +08:00
Eric Liang c71b25481a [SPARK-19183][SQL] Add deleteWithJob hook to internal commit protocol API
## What changes were proposed in this pull request?

Currently in SQL we implement overwrites by calling fs.delete() directly on the original data. This is not ideal since we the original files end up deleted even if the job aborts. We should extend the commit protocol to allow file overwrites to be managed as well.

## How was this patch tested?

Existing tests. I also fixed a bunch of tests that were depending on the commit protocol implementation being set to the legacy mapreduce one.

cc rxin cloud-fan

Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekhliang@gmail.com>

Closes #16554 from ericl/add-delete-protocol.
2017-01-12 17:45:55 +08:00
jiangxingbo 30a07071f0 [SPARK-18801][SQL] Support resolve a nested view
## What changes were proposed in this pull request?

We should be able to resolve a nested view. The main advantage is that if you update an underlying view, the current view also gets updated.
The new approach should be compatible with older versions of SPARK/HIVE, that means:
1. The new approach should be able to resolve the views that created by older versions of SPARK/HIVE;
2. The new approach should be able to resolve the views that are currently supported by SPARK SQL.

The new approach mainly brings in the following changes:
1. Add a new operator called `View` to keep track of the CatalogTable that describes the view, and the output attributes as well as the child of the view;
2. Update the `ResolveRelations` rule to resolve the relations and views, note that a nested view should be resolved correctly;
3. Add `viewDefaultDatabase` variable to `CatalogTable` to keep track of the default database name used to resolve a view, if the `CatalogTable` is not a view, then the variable should be `None`;
4. Add `AnalysisContext` to enable us to still support a view created with CTE/Windows query;
5. Enables the view support without enabling Hive support (i.e., enableHiveSupport);
6. Fix a weird behavior: the result of a view query may have different schema if the referenced table has been changed. After this PR, we try to cast the child output attributes to that from the view schema, throw an AnalysisException if cast is not allowed.

Note this is compatible with the views defined by older versions of Spark(before 2.2), which have empty `defaultDatabase` and all the relations in `viewText` have database part defined.

## How was this patch tested?
1. Add new tests in `SessionCatalogSuite` to test the function `lookupRelation`;
2. Add new test case in `SQLViewSuite` to test resolve a nested view.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #16233 from jiangxb1987/resolve-view.
2017-01-11 13:44:07 -08:00
Bryan Cutler 3bc2eff888 [SPARK-17568][CORE][DEPLOY] Add spark-submit option to override ivy settings used to resolve packages/artifacts
## What changes were proposed in this pull request?

Adding option in spark-submit to allow overriding the default IvySettings used to resolve artifacts as part of the Spark Packages functionality.  This will allow all artifact resolution to go through a central managed repository, such as Nexus or Artifactory, where site admins can better approve and control what is used with Spark apps.

This change restructures the creation of the IvySettings object in two distinct ways.  First, if the `spark.ivy.settings` option is not defined then `buildIvySettings` will create a default settings instance, as before, with defined repositories (Maven Central) included.  Second, if the option is defined, the ivy settings file will be loaded from the given path and only repositories defined within will be used for artifact resolution.
## How was this patch tested?

Existing tests for default behaviour, Manual tests that load a ivysettings.xml file with local and Nexus repositories defined.  Added new test to load a simple Ivy settings file with a local filesystem resolver.

Author: Bryan Cutler <cutlerb@gmail.com>
Author: Ian Hummel <ian@themodernlife.net>

Closes #15119 from BryanCutler/spark-custom-IvySettings.
2017-01-11 11:57:38 -08:00
wangzhenhua a615513569 [SPARK-19149][SQL] Unify two sets of statistics in LogicalPlan
## What changes were proposed in this pull request?

Currently we have two sets of statistics in LogicalPlan: a simple stats and a stats estimated by cbo, but the computing logic and naming are quite confusing, we need to unify these two sets of stats.

## How was this patch tested?

Just modify existing tests.

Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>

Closes #16529 from wzhfy/unifyStats.
2017-01-10 22:34:44 -08:00
Wenchen Fan 3b19c74e71 [SPARK-19157][SQL] should be able to change spark.sql.runSQLOnFiles at runtime
## What changes were proposed in this pull request?

The analyzer rule that supports to query files directly will be added to `Analyzer.extendedResolutionRules` when SparkSession is created, according to the `spark.sql.runSQLOnFiles` flag. If the flag is off when we create `SparkSession`, this rule is not added and we can not query files directly even we turn on the flag later.

This PR fixes this bug by always adding that rule to `Analyzer.extendedResolutionRules`.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16531 from cloud-fan/sql-on-files.
2017-01-10 21:33:44 -08:00
hyukjinkwon 2cfd41ac02
[SPARK-19117][TESTS] Skip the tests using script transformation on Windows
## What changes were proposed in this pull request?

This PR proposes to skip the tests for script transformation failed on Windows due to fixed bash location.

```
SQLQuerySuite:
 - script *** FAILED *** (553 milliseconds)
   org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 56.0 failed 1 times, most recent failure: Lost task 0.0 in stage 56.0 (TID 54, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - Star Expansion - script transform *** FAILED *** (2 seconds, 375 milliseconds)
   org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 389.0 failed 1 times, most recent failure: Lost task 0.0 in stage 389.0 (TID 725, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - test script transform for stdout *** FAILED *** (2 seconds, 813 milliseconds)
   org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 391.0 failed 1 times, most recent failure: Lost task 0.0 in stage 391.0 (TID 726, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - test script transform for stderr *** FAILED *** (2 seconds, 407 milliseconds)
   org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 393.0 failed 1 times, most recent failure: Lost task 0.0 in stage 393.0 (TID 727, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - test script transform data type *** FAILED *** (171 milliseconds)
   org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 395.0 failed 1 times, most recent failure: Lost task 0.0 in stage 395.0 (TID 728, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified
```

```
HiveQuerySuite:
 - transform *** FAILED *** (359 milliseconds)
   Failed to execute query using catalyst:
   Error: Job aborted due to stage failure: Task 0 in stage 1347.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1347.0 (TID 2395, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - schema-less transform *** FAILED *** (344 milliseconds)
   Failed to execute query using catalyst:
   Error: Job aborted due to stage failure: Task 0 in stage 1348.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1348.0 (TID 2396, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - transform with custom field delimiter *** FAILED *** (296 milliseconds)
   Failed to execute query using catalyst:
   Error: Job aborted due to stage failure: Task 0 in stage 1349.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1349.0 (TID 2397, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - transform with custom field delimiter2 *** FAILED *** (297 milliseconds)
   Failed to execute query using catalyst:
   Error: Job aborted due to stage failure: Task 0 in stage 1350.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1350.0 (TID 2398, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - transform with custom field delimiter3 *** FAILED *** (312 milliseconds)
   Failed to execute query using catalyst:
   Error: Job aborted due to stage failure: Task 0 in stage 1351.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1351.0 (TID 2399, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - transform with SerDe2 *** FAILED *** (437 milliseconds)
   org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1355.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1355.0 (TID 2403, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified
```

```
LogicalPlanToSQLSuite:
 - script transformation - schemaless *** FAILED *** (78 milliseconds)
   ...
   Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1968.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1968.0 (TID 3932, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified
  - script transformation - alias list *** FAILED *** (94 milliseconds)
   ...
   Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1969.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1969.0 (TID 3933, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - script transformation - alias list with type *** FAILED *** (93 milliseconds)
   ...
   Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1970.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1970.0 (TID 3934, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - script transformation - row format delimited clause with only one format property *** FAILED *** (78 milliseconds)
   ...
   Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1971.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1971.0 (TID 3935, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - script transformation - row format delimited clause with multiple format properties *** FAILED *** (94 milliseconds)
   ...
   Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1972.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1972.0 (TID 3936, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - script transformation - row format serde clauses with SERDEPROPERTIES *** FAILED *** (78 milliseconds)
   ...
   Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1973.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1973.0 (TID 3937, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - script transformation - row format serde clauses without SERDEPROPERTIES *** FAILED *** (78 milliseconds)
   ...
   Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1974.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1974.0 (TID 3938, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified
```

```
ScriptTransformationSuite:
 - cat without SerDe *** FAILED *** (156 milliseconds)
   ...
   Caused by: java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - cat with LazySimpleSerDe *** FAILED *** (63 milliseconds)
    ...
    org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 2383.0 failed 1 times, most recent failure: Lost task 0.0 in stage 2383.0 (TID 4819, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - script transformation should not swallow errors from upstream operators (no serde) *** FAILED *** (78 milliseconds)
    ...
    org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 2384.0 failed 1 times, most recent failure: Lost task 0.0 in stage 2384.0 (TID 4820, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - script transformation should not swallow errors from upstream operators (with serde) *** FAILED *** (47 milliseconds)
    ...
    org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 2385.0 failed 1 times, most recent failure: Lost task 0.0 in stage 2385.0 (TID 4821, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - SPARK-14400 script transformation should fail for bad script command *** FAILED *** (47 milliseconds)
   "Job aborted due to stage failure: Task 0 in stage 2386.0 failed 1 times, most recent failure: Lost task 0.0 in stage 2386.0 (TID 4822, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified
```

## How was this patch tested?

AppVeyor as below:

```
SQLQuerySuite:
  - script !!! CANCELED !!! (63 milliseconds)
  - Star Expansion - script transform !!! CANCELED !!! (0 milliseconds)
  - test script transform for stdout !!! CANCELED !!! (0 milliseconds)
  - test script transform for stderr !!! CANCELED !!! (0 milliseconds)
  - test script transform data type !!! CANCELED !!! (0 milliseconds)
```

```
HiveQuerySuite:
  - transform !!! CANCELED !!! (31 milliseconds)
  - schema-less transform !!! CANCELED !!! (0 milliseconds)
  - transform with custom field delimiter !!! CANCELED !!! (0 milliseconds)
  - transform with custom field delimiter2 !!! CANCELED !!! (0 milliseconds)
  - transform with custom field delimiter3 !!! CANCELED !!! (0 milliseconds)
  - transform with SerDe2 !!! CANCELED !!! (0 milliseconds)
```

```
LogicalPlanToSQLSuite:
  - script transformation - schemaless !!! CANCELED !!! (78 milliseconds)
  - script transformation - alias list !!! CANCELED !!! (0 milliseconds)
  - script transformation - alias list with type !!! CANCELED !!! (0 milliseconds)
  - script transformation - row format delimited clause with only one format property !!! CANCELED !!! (15 milliseconds)
  - script transformation - row format delimited clause with multiple format properties !!! CANCELED !!! (0 milliseconds)
  - script transformation - row format serde clauses with SERDEPROPERTIES !!! CANCELED !!! (0 milliseconds)
  - script transformation - row format serde clauses without SERDEPROPERTIES !!! CANCELED !!! (0 milliseconds)
```

```
ScriptTransformationSuite:
  - cat without SerDe !!! CANCELED !!! (62 milliseconds)
  - cat with LazySimpleSerDe !!! CANCELED !!! (0 milliseconds)
  - script transformation should not swallow errors from upstream operators (no serde) !!! CANCELED !!! (0 milliseconds)
  - script transformation should not swallow errors from upstream operators (with serde) !!! CANCELED !!! (0 milliseconds)
  - SPARK-14400 script transformation should fail for bad script command !!! CANCELED !!! (0 milliseconds)
```

Jenkins tests

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16501 from HyukjinKwon/windows-bash.
2017-01-10 13:22:35 +00:00
hyukjinkwon 4e27578faa
[SPARK-18922][SQL][CORE][STREAMING][TESTS] Fix all identified tests failed due to path and resource-not-closed problems on Windows
## What changes were proposed in this pull request?

This PR proposes to fix all the test failures identified by testing with AppVeyor.

**Scala - aborted tests**

```
WindowQuerySuite:
  Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.execution.WindowQuerySuite *** ABORTED *** (156 milliseconds)
   org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: C:projectssparksqlhive   argetscala-2.11   est-classesdatafilespart_tiny.txt;

OrcSourceSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.orc.OrcSourceSuite *** ABORTED *** (62 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

ParquetMetastoreSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.ParquetMetastoreSuite *** ABORTED *** (4 seconds, 703 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

ParquetSourceSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.ParquetSourceSuite *** ABORTED *** (3 seconds, 907 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark  arget mpspark-581a6575-454f-4f21-a516-a07f95266143;

KafkaRDDSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.KafkaRDDSuite *** ABORTED *** (5 seconds, 212 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-4722304d-213e-4296-b556-951df1a46807

DirectKafkaStreamSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.DirectKafkaStreamSuite *** ABORTED *** (7 seconds, 127 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-d0d3eba7-4215-4e10-b40e-bb797e89338e
   at org.apache.spark.util.Utils$.deleteRecursively(Utils.scala:1010)

ReliableKafkaStreamSuite
 Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.ReliableKafkaStreamSuite *** ABORTED *** (5 seconds, 498 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-d33e45a0-287e-4bed-acae-ca809a89d888

KafkaStreamSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.KafkaStreamSuite *** ABORTED *** (2 seconds, 892 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-59c9d169-5a56-4519-9ef0-cefdbd3f2e6c

KafkaClusterSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.KafkaClusterSuite *** ABORTED *** (1 second, 690 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-3ef402b0-8689-4a60-85ae-e41e274f179d

DirectKafkaStreamSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka010.DirectKafkaStreamSuite *** ABORTED *** (59 seconds, 626 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-426107da-68cf-4d94-b0d6-1f428f1c53f6

KafkaRDDSuite:
Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka010.KafkaRDDSuite *** ABORTED *** (2 minutes, 6 seconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-b9ce7929-5dae-46ab-a0c4-9ef6f58fbc2
```

**Java - failed tests**

```
Test org.apache.spark.streaming.kafka.JavaKafkaRDDSuite.testKafkaRDD failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-1cee32f4-4390-4321-82c9-e8616b3f0fb0, took 9.61 sec

Test org.apache.spark.streaming.kafka.JavaKafkaStreamSuite.testKafkaStream failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-f42695dd-242e-4b07-847c-f299b8e4676e, took 11.797 sec

Test org.apache.spark.streaming.kafka.JavaDirectKafkaStreamSuite.testKafkaStream failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-85c0d062-78cf-459c-a2dd-7973572101ce, took 1.581 sec

Test org.apache.spark.streaming.kafka010.JavaKafkaRDDSuite.testKafkaRDD failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-49eb6b5c-8366-47a6-83f2-80c443c48280, took 17.895 sec

org.apache.spark.streaming.kafka010.JavaDirectKafkaStreamSuite.testKafkaStream failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-898cf826-d636-4b1c-a61a-c12a364c02e7, took 8.858 sec
```

**Scala - failed tests**

```
PartitionProviderCompatibilitySuite:
 - insert overwrite partition of new datasource table overwrites just partition *** FAILED *** (828 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-bb6337b9-4f99-45ab-ad2c-a787ab965c09

 - SPARK-18635 special chars in partition values - partition management true *** FAILED *** (5 seconds, 360 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - SPARK-18635 special chars in partition values - partition management false *** FAILED *** (141 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

```
UtilsSuite:
 - reading offset bytes of a file (compressed) *** FAILED *** (0 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-ecb2b7d5-db8b-43a7-b268-1bf242b5a491

 - reading offset bytes across multiple files (compressed) *** FAILED *** (0 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-25cc47a8-1faa-4da5-8862-cf174df63ce0
```

```
StatisticsSuite:
 - MetastoreRelations fallback to HDFS for size estimation *** FAILED *** (110 milliseconds)
   org.apache.spark.sql.catalyst.analysis.NoSuchTableException: Table or view 'csv_table' not found in database 'default';
```

```
SQLQuerySuite:
 - permanent UDTF *** FAILED *** (125 milliseconds)
   org.apache.spark.sql.AnalysisException: Undefined function: 'udtf_count_temp'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 24

 - describe functions - user defined functions *** FAILED *** (125 milliseconds)
   org.apache.spark.sql.AnalysisException: Undefined function: 'udtf_count'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 7

 - CTAS without serde with location *** FAILED *** (16 milliseconds)
   java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: file:C:projectsspark%09arget%09mpspark-ed673d73-edfc-404e-829e-2e2b9725d94e/c1

 - derived from Hive query file: drop_database_removes_partition_dirs.q *** FAILED *** (47 milliseconds)
   java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: file:C:projectsspark%09arget%09mpspark-d2ddf08e-699e-45be-9ebd-3dfe619680fe/drop_database_removes_partition_dirs_table

 - derived from Hive query file: drop_table_removes_partition_dirs.q *** FAILED *** (0 milliseconds)
   java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: file:C:projectsspark%09arget%09mpspark-d2ddf08e-699e-45be-9ebd-3dfe619680fe/drop_table_removes_partition_dirs_table2

 - SPARK-17796 Support wildcard character in filename for LOAD DATA LOCAL INPATH *** FAILED *** (109 milliseconds)
   java.nio.file.InvalidPathException: Illegal char <:> at index 2: /C:/projects/spark/sql/hive/projectsspark	arget	mpspark-1a122f8c-dfb3-46c4-bab1-f30764baee0e/*part-r*
```

```
HiveDDLSuite:
 - drop external tables in default database *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - add/drop partitions - external table *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - create/drop database - location without pre-created directory *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - create/drop database - location with pre-created directory *** FAILED *** (32 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - drop database containing tables - CASCADE *** FAILED *** (94 milliseconds)
   CatalogDatabase(db1,,file:/C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be/db1.db,Map()) did not equal CatalogDatabase(db1,,file:C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be\db1.db,Map()) (HiveDDLSuite.scala:675)

 - drop an empty database - CASCADE *** FAILED *** (63 milliseconds)
   CatalogDatabase(db1,,file:/C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be/db1.db,Map()) did not equal CatalogDatabase(db1,,file:C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be\db1.db,Map()) (HiveDDLSuite.scala:675)

 - drop database containing tables - RESTRICT *** FAILED *** (47 milliseconds)
   CatalogDatabase(db1,,file:/C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be/db1.db,Map()) did not equal CatalogDatabase(db1,,file:C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be\db1.db,Map()) (HiveDDLSuite.scala:675)

 - drop an empty database - RESTRICT *** FAILED *** (47 milliseconds)
   CatalogDatabase(db1,,file:/C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be/db1.db,Map()) did not equal CatalogDatabase(db1,,file:C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be\db1.db,Map()) (HiveDDLSuite.scala:675)

 - CREATE TABLE LIKE an external data source table *** FAILED *** (140 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-c5eba16d-07ae-4186-95bb-21c5811cf888;

 - CREATE TABLE LIKE an external Hive serde table *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - desc table for data source table - no user-defined schema *** FAILED *** (125 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-e8bf5bf5-721a-4cbe-9d6	at scala.collection.immutable.List.foreach(List.scala:381)d-5543a8301c1d;
```

```
MetastoreDataSourcesSuite
 - CTAS: persisted bucketed data source table *** FAILED *** (16 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
```

```
ShowCreateTableSuite:
 - simple external hive table *** FAILED *** (0 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

```
PartitionedTablePerfStatsSuite:
 - hive table: partitioned pruned table reports only selected files *** FAILED *** (313 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - datasource table: partitioned pruned table reports only selected files *** FAILED *** (219 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-311f45f8-d064-4023-a4bb-e28235bff64d;

 - hive table: lazy partition pruning reads only necessary partition data *** FAILED *** (203 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - datasource table: lazy partition pruning reads only necessary partition data *** FAILED *** (187 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-fde874ca-66bd-4d0b-a40f-a043b65bf957;

 - hive table: lazy partition pruning with file status caching enabled *** FAILED *** (188 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - datasource table: lazy partition pruning with file status caching enabled *** FAILED *** (187 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-e6d20183-dd68-4145-acbe-4a509849accd;

 - hive table: file status caching respects refresh table and refreshByPath *** FAILED *** (172 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - datasource table: file status caching respects refresh table and refreshByPath *** FAILED *** (203 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-8b2c9651-2adf-4d58-874f-659007e21463;

 - hive table: file status cache respects size limit *** FAILED *** (219 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - datasource table: file status cache respects size limit *** FAILED *** (171 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-7835ab57-cb48-4d2c-bb1d-b46d5a4c47e4;

 - datasource table: table setup does not scan filesystem *** FAILED *** (266 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-20598d76-c004-42a7-8061-6c56f0eda5e2;

 - hive table: table setup does not scan filesystem *** FAILED *** (266 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - hive table: num hive client calls does not scale with partition count *** FAILED *** (2 seconds, 281 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - datasource table: num hive client calls does not scale with partition count *** FAILED *** (2 seconds, 422 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-4cfed321-4d1d-4b48-8d34-5c169afff383;

 - hive table: files read and cached when filesource partition management is off *** FAILED *** (234 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - datasource table: all partition data cached in memory when partition management is off *** FAILED *** (203 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-4bcc0398-15c9-4f6a-811e-12d40f3eec12;

 - SPARK-18700: table loaded only once even when resolved concurrently *** FAILED *** (1 second, 266 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

```
HiveSparkSubmitSuite:
 - temporary Hive UDF: define a UDF and use it *** FAILED *** (2 seconds, 94 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - permanent Hive UDF: define a UDF and use it *** FAILED *** (281 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - permanent Hive UDF: use a already defined permanent function *** FAILED *** (718 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-8368: includes jars passed in through --jars *** FAILED *** (3 seconds, 521 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-8020: set sql conf in spark conf *** FAILED *** (0 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-8489: MissingRequirementError during reflection *** FAILED *** (94 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-9757 Persist Parquet relation with decimal column *** FAILED *** (16 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-11009 fix wrong result of Window function in cluster mode *** FAILED *** (16 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-14244 fix window partition size attribute binding failure *** FAILED *** (78 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - set spark.sql.warehouse.dir *** FAILED *** (16 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - set hive.metastore.warehouse.dir *** FAILED *** (15 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-16901: set javax.jdo.option.ConnectionURL *** FAILED *** (16 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-18360: default table path of tables in default database should depend on the location of default database *** FAILED *** (15 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified
```

```
UtilsSuite:
 - resolveURIs with multiple paths *** FAILED *** (0 milliseconds)
   ".../jar3,file:/C:/pi.py[%23]py.pi,file:/C:/path%..." did not equal ".../jar3,file:/C:/pi.py[#]py.pi,file:/C:/path%..." (UtilsSuite.scala:468)
```

```
CheckpointSuite:
 - recovery with file input stream *** FAILED *** (10 seconds, 205 milliseconds)
   The code passed to eventually never returned normally. Attempted 660 times over 10.014272499999999 seconds. Last failure message: Unexpected internal error near index 1
   \
    ^. (CheckpointSuite.scala:680)
```

## How was this patch tested?

Manually via AppVeyor as below:

**Scala - aborted tests**

```
WindowQuerySuite - all passed
OrcSourceSuite:
- SPARK-18220: read Hive orc table with varchar column *** FAILED *** (4 seconds, 417 milliseconds)
  org.apache.spark.sql.execution.QueryExecutionException: FAILED: Execution Error, return code -101 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask. org.apache.hadoop.io.nativeio.NativeIO$Windows.access0(Ljava/lang/String;I)Z
  at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$runHive$1.apply(HiveClientImpl.scala:625)
ParquetMetastoreSuite - all passed
ParquetSourceSuite - all passed
KafkaRDDSuite - all passed
DirectKafkaStreamSuite - all passed
ReliableKafkaStreamSuite - all passed
KafkaStreamSuite - all passed
KafkaClusterSuite - all passed
DirectKafkaStreamSuite - all passed
KafkaRDDSuite - all passed
```

**Java - failed tests**

```
org.apache.spark.streaming.kafka.JavaKafkaRDDSuite - all passed
org.apache.spark.streaming.kafka.JavaDirectKafkaStreamSuite - all passed
org.apache.spark.streaming.kafka.JavaKafkaStreamSuite - all passed
org.apache.spark.streaming.kafka010.JavaDirectKafkaStreamSuite - all passed
org.apache.spark.streaming.kafka010.JavaKafkaRDDSuite - all passed
```

**Scala - failed tests**

```
PartitionProviderCompatibilitySuite:
- insert overwrite partition of new datasource table overwrites just partition (1 second, 953 milliseconds)
- SPARK-18635 special chars in partition values - partition management true (6 seconds, 31 milliseconds)
- SPARK-18635 special chars in partition values - partition management false (4 seconds, 578 milliseconds)
```

```
UtilsSuite:
- reading offset bytes of a file (compressed) (203 milliseconds)
- reading offset bytes across multiple files (compressed) (0 milliseconds)
```

```
StatisticsSuite:
- MetastoreRelations fallback to HDFS for size estimation (94 milliseconds)
```

```
SQLQuerySuite:
 - permanent UDTF (407 milliseconds)
 - describe functions - user defined functions (441 milliseconds)
 - CTAS without serde with location (2 seconds, 831 milliseconds)
 - derived from Hive query file: drop_database_removes_partition_dirs.q (734 milliseconds)
 - derived from Hive query file: drop_table_removes_partition_dirs.q (563 milliseconds)
 - SPARK-17796 Support wildcard character in filename for LOAD DATA LOCAL INPATH (453 milliseconds)
```

```
HiveDDLSuite:
 - drop external tables in default database (3 seconds, 5 milliseconds)
 - add/drop partitions - external table (2 seconds, 750 milliseconds)
 - create/drop database - location without pre-created directory (500 milliseconds)
 - create/drop database - location with pre-created directory (407 milliseconds)
 - drop database containing tables - CASCADE (453 milliseconds)
 - drop an empty database - CASCADE (375 milliseconds)
 - drop database containing tables - RESTRICT (328 milliseconds)
 - drop an empty database - RESTRICT (391 milliseconds)
 - CREATE TABLE LIKE an external data source table (953 milliseconds)
 - CREATE TABLE LIKE an external Hive serde table (3 seconds, 782 milliseconds)
 - desc table for data source table - no user-defined schema (1 second, 150 milliseconds)
```

```
MetastoreDataSourcesSuite
 - CTAS: persisted bucketed data source table (875 milliseconds)
```

```
ShowCreateTableSuite:
 - simple external hive table (78 milliseconds)
```

```
PartitionedTablePerfStatsSuite:
 - hive table: partitioned pruned table reports only selected files (1 second, 109 milliseconds)
- datasource table: partitioned pruned table reports only selected files (860 milliseconds)
 - hive table: lazy partition pruning reads only necessary partition data (859 milliseconds)
 - datasource table: lazy partition pruning reads only necessary partition data (1 second, 219 milliseconds)
 - hive table: lazy partition pruning with file status caching enabled (875 milliseconds)
 - datasource table: lazy partition pruning with file status caching enabled (890 milliseconds)
 - hive table: file status caching respects refresh table and refreshByPath (922 milliseconds)
 - datasource table: file status caching respects refresh table and refreshByPath (640 milliseconds)
 - hive table: file status cache respects size limit (469 milliseconds)
 - datasource table: file status cache respects size limit (453 milliseconds)
 - datasource table: table setup does not scan filesystem (328 milliseconds)
 - hive table: table setup does not scan filesystem (313 milliseconds)
 - hive table: num hive client calls does not scale with partition count (5 seconds, 431 milliseconds)
 - datasource table: num hive client calls does not scale with partition count (4 seconds, 79 milliseconds)
 - hive table: files read and cached when filesource partition management is off (656 milliseconds)
 - datasource table: all partition data cached in memory when partition management is off (484 milliseconds)
 - SPARK-18700: table loaded only once even when resolved concurrently (2 seconds, 578 milliseconds)
```

```
HiveSparkSubmitSuite:
 - temporary Hive UDF: define a UDF and use it (1 second, 745 milliseconds)
 - permanent Hive UDF: define a UDF and use it (406 milliseconds)
 - permanent Hive UDF: use a already defined permanent function (375 milliseconds)
 - SPARK-8368: includes jars passed in through --jars (391 milliseconds)
 - SPARK-8020: set sql conf in spark conf (156 milliseconds)
 - SPARK-8489: MissingRequirementError during reflection (187 milliseconds)
 - SPARK-9757 Persist Parquet relation with decimal column (157 milliseconds)
 - SPARK-11009 fix wrong result of Window function in cluster mode (156 milliseconds)
 - SPARK-14244 fix window partition size attribute binding failure (156 milliseconds)
 - set spark.sql.warehouse.dir (172 milliseconds)
 - set hive.metastore.warehouse.dir (156 milliseconds)
 - SPARK-16901: set javax.jdo.option.ConnectionURL (157 milliseconds)
 - SPARK-18360: default table path of tables in default database should depend on the location of default database (172 milliseconds)
```

```
UtilsSuite:
 - resolveURIs with multiple paths (0 milliseconds)
```

```
CheckpointSuite:
 - recovery with file input stream (4 seconds, 452 milliseconds)
```

Note: after resolving the aborted tests, there is a test failure identified as below:

```
OrcSourceSuite:
- SPARK-18220: read Hive orc table with varchar column *** FAILED *** (4 seconds, 417 milliseconds)
  org.apache.spark.sql.execution.QueryExecutionException: FAILED: Execution Error, return code -101 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask. org.apache.hadoop.io.nativeio.NativeIO$Windows.access0(Ljava/lang/String;I)Z
  at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$runHive$1.apply(HiveClientImpl.scala:625)
```

This does not look due to this problem so this PR does not fix it here.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16451 from HyukjinKwon/all-path-resource-fixes.
2017-01-10 13:19:21 +00:00
Wenchen Fan b0319c2ecb [SPARK-19107][SQL] support creating hive table with DataFrameWriter and Catalog
## What changes were proposed in this pull request?

After unifying the CREATE TABLE syntax in https://github.com/apache/spark/pull/16296, it's pretty easy to support creating hive table with `DataFrameWriter` and `Catalog` now.

This PR basically just removes the hive provider check in `DataFrameWriter.saveAsTable` and `Catalog.createExternalTable`, and add tests.

## How was this patch tested?

new tests in `HiveDDLSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16487 from cloud-fan/hive-table.
2017-01-10 19:26:51 +08:00
Wenchen Fan b3d39620c5 [SPARK-19085][SQL] cleanup OutputWriterFactory and OutputWriter
## What changes were proposed in this pull request?

`OutputWriterFactory`/`OutputWriter` are internal interfaces and we can remove some unnecessary APIs:
1. `OutputWriterFactory.newWriter(path: String)`: no one calls it and no one implements it.
2. `OutputWriter.write(row: Row)`: during execution we only call `writeInternal`, which is weird as `OutputWriter` is already an internal interface. We should rename `writeInternal` to `write` and remove `def write(row: Row)` and it's related converter code. All implementations should just implement `def write(row: InternalRow)`

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16479 from cloud-fan/hive-writer.
2017-01-08 00:42:09 +08:00
Wenchen Fan cca945b6aa [SPARK-18885][SQL] unify CREATE TABLE syntax for data source and hive serde tables
## What changes were proposed in this pull request?

Today we have different syntax to create data source or hive serde tables, we should unify them to not confuse users and step forward to make hive a data source.

Please read https://issues.apache.org/jira/secure/attachment/12843835/CREATE-TABLE.pdf for  details.

TODO(for follow-up PRs):
1. TBLPROPERTIES is not added to the new syntax, we should decide if we wanna add it later.
2. `SHOW CREATE TABLE` should be updated to use the new syntax.
3. we should decide if we wanna change the behavior of `SET LOCATION`.

## How was this patch tested?

new tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16296 from cloud-fan/create-table.
2017-01-05 17:40:27 -08:00
Wenchen Fan 30345c43b7 [SPARK-19058][SQL] fix partition related behaviors with DataFrameWriter.saveAsTable
## What changes were proposed in this pull request?

When we append data to a partitioned table with `DataFrameWriter.saveAsTable`, there are 2 issues:
1. doesn't work when the partition has custom location.
2. will recover all partitions

This PR fixes them by moving the special partition handling code from `DataSourceAnalysis` to `InsertIntoHadoopFsRelationCommand`, so that the `DataFrameWriter.saveAsTable` code path can also benefit from it.

## How was this patch tested?

newly added regression tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16460 from cloud-fan/append.
2017-01-05 14:11:05 +08:00
Niranjan Padmanabhan a1e40b1f5d
[MINOR][DOCS] Remove consecutive duplicated words/typo in Spark Repo
## What changes were proposed in this pull request?
There are many locations in the Spark repo where the same word occurs consecutively. Sometimes they are appropriately placed, but many times they are not. This PR removes the inappropriately duplicated words.

## How was this patch tested?
N/A since only docs or comments were updated.

Author: Niranjan Padmanabhan <niranjan.padmanabhan@gmail.com>

Closes #16455 from neurons/np.structure_streaming_doc.
2017-01-04 15:07:29 +00:00
Wenchen Fan 101556d0fa [SPARK-19060][SQL] remove the supportsPartial flag in AggregateFunction
## What changes were proposed in this pull request?

Now all aggregation functions support partial aggregate, we can remove the `supportsPartual` flag in `AggregateFunction`

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16461 from cloud-fan/partial.
2017-01-04 12:46:30 +01:00
gatorsmile b67b35f76b [SPARK-19048][SQL] Delete Partition Location when Dropping Managed Partitioned Tables in InMemoryCatalog
### What changes were proposed in this pull request?
The data in the managed table should be deleted after table is dropped. However, if the partition location is not under the location of the partitioned table, it is not deleted as expected. Users can specify any location for the partition when they adding a partition.

This PR is to delete partition location when dropping managed partitioned tables stored in `InMemoryCatalog`.

### How was this patch tested?
Added test cases for both HiveExternalCatalog and InMemoryCatalog

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16448 from gatorsmile/unsetSerdeProp.
2017-01-03 11:43:47 -08:00
hyukjinkwon 46b2126024
[SPARK-19002][BUILD][PYTHON] Check pep8 against all Python scripts
## What changes were proposed in this pull request?

This PR proposes to check pep8 against all other Python scripts and fix the errors as below:

```bash
./dev/create-release/generate-contributors.py
./dev/create-release/releaseutils.py
./dev/create-release/translate-contributors.py
./dev/lint-python
./python/docs/epytext.py
./examples/src/main/python/mllib/decision_tree_classification_example.py
./examples/src/main/python/mllib/decision_tree_regression_example.py
./examples/src/main/python/mllib/gradient_boosting_classification_example.py
./examples/src/main/python/mllib/gradient_boosting_regression_example.py
./examples/src/main/python/mllib/linear_regression_with_sgd_example.py
./examples/src/main/python/mllib/logistic_regression_with_lbfgs_example.py
./examples/src/main/python/mllib/naive_bayes_example.py
./examples/src/main/python/mllib/random_forest_classification_example.py
./examples/src/main/python/mllib/random_forest_regression_example.py
./examples/src/main/python/mllib/svm_with_sgd_example.py
./examples/src/main/python/streaming/network_wordjoinsentiments.py
./sql/hive/src/test/resources/data/scripts/cat.py
./sql/hive/src/test/resources/data/scripts/cat_error.py
./sql/hive/src/test/resources/data/scripts/doubleescapedtab.py
./sql/hive/src/test/resources/data/scripts/dumpdata_script.py
./sql/hive/src/test/resources/data/scripts/escapedcarriagereturn.py
./sql/hive/src/test/resources/data/scripts/escapednewline.py
./sql/hive/src/test/resources/data/scripts/escapedtab.py
./sql/hive/src/test/resources/data/scripts/input20_script.py
./sql/hive/src/test/resources/data/scripts/newline.py
```

## How was this patch tested?

- `./python/docs/epytext.py`

  ```bash
  cd ./python/docs $$ make html
  ```

- pep8 check (Python 2.7 / Python 3.3.6)

  ```
  ./dev/lint-python
  ```

- `./dev/merge_spark_pr.py` (Python 2.7 only / Python 3.3.6 not working)

  ```bash
  python -m doctest -v ./dev/merge_spark_pr.py
  ```

- `./dev/create-release/releaseutils.py` `./dev/create-release/generate-contributors.py` `./dev/create-release/translate-contributors.py` (Python 2.7 only / Python 3.3.6 not working)

  ```bash
  python generate-contributors.py
  python translate-contributors.py
  ```

- Examples (Python 2.7 / Python 3.3.6)

  ```bash
  ./bin/spark-submit examples/src/main/python/mllib/decision_tree_classification_example.py
  ./bin/spark-submit examples/src/main/python/mllib/decision_tree_regression_example.py
  ./bin/spark-submit examples/src/main/python/mllib/gradient_boosting_classification_example.py
  ./bin/spark-submit examples/src/main/python/mllib/gradient_boosting_regression_example.p
  ./bin/spark-submit examples/src/main/python/mllib/random_forest_classification_example.py
  ./bin/spark-submit examples/src/main/python/mllib/random_forest_regression_example.py
  ```

- Examples (Python 2.7 only / Python 3.3.6 not working)
  ```
  ./bin/spark-submit examples/src/main/python/mllib/linear_regression_with_sgd_example.py
  ./bin/spark-submit examples/src/main/python/mllib/logistic_regression_with_lbfgs_example.py
  ./bin/spark-submit examples/src/main/python/mllib/naive_bayes_example.py
  ./bin/spark-submit examples/src/main/python/mllib/svm_with_sgd_example.py
  ```

- `sql/hive/src/test/resources/data/scripts/*.py` (Python 2.7 / Python 3.3.6 within suggested changes)

  Manually tested only changed ones.

- `./dev/github_jira_sync.py` (Python 2.7 only / Python 3.3.6 not working)

  Manually tested this after disabling actually adding comments and links.

And also via Jenkins tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16405 from HyukjinKwon/minor-pep8.
2017-01-02 15:23:19 +00:00
gatorsmile 35e974076d [SPARK-19028][SQL] Fixed non-thread-safe functions used in SessionCatalog
### What changes were proposed in this pull request?
Fixed non-thread-safe functions used in SessionCatalog:
- refreshTable
- lookupRelation

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16437 from gatorsmile/addSyncToLookUpTable.
2016-12-31 19:40:28 +08:00
hyukjinkwon 852782b83c
[SPARK-18922][TESTS] Fix more path-related test failures on Windows
## What changes were proposed in this pull request?

This PR proposes to fix the test failures due to different format of paths on Windows.

Failed tests are as below:

```
ColumnExpressionSuite:
- input_file_name, input_file_block_start, input_file_block_length - FileScanRDD *** FAILED *** (187 milliseconds)
  "file:///C:/projects/spark/target/tmp/spark-0b21b963-6cfa-411c-8d6f-e6a5e1e73bce/part-00001-c083a03a-e55e-4b05-9073-451de352d006.snappy.parquet" did not contain "C:\projects\spark\target\tmp\spark-0b21b963-6cfa-411c-8d6f-e6a5e1e73bce" (ColumnExpressionSuite.scala:545)

- input_file_name, input_file_block_start, input_file_block_length - HadoopRDD *** FAILED *** (172 milliseconds)
  "file:/C:/projects/spark/target/tmp/spark-5d0afa94-7c2f-463b-9db9-2e8403e2bc5f/part-00000-f6530138-9ad3-466d-ab46-0eeb6f85ed0b.txt" did not contain "C:\projects\spark\target\tmp\spark-5d0afa94-7c2f-463b-9db9-2e8403e2bc5f" (ColumnExpressionSuite.scala:569)

- input_file_name, input_file_block_start, input_file_block_length - NewHadoopRDD *** FAILED *** (156 milliseconds)
  "file:/C:/projects/spark/target/tmp/spark-a894c7df-c74d-4d19-82a2-a04744cb3766/part-00000-29674e3f-3fcf-4327-9b04-4dab1d46338d.txt" did not contain "C:\projects\spark\target\tmp\spark-a894c7df-c74d-4d19-82a2-a04744cb3766" (ColumnExpressionSuite.scala:598)
```

```
DataStreamReaderWriterSuite:
- source metadataPath *** FAILED *** (62 milliseconds)
  org.mockito.exceptions.verification.junit.ArgumentsAreDifferent: Argument(s) are different! Wanted:
streamSourceProvider.createSource(
    org.apache.spark.sql.SQLContext3b04133b,
    "C:\projects\spark\target\tmp\streaming.metadata-b05db6ae-c8dc-4ce4-b0d9-1eb8c84876c0/sources/0",
    None,
    "org.apache.spark.sql.streaming.test",
    Map()
);
-> at org.apache.spark.sql.streaming.test.DataStreamReaderWriterSuite$$anonfun$12.apply$mcV$sp(DataStreamReaderWriterSuite.scala:374)
Actual invocation has different arguments:
streamSourceProvider.createSource(
    org.apache.spark.sql.SQLContext3b04133b,
    "/C:/projects/spark/target/tmp/streaming.metadata-b05db6ae-c8dc-4ce4-b0d9-1eb8c84876c0/sources/0",
    None,
    "org.apache.spark.sql.streaming.test",
    Map()
);
```

```
GlobalTempViewSuite:
- CREATE GLOBAL TEMP VIEW USING *** FAILED *** (110 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark  arget mpspark-960398ba-a0a1-45f6-a59a-d98533f9f519;
```

```
CreateTableAsSelectSuite:
- CREATE TABLE USING AS SELECT *** FAILED *** (0 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string

- create a table, drop it and create another one with the same name *** FAILED *** (16 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string

- create table using as select - with partitioned by *** FAILED *** (0 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string

- create table using as select - with non-zero buckets *** FAILED *** (0 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string
```

```
HiveMetadataCacheSuite:
- partitioned table is cached when partition pruning is true *** FAILED *** (532 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- partitioned table is cached when partition pruning is false *** FAILED *** (297 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

```
MultiDatabaseSuite:
- createExternalTable() to non-default database - with USE *** FAILED *** (954 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark  arget mpspark-0839d9a7-5e29-467a-9e3e-3e4cd618ee09;

- createExternalTable() to non-default database - without USE *** FAILED *** (500 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark  arget mpspark-c7e24d73-1d8f-45e8-ab7d-53a83087aec3;

 - invalid database name and table names *** FAILED *** (31 milliseconds)
   "Path does not exist: file:/C:projectsspark  arget mpspark-15a2a494-3483-4876-80e5-ec396e704b77;" did not contain "`t:a` is not a valid name for tables/databases. Valid names only contain alphabet characters, numbers and _." (MultiDatabaseSuite.scala:296)
```

```
OrcQuerySuite:
 - SPARK-8501: Avoids discovery schema from empty ORC files *** FAILED *** (15 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - Verify the ORC conversion parameter: CONVERT_METASTORE_ORC *** FAILED *** (78 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - converted ORC table supports resolving mixed case field *** FAILED *** (297 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

```
HadoopFsRelationTest - JsonHadoopFsRelationSuite, OrcHadoopFsRelationSuite, ParquetHadoopFsRelationSuite, SimpleTextHadoopFsRelationSuite:
 - Locality support for FileScanRDD *** FAILED *** (15 milliseconds)
   java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-383d1f13-8783-47fd-964d-9c75e5eec50f, expected: file:///
```

```
HiveQuerySuite:
- CREATE TEMPORARY FUNCTION *** FAILED *** (0 milliseconds)
   java.net.MalformedURLException: For input string: "%5Cprojects%5Cspark%5Csql%5Chive%5Ctarget%5Cscala-2.11%5Ctest-classes%5CTestUDTF.jar"

 - ADD FILE command *** FAILED *** (500 milliseconds)
   java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\sql\hive\target\scala-2.11\test-classes\data\files\v1.txt

 - ADD JAR command 2 *** FAILED *** (110 milliseconds)
   org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: C:projectssparksqlhive  argetscala-2.11 est-classesdatafilessample.json;
```

```
PruneFileSourcePartitionsSuite:
 - PruneFileSourcePartitions should not change the output of LogicalRelation *** FAILED *** (15 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

```
HiveCommandSuite:
 - LOAD DATA LOCAL *** FAILED *** (109 milliseconds)
   org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: C:projectssparksqlhive  argetscala-2.11 est-classesdatafilesemployee.dat;

 - LOAD DATA *** FAILED *** (93 milliseconds)
   java.net.URISyntaxException: Illegal character in opaque part at index 15: C:projectsspark arget mpemployee.dat7496657117354281006.tmp

 - Truncate Table *** FAILED *** (78 milliseconds)
   org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: C:projectssparksqlhive  argetscala-2.11 est-classesdatafilesemployee.dat;
```

```
HiveExternalCatalogBackwardCompatibilitySuite:
- make sure we can read table created by old version of Spark *** FAILED *** (0 milliseconds)
  "[/C:/projects/spark/target/tmp/]spark-0554d859-74e1-..." did not equal "[C:\projects\spark\target\tmp\]spark-0554d859-74e1-..." (HiveExternalCatalogBackwardCompatibilitySuite.scala:213)
  org.scalatest.exceptions.TestFailedException

- make sure we can alter table location created by old version of Spark *** FAILED *** (110 milliseconds)
  java.net.URISyntaxException: Illegal character in opaque part at index 15: C:projectsspark	arget	mpspark-0e9b2c5f-49a1-4e38-a32a-c0ab1813a79f
```

```
ExternalCatalogSuite:
- create/drop/rename partitions should create/delete/rename the directory *** FAILED *** (610 milliseconds)
  java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-4c24f010-18df-437b-9fed-990c6f9adece
```

```
SQLQuerySuite:
- describe functions - temporary user defined functions *** FAILED *** (16 milliseconds)
  java.net.URISyntaxException: Illegal character in opaque part at index 22: C:projectssparksqlhive	argetscala-2.11	est-classesTestUDTF.jar

- specifying database name for a temporary table is not allowed *** FAILED *** (125 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-a34c9814-a483-43f2-be29-37f616b6df91;
```

```
PartitionProviderCompatibilitySuite:
- convert partition provider to hive with repair table *** FAILED *** (281 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-ee5fc96d-8c7d-4ebf-8571-a1d62736473e;

- when partition management is enabled, new tables have partition provider hive *** FAILED *** (187 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-803ad4d6-3e8c-498d-9ca5-5cda5d9b2a48;

- when partition management is disabled, new tables have no partition provider *** FAILED *** (172 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-c9fda9e2-4020-465f-8678-52cd72d0a58f;

- when partition management is disabled, we preserve the old behavior even for new tables *** FAILED *** (203 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget
mpspark-f4a518a6-c49d-43d3-b407-0ddd76948e13;

- insert overwrite partition of legacy datasource table *** FAILED *** (188 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-f4a518a6-c49d-43d3-b407-0ddd76948e79;

- insert overwrite partition of new datasource table overwrites just partition *** FAILED *** (219 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-6ba3a88d-6f6c-42c5-a9f4-6d924a0616ff;

- SPARK-18544 append with saveAsTable - partition management true *** FAILED *** (173 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-cd234a6d-9cb4-4d1d-9e51-854ae9543bbd;

- SPARK-18635 special chars in partition values - partition management true *** FAILED *** (2 seconds, 967 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- SPARK-18635 special chars in partition values - partition management false *** FAILED *** (62 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- SPARK-18659 insert overwrite table with lowercase - partition management true *** FAILED *** (63 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- SPARK-18544 append with saveAsTable - partition management false *** FAILED *** (266 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- SPARK-18659 insert overwrite table files - partition management false *** FAILED *** (63 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- SPARK-18659 insert overwrite table with lowercase - partition management false *** FAILED *** (78 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- sanity check table setup *** FAILED *** (31 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- insert into partial dynamic partitions *** FAILED *** (47 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- insert into fully dynamic partitions *** FAILED *** (62 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- insert into static partition *** FAILED *** (78 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- overwrite partial dynamic partitions *** FAILED *** (63 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- overwrite fully dynamic partitions *** FAILED *** (47 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- overwrite static partition *** FAILED *** (63 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

```
MetastoreDataSourcesSuite:
- check change without refresh *** FAILED *** (203 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-00713fe4-ca04-448c-bfc7-6c5e9a2ad2a1;

- drop, change, recreate *** FAILED *** (78 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-2030a21b-7d67-4385-a65b-bb5e2bed4861;

- SPARK-15269 external data source table creation *** FAILED *** (78 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-4d50fd4a-14bc-41d6-9232-9554dd233f86;

- CTAS *** FAILED *** (109 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string

- CTAS with IF NOT EXISTS *** FAILED *** (109 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string

- CTAS: persisted partitioned bucketed data source table *** FAILED *** (0 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string

- SPARK-15025: create datasource table with path with select *** FAILED *** (16 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string

- CTAS: persisted partitioned data source table *** FAILED *** (47 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string
```

```
HiveMetastoreCatalogSuite:
- Persist non-partitioned parquet relation into metastore as managed table using CTAS *** FAILED *** (16 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string

- Persist non-partitioned orc relation into metastore as managed table using CTAS *** FAILED *** (16 milliseconds)
  java.lang.IllegalArgumentException: Can not create a Path from an empty string
```

```
HiveUDFSuite:
- SPARK-11522 select input_file_name from non-parquet table *** FAILED *** (16 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

```
QueryPartitionSuite:
- SPARK-13709: reading partitioned Avro table with nested schema *** FAILED *** (250 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

```
ParquetHiveCompatibilitySuite:
- simple primitives *** FAILED *** (16 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- SPARK-10177 timestamp *** FAILED *** (0 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- array *** FAILED *** (16 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- map *** FAILED *** (16 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- struct *** FAILED *** (0 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

- SPARK-16344: array of struct with a single field named 'array_element' *** FAILED *** (15 milliseconds)
  org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

## How was this patch tested?

Manually tested via AppVeyor.

```
ColumnExpressionSuite:
- input_file_name, input_file_block_start, input_file_block_length - FileScanRDD (234 milliseconds)
- input_file_name, input_file_block_start, input_file_block_length - HadoopRDD (235 milliseconds)
- input_file_name, input_file_block_start, input_file_block_length - NewHadoopRDD (203 milliseconds)
```

```
DataStreamReaderWriterSuite:
- source metadataPath (63 milliseconds)
```

```
GlobalTempViewSuite:
 - CREATE GLOBAL TEMP VIEW USING (436 milliseconds)
```

```
CreateTableAsSelectSuite:
- CREATE TABLE USING AS SELECT (171 milliseconds)
- create a table, drop it and create another one with the same name (422 milliseconds)
- create table using as select - with partitioned by (141 milliseconds)
- create table using as select - with non-zero buckets (125 milliseconds)
```

```
HiveMetadataCacheSuite:
- partitioned table is cached when partition pruning is true (3 seconds, 211 milliseconds)
- partitioned table is cached when partition pruning is false (1 second, 781 milliseconds)
```

```
MultiDatabaseSuite:
 - createExternalTable() to non-default database - with USE (797 milliseconds)
 - createExternalTable() to non-default database - without USE (640 milliseconds)
 - invalid database name and table names (62 milliseconds)
```

```
OrcQuerySuite:
 - SPARK-8501: Avoids discovery schema from empty ORC files (703 milliseconds)
 - Verify the ORC conversion parameter: CONVERT_METASTORE_ORC (750 milliseconds)
 - converted ORC table supports resolving mixed case field (625 milliseconds)
```

```
HadoopFsRelationTest - JsonHadoopFsRelationSuite, OrcHadoopFsRelationSuite, ParquetHadoopFsRelationSuite, SimpleTextHadoopFsRelationSuite:
 - Locality support for FileScanRDD (296 milliseconds)
```

```
HiveQuerySuite:
 - CREATE TEMPORARY FUNCTION (125 milliseconds)
 - ADD FILE command (250 milliseconds)
 - ADD JAR command 2 (609 milliseconds)
```

```
PruneFileSourcePartitionsSuite:
- PruneFileSourcePartitions should not change the output of LogicalRelation (359 milliseconds)
```

```
HiveCommandSuite:
 - LOAD DATA LOCAL (1 second, 829 milliseconds)
 - LOAD DATA (1 second, 735 milliseconds)
 - Truncate Table (1 second, 641 milliseconds)
```

```
HiveExternalCatalogBackwardCompatibilitySuite:
 - make sure we can read table created by old version of Spark (32 milliseconds)
 - make sure we can alter table location created by old version of Spark (125 milliseconds)
 - make sure we can rename table created by old version of Spark (281 milliseconds)
```

```
ExternalCatalogSuite:
- create/drop/rename partitions should create/delete/rename the directory (625 milliseconds)
```

```
SQLQuerySuite:
- describe functions - temporary user defined functions (31 milliseconds)
- specifying database name for a temporary table is not allowed (390 milliseconds)
```

```
PartitionProviderCompatibilitySuite:
 - convert partition provider to hive with repair table (813 milliseconds)
 - when partition management is enabled, new tables have partition provider hive (562 milliseconds)
 - when partition management is disabled, new tables have no partition provider (344 milliseconds)
 - when partition management is disabled, we preserve the old behavior even for new tables (422 milliseconds)
 - insert overwrite partition of legacy datasource table (750 milliseconds)
 - SPARK-18544 append with saveAsTable - partition management true (985 milliseconds)
 - SPARK-18635 special chars in partition values - partition management true (3 seconds, 328 milliseconds)
 - SPARK-18635 special chars in partition values - partition management false (2 seconds, 891 milliseconds)
 - SPARK-18659 insert overwrite table with lowercase - partition management true (750 milliseconds)
 - SPARK-18544 append with saveAsTable - partition management false (656 milliseconds)
 - SPARK-18659 insert overwrite table files - partition management false (922 milliseconds)
 - SPARK-18659 insert overwrite table with lowercase - partition management false (469 milliseconds)
 - sanity check table setup (937 milliseconds)
 - insert into partial dynamic partitions (2 seconds, 985 milliseconds)
 - insert into fully dynamic partitions (1 second, 937 milliseconds)
 - insert into static partition (1 second, 578 milliseconds)
 - overwrite partial dynamic partitions (7 seconds, 561 milliseconds)
 - overwrite fully dynamic partitions (1 second, 766 milliseconds)
 - overwrite static partition (1 second, 797 milliseconds)
```

```
MetastoreDataSourcesSuite:
 - check change without refresh (610 milliseconds)
 - drop, change, recreate (437 milliseconds)
 - SPARK-15269 external data source table creation (297 milliseconds)
 - CTAS with IF NOT EXISTS (437 milliseconds)
 - CTAS: persisted partitioned bucketed data source table (422 milliseconds)
 - SPARK-15025: create datasource table with path with select (265 milliseconds)
 - CTAS (438 milliseconds)
 - CTAS with IF NOT EXISTS (469 milliseconds)
 - CTAS: persisted partitioned bucketed data source table (406 milliseconds)
```

```
HiveMetastoreCatalogSuite:
 - Persist non-partitioned parquet relation into metastore as managed table using CTAS (406 milliseconds)
 - Persist non-partitioned orc relation into metastore as managed table using CTAS (313 milliseconds)
```

```
HiveUDFSuite:
 - SPARK-11522 select input_file_name from non-parquet table (3 seconds, 144 milliseconds)
```

```
QueryPartitionSuite:
 - SPARK-13709: reading partitioned Avro table with nested schema (1 second, 67 milliseconds)
```

```
ParquetHiveCompatibilitySuite:
 - simple primitives (745 milliseconds)
 - SPARK-10177 timestamp (375 milliseconds)
 - array (407 milliseconds)
 - map (409 milliseconds)
 - struct (437 milliseconds)
 - SPARK-16344: array of struct with a single field named 'array_element' (391 milliseconds)
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16397 from HyukjinKwon/SPARK-18922-paths.
2016-12-30 11:16:03 +00:00
Wenchen Fan 7d19b6ab7d [SPARK-18567][SQL] Simplify CreateDataSourceTableAsSelectCommand
## What changes were proposed in this pull request?

The `CreateDataSourceTableAsSelectCommand` is quite complex now, as it has a lot of work to do if the table already exists:

1. throw exception if we don't want to ignore it.
2. do some check and adjust the schema if we want to append data.
3. drop the table and create it again if we want to overwrite.

The work 2 and 3 should be done by analyzer, so that we can also apply it to hive tables.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15996 from cloud-fan/append.
2016-12-28 21:50:21 -08:00
Kazuaki Ishizaki 93f35569fd [SPARK-16213][SQL] Reduce runtime overhead of a program that creates an primitive array in DataFrame
## What changes were proposed in this pull request?

This PR reduces runtime overhead of a program the creates an primitive array in DataFrame by using the similar approach to #15044. Generated code performs boxing operation in an assignment from InternalRow to an `Object[]` temporary array (at Lines 051 and 061 in the generated code before without this PR). If we know that type of array elements is primitive, we apply the following optimizations:
1. Eliminate a pair of `isNullAt()` and a null assignment
2. Allocate an primitive array instead of `Object[]` (eliminate boxing operations)
3. Create `UnsafeArrayData` by using `UnsafeArrayWriter` to keep a primitive array in a row format instead of doing non-lightweight operations in constructor of `GenericArrayData`
The PR also performs the same things for `CreateMap`.

Here are performance results of [DataFrame programs](6bf54ec5e2/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/PrimitiveArrayBenchmark.scala (L83-L112)) by up to 17.9x over without this PR.

```
Without SPARK-16043
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)
Read a primitive array in DataFrame:     Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                           3805 / 4150          0.0      507308.9       1.0X
Double                                        3593 / 3852          0.0      479056.9       1.1X

With SPARK-16043
Read a primitive array in DataFrame:     Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            213 /  271          0.0       28387.5       1.0X
Double                                         204 /  223          0.0       27250.9       1.0X
```
Note : #15780 is enabled for these measurements

An motivating example

``` java
val df = sparkContext.parallelize(Seq(0.0d, 1.0d), 1).toDF
df.selectExpr("Array(value + 1.1d, value + 2.2d)").show
```

Generated code without this PR

``` java
/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private scala.collection.Iterator inputadapter_input;
/* 009 */   private UnsafeRow serializefromobject_result;
/* 010 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
/* 011 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
/* 012 */   private Object[] project_values;
/* 013 */   private UnsafeRow project_result;
/* 014 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder project_holder;
/* 015 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter project_rowWriter;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter project_arrayWriter;
/* 017 */
/* 018 */   public GeneratedIterator(Object[] references) {
/* 019 */     this.references = references;
/* 020 */   }
/* 021 */
/* 022 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 023 */     partitionIndex = index;
/* 024 */     this.inputs = inputs;
/* 025 */     inputadapter_input = inputs[0];
/* 026 */     serializefromobject_result = new UnsafeRow(1);
/* 027 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 0);
/* 028 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
/* 029 */     this.project_values = null;
/* 030 */     project_result = new UnsafeRow(1);
/* 031 */     this.project_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(project_result, 32);
/* 032 */     this.project_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(project_holder, 1);
/* 033 */     this.project_arrayWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter();
/* 034 */
/* 035 */   }
/* 036 */
/* 037 */   protected void processNext() throws java.io.IOException {
/* 038 */     while (inputadapter_input.hasNext()) {
/* 039 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 040 */       double inputadapter_value = inputadapter_row.getDouble(0);
/* 041 */
/* 042 */       final boolean project_isNull = false;
/* 043 */       this.project_values = new Object[2];
/* 044 */       boolean project_isNull1 = false;
/* 045 */
/* 046 */       double project_value1 = -1.0;
/* 047 */       project_value1 = inputadapter_value + 1.1D;
/* 048 */       if (false) {
/* 049 */         project_values[0] = null;
/* 050 */       } else {
/* 051 */         project_values[0] = project_value1;
/* 052 */       }
/* 053 */
/* 054 */       boolean project_isNull4 = false;
/* 055 */
/* 056 */       double project_value4 = -1.0;
/* 057 */       project_value4 = inputadapter_value + 2.2D;
/* 058 */       if (false) {
/* 059 */         project_values[1] = null;
/* 060 */       } else {
/* 061 */         project_values[1] = project_value4;
/* 062 */       }
/* 063 */
/* 064 */       final ArrayData project_value = new org.apache.spark.sql.catalyst.util.GenericArrayData(project_values);
/* 065 */       this.project_values = null;
/* 066 */       project_holder.reset();
/* 067 */
/* 068 */       project_rowWriter.zeroOutNullBytes();
/* 069 */
/* 070 */       if (project_isNull) {
/* 071 */         project_rowWriter.setNullAt(0);
/* 072 */       } else {
/* 073 */         // Remember the current cursor so that we can calculate how many bytes are
/* 074 */         // written later.
/* 075 */         final int project_tmpCursor = project_holder.cursor;
/* 076 */
/* 077 */         if (project_value instanceof UnsafeArrayData) {
/* 078 */           final int project_sizeInBytes = ((UnsafeArrayData) project_value).getSizeInBytes();
/* 079 */           // grow the global buffer before writing data.
/* 080 */           project_holder.grow(project_sizeInBytes);
/* 081 */           ((UnsafeArrayData) project_value).writeToMemory(project_holder.buffer, project_holder.cursor);
/* 082 */           project_holder.cursor += project_sizeInBytes;
/* 083 */
/* 084 */         } else {
/* 085 */           final int project_numElements = project_value.numElements();
/* 086 */           project_arrayWriter.initialize(project_holder, project_numElements, 8);
/* 087 */
/* 088 */           for (int project_index = 0; project_index < project_numElements; project_index++) {
/* 089 */             if (project_value.isNullAt(project_index)) {
/* 090 */               project_arrayWriter.setNullDouble(project_index);
/* 091 */             } else {
/* 092 */               final double project_element = project_value.getDouble(project_index);
/* 093 */               project_arrayWriter.write(project_index, project_element);
/* 094 */             }
/* 095 */           }
/* 096 */         }
/* 097 */
/* 098 */         project_rowWriter.setOffsetAndSize(0, project_tmpCursor, project_holder.cursor - project_tmpCursor);
/* 099 */       }
/* 100 */       project_result.setTotalSize(project_holder.totalSize());
/* 101 */       append(project_result);
/* 102 */       if (shouldStop()) return;
/* 103 */     }
/* 104 */   }
/* 105 */ }
```

Generated code with this PR

``` java
/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private scala.collection.Iterator inputadapter_input;
/* 009 */   private UnsafeRow serializefromobject_result;
/* 010 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
/* 011 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
/* 012 */   private UnsafeArrayData project_arrayData;
/* 013 */   private UnsafeRow project_result;
/* 014 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder project_holder;
/* 015 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter project_rowWriter;
/* 016 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter project_arrayWriter;
/* 017 */
/* 018 */   public GeneratedIterator(Object[] references) {
/* 019 */     this.references = references;
/* 020 */   }
/* 021 */
/* 022 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 023 */     partitionIndex = index;
/* 024 */     this.inputs = inputs;
/* 025 */     inputadapter_input = inputs[0];
/* 026 */     serializefromobject_result = new UnsafeRow(1);
/* 027 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 0);
/* 028 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
/* 029 */
/* 030 */     project_result = new UnsafeRow(1);
/* 031 */     this.project_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(project_result, 32);
/* 032 */     this.project_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(project_holder, 1);
/* 033 */     this.project_arrayWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter();
/* 034 */
/* 035 */   }
/* 036 */
/* 037 */   protected void processNext() throws java.io.IOException {
/* 038 */     while (inputadapter_input.hasNext()) {
/* 039 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 040 */       double inputadapter_value = inputadapter_row.getDouble(0);
/* 041 */
/* 042 */       byte[] project_array = new byte[32];
/* 043 */       project_arrayData = new UnsafeArrayData();
/* 044 */       Platform.putLong(project_array, 16, 2);
/* 045 */       project_arrayData.pointTo(project_array, 16, 32);
/* 046 */
/* 047 */       boolean project_isNull1 = false;
/* 048 */
/* 049 */       double project_value1 = -1.0;
/* 050 */       project_value1 = inputadapter_value + 1.1D;
/* 051 */       if (false) {
/* 052 */         project_arrayData.setNullAt(0);
/* 053 */       } else {
/* 054 */         project_arrayData.setDouble(0, project_value1);
/* 055 */       }
/* 056 */
/* 057 */       boolean project_isNull4 = false;
/* 058 */
/* 059 */       double project_value4 = -1.0;
/* 060 */       project_value4 = inputadapter_value + 2.2D;
/* 061 */       if (false) {
/* 062 */         project_arrayData.setNullAt(1);
/* 063 */       } else {
/* 064 */         project_arrayData.setDouble(1, project_value4);
/* 065 */       }
/* 066 */       project_holder.reset();
/* 067 */
/* 068 */       // Remember the current cursor so that we can calculate how many bytes are
/* 069 */       // written later.
/* 070 */       final int project_tmpCursor = project_holder.cursor;
/* 071 */
/* 072 */       if (project_arrayData instanceof UnsafeArrayData) {
/* 073 */         final int project_sizeInBytes = ((UnsafeArrayData) project_arrayData).getSizeInBytes();
/* 074 */         // grow the global buffer before writing data.
/* 075 */         project_holder.grow(project_sizeInBytes);
/* 076 */         ((UnsafeArrayData) project_arrayData).writeToMemory(project_holder.buffer, project_holder.cursor);
/* 077 */         project_holder.cursor += project_sizeInBytes;
/* 078 */
/* 079 */       } else {
/* 080 */         final int project_numElements = project_arrayData.numElements();
/* 081 */         project_arrayWriter.initialize(project_holder, project_numElements, 8);
/* 082 */
/* 083 */         for (int project_index = 0; project_index < project_numElements; project_index++) {
/* 084 */           if (project_arrayData.isNullAt(project_index)) {
/* 085 */             project_arrayWriter.setNullDouble(project_index);
/* 086 */           } else {
/* 087 */             final double project_element = project_arrayData.getDouble(project_index);
/* 088 */             project_arrayWriter.write(project_index, project_element);
/* 089 */           }
/* 090 */         }
/* 091 */       }
/* 092 */
/* 093 */       project_rowWriter.setOffsetAndSize(0, project_tmpCursor, project_holder.cursor - project_tmpCursor);
/* 094 */       project_result.setTotalSize(project_holder.totalSize());
/* 095 */       append(project_result);
/* 096 */       if (shouldStop()) return;
/* 097 */     }
/* 098 */   }
/* 099 */ }
```
## How was this patch tested?

Added unit tests into `DataFrameComplexTypeSuite`

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #13909 from kiszk/SPARK-16213.
2016-12-29 10:59:37 +08:00
gatorsmile 5ac62043cf [SPARK-18992][SQL] Move spark.sql.hive.thriftServer.singleSession to SQLConf
### What changes were proposed in this pull request?

Since `spark.sql.hive.thriftServer.singleSession` is a configuration of SQL component, this conf can be moved from `SparkConf` to `StaticSQLConf`.

When we introduced `spark.sql.hive.thriftServer.singleSession`, all the SQL configuration are session specific. They can be modified in different sessions.

In Spark 2.1, static SQL configuration is added. It is a perfect fit for `spark.sql.hive.thriftServer.singleSession`. Previously, we did the same move for `spark.sql.warehouse.dir` from `SparkConf` to `StaticSQLConf`

### How was this patch tested?
Added test cases in HiveThriftServer2Suites.scala

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16392 from gatorsmile/hiveThriftServerSingleSession.
2016-12-28 10:16:22 +08:00
Wenchen Fan dd724c84c8 [SPARK-18989][SQL] DESC TABLE should not fail with format class not found
## What changes were proposed in this pull request?

When we describe a table, we only wanna see the information of this table, not read it, so it's ok even if the format class is not present at the classpath.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16388 from cloud-fan/hive.
2016-12-26 11:27:56 -08:00
Wenchen Fan 8a7db8a608 [SPARK-18980][SQL] implement Aggregator with TypedImperativeAggregate
## What changes were proposed in this pull request?

Currently we implement `Aggregator` with `DeclarativeAggregate`, which will serialize/deserialize the buffer object every time we process an input.

This PR implements `Aggregator` with `TypedImperativeAggregate` and avoids to serialize/deserialize buffer object many times. The benchmark shows we get about 2 times speed up.

For simple buffer object that doesn't need serialization, we still go with `DeclarativeAggregate`, to avoid performance regression.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16383 from cloud-fan/aggregator.
2016-12-26 22:10:20 +08:00
wangzhenhua 3cff816157 [SPARK-18911][SQL] Define CatalogStatistics to interact with metastore and convert it to Statistics in relations
## What changes were proposed in this pull request?

Statistics in LogicalPlan should use attributes to refer to columns rather than column names, because two columns from two relations can have the same column name. But CatalogTable doesn't have the concepts of attribute or broadcast hint in Statistics. Therefore, putting Statistics in CatalogTable is confusing.

We define a different statistic structure in CatalogTable, which is only responsible for interacting with metastore, and is converted to statistics in LogicalPlan when it is used.

## How was this patch tested?

add test cases

Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>

Closes #16323 from wzhfy/nameToAttr.
2016-12-24 15:34:44 +08:00
hyukjinkwon 76622c661f [SPARK-16975][SQL][FOLLOWUP] Do not duplicately check file paths in data sources implementing FileFormat
## What changes were proposed in this pull request?

This PR cleans up duplicated checking for file paths in implemented data sources and prevent to attempt to list twice in ORC data source.

https://github.com/apache/spark/pull/14585 handles a problem for the partition column name having `_` and the issue itself is resolved correctly. However, it seems the data sources implementing `FileFormat` are validating the paths duplicately. Assuming from the comment in `CSVFileFormat`, `// TODO: Move filtering.`, I guess we don't have to check this duplicately.

   Currently, this seems being filtered in `PartitioningAwareFileIndex.shouldFilterOut` and`PartitioningAwareFileIndex.isDataPath`. So, `FileFormat.inferSchema` will always receive leaf files. For example, running to codes below:

   ``` scala
   spark.range(10).withColumn("_locality_code", $"id").write.partitionBy("_locality_code").save("/tmp/parquet")
   spark.read.parquet("/tmp/parquet")
   ```

   gives the paths below without directories but just valid data files:

   ``` bash
   /tmp/parquet/_col=0/part-r-00000-094a8efa-bece-4b50-b54c-7918d1f7b3f8.snappy.parquet
   /tmp/parquet/_col=1/part-r-00000-094a8efa-bece-4b50-b54c-7918d1f7b3f8.snappy.parquet
   /tmp/parquet/_col=2/part-r-00000-25de2b50-225a-4bcf-a2bc-9eb9ed407ef6.snappy.parquet
   ...
   ```

   to `FileFormat.inferSchema`.

## How was this patch tested?

Unit test added in `HadoopFsRelationTest` and related existing tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #14627 from HyukjinKwon/SPARK-16975.
2016-12-22 10:00:20 -08:00
Ryan Williams afd9bc1d8a [SPARK-17807][CORE] split test-tags into test-JAR
Remove spark-tag's compile-scope dependency (and, indirectly, spark-core's compile-scope transitive-dependency) on scalatest by splitting test-oriented tags into spark-tags' test JAR.

Alternative to #16303.

Author: Ryan Williams <ryan.blake.williams@gmail.com>

Closes #16311 from ryan-williams/tt.
2016-12-21 16:37:20 -08:00
gatorsmile 24c0c94128 [SPARK-18949][SQL] Add recoverPartitions API to Catalog
### What changes were proposed in this pull request?

Currently, we only have a SQL interface for recovering all the partitions in the directory of a table and update the catalog. `MSCK REPAIR TABLE` or `ALTER TABLE table RECOVER PARTITIONS`. (Actually, very hard for me to remember `MSCK` and have no clue what it means)

After the new "Scalable Partition Handling", the table repair becomes much more important for making visible the data in the created data source partitioned table.

Thus, this PR is to add it into the Catalog interface. After this PR, users can repair the table by
```Scala
spark.catalog.recoverPartitions("testTable")
```

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16356 from gatorsmile/repairTable.
2016-12-20 23:40:02 -08:00
Wenchen Fan f923c849e5 [SPARK-18899][SPARK-18912][SPARK-18913][SQL] refactor the error checking when append data to an existing table
## What changes were proposed in this pull request?

When we append data to an existing table with `DataFrameWriter.saveAsTable`, we will do various checks to make sure the appended data is consistent with the existing data.

However, we get the information of the existing table by matching the table relation, instead of looking at the table metadata. This is error-prone, e.g. we only check the number of columns for `HadoopFsRelation`, we forget to check bucketing, etc.

This PR refactors the error checking by looking at the metadata of the existing table, and fix several bugs:
* SPARK-18899: We forget to check if the specified bucketing matched the existing table, which may lead to a problematic table that has different bucketing in different data files.
* SPARK-18912: We forget to check the number of columns for non-file-based data source table
* SPARK-18913: We don't support append data to a table with special column names.

## How was this patch tested?
new regression test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16313 from cloud-fan/bug1.
2016-12-19 20:03:33 -08:00
Wenchen Fan 7a75ee1c92 [SPARK-18921][SQL] check database existence with Hive.databaseExists instead of getDatabase
## What changes were proposed in this pull request?

It's weird that we use `Hive.getDatabase` to check the existence of a database, while Hive has a `databaseExists` interface.

What's worse, `Hive.getDatabase` will produce an error message if the database doesn't exist, which is annoying when we only want to check the database existence.

This PR fixes this and use `Hive.databaseExists` to check database existence.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16332 from cloud-fan/minor.
2016-12-19 11:42:59 -08:00
xuanyuanking 24482858e0 [SPARK-18700][SQL] Add StripedLock for each table's relation in cache
## What changes were proposed in this pull request?

As the scenario describe in [SPARK-18700](https://issues.apache.org/jira/browse/SPARK-18700), when cachedDataSourceTables invalided, the coming few queries will fetch all FileStatus in listLeafFiles function. In the condition of table has many partitions, these jobs will occupy much memory of driver finally may cause driver OOM.

In this patch, add StripedLock for each table's relation in cache not for the whole cachedDataSourceTables, each table's load cache operation protected by it.

## How was this patch tested?

Add a multi-thread access table test in `PartitionedTablePerfStatsSuite` and check it only loading once using metrics in `HiveCatalogMetrics`

Author: xuanyuanking <xyliyuanjian@gmail.com>

Closes #16135 from xuanyuanking/SPARK-18700.
2016-12-19 20:31:43 +01:00
gatorsmile 8db4d95c02 [SPARK-18703][SQL] Drop Staging Directories and Data Files After each Insertion/CTAS of Hive serde Tables
### What changes were proposed in this pull request?
Below are the files/directories generated for three inserts againsts a Hive table:
```
/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-29_149_4298858301766472202-1
/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-29_149_4298858301766472202-1/-ext-10000
/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-29_149_4298858301766472202-1/-ext-10000/._SUCCESS.crc
/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-29_149_4298858301766472202-1/-ext-10000/.part-00000.crc
/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-29_149_4298858301766472202-1/-ext-10000/_SUCCESS
/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-29_149_4298858301766472202-1/-ext-10000/part-00000
/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-30_454_6445008511655931341-1
/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-30_454_6445008511655931341-1/-ext-10000
/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-30_454_6445008511655931341-1/-ext-10000/._SUCCESS.crc
/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-30_454_6445008511655931341-1/-ext-10000/.part-00000.crc
/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-30_454_6445008511655931341-1/-ext-10000/_SUCCESS
/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-30_454_6445008511655931341-1/-ext-10000/part-00000
/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-30_722_3388423608658711001-1
/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-30_722_3388423608658711001-1/-ext-10000
/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-30_722_3388423608658711001-1/-ext-10000/._SUCCESS.crc
/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-30_722_3388423608658711001-1/-ext-10000/.part-00000.crc
/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-30_722_3388423608658711001-1/-ext-10000/_SUCCESS
/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.hive-staging_hive_2016-12-03_20-56-30_722_3388423608658711001-1/-ext-10000/part-00000
/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.part-00000.crc
/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/part-00000
```

The first 18 files are temporary. We do not drop it until the end of JVM termination. If JVM does not appropriately terminate, these temporary files/directories will not be dropped.

Only the last two files are needed, as shown below.
```
/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/.part-00000.crc
/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/spark-41eaa5ce-0288-471e-bba1-09cc482813ff/part-00000
```
The temporary files/directories could accumulate a lot when we issue many inserts, since each insert generats at least six files. This could eat a lot of spaces and slow down the JVM termination. When the JVM does not terminates approprately, the files might not be dropped.

This PR is to drop the created staging files and temporary data files after each insert/CTAS.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16134 from gatorsmile/deleteFiles.
2016-12-15 09:23:55 +08:00
Wenchen Fan 3e307b4959 [SPARK-18566][SQL] remove OverwriteOptions
## What changes were proposed in this pull request?

`OverwriteOptions` was introduced in https://github.com/apache/spark/pull/15705, to carry the information of static partitions. However, after further refactor, this information becomes duplicated and we can remove `OverwriteOptions`.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15995 from cloud-fan/overwrite.
2016-12-14 11:30:34 +08:00
Wenchen Fan d53f18cae4 [SPARK-18675][SQL] CTAS for hive serde table should work for all hive versions
## What changes were proposed in this pull request?

Before hive 1.1, when inserting into a table, hive will create the staging directory under a common scratch directory. After the writing is finished, hive will simply empty the table directory and move the staging directory to it.

After hive 1.1, hive will create the staging directory under the table directory, and when moving staging directory to table directory, hive will still empty the table directory, but will exclude the staging directory there.

In `InsertIntoHiveTable`, we simply copy the code from hive 1.2, which means we will always create the staging directory under the table directory, no matter what the hive version is. This causes problems if the hive version is prior to 1.1, because the staging directory will be removed by hive when hive is trying to empty the table directory.

This PR copies the code from hive 0.13, so that we have 2 branches to create staging directory. If hive version is prior to 1.1, we'll go to the old style branch(i.e. create the staging directory under a common scratch directory), else, go to the new style branch(i.e. create the staging directory under the table directory)

## How was this patch tested?

new test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16104 from cloud-fan/hive-0.13.
2016-12-13 09:46:58 -08:00
Yuming Wang 90abfd15f4 [SPARK-18681][SQL] Fix filtering to compatible with partition keys of type int
## What changes were proposed in this pull request?

Cloudera put `/var/run/cloudera-scm-agent/process/15000-hive-HIVEMETASTORE/hive-site.xml` as the configuration file for the Hive Metastore Server, where `hive.metastore.try.direct.sql=false`. But Spark isn't reading this configuration file and get default value `hive.metastore.try.direct.sql=true`. As mallman said, we should use `getMetaConf` method to obtain the original configuration from Hive Metastore Server. I have tested this method few times and the return value is always consistent with Hive Metastore Server.

## How was this patch tested?

The existing tests.

Author: Yuming Wang <wgyumg@gmail.com>

Closes #16122 from wangyum/SPARK-18681.
2016-12-12 23:38:36 +01:00
Marcelo Vanzin 476b34c23a [SPARK-18752][HIVE] isSrcLocal" value should be set from user query.
The value of the "isSrcLocal" parameter passed to Hive's loadTable and
loadPartition methods needs to be set according to the user query (e.g.
"LOAD DATA LOCAL"), and not the current code that tries to guess what
it should be.

For existing versions of Hive the current behavior is probably ok, but
some recent changes in the Hive code changed the semantics slightly,
making code that sets "isSrcLocal" to "true" incorrectly to do the
wrong thing. It would end up moving the parent directory of the files
into the final location, instead of the file themselves, resulting
in a table that cannot be read.

I modified HiveCommandSuite so that existing "LOAD DATA" tests are run
both in local and non-local mode, since the semantics are slightly different.
The tests include a few new checks to make sure the semantics follow
what Hive describes in its documentation.

Tested with existing unit tests and also ran some Hive integration tests
with a version of Hive containing the changes that surfaced the problem.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #16179 from vanzin/SPARK-18752.
2016-12-12 14:19:42 -08:00
Zhan Zhang 67587d961d [SPARK-18637][SQL] Stateful UDF should be considered as nondeterministic
## What changes were proposed in this pull request?

Make stateful udf as nondeterministic

## How was this patch tested?
Add new test cases with both Stateful and Stateless UDF.
Without the patch, the test cases will throw exception:

1 did not equal 10
ScalaTestFailureLocation: org.apache.spark.sql.hive.execution.HiveUDFSuite$$anonfun$21 at (HiveUDFSuite.scala:501)
org.scalatest.exceptions.TestFailedException: 1 did not equal 10
        at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500)
        at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555)
        ...

Author: Zhan Zhang <zhanzhang@fb.com>

Closes #16068 from zhzhan/state.
2016-12-09 16:35:06 +08:00
Michael Allman 772ddbeaa6 [SPARK-18572][SQL] Add a method listPartitionNames to ExternalCatalog
(Link to Jira issue: https://issues.apache.org/jira/browse/SPARK-18572)

## What changes were proposed in this pull request?

Currently Spark answers the `SHOW PARTITIONS` command by fetching all of the table's partition metadata from the external catalog and constructing partition names therefrom. The Hive client has a `getPartitionNames` method which is many times faster for this purpose, with the performance improvement scaling with the number of partitions in a table.

To test the performance impact of this PR, I ran the `SHOW PARTITIONS` command on two Hive tables with large numbers of partitions. One table has ~17,800 partitions, and the other has ~95,000 partitions. For the purposes of this PR, I'll call the former table `table1` and the latter table `table2`. I ran 5 trials for each table with before-and-after versions of this PR. The results are as follows:

Spark at bdc8153, `SHOW PARTITIONS table1`, times in seconds:
7.901
3.983
4.018
4.331
4.261

Spark at bdc8153, `SHOW PARTITIONS table2`
(Timed out after 10 minutes with a `SocketTimeoutException`.)

Spark at this PR, `SHOW PARTITIONS table1`, times in seconds:
3.801
0.449
0.395
0.348
0.336

Spark at this PR, `SHOW PARTITIONS table2`, times in seconds:
5.184
1.63
1.474
1.519
1.41

Taking the best times from each trial, we get a 12x performance improvement for a table with ~17,800 partitions and at least a 426x improvement for a table with ~95,000 partitions. More significantly, the latter command doesn't even complete with the current code in master.

This is actually a patch we've been using in-house at VideoAmp since Spark 1.1. It's made all the difference in the practical usability of our largest tables. Even with tables with about 1,000 partitions there's a performance improvement of about 2-3x.

## How was this patch tested?

I added a unit test to `VersionsSuite` which tests that the Hive client's `getPartitionNames` method returns the correct number of partitions.

Author: Michael Allman <michael@videoamp.com>

Closes #15998 from mallman/spark-18572-list_partition_names.
2016-12-06 11:33:35 +08:00
Eric Liang d9eb4c7215 [SPARK-18661][SQL] Creating a partitioned datasource table should not scan all files for table
## What changes were proposed in this pull request?

Even though in 2.1 creating a partitioned datasource table will not populate the partition data by default (until the user issues MSCK REPAIR TABLE), it seems we still scan the filesystem for no good reason.

We should avoid doing this when the user specifies a schema.

## How was this patch tested?

Perf stat tests.

Author: Eric Liang <ekl@databricks.com>

Closes #16090 from ericl/spark-18661.
2016-12-04 20:44:04 +08:00
Reynold Xin c7c7265950 [SPARK-18695] Bump master branch version to 2.2.0-SNAPSHOT
## What changes were proposed in this pull request?
This patch bumps master branch version to 2.2.0-SNAPSHOT.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #16126 from rxin/SPARK-18695.
2016-12-02 21:09:37 -08:00
Eric Liang 7935c8470c [SPARK-18659][SQL] Incorrect behaviors in overwrite table for datasource tables
## What changes were proposed in this pull request?

Two bugs are addressed here
1. INSERT OVERWRITE TABLE sometime crashed when catalog partition management was enabled. This was because when dropping partitions after an overwrite operation, the Hive client will attempt to delete the partition files. If the entire partition directory was dropped, this would fail. The PR fixes this by adding a flag to control whether the Hive client should attempt to delete files.
2. The static partition spec for OVERWRITE TABLE was not correctly resolved to the case-sensitive original partition names. This resulted in the entire table being overwritten if you did not correctly capitalize your partition names.

cc yhuai cloud-fan

## How was this patch tested?

Unit tests. Surprisingly, the existing overwrite table tests did not catch these edge cases.

Author: Eric Liang <ekl@databricks.com>

Closes #16088 from ericl/spark-18659.
2016-12-02 21:59:02 +08:00
Nathan Howell c82f16c15e [SPARK-18658][SQL] Write text records directly to a FileOutputStream
## What changes were proposed in this pull request?

This replaces uses of `TextOutputFormat` with an `OutputStream`, which will either write directly to the filesystem or indirectly via a compressor (if so configured). This avoids intermediate buffering.

The inverse of this (reading directly from a stream) is necessary for streaming large JSON records (when `wholeFile` is enabled) so I wanted to keep the read and write paths symmetric.

## How was this patch tested?

Existing unit tests.

Author: Nathan Howell <nhowell@godaddy.com>

Closes #16089 from NathanHowell/SPARK-18658.
2016-12-01 21:40:49 -08:00
Wenchen Fan a5f02b0029 [SPARK-18647][SQL] do not put provider in table properties for Hive serde table
## What changes were proposed in this pull request?

In Spark 2.1, we make Hive serde tables case-preserving by putting the table metadata in table properties, like what we did for data source table. However, we should not put table provider, as it will break forward compatibility. e.g. if we create a Hive serde table with Spark 2.1, using `sql("create table test stored as parquet as select 1")`, we will fail to read it with Spark 2.0, as Spark 2.0 mistakenly treat it as data source table because there is a `provider` entry in table properties.

Logically Hive serde table's provider is always hive, we don't need to store it in table properties, this PR removes it.

## How was this patch tested?

manually test the forward compatibility issue.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16080 from cloud-fan/hive.
2016-12-02 12:54:12 +08:00
Eric Liang 88f559f20a [SPARK-18635][SQL] Partition name/values not escaped correctly in some cases
## What changes were proposed in this pull request?

Due to confusion between URI vs paths, in certain cases we escape partition values too many times, which causes some Hive client operations to fail or write data to the wrong location. This PR fixes at least some of these cases.

To my understanding this is how values, filesystem paths, and URIs interact.
- Hive stores raw (unescaped) partition values that are returned to you directly when you call listPartitions.
- Internally, we convert these raw values to filesystem paths via `ExternalCatalogUtils.[un]escapePathName`.
- In some circumstances we store URIs instead of filesystem paths. When a path is converted to a URI via `path.toURI`, the escaped partition values are further URI-encoded. This means that to get a path back from a URI, you must call `new Path(new URI(uriTxt))` in order to decode the URI-encoded string.
- In `CatalogStorageFormat` we store URIs as strings. This makes it easy to forget to URI-decode the value before converting it into a path.
- Finally, the Hive client itself uses mostly Paths for representing locations, and only URIs occasionally.

In the future we should probably clean this up, perhaps by dropping use of URIs when unnecessary. We should also try fixing escaping for partition names as well as values, though names are unlikely to contain special characters.

cc mallman cloud-fan yhuai

## How was this patch tested?

Unit tests.

Author: Eric Liang <ekl@databricks.com>

Closes #16071 from ericl/spark-18635.
2016-12-01 16:48:10 +08:00
Wenchen Fan 3f03c90a80 [SPARK-18220][SQL] read Hive orc table with varchar column should not fail
## What changes were proposed in this pull request?

Spark SQL only has `StringType`, when reading hive table with varchar column, we will read that column as `StringType`. However, we still need to use varchar `ObjectInspector` to read varchar column in hive table, which means we need to know the actual column type at hive side.

In Spark 2.1, after https://github.com/apache/spark/pull/14363 , we parse hive type string to catalyst type, which means the actual column type at hive side is erased. Then we may use string `ObjectInspector` to read varchar column and fail.

This PR keeps the original hive column type string in the metadata of `StructField`, and use it when we convert it to a hive column.

## How was this patch tested?

newly added regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16060 from cloud-fan/varchar.
2016-11-30 09:47:30 -08:00
gatorsmile a1d9138ab2 [SPARK-17680][SQL][TEST] Added a Testcase for Verifying Unicode Character Support for Column Names and Comments
### What changes were proposed in this pull request?

Spark SQL supports Unicode characters for column names when specified within backticks(`). When the Hive support is enabled, the version of the Hive metastore must be higher than 0.12,  See the JIRA: https://issues.apache.org/jira/browse/HIVE-6013 Hive metastore supports Unicode characters for column names since 0.13.

In Spark SQL, table comments, and view comments always allow Unicode characters without backticks.

BTW, a separate PR has been submitted for database and table name validation because we do not support Unicode characters in these two cases.
### How was this patch tested?

N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15255 from gatorsmile/unicodeSupport.
2016-11-30 15:17:29 +08:00
Herman van Hovell af9789a4f5 [SPARK-18632][SQL] AggregateFunction should not implement ImplicitCastInputTypes
## What changes were proposed in this pull request?
`AggregateFunction` currently implements `ImplicitCastInputTypes` (which enables implicit input type casting). There are actually quite a few situations in which we don't need this, or require more control over our input. A recent example is the aggregate for `CountMinSketch` which should only take string, binary or integral types inputs.

This PR removes `ImplicitCastInputTypes` from the `AggregateFunction` and makes a case-by-case decision on what kind of input validation we should use.

## How was this patch tested?
Refactoring only. Existing tests.

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

Closes #16066 from hvanhovell/SPARK-18632.
2016-11-29 20:05:15 -08:00
hyukjinkwon f830bb9170
[SPARK-3359][DOCS] Make javadoc8 working for unidoc/genjavadoc compatibility in Java API documentation
## What changes were proposed in this pull request?

This PR make `sbt unidoc` complete with Java 8.

This PR roughly includes several fixes as below:

- Fix unrecognisable class and method links in javadoc by changing it from `[[..]]` to `` `...` ``

  ```diff
  - * A column that will be computed based on the data in a [[DataFrame]].
  + * A column that will be computed based on the data in a `DataFrame`.
  ```

- Fix throws annotations so that they are recognisable in javadoc

- Fix URL links to `<a href="http..."></a>`.

  ```diff
  - * [[http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree]] model for regression.
  + * <a href="http://en.wikipedia.org/wiki/Decision_tree_learning">
  + * Decision tree (Wikipedia)</a> model for regression.
  ```

  ```diff
  -   * see http://en.wikipedia.org/wiki/Receiver_operating_characteristic
  +   * see <a href="http://en.wikipedia.org/wiki/Receiver_operating_characteristic">
  +   * Receiver operating characteristic (Wikipedia)</a>
  ```

- Fix < to > to

  - `greater than`/`greater than or equal to` or `less than`/`less than or equal to` where applicable.

  - Wrap it with `{{{...}}}` to print them in javadoc or use `{code ...}` or `{literal ..}`. Please refer https://github.com/apache/spark/pull/16013#discussion_r89665558

- Fix `</p>` complaint

## How was this patch tested?

Manually tested by `jekyll build` with Java 7 and 8

```
java version "1.7.0_80"
Java(TM) SE Runtime Environment (build 1.7.0_80-b15)
Java HotSpot(TM) 64-Bit Server VM (build 24.80-b11, mixed mode)
```

```
java version "1.8.0_45"
Java(TM) SE Runtime Environment (build 1.8.0_45-b14)
Java HotSpot(TM) 64-Bit Server VM (build 25.45-b02, mixed mode)
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16013 from HyukjinKwon/SPARK-3359-errors-more.
2016-11-29 09:41:32 +00:00
Eric Liang e2318ede04 [SPARK-18544][SQL] Append with df.saveAsTable writes data to wrong location
## What changes were proposed in this pull request?

We failed to properly propagate table metadata for existing tables for the saveAsTable command. This caused a downstream component to think the table was MANAGED, writing data to the wrong location.

## How was this patch tested?

Unit test that fails before the patch.

Author: Eric Liang <ekl@databricks.com>

Closes #15983 from ericl/spark-18544.
2016-11-28 21:58:01 -08:00
Cheng Lian 2e809903d4 [SPARK-18403][SQL] Fix unsafe data false sharing issue in ObjectHashAggregateExec
## What changes were proposed in this pull request?

This PR fixes a random OOM issue occurred while running `ObjectHashAggregateSuite`.

This issue can be steadily reproduced under the following conditions:

1. The aggregation must be evaluated using `ObjectHashAggregateExec`;
2. There must be an input column whose data type involves `ArrayType` (an input column of `MapType` may even cause SIGSEGV);
3. Sort-based aggregation fallback must be triggered during evaluation.

The root cause is that while falling back to sort-based aggregation, we must sort and feed already evaluated partial aggregation buffers living in the hash map to the sort-based aggregator using an external sorter. However, the underlying mutable byte buffer of `UnsafeRow`s produced by the iterator of the external sorter is reused and may get overwritten when the iterator steps forward. After the last entry is consumed, the byte buffer points to a block of uninitialized memory filled by `5a`. Therefore, while reading an `UnsafeArrayData` out of the `UnsafeRow`, `5a5a5a5a` is treated as array size and triggers a memory allocation for a ridiculously large array and immediately blows up the JVM with an OOM.

To fix this issue, we only need to add `.copy()` accordingly.

## How was this patch tested?

New regression test case added in `ObjectHashAggregateSuite`.

Author: Cheng Lian <lian@databricks.com>

Closes #15976 from liancheng/investigate-oom.
2016-11-29 09:01:03 +08:00
jiangxingbo 0f5f52a3d1 [SPARK-16282][SQL] Implement percentile SQL function.
## What changes were proposed in this pull request?

Implement percentile SQL function. It computes the exact percentile(s) of expr at pc with range in [0, 1].

## How was this patch tested?

Add a new testsuite `PercentileSuite` to test percentile directly.
Updated related testcases in `ExpressionToSQLSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>
Author: 蒋星博 <jiangxingbo@meituan.com>
Author: jiangxingbo <jiangxingbo@meituan.com>

Closes #14136 from jiangxb1987/percentile.
2016-11-28 11:05:58 -08:00
Wenchen Fan d31ff9b7ca [SPARK-17732][SQL] Revert ALTER TABLE DROP PARTITION should support comparators
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/15704 will fail if we use int literal in `DROP PARTITION`, and we have reverted it in branch-2.1.

This PR reverts it in master branch, and add a regression test for it, to make sure the master branch is healthy.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16036 from cloud-fan/revert.
2016-11-28 08:46:00 -08:00
Wenchen Fan fc2c13bdf0 [SPARK-18482][SQL] make sure Spark can access the table metadata created by older version of spark
## What changes were proposed in this pull request?

In Spark 2.1, we did a lot of refactor for `HiveExternalCatalog` and related code path. These refactor may introduce external behavior changes and break backward compatibility. e.g. http://issues.apache.org/jira/browse/SPARK-18464

To avoid future compatibility problems of `HiveExternalCatalog`, this PR dumps some typical table metadata from tables created by 2.0, and test if they can recognized by current version of Spark.

## How was this patch tested?

test only change

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16003 from cloud-fan/test.
2016-11-27 21:45:50 -08:00
gatorsmile 07f32c2283 [SPARK-18594][SQL] Name Validation of Databases/Tables
### What changes were proposed in this pull request?
Currently, the name validation checks are limited to table creation. It is enfored by Analyzer rule: `PreWriteCheck`.

However, table renaming and database creation have the same issues. It makes more sense to do the checks in `SessionCatalog`. This PR is to add it into `SessionCatalog`.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16018 from gatorsmile/nameValidate.
2016-11-27 19:43:24 -08:00
hyukjinkwon 51b1c1551d
[SPARK-3359][BUILD][DOCS] More changes to resolve javadoc 8 errors that will help unidoc/genjavadoc compatibility
## What changes were proposed in this pull request?

This PR only tries to fix things that looks pretty straightforward and were fixed in other previous PRs before.

This PR roughly fixes several things as below:

- Fix unrecognisable class and method links in javadoc by changing it from `[[..]]` to `` `...` ``

  ```
  [error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/DataStreamReader.java:226: error: reference not found
  [error]    * Loads text files and returns a {link DataFrame} whose schema starts with a string column named
  ```

- Fix an exception annotation and remove code backticks in `throws` annotation

  Currently, sbt unidoc with Java 8 complains as below:

  ```
  [error] .../java/org/apache/spark/sql/streaming/StreamingQuery.java:72: error: unexpected text
  [error]    * throws StreamingQueryException, if <code>this</code> query has terminated with an exception.
  ```

  `throws` should specify the correct class name from `StreamingQueryException,` to `StreamingQueryException` without backticks. (see [JDK-8007644](https://bugs.openjdk.java.net/browse/JDK-8007644)).

- Fix `[[http..]]` to `<a href="http..."></a>`.

  ```diff
  -   * [[https://blogs.oracle.com/java-platform-group/entry/diagnosing_tls_ssl_and_https Oracle
  -   * blog page]].
  +   * <a href="https://blogs.oracle.com/java-platform-group/entry/diagnosing_tls_ssl_and_https">
  +   * Oracle blog page</a>.
  ```

   `[[http...]]` link markdown in scaladoc is unrecognisable in javadoc.

- It seems class can't have `return` annotation. So, two cases of this were removed.

  ```
  [error] .../java/org/apache/spark/mllib/regression/IsotonicRegression.java:27: error: invalid use of return
  [error]    * return New instance of IsotonicRegression.
  ```

- Fix < to `&lt;` and > to `&gt;` according to HTML rules.

- Fix `</p>` complaint

- Exclude unrecognisable in javadoc, `constructor`, `todo` and `groupname`.

## How was this patch tested?

Manually tested by `jekyll build` with Java 7 and 8

```
java version "1.7.0_80"
Java(TM) SE Runtime Environment (build 1.7.0_80-b15)
Java HotSpot(TM) 64-Bit Server VM (build 24.80-b11, mixed mode)
```

```
java version "1.8.0_45"
Java(TM) SE Runtime Environment (build 1.8.0_45-b14)
Java HotSpot(TM) 64-Bit Server VM (build 25.45-b02, mixed mode)
```

Note: this does not yet make sbt unidoc suceed with Java 8 yet but it reduces the number of errors with Java 8.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15999 from HyukjinKwon/SPARK-3359-errors.
2016-11-25 11:27:07 +00:00
Reynold Xin 70ad07a9d2 [SPARK-18522][SQL] Explicit contract for column stats serialization
## What changes were proposed in this pull request?
The current implementation of column stats uses the base64 encoding of the internal UnsafeRow format to persist statistics (in table properties in Hive metastore). This is an internal format that is not stable across different versions of Spark and should NOT be used for persistence. In addition, it would be better if statistics stored in the catalog is human readable.

This pull request introduces the following changes:

1. Created a single ColumnStat class to for all data types. All data types track the same set of statistics.
2. Updated the implementation for stats collection to get rid of the dependency on internal data structures (e.g. InternalRow, or storing DateType as an int32). For example, previously dates were stored as a single integer, but are now stored as java.sql.Date. When we implement the next steps of CBO, we can add code to convert those back into internal types again.
3. Documented clearly what JVM data types are being used to store what data.
4. Defined a simple Map[String, String] interface for serializing and deserializing column stats into/from the catalog.
5. Rearranged the method/function structure so it is more clear what the supported data types are, and also moved how stats are generated into ColumnStat class so they are easy to find.

## How was this patch tested?
Removed most of the original test cases created for column statistics, and added three very simple ones to cover all the cases. The three test cases validate:
1. Roundtrip serialization works.
2. Behavior when analyzing non-existent column or unsupported data type column.
3. Result for stats collection for all valid data types.

Also moved parser related tests into a parser test suite and added an explicit serialization test for the Hive external catalog.

Author: Reynold Xin <rxin@databricks.com>

Closes #15959 from rxin/SPARK-18522.
2016-11-23 20:48:41 +08:00
Eric Liang 85235ed6c6 [SPARK-18545][SQL] Verify number of hive client RPCs in PartitionedTablePerfStatsSuite
## What changes were proposed in this pull request?

This would help catch accidental O(n) calls to the hive client as in https://issues.apache.org/jira/browse/SPARK-18507

## How was this patch tested?

Checked that the test fails before https://issues.apache.org/jira/browse/SPARK-18507 was patched. cc cloud-fan

Author: Eric Liang <ekl@databricks.com>

Closes #15985 from ericl/spark-18545.
2016-11-23 20:14:08 +08:00
gatorsmile 9c42d4a76c [SPARK-16803][SQL] SaveAsTable does not work when target table is a Hive serde table
### What changes were proposed in this pull request?

In Spark 2.0, `SaveAsTable` does not work when the target table is a Hive serde table, but Spark 1.6 works.

**Spark 1.6**

``` Scala
scala> sql("create table sample.sample stored as SEQUENCEFILE as select 1 as key, 'abc' as value")
res2: org.apache.spark.sql.DataFrame = []

scala> val df = sql("select key, value as value from sample.sample")
df: org.apache.spark.sql.DataFrame = [key: int, value: string]

scala> df.write.mode("append").saveAsTable("sample.sample")

scala> sql("select * from sample.sample").show()
+---+-----+
|key|value|
+---+-----+
|  1|  abc|
|  1|  abc|
+---+-----+
```

**Spark 2.0**

``` Scala
scala> df.write.mode("append").saveAsTable("sample.sample")
org.apache.spark.sql.AnalysisException: Saving data in MetastoreRelation sample, sample
 is not supported.;
```

So far, we do not plan to support it in Spark 2.1 due to the risk. Spark 1.6 works because it internally uses insertInto. But, if we change it back it will break the semantic of saveAsTable (this method uses by-name resolution instead of using by-position resolution used by insertInto). More extra changes are needed to support `hive` as a `format` in DataFrameWriter.

Instead, users should use insertInto API. This PR corrects the error messages. Users can understand how to bypass it before we support it in a separate PR.
### How was this patch tested?

Test cases are added

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15926 from gatorsmile/saveAsTableFix5.
2016-11-22 15:10:49 -08:00
Burak Yavuz bdc8153e86 [SPARK-18465] Add 'IF EXISTS' clause to 'UNCACHE' to not throw exceptions when table doesn't exist
## What changes were proposed in this pull request?

While this behavior is debatable, consider the following use case:
```sql
UNCACHE TABLE foo;
CACHE TABLE foo AS
SELECT * FROM bar
```
The command above fails the first time you run it. But I want to run the command above over and over again, and I don't want to change my code just for the first run of it.
The issue is that subsequent `CACHE TABLE` commands do not overwrite the existing table.

Now we can do:
```sql
UNCACHE TABLE IF EXISTS foo;
CACHE TABLE foo AS
SELECT * FROM bar
```

## How was this patch tested?

Unit tests

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #15896 from brkyvz/uncache.
2016-11-22 13:03:50 -08:00
Wenchen Fan 702cd403fc [SPARK-18507][SQL] HiveExternalCatalog.listPartitions should only call getTable once
## What changes were proposed in this pull request?

HiveExternalCatalog.listPartitions should only call `getTable` once, instead of calling it for every partitions.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15978 from cloud-fan/perf.
2016-11-22 15:25:22 -05:00
hyukjinkwon a2d464770c [SPARK-17765][SQL] Support for writing out user-defined type in ORC datasource
## What changes were proposed in this pull request?

This PR adds the support for `UserDefinedType` when writing out instead of throwing `ClassCastException` in ORC data source.

In more details, `OrcStruct` is being created based on string from`DataType.catalogString`. For user-defined type, it seems it returns `sqlType.simpleString` for `catalogString` by default[1]. However, during type-dispatching to match the output with the schema, it tries to cast to, for example, `StructType`[2].

So, running the codes below (`MyDenseVector` was borrowed[3]) :

``` scala
val data = Seq((1, new UDT.MyDenseVector(Array(0.25, 2.25, 4.25))))
val udtDF = data.toDF("id", "vectors")
udtDF.write.orc("/tmp/test.orc")
```

ends up throwing an exception as below:

```
java.lang.ClassCastException: org.apache.spark.sql.UDT$MyDenseVectorUDT cannot be cast to org.apache.spark.sql.types.ArrayType
    at org.apache.spark.sql.hive.HiveInspectors$class.wrapperFor(HiveInspectors.scala:381)
    at org.apache.spark.sql.hive.orc.OrcSerializer.wrapperFor(OrcFileFormat.scala:164)
...
```

So, this PR uses `UserDefinedType.sqlType` during finding the correct converter when writing out in ORC data source.

[1]dfdcab00c7/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala (L95)
[2]d2dc8c4a16/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala (L326)
[3]2bfed1a0c5/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala (L38-L70)
## How was this patch tested?

Unit tests in `OrcQuerySuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15361 from HyukjinKwon/SPARK-17765.
2016-11-21 13:23:32 -08:00
Reynold Xin 6f7ff75091 [SPARK-18505][SQL] Simplify AnalyzeColumnCommand
## What changes were proposed in this pull request?
I'm spending more time at the design & code level for cost-based optimizer now, and have found a number of issues related to maintainability and compatibility that I will like to address.

This is a small pull request to clean up AnalyzeColumnCommand:

1. Removed warning on duplicated columns. Warnings in log messages are useless since most users that run SQL don't see them.
2. Removed the nested updateStats function, by just inlining the function.
3. Renamed a few functions to better reflect what they do.
4. Removed the factory apply method for ColumnStatStruct. It is a bad pattern to use a apply method that returns an instantiation of a class that is not of the same type (ColumnStatStruct.apply used to return CreateNamedStruct).
5. Renamed ColumnStatStruct to just AnalyzeColumnCommand.
6. Added more documentation explaining some of the non-obvious return types and code blocks.

In follow-up pull requests, I'd like to address the following:

1. Get rid of the Map[String, ColumnStat] map, since internally we should be using Attribute to reference columns, rather than strings.
2. Decouple the fields exposed by ColumnStat and internals of Spark SQL's execution path. Currently the two are coupled because ColumnStat takes in an InternalRow.
3. Correctness: Remove code path that stores statistics in the catalog using the base64 encoding of the UnsafeRow format, which is not stable across Spark versions.
4. Clearly document the data representation stored in the catalog for statistics.

## How was this patch tested?
Affected test cases have been updated.

Author: Reynold Xin <rxin@databricks.com>

Closes #15933 from rxin/SPARK-18505.
2016-11-18 16:34:11 -08:00
Andrew Ray 795e9fc921 [SPARK-18457][SQL] ORC and other columnar formats using HiveShim read all columns when doing a simple count
## What changes were proposed in this pull request?

When reading zero columns (e.g., count(*)) from ORC or any other format that uses HiveShim, actually set the read column list to empty for Hive to use.

## How was this patch tested?

Query correctness is handled by existing unit tests. I'm happy to add more if anyone can point out some case that is not covered.

Reduction in data read can be verified in the UI when built with a recent version of Hadoop say:
```
build/mvn -Pyarn -Phadoop-2.7 -Dhadoop.version=2.7.0 -Phive -DskipTests clean package
```
However the default Hadoop 2.2 that is used for unit tests does not report actual bytes read and instead just full file sizes (see FileScanRDD.scala line 80). Therefore I don't think there is a good way to add a unit test for this.

I tested with the following setup using above build options
```
case class OrcData(intField: Long, stringField: String)
spark.range(1,1000000).map(i => OrcData(i, s"part-$i")).toDF().write.format("orc").save("orc_test")

sql(
      s"""CREATE EXTERNAL TABLE orc_test(
         |  intField LONG,
         |  stringField STRING
         |)
         |STORED AS ORC
         |LOCATION '${System.getProperty("user.dir") + "/orc_test"}'
       """.stripMargin)
```

## Results

query | Spark 2.0.2 | this PR
---|---|---
`sql("select count(*) from orc_test").collect`|4.4 MB|199.4 KB
`sql("select intField from orc_test").collect`|743.4 KB|743.4 KB
`sql("select * from orc_test").collect`|4.4 MB|4.4 MB

Author: Andrew Ray <ray.andrew@gmail.com>

Closes #15898 from aray/sql-orc-no-col.
2016-11-18 11:19:49 -08:00
Wenchen Fan ce13c26723 [SPARK-18360][SQL] default table path of tables in default database should depend on the location of default database
## What changes were proposed in this pull request?

The current semantic of the warehouse config:

1. it's a static config, which means you can't change it once your spark application is launched.
2. Once a database is created, its location won't change even the warehouse path config is changed.
3. default database is a special case, although its location is fixed, but the locations of tables created in it are not. If a Spark app starts with warehouse path B(while the location of default database is A), then users create a table `tbl` in default database, its location will be `B/tbl` instead of `A/tbl`. If uses change the warehouse path config to C, and create another table `tbl2`, its location will still be `B/tbl2` instead of `C/tbl2`.

rule 3 doesn't make sense and I think we made it by mistake, not intentionally. Data source tables don't follow rule 3 and treat default database like normal ones.

This PR fixes hive serde tables to make it consistent with data source tables.

## How was this patch tested?

HiveSparkSubmitSuite

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15812 from cloud-fan/default-db.
2016-11-17 17:31:12 -08:00
Wenchen Fan 07b3f045cd [SPARK-18464][SQL] support old table which doesn't store schema in metastore
## What changes were proposed in this pull request?

Before Spark 2.1, users can create an external data source table without schema, and we will infer the table schema at runtime. In Spark 2.1, we decided to infer the schema when the table was created, so that we don't need to infer it again and again at runtime.

This is a good improvement, but we should still respect and support old tables which doesn't store table schema in metastore.

## How was this patch tested?

regression test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15900 from cloud-fan/hive-catalog.
2016-11-17 00:00:38 -08:00
Cheng Lian 2ca8ae9aa1 [SPARK-18186] Migrate HiveUDAFFunction to TypedImperativeAggregate for partial aggregation support
## What changes were proposed in this pull request?

While being evaluated in Spark SQL, Hive UDAFs don't support partial aggregation. This PR migrates `HiveUDAFFunction`s to `TypedImperativeAggregate`, which already provides partial aggregation support for aggregate functions that may use arbitrary Java objects as aggregation states.

The following snippet shows the effect of this PR:

```scala
import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMax
sql(s"CREATE FUNCTION hive_max AS '${classOf[GenericUDAFMax].getName}'")

spark.range(100).createOrReplaceTempView("t")

// A query using both Spark SQL native `max` and Hive `max`
sql(s"SELECT max(id), hive_max(id) FROM t").explain()
```

Before this PR:

```
== Physical Plan ==
SortAggregate(key=[], functions=[max(id#1L), default.hive_max(default.hive_max, HiveFunctionWrapper(org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMax,org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMax7475f57e), id#1L, false, 0, 0)])
+- Exchange SinglePartition
   +- *Range (0, 100, step=1, splits=Some(1))
```

After this PR:

```
== Physical Plan ==
SortAggregate(key=[], functions=[max(id#1L), default.hive_max(default.hive_max, HiveFunctionWrapper(org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMax,org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMax5e18a6a7), id#1L, false, 0, 0)])
+- Exchange SinglePartition
   +- SortAggregate(key=[], functions=[partial_max(id#1L), partial_default.hive_max(default.hive_max, HiveFunctionWrapper(org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMax,org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMax5e18a6a7), id#1L, false, 0, 0)])
      +- *Range (0, 100, step=1, splits=Some(1))
```

The tricky part of the PR is mostly about updating and passing around aggregation states of `HiveUDAFFunction`s since the aggregation state of a Hive UDAF may appear in three different forms. Let's take a look at the testing `MockUDAF` added in this PR as an example. This UDAF computes the count of non-null values together with the count of nulls of a given column. Its aggregation state may appear as the following forms at different time:

1. A `MockUDAFBuffer`, which is a concrete subclass of `GenericUDAFEvaluator.AggregationBuffer`

   The form used by Hive UDAF API. This form is required by the following scenarios:

   - Calling `GenericUDAFEvaluator.iterate()` to update an existing aggregation state with new input values.
   - Calling `GenericUDAFEvaluator.terminate()` to get the final aggregated value from an existing aggregation state.
   - Calling `GenericUDAFEvaluator.merge()` to merge other aggregation states into an existing aggregation state.

     The existing aggregation state to be updated must be in this form.

   Conversions:

   - To form 2:

     `GenericUDAFEvaluator.terminatePartial()`

   - To form 3:

     Convert to form 2 first, and then to 3.

2. An `Object[]` array containing two `java.lang.Long` values.

   The form used to interact with Hive's `ObjectInspector`s. This form is required by the following scenarios:

   - Calling `GenericUDAFEvaluator.terminatePartial()` to convert an existing aggregation state in form 1 to form 2.
   - Calling `GenericUDAFEvaluator.merge()` to merge other aggregation states into an existing aggregation state.

     The input aggregation state must be in this form.

   Conversions:

   - To form 1:

     No direct method. Have to create an empty `AggregationBuffer` and merge it into the empty buffer.

   - To form 3:

     `unwrapperFor()`/`unwrap()` method of `HiveInspectors`

3. The byte array that holds data of an `UnsafeRow` with two `LongType` fields.

   The form used by Spark SQL to shuffle partial aggregation results. This form is required because `TypedImperativeAggregate` always asks its subclasses to serialize their aggregation states into a byte array.

   Conversions:

   - To form 1:

     Convert to form 2 first, and then to 1.

   - To form 2:

     `wrapperFor()`/`wrap()` method of `HiveInspectors`

Here're some micro-benchmark results produced by the most recent master and this PR branch.

Master:

```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.5
Intel(R) Core(TM) i7-4960HQ CPU  2.60GHz

hive udaf vs spark af:                   Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
w/o groupBy                                    339 /  372          3.1         323.2       1.0X
w/ groupBy                                     503 /  529          2.1         479.7       0.7X
```

This PR:

```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.5
Intel(R) Core(TM) i7-4960HQ CPU  2.60GHz

hive udaf vs spark af:                   Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
w/o groupBy                                    116 /  126          9.0         110.8       1.0X
w/ groupBy                                     151 /  159          6.9         144.0       0.8X
```

Benchmark code snippet:

```scala
  test("Hive UDAF benchmark") {
    val N = 1 << 20

    sparkSession.sql(s"CREATE TEMPORARY FUNCTION hive_max AS '${classOf[GenericUDAFMax].getName}'")

    val benchmark = new Benchmark(
      name = "hive udaf vs spark af",
      valuesPerIteration = N,
      minNumIters = 5,
      warmupTime = 5.seconds,
      minTime = 5.seconds,
      outputPerIteration = true
    )

    benchmark.addCase("w/o groupBy") { _ =>
      sparkSession.range(N).agg("id" -> "hive_max").collect()
    }

    benchmark.addCase("w/ groupBy") { _ =>
      sparkSession.range(N).groupBy($"id" % 10).agg("id" -> "hive_max").collect()
    }

    benchmark.run()

    sparkSession.sql(s"DROP TEMPORARY FUNCTION IF EXISTS hive_max")
  }
```

## How was this patch tested?

New test suite `HiveUDAFSuite` is added.

Author: Cheng Lian <lian@databricks.com>

Closes #15703 from liancheng/partial-agg-hive-udaf.
2016-11-16 14:32:36 -08:00
Dongjoon Hyun 74f5c2176d [SPARK-18433][SQL] Improve DataSource option keys to be more case-insensitive
## What changes were proposed in this pull request?

This PR aims to improve DataSource option keys to be more case-insensitive

DataSource partially use CaseInsensitiveMap in code-path. For example, the following fails to find url.

```scala
val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2)
df.write.format("jdbc")
    .option("UrL", url1)
    .option("dbtable", "TEST.SAVETEST")
    .options(properties.asScala)
    .save()
```

This PR makes DataSource options to use CaseInsensitiveMap internally and also makes DataSource to use CaseInsensitiveMap generally except `InMemoryFileIndex` and `InsertIntoHadoopFsRelationCommand`. We can not pass them CaseInsensitiveMap because they creates new case-sensitive HadoopConfs by calling newHadoopConfWithOptions(options) inside.

## How was this patch tested?

Pass the Jenkins test with newly added test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15884 from dongjoon-hyun/SPARK-18433.
2016-11-16 17:12:18 +08:00
Wenchen Fan 4ac9759f80 [SPARK-18377][SQL] warehouse path should be a static conf
## What changes were proposed in this pull request?

it's weird that every session can set its own warehouse path at runtime, we should forbid it and make it a static conf.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15825 from cloud-fan/warehouse.
2016-11-15 20:24:36 -08:00
Dongjoon Hyun 3ce057d001 [SPARK-17732][SQL] ALTER TABLE DROP PARTITION should support comparators
## What changes were proposed in this pull request?

This PR aims to support `comparators`, e.g. '<', '<=', '>', '>=', again in Apache Spark 2.0 for backward compatibility.

**Spark 1.6**

``` scala
scala> sql("CREATE TABLE sales(id INT) PARTITIONED BY (country STRING, quarter STRING)")
res0: org.apache.spark.sql.DataFrame = [result: string]

scala> sql("ALTER TABLE sales DROP PARTITION (country < 'KR')")
res1: org.apache.spark.sql.DataFrame = [result: string]
```

**Spark 2.0**

``` scala
scala> sql("CREATE TABLE sales(id INT) PARTITIONED BY (country STRING, quarter STRING)")
res0: org.apache.spark.sql.DataFrame = []

scala> sql("ALTER TABLE sales DROP PARTITION (country < 'KR')")
org.apache.spark.sql.catalyst.parser.ParseException:
mismatched input '<' expecting {')', ','}(line 1, pos 42)
```

After this PR, it's supported.

## How was this patch tested?

Pass the Jenkins test with a newly added testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15704 from dongjoon-hyun/SPARK-17732-2.
2016-11-15 15:59:04 -08:00
Dongjoon Hyun d42bb7cc4e [SPARK-17982][SQL] SQLBuilder should wrap the generated SQL with parenthesis for LIMIT
## What changes were proposed in this pull request?

Currently, `SQLBuilder` handles `LIMIT` by always adding `LIMIT` at the end of the generated subSQL. It makes `RuntimeException`s like the following. This PR adds a parenthesis always except `SubqueryAlias` is used together with `LIMIT`.

**Before**

``` scala
scala> sql("CREATE TABLE tbl(id INT)")
scala> sql("CREATE VIEW v1(id2) AS SELECT id FROM tbl LIMIT 2")
java.lang.RuntimeException: Failed to analyze the canonicalized SQL: ...
```

**After**

``` scala
scala> sql("CREATE TABLE tbl(id INT)")
scala> sql("CREATE VIEW v1(id2) AS SELECT id FROM tbl LIMIT 2")
scala> sql("SELECT id2 FROM v1")
res4: org.apache.spark.sql.DataFrame = [id2: int]
```

**Fixed cases in this PR**

The following two cases are the detail query plans having problematic SQL generations.

1. `SELECT * FROM (SELECT id FROM tbl LIMIT 2)`

    Please note that **FROM SELECT** part of the generated SQL in the below. When we don't use '()' for limit, this fails.

```scala
# Original logical plan:
Project [id#1]
+- GlobalLimit 2
   +- LocalLimit 2
      +- Project [id#1]
         +- MetastoreRelation default, tbl

# Canonicalized logical plan:
Project [gen_attr_0#1 AS id#4]
+- SubqueryAlias tbl
   +- Project [gen_attr_0#1]
      +- GlobalLimit 2
         +- LocalLimit 2
            +- Project [gen_attr_0#1]
               +- SubqueryAlias gen_subquery_0
                  +- Project [id#1 AS gen_attr_0#1]
                     +- SQLTable default, tbl, [id#1]

# Generated SQL:
SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`tbl`) AS gen_subquery_0 LIMIT 2) AS tbl
```

2. `SELECT * FROM (SELECT id FROM tbl TABLESAMPLE (2 ROWS))`

    Please note that **((~~~) AS gen_subquery_0 LIMIT 2)** in the below. When we use '()' for limit on `SubqueryAlias`, this fails.

```scala
# Original logical plan:
Project [id#1]
+- Project [id#1]
   +- GlobalLimit 2
      +- LocalLimit 2
         +- MetastoreRelation default, tbl

# Canonicalized logical plan:
Project [gen_attr_0#1 AS id#4]
+- SubqueryAlias tbl
   +- Project [gen_attr_0#1]
      +- GlobalLimit 2
         +- LocalLimit 2
            +- SubqueryAlias gen_subquery_0
               +- Project [id#1 AS gen_attr_0#1]
                  +- SQLTable default, tbl, [id#1]

# Generated SQL:
SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM ((SELECT `id` AS `gen_attr_0` FROM `default`.`tbl`) AS gen_subquery_0 LIMIT 2)) AS tbl
```

## How was this patch tested?

Pass the Jenkins test with a newly added test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15546 from dongjoon-hyun/SPARK-17982.
2016-11-11 13:28:18 -08:00
Eric Liang a3356343cb [SPARK-18185] Fix all forms of INSERT / OVERWRITE TABLE for Datasource tables
## What changes were proposed in this pull request?

As of current 2.1, INSERT OVERWRITE with dynamic partitions against a Datasource table will overwrite the entire table instead of only the partitions matching the static keys, as in Hive. It also doesn't respect custom partition locations.

This PR adds support for all these operations to Datasource tables managed by the Hive metastore. It is implemented as follows
- During planning time, the full set of partitions affected by an INSERT or OVERWRITE command is read from the Hive metastore.
- The planner identifies any partitions with custom locations and includes this in the write task metadata.
- FileFormatWriter tasks refer to this custom locations map when determining where to write for dynamic partition output.
- When the write job finishes, the set of written partitions is compared against the initial set of matched partitions, and the Hive metastore is updated to reflect the newly added / removed partitions.

It was necessary to introduce a method for staging files with absolute output paths to `FileCommitProtocol`. These files are not handled by the Hadoop output committer but are moved to their final locations when the job commits.

The overwrite behavior of legacy Datasource tables is also changed: no longer will the entire table be overwritten if a partial partition spec is present.

cc cloud-fan yhuai

## How was this patch tested?

Unit tests, existing tests.

Author: Eric Liang <ekl@databricks.com>
Author: Wenchen Fan <wenchen@databricks.com>

Closes #15814 from ericl/sc-5027.
2016-11-10 17:00:43 -08:00
Cheng Lian e0deee1f7d [SPARK-18403][SQL] Temporarily disable flaky ObjectHashAggregateSuite
## What changes were proposed in this pull request?

Randomized tests in `ObjectHashAggregateSuite` is being flaky and breaks PR builds. This PR disables them temporarily to bring back the PR build.

## How was this patch tested?

N/A

Author: Cheng Lian <lian@databricks.com>

Closes #15845 from liancheng/ignore-flaky-object-hash-agg-suite.
2016-11-10 13:44:54 -08:00
Wenchen Fan 2f7461f313 [SPARK-17990][SPARK-18302][SQL] correct several partition related behaviours of ExternalCatalog
## What changes were proposed in this pull request?

This PR corrects several partition related behaviors of `ExternalCatalog`:

1. default partition location should not always lower case the partition column names in path string(fix `HiveExternalCatalog`)
2. rename partition should not always lower case the partition column names in updated partition path string(fix `HiveExternalCatalog`)
3. rename partition should update the partition location only for managed table(fix `InMemoryCatalog`)
4. create partition with existing directory should be fine(fix `InMemoryCatalog`)
5. create partition with non-existing directory should create that directory(fix `InMemoryCatalog`)
6. drop partition from external table should not delete the directory(fix `InMemoryCatalog`)

## How was this patch tested?

new tests in `ExternalCatalogSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15797 from cloud-fan/partition.
2016-11-10 13:42:48 -08:00
Michael Allman b533fa2b20 [SPARK-17993][SQL] Fix Parquet log output redirection
(Link to Jira issue: https://issues.apache.org/jira/browse/SPARK-17993)
## What changes were proposed in this pull request?

PR #14690 broke parquet log output redirection for converted partitioned Hive tables. For example, when querying parquet files written by Parquet-mr 1.6.0 Spark prints a torrent of (harmless) warning messages from the Parquet reader:

```
Oct 18, 2016 7:42:18 PM WARNING: org.apache.parquet.CorruptStatistics: Ignoring statistics because created_by could not be parsed (see PARQUET-251): parquet-mr version 1.6.0
org.apache.parquet.VersionParser$VersionParseException: Could not parse created_by: parquet-mr version 1.6.0 using format: (.+) version ((.*) )?\(build ?(.*)\)
    at org.apache.parquet.VersionParser.parse(VersionParser.java:112)
    at org.apache.parquet.CorruptStatistics.shouldIgnoreStatistics(CorruptStatistics.java:60)
    at org.apache.parquet.format.converter.ParquetMetadataConverter.fromParquetStatistics(ParquetMetadataConverter.java:263)
    at org.apache.parquet.hadoop.ParquetFileReader$Chunk.readAllPages(ParquetFileReader.java:583)
    at org.apache.parquet.hadoop.ParquetFileReader.readNextRowGroup(ParquetFileReader.java:513)
    at org.apache.spark.sql.execution.datasources.parquet.VectorizedParquetRecordReader.checkEndOfRowGroup(VectorizedParquetRecordReader.java:270)
    at org.apache.spark.sql.execution.datasources.parquet.VectorizedParquetRecordReader.nextBatch(VectorizedParquetRecordReader.java:225)
    at org.apache.spark.sql.execution.datasources.parquet.VectorizedParquetRecordReader.nextKeyValue(VectorizedParquetRecordReader.java:137)
    at org.apache.spark.sql.execution.datasources.RecordReaderIterator.hasNext(RecordReaderIterator.scala:39)
    at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:102)
    at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.nextIterator(FileScanRDD.scala:162)
    at org.apache.spark.sql.execution.datasources.FileScanRDD$$anon$1.hasNext(FileScanRDD.scala:102)
    at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.scan_nextBatch$(Unknown Source)
    at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source)
    at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
    at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:372)
    at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:231)
    at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:225)
    at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:803)
    at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:803)
    at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
    at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319)
    at org.apache.spark.rdd.RDD.iterator(RDD.scala:283)
    at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
    at org.apache.spark.scheduler.Task.run(Task.scala:99)
    at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)
    at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
    at java.lang.Thread.run(Thread.java:745)
```

This only happens during execution, not planning, and it doesn't matter what log level the `SparkContext` is set to. That's because Parquet (versions < 1.9) doesn't use slf4j for logging. Note, you can tell that log redirection is not working here because the log message format does not conform to the default Spark log message format.

This is a regression I noted as something we needed to fix as a follow up.

It appears that the problem arose because we removed the call to `inferSchema` during Hive table conversion. That call is what triggered the output redirection.

## How was this patch tested?

I tested this manually in four ways:
1. Executing `spark.sqlContext.range(10).selectExpr("id as a").write.mode("overwrite").parquet("test")`.
2. Executing `spark.read.format("parquet").load(legacyParquetFile).show` for a Parquet file `legacyParquetFile` written using Parquet-mr 1.6.0.
3. Executing `select * from legacy_parquet_table limit 1` for some unpartitioned Parquet-based Hive table written using Parquet-mr 1.6.0.
4. Executing `select * from legacy_partitioned_parquet_table where partcol=x limit 1` for some partitioned Parquet-based Hive table written using Parquet-mr 1.6.0.

I ran each test with a new instance of `spark-shell` or `spark-sql`.

Incidentally, I found that test case 3 was not a regression—redirection was not occurring in the master codebase prior to #14690.

I spent some time working on a unit test, but based on my experience working on this ticket I feel that automated testing here is far from feasible.

cc ericl dongjoon-hyun

Author: Michael Allman <michael@videoamp.com>

Closes #15538 from mallman/spark-17993-fix_parquet_log_redirection.
2016-11-10 13:41:13 -08:00
Herman van Hovell d8b81f778a [SPARK-18370][SQL] Add table information to InsertIntoHadoopFsRelationCommand
## What changes were proposed in this pull request?
`InsertIntoHadoopFsRelationCommand` does not keep track if it inserts into a table and what table it inserts to. This can make debugging these statements problematic. This PR adds table information the `InsertIntoHadoopFsRelationCommand`. Explaining this SQL command `insert into prq select * from range(0, 100000)` now yields the following executed plan:
```
== Physical Plan ==
ExecutedCommand
   +- InsertIntoHadoopFsRelationCommand file:/dev/assembly/spark-warehouse/prq, ParquetFormat, <function1>, Map(serialization.format -> 1, path -> file:/dev/assembly/spark-warehouse/prq), Append, CatalogTable(
	Table: `default`.`prq`
	Owner: hvanhovell
	Created: Wed Nov 09 17:42:30 CET 2016
	Last Access: Thu Jan 01 01:00:00 CET 1970
	Type: MANAGED
	Schema: [StructField(id,LongType,true)]
	Provider: parquet
	Properties: [transient_lastDdlTime=1478709750]
	Storage(Location: file:/dev/assembly/spark-warehouse/prq, InputFormat: org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat, OutputFormat: org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat, Serde: org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe, Properties: [serialization.format=1]))
         +- Project [id#7L]
            +- Range (0, 100000, step=1, splits=None)
```

## How was this patch tested?
Added extra checks to the `ParquetMetastoreSuite`

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

Closes #15832 from hvanhovell/SPARK-18370.
2016-11-09 12:26:09 -08:00
Cheng Lian 205e6d5867 [SPARK-18338][SQL][TEST-MAVEN] Fix test case initialization order under Maven builds
## What changes were proposed in this pull request?

Test case initialization order under Maven and SBT are different. Maven always creates instances of all test cases and then run them all together.

This fails `ObjectHashAggregateSuite` because the randomized test cases there register a temporary Hive function right before creating a test case, and can be cleared while initializing other successive test cases. In SBT, this is fine since the created test case is executed immediately after creating the temporary function.

To fix this issue, we should put initialization/destruction code into `beforeAll()` and `afterAll()`.

## How was this patch tested?

Existing tests.

Author: Cheng Lian <lian@databricks.com>

Closes #15802 from liancheng/fix-flaky-object-hash-agg-suite.
2016-11-09 09:49:02 -08:00
Dongjoon Hyun 02c5325b8f
[SPARK-18292][SQL] LogicalPlanToSQLSuite should not use resource dependent path for golden file generation
## What changes were proposed in this pull request?

`LogicalPlanToSQLSuite` uses the following command to update the existing answer files.

```bash
SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "hive/test-only *LogicalPlanToSQLSuite"
```

However, after introducing `getTestResourcePath`, it fails to update the previous golden answer files in the predefined directory. This issue aims to fix that.

## How was this patch tested?

It's a testsuite update. Manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15789 from dongjoon-hyun/SPARK-18292.
2016-11-09 17:48:16 +00:00
gatorsmile e256392a12 [SPARK-17659][SQL] Partitioned View is Not Supported By SHOW CREATE TABLE
### What changes were proposed in this pull request?

`Partitioned View` is not supported by SPARK SQL. For Hive partitioned view, SHOW CREATE TABLE is unable to generate the right DDL. Thus, SHOW CREATE TABLE should not support it like the other Hive-only features. This PR is to issue an exception when detecting the view is a partitioned view.
### How was this patch tested?

Added a test case

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15233 from gatorsmile/partitionedView.
2016-11-09 00:11:48 -08:00
Eric Liang 4afa39e223 [SPARK-18333][SQL] Revert hacks in parquet and orc reader to support case insensitive resolution
## What changes were proposed in this pull request?

These are no longer needed after https://issues.apache.org/jira/browse/SPARK-17183

cc cloud-fan

## How was this patch tested?

Existing parquet and orc tests.

Author: Eric Liang <ekl@databricks.com>

Closes #15799 from ericl/sc-4929.
2016-11-09 15:00:46 +08:00
jiangxingbo 9c419698fe [SPARK-18191][CORE] Port RDD API to use commit protocol
## What changes were proposed in this pull request?

This PR port RDD API to use commit protocol, the changes made here:
1. Add new internal helper class that saves an RDD using a Hadoop OutputFormat named `SparkNewHadoopWriter`, it's similar with `SparkHadoopWriter` but uses commit protocol. This class supports the newer `mapreduce` API, instead of the old `mapred` API which is supported by `SparkHadoopWriter`;
2. Rewrite `PairRDDFunctions.saveAsNewAPIHadoopDataset` function, so it uses commit protocol now.

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

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15769 from jiangxb1987/rdd-commit.
2016-11-08 09:41:01 -08:00
Wenchen Fan 73feaa30eb [SPARK-18346][SQL] TRUNCATE TABLE should fail if no partition is matched for the given non-partial partition spec
## What changes were proposed in this pull request?

a follow up of https://github.com/apache/spark/pull/15688

## How was this patch tested?

updated test in `DDLSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15805 from cloud-fan/truncate.
2016-11-08 22:28:29 +08:00
root c291bd2745 [SPARK-18137][SQL] Fix RewriteDistinctAggregates UnresolvedException when a UDAF has a foldable TypeCheck
## What changes were proposed in this pull request?

In RewriteDistinctAggregates rewrite funtion,after the UDAF's childs are mapped to AttributeRefference, If the UDAF(such as ApproximatePercentile) has a foldable TypeCheck for the input, It will failed because the AttributeRefference is not foldable,then the UDAF is not resolved, and then nullify on the unresolved object will throw a Exception.

In this PR, only map Unfoldable child to AttributeRefference, this can avoid the UDAF's foldable TypeCheck. and then only Expand Unfoldable child, there is no need to Expand a static value(foldable value).

**Before sql result**

> select percentile_approxy(key,0.99999),count(distinct key),sume(distinc key) from src limit 1
> org.apache.spark.sql.catalyst.analysis.UnresolvedException: Invalid call to dataType on unresolved object, tree: 'percentile_approx(CAST(src.`key` AS DOUBLE), CAST(0.99999BD AS DOUBLE), 10000)
> at org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute.dataType(unresolved.scala:92)
>     at org.apache.spark.sql.catalyst.optimizer.RewriteDistinctAggregates$.org$apache$spark$sql$catalyst$optimizer$RewriteDistinctAggregates$$nullify(RewriteDistinctAggregates.scala:261)

**After sql result**

> select percentile_approxy(key,0.99999),count(distinct key),sume(distinc key) from src limit 1
> [498.0,309,79136]
## How was this patch tested?

Add a test case in HiveUDFSuit.

Author: root <root@iZbp1gsnrlfzjxh82cz80vZ.(none)>

Closes #15668 from windpiger/RewriteDistinctUDAFUnresolveExcep.
2016-11-08 12:09:32 +01:00
gatorsmile 1da64e1fa0 [SPARK-18217][SQL] Disallow creating permanent views based on temporary views or UDFs
### What changes were proposed in this pull request?
Based on the discussion in [SPARK-18209](https://issues.apache.org/jira/browse/SPARK-18209). It doesn't really make sense to create permanent views based on temporary views or temporary UDFs.

To disallow the supports and issue the exceptions, this PR needs to detect whether a temporary view/UDF is being used when defining a permanent view. Basically, this PR can be split to two sub-tasks:

**Task 1:** detecting a temporary view from the query plan of view definition.
When finding an unresolved temporary view, Analyzer replaces it by a `SubqueryAlias` with the corresponding logical plan, which is stored in an in-memory HashMap. After replacement, it is impossible to detect whether the `SubqueryAlias` is added/generated from a temporary view. Thus, to detect the usage of a temporary view in view definition, this PR traverses the unresolved logical plan and uses the name of an `UnresolvedRelation` to detect whether it is a (global) temporary view.

**Task 2:** detecting a temporary UDF from the query plan of view definition.
Detecting usage of a temporary UDF in view definition is not straightfoward.

First, in the analyzed plan, we are having different forms to represent the functions. More importantly, some classes (e.g., `HiveGenericUDF`) are not accessible from `CreateViewCommand`, which is part of  `sql/core`. Thus, we used the unanalyzed plan `child` of `CreateViewCommand` to detect the usage of a temporary UDF. Because the plan has already been successfully analyzed, we can assume the functions have been defined/registered.

Second, in Spark, the functions have four forms: Spark built-in functions, built-in hash functions, permanent UDFs and temporary UDFs. We do not have any direct way to determine whether a function is temporary or not. Thus, we introduced a function `isTemporaryFunction` in `SessionCatalog`. This function contains the detailed logics to determine whether a function is temporary or not.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15764 from gatorsmile/blockTempFromPermViewCreation.
2016-11-07 18:34:21 -08:00
Ryan Blue 9b0593d5e9 [SPARK-18086] Add support for Hive session vars.
## What changes were proposed in this pull request?

This adds support for Hive variables:

* Makes values set via `spark-sql --hivevar name=value` accessible
* Adds `getHiveVar` and `setHiveVar` to the `HiveClient` interface
* Adds a SessionVariables trait for sessions like Hive that support variables (including Hive vars)
* Adds SessionVariables support to variable substitution
* Adds SessionVariables support to the SET command

## How was this patch tested?

* Adds a test to all supported Hive versions for accessing Hive variables
* Adds HiveVariableSubstitutionSuite

Author: Ryan Blue <blue@apache.org>

Closes #15738 from rdblue/SPARK-18086-add-hivevar-support.
2016-11-07 17:36:15 -08:00
Weiqing Yang 0d95662e7f [SPARK-17108][SQL] Fix BIGINT and INT comparison failure in spark sql
## What changes were proposed in this pull request?

Add a function to check if two integers are compatible when invoking `acceptsType()` in `DataType`.
## How was this patch tested?

Manually.
E.g.

```
    spark.sql("create table t3(a map<bigint, array<string>>)")
    spark.sql("select * from t3 where a[1] is not null")
```

Before:

```
cannot resolve 't.`a`[1]' due to data type mismatch: argument 2 requires bigint type, however, '1' is of int type.; line 1 pos 22
org.apache.spark.sql.AnalysisException: cannot resolve 't.`a`[1]' due to data type mismatch: argument 2 requires bigint type, however, '1' is of int type.; line 1 pos 22
    at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42)
    at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$1$$anonfun$apply$2.applyOrElse(CheckAnalysis.scala:82)
    at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$1$$anonfun$apply$2.applyOrElse(CheckAnalysis.scala:74)
at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:307)
```

After:
 Run the sql queries above. No errors.

Author: Weiqing Yang <yangweiqing001@gmail.com>

Closes #15448 from weiqingy/SPARK_17108.
2016-11-07 21:33:01 +01:00
gatorsmile 57626a5570 [SPARK-16904][SQL] Removal of Hive Built-in Hash Functions and TestHiveFunctionRegistry
### What changes were proposed in this pull request?

Currently, the Hive built-in `hash` function is not being used in Spark since Spark 2.0. The public interface does not allow users to unregister the Spark built-in functions. Thus, users will never use Hive's built-in `hash` function.

The only exception here is `TestHiveFunctionRegistry`, which allows users to unregister the built-in functions. Thus, we can load Hive's hash function in the test cases. If we disable it, 10+ test cases will fail because the results are different from the Hive golden answer files.

This PR is to remove `hash` from the list of `hiveFunctions` in `HiveSessionCatalog`. It will also remove `TestHiveFunctionRegistry`. This removal makes us easier to remove `TestHiveSessionState` in the future.
### How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14498 from gatorsmile/removeHash.
2016-11-07 01:16:37 -08:00
Reynold Xin 07ac3f09da [SPARK-18167][SQL] Disable flaky hive partition pruning test. 2016-11-06 22:42:05 -08:00
Wenchen Fan 46b2e49993 [SPARK-18173][SQL] data source tables should support truncating partition
## What changes were proposed in this pull request?

Previously `TRUNCATE TABLE ... PARTITION` will always truncate the whole table for data source tables, this PR fixes it and improve `InMemoryCatalog` to make this command work with it.
## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15688 from cloud-fan/truncate.
2016-11-06 18:57:13 -08:00
Wenchen Fan 95ec4e25bb [SPARK-17183][SPARK-17983][SPARK-18101][SQL] put hive serde table schema to table properties like data source table
## What changes were proposed in this pull request?

For data source tables, we will put its table schema, partition columns, etc. to table properties, to work around some hive metastore issues, e.g. not case-preserving, bad decimal type support, etc.

We should also do this for hive serde tables, to reduce the difference between hive serde tables and data source tables, e.g. column names should be case preserving.
## How was this patch tested?

existing tests, and a new test in `HiveExternalCatalog`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14750 from cloud-fan/minor1.
2016-11-05 00:58:50 -07:00
Eric Liang 4cee2ce251 [SPARK-18167] Re-enable the non-flaky parts of SQLQuerySuite
## What changes were proposed in this pull request?

It seems the proximate cause of the test failures is that `cast(str as decimal)` in derby will raise an exception instead of returning NULL. This is a problem since Hive sometimes inserts `__HIVE_DEFAULT_PARTITION__` entries into the partition table as documented here: https://github.com/apache/hive/blob/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java#L1034

Basically, when these special default partitions are present, partition pruning pushdown using the SQL-direct mode will fail due this cast exception. As commented on in `MetaStoreDirectSql.java` above, this is normally fine since Hive falls back to JDO pruning, however when the pruning predicate contains an unsupported operator such as `>`, that will fail as well.

The only remaining question is why this behavior is nondeterministic. We know that when the test flakes, retries do not help, therefore the cause must be environmental. The current best hypothesis is that some config is different between different jenkins runs, which is why this PR prints out the Spark SQL and Hive confs for the test. The hope is that by comparing the config state for failure vs success we can isolate the root cause of the flakiness.

**Update:** we could not isolate the issue. It does not seem to be due to configuration differences. As such, I'm going to enable the non-flaky parts of the test since we are fairly confident these issues only occur with Derby (which is not used in production).

## How was this patch tested?

N/A

Author: Eric Liang <ekl@databricks.com>

Closes #15725 from ericl/print-confs-out.
2016-11-04 15:54:28 -07:00
福星 16293311cd [SPARK-18237][HIVE] hive.exec.stagingdir have no effect
hive.exec.stagingdir have no effect in spark2.0.1,
Hive confs in hive-site.xml will be loaded in `hadoopConf`, so we should use `hadoopConf` in `InsertIntoHiveTable` instead of `SessionState.conf`

Author: 福星 <fuxing@wacai.com>

Closes #15744 from ClassNotFoundExp/master.
2016-11-03 12:02:01 -07:00
Reynold Xin b17057c0a6 [SPARK-18244][SQL] Rename partitionProviderIsHive -> tracksPartitionsInCatalog
## What changes were proposed in this pull request?
This patch renames partitionProviderIsHive to tracksPartitionsInCatalog, as the old name was too Hive specific.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #15750 from rxin/SPARK-18244.
2016-11-03 11:48:05 -07:00
Cheng Lian 27daf6bcde [SPARK-17949][SQL] A JVM object based aggregate operator
## What changes were proposed in this pull request?

This PR adds a new hash-based aggregate operator named `ObjectHashAggregateExec` that supports `TypedImperativeAggregate`, which may use arbitrary Java objects as aggregation states. Please refer to the [design doc](https://issues.apache.org/jira/secure/attachment/12834260/%5BDesign%20Doc%5D%20Support%20for%20Arbitrary%20Aggregation%20States.pdf) attached in [SPARK-17949](https://issues.apache.org/jira/browse/SPARK-17949) for more details about it.

The major benefit of this operator is better performance when evaluating `TypedImperativeAggregate` functions, especially when there are relatively few distinct groups. Functions like Hive UDAFs, `collect_list`, and `collect_set` may also benefit from this after being migrated to `TypedImperativeAggregate`.

The following feature flag is introduced to enable or disable the new aggregate operator:
- Name: `spark.sql.execution.useObjectHashAggregateExec`
- Default value: `true`

We can also configure the fallback threshold using the following SQL operation:
- Name: `spark.sql.objectHashAggregate.sortBased.fallbackThreshold`
- Default value: 128

  Fallback to sort-based aggregation when more than 128 distinct groups are accumulated in the aggregation hash map. This number is intentionally made small to avoid GC problems since aggregation buffers of this operator may contain arbitrary Java objects.

  This may be improved by implementing size tracking for this operator, but that can be done in a separate PR.

Code generation and size tracking are planned to be implemented in follow-up PRs.
## Benchmark results
### `ObjectHashAggregateExec` vs `SortAggregateExec`

The first benchmark compares `ObjectHashAggregateExec` and `SortAggregateExec` by evaluating `typed_count`, a testing `TypedImperativeAggregate` version of the SQL `count` function.

```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.5
Intel(R) Core(TM) i7-4960HQ CPU  2.60GHz

object agg v.s. sort agg:                Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
sort agg w/ group by                        31251 / 31908          3.4         298.0       1.0X
object agg w/ group by w/o fallback           6903 / 7141         15.2          65.8       4.5X
object agg w/ group by w/ fallback          20945 / 21613          5.0         199.7       1.5X
sort agg w/o group by                         4734 / 5463         22.1          45.2       6.6X
object agg w/o group by w/o fallback          4310 / 4529         24.3          41.1       7.3X
```

The next benchmark compares `ObjectHashAggregateExec` and `SortAggregateExec` by evaluating the Spark native version of `percentile_approx`.

Note that `percentile_approx` is so heavy an aggregate function that the bottleneck of the benchmark is evaluating the aggregate function itself rather than the aggregate operator since I couldn't run a large scale benchmark on my laptop. That's why the results are so close and looks counter-intuitive (aggregation with grouping is even faster than that aggregation without grouping).

```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.5
Intel(R) Core(TM) i7-4960HQ CPU  2.60GHz

object agg v.s. sort agg:                Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
sort agg w/ group by                          3418 / 3530          0.6        1630.0       1.0X
object agg w/ group by w/o fallback           3210 / 3314          0.7        1530.7       1.1X
object agg w/ group by w/ fallback            3419 / 3511          0.6        1630.1       1.0X
sort agg w/o group by                         4336 / 4499          0.5        2067.3       0.8X
object agg w/o group by w/o fallback          4271 / 4372          0.5        2036.7       0.8X
```
### Hive UDAF vs Spark AF

This benchmark compares the following two kinds of aggregate functions:
- "hive udaf": Hive implementation of `percentile_approx`, without partial aggregation supports, evaluated using `SortAggregateExec`.
- "spark af": Spark native implementation of `percentile_approx`, with partial aggregation support, evaluated using `ObjectHashAggregateExec`

The performance differences are mostly due to faster implementation and partial aggregation support in the Spark native version of `percentile_approx`.

This benchmark basically shows the performance differences between the worst case, where an aggregate function without partial aggregation support is evaluated using `SortAggregateExec`, and the best case, where a `TypedImperativeAggregate` with partial aggregation support is evaluated using `ObjectHashAggregateExec`.

```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.5
Intel(R) Core(TM) i7-4960HQ CPU  2.60GHz

hive udaf vs spark af:                   Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
hive udaf w/o group by                        5326 / 5408          0.0       81264.2       1.0X
spark af w/o group by                           93 /  111          0.7        1415.6      57.4X
hive udaf w/ group by                         3804 / 3946          0.0       58050.1       1.4X
spark af w/ group by w/o fallback               71 /   90          0.9        1085.7      74.8X
spark af w/ group by w/ fallback                98 /  111          0.7        1501.6      54.1X
```
### Real world benchmark

We also did a relatively large benchmark using a real world query involving `percentile_approx`:
- Hive UDAF implementation, sort-based aggregation, w/o partial aggregation support

  24.77 minutes
- Native implementation, sort-based aggregation, w/ partial aggregation support

  4.64 minutes
- Native implementation, object hash aggregator, w/ partial aggregation support

  1.80 minutes
## How was this patch tested?

New unit tests and randomized test cases are added in `ObjectAggregateFunctionSuite`.

Author: Cheng Lian <lian@databricks.com>

Closes #15590 from liancheng/obj-hash-agg.
2016-11-03 09:34:51 -07:00
Reynold Xin 0ea5d5b24c [SQL] minor - internal doc improvement for InsertIntoTable.
## What changes were proposed in this pull request?
I was reading this part of the code and was really confused by the "partition" parameter. This patch adds some documentation for it to reduce confusion in the future.

I also looked around other logical plans but most of them are either already documented, or pretty self-evident to people that know Spark SQL.

## How was this patch tested?
N/A - doc change only.

Author: Reynold Xin <rxin@databricks.com>

Closes #15749 from rxin/doc-improvement.
2016-11-03 02:45:54 -07:00
hyukjinkwon 7eb2ca8e33 [SPARK-17963][SQL][DOCUMENTATION] Add examples (extend) in each expression and improve documentation
## What changes were proposed in this pull request?

This PR proposes to change the documentation for functions. Please refer the discussion from https://github.com/apache/spark/pull/15513

The changes include
- Re-indent the documentation
- Add examples/arguments in `extended` where the arguments are multiple or specific format (e.g. xml/ json).

For examples, the documentation was updated as below:
### Functions with single line usage

**Before**
- `pow`

  ``` sql
  Usage: pow(x1, x2) - Raise x1 to the power of x2.
  Extended Usage:
  > SELECT pow(2, 3);
   8.0
  ```
- `current_timestamp`

  ``` sql
  Usage: current_timestamp() - Returns the current timestamp at the start of query evaluation.
  Extended Usage:
  No example for current_timestamp.
  ```

**After**
- `pow`

  ``` sql
  Usage: pow(expr1, expr2) - Raises `expr1` to the power of `expr2`.
  Extended Usage:
      Examples:
        > SELECT pow(2, 3);
         8.0
  ```

- `current_timestamp`

  ``` sql
  Usage: current_timestamp() - Returns the current timestamp at the start of query evaluation.
  Extended Usage:
      No example/argument for current_timestamp.
  ```
### Functions with (already) multiple line usage

**Before**
- `approx_count_distinct`

  ``` sql
  Usage: approx_count_distinct(expr) - Returns the estimated cardinality by HyperLogLog++.
      approx_count_distinct(expr, relativeSD=0.05) - Returns the estimated cardinality by HyperLogLog++
        with relativeSD, the maximum estimation error allowed.

  Extended Usage:
  No example for approx_count_distinct.
  ```
- `percentile_approx`

  ``` sql
  Usage:
        percentile_approx(col, percentage [, accuracy]) - Returns the approximate percentile value of numeric
        column `col` at the given percentage. The value of percentage must be between 0.0
        and 1.0. The `accuracy` parameter (default: 10000) is a positive integer literal which
        controls approximation accuracy at the cost of memory. Higher value of `accuracy` yields
        better accuracy, `1.0/accuracy` is the relative error of the approximation.

        percentile_approx(col, array(percentage1 [, percentage2]...) [, accuracy]) - Returns the approximate
        percentile array of column `col` at the given percentage array. Each value of the
        percentage array must be between 0.0 and 1.0. The `accuracy` parameter (default: 10000) is
        a positive integer literal which controls approximation accuracy at the cost of memory.
        Higher value of `accuracy` yields better accuracy, `1.0/accuracy` is the relative error of
        the approximation.

  Extended Usage:
  No example for percentile_approx.
  ```

**After**
- `approx_count_distinct`

  ``` sql
  Usage:
      approx_count_distinct(expr[, relativeSD]) - Returns the estimated cardinality by HyperLogLog++.
        `relativeSD` defines the maximum estimation error allowed.

  Extended Usage:
      No example/argument for approx_count_distinct.
  ```

- `percentile_approx`

  ``` sql
  Usage:
      percentile_approx(col, percentage [, accuracy]) - Returns the approximate percentile value of numeric
        column `col` at the given percentage. The value of percentage must be between 0.0
        and 1.0. The `accuracy` parameter (default: 10000) is a positive numeric literal which
        controls approximation accuracy at the cost of memory. Higher value of `accuracy` yields
        better accuracy, `1.0/accuracy` is the relative error of the approximation.
        When `percentage` is an array, each value of the percentage array must be between 0.0 and 1.0.
        In this case, returns the approximate percentile array of column `col` at the given
        percentage array.

  Extended Usage:
      Examples:
        > SELECT percentile_approx(10.0, array(0.5, 0.4, 0.1), 100);
         [10.0,10.0,10.0]
        > SELECT percentile_approx(10.0, 0.5, 100);
         10.0
  ```
## How was this patch tested?

Manually tested

**When examples are multiple**

``` sql
spark-sql> describe function extended reflect;
Function: reflect
Class: org.apache.spark.sql.catalyst.expressions.CallMethodViaReflection
Usage: reflect(class, method[, arg1[, arg2 ..]]) - Calls a method with reflection.
Extended Usage:
    Examples:
      > SELECT reflect('java.util.UUID', 'randomUUID');
       c33fb387-8500-4bfa-81d2-6e0e3e930df2
      > SELECT reflect('java.util.UUID', 'fromString', 'a5cf6c42-0c85-418f-af6c-3e4e5b1328f2');
       a5cf6c42-0c85-418f-af6c-3e4e5b1328f2
```

**When `Usage` is in single line**

``` sql
spark-sql> describe function extended min;
Function: min
Class: org.apache.spark.sql.catalyst.expressions.aggregate.Min
Usage: min(expr) - Returns the minimum value of `expr`.
Extended Usage:
    No example/argument for min.
```

**When `Usage` is already in multiple lines**

``` sql
spark-sql> describe function extended percentile_approx;
Function: percentile_approx
Class: org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile
Usage:
    percentile_approx(col, percentage [, accuracy]) - Returns the approximate percentile value of numeric
      column `col` at the given percentage. The value of percentage must be between 0.0
      and 1.0. The `accuracy` parameter (default: 10000) is a positive numeric literal which
      controls approximation accuracy at the cost of memory. Higher value of `accuracy` yields
      better accuracy, `1.0/accuracy` is the relative error of the approximation.
      When `percentage` is an array, each value of the percentage array must be between 0.0 and 1.0.
      In this case, returns the approximate percentile array of column `col` at the given
      percentage array.

Extended Usage:
    Examples:
      > SELECT percentile_approx(10.0, array(0.5, 0.4, 0.1), 100);
       [10.0,10.0,10.0]
      > SELECT percentile_approx(10.0, 0.5, 100);
       10.0
```

**When example/argument is missing**

``` sql
spark-sql> describe function extended rank;
Function: rank
Class: org.apache.spark.sql.catalyst.expressions.Rank
Usage:
    rank() - Computes the rank of a value in a group of values. The result is one plus the number
      of rows preceding or equal to the current row in the ordering of the partition. The values
      will produce gaps in the sequence.

Extended Usage:
    No example/argument for rank.
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15677 from HyukjinKwon/SPARK-17963-1.
2016-11-02 20:56:30 -07:00
Wenchen Fan 3a1bc6f478 [SPARK-17470][SQL] unify path for data source table and locationUri for hive serde table
## What changes were proposed in this pull request?

Due to a limitation of hive metastore(table location must be directory path, not file path), we always store `path` for data source table in storage properties, instead of the `locationUri` field. However, we should not expose this difference to `CatalogTable` level, but just treat it as a hack in `HiveExternalCatalog`, like we store table schema of data source table in table properties.

This PR unifies `path` and `locationUri` outside of `HiveExternalCatalog`, both data source table and hive serde table should use the `locationUri` field.

This PR also unifies the way we handle default table location for managed table. Previously, the default table location of hive serde managed table is set by external catalog, but the one of data source table is set by command. After this PR, we follow the hive way and the default table location is always set by external catalog.

For managed non-file-based tables, we will assign a default table location and create an empty directory for it, the table location will be removed when the table is dropped. This is reasonable as metastore doesn't care about whether a table is file-based or not, and an empty table directory has no harm.
For external non-file-based tables, ideally we can omit the table location, but due to a hive metastore issue, we will assign a random location to it, and remove it right after the table is created. See SPARK-15269 for more details. This is fine as it's well isolated in `HiveExternalCatalog`.

To keep the existing behaviour of the `path` option, in this PR we always add the `locationUri` to storage properties using key `path`, before passing storage properties to `DataSource` as data source options.
## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15024 from cloud-fan/path.
2016-11-02 18:05:14 -07:00
Xiangrui Meng 02f203107b [SPARK-14393][SQL] values generated by non-deterministic functions shouldn't change after coalesce or union
## What changes were proposed in this pull request?

When a user appended a column using a "nondeterministic" function to a DataFrame, e.g., `rand`, `randn`, and `monotonically_increasing_id`, the expected semantic is the following:
- The value in each row should remain unchanged, as if we materialize the column immediately, regardless of later DataFrame operations.

However, since we use `TaskContext.getPartitionId` to get the partition index from the current thread, the values from nondeterministic columns might change if we call `union` or `coalesce` after. `TaskContext.getPartitionId` returns the partition index of the current Spark task, which might not be the corresponding partition index of the DataFrame where we defined the column.

See the unit tests below or JIRA for examples.

This PR uses the partition index from `RDD.mapPartitionWithIndex` instead of `TaskContext` and fixes the partition initialization logic in whole-stage codegen, normal codegen, and codegen fallback. `initializeStatesForPartition(partitionIndex: Int)` was added to `Projection`, `Nondeterministic`, and `Predicate` (codegen) and initialized right after object creation in `mapPartitionWithIndex`. `newPredicate` now returns a `Predicate` instance rather than a function for proper initialization.
## How was this patch tested?

Unit tests. (Actually I'm not very confident that this PR fixed all issues without introducing new ones ...)

cc: rxin davies

Author: Xiangrui Meng <meng@databricks.com>

Closes #15567 from mengxr/SPARK-14393.
2016-11-02 11:41:49 -07:00
eyal farago f151bd1af8 [SPARK-16839][SQL] Simplify Struct creation code path
## What changes were proposed in this pull request?

Simplify struct creation, especially the aspect of `CleanupAliases` which missed some aliases when handling trees created by `CreateStruct`.

This PR includes:

1. A failing test (create struct with nested aliases, some of the aliases survive `CleanupAliases`).
2. A fix that transforms `CreateStruct` into a `CreateNamedStruct` constructor, effectively eliminating `CreateStruct` from all expression trees.
3. A `NamePlaceHolder` used by `CreateStruct` when column names cannot be extracted from unresolved `NamedExpression`.
4. A new Analyzer rule that resolves `NamePlaceHolder` into a string literal once the `NamedExpression` is resolved.
5. `CleanupAliases` code was simplified as it no longer has to deal with `CreateStruct`'s top level columns.

## How was this patch tested?
Running all tests-suits in package org.apache.spark.sql, especially including the analysis suite, making sure added test initially fails, after applying suggested fix rerun the entire analysis package successfully.

Modified few tests that expected `CreateStruct` which is now transformed into `CreateNamedStruct`.

Author: eyal farago <eyal farago>
Author: Herman van Hovell <hvanhovell@databricks.com>
Author: eyal farago <eyal.farago@gmail.com>
Author: Eyal Farago <eyal.farago@actimize.com>
Author: Hyukjin Kwon <gurwls223@gmail.com>
Author: eyalfa <eyal.farago@gmail.com>

Closes #15718 from hvanhovell/SPARK-16839-2.
2016-11-02 11:12:20 +01:00
Sean Owen 9c8deef64e
[SPARK-18076][CORE][SQL] Fix default Locale used in DateFormat, NumberFormat to Locale.US
## What changes were proposed in this pull request?

Fix `Locale.US` for all usages of `DateFormat`, `NumberFormat`
## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #15610 from srowen/SPARK-18076.
2016-11-02 09:39:15 +00:00
Eric Liang abefe2ec42 [SPARK-18183][SPARK-18184] Fix INSERT [INTO|OVERWRITE] TABLE ... PARTITION for Datasource tables
## What changes were proposed in this pull request?

There are a couple issues with the current 2.1 behavior when inserting into Datasource tables with partitions managed by Hive.

(1) OVERWRITE TABLE ... PARTITION will actually overwrite the entire table instead of just the specified partition.
(2) INSERT|OVERWRITE does not work with partitions that have custom locations.

This PR fixes both of these issues for Datasource tables managed by Hive. The behavior for legacy tables or when `manageFilesourcePartitions = false` is unchanged.

There is one other issue in that INSERT OVERWRITE with dynamic partitions will overwrite the entire table instead of just the updated partitions, but this behavior is pretty complicated to implement for Datasource tables. We should address that in a future release.

## How was this patch tested?

Unit tests.

Author: Eric Liang <ekl@databricks.com>

Closes #15705 from ericl/sc-4942.
2016-11-02 14:15:10 +08:00
Michael Allman 1bbf9ff634 [SPARK-17992][SQL] Return all partitions from HiveShim when Hive throws a metastore exception when attempting to fetch partitions by filter
(Link to Jira issue: https://issues.apache.org/jira/browse/SPARK-17992)
## What changes were proposed in this pull request?

We recently added table partition pruning for partitioned Hive tables converted to using `TableFileCatalog`. When the Hive configuration option `hive.metastore.try.direct.sql` is set to `false`, Hive will throw an exception for unsupported filter expressions. For example, attempting to filter on an integer partition column will throw a `org.apache.hadoop.hive.metastore.api.MetaException`.

I discovered this behavior because VideoAmp uses the CDH version of Hive with a Postgresql metastore DB. In this configuration, CDH sets `hive.metastore.try.direct.sql` to `false` by default, and queries that filter on a non-string partition column will fail.

Rather than throw an exception in query planning, this patch catches this exception, logs a warning and returns all table partitions instead. Clients of this method are already expected to handle the possibility that the filters will not be honored.
## How was this patch tested?

A unit test was added.

Author: Michael Allman <michael@videoamp.com>

Closes #15673 from mallman/spark-17992-catch_hive_partition_filter_exception.
2016-11-01 22:20:19 -07:00
Eric Liang cfac17ee1c [SPARK-18167] Disable flaky SQLQuerySuite test
We now know it's a persistent environmental issue that is causing this test to sometimes fail. One hypothesis is that some configuration is leaked from another suite, and depending on suite ordering this can cause this test to fail.

I am planning on mining the jenkins logs to try to narrow down which suite could be causing this. For now, disable the test.

Author: Eric Liang <ekl@databricks.com>

Closes #15720 from ericl/disable-flaky-test.
2016-11-01 12:35:34 -07:00
Herman van Hovell 0cba535af3 Revert "[SPARK-16839][SQL] redundant aliases after cleanupAliases"
This reverts commit 5441a6269e.
2016-11-01 17:30:37 +01:00
eyal farago 5441a6269e [SPARK-16839][SQL] redundant aliases after cleanupAliases
## What changes were proposed in this pull request?

Simplify struct creation, especially the aspect of `CleanupAliases` which missed some aliases when handling trees created by `CreateStruct`.

This PR includes:

1. A failing test (create struct with nested aliases, some of the aliases survive `CleanupAliases`).
2. A fix that transforms `CreateStruct` into a `CreateNamedStruct` constructor, effectively eliminating `CreateStruct` from all expression trees.
3. A `NamePlaceHolder` used by `CreateStruct` when column names cannot be extracted from unresolved `NamedExpression`.
4. A new Analyzer rule that resolves `NamePlaceHolder` into a string literal once the `NamedExpression` is resolved.
5. `CleanupAliases` code was simplified as it no longer has to deal with `CreateStruct`'s top level columns.

## How was this patch tested?

running all tests-suits in package org.apache.spark.sql, especially including the analysis suite, making sure added test initially fails, after applying suggested fix rerun the entire analysis package successfully.

modified few tests that expected `CreateStruct` which is now transformed into `CreateNamedStruct`.

Credit goes to hvanhovell for assisting with this PR.

Author: eyal farago <eyal farago>
Author: eyal farago <eyal.farago@gmail.com>
Author: Herman van Hovell <hvanhovell@databricks.com>
Author: Eyal Farago <eyal.farago@actimize.com>
Author: Hyukjin Kwon <gurwls223@gmail.com>
Author: eyalfa <eyal.farago@gmail.com>

Closes #14444 from eyalfa/SPARK-16839_redundant_aliases_after_cleanupAliases.
2016-11-01 17:12:20 +01:00
Liang-Chi Hsieh dd85eb5448 [SPARK-18107][SQL] Insert overwrite statement runs much slower in spark-sql than it does in hive-client
## What changes were proposed in this pull request?

As reported on the jira, insert overwrite statement runs much slower in Spark, compared with hive-client.

It seems there is a patch [HIVE-11940](ba21806b77) which largely improves insert overwrite performance on Hive. HIVE-11940 is patched after Hive 2.0.0.

Because Spark SQL uses older Hive library, we can not benefit from such improvement.

The reporter verified that there is also a big performance gap between Hive 1.2.1 (520.037 secs) and Hive 2.0.1 (35.975 secs) on insert overwrite execution.

Instead of upgrading to Hive 2.0 in Spark SQL, which might not be a trivial task, this patch provides an approach to delete the partition before asking Hive to load data files into the partition.

Note: The case reported on the jira is insert overwrite to partition. Since `Hive.loadTable` also uses the function to replace files, insert overwrite to table should has the same issue. We can take the same approach to delete the table first. I will upgrade this to include this.
## How was this patch tested?

Jenkins tests.

There are existing tests using insert overwrite statement. Those tests should be passed. I added a new test to specially test insert overwrite into partition.

For performance issue, as I don't have Hive 2.0 environment, this needs the reporter to verify it. Please refer to the jira.

Please review https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark before opening a pull request.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #15667 from viirya/improve-hive-insertoverwrite.
2016-11-01 00:24:08 -07:00
Reynold Xin d9d1465009 [SPARK-18024][SQL] Introduce an internal commit protocol API
## What changes were proposed in this pull request?
This patch introduces an internal commit protocol API that is used by the batch data source to do write commits. It currently has only one implementation that uses Hadoop MapReduce's OutputCommitter API. In the future, this commit API can be used to unify streaming and batch commits.

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

Author: Reynold Xin <rxin@databricks.com>
Author: Eric Liang <ekl@databricks.com>

Closes #15707 from rxin/SPARK-18024-2.
2016-10-31 22:23:38 -07:00
Eric Liang 7d6c87155c [SPARK-18167][SQL] Retry when the SQLQuerySuite test flakes
## What changes were proposed in this pull request?

This will re-run the flaky test a few times after it fails. This will help determine if it's due to nondeterministic test setup, or because of some environment issue (e.g. leaked config from another test).

cc yhuai

Author: Eric Liang <ekl@databricks.com>

Closes #15708 from ericl/spark-18167-3.
2016-10-31 20:23:22 -07:00
Eric Liang 6633b97b57 [SPARK-18167][SQL] Also log all partitions when the SQLQuerySuite test flakes
## What changes were proposed in this pull request?

One possibility for this test flaking is that we have corrupted the partition schema somehow in the tests, which causes the cast to decimal to fail in the call. This should at least show us the actual partition values.

## How was this patch tested?

Run it locally, it prints out something like `ArrayBuffer(test(partcol=0), test(partcol=1), test(partcol=2), test(partcol=3), test(partcol=4))`.

Author: Eric Liang <ekl@databricks.com>

Closes #15701 from ericl/print-more-info.
2016-10-31 16:26:52 -07:00
Eric Liang 90d3b91f4c [SPARK-18103][SQL] Rename *FileCatalog to *FileIndex
## What changes were proposed in this pull request?

To reduce the number of components in SQL named *Catalog, rename *FileCatalog to *FileIndex. A FileIndex is responsible for returning the list of partitions / files to scan given a filtering expression.

```
TableFileCatalog => CatalogFileIndex
FileCatalog => FileIndex
ListingFileCatalog => InMemoryFileIndex
MetadataLogFileCatalog => MetadataLogFileIndex
PrunedTableFileCatalog => PrunedInMemoryFileIndex
```

cc yhuai marmbrus

## How was this patch tested?

N/A

Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekhliang@gmail.com>

Closes #15634 from ericl/rename-file-provider.
2016-10-30 13:14:45 -07:00
Eric Liang d2d438d1d5 [SPARK-18167][SQL] Add debug code for SQLQuerySuite flakiness when metastore partition pruning is enabled
## What changes were proposed in this pull request?

org.apache.spark.sql.hive.execution.SQLQuerySuite is flaking when hive partition pruning is enabled.
Based on the stack traces, it seems to be an old issue where Hive fails to cast a numeric partition column ("Invalid character string format for type DECIMAL"). There are two possibilities here: either we are somehow corrupting the partition table to have non-decimal values in that column, or there is a transient issue with Derby.

This PR logs the result of the retry when this exception is encountered, so we can confirm what is going on.

## How was this patch tested?

n/a

cc yhuai

Author: Eric Liang <ekl@databricks.com>

Closes #15676 from ericl/spark-18167.
2016-10-29 06:49:57 +02:00
Sunitha Kambhampati ab5f938bc7 [SPARK-18121][SQL] Unable to query global temp views when hive support is enabled
## What changes were proposed in this pull request?

Issue:
Querying on a global temp view throws Table or view not found exception.

Fix:
Update the lookupRelation in HiveSessionCatalog to check for global temp views similar to the SessionCatalog.lookupRelation.

Before fix:
Querying on a global temp view ( for. e.g.:  select * from global_temp.v1)  throws Table or view not found exception

After fix:
Query succeeds and returns the right result.

## How was this patch tested?
- Two unit tests are added to check for global temp view for the code path when hive support is enabled.
- Regression unit tests were run successfully. ( build/sbt -Phive hive/test, build/sbt sql/test, build/sbt catalyst/test)

Author: Sunitha Kambhampati <skambha@us.ibm.com>

Closes #15649 from skambha/lookuprelationChanges.
2016-10-28 08:39:02 +08:00
Eric Liang ccb1154304 [SPARK-17970][SQL] store partition spec in metastore for data source table
## What changes were proposed in this pull request?

We should follow hive table and also store partition spec in metastore for data source table.
This brings 2 benefits:

1. It's more flexible to manage the table data files, as users can use `ADD PARTITION`, `DROP PARTITION` and `RENAME PARTITION`
2. We don't need to cache all file status for data source table anymore.

## How was this patch tested?

existing tests.

Author: Eric Liang <ekl@databricks.com>
Author: Michael Allman <michael@videoamp.com>
Author: Eric Liang <ekhliang@gmail.com>
Author: Wenchen Fan <wenchen@databricks.com>

Closes #15515 from cloud-fan/partition.
2016-10-27 14:22:30 -07:00
Wenchen Fan 6f31833dbe [SPARK-18026][SQL] should not always lowercase partition columns of partition spec in parser
## What changes were proposed in this pull request?

Currently we always lowercase the partition columns of partition spec in parser, with the assumption that table partition columns are always lowercased.

However, this is not true for data source tables, which are case preserving. It's safe for now because data source tables don't store partition spec in metastore and don't support `ADD PARTITION`, `DROP PARTITION`, `RENAME PARTITION`, but we should make our code future-proof.

This PR makes partition spec case preserving at parser, and improve the `PreprocessTableInsertion` analyzer rule to normalize the partition columns in partition spec, w.r.t. the table partition columns.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15566 from cloud-fan/partition-spec.
2016-10-25 15:00:33 +08:00
gatorsmile d479c52622 [SPARK-17409][SQL][FOLLOW-UP] Do Not Optimize Query in CTAS More Than Once
### What changes were proposed in this pull request?
This follow-up PR is for addressing the [comment](https://github.com/apache/spark/pull/15048).

We added two test cases based on the suggestion from yhuai . One is a new test case using the `saveAsTable` API to create a data source table. Another is for CTAS on Hive serde table.

Note: No need to backport this PR to 2.0. Will submit a new PR to backport the whole fix with new test cases to Spark 2.0

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15459 from gatorsmile/ctasOptimizedTestCases.
2016-10-25 10:47:11 +08:00
Wenchen Fan 84a3399908 [SPARK-18028][SQL] simplify TableFileCatalog
## What changes were proposed in this pull request?

Simplify/cleanup TableFileCatalog:

1. pass a `CatalogTable` instead of `databaseName` and `tableName` into `TableFileCatalog`, so that we don't need to fetch table metadata from metastore again
2. In `TableFileCatalog.filterPartitions0`, DO NOT set `PartitioningAwareFileCatalog.BASE_PATH_PARAM`. According to the [classdoc](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala#L189-L209), the default value of `basePath` already satisfies our need. What's more, if we set this parameter, we may break the case 2 which is metioned in the classdoc.
3. add `equals` and `hashCode` to `TableFileCatalog`
4. add `SessionCatalog.listPartitionsByFilter` which handles case sensitivity.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15568 from cloud-fan/table-file-catalog.
2016-10-25 08:42:21 +08:00
Sean Owen 4ecbe1b92f
[SPARK-17810][SQL] Default spark.sql.warehouse.dir is relative to local FS but can resolve as HDFS path
## What changes were proposed in this pull request?

Always resolve spark.sql.warehouse.dir as a local path, and as relative to working dir not home dir

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #15382 from srowen/SPARK-17810.
2016-10-24 10:44:45 +01:00
jiangxingbo b158256c2e [SPARK-18045][SQL][TESTS] Move HiveDataFrameAnalyticsSuite to package sql
## What changes were proposed in this pull request?

The testsuite `HiveDataFrameAnalyticsSuite` has nothing to do with HIVE, we should move it to package `sql`.
The original test cases in that suite are splited into two existing testsuites: `DataFrameAggregateSuite` tests for the functions and ~~`SQLQuerySuite`~~`SQLQueryTestSuite` tests for the SQL statements.

## How was this patch tested?
~~Modified `SQLQuerySuite` in package `sql`.~~
Add query file for `SQLQueryTestSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15582 from jiangxb1987/group-analytics-test.
2016-10-23 13:28:35 +02:00
Tejas Patil 21c7539a52 [SPARK-18038][SQL] Move output partitioning definition from UnaryNodeExec to its children
## What changes were proposed in this pull request?

Jira : https://issues.apache.org/jira/browse/SPARK-18038

This was a suggestion by rxin over one of the dev list discussion : http://apache-spark-developers-list.1001551.n3.nabble.com/Project-not-preserving-child-partitioning-td19417.html

His words:

>> It would be better (safer) to move the output partitioning definition into each of the operator and remove it from UnaryExecNode.

With this PR, following is the output partitioning and ordering for all the impls of `UnaryExecNode`.

UnaryExecNode's impl | outputPartitioning | outputOrdering | comment
------------ | ------------- | ------------ | ------------
AppendColumnsExec | child's | Nil | child's ordering can be used
AppendColumnsWithObjectExec | child's | Nil | child's ordering can be used
BroadcastExchangeExec | BroadcastPartitioning | Nil | -
CoalesceExec | UnknownPartitioning | Nil | -
CollectLimitExec | SinglePartition | Nil | -
DebugExec | child's | Nil | child's ordering can be used
DeserializeToObjectExec | child's | Nil | child's ordering can be used
ExpandExec | UnknownPartitioning | Nil | -
FilterExec | child's | child's | -
FlatMapGroupsInRExec | child's | Nil | child's ordering can be used
GenerateExec | child's | Nil | need to dig more
GlobalLimitExec | child's | child's | -
HashAggregateExec | child's | Nil | -
InputAdapter | child's | child's | -
InsertIntoHiveTable | child's | Nil | terminal node, doesn't need partitioning
LocalLimitExec | child's | child's | -
MapElementsExec | child's | child's | -
MapGroupsExec | child's | Nil | child's ordering can be used
MapPartitionsExec | child's | Nil | child's ordering can be used
ProjectExec | child's | child's | -
SampleExec | child's | Nil | child's ordering can be used
ScriptTransformation | child's | Nil | child's ordering can be used
SerializeFromObjectExec | child's | Nil | child's ordering can be used
ShuffleExchange | custom | Nil | -
SortAggregateExec | child's | sort over grouped exprs | -
SortExec | child's | custom | -
StateStoreRestoreExec  | child's | Nil | child's ordering can be used
StateStoreSaveExec | child's | Nil | child's ordering can be used
SubqueryExec | child's | child's | -
TakeOrderedAndProjectExec | SinglePartition | custom | -
WholeStageCodegenExec | child's | child's | -
WindowExec | child's | child's | -

## How was this patch tested?

This does NOT change any existing functionality so relying on existing tests

Author: Tejas Patil <tejasp@fb.com>

Closes #15575 from tejasapatil/SPARK-18038_UnaryNodeExec_output_partitioning.
2016-10-23 13:25:47 +02:00
Tejas Patil eff4aed1ac [SPARK-18035][SQL] Introduce performant and memory efficient APIs to create ArrayBasedMapData
## What changes were proposed in this pull request?

Jira: https://issues.apache.org/jira/browse/SPARK-18035

In HiveInspectors, I saw that converting Java map to Spark's `ArrayBasedMapData` spent quite sometime in buffer copying : https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala#L658

The reason being `map.toSeq` allocates a new buffer and copies the map entries to it: https://github.com/scala/scala/blob/2.11.x/src/library/scala/collection/MapLike.scala#L323

This copy is not needed as we get rid of it once we extract the key and value arrays.

Here is the call trace:

```
org.apache.spark.sql.hive.HiveInspectors$$anonfun$unwrapperFor$41.apply(HiveInspectors.scala:664)
scala.collection.AbstractMap.toSeq(Map.scala:59)
scala.collection.MapLike$class.toSeq(MapLike.scala:323)
scala.collection.AbstractMap.toBuffer(Map.scala:59)
scala.collection.MapLike$class.toBuffer(MapLike.scala:326)
scala.collection.AbstractTraversable.copyToBuffer(Traversable.scala:104)
scala.collection.TraversableOnce$class.copyToBuffer(TraversableOnce.scala:275)
scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:48)
scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:104)
scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:59)
scala.collection.AbstractIterable.foreach(Iterable.scala:54)
scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
scala.collection.AbstractIterator.foreach(Iterator.scala:1336)
scala.collection.Iterator$class.foreach(Iterator.scala:893)
scala.collection.generic.Growable$$anonfun$$plus$plus$eq$1.apply(Growable.scala:59)
scala.collection.generic.Growable$$anonfun$$plus$plus$eq$1.apply(Growable.scala:59)
```

Also, earlier code was populating keys and values arrays separately by iterating twice. The PR avoids double iteration of the map and does it in one iteration.

EDIT: During code review, there were several more places in the code which were found to do similar thing. The PR dedupes those instances and introduces convenient APIs which are performant and memory efficient

## Performance gains

The number is subjective and depends on how many map columns are accessed in the query and average entries per map. For one the queries that I tried out, I saw 3% CPU savings (end-to-end) for the query.

## How was this patch tested?

This does not change the end result produced so relying on existing tests.

Author: Tejas Patil <tejasp@fb.com>

Closes #15573 from tejasapatil/SPARK-18035_avoid_toSeq.
2016-10-22 20:43:43 -07:00
Eric Liang 3eca283aca [SPARK-17994][SQL] Add back a file status cache for catalog tables
## What changes were proposed in this pull request?

In SPARK-16980, we removed the full in-memory cache of table partitions in favor of loading only needed partitions from the metastore. This greatly improves the initial latency of queries that only read a small fraction of table partitions.

However, since the metastore does not store file statistics, we need to discover those from remote storage. With the loss of the in-memory file status cache this has to happen on each query, increasing the latency of repeated queries over the same partitions.

The proposal is to add back a per-table cache of partition contents, i.e. Map[Path, Array[FileStatus]]. This cache would be retained per-table, and can be invalidated through refreshTable() and refreshByPath(). Unlike the prior cache, it can be incrementally updated as new partitions are read.

## How was this patch tested?

Existing tests and new tests in `HiveTablePerfStatsSuite`.

cc mallman

Author: Eric Liang <ekl@databricks.com>
Author: Michael Allman <michael@videoamp.com>
Author: Eric Liang <ekhliang@gmail.com>

Closes #15539 from ericl/meta-cache.
2016-10-22 22:08:28 +08:00
Reynold Xin 3fbf5a58c2 [SPARK-18042][SQL] OutputWriter should expose file path written
## What changes were proposed in this pull request?
This patch adds a new "path" method on OutputWriter that returns the path of the file written by the OutputWriter. This is part of the necessary work to consolidate structured streaming and batch write paths.

The batch write path has a nice feature that each data source can define the extension of the files, and allow Spark to specify the staging directory and the prefix for the files. However, in the streaming path we need to collect the list of files written, and there is no interface right now to do that.

## How was this patch tested?
N/A - there is no behavior change and this should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #15580 from rxin/SPARK-18042.
2016-10-21 17:27:18 -07:00
Wenchen Fan 57e97fcbd6 [SPARK-18029][SQL] PruneFileSourcePartitions should not change the output of LogicalRelation
## What changes were proposed in this pull request?

In `PruneFileSourcePartitions`, we will replace the `LogicalRelation` with a pruned one. However, this replacement may change the output of the `LogicalRelation` if it doesn't have `expectedOutputAttributes`. This PR fixes it.

## How was this patch tested?

the new `PruneFileSourcePartitionsSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15569 from cloud-fan/partition-bug.
2016-10-21 12:27:53 +08:00
Reynold Xin 7f9ec19eae [SPARK-18021][SQL] Refactor file name specification for data sources
## What changes were proposed in this pull request?
Currently each data source OutputWriter is responsible for specifying the entire file name for each file output. This, however, does not make any sense because we rely on file naming schemes for certain behaviors in Spark SQL, e.g. bucket id. The current approach allows individual data sources to break the implementation of bucketing.

On the flip side, we also don't want to move file naming entirely out of data sources, because different data sources do want to specify different extensions.

This patch divides file name specification into two parts: the first part is a prefix specified by the caller of OutputWriter (in WriteOutput), and the second part is the suffix that can be specified by the OutputWriter itself. Note that a side effect of this change is that now all file based data sources also support bucketing automatically.

There are also some other minor cleanups:

- Removed the UUID passed through generic Configuration string
- Some minor rewrites for better clarity
- Renamed "path" in multiple places to "stagingDir", to more accurately reflect its meaning

## How was this patch tested?
This should be covered by existing data source tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #15562 from rxin/SPARK-18021.
2016-10-20 12:18:56 -07:00
Tejas Patil fb0894b3a8 [SPARK-17698][SQL] Join predicates should not contain filter clauses
## What changes were proposed in this pull request?

Jira : https://issues.apache.org/jira/browse/SPARK-17698

`ExtractEquiJoinKeys` is incorrectly using filter predicates as the join condition for joins. `canEvaluate` [0] tries to see if the an `Expression` can be evaluated using output of a given `Plan`. In case of filter predicates (eg. `a.id='1'`), the `Expression` passed for the right hand side (ie. '1' ) is a `Literal` which does not have any attribute references. Thus `expr.references` is an empty set which theoretically is a subset of any set. This leads to `canEvaluate` returning `true` and `a.id='1'` is treated as a join predicate. While this does not lead to incorrect results but in case of bucketed + sorted tables, we might miss out on avoiding un-necessary shuffle + sort. See example below:

[0] : https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala#L91

eg.

```
val df = (1 until 10).toDF("id").coalesce(1)
hc.sql("DROP TABLE IF EXISTS table1").collect
df.write.bucketBy(8, "id").sortBy("id").saveAsTable("table1")
hc.sql("DROP TABLE IF EXISTS table2").collect
df.write.bucketBy(8, "id").sortBy("id").saveAsTable("table2")

sqlContext.sql("""
  SELECT a.id, b.id
  FROM table1 a
  FULL OUTER JOIN table2 b
  ON a.id = b.id AND a.id='1' AND b.id='1'
""").explain(true)
```

BEFORE: This is doing shuffle + sort over table scan outputs which is not needed as both tables are bucketed and sorted on the same columns and have same number of buckets. This should be a single stage job.

```
SortMergeJoin [id#38, cast(id#38 as double), 1.0], [id#39, 1.0, cast(id#39 as double)], FullOuter
:- *Sort [id#38 ASC NULLS FIRST, cast(id#38 as double) ASC NULLS FIRST, 1.0 ASC NULLS FIRST], false, 0
:  +- Exchange hashpartitioning(id#38, cast(id#38 as double), 1.0, 200)
:     +- *FileScan parquet default.table1[id#38] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table1, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<id:int>
+- *Sort [id#39 ASC NULLS FIRST, 1.0 ASC NULLS FIRST, cast(id#39 as double) ASC NULLS FIRST], false, 0
   +- Exchange hashpartitioning(id#39, 1.0, cast(id#39 as double), 200)
      +- *FileScan parquet default.table2[id#39] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table2, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<id:int>
```

AFTER :

```
SortMergeJoin [id#32], [id#33], FullOuter, ((cast(id#32 as double) = 1.0) && (cast(id#33 as double) = 1.0))
:- *FileScan parquet default.table1[id#32] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table1, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<id:int>
+- *FileScan parquet default.table2[id#33] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table2, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<id:int>
```

## How was this patch tested?

- Added a new test case for this scenario : `SPARK-17698 Join predicates should not contain filter clauses`
- Ran all the tests in `BucketedReadSuite`

Author: Tejas Patil <tejasp@fb.com>

Closes #15272 from tejasapatil/SPARK-17698_join_predicate_filter_clause.
2016-10-20 09:50:55 -07:00
Dilip Biswal e895bc2548 [SPARK-17860][SQL] SHOW COLUMN's database conflict check should respect case sensitivity configuration
## What changes were proposed in this pull request?
SHOW COLUMNS command validates the user supplied database
name with database name from qualified table name name to make
sure both of them are consistent. This comparison should respect
case sensitivity.

## How was this patch tested?
Added tests in DDLSuite and existing tests were moved to use new sql based test infrastructure.

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

Closes #15423 from dilipbiswal/dkb_show_column_fix.
2016-10-20 19:39:25 +08:00
Dongjoon Hyun 986a3b8b5b
[SPARK-17796][SQL] Support wildcard character in filename for LOAD DATA LOCAL INPATH
## What changes were proposed in this pull request?

Currently, Spark 2.0 raises an `input path does not exist` AnalysisException if the file name contains '*'. It is misleading since it occurs when there exist some matched files. Also, it was a supported feature in Spark 1.6.2. This PR aims to support wildcard characters in filename for `LOAD DATA LOCAL INPATH` SQL command like Spark 1.6.2.

**Reported Error Scenario**
```scala
scala> sql("CREATE TABLE t(a string)")
res0: org.apache.spark.sql.DataFrame = []

scala> sql("LOAD DATA LOCAL INPATH '/tmp/x*' INTO TABLE t")
org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: /tmp/x*;
```

## How was this patch tested?

Pass the Jenkins test with a new test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15376 from dongjoon-hyun/SPARK-17796.
2016-10-20 09:53:12 +01:00
Eric Liang 5f20ae0394 [SPARK-17980][SQL] Fix refreshByPath for converted Hive tables
## What changes were proposed in this pull request?

There was a bug introduced in https://github.com/apache/spark/pull/14690 which broke refreshByPath with converted hive tables (though, it turns out it was very difficult to refresh converted hive tables anyways, since you had to specify the exact path of one of the partitions).

This changes refreshByPath to invalidate by prefix instead of exact match, and fixes the issue.

cc sameeragarwal for refreshByPath changes
mallman

## How was this patch tested?

Extended unit test.

Author: Eric Liang <ekl@databricks.com>

Closes #15521 from ericl/fix-caching.
2016-10-19 10:20:12 +08:00
Eric Liang 4ef39c2f44 [SPARK-17974] try 2) Refactor FileCatalog classes to simplify the inheritance tree
## What changes were proposed in this pull request?

This renames `BasicFileCatalog => FileCatalog`, combines  `SessionFileCatalog` with `PartitioningAwareFileCatalog`, and removes the old `FileCatalog` trait.

In summary,
```
MetadataLogFileCatalog extends PartitioningAwareFileCatalog
ListingFileCatalog extends PartitioningAwareFileCatalog
PartitioningAwareFileCatalog extends FileCatalog
TableFileCatalog extends FileCatalog
```

(note that this is a re-submission of https://github.com/apache/spark/pull/15518 which got reverted)

## How was this patch tested?

Existing tests

Author: Eric Liang <ekl@databricks.com>

Closes #15533 from ericl/fix-scalastyle-revert.
2016-10-18 13:33:46 -07:00
Wenchen Fan e59df62e62 [SPARK-17899][SQL][FOLLOW-UP] debug mode should work for corrupted table
## What changes were proposed in this pull request?

Debug mode should work for corrupted table, so that we can really debug

## How was this patch tested?

new test in `MetastoreDataSourcesSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15528 from cloud-fan/debug.
2016-10-18 11:03:10 -07:00
Reynold Xin 1c5a7d7f64 Revert "[SPARK-17974] Refactor FileCatalog classes to simplify the inheritance tree"
This reverts commit 8daa1a29b6.
2016-10-17 21:26:28 -07:00
Eric Liang 8daa1a29b6 [SPARK-17974] Refactor FileCatalog classes to simplify the inheritance tree
## What changes were proposed in this pull request?

This renames `BasicFileCatalog => FileCatalog`, combines  `SessionFileCatalog` with `PartitioningAwareFileCatalog`, and removes the old `FileCatalog` trait.

In summary,
```
MetadataLogFileCatalog extends PartitioningAwareFileCatalog
ListingFileCatalog extends PartitioningAwareFileCatalog
PartitioningAwareFileCatalog extends FileCatalog
TableFileCatalog extends FileCatalog
```

cc cloud-fan mallman

## How was this patch tested?

Existing tests

Author: Eric Liang <ekl@databricks.com>

Closes #15518 from ericl/refactor-session-file-catalog.
2016-10-17 21:01:22 -07:00
Dilip Biswal 813ab5e025 [SPARK-17620][SQL] Determine Serde by hive.default.fileformat when Creating Hive Serde Tables
## What changes were proposed in this pull request?
Reopens the closed PR https://github.com/apache/spark/pull/15190
(Please refer to the above link for review comments on the PR)

Make sure the hive.default.fileformat is used to when creating the storage format metadata.

Output
``` SQL
scala> spark.sql("SET hive.default.fileformat=orc")
res1: org.apache.spark.sql.DataFrame = [key: string, value: string]

scala> spark.sql("CREATE TABLE tmp_default(id INT)")
res2: org.apache.spark.sql.DataFrame = []
```
Before
```SQL
scala> spark.sql("DESC FORMATTED tmp_default").collect.foreach(println)
..
[# Storage Information,,]
[SerDe Library:,org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe,]
[InputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcInputFormat,]
[OutputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat,]
[Compressed:,No,]
[Storage Desc Parameters:,,]
[  serialization.format,1,]
```
After
```SQL
scala> spark.sql("DESC FORMATTED tmp_default").collect.foreach(println)
..
[# Storage Information,,]
[SerDe Library:,org.apache.hadoop.hive.ql.io.orc.OrcSerde,]
[InputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcInputFormat,]
[OutputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat,]
[Compressed:,No,]
[Storage Desc Parameters:,,]
[  serialization.format,1,]

```
## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
Added new tests to HiveDDLCommandSuite, SQLQuerySuite

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

Closes #15495 from dilipbiswal/orc2.
2016-10-17 20:46:30 -07:00
Michael Allman 6ce1b675ee [SPARK-16980][SQL] Load only catalog table partition metadata required to answer a query
(This PR addresses https://issues.apache.org/jira/browse/SPARK-16980.)

## What changes were proposed in this pull request?

In a new Spark session, when a partitioned Hive table is converted to use Spark's `HadoopFsRelation` in `HiveMetastoreCatalog`, metadata for every partition of that table are retrieved from the metastore and loaded into driver memory. In addition, every partition's metadata files are read from the filesystem to perform schema inference.

If a user queries such a table with predicates which prune that table's partitions, we would like to be able to answer that query without consulting partition metadata which are not involved in the query. When querying a table with a large number of partitions for some data from a small number of partitions (maybe even a single partition), the current conversion strategy is highly inefficient. I suspect this scenario is not uncommon in the wild.

In addition to being inefficient in running time, the current strategy is inefficient in its use of driver memory. When the sum of the number of partitions of all tables loaded in a driver reaches a certain level (somewhere in the tens of thousands), their cached data exhaust all driver heap memory in the default configuration. I suspect this scenario is less common (in that not too many deployments work with tables with tens of thousands of partitions), however this does illustrate how large the memory footprint of this metadata can be. With tables with hundreds or thousands of partitions, I would expect the `HiveMetastoreCatalog` table cache to represent a significant portion of the driver's heap space.

This PR proposes an alternative approach. Basically, it makes four changes:

1. It adds a new method, `listPartitionsByFilter` to the Catalyst `ExternalCatalog` trait which returns the partition metadata for a given sequence of partition pruning predicates.
1. It refactors the `FileCatalog` type hierarchy to include a new `TableFileCatalog` to efficiently return files only for partitions matching a sequence of partition pruning predicates.
1. It removes partition loading and caching from `HiveMetastoreCatalog`.
1. It adds a new Catalyst optimizer rule, `PruneFileSourcePartitions`, which applies a plan's partition-pruning predicates to prune out unnecessary partition files from a `HadoopFsRelation`'s underlying file catalog.

The net effect is that when a query over a partitioned Hive table is planned, the analyzer retrieves the table metadata from `HiveMetastoreCatalog`. As part of this operation, the `HiveMetastoreCatalog` builds a `HadoopFsRelation` with a `TableFileCatalog`. It does not load any partition metadata or scan any files. The optimizer prunes-away unnecessary table partitions by sending the partition-pruning predicates to the relation's `TableFileCatalog `. The `TableFileCatalog` in turn calls the `listPartitionsByFilter` method on its external catalog. This queries the Hive metastore, passing along those filters.

As a bonus, performing partition pruning during optimization leads to a more accurate relation size estimate. This, along with c481bdf, can lead to automatic, safe application of the broadcast optimization in a join where it might previously have been omitted.

## Open Issues

1. This PR omits partition metadata caching. I can add this once the overall strategy for the cold path is established, perhaps in a future PR.
1. This PR removes and omits partitioned Hive table schema reconciliation. As a result, it fails to find Parquet schema columns with upper case letters because of the Hive metastore's case-insensitivity. This issue may be fixed by #14750, but that PR appears to have stalled. ericl has contributed to this PR a workaround for Parquet wherein schema reconciliation occurs at query execution time instead of planning. Whether ORC requires a similar patch is an open issue.
1. This PR omits an implementation of `listPartitionsByFilter` for the `InMemoryCatalog`.
1. This PR breaks parquet log output redirection during query execution. I can work around this by running `Class.forName("org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat$")` first thing in a Spark shell session, but I haven't figured out how to fix this properly.

## How was this patch tested?

The current Spark unit tests were run, and some ad-hoc tests were performed to validate that only the necessary partition metadata is loaded.

Author: Michael Allman <michael@videoamp.com>
Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekhliang@gmail.com>

Closes #14690 from mallman/spark-16980-lazy_partition_fetching.
2016-10-14 18:26:18 -07:00
Yin Huai 522dd0d0e5 Revert "[SPARK-17620][SQL] Determine Serde by hive.default.fileformat when Creating Hive Serde Tables"
This reverts commit 7ab86244e3.
2016-10-14 14:09:35 -07:00
Dilip Biswal 7ab86244e3 [SPARK-17620][SQL] Determine Serde by hive.default.fileformat when Creating Hive Serde Tables
## What changes were proposed in this pull request?
Make sure the hive.default.fileformat is used to when creating the storage format metadata.

Output
``` SQL
scala> spark.sql("SET hive.default.fileformat=orc")
res1: org.apache.spark.sql.DataFrame = [key: string, value: string]

scala> spark.sql("CREATE TABLE tmp_default(id INT)")
res2: org.apache.spark.sql.DataFrame = []
```
Before
```SQL
scala> spark.sql("DESC FORMATTED tmp_default").collect.foreach(println)
..
[# Storage Information,,]
[SerDe Library:,org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe,]
[InputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcInputFormat,]
[OutputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat,]
[Compressed:,No,]
[Storage Desc Parameters:,,]
[  serialization.format,1,]
```
After
```SQL
scala> spark.sql("DESC FORMATTED tmp_default").collect.foreach(println)
..
[# Storage Information,,]
[SerDe Library:,org.apache.hadoop.hive.ql.io.orc.OrcSerde,]
[InputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcInputFormat,]
[OutputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat,]
[Compressed:,No,]
[Storage Desc Parameters:,,]
[  serialization.format,1,]

```

## How was this patch tested?
Added new tests to HiveDDLCommandSuite

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

Closes #15190 from dilipbiswal/orc.
2016-10-14 13:22:59 -07:00
wangzhenhua 7486442fe0 [SPARK-17073][SQL][FOLLOWUP] generate column-level statistics
## What changes were proposed in this pull request?
This pr adds some test cases for statistics: case sensitive column names, non ascii column names, refresh table, and also improves some documentation.

## How was this patch tested?
add test cases

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #15360 from wzhfy/colStats2.
2016-10-14 21:18:49 +08:00
Wenchen Fan 2fb12b0a33 [SPARK-17903][SQL] MetastoreRelation should talk to external catalog instead of hive client
## What changes were proposed in this pull request?

`HiveExternalCatalog` should be the only interface to talk to the hive metastore. In `MetastoreRelation` we can just use `ExternalCatalog` instead of `HiveClient` to interact with hive metastore,  and add missing API in `ExternalCatalog`.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15460 from cloud-fan/relation.
2016-10-14 15:53:50 +08:00
Pete Robbins 84f149e414 [SPARK-17827][SQL] maxColLength type should be Int for String and Binary
## What changes were proposed in this pull request?
correct the expected type from Length function to be Int

## How was this patch tested?
Test runs on little endian and big endian platforms

Author: Pete Robbins <robbinspg@gmail.com>

Closes #15464 from robbinspg/SPARK-17827.
2016-10-13 11:26:30 -07:00
gatorsmile 0a8e51a5e4 [SPARK-17657][SQL] Disallow Users to Change Table Type
### What changes were proposed in this pull request?
Hive allows users to change the table type from `Managed` to `External` or from `External` to `Managed` by altering table's property `EXTERNAL`. See the JIRA: https://issues.apache.org/jira/browse/HIVE-1329

So far, Spark SQL does not correctly support it, although users can do it. Many assumptions are broken in the implementation. Thus, this PR is to disallow users to change it.

In addition, we also do not allow users to set the property `EXTERNAL` when creating a table.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15230 from gatorsmile/alterTableSetExternal.
2016-10-13 21:36:39 +08:00
Wenchen Fan db8784feaa [SPARK-17899][SQL] add a debug mode to keep raw table properties in HiveExternalCatalog
## What changes were proposed in this pull request?

Currently `HiveExternalCatalog` will filter out the Spark SQL internal table properties, e.g. `spark.sql.sources.provider`, `spark.sql.sources.schema`, etc. This is reasonable for external users as they don't want to see these internal properties in `DESC TABLE`.

However, as a Spark developer, sometimes we do wanna see the raw table properties. This PR adds a new internal SQL conf, `spark.sql.debug`, to enable debug mode and keep these raw table properties.

This config can also be used in similar places where we wanna retain debug information in the future.

## How was this patch tested?

new test in MetastoreDataSourcesSuite

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15458 from cloud-fan/debug.
2016-10-13 03:26:29 -04:00
Wenchen Fan b9a147181d [SPARK-17720][SQL] introduce static SQL conf
## What changes were proposed in this pull request?

SQLConf is session-scoped and mutable. However, we do have the requirement for a static SQL conf, which is global and immutable, e.g. the `schemaStringThreshold` in `HiveExternalCatalog`, the flag to enable/disable hive support, the global temp view database in https://github.com/apache/spark/pull/14897.

Actually we've already implemented static SQL conf implicitly via `SparkConf`, this PR just make it explicit and expose it to users, so that they can see the config value via SQL command or `SparkSession.conf`, and forbid users to set/unset static SQL conf.

## How was this patch tested?

new tests in SQLConfSuite

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15295 from cloud-fan/global-conf.
2016-10-11 20:27:08 -07:00
Wenchen Fan 23ddff4b2b [SPARK-17338][SQL] add global temp view
## What changes were proposed in this pull request?

Global temporary view is a cross-session temporary view, which means it's shared among all sessions. Its lifetime is the lifetime of the Spark application, i.e. it will be automatically dropped when the application terminates. It's tied to a system preserved database `global_temp`(configurable via SparkConf), and we must use the qualified name to refer a global temp view, e.g. SELECT * FROM global_temp.view1.

changes for `SessionCatalog`:

1. add a new field `gloabalTempViews: GlobalTempViewManager`, to access the shared global temp views, and the global temp db name.
2. `createDatabase` will fail if users wanna create `global_temp`, which is system preserved.
3. `setCurrentDatabase` will fail if users wanna set `global_temp`, which is system preserved.
4. add `createGlobalTempView`, which is used in `CreateViewCommand` to create global temp views.
5. add `dropGlobalTempView`, which is used in `CatalogImpl` to drop global temp view.
6. add `alterTempViewDefinition`, which is used in `AlterViewAsCommand` to update the view definition for local/global temp views.
7. `renameTable`/`dropTable`/`isTemporaryTable`/`lookupRelation`/`getTempViewOrPermanentTableMetadata`/`refreshTable` will handle global temp views.

changes for SQL commands:

1. `CreateViewCommand`/`AlterViewAsCommand` is updated to support global temp views
2. `ShowTablesCommand` outputs a new column `database`, which is used to distinguish global and local temp views.
3. other commands can also handle global temp views if they call `SessionCatalog` APIs which accepts global temp views, e.g. `DropTableCommand`, `AlterTableRenameCommand`, `ShowColumnsCommand`, etc.

changes for other public API

1. add a new method `dropGlobalTempView` in `Catalog`
2. `Catalog.findTable` can find global temp view
3. add a new method `createGlobalTempView` in `Dataset`

## How was this patch tested?

new tests in `SQLViewSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14897 from cloud-fan/global-temp-view.
2016-10-10 15:48:57 +08:00
jiangxingbo 16590030c1 [SPARK-17741][SQL] Grammar to parse top level and nested data fields separately
## What changes were proposed in this pull request?

Currently we use the same rule to parse top level and nested data fields. For example:
```
create table tbl_x(
  id bigint,
  nested struct<col1:string,col2:string>
)
```
Shows both syntaxes. In this PR we split this rule in a top-level and nested rule.

Before this PR,
```
sql("CREATE TABLE my_tab(column1: INT)")
```
works fine.
After this PR, it will throw a `ParseException`:
```
scala> sql("CREATE TABLE my_tab(column1: INT)")
org.apache.spark.sql.catalyst.parser.ParseException:
no viable alternative at input 'CREATE TABLE my_tab(column1:'(line 1, pos 27)
```

## How was this patch tested?
Add new testcases in `SparkSqlParserSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15346 from jiangxb1987/cdt.
2016-10-09 22:00:54 -07:00
Weiqing Yang 8a6bbe095b
[MINOR][SQL] Use resource path for test_script.sh
## What changes were proposed in this pull request?
This PR modified the test case `test("script")` to use resource path for `test_script.sh`. Make the test case portable (even in IntelliJ).

## How was this patch tested?
Passed the test case.
Before:
Run `test("script")` in IntelliJ:
```
Caused by: org.apache.spark.SparkException: Subprocess exited with status 127. Error: bash: src/test/resources/test_script.sh: No such file or directory
```
After:
Test passed.

Author: Weiqing Yang <yangweiqing001@gmail.com>

Closes #15246 from weiqingy/hivetest.
2016-10-08 12:12:35 +01:00
Herman van Hovell 97594c29b7 [SPARK-17761][SQL] Remove MutableRow
## What changes were proposed in this pull request?
In practice we cannot guarantee that an `InternalRow` is immutable. This makes the `MutableRow` almost redundant. This PR folds `MutableRow` into `InternalRow`.

The code below illustrates the immutability issue with InternalRow:
```scala
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.GenericMutableRow
val struct = new GenericMutableRow(1)
val row = InternalRow(struct, 1)
println(row)
scala> [[null], 1]
struct.setInt(0, 42)
println(row)
scala> [[42], 1]
```

This might be somewhat controversial, so feedback is appreciated.

## How was this patch tested?
Existing tests.

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

Closes #15333 from hvanhovell/SPARK-17761.
2016-10-07 14:03:45 -07:00
Dongjoon Hyun 92b7e57280 [SPARK-17750][SQL] Fix CREATE VIEW with INTERVAL arithmetic.
## What changes were proposed in this pull request?

Currently, Spark raises `RuntimeException` when creating a view with timestamp with INTERVAL arithmetic like the following. The root cause is the arithmetic expression, `TimeAdd`, was transformed into `timeadd` function as a VIEW definition. This PR fixes the SQL definition of `TimeAdd` and `TimeSub` expressions.

```scala
scala> sql("CREATE TABLE dates (ts TIMESTAMP)")

scala> sql("CREATE VIEW view1 AS SELECT ts + INTERVAL 1 DAY FROM dates")
java.lang.RuntimeException: Failed to analyze the canonicalized SQL: ...
```

## How was this patch tested?

Pass Jenkins with a new testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15318 from dongjoon-hyun/SPARK-17750.
2016-10-06 09:42:30 -07:00
Herman van Hovell 89516c1c4a [SPARK-17258][SQL] Parse scientific decimal literals as decimals
## What changes were proposed in this pull request?
Currently Spark SQL parses regular decimal literals (e.g. `10.00`) as decimals and scientific decimal literals (e.g. `10.0e10`) as doubles. The difference between the two confuses most users. This PR unifies the parsing behavior and also parses scientific decimal literals as decimals.

This implications in tests are limited to a single Hive compatibility test.

## How was this patch tested?
Updated tests in `ExpressionParserSuite` and `SQLQueryTestSuite`.

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

Closes #14828 from hvanhovell/SPARK-17258.
2016-10-04 23:48:26 -07:00
Ergin Seyfe d2dc8c4a16 [SPARK-17773] Input/Output] Add VoidObjectInspector
## What changes were proposed in this pull request?
Added VoidObjectInspector to the list of PrimitiveObjectInspectors

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
Executing following query was failing.
select SOME_UDAF*(a.arr)
from (
select Array(null) as arr from dim_one_row
) a

After the fix, I am getting the correct output:
res0: Array[org.apache.spark.sql.Row] = Array([null])

Author: Ergin Seyfe <eseyfe@fb.com>

Closes #15337 from seyfe/add_void_object_inspector.
2016-10-03 23:28:39 -07:00
Zhenhua Wang 7bf9212764 [SPARK-17073][SQL] generate column-level statistics
## What changes were proposed in this pull request?

Generate basic column statistics for all the atomic types:
- numeric types: max, min, num of nulls, ndv (number of distinct values)
- date/timestamp types: they are also represented as numbers internally, so they have the same stats as above.
- string: avg length, max length, num of nulls, ndv
- binary: avg length, max length, num of nulls
- boolean: num of nulls, num of trues, num of falsies

Also support storing and loading these statistics.

One thing to notice:
We support analyzing columns independently, e.g.:
sql1: `ANALYZE TABLE src COMPUTE STATISTICS FOR COLUMNS key;`
sql2: `ANALYZE TABLE src COMPUTE STATISTICS FOR COLUMNS value;`
when running sql2 to collect column stats for `value`, we don’t remove stats of columns `key` which are analyzed in sql1 and not in sql2. As a result, **users need to guarantee consistency** between sql1 and sql2. If the table has been changed before sql2, users should re-analyze column `key` when they want to analyze column `value`:
`ANALYZE TABLE src COMPUTE STATISTICS FOR COLUMNS key, value;`

## How was this patch tested?

add unit tests

Author: Zhenhua Wang <wzh_zju@163.com>

Closes #15090 from wzhfy/colStats.
2016-10-03 10:12:02 -07:00
Sital Kedia f8d7fade4b [SPARK-17509][SQL] When wrapping catalyst datatype to Hive data type avoid…
## What changes were proposed in this pull request?

When wrapping catalyst datatypes to Hive data type, wrap function was doing an expensive pattern matching which was consuming around 11% of cpu time. Avoid the pattern matching by returning the wrapper only once and reuse it.

## How was this patch tested?

Tested by running the job on cluster and saw around 8% cpu improvements.

Author: Sital Kedia <skedia@fb.com>

Closes #15064 from sitalkedia/skedia/hive_wrapper.
2016-10-02 15:47:36 -07:00
Dongjoon Hyun 4ecc648ad7 [SPARK-17612][SQL] Support DESCRIBE table PARTITION SQL syntax
## What changes were proposed in this pull request?

This PR implements `DESCRIBE table PARTITION` SQL Syntax again. It was supported until Spark 1.6.2, but was dropped since 2.0.0.

**Spark 1.6.2**
```scala
scala> sql("CREATE TABLE partitioned_table (a STRING, b INT) PARTITIONED BY (c STRING, d STRING)")
res1: org.apache.spark.sql.DataFrame = [result: string]

scala> sql("ALTER TABLE partitioned_table ADD PARTITION (c='Us', d=1)")
res2: org.apache.spark.sql.DataFrame = [result: string]

scala> sql("DESC partitioned_table PARTITION (c='Us', d=1)").show(false)
+----------------------------------------------------------------+
|result                                                          |
+----------------------------------------------------------------+
|a                      string                                   |
|b                      int                                      |
|c                      string                                   |
|d                      string                                   |
|                                                                |
|# Partition Information                                         |
|# col_name             data_type               comment          |
|                                                                |
|c                      string                                   |
|d                      string                                   |
+----------------------------------------------------------------+
```

**Spark 2.0**
- **Before**
```scala
scala> sql("CREATE TABLE partitioned_table (a STRING, b INT) PARTITIONED BY (c STRING, d STRING)")
res0: org.apache.spark.sql.DataFrame = []

scala> sql("ALTER TABLE partitioned_table ADD PARTITION (c='Us', d=1)")
res1: org.apache.spark.sql.DataFrame = []

scala> sql("DESC partitioned_table PARTITION (c='Us', d=1)").show(false)
org.apache.spark.sql.catalyst.parser.ParseException:
Unsupported SQL statement
```

- **After**
```scala
scala> sql("CREATE TABLE partitioned_table (a STRING, b INT) PARTITIONED BY (c STRING, d STRING)")
res0: org.apache.spark.sql.DataFrame = []

scala> sql("ALTER TABLE partitioned_table ADD PARTITION (c='Us', d=1)")
res1: org.apache.spark.sql.DataFrame = []

scala> sql("DESC partitioned_table PARTITION (c='Us', d=1)").show(false)
+-----------------------+---------+-------+
|col_name               |data_type|comment|
+-----------------------+---------+-------+
|a                      |string   |null   |
|b                      |int      |null   |
|c                      |string   |null   |
|d                      |string   |null   |
|# Partition Information|         |       |
|# col_name             |data_type|comment|
|c                      |string   |null   |
|d                      |string   |null   |
+-----------------------+---------+-------+

scala> sql("DESC EXTENDED partitioned_table PARTITION (c='Us', d=1)").show(100,false)
+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+---------+-------+
|col_name                                                                                                                                                                                                                                                                                                                                                                                                                                                                           |data_type|comment|
+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+---------+-------+
|a                                                                                                                                                                                                                                                                                                                                                                                                                                                                                  |string   |null   |
|b                                                                                                                                                                                                                                                                                                                                                                                                                                                                                  |int      |null   |
|c                                                                                                                                                                                                                                                                                                                                                                                                                                                                                  |string   |null   |
|d                                                                                                                                                                                                                                                                                                                                                                                                                                                                                  |string   |null   |
|# Partition Information                                                                                                                                                                                                                                                                                                                                                                                                                                                            |         |       |
|# col_name                                                                                                                                                                                                                                                                                                                                                                                                                                                                         |data_type|comment|
|c                                                                                                                                                                                                                                                                                                                                                                                                                                                                                  |string   |null   |
|d                                                                                                                                                                                                                                                                                                                                                                                                                                                                                  |string   |null   |
|                                                                                                                                                                                                                                                                                                                                                                                                                                                                                   |         |       |
|Detailed Partition Information CatalogPartition(
        Partition Values: [Us, 1]
        Storage(Location: file:/Users/dhyun/SPARK-17612-DESC-PARTITION/spark-warehouse/partitioned_table/c=Us/d=1, InputFormat: org.apache.hadoop.mapred.TextInputFormat, OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, Serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, Properties: [serialization.format=1])
        Partition Parameters:{transient_lastDdlTime=1475001066})|         |       |
+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+---------+-------+

scala> sql("DESC FORMATTED partitioned_table PARTITION (c='Us', d=1)").show(100,false)
+--------------------------------+---------------------------------------------------------------------------------------+-------+
|col_name                        |data_type                                                                              |comment|
+--------------------------------+---------------------------------------------------------------------------------------+-------+
|a                               |string                                                                                 |null   |
|b                               |int                                                                                    |null   |
|c                               |string                                                                                 |null   |
|d                               |string                                                                                 |null   |
|# Partition Information         |                                                                                       |       |
|# col_name                      |data_type                                                                              |comment|
|c                               |string                                                                                 |null   |
|d                               |string                                                                                 |null   |
|                                |                                                                                       |       |
|# Detailed Partition Information|                                                                                       |       |
|Partition Value:                |[Us, 1]                                                                                |       |
|Database:                       |default                                                                                |       |
|Table:                          |partitioned_table                                                                      |       |
|Location:                       |file:/Users/dhyun/SPARK-17612-DESC-PARTITION/spark-warehouse/partitioned_table/c=Us/d=1|       |
|Partition Parameters:           |                                                                                       |       |
|  transient_lastDdlTime         |1475001066                                                                             |       |
|                                |                                                                                       |       |
|# 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                             |       |
|Compressed:                     |No                                                                                     |       |
|Storage Desc Parameters:        |                                                                                       |       |
|  serialization.format          |1                                                                                      |       |
+--------------------------------+---------------------------------------------------------------------------------------+-------+
```

## How was this patch tested?

Pass the Jenkins tests with a new testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15168 from dongjoon-hyun/SPARK-17612.
2016-09-29 15:30:18 -07:00
Josh Rosen b03b4adf6d [SPARK-17666] Ensure that RecordReaders are closed by data source file scans
## What changes were proposed in this pull request?

This patch addresses a potential cause of resource leaks in data source file scans. As reported in [SPARK-17666](https://issues.apache.org/jira/browse/SPARK-17666), tasks which do not fully-consume their input may cause file handles / network connections (e.g. S3 connections) to be leaked. Spark's `NewHadoopRDD` uses a TaskContext callback to [close its record readers](https://github.com/apache/spark/blame/master/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala#L208), but the new data source file scans will only close record readers once their iterators are fully-consumed.

This patch modifies `RecordReaderIterator` and `HadoopFileLinesReader` to add `close()` methods and modifies all six implementations of `FileFormat.buildReader()` to register TaskContext task completion callbacks to guarantee that cleanup is eventually performed.

## How was this patch tested?

Tested manually for now.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #15245 from JoshRosen/SPARK-17666-close-recordreader.
2016-09-27 17:52:57 -07:00
gatorsmile 2ab24a7bf6 [SPARK-17660][SQL] DESC FORMATTED for VIEW Lacks View Definition
### What changes were proposed in this pull request?
Before this PR, `DESC FORMATTED` does not have a section for the view definition. We should add it for permanent views, like what Hive does.

```
+----------------------------+-------------------------------------------------------------------------------------------------------------------------------------+-------+
|col_name                    |data_type                                                                                                                            |comment|
+----------------------------+-------------------------------------------------------------------------------------------------------------------------------------+-------+
|a                           |int                                                                                                                                  |null   |
|                            |                                                                                                                                     |       |
|# Detailed Table Information|                                                                                                                                     |       |
|Database:                   |default                                                                                                                              |       |
|Owner:                      |xiaoli                                                                                                                               |       |
|Create Time:                |Sat Sep 24 21:46:19 PDT 2016                                                                                                         |       |
|Last Access Time:           |Wed Dec 31 16:00:00 PST 1969                                                                                                         |       |
|Location:                   |                                                                                                                                     |       |
|Table Type:                 |VIEW                                                                                                                                 |       |
|Table Parameters:           |                                                                                                                                     |       |
|  transient_lastDdlTime     |1474778779                                                                                                                           |       |
|                            |                                                                                                                                     |       |
|# 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                                                                                                                                   |       |
|Storage Desc Parameters:    |                                                                                                                                     |       |
|  serialization.format      |1                                                                                                                                    |       |
|                            |                                                                                                                                     |       |
|# View Information          |                                                                                                                                     |       |
|View Original Text:         |SELECT * FROM tbl                                                                                                                    |       |
|View Expanded Text:         |SELECT `gen_attr_0` AS `a` FROM (SELECT `gen_attr_0` FROM (SELECT `a` AS `gen_attr_0` FROM `default`.`tbl`) AS gen_subquery_0) AS tbl|       |
+----------------------------+-------------------------------------------------------------------------------------------------------------------------------------+-------+
```

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15234 from gatorsmile/descFormattedView.
2016-09-27 10:52:26 -07:00
hyukjinkwon 2cac3b2d4a [SPARK-16516][SQL] Support for pushing down filters for decimal and timestamp types in ORC
## 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 timestamp type and decimal type.

In more details, the types listed in [`SearchArgumentImpl.boxLiteral()`](https://github.com/apache/hive/blob/branch-1.2/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java#L1068-L1093) can be used as a filter value.

FYI, inital `case` caluse for supported types was introduced in 65d71bd9fb and this was not changed overtime. At that time, Hive version was, 0.13 which supports only some types for filter-push down (See [SearchArgumentImpl.java#L945-L965](https://github.com/apache/hive/blob/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java#L945-L965) at 0.13).

However, the version was upgraded into 1.2.x and now it supports more types (See [SearchArgumentImpl.java#L1068-L1093](https://github.com/apache/hive/blob/branch-1.2/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java#L1068-L1093) at 1.2.0)

## How was this patch tested?

Unit tests in `OrcFilterSuite` and `OrcQuerySuite`

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #14172 from HyukjinKwon/SPARK-16516.
2016-09-28 00:50:12 +08:00
Yadong Qi cb324f6115 [SPARK-17425][SQL] Override sameResult in HiveTableScanExec to make ReuseExchange work in text format table
## What changes were proposed in this pull request?
The PR will override the `sameResult` in `HiveTableScanExec` to make `ReuseExchange` work in text format table.

## How was this patch tested?
# SQL
```sql
SELECT * FROM src t1
JOIN src t2 ON t1.key = t2.key
JOIN src t3 ON t1.key = t3.key;
```

# Before
```
== Physical Plan ==
*BroadcastHashJoin [key#30], [key#34], Inner, BuildRight
:- *BroadcastHashJoin [key#30], [key#32], Inner, BuildRight
:  :- *Filter isnotnull(key#30)
:  :  +- HiveTableScan [key#30, value#31], MetastoreRelation default, src
:  +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)))
:     +- *Filter isnotnull(key#32)
:        +- HiveTableScan [key#32, value#33], MetastoreRelation default, src
+- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)))
   +- *Filter isnotnull(key#34)
      +- HiveTableScan [key#34, value#35], MetastoreRelation default, src
```

# After
```
== Physical Plan ==
*BroadcastHashJoin [key#2], [key#6], Inner, BuildRight
:- *BroadcastHashJoin [key#2], [key#4], Inner, BuildRight
:  :- *Filter isnotnull(key#2)
:  :  +- HiveTableScan [key#2, value#3], MetastoreRelation default, src
:  +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)))
:     +- *Filter isnotnull(key#4)
:        +- HiveTableScan [key#4, value#5], MetastoreRelation default, src
+- ReusedExchange [key#6, value#7], BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)))
```

cc: davies cloud-fan

Author: Yadong Qi <qiyadong2010@gmail.com>

Closes #14988 from watermen/SPARK-17425.
2016-09-22 13:04:42 +08:00
Wenchen Fan b50b34f561 [SPARK-17609][SQL] SessionCatalog.tableExists should not check temp view
## What changes were proposed in this pull request?

After #15054 , there is no place in Spark SQL that need `SessionCatalog.tableExists` to check temp views, so this PR makes `SessionCatalog.tableExists` only check permanent table/view and removes some hacks.

This PR also improves the `getTempViewOrPermanentTableMetadata` that is introduced in  #15054 , to make the code simpler.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15160 from cloud-fan/exists.
2016-09-22 12:52:09 +08:00
Wenchen Fan eb004c6620 [SPARK-17051][SQL] we should use hadoopConf in InsertIntoHiveTable
## What changes were proposed in this pull request?

Hive confs in hive-site.xml will be loaded in `hadoopConf`, so we should use `hadoopConf` in `InsertIntoHiveTable` instead of `SessionState.conf`

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14634 from cloud-fan/bug.
2016-09-20 09:53:28 -07:00
gatorsmile d5ec5dbb0d [SPARK-17502][SQL] Fix Multiple Bugs in DDL Statements on Temporary Views
### What changes were proposed in this pull request?
- When the permanent tables/views do not exist but the temporary view exists, the expected error should be `NoSuchTableException` for partition-related ALTER TABLE commands. However, it always reports a confusing error message. For example,
```
Partition spec is invalid. The spec (a, b) must match the partition spec () defined in table '`testview`';
```
- When the permanent tables/views do not exist but the temporary view exists, the expected error should be `NoSuchTableException` for `ALTER TABLE ... UNSET TBLPROPERTIES`. However, it reports a missing table property. For example,
```
Attempted to unset non-existent property 'p' in table '`testView`';
```
- When `ANALYZE TABLE` is called on a view or a temporary view, we should issue an error message. However, it reports a strange error:
```
ANALYZE TABLE is not supported for Project
```

- When inserting into a temporary view that is generated from `Range`, we will get the following error message:
```
assertion failed: No plan for 'InsertIntoTable Range (0, 10, step=1, splits=Some(1)), false, false
+- Project [1 AS 1#20]
   +- OneRowRelation$
```

This PR is to fix the above four issues.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15054 from gatorsmile/tempViewDDL.
2016-09-20 20:11:48 +08:00
Wenchen Fan 3fe630d314 [SPARK-17541][SQL] fix some DDL bugs about table management when same-name temp view exists
## What changes were proposed in this pull request?

In `SessionCatalog`, we have several operations(`tableExists`, `dropTable`, `loopupRelation`, etc) that handle both temp views and metastore tables/views. This brings some bugs to DDL commands that want to handle temp view only or metastore table/view only. These bugs are:

1. `CREATE TABLE USING` will fail if a same-name temp view exists
2. `Catalog.dropTempView`will un-cache and drop metastore table if a same-name table exists
3. `saveAsTable` will fail or have unexpected behaviour if a same-name temp view exists.

These bug fixes are pulled out from https://github.com/apache/spark/pull/14962 and targets both master and 2.0 branch

## How was this patch tested?

new regression tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15099 from cloud-fan/fix-view.
2016-09-18 21:15:35 +08:00
gatorsmile 3a3c9ffbd2 [SPARK-17518][SQL] Block Users to Specify the Internal Data Source Provider Hive
### What changes were proposed in this pull request?
In Spark 2.1, we introduced a new internal provider `hive` for telling Hive serde tables from data source tables. This PR is to block users to specify this in `DataFrameWriter` and SQL APIs.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15073 from gatorsmile/formatHive.
2016-09-18 15:37:15 +08:00
hyukjinkwon 86c2d393a5
[SPARK-17480][SQL][FOLLOWUP] Fix more instances which calls List.length/size which is O(n)
## What changes were proposed in this pull request?

This PR fixes all the instances which was fixed in the previous PR.

To make sure, I manually debugged and also checked the Scala source. `length` in [LinearSeqOptimized.scala#L49-L57](https://github.com/scala/scala/blob/2.11.x/src/library/scala/collection/LinearSeqOptimized.scala#L49-L57) is O(n). Also, `size` calls `length` via [SeqLike.scala#L106](https://github.com/scala/scala/blob/2.11.x/src/library/scala/collection/SeqLike.scala#L106).

For debugging, I have created these as below:

```scala
ArrayBuffer(1, 2, 3)
Array(1, 2, 3)
List(1, 2, 3)
Seq(1, 2, 3)
```

and then called `size` and `length` for each to debug.

## How was this patch tested?

I ran the bash as below on Mac

```bash
find . -name *.scala -type f -exec grep -il "while (.*\\.length)" {} \; | grep "src/main"
find . -name *.scala -type f -exec grep -il "while (.*\\.size)" {} \; | grep "src/main"
```

and then checked each.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15093 from HyukjinKwon/SPARK-17480-followup.
2016-09-17 16:52:30 +01:00
gatorsmile 6a6adb1673 [SPARK-17440][SPARK-17441] Fixed Multiple Bugs in ALTER TABLE
### What changes were proposed in this pull request?
For the following `ALTER TABLE` DDL, we should issue an exception when the target table is a `VIEW`:
```SQL
 ALTER TABLE viewName SET LOCATION '/path/to/your/lovely/heart'

 ALTER TABLE viewName SET SERDE 'whatever'

 ALTER TABLE viewName SET SERDEPROPERTIES ('x' = 'y')

 ALTER TABLE viewName PARTITION (a=1, b=2) SET SERDEPROPERTIES ('x' = 'y')

 ALTER TABLE viewName ADD IF NOT EXISTS PARTITION (a='4', b='8')

 ALTER TABLE viewName DROP IF EXISTS PARTITION (a='2')

 ALTER TABLE viewName RECOVER PARTITIONS

 ALTER TABLE viewName PARTITION (a='1', b='q') RENAME TO PARTITION (a='100', b='p')
```

In addition, `ALTER TABLE RENAME PARTITION` is unable to handle data source tables, just like the other `ALTER PARTITION` commands. We should issue an exception instead.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15004 from gatorsmile/altertable.
2016-09-15 14:43:10 +08:00
Xin Wu 040e46979d [SPARK-10747][SQL] Support NULLS FIRST|LAST clause in ORDER BY
## What changes were proposed in this pull request?
Currently, ORDER BY clause returns nulls value according to sorting order (ASC|DESC), considering null value is always smaller than non-null values.
However, SQL2003 standard support NULLS FIRST or NULLS LAST to allow users to specify whether null values should be returned first or last, regardless of sorting order (ASC|DESC).

This PR is to support this new feature.

## How was this patch tested?
New test cases are added to test NULLS FIRST|LAST for regular select queries and windowing queries.

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

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

Closes #14842 from xwu0226/SPARK-10747.
2016-09-14 21:14:29 +02:00
gatorsmile 52738d4e09 [SPARK-17409][SQL] Do Not Optimize Query in CTAS More Than Once
### What changes were proposed in this pull request?
As explained in https://github.com/apache/spark/pull/14797:
>Some analyzer rules have assumptions on logical plans, optimizer may break these assumption, we should not pass an optimized query plan into QueryExecution (will be analyzed again), otherwise we may some weird bugs.
For example, we have a rule for decimal calculation to promote the precision before binary operations, use PromotePrecision as placeholder to indicate that this rule should not apply twice. But a Optimizer rule will remove this placeholder, that break the assumption, then the rule applied twice, cause wrong result.

We should not optimize the query in CTAS more than once. For example,
```Scala
spark.range(99, 101).createOrReplaceTempView("tab1")
val sqlStmt = "SELECT id, cast(id as long) * cast('1.0' as decimal(38, 18)) as num FROM tab1"
sql(s"CREATE TABLE tab2 USING PARQUET AS $sqlStmt")
checkAnswer(spark.table("tab2"), sql(sqlStmt))
```
Before this PR, the results do not match
```
== Results ==
!== Correct Answer - 2 ==       == Spark Answer - 2 ==
![100,100.000000000000000000]   [100,null]
 [99,99.000000000000000000]     [99,99.000000000000000000]
```
After this PR, the results match.
```
+---+----------------------+
|id |num                   |
+---+----------------------+
|99 |99.000000000000000000 |
|100|100.000000000000000000|
+---+----------------------+
```

In this PR, we do not treat the `query` in CTAS as a child. Thus, the `query` will not be optimized when optimizing CTAS statement. However, we still need to analyze it for normalizing and verifying the CTAS in the Analyzer. Thus, we do it in the analyzer rule `PreprocessDDL`, because so far only this rule needs the analyzed plan of the `query`.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15048 from gatorsmile/ctasOptimized.
2016-09-14 23:10:20 +08:00
Ergin Seyfe 4cea9da2ae [SPARK-17480][SQL] Improve performance by removing or caching List.length which is O(n)
## What changes were proposed in this pull request?
Scala's List.length method is O(N) and it makes the gatherCompressibilityStats function O(N^2). Eliminate the List.length calls by writing it in Scala way.

https://github.com/scala/scala/blob/2.10.x/src/library/scala/collection/LinearSeqOptimized.scala#L36

As suggested. Extended the fix to HiveInspectors and AggregationIterator classes as well.

## How was this patch tested?
Profiled a Spark job and found that CompressibleColumnBuilder is using 39% of the CPU. Out of this 39% CompressibleColumnBuilder->gatherCompressibilityStats is using 23% of it. 6.24% of the CPU is spend on List.length which is called inside gatherCompressibilityStats.

After this change we started to save 6.24% of the CPU.

Author: Ergin Seyfe <eseyfe@fb.com>

Closes #15032 from seyfe/gatherCompressibilityStats.
2016-09-14 09:51:14 +01:00
Burak Yavuz 72edc7e958 [SPARK-17531] Don't initialize Hive Listeners for the Execution Client
## What changes were proposed in this pull request?

If a user provides listeners inside the Hive Conf, the configuration for these listeners are passed to the Hive Execution Client as well. This may cause issues for two reasons:
1. The Execution Client will actually generate garbage
2. The listener class needs to be both in the Spark Classpath and Hive Classpath

This PR empties the listener configurations in `HiveUtils.newTemporaryConfiguration` so that the execution client will not contain the listener confs, but the metadata client will.

## How was this patch tested?

Unit tests

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #15086 from brkyvz/null-listeners.
2016-09-13 15:11:55 -07:00
tone-zhang bf22217377 [SPARK-17330][SPARK UT] Clean up spark-warehouse in UT
## What changes were proposed in this pull request?

Check the database warehouse used in Spark UT, and remove the existing database file before run the UT (SPARK-8368).

## How was this patch tested?

Run Spark UT with the command for several times:
./build/sbt -Pyarn -Phadoop-2.6 -Phive -Phive-thriftserver "test-only *HiveSparkSubmitSuit*"
Without the patch, the test case can be passed only at the first time, and always failed from the second time.
With the patch the test case always can be passed correctly.

Author: tone-zhang <tone.zhang@linaro.org>

Closes #14894 from tone-zhang/issue1.
2016-09-11 10:17:53 +01:00
Tejas Patil 335491704c [SPARK-15453][SQL] FileSourceScanExec to extract outputOrdering information
## What changes were proposed in this pull request?

Jira : https://issues.apache.org/jira/browse/SPARK-15453

Extracting sort ordering information in `FileSourceScanExec` so that planner can make use of it. My motivation to make this change was to get Sort Merge join in par with Hive's Sort-Merge-Bucket join when the source tables are bucketed + sorted.

Query:

```
val df = (0 until 16).map(i => (i % 8, i * 2, i.toString)).toDF("i", "j", "k").coalesce(1)
df.write.bucketBy(8, "j", "k").sortBy("j", "k").saveAsTable("table8")
df.write.bucketBy(8, "j", "k").sortBy("j", "k").saveAsTable("table9")
context.sql("SELECT * FROM table8 a JOIN table9 b ON a.j=b.j AND a.k=b.k").explain(true)
```

Before:

```
== Physical Plan ==
*SortMergeJoin [j#120, k#121], [j#123, k#124], Inner
:- *Sort [j#120 ASC, k#121 ASC], false, 0
:  +- *Project [i#119, j#120, k#121]
:     +- *Filter (isnotnull(k#121) && isnotnull(j#120))
:        +- *FileScan orc default.table8[i#119,j#120,k#121] Batched: false, Format: ORC, InputPaths: file:/Users/tejasp/Desktop/dev/tp-spark/spark-warehouse/table8, PartitionFilters: [], PushedFilters: [IsNotNull(k), IsNotNull(j)], ReadSchema: struct<i:int,j:int,k:string>
+- *Sort [j#123 ASC, k#124 ASC], false, 0
+- *Project [i#122, j#123, k#124]
+- *Filter (isnotnull(k#124) && isnotnull(j#123))
 +- *FileScan orc default.table9[i#122,j#123,k#124] Batched: false, Format: ORC, InputPaths: file:/Users/tejasp/Desktop/dev/tp-spark/spark-warehouse/table9, PartitionFilters: [], PushedFilters: [IsNotNull(k), IsNotNull(j)], ReadSchema: struct<i:int,j:int,k:string>
```

After:  (note that the `Sort` step is no longer there)

```
== Physical Plan ==
*SortMergeJoin [j#49, k#50], [j#52, k#53], Inner
:- *Project [i#48, j#49, k#50]
:  +- *Filter (isnotnull(k#50) && isnotnull(j#49))
:     +- *FileScan orc default.table8[i#48,j#49,k#50] Batched: false, Format: ORC, InputPaths: file:/Users/tejasp/Desktop/dev/tp-spark/spark-warehouse/table8, PartitionFilters: [], PushedFilters: [IsNotNull(k), IsNotNull(j)], ReadSchema: struct<i:int,j:int,k:string>
+- *Project [i#51, j#52, k#53]
   +- *Filter (isnotnull(j#52) && isnotnull(k#53))
      +- *FileScan orc default.table9[i#51,j#52,k#53] Batched: false, Format: ORC, InputPaths: file:/Users/tejasp/Desktop/dev/tp-spark/spark-warehouse/table9, PartitionFilters: [], PushedFilters: [IsNotNull(j), IsNotNull(k)], ReadSchema: struct<i:int,j:int,k:string>
```

## How was this patch tested?

Added a test case in `JoinSuite`. Ran all other tests in `JoinSuite`

Author: Tejas Patil <tejasp@fb.com>

Closes #14864 from tejasapatil/SPARK-15453_smb_optimization.
2016-09-10 09:27:22 +08:00
hyukjinkwon f7d2143705 [SPARK-17354] [SQL] Partitioning by dates/timestamps should work with Parquet vectorized reader
## What changes were proposed in this pull request?

This PR fixes `ColumnVectorUtils.populate` so that Parquet vectorized reader can read partitioned table with dates/timestamps. This works fine with Parquet normal reader.

This is being only called within [VectorizedParquetRecordReader.java#L185](https://github.com/apache/spark/blob/master/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java#L185).

When partition column types are explicitly given to `DateType` or `TimestampType` (rather than inferring the type of partition column), this fails with the exception below:

```
16/09/01 10:30:07 ERROR Executor: Exception in task 0.0 in stage 5.0 (TID 6)
java.lang.ClassCastException: java.lang.Integer cannot be cast to java.sql.Date
	at org.apache.spark.sql.execution.vectorized.ColumnVectorUtils.populate(ColumnVectorUtils.java:89)
	at org.apache.spark.sql.execution.datasources.parquet.VectorizedParquetRecordReader.initBatch(VectorizedParquetRecordReader.java:185)
	at org.apache.spark.sql.execution.datasources.parquet.VectorizedParquetRecordReader.initBatch(VectorizedParquetRecordReader.java:204)
	at org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat$$anonfun$buildReader$1.apply(ParquetFileFormat.scala:362)
...
```

## How was this patch tested?

Unit tests in `SQLQuerySuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #14919 from HyukjinKwon/SPARK-17354.
2016-09-09 14:23:05 -07:00
gatorsmile b230fb92a5 [SPARK-17052][SQL] Remove Duplicate Test Cases auto_join from HiveCompatibilitySuite.scala
### What changes were proposed in this pull request?
The original [JIRA Hive-1642](https://issues.apache.org/jira/browse/HIVE-1642) delivered the test cases `auto_joinXYZ` for verifying the results when the joins are automatically converted to map-join. Basically, most of them are just copied from the corresponding `joinXYZ`.

After comparison between `auto_joinXYZ` and `joinXYZ`, below is a list of duplicate cases:
```
    "auto_join0",
    "auto_join1",
    "auto_join10",
    "auto_join11",
    "auto_join12",
    "auto_join13",
    "auto_join14",
    "auto_join14_hadoop20",
    "auto_join15",
    "auto_join17",
    "auto_join18",
    "auto_join2",
    "auto_join20",
    "auto_join21",
    "auto_join23",
    "auto_join24",
    "auto_join3",
    "auto_join4",
    "auto_join5",
    "auto_join6",
    "auto_join7",
    "auto_join8",
    "auto_join9"
```

We can remove all of them without affecting the test coverage.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14635 from gatorsmile/removeAuto.
2016-09-07 14:03:14 -07:00
Wenchen Fan d6eede9a36 [SPARK-17238][SQL] simplify the logic for converting data source table into hive compatible format
## What changes were proposed in this pull request?

Previously we have 2 conditions to decide whether a data source table is hive-compatible:

1. the data source is file-based and has a corresponding Hive serde
2. have a `path` entry in data source options/storage properties

However, if condition 1 is true, condition 2 must be true too, as we will put the default table path into data source options/storage properties for managed data source tables.

There is also a potential issue: we will set the `locationUri` even for managed table.

This PR removes the condition 2 and only set the `locationUri` for external data source tables.

Note: this is also a first step to unify the `path` of data source tables and `locationUri` of hive serde tables. For hive serde tables, `locationUri` is only set for external table. For data source tables, `path` is always set. We can make them consistent after this PR.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14809 from cloud-fan/minor2.
2016-09-07 09:36:53 +08:00
gatorsmile a40657bfd3 [SPARK-17408][TEST] Flaky test: org.apache.spark.sql.hive.StatisticsSuite
### What changes were proposed in this pull request?
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/64956/testReport/junit/org.apache.spark.sql.hive/StatisticsSuite/test_statistics_of_LogicalRelation_converted_from_MetastoreRelation/
```
org.apache.spark.sql.hive.StatisticsSuite.test statistics of LogicalRelation converted from MetastoreRelation

Failing for the past 1 build (Since Failed#64956 )
Took 1.4 sec.
Error Message

org.scalatest.exceptions.TestFailedException: 6871 did not equal 4236
Stacktrace

sbt.ForkMain$ForkError: org.scalatest.exceptions.TestFailedException: 6871 did not equal 4236
	at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500)
```

This fix does not check the exact value of `sizeInBytes`. Instead, we compare whether it is larger than zero and compare the values between different values.

In addition, we also combine `checkMetastoreRelationStats` and `checkLogicalRelationStats` into the same checking function.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14978 from gatorsmile/spark17408.
2016-09-07 08:13:12 +08:00
Wenchen Fan c0ae6bc6ea [SPARK-17361][SQL] file-based external table without path should not be created
## What changes were proposed in this pull request?

Using the public `Catalog` API, users can create a file-based data source table, without giving the path options. For this case, currently we can create the table successfully, but fail when we read it. Ideally we should fail during creation.

This is because when we create data source table, we resolve the data source relation without validating path: `resolveRelation(checkPathExist = false)`.

Looking back to why we add this trick(`checkPathExist`), it's because when we call `resolveRelation` for managed table, we add the path to data source options but the path is not created yet. So why we add this not-yet-created path to data source options? This PR fix the problem by adding path to options after we call `resolveRelation`. Then we can remove the `checkPathExist` parameter in `DataSource.resolveRelation` and do some related cleanups.

## How was this patch tested?

existing tests and new test in `CatalogSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14921 from cloud-fan/check-path.
2016-09-06 14:17:47 +08:00
Yadong Qi 64e826f91e [SPARK-17358][SQL] Cached table(parquet/orc) should be shard between beelines
## What changes were proposed in this pull request?
Cached table(parquet/orc) couldn't be shard between beelines, because the `sameResult` method used by `CacheManager` always return false(`sparkSession` are different) when compare two `HadoopFsRelation` in different beelines. So we make `sparkSession` a curry parameter.

## How was this patch tested?
Beeline1
```
1: jdbc:hive2://localhost:10000> CACHE TABLE src_pqt;
+---------+--+
| Result  |
+---------+--+
+---------+--+
No rows selected (5.143 seconds)
1: jdbc:hive2://localhost:10000> EXPLAIN SELECT * FROM src_pqt;
+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--+
|                                                                                                                                                                                                            plan                                                                                                                                                                                                            |
+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--+
| == Physical Plan ==
InMemoryTableScan [key#49, value#50]
   +- InMemoryRelation [key#49, value#50], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas), `src_pqt`
         +- *FileScan parquet default.src_pqt[key#0,value#1] Batched: true, Format: ParquetFormat, InputPaths: hdfs://199.0.0.1:9000/qiyadong/src_pqt, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<key:int,value:string>  |
+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--+
```

Beeline2
```
0: jdbc:hive2://localhost:10000> EXPLAIN SELECT * FROM src_pqt;
+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--+
|                                                                                                                                                                                                            plan                                                                                                                                                                                                            |
+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--+
| == Physical Plan ==
InMemoryTableScan [key#68, value#69]
   +- InMemoryRelation [key#68, value#69], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas), `src_pqt`
         +- *FileScan parquet default.src_pqt[key#0,value#1] Batched: true, Format: ParquetFormat, InputPaths: hdfs://199.0.0.1:9000/qiyadong/src_pqt, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<key:int,value:string>  |
+----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+--+
```

Author: Yadong Qi <qiyadong2010@gmail.com>

Closes #14913 from watermen/SPARK-17358.
2016-09-06 10:57:21 +08:00
Sean Zhong afb3d5d301 [SPARK-17369][SQL] MetastoreRelation toJSON throws AssertException due to missing otherCopyArgs
## What changes were proposed in this pull request?

`TreeNode.toJSON` requires a subclass to explicitly override otherCopyArgs to include currying construction arguments, otherwise it reports AssertException telling that the construction argument values' count doesn't match the construction argument names' count.

For class `MetastoreRelation`, it has a currying construction parameter `client: HiveClient`, but Spark forgets to add it to the list of otherCopyArgs.

## How was this patch tested?

Unit tests.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #14928 from clockfly/metastore_relation_toJSON.
2016-09-06 10:50:07 +08:00
wangzhenhua 6d86403d8b [SPARK-17072][SQL] support table-level statistics generation and storing into/loading from metastore
## What changes were proposed in this pull request?

1. Support generation table-level statistics for
    - hive tables in HiveExternalCatalog
    - data source tables in HiveExternalCatalog
    - data source tables in InMemoryCatalog.
2. Add a property "catalogStats" in CatalogTable to hold statistics in Spark side.
3. Put logics of statistics transformation between Spark and Hive in HiveClientImpl.
4. Extend Statistics class by adding rowCount (will add estimatedSize when we have column stats).

## How was this patch tested?

add unit tests

Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #14712 from wzhfy/tableStats.
2016-09-05 17:32:31 +02:00
gatorsmile c1e9a6d274 [SPARK-17393][SQL] Error Handling when CTAS Against the Same Data Source Table Using Overwrite Mode
### What changes were proposed in this pull request?
When we trying to read a table and then write to the same table using the `Overwrite` save mode, we got a very confusing error message:
For example,
```Scala
      Seq((1, 2)).toDF("i", "j").write.saveAsTable("tab1")
      table("tab1").write.mode(SaveMode.Overwrite).saveAsTable("tab1")
```

```
Job aborted.
org.apache.spark.SparkException: Job aborted.
	at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand$$anonfun$run$1.apply$mcV$sp
...
Caused by: org.apache.spark.SparkException: Task failed while writing rows
	at org.apache.spark.sql.execution.datasources.DefaultWriterContainer.writeRows(WriterContainer.scala:266)
	at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand$$anonfun$run$1$$anonfun$apply$mcV$sp$1.apply(InsertIntoHadoopFsRelationCommand.scala:143)
	at org.apache.spark.sql.execution.datasources
```

After the PR, we will issue an `AnalysisException`:
```
Cannot overwrite table `tab1` that is also being read from
```
### How was this patch tested?
Added test cases.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14954 from gatorsmile/ctasQueryAnalyze.
2016-09-05 11:28:19 +08:00
gatorsmile 6b156e2fcf [SPARK-17324][SQL] Remove Direct Usage of HiveClient in InsertIntoHiveTable
### What changes were proposed in this pull request?
This is another step to get rid of HiveClient from `HiveSessionState`. All the metastore interactions should be through `ExternalCatalog` interface. However, the existing implementation of `InsertIntoHiveTable ` still requires Hive clients. This PR is to remove HiveClient by moving the metastore interactions into `ExternalCatalog`.

### How was this patch tested?
Existing test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14888 from gatorsmile/removeClientFromInsertIntoHiveTable.
2016-09-04 15:04:33 +08:00
Sandeep Singh a8a35b39b9 [MINOR][SQL] Not dropping all necessary tables
## What changes were proposed in this pull request?
was not dropping table `parquet_t3`

## How was this patch tested?
tested `LogicalPlanToSQLSuite` locally

Author: Sandeep Singh <sandeep@techaddict.me>

Closes #13767 from techaddict/minor-8.
2016-09-03 15:35:19 +01:00
Srinath Shankar e6132a6cf1 [SPARK-17298][SQL] Require explicit CROSS join for cartesian products
## What changes were proposed in this pull request?

Require the use of CROSS join syntax in SQL (and a new crossJoin
DataFrame API) to specify explicit cartesian products between relations.
By cartesian product we mean a join between relations R and S where
there is no join condition involving columns from both R and S.

If a cartesian product is detected in the absence of an explicit CROSS
join, an error must be thrown. Turning on the
"spark.sql.crossJoin.enabled" configuration flag will disable this check
and allow cartesian products without an explicit CROSS join.

The new crossJoin DataFrame API must be used to specify explicit cross
joins. The existing join(DataFrame) method will produce a INNER join
that will require a subsequent join condition.
That is df1.join(df2) is equivalent to select * from df1, df2.

## How was this patch tested?

Added cross-join.sql to the SQLQueryTestSuite to test the check for cartesian products. Added a couple of tests to the DataFrameJoinSuite to test the crossJoin API. Modified various other test suites to explicitly specify a cross join where an INNER join or a comma-separated list was previously used.

Author: Srinath Shankar <srinath@databricks.com>

Closes #14866 from srinathshankar/crossjoin.
2016-09-03 00:20:43 +02:00
Davies Liu ed9c884dcf [SPARK-17230] [SQL] Should not pass optimized query into QueryExecution in DataFrameWriter
## What changes were proposed in this pull request?

Some analyzer rules have assumptions on logical plans, optimizer may break these assumption, we should not pass an optimized query plan into QueryExecution (will be analyzed again), otherwise we may some weird bugs.

For example, we have a rule for decimal calculation to promote the precision before binary operations, use PromotePrecision as placeholder to indicate that this rule should not apply twice. But a Optimizer rule will remove this placeholder, that break the assumption, then the rule applied twice, cause wrong result.

Ideally, we should make all the analyzer rules all idempotent, that may require lots of effort to double checking them one by one (may be not easy).

An easier approach could be never feed a optimized plan into Analyzer, this PR fix the case for RunnableComand, they will be optimized, during execution, the passed `query` will also be passed into QueryExecution again. This PR make these `query` not part of the children, so they will not be optimized and analyzed again.

Right now, we did not know a logical plan is optimized or not, we could introduce a flag for that, and make sure a optimized logical plan will not be analyzed again.

## How was this patch tested?

Added regression tests.

Author: Davies Liu <davies@databricks.com>

Closes #14797 from davies/fix_writer.
2016-09-02 15:10:12 -07:00
gatorsmile 247a4faf06 [SPARK-16935][SQL] Verification of Function-related ExternalCatalog APIs
### What changes were proposed in this pull request?
Function-related `HiveExternalCatalog` APIs do not have enough verification logics. After the PR, `HiveExternalCatalog` and `InMemoryCatalog` become consistent in the error handling.

For example, below is the exception we got when calling `renameFunction`.
```
15:13:40.369 WARN org.apache.hadoop.hive.metastore.ObjectStore: Failed to get database db1, returning NoSuchObjectException
15:13:40.377 WARN org.apache.hadoop.hive.metastore.ObjectStore: Failed to get database db2, returning NoSuchObjectException
15:13:40.739 ERROR DataNucleus.Datastore.Persist: Update of object "org.apache.hadoop.hive.metastore.model.MFunction205629e9" using statement "UPDATE FUNCS SET FUNC_NAME=? WHERE FUNC_ID=?" failed : org.apache.derby.shared.common.error.DerbySQLIntegrityConstraintViolationException: The statement was aborted because it would have caused a duplicate key value in a unique or primary key constraint or unique index identified by 'UNIQUEFUNCTION' defined on 'FUNCS'.
	at org.apache.derby.impl.jdbc.SQLExceptionFactory.getSQLException(Unknown Source)
	at org.apache.derby.impl.jdbc.Util.generateCsSQLException(Unknown Source)
	at org.apache.derby.impl.jdbc.TransactionResourceImpl.wrapInSQLException(Unknown Source)
	at org.apache.derby.impl.jdbc.TransactionResourceImpl.handleException(Unknown Source)
```

### How was this patch tested?
Improved the existing test cases to check whether the messages are right.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14521 from gatorsmile/functionChecking.
2016-09-02 22:31:01 +08:00
Brian Cho f2d6e2ef23 [SPARK-16926][SQL] Add unit test to compare table and partition column metadata.
## What changes were proposed in this pull request?

Add unit test for changes made in PR #14515. It makes sure that a newly created table has the same number of columns in table and partition metadata. This test fails before the changes introduced in #14515.

## How was this patch tested?

Run new unit test.

Author: Brian Cho <bcho@fb.com>

Closes #14930 from dafrista/partition-metadata-unit-test.
2016-09-02 11:12:34 +08:00
Qifan Pu 03d77af9ec [SPARK-16525] [SQL] Enable Row Based HashMap in HashAggregateExec
## What changes were proposed in this pull request?

This PR is the second step for the following feature:

For hash aggregation in Spark SQL, we use a fast aggregation hashmap to act as a "cache" in order to boost aggregation performance. Previously, the hashmap is backed by a `ColumnarBatch`. This has performance issues when we have wide schema for the aggregation table (large number of key fields or value fields).
In this JIRA, we support another implementation of fast hashmap, which is backed by a `RowBatch`. We then automatically pick between the two implementations based on certain knobs.

In this second-step PR, we enable `RowBasedHashMapGenerator` in `HashAggregateExec`.

## How was this patch tested?

Added tests: `RowBasedAggregateHashMapSuite` and ` VectorizedAggregateHashMapSuite`
Additional micro-benchmarks tests and TPCDS results will be added in a separate PR in the series.

Author: Qifan Pu <qifan.pu@gmail.com>
Author: ooq <qifan.pu@gmail.com>

Closes #14176 from ooq/rowbasedfastaggmap-pr2.
2016-09-01 16:56:35 -07:00
Brian Cho 473d78649d [SPARK-16926] [SQL] Remove partition columns from partition metadata.
## What changes were proposed in this pull request?

This removes partition columns from column metadata of partitions to match tables.

A change introduced in SPARK-14388 removed partition columns from the column metadata of tables, but not for partitions. This causes TableReader to believe that the schema is different between table and partition, and create an unnecessary conversion object inspector in TableReader.

## How was this patch tested?

Existing unit tests.

Author: Brian Cho <bcho@fb.com>

Closes #14515 from dafrista/partition-columns-metadata.
2016-09-01 14:13:17 -07:00
Herman van Hovell 2be5f8d7e0 [SPARK-17263][SQL] Add hexadecimal literal parsing
## What changes were proposed in this pull request?
This PR adds the ability to parse SQL (hexadecimal) binary literals (AKA bit strings). It follows the following syntax `X'[Hexadecimal Characters]+'`, for example: `X'01AB'` would create a binary the following binary array `0x01AB`.

If an uneven number of hexadecimal characters is passed, then the upper 4 bits of the initial byte are kept empty, and the lower 4 bits are filled using the first character. For example `X'1C7'` would create the following binary array `0x01C7`.

Binary data (Array[Byte]) does not have a proper `hashCode` and `equals` functions. This meant that comparing `Literal`s containing binary data was a pain. I have updated Literal.hashCode and Literal.equals to deal properly with binary data.

## How was this patch tested?
Added tests to the `ExpressionParserSuite`, `SQLQueryTestSuite` and `ExpressionSQLBuilderSuite`.

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

Closes #14832 from hvanhovell/SPARK-17263.
2016-09-01 12:01:22 -07:00
Wenchen Fan 8e740ae44d [SPARK-17257][SQL] the physical plan of CREATE TABLE or CTAS should take CatalogTable
## What changes were proposed in this pull request?

This is kind of a follow-up of https://github.com/apache/spark/pull/14482 . As we put `CatalogTable` in the logical plan directly, it makes sense to let physical plans take `CatalogTable` directly, instead of extracting some fields of `CatalogTable` in planner and then construct a new `CatalogTable` in physical plan.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14823 from cloud-fan/create-table.
2016-09-01 16:45:22 +08:00
gatorsmile 1f06a5b6a0 [SPARK-17353][SPARK-16943][SPARK-16942][SQL] Fix multiple bugs in CREATE TABLE LIKE command
### What changes were proposed in this pull request?
The existing `CREATE TABLE LIKE` command has multiple issues:

- The generated table is non-empty when the source table is a data source table. The major reason is the data source table is using the table property `path` to store the location of table contents. Currently, we keep it unchanged. Thus, we still create the same table with the same location.

- The table type of the generated table is `EXTERNAL` when the source table is an external Hive Serde table. Currently, we explicitly set it to `MANAGED`, but Hive is checking the table property `EXTERNAL` to decide whether the table is `EXTERNAL` or not. (See https://github.com/apache/hive/blob/master/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java#L1407-L1408) Thus, the created table is still `EXTERNAL`.

- When the source table is a `VIEW`, the metadata of the generated table contains the original view text and view original text. So far, this does not break anything, but it could cause something wrong in Hive. (For example, https://github.com/apache/hive/blob/master/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java#L1405-L1406)

- The issue regarding the table `comment`. To follow what Hive does, the table comment should be cleaned, but the column comments should be still kept.

- The `INDEX` table is not supported. Thus, we should throw an exception in this case.

- `owner` should not be retained. `ToHiveTable` set it [here](e679bc3c1c/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala (L793)) no matter which value we set in `CatalogTable`. We set it to an empty string for avoiding the confusing output in Explain.

- Add a support for temp tables

- Like Hive, we should not copy the table properties from the source table to the created table, especially for the statistics-related properties, which could be wrong in the created table.

- `unsupportedFeatures` should not be copied from the source table. The created table does not have these unsupported features.

- When the type of source table is a view, the target table is using the default format of data source tables: `spark.sql.sources.default`.

This PR is to fix the above issues.

### How was this patch tested?
Improve the test coverage by adding more test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14531 from gatorsmile/createTableLike.
2016-09-01 16:36:14 +08:00
Sean Zhong a18c169fd0 [SPARK-16283][SQL] Implements percentile_approx aggregation function which supports partial aggregation.
## What changes were proposed in this pull request?

This PR implements aggregation function `percentile_approx`. Function `percentile_approx` returns the approximate percentile(s) of a column at the given percentage(s). A percentile is a watermark value below which a given percentage of the column values fall. For example, the percentile of column `col` at percentage 50% is the median value of column `col`.

### Syntax:
```
# Returns percentile at a given percentage value. The approximation error can be reduced by increasing parameter accuracy, at the cost of memory.
percentile_approx(col, percentage [, accuracy])

# Returns percentile value array at given percentage value array
percentile_approx(col, array(percentage1 [, percentage2]...) [, accuracy])
```

### Features:
1. This function supports partial aggregation.
2. The memory consumption is bounded. The larger `accuracy` parameter we choose, we smaller error we get. The default accuracy value is 10000, to match with Hive default setting. Choose a smaller value for smaller memory footprint.
3.  This function supports window function aggregation.

### Example usages:
```
## Returns the 25th percentile value, with default accuracy
SELECT percentile_approx(col, 0.25) FROM table

## Returns an array of percentile value (25th, 50th, 75th), with default accuracy
SELECT percentile_approx(col, array(0.25, 0.5, 0.75)) FROM table

## Returns 25th percentile value, with custom accuracy value 100, larger accuracy parameter yields smaller approximation error
SELECT percentile_approx(col, 0.25, 100) FROM table

## Returns the 25th, and 50th percentile values, with custom accuracy value 100
SELECT percentile_approx(col, array(0.25, 0.5), 100) FROM table
```

### NOTE:
1. The `percentile_approx` implementation is different from Hive, so the result returned on same query maybe slightly different with Hive. This implementation uses `QuantileSummaries` as the underlying probabilistic data structure, and mainly follows paper `Space-efficient Online Computation of Quantile Summaries` by Greenwald, Michael and Khanna, Sanjeev. (http://dx.doi.org/10.1145/375663.375670)`
2. The current implementation of `QuantileSummaries` doesn't support automatic compression. This PR has a rule to do compression automatically at the caller side, but it may not be optimal.

## How was this patch tested?

Unit test, and Sql query test.

## Acknowledgement
1. This PR's work in based on lw-lin's PR https://github.com/apache/spark/pull/14298, with improvements like supporting partial aggregation, fixing out of memory issue.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #14868 from clockfly/appro_percentile_try_2.
2016-09-01 16:31:13 +08:00
Wenchen Fan 12fd0cd615 [SPARK-17180][SPARK-17309][SPARK-17323][SQL] create AlterViewAsCommand to handle ALTER VIEW AS
## What changes were proposed in this pull request?

Currently we use `CreateViewCommand` to implement ALTER VIEW AS, which has 3 bugs:

1. SPARK-17180: ALTER VIEW AS should alter temp view if view name has no database part and temp view exists
2. SPARK-17309: ALTER VIEW AS should issue exception if view does not exist.
3. SPARK-17323: ALTER VIEW AS should keep the previous table properties, comment, create_time, etc.

The root cause is, ALTER VIEW AS is quite different from CREATE VIEW, we need different code path to handle them. However, in `CreateViewCommand`, there is no way to distinguish ALTER VIEW AS and CREATE VIEW, we have to introduce extra flag. But instead of doing this, I think a more natural way is to separate the ALTER VIEW AS logic into a new command.

## How was this patch tested?

new tests in SQLViewSuite

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14874 from cloud-fan/minor4.
2016-08-31 17:08:08 +08:00
Xin Ren 2d76cb11f5 [SPARK-17276][CORE][TEST] Stop env params output on Jenkins job page
https://issues.apache.org/jira/browse/SPARK-17276

## What changes were proposed in this pull request?

When trying to find error msg in a failed Jenkins build job, I'm annoyed by the huge env output.
The env parameter output should be muted.

![screen shot 2016-08-26 at 10 52 07 pm](https://cloud.githubusercontent.com/assets/3925641/18025581/b8d567ba-6be2-11e6-9eeb-6aec223f1730.png)

## How was this patch tested?

Tested manually on local laptop.

Author: Xin Ren <iamshrek@126.com>

Closes #14848 from keypointt/SPARK-17276.
2016-08-30 11:18:29 +01:00
gatorsmile bca79c8230 [SPARK-17234][SQL] Table Existence Checking when Index Table with the Same Name Exists
### What changes were proposed in this pull request?
Hive Index tables are not supported by Spark SQL. Thus, we issue an exception when users try to access Hive Index tables. When the internal function `tableExists` tries to access Hive Index tables, it always gets the same error message: ```Hive index table is not supported```. This message could be confusing to users, since their SQL operations could be completely unrelated to Hive Index tables. For example, when users try to alter a table to a new name and there exists an index table with the same name, the expected exception should be a `TableAlreadyExistsException`.

This PR made the following changes:
- Introduced a new `AnalysisException` type: `SQLFeatureNotSupportedException`. When users try to access an `Index Table`, we will issue a `SQLFeatureNotSupportedException`.
- `tableExists` returns `true` when hitting a `SQLFeatureNotSupportedException` and the feature is `Hive index table`.
- Add a checking `requireTableNotExists` for `SessionCatalog`'s `createTable` API; otherwise, the current implementation relies on the Hive's internal checking.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14801 from gatorsmile/tableExists.
2016-08-30 17:27:00 +08:00
Davies Liu 48caec2516 [SPARK-17063] [SQL] Improve performance of MSCK REPAIR TABLE with Hive metastore
## What changes were proposed in this pull request?

This PR split the the single `createPartitions()` call into smaller batches, which could prevent Hive metastore from OOM (caused by millions of partitions).

It will also try to gather all the fast stats (number of files and total size of all files) in parallel to avoid the bottle neck of listing the files in metastore sequential, which is controlled by spark.sql.gatherFastStats (enabled by default).

## How was this patch tested?

Tested locally with 10000 partitions and 100 files with embedded metastore, without gathering fast stats in parallel, adding partitions took 153 seconds, after enable that, gathering the fast stats took about 34 seconds, adding these partitions took 25 seconds (most of the time spent in object store), 59 seconds in total, 2.5X faster (with larger cluster, gathering will much faster).

Author: Davies Liu <davies@databricks.com>

Closes #14607 from davies/repair_batch.
2016-08-29 11:23:53 -07:00
Herman van Hovell a11d10f182 [SPARK-17246][SQL] Add BigDecimal literal
## What changes were proposed in this pull request?
This PR adds parser support for `BigDecimal` literals. If you append the suffix `BD` to a valid number then this will be interpreted as a `BigDecimal`, for example `12.0E10BD` will interpreted into a BigDecimal with scale -9 and precision 3. This is useful in situations where you need exact values.

## How was this patch tested?
Added tests to `ExpressionParserSuite`, `ExpressionSQLBuilderSuite` and `SQLQueryTestSuite`.

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

Closes #14819 from hvanhovell/SPARK-17246.
2016-08-26 13:29:22 -07:00
gatorsmile 261c55dd88 [SPARK-17250][SQL] Remove HiveClient and setCurrentDatabase from HiveSessionCatalog
### What changes were proposed in this pull request?
This is the first step to remove `HiveClient` from `HiveSessionState`. In the metastore interaction, we always use the fully qualified table name when accessing/operating a table. That means, we always specify the database. Thus, it is not necessary to use `HiveClient` to change the active database in Hive metastore.

In `HiveSessionCatalog `, `setCurrentDatabase` is the only function that uses `HiveClient`. Thus, we can remove it after removing `setCurrentDatabase`

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14821 from gatorsmile/setCurrentDB.
2016-08-26 11:19:03 -07:00
gatorsmile fd4ba3f626 [SPARK-17192][SQL] Issue Exception when Users Specify the Partitioning Columns without a Given Schema
### What changes were proposed in this pull request?
Address the comments by yhuai in the original PR: https://github.com/apache/spark/pull/14207

First, issue an exception instead of logging a warning when users specify the partitioning columns without a given schema.

Second, refactor the codes a little.

### How was this patch tested?
Fixed the test cases.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14572 from gatorsmile/followup16552.
2016-08-26 11:13:38 -07:00
Wenchen Fan 28ab17922a [SPARK-17260][MINOR] move CreateTables to HiveStrategies
## What changes were proposed in this pull request?

`CreateTables` rule turns a general `CreateTable` plan to `CreateHiveTableAsSelectCommand` for hive serde table. However, this rule is logically a planner strategy, we should move it to `HiveStrategies`, to be consistent with other DDL commands.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14825 from cloud-fan/ctas.
2016-08-26 08:52:10 -07:00
hyukjinkwon 6063d5963f [SPARK-16216][SQL][FOLLOWUP] Enable timestamp type tests for JSON and verify all unsupported types in CSV
## What changes were proposed in this pull request?

This PR enables the tests for `TimestampType` for JSON and unifies the logics for verifying schema when writing in CSV.

In more details, this PR,

- Enables the tests for `TimestampType` for JSON and

  This was disabled due to an issue in `DatatypeConverter.parseDateTime` which parses dates incorrectly, for example as below:

  ```scala
   val d = javax.xml.bind.DatatypeConverter.parseDateTime("0900-01-01T00:00:00.000").getTime
  println(d.toString)
  ```
  ```
  Fri Dec 28 00:00:00 KST 899
  ```

  However, since we use `FastDateFormat`, it seems we are safe now.

  ```scala
  val d = FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSS").parse("0900-01-01T00:00:00.000")
  println(d)
  ```
  ```
  Tue Jan 01 00:00:00 PST 900
  ```

- Verifies all unsupported types in CSV

  There is a separate logics to verify the schemas in `CSVFileFormat`. This is actually not quite correct enough because we don't support `NullType` and `CalanderIntervalType` as well `StructType`, `ArrayType`, `MapType`. So, this PR adds both types.

## How was this patch tested?

Tests in `JsonHadoopFsRelation` and `CSVSuite`

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #14829 from HyukjinKwon/SPARK-16216-followup.
2016-08-26 17:29:37 +02:00
Josh Rosen 3e4c7db4d1 [SPARK-17205] Literal.sql should handle Infinity and NaN
This patch updates `Literal.sql` to properly generate SQL for `NaN` and `Infinity` float and double literals: these special values need to be handled differently from regular values, since simply appending a suffix to the value's `toString()` representation will not work for these values.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #14777 from JoshRosen/SPARK-17205.
2016-08-26 00:15:01 +02:00
gatorsmile 4d0706d616 [SPARK-17190][SQL] Removal of HiveSharedState
### What changes were proposed in this pull request?
Since `HiveClient` is used to interact with the Hive metastore, it should be hidden in `HiveExternalCatalog`. After moving `HiveClient` into `HiveExternalCatalog`, `HiveSharedState` becomes a wrapper of `HiveExternalCatalog`. Thus, removal of `HiveSharedState` becomes straightforward. After removal of `HiveSharedState`, the reflection logic is directly applied on the choice of `ExternalCatalog` types, based on the configuration of `CATALOG_IMPLEMENTATION`.

~~`HiveClient` is also used/invoked by the other entities besides HiveExternalCatalog, we defines the following two APIs: getClient and getNewClient~~

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14757 from gatorsmile/removeHiveClient.
2016-08-25 12:50:03 +08:00
hyukjinkwon 29952ed096 [SPARK-16216][SQL] Read/write timestamps and dates in ISO 8601 and dateFormat/timestampFormat option for CSV and JSON
## What changes were proposed in this pull request?

### Default - ISO 8601

Currently, CSV datasource is writing `Timestamp` and `Date` as numeric form and JSON datasource is writing both as below:

- CSV
  ```
  // TimestampType
  1414459800000000
  // DateType
  16673
  ```

- Json

  ```
  // TimestampType
  1970-01-01 11:46:40.0
  // DateType
  1970-01-01
  ```

So, for CSV we can't read back what we write and for JSON it becomes ambiguous because the timezone is being missed.

So, this PR make both **write** `Timestamp` and `Date` in ISO 8601 formatted string (please refer the [ISO 8601 specification](https://www.w3.org/TR/NOTE-datetime)).

- For `Timestamp` it becomes as below: (`yyyy-MM-dd'T'HH:mm:ss.SSSZZ`)

  ```
  1970-01-01T02:00:01.000-01:00
  ```

- For `Date` it becomes as below (`yyyy-MM-dd`)

  ```
  1970-01-01
  ```

### Custom date format option - `dateFormat`

This PR also adds the support to write and read dates and timestamps in a formatted string as below:

- **DateType**

  - With `dateFormat` option (e.g. `yyyy/MM/dd`)

    ```
    +----------+
    |      date|
    +----------+
    |2015/08/26|
    |2014/10/27|
    |2016/01/28|
    +----------+
    ```

### Custom date format option - `timestampFormat`

- **TimestampType**

  - With `dateFormat` option (e.g. `dd/MM/yyyy HH:mm`)

    ```
    +----------------+
    |            date|
    +----------------+
    |2015/08/26 18:00|
    |2014/10/27 18:30|
    |2016/01/28 20:00|
    +----------------+
    ```

## How was this patch tested?

Unit tests were added in `CSVSuite` and `JsonSuite`. For JSON, existing tests cover the default cases.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #14279 from HyukjinKwon/SPARK-16216-json-csv.
2016-08-24 22:16:20 +02:00
Wenchen Fan 52fa45d62a [SPARK-17186][SQL] remove catalog table type INDEX
## What changes were proposed in this pull request?

Actually Spark SQL doesn't support index, the catalog table type `INDEX` is from Hive. However, most operations in Spark SQL can't handle index table, e.g. create table, alter table, etc.

Logically index table should be invisible to end users, and Hive also generates special table name for index table to avoid users accessing it directly. Hive has special SQL syntax to create/show/drop index tables.

At Spark SQL side, although we can describe index table directly, but the result is unreadable, we should use the dedicated SQL syntax to do it(e.g. `SHOW INDEX ON tbl`). Spark SQL can also read index table directly, but the result is always empty.(Can hive read index table directly?)

This PR remove the table type `INDEX`, to make it clear that Spark SQL doesn't support index currently.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14752 from cloud-fan/minor2.
2016-08-23 23:46:09 -07:00
Weiqing Yang b9994ad056 [MINOR][SQL] Remove implemented functions from comments of 'HiveSessionCatalog.scala'
## What changes were proposed in this pull request?
This PR removes implemented functions from comments of `HiveSessionCatalog.scala`: `java_method`, `posexplode`, `str_to_map`.

## How was this patch tested?
Manual.

Author: Weiqing Yang <yangweiqing001@gmail.com>

Closes #14769 from Sherry302/cleanComment.
2016-08-23 23:44:45 -07:00
Josh Rosen bf8ff833e3 [SPARK-17194] Use single quotes when generating SQL for string literals
When Spark emits SQL for a string literal, it should wrap the string in single quotes, not double quotes. Databases which adhere more strictly to the ANSI SQL standards, such as Postgres, allow only single-quotes to be used for denoting string literals (see http://stackoverflow.com/a/1992331/590203).

Author: Josh Rosen <joshrosen@databricks.com>

Closes #14763 from JoshRosen/SPARK-17194.
2016-08-23 22:31:58 +02:00
Eric Liang 84770b59f7 [SPARK-17162] Range does not support SQL generation
## What changes were proposed in this pull request?

The range operator previously didn't support SQL generation, which made it not possible to use in views.

## How was this patch tested?

Unit tests.

cc hvanhovell

Author: Eric Liang <ekl@databricks.com>

Closes #14724 from ericl/spark-17162.
2016-08-22 15:48:35 -07:00
Wenchen Fan b2074b664a [SPARK-16498][SQL] move hive hack for data source table into HiveExternalCatalog
## What changes were proposed in this pull request?

Spark SQL doesn't have its own meta store yet, and use hive's currently. However, hive's meta store has some limitations(e.g. columns can't be too many, not case-preserving, bad decimal type support, etc.), so we have some hacks to successfully store data source table metadata into hive meta store, i.e. put all the information in table properties.

This PR moves these hacks to `HiveExternalCatalog`, tries to isolate hive specific logic in one place.

changes overview:

1.  **before this PR**: we need to put metadata(schema, partition columns, etc.) of data source tables to table properties before saving it to external catalog, even the external catalog doesn't use hive metastore(e.g. `InMemoryCatalog`)
**after this PR**: the table properties tricks are only in `HiveExternalCatalog`, the caller side doesn't need to take care of it anymore.

2. **before this PR**: because the table properties tricks are done outside of external catalog, so we also need to revert these tricks when we read the table metadata from external catalog and use it. e.g. in `DescribeTableCommand` we will read schema and partition columns from table properties.
**after this PR**: The table metadata read from external catalog is exactly the same with what we saved to it.

bonus: now we can create data source table using `SessionCatalog`, if schema is specified.
breaks: `schemaStringLengthThreshold` is not configurable anymore. `hive.default.rcfile.serde` is not configurable anymore.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14155 from cloud-fan/catalog-table.
2016-08-21 22:23:14 -07:00
Liang-Chi Hsieh 31a0155720 [SPARK-17104][SQL] LogicalRelation.newInstance should follow the semantics of MultiInstanceRelation
## What changes were proposed in this pull request?

Currently `LogicalRelation.newInstance()` simply creates another `LogicalRelation` object with the same parameters. However, the `newInstance()` method inherited from `MultiInstanceRelation` should return a copy of object with unique expression ids. Current `LogicalRelation.newInstance()` can cause failure when doing self-join.

## How was this patch tested?

Jenkins tests.

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

Closes #14682 from viirya/fix-localrelation.
2016-08-20 23:29:48 +08:00
petermaxlee 45d40d9f66 [SPARK-17150][SQL] Support SQL generation for inline tables
## What changes were proposed in this pull request?
This patch adds support for SQL generation for inline tables. With this, it would be possible to create a view that depends on inline tables.

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

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14709 from petermaxlee/SPARK-17150.
2016-08-20 13:19:38 +08:00
petermaxlee a117afa7c2 [SPARK-17149][SQL] array.sql for testing array related functions
## What changes were proposed in this pull request?
This patch creates array.sql in SQLQueryTestSuite for testing array related functions, including:

- indexing
- array creation
- size
- array_contains
- sort_array

## How was this patch tested?
The patch itself is about adding tests.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14708 from petermaxlee/SPARK-17149.
2016-08-19 18:14:45 -07:00
Steve Loughran cc97ea188e [SPARK-16736][CORE][SQL] purge superfluous fs calls
A review of the code, working back from Hadoop's `FileSystem.exists()` and `FileSystem.isDirectory()` code, then removing uses of the calls when superfluous.

1. delete is harmless if called on a nonexistent path, so don't do any checks before deletes
1. any `FileSystem.exists()`  check before `getFileStatus()` or `open()` is superfluous as the operation itself does the check. Instead the `FileNotFoundException` is caught and triggers the downgraded path. When a `FileNotFoundException` was thrown before, the code still creates a new FNFE with the error messages. Though now the inner exceptions are nested, for easier diagnostics.

Initially, relying on Jenkins test runs.

One troublespot here is that some of the codepaths are clearly error situations; it's not clear that they have coverage anyway. Trying to create the failure conditions in tests would be ideal, but it will also be hard.

Author: Steve Loughran <stevel@apache.org>

Closes #14371 from steveloughran/cloud/SPARK-16736-superfluous-fs-calls.
2016-08-17 11:43:01 -07:00
Herman van Hovell f7c9ff57c1 [SPARK-17068][SQL] Make view-usage visible during analysis
## What changes were proposed in this pull request?
This PR adds a field to subquery alias in order to make the usage of views in a resolved `LogicalPlan` more visible (and more understandable).

For example, the following view and query:
```sql
create view constants as select 1 as id union all select 1 union all select 42
select * from constants;
```
...now yields the following analyzed plan:
```
Project [id#39]
+- SubqueryAlias c, `default`.`constants`
   +- Project [gen_attr_0#36 AS id#39]
      +- SubqueryAlias gen_subquery_0
         +- Union
            :- Union
            :  :- Project [1 AS gen_attr_0#36]
            :  :  +- OneRowRelation$
            :  +- Project [1 AS gen_attr_1#37]
            :     +- OneRowRelation$
            +- Project [42 AS gen_attr_2#38]
               +- OneRowRelation$
```
## How was this patch tested?
Added tests for the two code paths in `SessionCatalogSuite` (sql/core) and `HiveMetastoreCatalogSuite` (sql/hive)

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

Closes #14657 from hvanhovell/SPARK-17068.
2016-08-16 23:09:53 -07:00
Herman van Hovell 8fdc6ce400 [SPARK-16964][SQL] Remove private[hive] from sql.hive.execution package
## What changes were proposed in this pull request?
This PR is a small follow-up to https://github.com/apache/spark/pull/14554. This also widens the visibility of a few (similar) Hive classes.

## How was this patch tested?
No test. Only a visibility change.

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

Closes #14654 from hvanhovell/SPARK-16964-hive.
2016-08-16 01:12:27 -07:00
petermaxlee 00e103a6ed [SPARK-17013][SQL] Parse negative numeric literals
## What changes were proposed in this pull request?
This patch updates the SQL parser to parse negative numeric literals as numeric literals, instead of unary minus of positive literals.

This allows the parser to parse the minimal value for each data type, e.g. "-32768S".

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

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14608 from petermaxlee/SPARK-17013.
2016-08-11 23:56:55 -07:00
petermaxlee b9f8a11709 [SPARK-16866][SQL] Infrastructure for file-based SQL end-to-end tests
## What changes were proposed in this pull request?
This patch introduces SQLQueryTestSuite, a basic framework for end-to-end SQL test cases defined in spark/sql/core/src/test/resources/sql-tests. This is a more standard way to test SQL queries end-to-end in different open source database systems, because it is more manageable to work with files.

This is inspired by HiveCompatibilitySuite, but simplified for general Spark SQL tests. Once this is merged, I can work towards porting SQLQuerySuite over, and eventually also move the existing HiveCompatibilitySuite to use this framework.

Unlike HiveCompatibilitySuite, SQLQueryTestSuite compares both the output schema and the output data (in string form).

When there is a mismatch, the error message looks like the following:

```
[info] - blacklist.sql !!! IGNORED !!!
[info] - number-format.sql *** FAILED *** (2 seconds, 405 milliseconds)
[info]   Expected "...147483648	-214748364[8]", but got "...147483648	-214748364[9]" Result should match for query #1 (SQLQueryTestSuite.scala:171)
[info]   org.scalatest.exceptions.TestFailedException:
[info]   at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:495)
[info]   at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555)
[info]   at org.scalatest.Assertions$class.assertResult(Assertions.scala:1171)
```

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

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14472 from petermaxlee/SPARK-16866.
2016-08-10 17:17:21 +08:00
gatorsmile bdd537164d [SPARK-16959][SQL] Rebuild Table Comment when Retrieving Metadata from Hive Metastore
### What changes were proposed in this pull request?
The `comment` in `CatalogTable` returned from Hive is always empty. We store it in the table property when creating a table. However, when we try to retrieve the table metadata from Hive metastore, we do not rebuild it. The `comment` is always empty.

This PR is to fix the issue.

### How was this patch tested?
Fixed the test case to verify the change.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14550 from gatorsmile/tableComment.
2016-08-10 16:25:01 +08:00
Davies Liu 92da22878b [SPARK-16905] SQL DDL: MSCK REPAIR TABLE
## What changes were proposed in this pull request?

MSCK REPAIR TABLE could be used to recover the partitions in external catalog based on partitions in file system.

Another syntax is: ALTER TABLE table RECOVER PARTITIONS

The implementation in this PR will only list partitions (not the files with a partition) in driver (in parallel if needed).

## How was this patch tested?

Added unit tests for it and Hive compatibility test suite.

Author: Davies Liu <davies@databricks.com>

Closes #14500 from davies/repair_table.
2016-08-09 10:04:36 -07:00
Reynold Xin 511f52f842 [SPARK-16964][SQL] Remove private[sql] and private[spark] from sql.execution package
## What changes were proposed in this pull request?
This package is meant to be internal, and as a result it does not make sense to mark things as private[sql] or private[spark]. It simply makes debugging harder when Spark developers need to inspect the plans at runtime.

This patch removes all private[sql] and private[spark] visibility modifiers in org.apache.spark.sql.execution.

## How was this patch tested?
N/A - just visibility changes.

Author: Reynold Xin <rxin@databricks.com>

Closes #14554 from rxin/remote-private.
2016-08-09 18:22:14 +08:00
hyukjinkwon bb2b9d0a42 [SPARK-16610][SQL] Add orc.compress as an alias for compression option.
## What changes were proposed in this pull request?

For ORC source, Spark SQL has a writer option `compression`, which is used to set the codec and its value will be also set to `orc.compress` (the orc conf used for codec). However, if a user only set `orc.compress` in the writer option, we should not use the default value of `compression` (snappy) as the codec. Instead, we should respect the value of `orc.compress`.

This PR makes ORC data source not ignoring `orc.compress` when `comperssion` is unset.

So, here is the behaviour,

 1. Check `compression` and use this if it is set.
 2. If `compression` is not set, check `orc.compress` and use it.
 3. If `compression` and `orc.compress` are not set, then use the default snappy.

## How was this patch tested?

Unit test in `OrcQuerySuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #14518 from HyukjinKwon/SPARK-16610.
2016-08-09 10:23:54 +08:00
Holden Karau 9216901d52 [SPARK-16779][TRIVIAL] Avoid using postfix operators where they do not add much and remove whitelisting
## What changes were proposed in this pull request?

Avoid using postfix operation for command execution in SQLQuerySuite where it wasn't whitelisted and audit existing whitelistings removing postfix operators from most places. Some notable places where postfix operation remains is in the XML parsing & time units (seconds, millis, etc.) where it arguably can improve readability.

## How was this patch tested?

Existing tests.

Author: Holden Karau <holden@us.ibm.com>

Closes #14407 from holdenk/SPARK-16779.
2016-08-08 15:54:03 -07:00
gatorsmile ab126909ce [SPARK-16457][SQL] Fix Wrong Messages when CTAS with a Partition By Clause
#### What changes were proposed in this pull request?
When doing a CTAS with a Partition By clause, we got a wrong error message.

For example,
```SQL
CREATE TABLE gen__tmp
PARTITIONED BY (key string)
AS SELECT key, value FROM mytable1
```
The error message we get now is like
```
Operation not allowed: Schema may not be specified in a Create Table As Select (CTAS) statement(line 2, pos 0)
```

However, based on the code, the message we should get is like
```
Operation not allowed: A Create Table As Select (CTAS) statement is not allowed to create a partitioned table using Hive's file formats. Please use the syntax of "CREATE TABLE tableName USING dataSource OPTIONS (...) PARTITIONED BY ...\" to create a partitioned table through a CTAS statement.(line 2, pos 0)
```

Currently, partitioning columns is part of the schema. This PR fixes the bug by changing the detection orders.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14113 from gatorsmile/ctas.
2016-08-08 22:26:44 +08:00
Yin Huai e679bc3c1c [SPARK-16901] Hive settings in hive-site.xml may be overridden by Hive's default values
## What changes were proposed in this pull request?
When we create the HiveConf for metastore client, we use a Hadoop Conf as the base, which may contain Hive settings in hive-site.xml (https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/internal/SharedState.scala#L49). However, HiveConf's initialize function basically ignores the base Hadoop Conf and always its default values (i.e. settings with non-null default values) as the base (https://github.com/apache/hive/blob/release-1.2.1/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java#L2687). So, even a user put javax.jdo.option.ConnectionURL in hive-site.xml, it is not used and Hive will use its default, which is jdbc:derby:;databaseName=metastore_db;create=true.

This issue only shows up when `spark.sql.hive.metastore.jars` is not set to builtin.

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

Author: Yin Huai <yhuai@databricks.com>

Closes #14497 from yhuai/SPARK-16901.
2016-08-05 15:52:02 -07:00
Wenchen Fan 5effc016c8 [SPARK-16879][SQL] unify logical plans for CREATE TABLE and CTAS
## What changes were proposed in this pull request?

we have various logical plans for CREATE TABLE and CTAS: `CreateTableUsing`, `CreateTableUsingAsSelect`, `CreateHiveTableAsSelectLogicalPlan`. This PR unifies them to reduce the complexity and centralize the error handling.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14482 from cloud-fan/table.
2016-08-05 10:50:26 +02:00
Wenchen Fan 43f4fd6f9b [SPARK-16867][SQL] createTable and alterTable in ExternalCatalog should not take db
## What changes were proposed in this pull request?

These 2 methods take `CatalogTable` as parameter, which already have the database information.

## How was this patch tested?

existing test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14476 from cloud-fan/minor5.
2016-08-04 16:48:30 +08:00
Eric Liang e6f226c567 [SPARK-16596] [SQL] Refactor DataSourceScanExec to do partition discovery at execution instead of planning time
## What changes were proposed in this pull request?

Partition discovery is rather expensive, so we should do it at execution time instead of during physical planning. Right now there is not much benefit since ListingFileCatalog will read scan for all partitions at planning time anyways, but this can be optimized in the future. Also, there might be more information for partition pruning not available at planning time.

This PR moves a lot of the file scan logic from planning to execution time. All file scan operations are handled by `FileSourceScanExec`, which handles both batched and non-batched file scans. This requires some duplication with `RowDataSourceScanExec`, but is probably worth it so that `FileSourceScanExec` does not need to depend on an input RDD.

TODO: In another pr, move DataSourceScanExec to it's own file.

## How was this patch tested?

Existing tests (it might be worth adding a test that catalog.listFiles() is delayed until execution, but this can be delayed until there is an actual benefit to doing so).

Author: Eric Liang <ekl@databricks.com>

Closes #14241 from ericl/refactor.
2016-08-03 11:19:55 -07:00
gatorsmile b73a570603 [SPARK-16858][SQL][TEST] Removal of TestHiveSharedState
### What changes were proposed in this pull request?
This PR is to remove `TestHiveSharedState`.

Also, this is also associated with the Hive refractoring for removing `HiveSharedState`.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14463 from gatorsmile/removeTestHiveSharedState.
2016-08-02 14:17:45 -07:00
jiangxingbo 5184df06b3 [SPARK-16793][SQL] Set the temporary warehouse path to sc'conf in TestHive.
## What changes were proposed in this pull request?

With SPARK-15034, we could use the value of spark.sql.warehouse.dir to set the warehouse location. In TestHive, we can now simply set the temporary warehouse path in sc's conf, and thus, param "warehousePath" could be removed.

## How was this patch tested?

exsiting testsuites.

Author: jiangxingbo <jiangxingbo@meituan.com>

Closes #14401 from jiangxb1987/warehousePath.
2016-08-01 23:08:06 -07:00
Wenchen Fan 301fb0d723 [SPARK-16731][SQL] use StructType in CatalogTable and remove CatalogColumn
## What changes were proposed in this pull request?

`StructField` has very similar semantic with `CatalogColumn`, except that `CatalogColumn` use string to express data type. I think it's reasonable to use `StructType` as the `CatalogTable.schema` and remove `CatalogColumn`.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14363 from cloud-fan/column.
2016-07-31 18:18:53 -07:00
gatorsmile 762366fd87 [SPARK-16552][SQL] Store the Inferred Schemas into External Catalog Tables when Creating Tables
#### What changes were proposed in this pull request?

Currently, in Spark SQL, the initial creation of schema can be classified into two groups. It is applicable to both Hive tables and Data Source tables:

**Group A. Users specify the schema.**

_Case 1 CREATE TABLE AS SELECT_: the schema is determined by the result schema of the SELECT clause. For example,
```SQL
CREATE TABLE tab STORED AS TEXTFILE
AS SELECT * from input
```

_Case 2 CREATE TABLE_: users explicitly specify the schema. For example,
```SQL
CREATE TABLE jsonTable (_1 string, _2 string)
USING org.apache.spark.sql.json
```

**Group B. Spark SQL infers the schema at runtime.**

_Case 3 CREATE TABLE_. Users do not specify the schema but the path to the file location. For example,
```SQL
CREATE TABLE jsonTable
USING org.apache.spark.sql.json
OPTIONS (path '${tempDir.getCanonicalPath}')
```

Before this PR, Spark SQL does not store the inferred schema in the external catalog for the cases in Group B. When users refreshing the metadata cache, accessing the table at the first time after (re-)starting Spark, Spark SQL will infer the schema and store the info in the metadata cache for improving the performance of subsequent metadata requests. However, the runtime schema inference could cause undesirable schema changes after each reboot of Spark.

This PR is to store the inferred schema in the external catalog when creating the table. When users intend to refresh the schema after possible changes on external files (table location), they issue `REFRESH TABLE`. Spark SQL will infer the schema again based on the previously specified table location and update/refresh the schema in the external catalog and metadata cache.

In this PR, we do not use the inferred schema to replace the user specified schema for avoiding external behavior changes . Based on the design, user-specified schemas (as described in Group A) can be changed by ALTER TABLE commands, although we do not support them now.

#### How was this patch tested?
TODO: add more cases to cover the changes.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14207 from gatorsmile/userSpecifiedSchema.
2016-07-28 17:29:26 +08:00
Dongjoon Hyun 5c2ae79bfc [SPARK-15232][SQL] Add subquery SQL building tests to LogicalPlanToSQLSuite
## What changes were proposed in this pull request?

We currently test subquery SQL building using the `HiveCompatibilitySuite`. The is not desired since SQL building is actually a part of `sql/core` and because we are slowly reducing our dependency on Hive. This PR adds the same tests from the whitelist of `HiveCompatibilitySuite` into `LogicalPlanToSQLSuite`.

## How was this patch tested?

This adds more testcases. Pass the Jenkins tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14383 from dongjoon-hyun/SPARK-15232.
2016-07-27 23:29:26 -07:00
Dongjoon Hyun 5b8e848bbf [SPARK-16621][SQL] Generate stable SQLs in SQLBuilder
## What changes were proposed in this pull request?

Currently, the generated SQLs have not-stable IDs for generated attributes.
The stable generated SQL will give more benefit for understanding or testing the queries.
This PR provides stable SQL generation by the followings.

 - Provide unique ids for generated subqueries, `gen_subquery_xxx`.
 - Provide unique and stable ids for generated attributes, `gen_attr_xxx`.

**Before**
```scala
scala> new org.apache.spark.sql.catalyst.SQLBuilder(sql("select 1")).toSQL
res0: String = SELECT `gen_attr_0` AS `1` FROM (SELECT 1 AS `gen_attr_0`) AS gen_subquery_0
scala> new org.apache.spark.sql.catalyst.SQLBuilder(sql("select 1")).toSQL
res1: String = SELECT `gen_attr_4` AS `1` FROM (SELECT 1 AS `gen_attr_4`) AS gen_subquery_0
```

**After**
```scala
scala> new org.apache.spark.sql.catalyst.SQLBuilder(sql("select 1")).toSQL
res1: String = SELECT `gen_attr_0` AS `1` FROM (SELECT 1 AS `gen_attr_0`) AS gen_subquery_0
scala> new org.apache.spark.sql.catalyst.SQLBuilder(sql("select 1")).toSQL
res2: String = SELECT `gen_attr_0` AS `1` FROM (SELECT 1 AS `gen_attr_0`) AS gen_subquery_0
```

## How was this patch tested?

Pass the existing Jenkins tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14257 from dongjoon-hyun/SPARK-16621.
2016-07-27 13:23:59 +08:00
Wenchen Fan a2abb583ca [SPARK-16663][SQL] desc table should be consistent between data source and hive serde tables
## What changes were proposed in this pull request?

Currently there are 2 inconsistence:

1. for data source table, we only print partition names, for hive table, we also print partition schema. After this PR, we will always print schema
2. if column doesn't have comment, data source table will print empty string, hive table will print null. After this PR, we will always print null

## How was this patch tested?

new test in `HiveDDLSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14302 from cloud-fan/minor3.
2016-07-26 18:46:12 +08:00
Yin Huai 815f3eece5 [SPARK-16633][SPARK-16642][SPARK-16721][SQL] Fixes three issues related to lead and lag functions
## What changes were proposed in this pull request?
This PR contains three changes.

First, this PR changes the behavior of lead/lag back to Spark 1.6's behavior, which is described as below:
1. lead/lag respect null input values, which means that if the offset row exists and the input value is null, the result will be null instead of the default value.
2. If the offset row does not exist, the default value will be used.
3. OffsetWindowFunction's nullable setting also considers the nullability of its input (because of the first change).

Second, this PR fixes the evaluation of lead/lag when the input expression is a literal. This fix is a result of the first change. In current master, if a literal is used as the input expression of a lead or lag function, the result will be this literal even if the offset row does not exist.

Third, this PR makes ResolveWindowFrame not fire if a window function is not resolved.

## How was this patch tested?
New tests in SQLWindowFunctionSuite

Author: Yin Huai <yhuai@databricks.com>

Closes #14284 from yhuai/lead-lag.
2016-07-25 20:58:07 -07:00
Dongjoon Hyun 8a8d26f1e2 [SPARK-16672][SQL] SQLBuilder should not raise exceptions on EXISTS queries
## What changes were proposed in this pull request?

Currently, `SQLBuilder` raises `empty.reduceLeft` exceptions on *unoptimized* `EXISTS` queries. We had better prevent this.
```scala
scala> sql("CREATE TABLE t1(a int)")
scala> val df = sql("select * from t1 b where exists (select * from t1 a)")
scala> new org.apache.spark.sql.catalyst.SQLBuilder(df).toSQL
java.lang.UnsupportedOperationException: empty.reduceLeft
```

## How was this patch tested?

Pass the Jenkins tests with a new test suite.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14307 from dongjoon-hyun/SPARK-16672.
2016-07-25 19:52:17 -07:00
gatorsmile 3fc4566941 [SPARK-16678][SPARK-16677][SQL] Fix two View-related bugs
## What changes were proposed in this pull request?
**Issue 1: Disallow Creating/Altering a View when the same-name Table Exists (without IF NOT EXISTS)**
When we create OR alter a view, we check whether the view already exists. In the current implementation, if a table with the same name exists, we treat it as a view. However, this is not the right behavior. We should follow what Hive does. For example,
```
hive> CREATE TABLE tab1 (id int);
OK
Time taken: 0.196 seconds
hive> CREATE OR REPLACE VIEW tab1 AS SELECT * FROM t1;
FAILED: SemanticException [Error 10218]: Existing table is not a view
 The following is an existing table, not a view: default.tab1
hive> ALTER VIEW tab1 AS SELECT * FROM t1;
FAILED: SemanticException [Error 10218]: Existing table is not a view
 The following is an existing table, not a view: default.tab1
hive> CREATE VIEW IF NOT EXISTS tab1 AS SELECT * FROM t1;
OK
Time taken: 0.678 seconds
```

**Issue 2: Strange Error when Issuing Load Table Against A View**
Users should not be allowed to issue LOAD DATA against a view. Currently, when users doing it, we got a very strange runtime error. For example,
```SQL
LOAD DATA LOCAL INPATH "$testData" INTO TABLE $viewName
```
```
java.lang.reflect.InvocationTargetException was thrown.
java.lang.reflect.InvocationTargetException
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:606)
	at org.apache.spark.sql.hive.client.Shim_v0_14.loadTable(HiveShim.scala:680)
```
## How was this patch tested?
Added test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14314 from gatorsmile/tableDDLAgainstView.
2016-07-26 09:32:29 +08:00
Cheng Lian 7ea6d282b9 [SPARK-16703][SQL] Remove extra whitespace in SQL generation for window functions
## What changes were proposed in this pull request?

This PR fixes a minor formatting issue of `WindowSpecDefinition.sql` when no partitioning expressions are present.

Before:

```sql
( ORDER BY `a` ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)
```

After:

```sql
(ORDER BY `a` ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)
```

## How was this patch tested?

New test case added in `ExpressionSQLBuilderSuite`.

Author: Cheng Lian <lian@databricks.com>

Closes #14334 from liancheng/window-spec-sql-format.
2016-07-25 09:42:39 -07:00
Wenchen Fan 64529b186a [SPARK-16691][SQL] move BucketSpec to catalyst module and use it in CatalogTable
## What changes were proposed in this pull request?

It's weird that we have `BucketSpec` to abstract bucket info, but don't use it in `CatalogTable`. This PR moves `BucketSpec` into catalyst module.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14331 from cloud-fan/check.
2016-07-25 22:05:48 +08:00
Wenchen Fan d27d362eba [SPARK-16660][SQL] CreateViewCommand should not take CatalogTable
## What changes were proposed in this pull request?

`CreateViewCommand` only needs some information of a `CatalogTable`, but not all of them. We have some tricks(e.g. we need to check the table type is `VIEW`, we need to make `CatalogColumn.dataType` nullable) to allow it to take a `CatalogTable`.
This PR cleans it up and only pass in necessary information to `CreateViewCommand`.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14297 from cloud-fan/minor2.
2016-07-25 22:02:00 +08:00
Cheng Lian 68b4020d0c [SPARK-16648][SQL] Make ignoreNullsExpr a child expression of First and Last
## What changes were proposed in this pull request?

Default `TreeNode.withNewChildren` implementation doesn't work for `Last` and when both constructor arguments are the same, e.g.:

```sql
LAST_VALUE(FALSE) -- The 2nd argument defaults to FALSE
LAST_VALUE(FALSE, FALSE)
LAST_VALUE(TRUE, TRUE)
```

This is because although `Last` is a unary expression, both of its constructor arguments, `child` and `ignoreNullsExpr`, are `Expression`s. When they have the same value, `TreeNode.withNewChildren` treats both of them as child nodes by mistake. `First` is also affected by this issue in exactly the same way.

This PR fixes this issue by making `ignoreNullsExpr` a child expression of `First` and `Last`.

## How was this patch tested?

New test case added in `WindowQuerySuite`.

Author: Cheng Lian <lian@databricks.com>

Closes #14295 from liancheng/spark-16648-last-value.
2016-07-25 17:22:29 +08:00
Wenchen Fan 1221ce0402 [SPARK-16645][SQL] rename CatalogStorageFormat.serdeProperties to properties
## What changes were proposed in this pull request?

we also store data source table options in this field, it's unreasonable to call it `serdeProperties`.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14283 from cloud-fan/minor1.
2016-07-25 09:28:56 +08:00
Liwei Lin d6795c7a25 [SPARK-16515][SQL][FOLLOW-UP] Fix test script on OS X/Windows...
## Problem

The current `sed` in `test_script.sh` is missing a `$`, leading to the failure of `script` test on OS X:
```
== Results ==
!== Correct Answer - 2 ==   == Spark Answer - 2 ==
![x1_y1]                    [x1]
![x2_y2]                    [x2]
```

In addition, this `script` test would also fail on systems like Windows where we couldn't be able to invoke `bash` or `echo | sed`.

## What changes were proposed in this pull request?
This patch
- fixes `sed` in `test_script.sh`
- adds command guards so that the `script` test would pass on systems like Windows

## How was this patch tested?

- Jenkins
- Manually verified tests pass on OS X

Author: Liwei Lin <lwlin7@gmail.com>

Closes #14280 from lw-lin/osx-sed.
2016-07-24 08:35:57 +01:00
Wenchen Fan 86c2752066 [SPARK-16690][TEST] rename SQLTestUtils.withTempTable to withTempView
## What changes were proposed in this pull request?

after https://github.com/apache/spark/pull/12945, we renamed the `registerTempTable` to `createTempView`, as we do create a view actually. This PR renames `SQLTestUtils.withTempTable` to reflect this change.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14318 from cloud-fan/minor4.
2016-07-23 11:39:48 -07:00
Sandeep Singh df2c6d59d0 [SPARK-16287][SQL] Implement str_to_map SQL function
## What changes were proposed in this pull request?
This PR adds `str_to_map` SQL function in order to remove Hive fallback.

## How was this patch tested?
Pass the Jenkins tests with newly added.

Author: Sandeep Singh <sandeep@techaddict.me>

Closes #13990 from techaddict/SPARK-16287.
2016-07-22 10:05:21 +08:00
Cheng Lian e651900bd5 [SPARK-16344][SQL] Decoding Parquet array of struct with a single field named "element"
## What changes were proposed in this pull request?

Due to backward-compatibility reasons, the following Parquet schema is ambiguous:

```
optional group f (LIST) {
  repeated group list {
    optional group element {
      optional int32 element;
    }
  }
}
```

According to the parquet-format spec, when interpreted as a standard 3-level layout, this type is equivalent to the following SQL type:

```
ARRAY<STRUCT<element: INT>>
```

However, when interpreted as a legacy 2-level layout, it's equivalent to

```
ARRAY<STRUCT<element: STRUCT<element: INT>>>
```

Historically, to disambiguate these cases, we employed two methods:

- `ParquetSchemaConverter.isElementType()`

  Used to disambiguate the above cases while converting Parquet types to Spark types.

- `ParquetRowConverter.isElementType()`

  Used to disambiguate the above cases while instantiating row converters that convert Parquet records to Spark rows.

Unfortunately, these two methods make different decision about the above problematic Parquet type, and caused SPARK-16344.

`ParquetRowConverter.isElementType()` is necessary for Spark 1.4 and earlier versions because Parquet requested schemata are directly converted from Spark schemata in these versions. The converted Parquet schemata may be incompatible with actual schemata of the underlying physical files when the files are written by a system/library that uses a schema conversion scheme that is different from Spark when writing Parquet LIST and MAP fields.

In Spark 1.5, Parquet requested schemata are always properly tailored from schemata of physical files to be read. Thus `ParquetRowConverter.isElementType()` is no longer necessary. This PR replaces this method with a simply yet accurate scheme: whenever an ambiguous Parquet type is hit, convert the type in question back to a Spark type using `ParquetSchemaConverter` and check whether it matches the corresponding Spark type.

## How was this patch tested?

New test cases added in `ParquetHiveCompatibilitySuite` and `ParquetQuerySuite`.

Author: Cheng Lian <lian@databricks.com>

Closes #14014 from liancheng/spark-16344-for-master-and-2.0.
2016-07-20 16:49:46 -07:00
Yin Huai 2ae7b88a07 [SPARK-15705][SQL] Change the default value of spark.sql.hive.convertMetastoreOrc to false.
## What changes were proposed in this pull request?
In 2.0, we add a new logic to convert HiveTableScan on ORC tables to Spark's native code path. However, during this conversion, we drop the original metastore schema (https://issues.apache.org/jira/browse/SPARK-15705). Because of this regression, I am changing the default value of `spark.sql.hive.convertMetastoreOrc` to false.

Author: Yin Huai <yhuai@databricks.com>

Closes #14267 from yhuai/SPARK-15705-changeDefaultValue.
2016-07-19 12:58:08 -07:00
Xin Ren 21a6dd2aef [SPARK-16535][BUILD] In pom.xml, remove groupId which is redundant definition and inherited from the parent
https://issues.apache.org/jira/browse/SPARK-16535

## What changes were proposed in this pull request?

When I scan through the pom.xml of sub projects, I found this warning as below and attached screenshot
```
Definition of groupId is redundant, because it's inherited from the parent
```
![screen shot 2016-07-13 at 3 13 11 pm](https://cloud.githubusercontent.com/assets/3925641/16823121/744f893e-4916-11e6-8a52-042f83b9db4e.png)

I've tried to remove some of the lines with groupId definition, and the build on my local machine is still ok.
```
<groupId>org.apache.spark</groupId>
```
As I just find now `<maven.version>3.3.9</maven.version>` is being used in Spark 2.x, and Maven-3 supports versionless parent elements: Maven 3 will remove the need to specify the parent version in sub modules. THIS is great (in Maven 3.1).

ref: http://stackoverflow.com/questions/3157240/maven-3-worth-it/3166762#3166762

## How was this patch tested?

I've tested by re-building the project, and build succeeded.

Author: Xin Ren <iamshrek@126.com>

Closes #14189 from keypointt/SPARK-16535.
2016-07-19 11:59:46 +01:00
Reynold Xin c4524f5193 [HOTFIX] Fix Scala 2.10 compilation 2016-07-18 17:56:36 -07:00
Dongjoon Hyun ea78edb80b [SPARK-16590][SQL] Improve LogicalPlanToSQLSuite to check generated SQL directly
## What changes were proposed in this pull request?

This PR improves `LogicalPlanToSQLSuite` to check the generated SQL directly by **structure**. So far, `LogicalPlanToSQLSuite` relies on  `checkHiveQl` to ensure the **successful SQL generation** and **answer equality**. However, it does not guarantee the generated SQL is the same or will not be changed unnoticeably.

## How was this patch tested?

Pass the Jenkins. This is only a testsuite change.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14235 from dongjoon-hyun/SPARK-16590.
2016-07-18 17:17:37 -07:00
Daoyuan Wang 96e9afaae9 [SPARK-16515][SQL] set default record reader and writer for script transformation
## What changes were proposed in this pull request?
In ScriptInputOutputSchema, we read default RecordReader and RecordWriter from conf. Since Spark 2.0 has deleted those config keys from hive conf, we have to set default reader/writer class name by ourselves. Otherwise we will get None for LazySimpleSerde, the data written would not be able to read by script. The test case added worked fine with previous version of Spark, but would fail now.

## How was this patch tested?
added a test case in SQLQuerySuite.

Closes #14169

Author: Daoyuan Wang <daoyuan.wang@intel.com>
Author: Yin Huai <yhuai@databricks.com>

Closes #14249 from yhuai/scriptTransformation.
2016-07-18 13:58:12 -07:00
Jacek Lewandowski 31ca741aef [SPARK-16528][SQL] Fix NPE problem in HiveClientImpl
## What changes were proposed in this pull request?

There are some calls to methods or fields (getParameters, properties) which are then passed to Java/Scala collection converters. Unfortunately those fields can be null in some cases and then the conversions throws NPE. We fix it by wrapping calls to those fields and methods with option and then do the conversion.

## How was this patch tested?

Manually tested with a custom Hive metastore.

Author: Jacek Lewandowski <lewandowski.jacek@gmail.com>

Closes #14200 from jacek-lewandowski/SPARK-16528.
2016-07-14 10:18:31 -07:00
gatorsmile c5ec879828 [SPARK-16482][SQL] Describe Table Command for Tables Requiring Runtime Inferred Schema
#### What changes were proposed in this pull request?
If we create a table pointing to a parquet/json datasets without specifying the schema, describe table command does not show the schema at all. It only shows `# Schema of this table is inferred at runtime`. In 1.6, describe table does show the schema of such a table.

~~For data source tables, to infer the schema, we need to load the data source tables at runtime. Thus, this PR calls the function `lookupRelation`.~~

For data source tables, we infer the schema before table creation. Thus, this PR set the inferred schema as the table schema when table creation.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14148 from gatorsmile/describeSchema.
2016-07-13 15:23:37 -07:00
petermaxlee 56bd399a86 [SPARK-16284][SQL] Implement reflect SQL function
## What changes were proposed in this pull request?
This patch implements reflect SQL function, which can be used to invoke a Java method in SQL. Slightly different from Hive, this implementation requires the class name and the method name to be literals. This implementation also supports only a smaller number of data types, and requires the function to be static, as suggested by rxin in #13969.

java_method is an alias for reflect, so this should also resolve SPARK-16277.

## How was this patch tested?
Added expression unit tests and an end-to-end test.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #14138 from petermaxlee/reflect-static.
2016-07-13 08:05:20 +08:00
Marcelo Vanzin 7f968867ff [SPARK-16119][SQL] Support PURGE option to drop table / partition.
This option is used by Hive to directly delete the files instead of
moving them to the trash. This is needed in certain configurations
where moving the files does not work. For non-Hive tables and partitions,
Spark already behaves as if the PURGE option was set, so there's no
need to do anything.

Hive support for PURGE was added in 0.14 (for tables) and 1.2 (for
partitions), so the code reflects that: trying to use the option with
older versions of Hive will cause an exception to be thrown.

The change is a little noisier than I would like, because of the code
to propagate the new flag through all the interfaces and implementations;
the main changes are in the parser and in HiveShim, aside from the tests
(DDLCommandSuite, VersionsSuite).

Tested by running sql and catalyst unit tests, plus VersionsSuite which
has been updated to test the version-specific behavior. I also ran an
internal test suite that uses PURGE and would not pass previously.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #13831 from vanzin/SPARK-16119.
2016-07-12 12:47:46 -07:00
Lianhui Wang 5ad68ba5ce [SPARK-15752][SQL] Optimize metadata only query that has an aggregate whose children are deterministic project or filter operators.
## What changes were proposed in this pull request?
when query only use metadata (example: partition key), it can return results based on metadata without scanning files. Hive did it in HIVE-1003.

## How was this patch tested?
add unit tests

Author: Lianhui Wang <lianhuiwang09@gmail.com>
Author: Wenchen Fan <wenchen@databricks.com>
Author: Lianhui Wang <lianhuiwang@users.noreply.github.com>

Closes #13494 from lianhuiwang/metadata-only.
2016-07-12 18:52:15 +02:00
Russell Spitzer b1e5281c5c [SPARK-12639][SQL] Mark Filters Fully Handled By Sources with *
## What changes were proposed in this pull request?

In order to make it clear which filters are fully handled by the
underlying datasource we will mark them with an *. This will give a
clear visual queue to users that the filter is being treated differently
by catalyst than filters which are just presented to the underlying
DataSource.

Examples from the FilteredScanSuite, in this example `c IN (...)` is handled by the source, `b < ...` is not
### Before
```
//SELECT a FROM oneToTenFiltered WHERE a + b > 9 AND b < 16 AND c IN ('bbbbbBBBBB', 'cccccCCCCC', 'dddddDDDDD', 'foo')
== Physical Plan ==
Project [a#0]
+- Filter (((a#0 + b#1) > 9) && (b#1 < 16))
   +- Scan SimpleFilteredScan(1,10)[a#0,b#1] PushedFilters: [LessThan(b,16), In(c, [bbbbbBBBBB,cccccCCCCC,dddddDDDDD,foo]]
```

### After
```
== Physical Plan ==
Project [a#0]
+- Filter (((a#0 + b#1) > 9) && (b#1 < 16))
   +- Scan SimpleFilteredScan(1,10)[a#0,b#1] PushedFilters: [LessThan(b,16), *In(c, [bbbbbBBBBB,cccccCCCCC,dddddDDDDD,foo]]
```

## How was the this patch tested?

Manually tested with the Spark Cassandra Connector, a source which fully handles underlying filters. Now fully handled filters appear with an * next to their names. I can add an automated test as well if requested

Post 1.6.1
Tested by modifying the FilteredScanSuite to run explains.

Author: Russell Spitzer <Russell.Spitzer@gmail.com>

Closes #11317 from RussellSpitzer/SPARK-12639-Star.
2016-07-11 21:40:09 -07:00
Marcelo Vanzin b4fbe140be [SPARK-16349][SQL] Fall back to isolated class loader when classes not found.
Some Hadoop classes needed by the Hive metastore client jars are not present
in Spark's packaging (for example, "org/apache/hadoop/mapred/MRVersion"). So
if the parent class loader fails to find a class, try to load it from the
isolated class loader, in case it's available there.

Tested by setting spark.sql.hive.metastore.jars to local paths with Hive/Hadoop
libraries and verifying that Spark can talk to the metastore.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #14020 from vanzin/SPARK-16349.
2016-07-11 15:20:48 -07:00
Reynold Xin ffcb6e055a [SPARK-16477] Bump master version to 2.1.0-SNAPSHOT
## What changes were proposed in this pull request?
After SPARK-16476 (committed earlier today as #14128), we can finally bump the version number.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #14130 from rxin/SPARK-16477.
2016-07-11 09:42:56 -07:00
Dongjoon Hyun 7ac79da0e4 [SPARK-16459][SQL] Prevent dropping current database
## What changes were proposed in this pull request?

This PR prevents dropping the current database to avoid errors like the followings.

```scala
scala> sql("create database delete_db")
scala> sql("use delete_db")
scala> sql("drop database delete_db")
scala> sql("create table t as select 1")
org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException: Database `delete_db` not found;
```

## How was this patch tested?

Pass the Jenkins tests including an updated testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14115 from dongjoon-hyun/SPARK-16459.
2016-07-11 15:15:47 +02:00
petermaxlee 82f0874453 [SPARK-16318][SQL] Implement all remaining xpath functions
## What changes were proposed in this pull request?
This patch implements all remaining xpath functions that Hive supports and not natively supported in Spark: xpath_int, xpath_short, xpath_long, xpath_float, xpath_double, xpath_string, and xpath.

## How was this patch tested?
Added unit tests and end-to-end tests.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #13991 from petermaxlee/SPARK-16318.
2016-07-11 13:28:34 +08:00
wujian f5fef69143 [SPARK-16281][SQL] Implement parse_url SQL function
## What changes were proposed in this pull request?

This PR adds parse_url SQL functions in order to remove Hive fallback.

A new implementation of #13999

## How was this patch tested?

Pass the exist tests including new testcases.

Author: wujian <jan.chou.wu@gmail.com>

Closes #14008 from janplus/SPARK-16281.
2016-07-08 14:38:05 -07:00
Dongjoon Hyun a54438cb23 [SPARK-16285][SQL] Implement sentences SQL functions
## What changes were proposed in this pull request?

This PR implements `sentences` SQL function.

## How was this patch tested?

Pass the Jenkins tests with a new testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14004 from dongjoon-hyun/SPARK_16285.
2016-07-08 17:05:24 +08:00
Daoyuan Wang 28710b42b0 [SPARK-16415][SQL] fix catalog string error
## What changes were proposed in this pull request?

In #13537 we truncate `simpleString` if it is a long `StructType`. But sometimes we need `catalogString` to reconstruct `TypeInfo`, for example in description of [SPARK-16415 ](https://issues.apache.org/jira/browse/SPARK-16415). So we need to keep the implementation of `catalogString` not affected by our truncate.

## How was this patch tested?

added a test case.

Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #14089 from adrian-wang/catalogstring.
2016-07-07 11:08:06 -07:00
gatorsmile ab05db0b48 [SPARK-16368][SQL] Fix Strange Errors When Creating View With Unmatched Column Num
#### What changes were proposed in this pull request?
When creating a view, a common user error is the number of columns produced by the `SELECT` clause does not match the number of column names specified by `CREATE VIEW`.

For example, given Table `t1` only has 3 columns
```SQL
create view v1(col2, col4, col3, col5) as select * from t1
```
Currently, Spark SQL reports the following error:
```
requirement failed
java.lang.IllegalArgumentException: requirement failed
	at scala.Predef$.require(Predef.scala:212)
	at org.apache.spark.sql.execution.command.CreateViewCommand.run(views.scala:90)
```

This error message is very confusing. This PR is to detect the error and issue a meaningful error message.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14047 from gatorsmile/viewMismatchedColumns.
2016-07-07 00:07:25 -07:00
gatorsmile 42279bff68 [SPARK-16374][SQL] Remove Alias from MetastoreRelation and SimpleCatalogRelation
#### What changes were proposed in this pull request?
Different from the other leaf nodes, `MetastoreRelation` and `SimpleCatalogRelation` have a pre-defined `alias`, which is used to change the qualifier of the node. However, based on the existing alias handling, alias should be put in `SubqueryAlias`.

This PR is to separate alias handling from `MetastoreRelation` and `SimpleCatalogRelation` to make it consistent with the other nodes. It simplifies the signature and conversion to a `BaseRelation`.

For example, below is an example query for `MetastoreRelation`,  which is converted to a `LogicalRelation`:
```SQL
SELECT tmp.a + 1 FROM test_parquet_ctas tmp WHERE tmp.a > 2
```

Before changes, the analyzed plan is
```
== Analyzed Logical Plan ==
(a + 1): int
Project [(a#951 + 1) AS (a + 1)#952]
+- Filter (a#951 > 2)
   +- SubqueryAlias tmp
      +- Relation[a#951] parquet
```
After changes, the analyzed plan becomes
```
== Analyzed Logical Plan ==
(a + 1): int
Project [(a#951 + 1) AS (a + 1)#952]
+- Filter (a#951 > 2)
   +- SubqueryAlias tmp
      +- SubqueryAlias test_parquet_ctas
         +- Relation[a#951] parquet
```

**Note: the optimized plans are the same.**

For `SimpleCatalogRelation`, the existing code always generates two Subqueries. Thus, no change is needed.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14053 from gatorsmile/removeAliasFromMetastoreRelation.
2016-07-07 12:07:19 +08:00
gatorsmile 21eadd1d8c [SPARK-16229][SQL] Drop Empty Table After CREATE TABLE AS SELECT fails
#### What changes were proposed in this pull request?
In `CREATE TABLE AS SELECT`, if the `SELECT` query failed, the table should not exist. For example,

```SQL
CREATE TABLE tab
STORED AS TEXTFILE
SELECT 1 AS a, (SELECT a FROM (SELECT 1 AS a UNION ALL SELECT 2 AS a) t) AS b
```
The above query failed as expected but an empty table `t` is created.

This PR is to drop the created table when hitting any non-fatal exception.

#### How was this patch tested?
Added a test case to verify the behavior

Author: gatorsmile <gatorsmile@gmail.com>

Closes #13926 from gatorsmile/dropTableAfterException.
2016-07-06 21:43:55 +08:00
Reynold Xin 7e28fabdff [SPARK-16388][SQL] Remove spark.sql.nativeView and spark.sql.nativeView.canonical config
## What changes were proposed in this pull request?
These two configs should always be true after Spark 2.0. This patch removes them from the config list. Note that ideally this should've gone into branch-2.0, but due to the timing of the release we should only merge this in master for Spark 2.1.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #14061 from rxin/SPARK-16388.
2016-07-06 17:40:55 +08:00
Tejas Patil 5f342049cc [SPARK-16339][CORE] ScriptTransform does not print stderr when outstream is lost
## What changes were proposed in this pull request?

Currently, if due to some failure, the outstream gets destroyed or closed and later `outstream.close()` leads to IOException in such case. Due to this, the `stderrBuffer` does not get logged and there is no way for users to see why the job failed.

The change is to first display the stderr buffer and then try closing the outstream.

## How was this patch tested?

The correct way to test this fix would be to grep the log to see if the `stderrBuffer` gets logged but I dont think having test cases which do that is a good idea.

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

…

Author: Tejas Patil <tejasp@fb.com>

Closes #13834 from tejasapatil/script_transform.
2016-07-06 09:18:04 +01:00
gatorsmile ec18cd0af4 [SPARK-16389][SQL] Remove MetastoreRelation from SparkHiveWriterContainer and SparkHiveDynamicPartitionWriterContainer
#### What changes were proposed in this pull request?
- Remove useless `MetastoreRelation` from the signature of `SparkHiveWriterContainer` and `SparkHiveDynamicPartitionWriterContainer`.
- Avoid unnecessary metadata retrieval using Hive client in `InsertIntoHiveTable`.

#### How was this patch tested?
Existing test cases already cover it.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14062 from gatorsmile/removeMetastoreRelation.
2016-07-06 12:09:53 +08:00
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