Commit graph

4879 commits

Author SHA1 Message Date
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
sureshthalamati f174cdc747 [SPARK-14536][SQL] fix to handle null value in array type column for postgres.
## What changes were proposed in this pull request?

JDBC  read  is failing with  NPE due to missing null value check for array data type if the source table has null values in the array type column.  For null values Resultset.getArray()  returns null.
This PR adds null safe check to the Resultset.getArray() value before invoking method on the Array object.
## How was this patch tested?

Updated the PostgresIntegration test suite to test null values. Ran docker integration tests on my laptop.

Author: sureshthalamati <suresh.thalamati@gmail.com>

Closes #15192 from sureshthalamati/jdbc_array_null_fix-SPARK-14536.
2017-01-20 19:23:20 -08:00
hyukjinkwon 54268b42dc [SPARK-16101][SQL] Refactoring CSV write path to be consistent with JSON data source
## What changes were proposed in this pull request?

This PR refactors CSV write path to be consistent with JSON data source.

This PR makes the methods in classes have consistent arguments with JSON ones.
  - `UnivocityGenerator` and `JacksonGenerator`

    ``` scala
    private[csv] class UnivocityGenerator(
        schema: StructType,
        writer: Writer,
        options: CSVOptions = new CSVOptions(Map.empty[String, String])) {
    ...

    def write ...
    def close ...
    def flush ...
    ```

    ``` scala
    private[sql] class JacksonGenerator(
       schema: StructType,
       writer: Writer,
       options: JSONOptions = new JSONOptions(Map.empty[String, String])) {
    ...

    def write ...
    def close ...
    def flush ...
    ```

- This PR also makes the classes put in together in a consistent manner with JSON.
  - `CsvFileFormat`

    ``` scala
    CsvFileFormat
    CsvOutputWriter
    ```

  - `JsonFileFormat`

    ``` scala
    JsonFileFormat
    JsonOutputWriter
    ```

## How was this patch tested?

Existing tests should cover this.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16496 from HyukjinKwon/SPARK-16101-write.
2017-01-21 10:43:52 +08:00
Shixiong Zhu ea31f92bb8 [SPARK-19267][SS] Fix a race condition when stopping StateStore
## What changes were proposed in this pull request?

There is a race condition when stopping StateStore which makes `StateStoreSuite.maintenance` flaky. `StateStore.stop` doesn't wait for the running task to finish, and an out-of-date task may fail `doMaintenance` and cancel the new task. Here is a reproducer: dde1b5b106

This PR adds MaintenanceTask to eliminate the race condition.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>
Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #16627 from zsxwing/SPARK-19267.
2017-01-20 17:49:26 -08:00
Davies Liu 9b7a03f15a [SPARK-18589][SQL] Fix Python UDF accessing attributes from both side of join
## What changes were proposed in this pull request?

PythonUDF is unevaluable, which can not be used inside a join condition, currently the optimizer will push a PythonUDF which accessing both side of join into the join condition, then the query will fail to plan.

This PR fix this issue by checking the expression is evaluable  or not before pushing it into Join.

## How was this patch tested?

Add a regression test.

Author: Davies Liu <davies@databricks.com>

Closes #16581 from davies/pyudf_join.
2017-01-20 16:11:40 -08:00
Tathagata Das 552e5f0884 [SPARK-19314][SS][CATALYST] Do not allow sort before aggregation in Structured Streaming plan
## What changes were proposed in this pull request?

Sort in a streaming plan should be allowed only after a aggregation in complete mode. Currently it is incorrectly allowed when present anywhere in the plan. It gives unpredictable potentially incorrect results.

## How was this patch tested?
New test

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

Closes #16662 from tdas/SPARK-19314.
2017-01-20 14:04:51 -08:00
wangzhenhua 039ed9fe8a [SPARK-19271][SQL] Change non-cbo estimation of aggregate
## What changes were proposed in this pull request?

Change non-cbo estimation behavior of aggregate:
- If groupExpression is empty, we can know row count (=1) and the corresponding size;
- otherwise, estimation falls back to UnaryNode's computeStats method, which should not propagate rowCount and attributeStats in Statistics because they are not estimated in that method.

## How was this patch tested?

Added test case

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #16631 from wzhfy/aggNoCbo.
2017-01-19 22:18:47 -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
Kazuaki Ishizaki 148a84b370 [SPARK-17912] [SQL] Refactor code generation to get data for ColumnVector/ColumnarBatch
## What changes were proposed in this pull request?

This PR refactors the code generation part to get data from `ColumnarVector` and `ColumnarBatch` by using a trait `ColumnarBatchScan` for ease of reuse. This is because this part will be reused by several components (e.g. parquet reader, Dataset.cache, and others) since `ColumnarBatch` will be first citizen.

This PR is a part of https://github.com/apache/spark/pull/15219. In advance, this PR makes the code generation for  `ColumnarVector` and `ColumnarBatch` reuseable as a trait. In general, this is very useful for other components from the reuseability view, too.
## How was this patch tested?

tested existing test suites

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #15467 from kiszk/columnarrefactor.
2017-01-19 15:16:05 -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
jayadevanmurali 064fadd2a2 [SPARK-19059][SQL] Unable to retrieve data from parquet table whose name startswith underscore
## What changes were proposed in this pull request?
The initial shouldFilterOut() method invocation filter the root path name(table name in the intial call) and remove if it contains _. I moved the check one level below, so it first list files/directories in the given root path and then apply filter.
(Please fill in changes proposed in this fix)

## How was this patch tested?
Added new test case for this scenario
(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

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

Author: jayadevanmurali <jayadevan.m@tcs.com>
Author: jayadevan <jayadevan.m@tcs.com>

Closes #16635 from jayadevanmurali/branch-0.1-SPARK-19059.
2017-01-19 20:07:52 +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
Liwei Lin 569e50680f [SPARK-19168][STRUCTURED STREAMING] StateStore should be aborted upon error
## What changes were proposed in this pull request?

We should call `StateStore.abort()` when there should be any error before the store is committed.

## How was this patch tested?

Manually.

Author: Liwei Lin <lwlin7@gmail.com>

Closes #16547 from lw-lin/append-filter.
2017-01-18 10:52:47 -08:00
Shixiong Zhu c050c12274 [SPARK-19113][SS][TESTS] Ignore StreamingQueryException thrown from awaitInitialization to avoid breaking tests
## What changes were proposed in this pull request?

#16492 missed one race condition: `StreamExecution.awaitInitialization` may throw fatal errors and fail the test. This PR just ignores `StreamingQueryException` thrown from `awaitInitialization` so that we can verify the exception in the `ExpectFailure` action later. It's fine since `StopStream` or `ExpectFailure` will catch `StreamingQueryException` as well.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16567 from zsxwing/SPARK-19113-2.
2017-01-18 10:50:51 -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
Reynold Xin 83dff87ded [SPARK-18917][SQL] Remove schema check in appending data
## What changes were proposed in this pull request?
In append mode, we check whether the schema of the write is compatible with the schema of the existing data. It can be a significant performance issue in cloud environment to find the existing schema for files. This patch removes the check.

Note that for catalog tables, we always do the check, as discussed in https://github.com/apache/spark/pull/16339#discussion_r96208357

## How was this patch tested?
N/A

Closes #16339.

Author: Reynold Xin <rxin@databricks.com>

Closes #16622 from rxin/SPARK-18917.
2017-01-17 15:06:28 -08:00
jiangxingbo fee20df143 [MINOR][SQL] Remove duplicate call of reset() function in CurrentOrigin.withOrigin()
## What changes were proposed in this pull request?

Remove duplicate call of reset() function in CurrentOrigin.withOrigin().

## How was this patch tested?

Existing test cases.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #16615 from jiangxb1987/dummy-code.
2017-01-17 10:47:46 -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
Shixiong Zhu a83accfcfd [SPARK-19065][SQL] Don't inherit expression id in dropDuplicates
## What changes were proposed in this pull request?

`dropDuplicates` will create an Alias using the same exprId, so `StreamExecution` should also replace Alias if necessary.

## How was this patch tested?

test("SPARK-19065: dropDuplicates should not create expressions using the same id")

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16564 from zsxwing/SPARK-19065.
2017-01-18 01:57:12 +08:00
hyukjinkwon 6c00c069e3
[SPARK-3249][DOC] Fix links in ScalaDoc that cause warning messages in sbt/sbt unidoc
## What changes were proposed in this pull request?

This PR proposes to fix ambiguous link warnings by simply making them as code blocks for both javadoc and scaladoc.

```
[warn] .../spark/core/src/main/scala/org/apache/spark/Accumulator.scala:20: The link target "SparkContext#accumulator" is ambiguous. Several members fit the target:
[warn] .../spark/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala:281: The link target "runMiniBatchSGD" is ambiguous. Several members fit the target:
[warn] .../spark/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala:83: The link target "run" is ambiguous. Several members fit the target:
...
```

This PR also fixes javadoc8 break as below:

```
[error] .../spark/sql/core/target/java/org/apache/spark/sql/LowPrioritySQLImplicits.java:7: error: reference not found
[error]  * newProductEncoder - to disambiguate for {link List}s which are both {link Seq} and {link Product}
[error]                                                   ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/LowPrioritySQLImplicits.java:7: error: reference not found
[error]  * newProductEncoder - to disambiguate for {link List}s which are both {link Seq} and {link Product}
[error]                                                                                ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/LowPrioritySQLImplicits.java:7: error: reference not found
[error]  * newProductEncoder - to disambiguate for {link List}s which are both {link Seq} and {link Product}
[error]                                                                                                ^
[info] 3 errors
```

## How was this patch tested?

Manually via `sbt unidoc > output.txt` and the checked it via `cat output.txt | grep ambiguous`

and `sbt unidoc | grep error`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16604 from HyukjinKwon/SPARK-3249.
2017-01-17 12:28:15 +00: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
Wenchen Fan a774bca05e [SPARK-19240][SQL][TEST] add test for setting location for managed table
## What changes were proposed in this pull request?

SET LOCATION can also work on managed table(or table created without custom path), the behavior is a little weird, but as we have already supported it, we should add a test to explicitly show the behavior.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16597 from cloud-fan/set-location.
2017-01-17 19:42:02 +08:00
Wenchen Fan 18ee55dd5d [SPARK-19148][SQL] do not expose the external table concept in Catalog
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/16296 , we reached a consensus that we should hide the external/managed table concept to users and only expose custom table path.

This PR renames `Catalog.createExternalTable` to `createTable`(still keep the old versions for backward compatibility), and only set the table type to EXTERNAL if `path` is specified in options.

## How was this patch tested?

new tests in `CatalogSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16528 from cloud-fan/create-table.
2017-01-17 12:54:50 +08: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
Liang-Chi Hsieh 61e48f52d1 [SPARK-19082][SQL] Make ignoreCorruptFiles work for Parquet
## What changes were proposed in this pull request?

We have a config `spark.sql.files.ignoreCorruptFiles` which can be used to ignore corrupt files when reading files in SQL. Currently the `ignoreCorruptFiles` config has two issues and can't work for Parquet:

1. We only ignore corrupt files in `FileScanRDD` . Actually, we begin to read those files as early as inferring data schema from the files. For corrupt files, we can't read the schema and fail the program. A related issue reported at http://apache-spark-developers-list.1001551.n3.nabble.com/Skip-Corrupted-Parquet-blocks-footer-tc20418.html
2. In `FileScanRDD`, we assume that we only begin to read the files when starting to consume the iterator. However, it is possibly the files are read before that. In this case, `ignoreCorruptFiles` config doesn't work too.

This patch targets Parquet datasource. If this direction is ok, we can address the same issue for other datasources like Orc.

Two main changes in this patch:

1. Replace `ParquetFileReader.readAllFootersInParallel` by implementing the logic to read footers in multi-threaded manner

    We can't ignore corrupt files if we use `ParquetFileReader.readAllFootersInParallel`. So this patch implements the logic to do the similar thing in `readParquetFootersInParallel`.

2. In `FileScanRDD`, we need to ignore corrupt file too when we call `readFunction` to return iterator.

One thing to notice is:

We read schema from Parquet file's footer. The method to read footer `ParquetFileReader.readFooter` throws `RuntimeException`, instead of `IOException`, if it can't successfully read the footer. Please check out df9d8e4154/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java (L470). So this patch catches `RuntimeException`.  One concern is that it might also shadow other runtime exceptions other than reading corrupt files.

## How was this patch tested?

Jenkins tests.

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

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #16474 from viirya/fix-ignorecorrupted-parquet-files.
2017-01-16 15:26:41 +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
Tsuyoshi Ozawa 9112f31bb8
[SPARK-19207][SQL] LocalSparkSession should use Slf4JLoggerFactory.INSTANCE
## What changes were proposed in this pull request?

Using Slf4JLoggerFactory.INSTANCE instead of creating Slf4JLoggerFactory's object with constructor. It's deprecated.

## How was this patch tested?

With running StateStoreRDDSuite.

Author: Tsuyoshi Ozawa <ozawa@apache.org>

Closes #16570 from oza/SPARK-19207.
2017-01-15 11:11:21 +00: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
Yucai Yu ad0dadaa25 [SPARK-19180] [SQL] the offset of short should be 2 in OffHeapColumn
## What changes were proposed in this pull request?

the offset of short is 4 in OffHeapColumnVector's putShorts, but actually it should be 2.

## How was this patch tested?

unit test

Author: Yucai Yu <yucai.yu@intel.com>

Closes #16555 from yucai/offheap_short.
2017-01-13 13:40:53 -08:00
Andrew Ash b040cef2ed Fix missing close-parens for In filter's toString
Otherwise the open parentheses isn't closed in query plan descriptions of batch scans.

    PushedFilters: [In(COL_A, [1,2,4,6,10,16,219,815], IsNotNull(COL_B), ...

Author: Andrew Ash <andrew@andrewash.com>

Closes #16558 from ash211/patch-9.
2017-01-12 23:14:07 -08:00
Wenchen Fan 6b34e745bb [SPARK-19178][SQL] convert string of large numbers to int should return null
## What changes were proposed in this pull request?

When we convert a string to integral, we will convert that string to `decimal(20, 0)` first, so that we can turn a string with decimal format to truncated integral, e.g. `CAST('1.2' AS int)` will return `1`.

However, this brings problems when we convert a string with large numbers to integral, e.g. `CAST('1234567890123' AS int)` will return `1912276171`, while Hive returns null as we expected.

This is a long standing bug(seems it was there the first day Spark SQL was created), this PR fixes this bug by adding the native support to convert `UTF8String` to integral.

## How was this patch tested?

new regression tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16550 from cloud-fan/string-to-int.
2017-01-12 22:52:34 -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
Takeshi YAMAMURO 5585ed93b0 [SPARK-17237][SQL] Remove backticks in a pivot result schema
## What changes were proposed in this pull request?
Pivoting adds backticks (e.g. 3_count(\`c\`)) in column names and, in some cases,
thes causes analysis exceptions  like;
```
scala> val df = Seq((2, 3, 4), (3, 4, 5)).toDF("a", "x", "y")
scala> df.groupBy("a").pivot("x").agg(count("y"), avg("y")).na.fill(0)
org.apache.spark.sql.AnalysisException: syntax error in attribute name: `3_count(`y`)`;
  at org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute$.e$1(unresolved.scala:134)
  at org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute$.parseAttributeName(unresolved.scala:144)
...
```
So, this pr proposes to remove these backticks from column names.

## How was this patch tested?
Added a test in `DataFrameAggregateSuite`.

Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>

Closes #14812 from maropu/SPARK-17237.
2017-01-12 09:46:53 -08:00
Wenchen Fan 871d266649 [SPARK-18969][SQL] Support grouping by nondeterministic expressions
## What changes were proposed in this pull request?

Currently nondeterministic expressions are allowed in `Aggregate`(see the [comment](https://github.com/apache/spark/blob/v2.0.2/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala#L249-L251)), but the `PullOutNondeterministic` analyzer rule failed to handle `Aggregate`, this PR fixes it.

close https://github.com/apache/spark/pull/16379

There is still one remaining issue: `SELECT a + rand() FROM t GROUP BY a + rand()` is not allowed, because the 2 `rand()` are different(we generate random seed as the default seed for `rand()`). https://issues.apache.org/jira/browse/SPARK-19035 is tracking this issue.

## How was this patch tested?

a new test suite

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16404 from cloud-fan/groupby.
2017-01-12 20:21:04 +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
hyukjinkwon 24100f162d [SPARK-16848][SQL] Check schema validation for user-specified schema in jdbc and table APIs
## What changes were proposed in this pull request?

This PR proposes to throw an exception for both jdbc APIs when user specified schemas are not allowed or useless.

**DataFrameReader.jdbc(...)**

``` scala
spark.read.schema(StructType(Nil)).jdbc(...)
```

**DataFrameReader.table(...)**

```scala
spark.read.schema(StructType(Nil)).table("usrdb.test")
```

## How was this patch tested?

Unit test in `JDBCSuite` and `DataFrameReaderWriterSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #14451 from HyukjinKwon/SPARK-16848.
2017-01-11 21:03:48 -08:00
wangzhenhua 43fa21b3e6 [SPARK-19132][SQL] Add test cases for row size estimation and aggregate estimation
## What changes were proposed in this pull request?

In this pr, we add more test cases for project and aggregate estimation.

## How was this patch tested?

Add test cases.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #16551 from wzhfy/addTests.
2017-01-11 15:00:58 -08:00
Reynold Xin 66fe819ada [SPARK-19149][SQL] Follow-up: simplify cache implementation.
## What changes were proposed in this pull request?
This patch simplifies slightly the logical plan statistics cache implementation, as discussed in https://github.com/apache/spark/pull/16529

## How was this patch tested?
N/A - this has no behavior change.

Author: Reynold Xin <rxin@databricks.com>

Closes #16544 from rxin/SPARK-19149.
2017-01-11 14:25:36 -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
Shixiong Zhu bc6c56e940 [SPARK-19140][SS] Allow update mode for non-aggregation streaming queries
## What changes were proposed in this pull request?

This PR allow update mode for non-aggregation streaming queries. It will be same as the append mode if a query has no aggregations.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16520 from zsxwing/update-without-agg.
2017-01-10 17:58:11 -08:00
Dongjoon Hyun d5b1dc934a [SPARK-19137][SQL] Fix withSQLConf to reset OptionalConfigEntry correctly
## What changes were proposed in this pull request?

`DataStreamReaderWriterSuite` makes test files in source folder like the followings. Interestingly, the root cause is `withSQLConf` fails to reset `OptionalConfigEntry` correctly. In other words, it resets the config into `Some(undefined)`.

```bash
$ git status
Untracked files:
  (use "git add <file>..." to include in what will be committed)

        sql/core/%253Cundefined%253E/
        sql/core/%3Cundefined%3E/
```

## How was this patch tested?

Manual.
```
build/sbt "project sql" test
git status
```

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #16522 from dongjoon-hyun/SPARK-19137.
2017-01-10 10:49:44 -08:00
Shixiong Zhu 3ef183a941
[SPARK-19113][SS][TESTS] Set UncaughtExceptionHandler in onQueryStarted to ensure catching fatal errors during query initialization
## What changes were proposed in this pull request?

StreamTest sets `UncaughtExceptionHandler` after starting the query now. It may not be able to catch fatal errors during query initialization. This PR uses `onQueryStarted` callback to fix it.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16492 from zsxwing/SPARK-19113.
2017-01-10 14:24:45 +00:00