Commit graph

5069 commits

Author SHA1 Message Date
windpiger 6a9a85b84d [SPARK-19329][SQL] Reading from or writing to a datasource table with a non pre-existing location should succeed
## What changes were proposed in this pull request?

when we insert data into a datasource table use `sqlText`, and the table has an not exists location,
this will throw an Exception.

example:

```
spark.sql("create table t(a string, b int) using parquet")
spark.sql("alter table t set location '/xx'")
spark.sql("insert into table t select 'c', 1")
```

Exception:
```
com.google.common.util.concurrent.UncheckedExecutionException: org.apache.spark.sql.AnalysisException: Path does not exist: /xx;
at com.google.common.cache.LocalCache$LocalLoadingCache.getUnchecked(LocalCache.java:4814)
at com.google.common.cache.LocalCache$LocalLoadingCache.apply(LocalCache.java:4830)
at org.apache.spark.sql.hive.HiveMetastoreCatalog.lookupRelation(HiveMetastoreCatalog.scala:122)
at org.apache.spark.sql.hive.HiveSessionCatalog.lookupRelation(HiveSessionCatalog.scala:69)
at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$.org$apache$spark$sql$catalyst$analysis$Analyzer$ResolveRelations$$lookupTableFromCatalog(Analyzer.scala:456)
at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$$anonfun$apply$8.applyOrElse(Analyzer.scala:465)
at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$$anonfun$apply$8.applyOrElse(Analyzer.scala:463)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolveOperators$1.apply(LogicalPlan.scala:61)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolveOperators$1.apply(LogicalPlan.scala:61)
at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperators(LogicalPlan.scala:60)
at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$.apply(Analyzer.scala:463)
at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$.apply(Analyzer.scala:453)
```

As discussed following comments, we should unify the action when we reading from or writing to a datasource table with a non pre-existing locaiton:

1. reading from a datasource table: return 0 rows
2. writing to a datasource table:  write data successfully

## How was this patch tested?
unit test added

Author: windpiger <songjun@outlook.com>

Closes #16672 from windpiger/insertNotExistLocation.
2017-02-15 13:21:48 -08:00
Dongjoon Hyun 59dc26e378 [SPARK-19607][HOTFIX] Finding QueryExecution that matches provided executionId
## What changes were proposed in this pull request?

#16940 adds a test case which does not stop the spark job. It causes many failures of other test cases.

- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.7/2403/consoleFull
- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7/2600/consoleFull

```
[info]   org.apache.spark.SparkException: Only one SparkContext may be running in this JVM (see SPARK-2243). To ignore this error, set spark.driver.allowMultipleContexts = true. The currently running SparkContext was created at:
```

## How was this patch tested?

Pass the Jenkins test.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #16943 from dongjoon-hyun/SPARK-19607-2.
2017-02-15 21:57:49 +01:00
jiangxingbo 3755da76c3 [SPARK-19331][SQL][TESTS] Improve the test coverage of SQLViewSuite
Move `SQLViewSuite` from `sql/hive` to `sql/core`, so we can test the view supports without hive metastore. Also moved the test cases that specified to hive to `HiveSQLViewSuite`.

Improve the test coverage of SQLViewSuite, cover the following cases:
1. view resolution(possibly a referenced table/view have changed after the view creation);
2. handle a view with user specified column names;
3. improve the test cases for a nested view.

Also added a test case for cyclic view reference, which is a known issue that is not fixed yet.

N/A

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #16674 from jiangxb1987/view-test.
2017-02-15 10:47:11 -08:00
Felix Cheung 671bc08ed5 [SPARK-19399][SPARKR] Add R coalesce API for DataFrame and Column
## What changes were proposed in this pull request?

Add coalesce on DataFrame for down partitioning without shuffle and coalesce on Column

## How was this patch tested?

manual, unit tests

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #16739 from felixcheung/rcoalesce.
2017-02-15 10:45:37 -08:00
Liang-Chi Hsieh acf71c63cd [SPARK-16475][SQL] broadcast hint for SQL queries - disallow space as the delimiter
## What changes were proposed in this pull request?

A follow-up to disallow space as the delimiter in broadcast hint.

## How was this patch tested?

Jenkins test.

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

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

Closes #16941 from viirya/disallow-space-delimiter.
2017-02-15 18:48:02 +01:00
Dilip Biswal a8a139820c [SPARK-18872][SQL][TESTS] New test cases for EXISTS subquery (Joins + CTE)
## What changes were proposed in this pull request?

This PR adds the third and final set of tests for EXISTS subquery.

File name                        | Brief description
------------------------| -----------------
exists-cte.sql              |Tests Exist subqueries referencing CTE
exists-joins-and-set-ops.sql|Tests Exists subquery used in Joins (Both when joins occurs in outer and suquery blocks)

DB2 results are attached here as reference :

[exists-cte-db2.txt](https://github.com/apache/spark/files/752091/exists-cte-db2.txt)
[exists-joins-and-set-ops-db2.txt](https://github.com/apache/spark/files/753283/exists-joins-and-set-ops-db2.txt) (updated)

## How was this patch tested?
The test result is compared with the result run from another SQL engine (in this case is IBM DB2). If the result are equivalent, we assume the result is correct.

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

Closes #16802 from dilipbiswal/exists-pr3.
2017-02-15 17:34:05 +01:00
Nattavut Sutyanyong 5ad10c5310 [SPARK-18873][SQL][TEST] New test cases for scalar subquery (part 2 of 2) - scalar subquery in predicate context
## What changes were proposed in this pull request?
This PR adds new test cases for scalar subquery in predicate context

## How was this patch tested?
The test result is compared with the result run from another SQL engine (in this case is IBM DB2). If the result are equivalent, we assume the result is correct.

Author: Nattavut Sutyanyong <nsy.can@gmail.com>

Closes #16798 from nsyca/18873-2.
2017-02-15 17:30:55 +01:00
Kevin Yu d22db62785 [SPARK-18871][SQL][TESTS] New test cases for IN/NOT IN subquery 2nd batch
## What changes were proposed in this pull request?

This is 2nd batch of test case for IN/NOT IN subquery.  In this PR, it has these test cases:
`in-limit.sql`
`in-order-by.sql`
`not-in-group-by.sql`

These are the queries and results from running on DB2.
[in-limit DB2 version](https://github.com/apache/spark/files/743267/in-limit.sql.db2.out.txt)
[in-order-by DB2 version](https://github.com/apache/spark/files/743269/in-order-by.sql.db2.txt)
[not-in-group-by DB2 version](https://github.com/apache/spark/files/743271/not-in-group-by.sql.db2.txt)
[output of in-limit.sql DB2](https://github.com/apache/spark/files/743276/in-limit.sql.db2.out.txt)
[output of in-order-by.sql DB2](https://github.com/apache/spark/files/743278/in-order-by.sql.db2.out.txt)
[output of not-in-group-by.sql DB2](https://github.com/apache/spark/files/743279/not-in-group-by.sql.db2.out.txt)

## How was this patch tested?

This pr is adding new test cases.

Author: Kevin Yu <qyu@us.ibm.com>

Closes #16759 from kevinyu98/spark-18871-2.
2017-02-15 17:28:42 +01:00
Zhenhua Wang 601b9c3e68 [SPARK-17076][SQL] Cardinality estimation for join based on basic column statistics
## What changes were proposed in this pull request?

Support cardinality estimation and stats propagation for all join types.

Limitations:
- For inner/outer joins without any equal condition, we estimate it like cartesian product.
- For left semi/anti joins, since we can't apply the heuristics for inner join to it, for now we just propagate the statistics from left side. We should support them when other advanced stats (e.g. histograms) are available in spark.

## How was this patch tested?

Add a new test suite.

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

Closes #16228 from wzhfy/joinEstimate.
2017-02-15 08:21:51 -08:00
Wenchen Fan 8b75f8c1c9 [SPARK-19587][SQL] bucket sorting columns should not be picked from partition columns
## What changes were proposed in this pull request?

We will throw an exception if bucket columns are part of partition columns, this should also apply to sort columns.

This PR also move the checking logic from `DataFrameWriter` to `PreprocessTableCreation`, which is the central place for checking and normailization.

## How was this patch tested?

updated test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16931 from cloud-fan/bucket.
2017-02-15 08:15:03 -08:00
Reynold Xin 733c59ec1e [SPARK-16475][SQL] broadcast hint for SQL queries - follow up
## What changes were proposed in this pull request?
A small update to https://github.com/apache/spark/pull/16925

1. Rename SubstituteHints -> ResolveHints to be more consistent with rest of the rules.
2. Added more documentation in the rule and be more defensive / future proof to skip views as well as CTEs.

## How was this patch tested?
This pull request contains no real logic change and all behavior should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #16939 from rxin/SPARK-16475.
2017-02-15 17:10:49 +01:00
Ala Luszczak b55563c17e [SPARK-19607] Finding QueryExecution that matches provided executionId
## What changes were proposed in this pull request?

Implementing a mapping between executionId and corresponding QueryExecution in SQLExecution.

## How was this patch tested?

Adds a unit test.

Author: Ala Luszczak <ala@databricks.com>

Closes #16940 from ala/execution-id.
2017-02-15 17:06:04 +01:00
sureshthalamati f48c5a57d6 [SPARK-19318][SQL] Fix to treat JDBC connection properties specified by the user in case-sensitive manner.
## What changes were proposed in this pull request?
The reason for test failure is that the property “oracle.jdbc.mapDateToTimestamp” set by the test was getting converted into all lower case. Oracle database expects this property in case-sensitive manner.

This test was passing in previous releases because connection properties were sent as user specified for the test case scenario. Fixes to handle all option uniformly in case-insensitive manner, converted the JDBC connection properties also to lower case.

This PR  enhances CaseInsensitiveMap to keep track of input case-sensitive keys , and uses those when creating connection properties that are passed to the JDBC connection.

Alternative approach PR https://github.com/apache/spark/pull/16847  is to pass original input keys to JDBC data source by adding check in the  Data source class and handle case-insensitivity in the JDBC source code.

## How was this patch tested?
Added new test cases to JdbcSuite , and OracleIntegrationSuite. Ran docker integration tests passed on my laptop, all tests passed successfully.

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

Closes #16891 from sureshthalamati/jdbc_case_senstivity_props_fix-SPARK-19318.
2017-02-14 15:34:12 -08:00
Reynold Xin da7aef7a0e [SPARK-16475][SQL] Broadcast hint for SQL Queries
## What changes were proposed in this pull request?
This pull request introduces a simple hint infrastructure to SQL and implements broadcast join hint using the infrastructure.

The hint syntax looks like the following:
```
SELECT /*+ BROADCAST(t) */ * FROM t
```

For broadcast hint, we accept "BROADCAST", "BROADCASTJOIN", and "MAPJOIN", and a sequence of relation aliases can be specified in the hint. A broadcast hint plan node will be inserted on top of any relation (that is not aliased differently), subquery, or common table expression that match the specified name.

The hint resolution works by recursively traversing down the query plan to find a relation or subquery that matches one of the specified broadcast aliases. The traversal does not go past beyond any existing broadcast hints, subquery aliases. This rule happens before common table expressions.

Note that there was an earlier patch in https://github.com/apache/spark/pull/14426. This is a rewrite of that patch, with different semantics and simpler test cases.

## How was this patch tested?
Added a new unit test suite for the broadcast hint rule (SubstituteHintsSuite) and new test cases for parser change (in PlanParserSuite). Also added end-to-end test case in BroadcastSuite.

Author: Reynold Xin <rxin@databricks.com>
Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #16925 from rxin/SPARK-16475-broadcast-hint.
2017-02-14 14:11:17 -08:00
Xiao Li 457850e6f5 [SPARK-19589][SQL] Removal of SQLGEN files
### What changes were proposed in this pull request?
SQLGen is removed. Thus, the generated files should be removed too.

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #16921 from gatorsmile/removeSQLGenFiles.
2017-02-14 09:35:08 +01:00
Xin Wu 1ab97310e8 [SPARK-19539][SQL] Block duplicate temp table during creation
## What changes were proposed in this pull request?
Current `CREATE TEMPORARY TABLE ... ` is deprecated and recommend users to use `CREATE TEMPORARY VIEW ...` And it does not support `IF NOT EXISTS `clause. However, if there is an existing temporary view defined, it is possible to unintentionally replace this existing view by issuing `CREATE TEMPORARY TABLE ...`  with the same table/view name.

This PR is to disallow `CREATE TEMPORARY TABLE ...` with an existing view name.
Under the cover, `CREATE TEMPORARY TABLE ...` will be changed to create temporary view, however, passing in a flag `replace=false`, instead of currently `true`. So when creating temporary view under the cover, if there is existing view with the same name, the operation will be blocked.

## How was this patch tested?
New unit test case is added and updated some existing test cases to adapt the new behavior

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

Closes #16878 from xwu0226/block_duplicate_temp_table.
2017-02-13 19:45:58 -08:00
ouyangxiaochen 6e45b547ce [SPARK-19115][SQL] Supporting Create Table Like Location
What changes were proposed in this pull request?

Support CREATE [EXTERNAL] TABLE LIKE LOCATION... syntax for Hive serde and datasource tables.
In this PR,we follow SparkSQL design rules :

    supporting create table like view or physical table or temporary view with location.
    creating a table with location,this table will be an external table other than managed table.

How was this patch tested?

Add new test cases and update existing test cases

Author: ouyangxiaochen <ou.yangxiaochen@zte.com.cn>

Closes #16868 from ouyangxiaochen/spark19115.
2017-02-13 19:41:44 -08:00
hyukjinkwon 9af8f743b0 [SPARK-19435][SQL] Type coercion between ArrayTypes
## What changes were proposed in this pull request?

This PR proposes to support type coercion between `ArrayType`s where the element types are compatible.

**Before**

```
Seq(Array(1)).toDF("a").selectExpr("greatest(a, array(1D))")
org.apache.spark.sql.AnalysisException: cannot resolve 'greatest(`a`, array(1.0D))' due to data type mismatch: The expressions should all have the same type, got GREATEST(array<int>, array<double>).; line 1 pos 0;

Seq(Array(1)).toDF("a").selectExpr("least(a, array(1D))")
org.apache.spark.sql.AnalysisException: cannot resolve 'least(`a`, array(1.0D))' due to data type mismatch: The expressions should all have the same type, got LEAST(array<int>, array<double>).; line 1 pos 0;

sql("SELECT * FROM values (array(0)), (array(1D)) as data(a)")
org.apache.spark.sql.AnalysisException: incompatible types found in column a for inline table; line 1 pos 14

Seq(Array(1)).toDF("a").union(Seq(Array(1D)).toDF("b"))
org.apache.spark.sql.AnalysisException: Union can only be performed on tables with the compatible column types. ArrayType(DoubleType,false) <> ArrayType(IntegerType,false) at the first column of the second table;;

sql("SELECT IF(1=1, array(1), array(1D))")
org.apache.spark.sql.AnalysisException: cannot resolve '(IF((1 = 1), array(1), array(1.0D)))' due to data type mismatch: differing types in '(IF((1 = 1), array(1), array(1.0D)))' (array<int> and array<double>).; line 1 pos 7;
```

**After**

```scala
Seq(Array(1)).toDF("a").selectExpr("greatest(a, array(1D))")
res5: org.apache.spark.sql.DataFrame = [greatest(a, array(1.0)): array<double>]

Seq(Array(1)).toDF("a").selectExpr("least(a, array(1D))")
res6: org.apache.spark.sql.DataFrame = [least(a, array(1.0)): array<double>]

sql("SELECT * FROM values (array(0)), (array(1D)) as data(a)")
res8: org.apache.spark.sql.DataFrame = [a: array<double>]

Seq(Array(1)).toDF("a").union(Seq(Array(1D)).toDF("b"))
res10: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [a: array<double>]

sql("SELECT IF(1=1, array(1), array(1D))")
res15: org.apache.spark.sql.DataFrame = [(IF((1 = 1), array(1), array(1.0))): array<double>]
```

## How was this patch tested?

Unit tests in `TypeCoercion` and Jenkins tests and

building with scala 2.10

```scala
./dev/change-scala-version.sh 2.10
./build/mvn -Pyarn -Phadoop-2.4 -Dscala-2.10 -DskipTests clean package
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16777 from HyukjinKwon/SPARK-19435.
2017-02-13 13:10:57 -08:00
Shixiong Zhu 3dbff9be06 [SPARK-19542][SS] Delete the temp checkpoint if a query is stopped without errors
## What changes were proposed in this pull request?

When a query uses a temp checkpoint dir, it's better to delete it if it's stopped without errors.

## How was this patch tested?

New unit tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16880 from zsxwing/delete-temp-checkpoint.
2017-02-13 11:54:54 -08:00
Ala Luszczak 0417ce8787 [SPARK-19514] Enhancing the test for Range interruption.
Improve the test for SPARK-19514, so that it's clear which stage is being cancelled.

Author: Ala Luszczak <ala@databricks.com>

Closes #16914 from ala/fix-range-test.
2017-02-13 20:07:39 +01:00
hyukjinkwon 4321ff9edd [SPARK-19544][SQL] Improve error message when some column types are compatible and others are not in set operations
## What changes were proposed in this pull request?

This PR proposes to fix the error message when some data types are compatible and others are not in set/union operation.

Currently, the code below:

```scala
Seq((1,("a", 1))).toDF.union(Seq((1L,("a", "b"))).toDF)
```

throws an exception saying `LongType` and `IntegerType` are incompatible types. It should say something about `StructType`s with more readable format as below:

**Before**

```
Union can only be performed on tables with the compatible column types.
LongType <> IntegerType at the first column of the second table;;
```

**After**

```
Union can only be performed on tables with the compatible column types.
struct<_1:string,_2:string> <> struct<_1:string,_2:int> at the second column of the second table;;
```

*I manually inserted a newline in the messages above for readability only in this PR description.

## How was this patch tested?

Unit tests in `AnalysisErrorSuite`, manual tests and build wth Scala 2.10.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16882 from HyukjinKwon/SPARK-19544.
2017-02-13 16:08:31 +01:00
windpiger 04ad822534 [SPARK-19496][SQL] to_date udf to return null when input date is invalid
## What changes were proposed in this pull request?

Currently the udf  `to_date` has different return value with an invalid date input.

```
SELECT to_date('2015-07-22', 'yyyy-dd-MM') ->  return `2016-10-07`
SELECT to_date('2014-31-12')    -> return null
```

As discussed in JIRA [SPARK-19496](https://issues.apache.org/jira/browse/SPARK-19496), we should return null in both situations when the input date is invalid

## How was this patch tested?
unit test added

Author: windpiger <songjun@outlook.com>

Closes #16870 from windpiger/to_date.
2017-02-13 12:25:13 +01:00
windpiger 3881f342b4 [SPARK-19448][SQL] optimize some duplication functions between HiveClientImpl and HiveUtils
## What changes were proposed in this pull request?

There are some duplicate functions between `HiveClientImpl` and `HiveUtils`, we can merge them to one place. such as: `toHiveTable` 、`toHivePartition`、`fromHivePartition`.

And additional modify is change `MetastoreRelation.attributes` to `MetastoreRelation.dataColKeys`
https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala#L234

## How was this patch tested?
N/A

Author: windpiger <songjun@outlook.com>

Closes #16787 from windpiger/todoInMetaStoreRelation.
2017-02-11 22:21:14 -08:00
Herman van Hovell 226d38840c [SPARK-19548][SQL] Support Hive UDFs which return typed Lists/Maps
## What changes were proposed in this pull request?
This PR adds support for Hive UDFs that return fully typed java Lists or Maps, for example `List<String>` or `Map<String, Integer>`.  It is also allowed to nest these structures, for example `Map<String, List<Integer>>`. Raw collections or collections using wildcards are still not supported, and cannot be supported due to the lack of type information.

## How was this patch tested?
Modified existing tests in `HiveUDFSuite`, and I have added test cases for raw collection and collection using wildcards.

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

Closes #16886 from hvanhovell/SPARK-19548.
2017-02-10 14:47:25 -08:00
Ala Luszczak d785217b79 [SPARK-19549] Allow providing reason for stage/job cancelling
## What changes were proposed in this pull request?

This change add an optional argument to `SparkContext.cancelStage()` and `SparkContext.cancelJob()` functions, which allows the caller to provide exact reason  for the cancellation.

## How was this patch tested?

Adds unit test.

Author: Ala Luszczak <ala@databricks.com>

Closes #16887 from ala/cancel.
2017-02-10 21:10:02 +01:00
Herman van Hovell de8a03e682 [SPARK-19459][SQL] Add Hive datatype (char/varchar) to StructField metadata
## What changes were proposed in this pull request?
Reading from an existing ORC table which contains `char` or `varchar` columns can fail with a `ClassCastException` if the table metadata has been created using Spark. This is caused by the fact that spark internally replaces `char` and `varchar` columns with a `string` column.

This PR fixes this by adding the hive type to the `StructField's` metadata under the `HIVE_TYPE_STRING` key. This is picked up by the `HiveClient` and the ORC reader, see https://github.com/apache/spark/pull/16060 for more details on how the metadata is used.

## How was this patch tested?
Added a regression test to `OrcSourceSuite`.

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

Closes #16804 from hvanhovell/SPARK-19459.
2017-02-10 11:06:57 -08:00
Burak Yavuz d5593f7f57 [SPARK-19543] from_json fails when the input row is empty
## What changes were proposed in this pull request?

Using from_json on a column with an empty string results in: java.util.NoSuchElementException: head of empty list.

This is because `parser.parse(input)` may return `Nil` when `input.trim.isEmpty`

## How was this patch tested?

Regression test in `JsonExpressionsSuite`

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #16881 from brkyvz/json-fix.
2017-02-10 12:55:06 +01:00
jiangxingbo af63c52fd3 [SPARK-19025][SQL] Remove SQL builder for operators
## What changes were proposed in this pull request?

With the new approach of view resolution, we can get rid of SQL generation on view creation, so let's remove SQL builder for operators.

Note that, since all sql generation for operators is defined in one file (org.apache.spark.sql.catalyst.SQLBuilder), it’d be trivial to recover it in the future.

## How was this patch tested?

N/A

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #16869 from jiangxb1987/SQLBuilder.
2017-02-09 19:35:39 +01:00
Bogdan Raducanu 1af0dee418 [SPARK-19512][SQL] codegen for compare structs fails
## What changes were proposed in this pull request?

Set currentVars to null in GenerateOrdering.genComparisons before genCode is called. genCode ignores INPUT_ROW if currentVars is not null and in genComparisons we want it to use INPUT_ROW.

## How was this patch tested?

Added test with 2 queries in WholeStageCodegenSuite

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

Closes #16852 from bogdanrdc/SPARK-19512.
2017-02-09 19:15:11 +01:00
Ala Luszczak 4064574d03 [SPARK-19514] Making range interruptible.
## What changes were proposed in this pull request?

Previously range operator could not be interrupted. For example, using DAGScheduler.cancelStage(...) on a query with range might have been ineffective.

This change adds periodic checks of TaskContext.isInterrupted to codegen version, and InterruptibleOperator to non-codegen version.

I benchmarked the performance of codegen version on a sample query `spark.range(1000L * 1000 * 1000 * 10).count()` and there is no measurable difference.

## How was this patch tested?

Adds a unit test.

Author: Ala Luszczak <ala@databricks.com>

Closes #16872 from ala/SPARK-19514b.
2017-02-09 19:07:06 +01:00
Liwei Lin 9d9d67c795 [SPARK-19265][SQL][FOLLOW-UP] Configurable tableRelationCache maximum size
## What changes were proposed in this pull request?

SPARK-19265 had made table relation cache general; this follow-up aims to make `tableRelationCache`'s maximum size configurable.

In order to do sanity-check, this patch also adds a `checkValue()` method to `TypedConfigBuilder`.

## How was this patch tested?

new test case: `test("conf entry: checkValue()")`

Author: Liwei Lin <lwlin7@gmail.com>

Closes #16736 from lw-lin/conf.
2017-02-09 00:48:47 -05:00
Wenchen Fan 50a991264c [SPARK-19359][SQL] renaming partition should not leave useless directories
## What changes were proposed in this pull request?

Hive metastore is not case-preserving and keep partition columns with lower case names. If Spark SQL creates a table with upper-case partition column names using `HiveExternalCatalog`, when we rename partition, it first calls the HiveClient to renamePartition, which will create a new lower case partition path, then Spark SQL renames the lower case path to upper-case.

However, when we rename a nested path, different file systems have different behaviors. e.g. in jenkins, renaming `a=1/b=2` to `A=2/B=2` will success, but leave an empty directory `a=1`. in mac os, the renaming doesn't work as expected and result to `a=1/B=2`.

This PR renames the partition directory recursively from the first partition column in `HiveExternalCatalog`, to be most compatible with different file systems.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16837 from cloud-fan/partition.
2017-02-09 00:39:22 -05:00
Dilip Biswal 64cae22f7c [SPARK-18872][SQL][TESTS] New test cases for EXISTS subquery (Aggregate, Having, Orderby, Limit)
## What changes were proposed in this pull request?
This PR adds the second set of tests for EXISTS subquery.

File name                        | Brief description
------------------------| -----------------
exists-aggregate.sql              |Tests aggregate expressions in outer query and EXISTS subquery.
exists-having.sql|Tests HAVING clause in subquery.
exists-orderby-limit.sql|Tests EXISTS subquery support with ORDER BY and LIMIT clauses.

DB2 results are attached here as reference :

[exists-aggregate-db2.txt](https://github.com/apache/spark/files/743287/exists-aggregate-db2.txt)
[exists-having-db2.txt](https://github.com/apache/spark/files/743286/exists-having-db2.txt)
[exists-orderby-limit-db2.txt](https://github.com/apache/spark/files/743288/exists-orderby-limit-db2.txt)

##  How the patch was tested.
The test result is compared with the result run from another SQL engine (in this case is IBM DB2). If the result are equivalent, we assume the result is correct.

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

Closes #16760 from dilipbiswal/exists-pr2.
2017-02-09 00:31:51 -05:00
gatorsmile 4d4d0de7f6 [SPARK-19279][SQL][FOLLOW-UP] Infer Schema for Hive Serde Tables
### What changes were proposed in this pull request?
`table.schema` is always not empty for partitioned tables, because `table.schema` also contains the partitioned columns, even if the original table does not have any column. This PR is to fix the issue.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16848 from gatorsmile/inferHiveSerdeSchema.
2017-02-08 10:11:44 -05:00
Dongjoon Hyun 0077bfcb93
[SPARK-19409][BUILD][TEST-MAVEN] Fix ParquetAvroCompatibilitySuite failure due to test dependency on avro
## What changes were proposed in this pull request?

After using Apache Parquet 1.8.2, `ParquetAvroCompatibilitySuite` fails on **Maven** test. It is because `org.apache.parquet.avro.AvroParquetWriter` in the test code used new `avro 1.8.0` specific class, `LogicalType`. This PR aims to fix the test dependency of `sql/core` module to use avro 1.8.0.

https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7/2530/consoleFull

```
ParquetAvroCompatibilitySuite:
*** RUN ABORTED ***
  java.lang.NoClassDefFoundError: org/apache/avro/LogicalType
  at org.apache.parquet.avro.AvroParquetWriter.writeSupport(AvroParquetWriter.java:144)
```

## How was this patch tested?

Pass the existing test with **Maven**.

```
$ build/mvn -Pyarn -Phadoop-2.7 -Pkinesis-asl -Phive -Phive-thriftserver test
...
[INFO] ------------------------------------------------------------------------
[INFO] BUILD SUCCESS
[INFO] ------------------------------------------------------------------------
[INFO] Total time: 02:07 h
[INFO] Finished at: 2017-02-04T05:41:43+00:00
[INFO] Final Memory: 77M/987M
[INFO] ------------------------------------------------------------------------
```

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #16795 from dongjoon-hyun/SPARK-19409-2.
2017-02-08 12:21:49 +00:00
Sean Owen e8d3fca450
[SPARK-19464][CORE][YARN][TEST-HADOOP2.6] Remove support for Hadoop 2.5 and earlier
## What changes were proposed in this pull request?

- Remove support for Hadoop 2.5 and earlier
- Remove reflection and code constructs only needed to support multiple versions at once
- Update docs to reflect newer versions
- Remove older versions' builds and profiles.

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #16810 from srowen/SPARK-19464.
2017-02-08 12:20:07 +00:00
windpiger d60dde26f9 [SPARK-19488][SQL] fix csv infer schema when the field is Nan/Inf etc
## What changes were proposed in this pull request?

when csv infer schema, it does not use user defined csvoptions to parse the field, such as `inf`, `-inf` which are should be parsed to DoubleType

this pr add  `options.nanValue`, `options.negativeInf`, `options.positiveIn`  to check if the field is a DoubleType

## How was this patch tested?
unit test added

Author: windpiger <songjun@outlook.com>

Closes #16834 from windpiger/fixinferInfSchemaCsv.
2017-02-08 14:30:28 +08:00
Nattavut Sutyanyong 266c1e7309 [SPARK-18873][SQL][TEST] New test cases for scalar subquery (part 1 of 2) - scalar subquery in SELECT clause
## What changes were proposed in this pull request?
This PR adds new test cases for scalar subquery in SELECT clause.

## How was this patch tested?
The test result is compared with the result run from another SQL engine (in this case is IBM DB2). If the result are equivalent, we assume the result is correct.

Author: Nattavut Sutyanyong <nsy.can@gmail.com>

Closes #16712 from nsyca/18873.
2017-02-07 23:36:34 -05:00
CodingCat d4cd975718 [SPARK-19499][SS] Add more notes in the comments of Sink.addBatch()
## What changes were proposed in this pull request?

addBatch method in Sink trait is supposed to be a synchronous method to coordinate with the fault-tolerance design in StreamingExecution (being different with the compute() method in DStream)

We need to add more notes in the comments of this method to remind the developers

## How was this patch tested?

existing tests

Author: CodingCat <zhunansjtu@gmail.com>

Closes #16840 from CodingCat/SPARK-19499.
2017-02-07 20:25:18 -08:00
Tathagata Das aeb80348dd [SPARK-19413][SS] MapGroupsWithState for arbitrary stateful operations
## What changes were proposed in this pull request?

`mapGroupsWithState` is a new API for arbitrary stateful operations in Structured Streaming, similar to `DStream.mapWithState`

*Requirements*
- Users should be able to specify a function that can do the following
- Access the input row corresponding to a key
- Access the previous state corresponding to a key
- Optionally, update or remove the state
- Output any number of new rows (or none at all)

*Proposed API*
```
// ------------ New methods on KeyValueGroupedDataset ------------
class KeyValueGroupedDataset[K, V] {
	// Scala friendly
	def mapGroupsWithState[S: Encoder, U: Encoder](func: (K, Iterator[V], KeyedState[S]) => U)
        def flatMapGroupsWithState[S: Encode, U: Encoder](func: (K, Iterator[V], KeyedState[S]) => Iterator[U])
	// Java friendly
       def mapGroupsWithState[S, U](func: MapGroupsWithStateFunction[K, V, S, R], stateEncoder: Encoder[S], resultEncoder: Encoder[U])
       def flatMapGroupsWithState[S, U](func: FlatMapGroupsWithStateFunction[K, V, S, R], stateEncoder: Encoder[S], resultEncoder: Encoder[U])
}

// ------------------- New Java-friendly function classes -------------------
public interface MapGroupsWithStateFunction<K, V, S, R> extends Serializable {
  R call(K key, Iterator<V> values, state: KeyedState<S>) throws Exception;
}
public interface FlatMapGroupsWithStateFunction<K, V, S, R> extends Serializable {
  Iterator<R> call(K key, Iterator<V> values, state: KeyedState<S>) throws Exception;
}

// ---------------------- Wrapper class for state data ----------------------
trait State[S] {
	def exists(): Boolean
  	def get(): S 			// throws Exception is state does not exist
	def getOption(): Option[S]
	def update(newState: S): Unit
	def remove(): Unit		// exists() will be false after this
}
```

Key Semantics of the State class
- The state can be null.
- If the state.remove() is called, then state.exists() will return false, and getOption will returm None.
- After that state.update(newState) is called, then state.exists() will return true, and getOption will return Some(...).
- None of the operations are thread-safe. This is to avoid memory barriers.

*Usage*
```
val stateFunc = (word: String, words: Iterator[String, runningCount: KeyedState[Long]) => {
    val newCount = words.size + runningCount.getOption.getOrElse(0L)
    runningCount.update(newCount)
   (word, newCount)
}

dataset					                        // type is Dataset[String]
  .groupByKey[String](w => w)        	                // generates KeyValueGroupedDataset[String, String]
  .mapGroupsWithState[Long, (String, Long)](stateFunc)	// returns Dataset[(String, Long)]
```

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

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

Closes #16758 from tdas/mapWithState.
2017-02-07 20:21:00 -08:00
gatorsmile e33aaa2ac5 [SPARK-19397][SQL] Make option names of LIBSVM and TEXT case insensitive
### What changes were proposed in this pull request?
Prior to Spark 2.1, the option names are case sensitive for all the formats. Since Spark 2.1, the option key names become case insensitive except the format `Text` and `LibSVM `. This PR is to fix these issues.

Also, add a check to know whether the input option vector type is legal for `LibSVM`.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16737 from gatorsmile/libSVMTextOptions.
2017-02-08 09:33:18 +08:00
Herman van Hovell 73ee73945e [SPARK-18609][SPARK-18841][SQL] Fix redundant Alias removal in the optimizer
## What changes were proposed in this pull request?
The optimizer tries to remove redundant alias only projections from the query plan using the `RemoveAliasOnlyProject` rule. The current rule identifies removes such a project and rewrites the project's attributes in the **entire** tree. This causes problems when parts of the tree are duplicated (for instance a self join on a temporary view/CTE)  and the duplicated part contains the alias only project, in this case the rewrite will break the tree.

This PR fixes these problems by using a blacklist for attributes that are not to be moved, and by making sure that attribute remapping is only done for the parent tree, and not for unrelated parts of the query plan.

The current tree transformation infrastructure works very well if the transformation at hand requires little or a global contextual information. In this case we need to know both the attributes that were not to be moved, and we also needed to know which child attributes were modified. This cannot be done easily using the current infrastructure, and solutions typically involves transversing the query plan multiple times (which is super slow). I have moved around some code in `TreeNode`, `QueryPlan` and `LogicalPlan`to make this much more straightforward; this basically allows you to manually traverse the tree.

This PR subsumes the following PRs by windpiger:
Closes https://github.com/apache/spark/pull/16267
Closes https://github.com/apache/spark/pull/16255

## How was this patch tested?
I have added unit tests to `RemoveRedundantAliasAndProjectSuite` and I have added integration tests to the `SQLQueryTestSuite.union` and `SQLQueryTestSuite.cte` test cases.

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

Closes #16757 from hvanhovell/SPARK-18609.
2017-02-07 22:28:59 +01:00
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
gagan taneja e99e34d0f3 [SPARK-19118][SQL] Percentile support for frequency distribution table
## What changes were proposed in this pull request?

I have a frequency distribution table with following entries
Age,    No of person
21, 10
22, 15
23, 18
..
..
30, 14
Moreover it is common to have data in frequency distribution format to further calculate Percentile, Median. With current implementation
It would be very difficult and complex to find the percentile.
Therefore i am proposing enhancement to current Percentile and Approx Percentile implementation to take frequency distribution column into consideration

## How was this patch tested?
1) Enhanced /sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala to cover the additional functionality
2) Run some performance benchmark test with 20 million row in local environment and did not see any performance degradation

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

Author: gagan taneja <tanejagagan@gagans-MacBook-Pro.local>

Closes #16497 from tanejagagan/branch-18940.
2017-02-07 14:05:22 +01:00
hyukjinkwon 3d314d08c9 [SPARK-16101][SQL] Refactoring CSV schema inference path to be consistent with JSON
## What changes were proposed in this pull request?

This PR refactors CSV schema inference path to be consistent with JSON data source and moves some filtering codes having the similar/same logics into `CSVUtils`.

 It makes the methods in classes have consistent arguments with JSON ones. (this PR renames `.../json/InferSchema.scala` → `.../json/JsonInferSchema.scala`)

`CSVInferSchema` and `JsonInferSchema`

``` scala
private[csv] object CSVInferSchema {
  ...

  def infer(
      csv: Dataset[String],
      caseSensitive: Boolean,
      options: CSVOptions): StructType = {
  ...
```

``` scala
private[sql] object JsonInferSchema {
  ...

  def infer(
      json: RDD[String],
      columnNameOfCorruptRecord: String,
      configOptions: JSONOptions): StructType = {
  ...
```

These allow schema inference from `Dataset[String]` directly, meaning the similar functionalities that use `JacksonParser`/`JsonInferSchema` for JSON can be easily implemented by `UnivocityParser`/`CSVInferSchema` for CSV.

This completes refactoring CSV datasource and they are now pretty consistent.

## How was this patch tested?

Existing tests should cover this and

```
./dev/change-scala-version.sh 2.10
./build/mvn -Pyarn -Phadoop-2.4 -Dscala-2.10 -DskipTests clean package
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16680 from HyukjinKwon/SPARK-16101-schema-inference.
2017-02-07 21:02:20 +08:00
Eyal Farago a97edc2cf4 [SPARK-18601][SQL] Simplify Create/Get complex expression pairs in optimizer
## What changes were proposed in this pull request?
It often happens that a complex object (struct/map/array) is created only to get elements from it in an subsequent expression. We can add an optimizer rule for this.

## How was this patch tested?
unit-tests

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

Author: Eyal Farago <eyal@nrgene.com>
Author: eyal farago <eyal.farago@gmail.com>

Closes #16043 from eyalfa/SPARK-18601.
2017-02-07 10:54:55 +01:00
uncleGen 7a0a630e0f [SPARK-19407][SS] defaultFS is used FileSystem.get instead of getting it from uri scheme
## What changes were proposed in this pull request?

```
Caused by: java.lang.IllegalArgumentException: Wrong FS: s3a://**************/checkpoint/7b2231a3-d845-4740-bfa3-681850e5987f/metadata, expected: file:///
	at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:649)
	at org.apache.hadoop.fs.RawLocalFileSystem.pathToFile(RawLocalFileSystem.java:82)
	at org.apache.hadoop.fs.RawLocalFileSystem.deprecatedGetFileStatus(RawLocalFileSystem.java:606)
	at org.apache.hadoop.fs.RawLocalFileSystem.getFileLinkStatusInternal(RawLocalFileSystem.java:824)
	at org.apache.hadoop.fs.RawLocalFileSystem.getFileStatus(RawLocalFileSystem.java:601)
	at org.apache.hadoop.fs.FilterFileSystem.getFileStatus(FilterFileSystem.java:421)
	at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:1426)
	at org.apache.spark.sql.execution.streaming.StreamMetadata$.read(StreamMetadata.scala:51)
	at org.apache.spark.sql.execution.streaming.StreamExecution.<init>(StreamExecution.scala:100)
	at org.apache.spark.sql.streaming.StreamingQueryManager.createQuery(StreamingQueryManager.scala:232)
	at org.apache.spark.sql.streaming.StreamingQueryManager.startQuery(StreamingQueryManager.scala:269)
	at org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:262)
```

Can easily replicate on spark standalone cluster by providing checkpoint location uri scheme anything other than "file://" and not overriding in config.

WorkAround  --conf spark.hadoop.fs.defaultFS=s3a://somebucket or set it in sparkConf or spark-default.conf

## How was this patch tested?

existing ut

Author: uncleGen <hustyugm@gmail.com>

Closes #16815 from uncleGen/SPARK-19407.
2017-02-06 21:03:20 -08:00
gatorsmile d6dc603ed4 [SPARK-19441][SQL] Remove IN type coercion from PromoteStrings
### What changes were proposed in this pull request?
The removed codes for `IN` are not reachable, because the previous rule `InConversion` already resolves the type coercion issues.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16783 from gatorsmile/typeCoercionIn.
2017-02-07 09:59:16 +08:00
Herman van Hovell cb2677b860 [SPARK-19472][SQL] Parser should not mistake CASE WHEN(...) for a function call
## What changes were proposed in this pull request?
The SQL parser can mistake a `WHEN (...)` used in `CASE` for a function call. This happens in cases like the following:
```sql
select case when (1) + case when 1 > 0 then 1 else 0 end = 2 then 1 else 0 end
from tb
```
This PR fixes this by re-organizing the case related parsing rules.

## How was this patch tested?
Added a regression test to the `ExpressionParserSuite`.

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

Closes #16821 from hvanhovell/SPARK-19472.
2017-02-06 15:28:13 -05: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
hyukjinkwon 0f16ff5b0e [SPARK-17213][SQL][FOLLOWUP] Re-enable Parquet filter tests for binary and string
## What changes were proposed in this pull request?

This PR proposes to enable the tests for Parquet filter pushdown with binary and string.

This was disabled in https://github.com/apache/spark/pull/16106 due to Parquet's issue but it is now revived in https://github.com/apache/spark/pull/16791 after upgrading Parquet to 1.8.2.

## How was this patch tested?

Manually tested `ParquetFilterSuite` via IDE.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16817 from HyukjinKwon/SPARK-17213.
2017-02-06 23:10:05 +08:00
Cheng Lian 7730426cb9 [SPARK-19409][SPARK-17213] Cleanup Parquet workarounds/hacks due to bugs of old Parquet versions
## What changes were proposed in this pull request?

We've already upgraded parquet-mr to 1.8.2. This PR does some further cleanup by removing a workaround of PARQUET-686 and a hack due to PARQUET-363 and PARQUET-278. All three Parquet issues are fixed in parquet-mr 1.8.2.

## How was this patch tested?

Existing unit tests.

Author: Cheng Lian <lian@databricks.com>

Closes #16791 from liancheng/parquet-1.8.2-cleanup.
2017-02-06 09:10:55 +01: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
Liang-Chi Hsieh 0674e7eb85 [SPARK-19425][SQL] Make ExtractEquiJoinKeys support UDT columns
## What changes were proposed in this pull request?

DataFrame.except doesn't work for UDT columns. It is because `ExtractEquiJoinKeys` will run `Literal.default` against UDT. However, we don't handle UDT in `Literal.default` and an exception will throw like:

    java.lang.RuntimeException: no default for type
    org.apache.spark.ml.linalg.VectorUDT3bfc3ba7
      at org.apache.spark.sql.catalyst.expressions.Literal$.default(literals.scala:179)
      at org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys$$anonfun$4.apply(patterns.scala:117)
      at org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys$$anonfun$4.apply(patterns.scala:110)

More simple fix is just let `Literal.default` handle UDT by its sql type. So we can use more efficient join type on UDT.

Besides `except`, this also fixes other similar scenarios, so in summary this fixes:

* `except` on two Datasets with UDT
* `intersect` on two Datasets with UDT
* `Join` with the join conditions using `<=>` on UDT columns

## 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 #16765 from viirya/df-except-for-udt.
2017-02-04 15:57:56 -08:00
hyukjinkwon 2f3c20bbdd [SPARK-19446][SQL] Remove unused findTightestCommonType in TypeCoercion
## What changes were proposed in this pull request?

This PR proposes to

- remove unused `findTightestCommonType` in `TypeCoercion` as suggested in https://github.com/apache/spark/pull/16777#discussion_r99283834
- rename `findTightestCommonTypeOfTwo ` to `findTightestCommonType`.
- fix comments accordingly

The usage was removed while refactoring/fixing in several JIRAs such as SPARK-16714, SPARK-16735 and SPARK-16646

## How was this patch tested?

Existing tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16786 from HyukjinKwon/SPARK-19446.
2017-02-03 22:10:17 -08:00
Dongjoon Hyun 52d4f61941 [SPARK-18909][SQL] The error messages in ExpressionEncoder.toRow/fromRow are too verbose
## What changes were proposed in this pull request?

In `ExpressionEncoder.toRow` and `fromRow`, we catch the exception and output `treeString` of serializer/deserializer expressions in the error message. However, encoder can be very complex and the serializer/deserializer expressions can be very large trees and blow up the log files(e.g. generate over 500mb logs for this single error message.) As a first attempt, this PR try to use `simpleString` instead.

**BEFORE**

```scala
scala> :paste
// Entering paste mode (ctrl-D to finish)

case class TestCaseClass(value: Int)
import spark.implicits._
Seq(TestCaseClass(1)).toDS().collect()

// Exiting paste mode, now interpreting.

java.lang.RuntimeException: Error while decoding: java.lang.NullPointerException
newInstance(class TestCaseClass)
+- assertnotnull(input[0, int, false], - field (class: "scala.Int", name: "value"), - root class: "TestCaseClass")
   +- input[0, int, false]

  at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.fromRow(ExpressionEncoder.scala:303)
...
```

**AFTER**

```scala
...
// Exiting paste mode, now interpreting.

java.lang.RuntimeException: Error while decoding: java.lang.NullPointerException
newInstance(class TestCaseClass)
  at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.fromRow(ExpressionEncoder.scala:303)
...
```

## How was this patch tested?

Manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #16701 from dongjoon-hyun/SPARK-18909-EXPR-ERROR.
2017-02-03 20:26:53 +08:00
Liang-Chi Hsieh bf493686eb [SPARK-19411][SQL] Remove the metadata used to mark optional columns in merged Parquet schema for filter predicate pushdown
## What changes were proposed in this pull request?

There is a metadata introduced before to mark the optional columns in merged Parquet schema for filter predicate pushdown. As we upgrade to Parquet 1.8.2 which includes the fix for the pushdown of optional columns, we don't need this metadata now.

## 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 #16756 from viirya/remove-optional-metadata.
2017-02-03 11:58:42 +01:00
Zheng RuiFeng b0985764f0 [SPARK-14352][SQL] approxQuantile should support multi columns
## What changes were proposed in this pull request?

1, add the multi-cols support based on current private api
2, add the multi-cols support to pyspark
## How was this patch tested?

unit tests

Author: Zheng RuiFeng <ruifengz@foxmail.com>
Author: Ruifeng Zheng <ruifengz@foxmail.com>

Closes #12135 from zhengruifeng/quantile4multicols.
2017-02-01 14:11:28 -08:00
hyukjinkwon 5ed397baa7 [SPARK-19296][SQL] Deduplicate url and table in JdbcUtils
## What changes were proposed in this pull request?

This PR deduplicates arguments, `url` and `table` in `JdbcUtils` with `JDBCOptions`.

It avoids to use duplicated arguments, for example, as below:

from

```scala
val jdbcOptions = new JDBCOptions(url, table, map)
JdbcUtils.saveTable(ds, url, table, jdbcOptions)
```

to

```scala
val jdbcOptions = new JDBCOptions(url, table, map)
JdbcUtils.saveTable(ds, jdbcOptions)
```

## How was this patch tested?

Running unit test in `JdbcSuite`/`JDBCWriteSuite`

Building with Scala 2.10 as below:

```
./dev/change-scala-version.sh 2.10
./build/mvn -Pyarn -Phadoop-2.4 -Dscala-2.10 -DskipTests clean package
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16753 from HyukjinKwon/SPARK-19296.
2017-02-01 09:43:35 -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
Burak Yavuz 081b7addaf [SPARK-19378][SS] Ensure continuity of stateOperator and eventTime metrics even if there is no new data in trigger
## What changes were proposed in this pull request?

In StructuredStreaming, if a new trigger was skipped because no new data arrived, we suddenly report nothing for the metrics `stateOperator`. We could however easily report the metrics from `lastExecution` to ensure continuity of metrics.

## How was this patch tested?

Regression test in `StreamingQueryStatusAndProgressSuite`

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #16716 from brkyvz/state-agg.
2017-01-31 16:52:53 -08:00
gatorsmile f9156d2956 [SPARK-19406][SQL] Fix function to_json to respect user-provided options
### What changes were proposed in this pull request?
Currently, the function `to_json` allows users to provide options for generating JSON. However, it does not pass it to `JacksonGenerator`. Thus, it ignores the user-provided options. This PR is to fix it. Below is an example.

```Scala
val df = Seq(Tuple1(Tuple1(java.sql.Timestamp.valueOf("2015-08-26 18:00:00.0")))).toDF("a")
val options = Map("timestampFormat" -> "dd/MM/yyyy HH:mm")
df.select(to_json($"a", options)).show(false)
```
The current output is like
```
+--------------------------------------+
|structtojson(a)                       |
+--------------------------------------+
|{"_1":"2015-08-26T18:00:00.000-07:00"}|
+--------------------------------------+
```

After the fix, the output is like
```
+-------------------------+
|structtojson(a)          |
+-------------------------+
|{"_1":"26/08/2015 18:00"}|
+-------------------------+
```
### How was this patch tested?
Added test cases for both `from_json` and `to_json`

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16745 from gatorsmile/toJson.
2017-01-30 18:38:14 -08:00
Liwei Lin ade075aed4 [SPARK-19385][SQL] During canonicalization, NOT(...(l, r)) should not expect such cases that l.hashcode > r.hashcode
## What changes were proposed in this pull request?

During canonicalization, `NOT(...(l, r))` should not expect such cases that `l.hashcode > r.hashcode`.

Take the rule `case NOT(GreaterThan(l, r)) if l.hashcode > r.hashcode` for example, it should never be matched since `GreaterThan(l, r)` itself would be re-written as `GreaterThan(r, l)` given `l.hashcode > r.hashcode` after canonicalization.

This patch consolidates rules like `case NOT(GreaterThan(l, r)) if l.hashcode > r.hashcode` and `case NOT(GreaterThan(l, r))`.

## How was this patch tested?

This patch expanded the `NOT` test case to cover both cases where:
- `l.hashcode > r.hashcode`
- `l.hashcode < r.hashcode`

Author: Liwei Lin <lwlin7@gmail.com>

Closes #16719 from lw-lin/canonicalize.
2017-01-29 13:00:50 -08:00
Dilip Biswal e2e7b12ce8 [SPARK-18872][SQL][TESTS] New test cases for EXISTS subquery
## What changes were proposed in this pull request?
This PR adds the first set of tests for EXISTS subquery.

File name                        | Brief description
------------------------| -----------------
exists-basic.sql              |Tests EXISTS and NOT EXISTS subqueries with both correlated and local predicates.
exists-within-and-or.sql|Tests EXISTS and NOT EXISTS subqueries embedded in AND or OR expression.

DB2 results are attached here as reference :

[exists-basic-db2.txt](https://github.com/apache/spark/files/733031/exists-basic-db2.txt)
[exists-and-or-db2.txt](https://github.com/apache/spark/files/733030/exists-and-or-db2.txt)

## How was this patch tested?
This patch is adding tests.

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

Closes #16710 from dilipbiswal/exist-basic.
2017-01-29 12:51:59 -08: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
Takeshi YAMAMURO 9f523d3192 [SPARK-19338][SQL] Add UDF names in explain
## What changes were proposed in this pull request?
This pr added a variable for a UDF name in `ScalaUDF`.
Then, if the variable filled, `DataFrame#explain` prints the name.

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

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

Closes #16707 from maropu/SPARK-19338.
2017-01-26 09:50:42 -08: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
Dilip Biswal 9effc2cdcb [TESTS][SQL] Setup testdata at the beginning for tests to run independently
## What changes were proposed in this pull request?

In CachedTableSuite, we are not setting up the test data at the beginning. Some tests fail while trying to run individually. When running the entire suite they run fine.

Here are some of the tests that fail -

- test("SELECT star from cached table")
- test("Self-join cached")

As part of this simplified a couple of tests by calling a support method to count the number of
InMemoryRelations.

## How was this patch tested?

Ran the failing tests individually.

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

Closes #16688 from dilipbiswal/cachetablesuite_simple.
2017-01-25 21:50:45 -08:00
gmoehler f6480b1467 [SPARK-19311][SQL] fix UDT hierarchy issue
## What changes were proposed in this pull request?
acceptType() in UDT will no only accept the same type but also all base types

## How was this patch tested?
Manual test using a set of generated UDTs fixing acceptType() in my user defined types

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

Author: gmoehler <moehler@de.ibm.com>

Closes #16660 from gmoehler/master.
2017-01-25 08:17:24 -08:00
Nattavut Sutyanyong f1ddca5fcc [SPARK-18863][SQL] Output non-aggregate expressions without GROUP BY in a subquery does not yield an error
## What changes were proposed in this pull request?
This PR will report proper error messages when a subquery expression contain an invalid plan. This problem is fixed by calling CheckAnalysis for the plan inside a subquery.

## How was this patch tested?
Existing tests and two new test cases on 2 forms of subquery, namely, scalar subquery and in/exists subquery.

````
-- TC 01.01
-- The column t2b in the SELECT of the subquery is invalid
-- because it is neither an aggregate function nor a GROUP BY column.
select t1a, t2b
from   t1, t2
where  t1b = t2c
and    t2b = (select max(avg)
              from   (select   t2b, avg(t2b) avg
                      from     t2
                      where    t2a = t1.t1b
                     )
             )
;

-- TC 01.02
-- Invalid due to the column t2b not part of the output from table t2.
select *
from   t1
where  t1a in (select   min(t2a)
               from     t2
               group by t2c
               having   t2c in (select   max(t3c)
                                from     t3
                                group by t3b
                                having   t3b > t2b ))
;
````

Author: Nattavut Sutyanyong <nsy.can@gmail.com>

Closes #16572 from nsyca/18863.
2017-01-25 17:04:36 +01:00
Kousuke Saruta 15ef3740de [SPARK-19334][SQL] Fix the code injection vulnerability related to Generator functions.
## What changes were proposed in this pull request?

Similar to SPARK-15165, codegen is in danger of arbitrary code injection. The root cause is how variable names are created by codegen.
In GenerateExec#codeGenAccessor, a variable name is created like as follows.

```
val value = ctx.freshName(name)
```

The variable `value` is named based on the value of the variable `name` and the value of `name` is from schema given by users so an attacker can attack with queries like as follows.

```
SELECT inline(array(cast(struct(1) AS struct<`=new Object() { {f();} public void f() {throw new RuntimeException("This exception is injected.");} public int x;}.x`:int>)))
```

In the example above, a RuntimeException is thrown but an attacker can replace it with arbitrary code.

## How was this patch tested?

Added a new test case.

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #16681 from sarutak/SPARK-19334.
2017-01-24 23:35:23 +01:00
Nattavut Sutyanyong cdb691eb4d [SPARK-19017][SQL] NOT IN subquery with more than one column may return incorrect results
## What changes were proposed in this pull request?

This PR fixes the code in Optimizer phase where the NULL-aware expression of a NOT IN query is expanded in Rule `RewritePredicateSubquery`.

Example:
The query

 select a1,b1
 from   t1
 where  (a1,b1) not in (select a2,b2
                        from   t2);

has the (a1, b1) = (a2, b2) rewritten from (before this fix):

Join LeftAnti, ((isnull((_1#2 = a2#16)) || isnull((_2#3 = b2#17))) || ((_1#2 = a2#16) && (_2#3 = b2#17)))

to (after this fix):

Join LeftAnti, (((_1#2 = a2#16) || isnull((_1#2 = a2#16))) && ((_2#3 = b2#17) || isnull((_2#3 = b2#17))))

## How was this patch tested?

sql/test, catalyst/test and new test cases in SQLQueryTestSuite.

Author: Nattavut Sutyanyong <nsy.can@gmail.com>

Closes #16467 from nsyca/19017.
2017-01-24 23:31:06 +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 752502be05 [SPARK-19246][SQL] CataLogTable's partitionSchema order and exist check
## What changes were proposed in this pull request?

CataLogTable's partitionSchema should check if each column name in partitionColumnNames must match one and only one field in schema, if not we should throw an exception

and CataLogTable's partitionSchema should keep order with partitionColumnNames

## How was this patch tested?
N/A

Author: windpiger <songjun@outlook.com>

Closes #16606 from windpiger/checkPartionColNameWithSchema.
2017-01-24 20:49:23 +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
hyukjinkwon ec9493b445 [SPARK-16101][HOTFIX] Fix the build with Scala 2.10 by explicit typed argument
## What changes were proposed in this pull request?

I goofed in https://github.com/apache/spark/pull/16669 which introduces the break in scala 2.10.

This fixes

```bash
[error] /home/jenkins/workspace/spark-master-compile-sbt-scala-2.10/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala:65: polymorphic expression cannot be instantiated to expected type;
[error]  found   : [B >: org.apache.spark.sql.types.StructField](B, Int) => Int
[error]  required: org.apache.spark.sql.types.StructField => ?
[error]     fields.map(schema.indexOf).toArray
[error]                       ^
[error] one error found
[error] (sql/compile:compileIncremental) Compilation failed
```

## How was this patch tested?

Manually via

```bash
./dev/change-scala-version.sh 2.10
./build/mvn -Pyarn -Phadoop-2.4 -Dscala-2.10 -DskipTests clean package
```
```
[INFO] ------------------------------------------------------------------------
[INFO] Reactor Summary:
[INFO]
[INFO] Spark Project Parent POM ........................... SUCCESS [  2.719 s]
[INFO] Spark Project Tags ................................. SUCCESS [  3.441 s]
[INFO] Spark Project Sketch ............................... SUCCESS [  3.411 s]
[INFO] Spark Project Networking ........................... SUCCESS [  5.088 s]
[INFO] Spark Project Shuffle Streaming Service ............ SUCCESS [  5.131 s]
[INFO] Spark Project Unsafe ............................... SUCCESS [  5.813 s]
[INFO] Spark Project Launcher ............................. SUCCESS [  6.567 s]
[INFO] Spark Project Core ................................. SUCCESS [01:39 min]
[INFO] Spark Project ML Local Library ..................... SUCCESS [  6.644 s]
[INFO] Spark Project GraphX ............................... SUCCESS [ 11.304 s]
[INFO] Spark Project Streaming ............................ SUCCESS [ 26.275 s]
[INFO] Spark Project Catalyst ............................. SUCCESS [01:04 min]
[INFO] Spark Project SQL .................................. SUCCESS [02:07 min]
[INFO] Spark Project ML Library ........................... SUCCESS [01:20 min]
[INFO] Spark Project Tools ................................ SUCCESS [  8.755 s]
[INFO] Spark Project Hive ................................. SUCCESS [ 51.141 s]
[INFO] Spark Project REPL ................................. SUCCESS [ 13.688 s]
[INFO] Spark Project YARN Shuffle Service ................. SUCCESS [  7.211 s]
[INFO] Spark Project YARN ................................. SUCCESS [ 10.908 s]
[INFO] Spark Project Assembly ............................. SUCCESS [  2.940 s]
[INFO] Spark Project External Flume Sink .................. SUCCESS [  4.386 s]
[INFO] Spark Project External Flume ....................... SUCCESS [  8.589 s]
[INFO] Spark Project External Flume Assembly .............. SUCCESS [  1.891 s]
[INFO] Spark Integration for Kafka 0.8 .................... SUCCESS [  8.458 s]
[INFO] Spark Project Examples ............................. SUCCESS [ 17.706 s]
[INFO] Spark Project External Kafka Assembly .............. SUCCESS [  3.070 s]
[INFO] Spark Integration for Kafka 0.10 ................... SUCCESS [ 11.227 s]
[INFO] Spark Integration for Kafka 0.10 Assembly .......... SUCCESS [  2.982 s]
[INFO] Kafka 0.10 Source for Structured Streaming ......... SUCCESS [  7.494 s]
[INFO] Spark Project Java 8 Tests ......................... SUCCESS [  3.748 s]
[INFO] ------------------------------------------------------------------------
[INFO] BUILD SUCCESS
[INFO] ------------------------------------------------------------------------
```

and manual test `CSVSuite` with Scala 2.11 with my IDE.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16684 from HyukjinKwon/hot-fix-type-ensurance.
2017-01-23 23:57:22 -08:00
Shixiong Zhu 60bd91a340 [SPARK-19268][SS] Disallow adaptive query execution for streaming queries
## What changes were proposed in this pull request?

As adaptive query execution may change the number of partitions in different batches, it may break streaming queries. Hence, we should disallow this feature in Structured Streaming.

## How was this patch tested?

`test("SPARK-19268: Adaptive query execution should be disallowed")`.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16683 from zsxwing/SPARK-19268.
2017-01-23 22:30:51 -08:00
hyukjinkwon e576c1ed79 [SPARK-9435][SQL] Reuse function in Java UDF to correctly support expressions that require equality comparison between ScalaUDF
## What changes were proposed in this pull request?

Currently, running the codes in Java

```java
spark.udf().register("inc", new UDF1<Long, Long>() {
  Override
  public Long call(Long i) {
    return i + 1;
  }
}, DataTypes.LongType);

spark.range(10).toDF("x").createOrReplaceTempView("tmp");
Row result = spark.sql("SELECT inc(x) FROM tmp GROUP BY inc(x)").head();
Assert.assertEquals(7, result.getLong(0));
```

fails as below:

```
org.apache.spark.sql.AnalysisException: expression 'tmp.`x`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.;;
Aggregate [UDF(x#19L)], [UDF(x#19L) AS UDF(x)#23L]
+- SubqueryAlias tmp, `tmp`
   +- Project [id#16L AS x#19L]
      +- Range (0, 10, step=1, splits=Some(8))

	at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.failAnalysis(CheckAnalysis.scala:40)
	at org.apache.spark.sql.catalyst.analysis.Analyzer.failAnalysis(Analyzer.scala:57)
```

The root cause is because we were creating the function every time when it needs to build as below:

```scala
scala> def inc(i: Int) = i + 1
inc: (i: Int)Int

scala> (inc(_: Int)).hashCode
res15: Int = 1231799381

scala> (inc(_: Int)).hashCode
res16: Int = 2109839984

scala> (inc(_: Int)) == (inc(_: Int))
res17: Boolean = false
```

This seems leading to the comparison failure between `ScalaUDF`s created from Java UDF API, for example, in `Expression.semanticEquals`.

In case of Scala one, it seems already fine.

Both can be tested easily as below if any reviewer is more comfortable with Scala:

```scala
val df = Seq((1, 10), (2, 11), (3, 12)).toDF("x", "y")
val javaUDF = new UDF1[Int, Int]  {
  override def call(i: Int): Int = i + 1
}
// spark.udf.register("inc", javaUDF, IntegerType) // Uncomment this for Java API
// spark.udf.register("inc", (i: Int) => i + 1)    // Uncomment this for Scala API
df.createOrReplaceTempView("tmp")
spark.sql("SELECT inc(y) FROM tmp GROUP BY inc(y)").show()
```

## How was this patch tested?

Unit test in `JavaUDFSuite.java` and `./dev/lint-java`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16553 from HyukjinKwon/SPARK-9435.
2017-01-23 22:20:42 -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
windpiger 0ef1421a64 [SPARK-19284][SQL] append to partitioned datasource table should without custom partition location
## What changes were proposed in this pull request?

when we append data to a existed partitioned datasource table, the InsertIntoHadoopFsRelationCommand.getCustomPartitionLocations currently
return the same location with Hive default, it should return None.

## How was this patch tested?

Author: windpiger <songjun@outlook.com>

Closes #16642 from windpiger/appendSchema.
2017-01-23 19:06:04 +08:00
Dongjoon Hyun c4a6519c44 [SPARK-19218][SQL] Fix SET command to show a result correctly and in a sorted order
## What changes were proposed in this pull request?

This PR aims to fix the following two things.

1. `sql("SET -v").collect()` or `sql("SET -v").show()` raises the following exceptions for String configuration with default value, `null`. For the test, please see [Jenkins result](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71539/testReport/) and 60953bf1f1 in #16624 .

```
sbt.ForkMain$ForkError: java.lang.RuntimeException: Error while decoding: java.lang.NullPointerException
createexternalrow(input[0, string, false].toString, input[1, string, false].toString, input[2, string, false].toString, StructField(key,StringType,false), StructField(value,StringType,false), StructField(meaning,StringType,false))
:- input[0, string, false].toString
:  +- input[0, string, false]
:- input[1, string, false].toString
:  +- input[1, string, false]
+- input[2, string, false].toString
   +- input[2, string, false]
```

2. Currently, `SET` and `SET -v` commands show unsorted result.
    We had better show a sorted result for UX. Also, this is compatible with Hive.

**BEFORE**
```
scala> sql("set").show(false)
...
|spark.driver.host              |10.22.16.140                                                                                                                                 |
|spark.driver.port              |63893                                                                                                                                        |
|spark.repl.class.uri           |spark://10.22.16.140:63893/classes                                                                                                           |
...
|spark.app.name                 |Spark shell                                                                                                                                  |
|spark.driver.memory            |4G                                                                                                                                           |
|spark.executor.id              |driver                                                                                                                                       |
|spark.submit.deployMode        |client                                                                                                                                       |
|spark.master                   |local[*]                                                                                                                                     |
|spark.home                     |/Users/dhyun/spark                                                                                                                           |
|spark.sql.catalogImplementation|hive                                                                                                                                         |
|spark.app.id                   |local-1484333618945                                                                                                                          |
```

**AFTER**

```
scala> sql("set").show(false)
...
|spark.app.id                   |local-1484333925649                                                                                                                          |
|spark.app.name                 |Spark shell                                                                                                                                  |
|spark.driver.host              |10.22.16.140                                                                                                                                 |
|spark.driver.memory            |4G                                                                                                                                           |
|spark.driver.port              |64994                                                                                                                                        |
|spark.executor.id              |driver                                                                                                                                       |
|spark.jars                     |                                                                                                                                             |
|spark.master                   |local[*]                                                                                                                                     |
|spark.repl.class.uri           |spark://10.22.16.140:64994/classes                                                                                                           |
|spark.sql.catalogImplementation|hive                                                                                                                                         |
|spark.submit.deployMode        |client                                                                                                                                       |
```

## How was this patch tested?

Jenkins with a new test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #16579 from dongjoon-hyun/SPARK-19218.
2017-01-23 01:21:44 -08:00
Wenchen Fan de6ad3dfa7 [SPARK-19309][SQL] disable common subexpression elimination for conditional expressions
## What changes were proposed in this pull request?

As I pointed out in https://github.com/apache/spark/pull/15807#issuecomment-259143655 , the current subexpression elimination framework has a problem, it always evaluates all common subexpressions at the beginning, even they are inside conditional expressions and may not be accessed.

Ideally we should implement it like scala lazy val, so we only evaluate it when it gets accessed at lease once. https://github.com/apache/spark/issues/15837 tries this approach, but it seems too complicated and may introduce performance regression.

This PR simply stops common subexpression elimination for conditional expressions, with some cleanup.

## How was this patch tested?

regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16659 from cloud-fan/codegen.
2017-01-23 13:31:26 +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
hyukjinkwon 74e65cb74a [SPARK-16101][SQL] Refactoring CSV read path to be consistent with JSON data source
## What changes were proposed in this pull request?

This PR refactors CSV read path to be consistent with JSON data source. It makes the methods in classes have consistent arguments with JSON ones.

`UnivocityParser` and `JacksonParser`

``` scala
private[csv] class UnivocityParser(
    schema: StructType,
    requiredSchema: StructType,
    options: CSVOptions) extends Logging {
  ...

def parse(input: String): Seq[InternalRow] = {
  ...
```

``` scala
class JacksonParser(
    schema: StructType,
    columnNameOfCorruptRecord: String,
    options: JSONOptions) extends Logging {
  ...

def parse(input: String): Option[InternalRow] = {
  ...
```

These allow parsing an iterator (`String` to `InternalRow`) as below for both JSON and CSV:

```scala
iter.flatMap(parser.parse)
```

## How was this patch tested?

Existing tests should cover this.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16669 from HyukjinKwon/SPARK-16101-read.
2017-01-23 12:23:12 +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
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
Dongjoon Hyun a2c6adcc5d
[SPARK-18857][SQL] Don't use Iterator.duplicate for incrementalCollect in Thrift Server
## What changes were proposed in this pull request?

To support `FETCH_FIRST`, SPARK-16563 used Scala `Iterator.duplicate`. However,
Scala `Iterator.duplicate` uses a **queue to buffer all items between both iterators**,
this causes GC and hangs for queries with large number of rows. We should not use this,
especially for `spark.sql.thriftServer.incrementalCollect`.

https://github.com/scala/scala/blob/2.12.x/src/library/scala/collection/Iterator.scala#L1262-L1300

## How was this patch tested?

Pass the existing tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #16440 from dongjoon-hyun/SPARK-18857.
2017-01-10 13:27:55 +00: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
Liwei Lin acfc5f3543 [SPARK-16845][SQL] GeneratedClass$SpecificOrdering grows beyond 64 KB
## What changes were proposed in this pull request?

Prior to this patch, we'll generate `compare(...)` for `GeneratedClass$SpecificOrdering` like below, leading to Janino exceptions saying the code grows beyond 64 KB.

``` scala
/* 005 */ class SpecificOrdering extends o.a.s.sql.catalyst.expressions.codegen.BaseOrdering {
/* ..... */   ...
/* 10969 */   private int compare(InternalRow a, InternalRow b) {
/* 10970 */     InternalRow i = null;  // Holds current row being evaluated.
/* 10971 */
/* 1.... */     code for comparing field0
/* 1.... */     code for comparing field1
/* 1.... */     ...
/* 1.... */     code for comparing field449
/* 15012 */
/* 15013 */     return 0;
/* 15014 */   }
/* 15015 */ }
```

This patch would break `compare(...)` into smaller `compare_xxx(...)` methods when necessary; then we'll get generated `compare(...)` like:

``` scala
/* 001 */ public SpecificOrdering generate(Object[] references) {
/* 002 */   return new SpecificOrdering(references);
/* 003 */ }
/* 004 */
/* 005 */ class SpecificOrdering extends o.a.s.sql.catalyst.expressions.codegen.BaseOrdering {
/* 006 */
/* 007 */     ...
/* 1.... */
/* 11290 */   private int compare_0(InternalRow a, InternalRow b) {
/* 11291 */     InternalRow i = null;  // Holds current row being evaluated.
/* 11292 */
/* 11293 */     i = a;
/* 11294 */     boolean isNullA;
/* 11295 */     UTF8String primitiveA;
/* 11296 */     {
/* 11297 */
/* 11298 */       Object obj = ((Expression) references[0]).eval(null);
/* 11299 */       UTF8String value = (UTF8String) obj;
/* 11300 */       isNullA = false;
/* 11301 */       primitiveA = value;
/* 11302 */     }
/* 11303 */     i = b;
/* 11304 */     boolean isNullB;
/* 11305 */     UTF8String primitiveB;
/* 11306 */     {
/* 11307 */
/* 11308 */       Object obj = ((Expression) references[0]).eval(null);
/* 11309 */       UTF8String value = (UTF8String) obj;
/* 11310 */       isNullB = false;
/* 11311 */       primitiveB = value;
/* 11312 */     }
/* 11313 */     if (isNullA && isNullB) {
/* 11314 */       // Nothing
/* 11315 */     } else if (isNullA) {
/* 11316 */       return -1;
/* 11317 */     } else if (isNullB) {
/* 11318 */       return 1;
/* 11319 */     } else {
/* 11320 */       int comp = primitiveA.compare(primitiveB);
/* 11321 */       if (comp != 0) {
/* 11322 */         return comp;
/* 11323 */       }
/* 11324 */     }
/* 11325 */
/* 11326 */
/* 11327 */     i = a;
/* 11328 */     boolean isNullA1;
/* 11329 */     UTF8String primitiveA1;
/* 11330 */     {
/* 11331 */
/* 11332 */       Object obj1 = ((Expression) references[1]).eval(null);
/* 11333 */       UTF8String value1 = (UTF8String) obj1;
/* 11334 */       isNullA1 = false;
/* 11335 */       primitiveA1 = value1;
/* 11336 */     }
/* 11337 */     i = b;
/* 11338 */     boolean isNullB1;
/* 11339 */     UTF8String primitiveB1;
/* 11340 */     {
/* 11341 */
/* 11342 */       Object obj1 = ((Expression) references[1]).eval(null);
/* 11343 */       UTF8String value1 = (UTF8String) obj1;
/* 11344 */       isNullB1 = false;
/* 11345 */       primitiveB1 = value1;
/* 11346 */     }
/* 11347 */     if (isNullA1 && isNullB1) {
/* 11348 */       // Nothing
/* 11349 */     } else if (isNullA1) {
/* 11350 */       return -1;
/* 11351 */     } else if (isNullB1) {
/* 11352 */       return 1;
/* 11353 */     } else {
/* 11354 */       int comp = primitiveA1.compare(primitiveB1);
/* 11355 */       if (comp != 0) {
/* 11356 */         return comp;
/* 11357 */       }
/* 11358 */     }
/* 1.... */
/* 1.... */   ...
/* 1.... */
/* 12652 */     return 0;
/* 12653 */   }
/* 1.... */
/* 1.... */   ...
/* 15387 */
/* 15388 */   public int compare(InternalRow a, InternalRow b) {
/* 15389 */
/* 15390 */     int comp_0 = compare_0(a, b);
/* 15391 */     if (comp_0 != 0) {
/* 15392 */       return comp_0;
/* 15393 */     }
/* 15394 */
/* 15395 */     int comp_1 = compare_1(a, b);
/* 15396 */     if (comp_1 != 0) {
/* 15397 */       return comp_1;
/* 15398 */     }
/* 1.... */
/* 1.... */     ...
/* 1.... */
/* 15450 */     return 0;
/* 15451 */   }
/* 15452 */ }
```
## How was this patch tested?
- a new added test case which
  - would fail prior to this patch
  - would pass with this patch
- ordering correctness should already be covered by existing tests like those in `OrderingSuite`

## Acknowledgement

A major part of this PR - the refactoring work of `splitExpression()` - has been done by ueshin.

Author: Liwei Lin <lwlin7@gmail.com>
Author: Takuya UESHIN <ueshin@happy-camper.st>
Author: Takuya Ueshin <ueshin@happy-camper.st>

Closes #15480 from lw-lin/spec-ordering-64k-.
2017-01-10 19:35:46 +08: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
Burak Yavuz faabe69cc0 [SPARK-18952] Regex strings not properly escaped in codegen for aggregations
## What changes were proposed in this pull request?

If I use the function regexp_extract, and then in my regex string, use `\`, i.e. escape character, this fails codegen, because the `\` character is not properly escaped when codegen'd.

Example stack trace:
```
/* 059 */     private int maxSteps = 2;
/* 060 */     private int numRows = 0;
/* 061 */     private org.apache.spark.sql.types.StructType keySchema = new org.apache.spark.sql.types.StructType().add("date_format(window#325.start, yyyy-MM-dd HH:mm)", org.apache.spark.sql.types.DataTypes.StringType)
/* 062 */     .add("regexp_extract(source#310.description, ([a-zA-Z]+)\[.*, 1)", org.apache.spark.sql.types.DataTypes.StringType);
/* 063 */     private org.apache.spark.sql.types.StructType valueSchema = new org.apache.spark.sql.types.StructType().add("sum", org.apache.spark.sql.types.DataTypes.LongType);
/* 064 */     private Object emptyVBase;

...

org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 62, Column 58: Invalid escape sequence
	at org.codehaus.janino.Scanner.scanLiteralCharacter(Scanner.java:918)
	at org.codehaus.janino.Scanner.produce(Scanner.java:604)
	at org.codehaus.janino.Parser.peekRead(Parser.java:3239)
	at org.codehaus.janino.Parser.parseArguments(Parser.java:3055)
	at org.codehaus.janino.Parser.parseSelector(Parser.java:2914)
	at org.codehaus.janino.Parser.parseUnaryExpression(Parser.java:2617)
	at org.codehaus.janino.Parser.parseMultiplicativeExpression(Parser.java:2573)
	at org.codehaus.janino.Parser.parseAdditiveExpression(Parser.java:2552)
```

In the codegend expression, the literal should use `\\` instead of `\`

A similar problem was solved here: https://github.com/apache/spark/pull/15156.

## How was this patch tested?

Regression test in `DataFrameAggregationSuite`

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #16361 from brkyvz/reg-break.
2017-01-09 14:25:38 -08:00
Zhenhua Wang 15c2bd01b0 [SPARK-19020][SQL] Cardinality estimation of aggregate operator
## What changes were proposed in this pull request?

Support cardinality estimation of aggregate operator

## How was this patch tested?

Add test cases

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

Closes #16431 from wzhfy/aggEstimation.
2017-01-09 11:29:42 -08:00
Zhenhua Wang 3ccabdfb4d [SPARK-17077][SQL] Cardinality estimation for project operator
## What changes were proposed in this pull request?

Support cardinality estimation for project operator.

## How was this patch tested?

Add a test suite and a base class in the catalyst package.

Author: Zhenhua Wang <wzh_zju@163.com>

Closes #16430 from wzhfy/projectEstimation.
2017-01-08 21:15:52 -08:00
anabranch 19d9d4c855 [SPARK-19126][DOCS] Update Join Documentation Across Languages
## What changes were proposed in this pull request?

- [X] Make sure all join types are clearly mentioned
- [X] Make join labeling/style consistent
- [X] Make join label ordering docs the same
- [X] Improve join documentation according to above for Scala
- [X] Improve join documentation according to above for Python
- [X] Improve join documentation according to above for R

## How was this patch tested?
No tests b/c docs.

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

Author: anabranch <wac.chambers@gmail.com>

Closes #16504 from anabranch/SPARK-19126.
2017-01-08 20:37:46 -08:00
anabranch 1f6ded6455 [SPARK-19127][DOCS] Update Rank Function Documentation
## What changes were proposed in this pull request?

- [X] Fix inconsistencies in function reference for dense rank and dense
- [X] Make all languages equivalent in their reference to `dense_rank` and `rank`.

## How was this patch tested?

N/A for docs.

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

Author: anabranch <wac.chambers@gmail.com>

Closes #16505 from anabranch/SPARK-19127.
2017-01-08 17:53:53 -08:00