Commit graph

5157 commits

Author SHA1 Message Date
Eric Liang 8e558041aa [SPARK-19820][CORE] Add interface to kill tasks w/ a reason
This commit adds a killTaskAttempt method to SparkContext, to allow users to
kill tasks so that they can be re-scheduled elsewhere.

This also refactors the task kill path to allow specifying a reason for the task kill. The reason is propagated opaquely through events, and will show up in the UI automatically as `(N killed: $reason)` and `TaskKilled: $reason`. Without this change, there is no way to provide the user feedback through the UI.

Currently used reasons are "stage cancelled", "another attempt succeeded", and "killed via SparkContext.killTask". The user can also specify a custom reason through `SparkContext.killTask`.

cc rxin

In the stage overview UI the reasons are summarized:
![1](https://cloud.githubusercontent.com/assets/14922/23929209/a83b2862-08e1-11e7-8b3e-ae1967bbe2e5.png)

Within the stage UI you can see individual task kill reasons:
![2](https://cloud.githubusercontent.com/assets/14922/23929200/9a798692-08e1-11e7-8697-72b27ad8a287.png)

Existing tests, tried killing some stages in the UI and verified the messages are as expected.

Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekl@google.com>

Closes #17166 from ericl/kill-reason.
2017-03-23 23:30:44 -07:00
Kazuaki Ishizaki bb823ca4b4 [SPARK-19959][SQL] Fix to throw NullPointerException in df[java.lang.Long].collect
## What changes were proposed in this pull request?

This PR fixes `NullPointerException` in the generated code by Catalyst. When we run the following code, we get the following `NullPointerException`. This is because there is no null checks for `inputadapter_value`  while `java.lang.Long inputadapter_value` at Line 30 may have `null`.

This happen when a type of DataFrame is nullable primitive type such as `java.lang.Long` and the wholestage codegen is used. While the physical plan keeps `nullable=true` in `input[0, java.lang.Long, true].longValue`, `BoundReference.doGenCode` ignores `nullable=true`. Thus, nullcheck code will not be generated and `NullPointerException` will occur.

This PR checks the nullability and correctly generates nullcheck if needed.
```java
sparkContext.parallelize(Seq[java.lang.Long](0L, null, 2L), 1).toDF.collect
```

```java
Caused by: java.lang.NullPointerException
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(generated.java:37)
	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:393)
...
```

Generated code without this PR
```java
/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private scala.collection.Iterator inputadapter_input;
/* 009 */   private UnsafeRow serializefromobject_result;
/* 010 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
/* 011 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
/* 012 */
/* 013 */   public GeneratedIterator(Object[] references) {
/* 014 */     this.references = references;
/* 015 */   }
/* 016 */
/* 017 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 018 */     partitionIndex = index;
/* 019 */     this.inputs = inputs;
/* 020 */     inputadapter_input = inputs[0];
/* 021 */     serializefromobject_result = new UnsafeRow(1);
/* 022 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 0);
/* 023 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
/* 024 */
/* 025 */   }
/* 026 */
/* 027 */   protected void processNext() throws java.io.IOException {
/* 028 */     while (inputadapter_input.hasNext() && !stopEarly()) {
/* 029 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 030 */       java.lang.Long inputadapter_value = (java.lang.Long)inputadapter_row.get(0, null);
/* 031 */
/* 032 */       boolean serializefromobject_isNull = true;
/* 033 */       long serializefromobject_value = -1L;
/* 034 */       if (!false) {
/* 035 */         serializefromobject_isNull = false;
/* 036 */         if (!serializefromobject_isNull) {
/* 037 */           serializefromobject_value = inputadapter_value.longValue();
/* 038 */         }
/* 039 */
/* 040 */       }
/* 041 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 042 */
/* 043 */       if (serializefromobject_isNull) {
/* 044 */         serializefromobject_rowWriter.setNullAt(0);
/* 045 */       } else {
/* 046 */         serializefromobject_rowWriter.write(0, serializefromobject_value);
/* 047 */       }
/* 048 */       append(serializefromobject_result);
/* 049 */       if (shouldStop()) return;
/* 050 */     }
/* 051 */   }
/* 052 */ }
```

Generated code with this PR

```java
/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private scala.collection.Iterator inputadapter_input;
/* 009 */   private UnsafeRow serializefromobject_result;
/* 010 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
/* 011 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
/* 012 */
/* 013 */   public GeneratedIterator(Object[] references) {
/* 014 */     this.references = references;
/* 015 */   }
/* 016 */
/* 017 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 018 */     partitionIndex = index;
/* 019 */     this.inputs = inputs;
/* 020 */     inputadapter_input = inputs[0];
/* 021 */     serializefromobject_result = new UnsafeRow(1);
/* 022 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 0);
/* 023 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
/* 024 */
/* 025 */   }
/* 026 */
/* 027 */   protected void processNext() throws java.io.IOException {
/* 028 */     while (inputadapter_input.hasNext() && !stopEarly()) {
/* 029 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 030 */       boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 031 */       java.lang.Long inputadapter_value = inputadapter_isNull ? null : ((java.lang.Long)inputadapter_row.get(0, null));
/* 032 */
/* 033 */       boolean serializefromobject_isNull = true;
/* 034 */       long serializefromobject_value = -1L;
/* 035 */       if (!inputadapter_isNull) {
/* 036 */         serializefromobject_isNull = false;
/* 037 */         if (!serializefromobject_isNull) {
/* 038 */           serializefromobject_value = inputadapter_value.longValue();
/* 039 */         }
/* 040 */
/* 041 */       }
/* 042 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 043 */
/* 044 */       if (serializefromobject_isNull) {
/* 045 */         serializefromobject_rowWriter.setNullAt(0);
/* 046 */       } else {
/* 047 */         serializefromobject_rowWriter.write(0, serializefromobject_value);
/* 048 */       }
/* 049 */       append(serializefromobject_result);
/* 050 */       if (shouldStop()) return;
/* 051 */     }
/* 052 */   }
/* 053 */ }
```

## How was this patch tested?

Added new test suites in `DataFrameSuites`

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

Closes #17302 from kiszk/SPARK-19959.
2017-03-24 12:57:56 +08:00
Burak Yavuz 93581fbc18 Fix compilation of the Scala 2.10 master branch
## What changes were proposed in this pull request?

Fixes break caused by: 746a558de2

## How was this patch tested?

Compiled with `build/sbt -Dscala2.10 sql/compile` locally

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #17403 from brkyvz/onceTrigger2.10.
2017-03-23 17:57:31 -07:00
sureshthalamati c791180705 [SPARK-10849][SQL] Adds option to the JDBC data source write for user to specify database column type for the create table
## What changes were proposed in this pull request?
Currently JDBC data source creates tables in the target database using the default type mapping, and the JDBC dialect mechanism.  If users want to specify different database data type for only some of columns, there is no option available. In scenarios where default mapping does not work, users are forced to create tables on the target database before writing. This workaround is probably not acceptable from a usability point of view. This PR is to provide a user-defined type mapping for specific columns.

The solution is to allow users to specify database column data type for the create table  as JDBC datasource option(createTableColumnTypes) on write. Data type information can be specified in the same format as table schema DDL format (e.g: `name CHAR(64), comments VARCHAR(1024)`).

All supported target database types can not be specified ,  the data types has to be valid spark sql data types also.  For example user can not specify target database  CLOB data type. This will be supported in the follow-up PR.

Example:
```Scala
df.write
.option("createTableColumnTypes", "name CHAR(64), comments VARCHAR(1024)")
.jdbc(url, "TEST.DBCOLTYPETEST", properties)
```
## How was this patch tested?
Added new test cases to the JDBCWriteSuite

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

Closes #16209 from sureshthalamati/jdbc_custom_dbtype_option_json-spark-10849.
2017-03-23 17:39:33 -07:00
Tyson Condie 746a558de2 [SPARK-19876][SS][WIP] OneTime Trigger Executor
## What changes were proposed in this pull request?

An additional trigger and trigger executor that will execute a single trigger only. One can use this OneTime trigger to have more control over the scheduling of triggers.

In addition, this patch requires an optimization to StreamExecution that logs a commit record at the end of successfully processing a batch. This new commit log will be used to determine the next batch (offsets) to process after a restart, instead of using the offset log itself to determine what batch to process next after restart; using the offset log to determine this would process the previously logged batch, always, thus not permitting a OneTime trigger feature.

## How was this patch tested?

A number of existing tests have been revised. These tests all assumed that when restarting a stream, the last batch in the offset log is to be re-processed. Given that we now have a commit log that will tell us if that last batch was processed successfully, the results/assumptions of those tests needed to be revised accordingly.

In addition, a OneTime trigger test was added to StreamingQuerySuite, which tests:
- The semantics of OneTime trigger (i.e., on start, execute a single batch, then stop).
- The case when the commit log was not able to successfully log the completion of a batch before restart, which would mean that we should fall back to what's in the offset log.
- A OneTime trigger execution that results in an exception being thrown.

marmbrus tdas zsxwing

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

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

Closes #17219 from tcondie/stream-commit.
2017-03-23 14:32:05 -07:00
hyukjinkwon aefe798905 [MINOR][BUILD] Fix javadoc8 break
## What changes were proposed in this pull request?

Several javadoc8 breaks have been introduced. This PR proposes fix those instances so that we can build Scala/Java API docs.

```
[error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/GroupState.java:6: error: reference not found
[error]  * <code>flatMapGroupsWithState</code> operations on {link KeyValueGroupedDataset}.
[error]                                                             ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/GroupState.java:10: error: reference not found
[error]  * Both, <code>mapGroupsWithState</code> and <code>flatMapGroupsWithState</code> in {link KeyValueGroupedDataset}
[error]                                                                                            ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/GroupState.java:51: error: reference not found
[error]  *    {link GroupStateTimeout.ProcessingTimeTimeout}) or event time (i.e.
[error]              ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/GroupState.java:52: error: reference not found
[error]  *    {link GroupStateTimeout.EventTimeTimeout}).
[error]              ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/GroupState.java:158: error: reference not found
[error]  *           Spark SQL types (see {link Encoder} for more details).
[error]                                          ^
[error] .../spark/mllib/target/java/org/apache/spark/ml/fpm/FPGrowthParams.java:26: error: bad use of '>'
[error]    * Number of partitions (>=1) used by parallel FP-growth. By default the param is not set, and
[error]                            ^
[error] .../spark/sql/core/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsWithStateFunction.java:30: error: reference not found
[error]  * {link org.apache.spark.sql.KeyValueGroupedDataset#flatMapGroupsWithState(
[error]           ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyValueGroupedDataset.java:211: error: reference not found
[error]    * See {link GroupState} for more details.
[error]                 ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyValueGroupedDataset.java:232: error: reference not found
[error]    * See {link GroupState} for more details.
[error]                 ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyValueGroupedDataset.java:254: error: reference not found
[error]    * See {link GroupState} for more details.
[error]                 ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyValueGroupedDataset.java:277: error: reference not found
[error]    * See {link GroupState} for more details.
[error]                 ^
[error] .../spark/core/target/java/org/apache/spark/TaskContextImpl.java:10: error: reference not found
[error]  * {link TaskMetrics} &amp; {link MetricsSystem} objects are not thread safe.
[error]           ^
[error] .../spark/core/target/java/org/apache/spark/TaskContextImpl.java:10: error: reference not found
[error]  * {link TaskMetrics} &amp; {link MetricsSystem} objects are not thread safe.
[error]                                     ^
[info] 13 errors
```

```
jekyll 3.3.1 | Error:  Unidoc generation failed
```

## How was this patch tested?

Manually via `jekyll build`

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17389 from HyukjinKwon/minor-javadoc8-fix.
2017-03-23 08:41:30 +00:00
hyukjinkwon 07c12c09a7 [SPARK-18579][SQL] Use ignoreLeadingWhiteSpace and ignoreTrailingWhiteSpace options in CSV writing
## What changes were proposed in this pull request?

This PR proposes to support _not_ trimming the white spaces when writing out. These are `false` by default in CSV reading path but these are `true` by default in CSV writing in univocity parser.

Both `ignoreLeadingWhiteSpace` and `ignoreTrailingWhiteSpace` options are not being used for writing and therefore, we are always trimming the white spaces.

It seems we should provide a way to keep this white spaces easily.

WIth the data below:

```scala
val df = spark.read.csv(Seq("a , b  , c").toDS)
df.show()
```

```
+---+----+---+
|_c0| _c1|_c2|
+---+----+---+
| a | b  |  c|
+---+----+---+
```

**Before**

```scala
df.write.csv("/tmp/text.csv")
spark.read.text("/tmp/text.csv").show()
```

```
+-----+
|value|
+-----+
|a,b,c|
+-----+
```

It seems this can't be worked around via `quoteAll` too.

```scala
df.write.option("quoteAll", true).csv("/tmp/text.csv")
spark.read.text("/tmp/text.csv").show()
```
```
+-----------+
|      value|
+-----------+
|"a","b","c"|
+-----------+
```

**After**

```scala
df.write.option("ignoreLeadingWhiteSpace", false).option("ignoreTrailingWhiteSpace", false).csv("/tmp/text.csv")
spark.read.text("/tmp/text.csv").show()
```

```
+----------+
|     value|
+----------+
|a , b  , c|
+----------+
```

Note that this case is possible in R

```r
> system("cat text.csv")
f1,f2,f3
a , b  , c
> df <- read.csv(file="text.csv")
> df
  f1   f2 f3
1 a   b    c
> write.csv(df, file="text1.csv", quote=F, row.names=F)
> system("cat text1.csv")
f1,f2,f3
a , b  , c
```

## How was this patch tested?

Unit tests in `CSVSuite` and manual tests for Python.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17310 from HyukjinKwon/SPARK-18579.
2017-03-23 00:25:01 -07:00
Sameer Agarwal 12cd00706c [BUILD][MINOR] Fix 2.10 build
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/17385 breaks the 2.10 sbt/maven builds by hitting an empty-string interpolation bug (https://issues.scala-lang.org/browse/SI-7919).

https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Compile/job/spark-master-compile-sbt-scala-2.10/4072/
https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Compile/job/spark-master-compile-maven-scala-2.10/3987/

## How was this patch tested?

Compiles

Author: Sameer Agarwal <sameerag@cs.berkeley.edu>

Closes #17391 from sameeragarwal/build-fix.
2017-03-22 15:58:42 -07:00
Tathagata Das 82b598b963 [SPARK-20057][SS] Renamed KeyedState to GroupState in mapGroupsWithState
## What changes were proposed in this pull request?

Since the state is tied a "group" in the "mapGroupsWithState" operations, its better to call the state "GroupState" instead of a key. This would make it more general if you extends this operation to RelationGroupedDataset and python APIs.

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

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

Closes #17385 from tdas/SPARK-20057.
2017-03-22 12:30:36 -07:00
hyukjinkwon 80fd070389 [SPARK-20018][SQL] Pivot with timestamp and count should not print internal representation
## What changes were proposed in this pull request?

Currently, when we perform count with timestamp types, it prints the internal representation as the column name as below:

```scala
Seq(new java.sql.Timestamp(1)).toDF("a").groupBy("a").pivot("a").count().show()
```

```
+--------------------+----+
|                   a|1000|
+--------------------+----+
|1969-12-31 16:00:...|   1|
+--------------------+----+
```

This PR proposes to use external Scala value instead of the internal representation in the column names as below:

```
+--------------------+-----------------------+
|                   a|1969-12-31 16:00:00.001|
+--------------------+-----------------------+
|1969-12-31 16:00:...|                      1|
+--------------------+-----------------------+
```

## How was this patch tested?

Unit test in `DataFramePivotSuite` and manual tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17348 from HyukjinKwon/SPARK-20018.
2017-03-22 09:58:46 -07:00
hyukjinkwon 465818389a [SPARK-19949][SQL][FOLLOW-UP] Clean up parse modes and update related comments
## What changes were proposed in this pull request?

This PR proposes to make `mode` options in both CSV and JSON to use `cass object` and fix some related comments related previous fix.

Also, this PR modifies some tests related parse modes.

## How was this patch tested?

Modified unit tests in both `CSVSuite.scala` and `JsonSuite.scala`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17377 from HyukjinKwon/SPARK-19949.
2017-03-22 09:52:37 -07:00
Prashant Sharma 0caade6340 [SPARK-20027][DOCS] Compilation fix in java docs.
## What changes were proposed in this pull request?

During build/sbt publish-local, build breaks due to javadocs errors. This patch fixes those errors.

## How was this patch tested?

Tested by running the sbt build.

Author: Prashant Sharma <prashsh1@in.ibm.com>

Closes #17358 from ScrapCodes/docs-fix.
2017-03-22 13:52:03 +00:00
Xiao Li 7343a09401 [SPARK-20023][SQL] Output table comment for DESC FORMATTED
### What changes were proposed in this pull request?
Currently, `DESC FORMATTED` did not output the table comment, unlike what `DESC EXTENDED` does. This PR is to fix it.

Also correct the following displayed names in `DESC FORMATTED`, for being consistent with `DESC EXTENDED`
- `"Create Time:"` -> `"Created:"`
- `"Last Access Time:"` -> `"Last Access:"`

### How was this patch tested?
Added test cases in `describe.sql`

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17381 from gatorsmile/descFormattedTableComment.
2017-03-22 19:08:28 +08:00
Tathagata Das c1e87e384d [SPARK-20030][SS] Event-time-based timeout for MapGroupsWithState
## What changes were proposed in this pull request?

Adding event time based timeout. The user sets the timeout timestamp directly using `KeyedState.setTimeoutTimestamp`. The keys times out when the watermark crosses the timeout timestamp.

## How was this patch tested?
Unit tests

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

Closes #17361 from tdas/SPARK-20030.
2017-03-21 21:27:08 -07:00
Kunal Khamar 2d73fcced0 [SPARK-20051][SS] Fix StreamSuite flaky test - recover from v2.1 checkpoint
## What changes were proposed in this pull request?

There is a race condition between calling stop on a streaming query and deleting directories in `withTempDir` that causes test to fail, fixing to do lazy deletion using delete on shutdown JVM hook.

## How was this patch tested?

- Unit test
  - repeated 300 runs with no failure

Author: Kunal Khamar <kkhamar@outlook.com>

Closes #17382 from kunalkhamar/partition-bugfix.
2017-03-21 18:56:14 -07:00
hyukjinkwon 9281a3d504 [SPARK-19919][SQL] Defer throwing the exception for empty paths in CSV datasource into DataSource
## What changes were proposed in this pull request?

This PR proposes to defer throwing the exception within `DataSource`.

Currently, if other datasources fail to infer the schema, it returns `None` and then this is being validated in `DataSource` as below:

```
scala> spark.read.json("emptydir")
org.apache.spark.sql.AnalysisException: Unable to infer schema for JSON. It must be specified manually.;
```

```
scala> spark.read.orc("emptydir")
org.apache.spark.sql.AnalysisException: Unable to infer schema for ORC. It must be specified manually.;
```

```
scala> spark.read.parquet("emptydir")
org.apache.spark.sql.AnalysisException: Unable to infer schema for Parquet. It must be specified manually.;
```

However, CSV it checks it within the datasource implementation and throws another exception message as below:

```
scala> spark.read.csv("emptydir")
java.lang.IllegalArgumentException: requirement failed: Cannot infer schema from an empty set of files
```

We could remove this duplicated check and validate this in one place in the same way with the same message.

## How was this patch tested?

Unit test in `CSVSuite` and manual test.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17256 from HyukjinKwon/SPARK-19919.
2017-03-22 08:41:46 +08:00
Will Manning a04dcde8cb clarify array_contains function description
## What changes were proposed in this pull request?

The description in the comment for array_contains is vague/incomplete (i.e., doesn't mention that it returns `null` if the array is `null`); this PR fixes that.

## How was this patch tested?

No testing, since it merely changes a comment.

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

Author: Will Manning <lwwmanning@gmail.com>

Closes #17380 from lwwmanning/patch-1.
2017-03-22 00:40:48 +01:00
zhaorongsheng 7dbc162f12 [SPARK-20017][SQL] change the nullability of function 'StringToMap' from 'false' to 'true'
## What changes were proposed in this pull request?

Change the nullability of function `StringToMap` from `false` to `true`.

Author: zhaorongsheng <334362872@qq.com>

Closes #17350 from zhaorongsheng/bug-fix_strToMap_NPE.
2017-03-21 11:30:55 -07:00
Xin Wu 4c0ff5f585 [SPARK-19261][SQL] Alter add columns for Hive serde and some datasource tables
## What changes were proposed in this pull request?
Support` ALTER TABLE ADD COLUMNS (...) `syntax for Hive serde and some datasource tables.
In this PR, we consider a few aspects:

1. View is not supported for `ALTER ADD COLUMNS`

2. Since tables created in SparkSQL with Hive DDL syntax will populate table properties with schema information, we need make sure the consistency of the schema before and after ALTER operation in order for future use.

3. For embedded-schema type of format, such as `parquet`, we need to make sure that the predicate on the newly-added columns can be evaluated properly, or pushed down properly. In case of the data file does not have the columns for the newly-added columns, such predicates should return as if the column values are NULLs.

4. For datasource table, this feature does not support the following:
4.1 TEXT format, since there is only one default column `value` is inferred for text format data.
4.2 ORC format, since SparkSQL native ORC reader does not support the difference between user-specified-schema and inferred schema from ORC files.
4.3 Third party datasource types that implements RelationProvider, including the built-in JDBC format, since different implementations by the vendors may have different ways to dealing with schema.
4.4 Other datasource types, such as `parquet`, `json`, `csv`, `hive` are supported.

5. Column names being added can not be duplicate of any existing data column or partition column names. Case sensitivity is taken into consideration according to the sql configuration.

6. This feature also supports In-Memory catalog, while Hive support is turned off.
## How was this patch tested?
Add new test cases

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

Closes #16626 from xwu0226/alter_add_columns.
2017-03-21 08:49:54 -07:00
wangzhenhua 14865d7ff7 [SPARK-17080][SQL][FOLLOWUP] Improve documentation, change buildJoin method structure and add a debug log
## What changes were proposed in this pull request?

1. Improve documentation for class `Cost` and `JoinReorderDP` and method `buildJoin()`.
2. Change code structure of `buildJoin()` to make the logic clearer.
3. Add a debug-level log to record information for join reordering, including time cost, the number of items and the number of plans in memo.

## How was this patch tested?

Not related.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17353 from wzhfy/reorderFollow.
2017-03-21 08:44:09 -07:00
Xiao Li d2dcd6792f [SPARK-20024][SQL][TEST-MAVEN] SessionCatalog reset need to set the current database of ExternalCatalog
### What changes were proposed in this pull request?
SessionCatalog API setCurrentDatabase does not set the current database of the underlying ExternalCatalog. Thus, weird errors could come in the test suites after we call reset. We need to fix it.

So far, have not found the direct impact in the other code paths because we expect all the SessionCatalog APIs should always use the current database value we managed, unless some of code paths skip it. Thus, we fix it in the test-only function reset().

### How was this patch tested?
Multiple test case failures are observed in mvn and add a test case in SessionCatalogSuite.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17354 from gatorsmile/useDB.
2017-03-20 22:52:45 -07:00
Wenchen Fan 68d65fae71 [SPARK-19949][SQL] unify bad record handling in CSV and JSON
## What changes were proposed in this pull request?

Currently JSON and CSV have exactly the same logic about handling bad records, this PR tries to abstract it and put it in a upper level to reduce code duplication.

The overall idea is, we make the JSON and CSV parser to throw a BadRecordException, then the upper level, FailureSafeParser, handles bad records according to the parse mode.

Behavior changes:
1. with PERMISSIVE mode, if the number of tokens doesn't match the schema, previously CSV parser will treat it as a legal record and parse as many tokens as possible. After this PR, we treat it as an illegal record, and put the raw record string in a special column, but we still parse as many tokens as possible.
2. all logging is removed as they are not very useful in practice.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>
Author: hyukjinkwon <gurwls223@gmail.com>
Author: Wenchen Fan <cloud0fan@gmail.com>

Closes #17315 from cloud-fan/bad-record2.
2017-03-20 21:43:14 -07:00
Dongjoon Hyun 21e366aea5 [SPARK-19912][SQL] String literals should be escaped for Hive metastore partition pruning
## What changes were proposed in this pull request?

Since current `HiveShim`'s `convertFilters` does not escape the string literals. There exists the following correctness issues. This PR aims to return the correct result and also shows the more clear exception message.

**BEFORE**

```scala
scala> Seq((1, "p1", "q1"), (2, "p1\" and q=\"q1", "q2")).toDF("a", "p", "q").write.partitionBy("p", "q").saveAsTable("t1")

scala> spark.table("t1").filter($"p" === "p1\" and q=\"q1").select($"a").show
+---+
|  a|
+---+
+---+

scala> spark.table("t1").filter($"p" === "'\"").select($"a").show
java.lang.RuntimeException: Caught Hive MetaException attempting to get partition metadata by filter from ...
```

**AFTER**

```scala
scala> spark.table("t1").filter($"p" === "p1\" and q=\"q1").select($"a").show
+---+
|  a|
+---+
|  2|
+---+

scala> spark.table("t1").filter($"p" === "'\"").select($"a").show
java.lang.UnsupportedOperationException: Partition filter cannot have both `"` and `'` characters
```

## How was this patch tested?

Pass the Jenkins test with new test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #17266 from dongjoon-hyun/SPARK-19912.
2017-03-21 12:17:26 +08:00
Takeshi Yamamuro 0ec1db5475 [SPARK-19980][SQL] Add NULL checks in Bean serializer
## What changes were proposed in this pull request?
A Bean serializer in `ExpressionEncoder`  could change values when Beans having NULL. A concrete example is as follows;
```
scala> :paste
class Outer extends Serializable {
  private var cls: Inner = _
  def setCls(c: Inner): Unit = cls = c
  def getCls(): Inner = cls
}

class Inner extends Serializable {
  private var str: String = _
  def setStr(s: String): Unit = str = str
  def getStr(): String = str
}

scala> Seq("""{"cls":null}""", """{"cls": {"str":null}}""").toDF().write.text("data")
scala> val encoder = Encoders.bean(classOf[Outer])
scala> val schema = encoder.schema
scala> val df = spark.read.schema(schema).json("data").as[Outer](encoder)
scala> df.show
+------+
|   cls|
+------+
|[null]|
|  null|
+------+

scala> df.map(x => x)(encoder).show()
+------+
|   cls|
+------+
|[null]|
|[null]|     // <-- Value changed
+------+
```

This is because the Bean serializer does not have the NULL-check expressions that the serializer of Scala's product types has. Actually, this value change does not happen in Scala's product types;

```
scala> :paste
case class Outer(cls: Inner)
case class Inner(str: String)

scala> val encoder = Encoders.product[Outer]
scala> val schema = encoder.schema
scala> val df = spark.read.schema(schema).json("data").as[Outer](encoder)
scala> df.show
+------+
|   cls|
+------+
|[null]|
|  null|
+------+

scala> df.map(x => x)(encoder).show()
+------+
|   cls|
+------+
|[null]|
|  null|
+------+
```

This pr added the NULL-check expressions in Bean serializer along with the serializer of Scala's product types.

## How was this patch tested?
Added tests in `JavaDatasetSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17347 from maropu/SPARK-19980.
2017-03-21 11:17:34 +08:00
wangzhenhua e9c91badce [SPARK-20010][SQL] Sort information is lost after sort merge join
## What changes were proposed in this pull request?

After sort merge join for inner join, now we only keep left key ordering. However, after inner join, right key has the same value and order as left key. So if we need another smj on right key, we will unnecessarily add a sort which causes additional cost.

As a more complicated example, A join B on A.key = B.key join C on B.key = C.key join D on A.key = D.key. We will unnecessarily add a sort on B.key when join {A, B} and C, and add a sort on A.key when join {A, B, C} and D.

To fix this, we need to propagate all sorted information (equivalent expressions) from bottom up through `outputOrdering` and `SortOrder`.

## How was this patch tested?

Test cases are added.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17339 from wzhfy/sortEnhance.
2017-03-21 10:43:17 +08:00
Zheng RuiFeng 10691d36de [SPARK-19573][SQL] Make NaN/null handling consistent in approxQuantile
## What changes were proposed in this pull request?
update `StatFunctions.multipleApproxQuantiles` to handle NaN/null

## How was this patch tested?
existing tests and added tests

Author: Zheng RuiFeng <ruifengz@foxmail.com>

Closes #16971 from zhengruifeng/quantiles_nan.
2017-03-20 18:25:59 -07:00
Dongjoon Hyun fc7554599a [SPARK-19970][SQL] Table owner should be USER instead of PRINCIPAL in kerberized clusters
## What changes were proposed in this pull request?

In the kerberized hadoop cluster, when Spark creates tables, the owner of tables are filled with PRINCIPAL strings instead of USER names. This is inconsistent with Hive and causes problems when using [ROLE](https://cwiki.apache.org/confluence/display/Hive/SQL+Standard+Based+Hive+Authorization) in Hive. We had better to fix this.

**BEFORE**
```scala
scala> sql("create table t(a int)").show
scala> sql("desc formatted t").show(false)
...
|Owner:                      |sparkEXAMPLE.COM                                         |       |
```

**AFTER**
```scala
scala> sql("create table t(a int)").show
scala> sql("desc formatted t").show(false)
...
|Owner:                      |spark                                         |       |
```

## How was this patch tested?

Manually do `create table` and `desc formatted` because this happens in Kerberized clusters.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #17311 from dongjoon-hyun/SPARK-19970.
2017-03-20 10:07:31 -07:00
windpiger 7ce30e00b2 [SPARK-19990][SQL][TEST-MAVEN] create a temp file for file in test.jar's resource when run mvn test accross different modules
## What changes were proposed in this pull request?

After we have merged the `HiveDDLSuite` and `DDLSuite` in [SPARK-19235](https://issues.apache.org/jira/browse/SPARK-19235), we have two subclasses of `DDLSuite`, that is `HiveCatalogedDDLSuite` and `InMemoryCatalogDDLSuite`.

While `DDLSuite` is in `sql/core module`, and `HiveCatalogedDDLSuite` is in `sql/hive module`, if we mvn test
`HiveCatalogedDDLSuite`, it will run the test in its parent class `DDLSuite`, this will cause some test case failed which will get and use the test file path in `sql/core module` 's `resource`.

Because the test file path getted will start with 'jar:' like "jar:file:/home/jenkins/workspace/spark-master-test-maven-hadoop-2.6/sql/core/target/spark-sql_2.11-2.2.0-SNAPSHOT-tests.jar!/test-data/cars.csv", which will failed when new Path() in datasource.scala

This PR fix this by copy file from resource to  a temp dir.

## How was this patch tested?
N/A

Author: windpiger <songjun@outlook.com>

Closes #17338 from windpiger/fixtestfailemvn.
2017-03-20 21:36:00 +08:00
Ioana Delaney 8163911594 [SPARK-17791][SQL] Join reordering using star schema detection
## What changes were proposed in this pull request?

Star schema consists of one or more fact tables referencing a number of dimension tables. In general, queries against star schema are expected to run fast because of the established RI constraints among the tables. This design proposes a join reordering based on natural, generally accepted heuristics for star schema queries:
- Finds the star join with the largest fact table and places it on the driving arm of the left-deep join. This plan avoids large tables on the inner, and thus favors hash joins.
- Applies the most selective dimensions early in the plan to reduce the amount of data flow.

The design document was included in SPARK-17791.

Link to the google doc: [StarSchemaDetection](https://docs.google.com/document/d/1UAfwbm_A6wo7goHlVZfYK99pqDMEZUumi7pubJXETEA/edit?usp=sharing)

## How was this patch tested?

A new test suite StarJoinSuite.scala was implemented.

Author: Ioana Delaney <ioanamdelaney@gmail.com>

Closes #15363 from ioana-delaney/starJoinReord2.
2017-03-20 16:04:58 +08:00
wangzhenhua 965a5abcff [SPARK-19994][SQL] Wrong outputOrdering for right/full outer smj
## What changes were proposed in this pull request?

For right outer join, values of the left key will be filled with nulls if it can't match the value of the right key, so `nullOrdering` of the left key can't be guaranteed. We should output right key order instead of left key order.

For full outer join, neither left key nor right key guarantees `nullOrdering`. We should not output any ordering.

In tests, besides adding three test cases for left/right/full outer sort merge join, this patch also reorganizes code in `PlannerSuite` by putting together tests for `Sort`, and also extracts common logic in Sort tests into a method.

## How was this patch tested?

Corresponding test cases are added.

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

Closes #17331 from wzhfy/wrongOrdering.
2017-03-20 14:37:23 +08:00
hyukjinkwon 0cdcf91145 [SPARK-19849][SQL] Support ArrayType in to_json to produce JSON array
## What changes were proposed in this pull request?

This PR proposes to support an array of struct type in `to_json` as below:

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

val df = Seq(Tuple1(Tuple1(1) :: Nil)).toDF("a")
df.select(to_json($"a").as("json")).show()
```

```
+----------+
|      json|
+----------+
|[{"_1":1}]|
+----------+
```

Currently, it throws an exception as below (a newline manually inserted for readability):

```
org.apache.spark.sql.AnalysisException: cannot resolve 'structtojson(`array`)' due to data type
mismatch: structtojson requires that the expression is a struct expression.;;
```

This allows the roundtrip with `from_json` as below:

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

val schema = ArrayType(StructType(StructField("a", IntegerType) :: Nil))
val df = Seq("""[{"a":1}, {"a":2}]""").toDF("json").select(from_json($"json", schema).as("array"))
df.show()

// Read back.
df.select(to_json($"array").as("json")).show()
```

```
+----------+
|     array|
+----------+
|[[1], [2]]|
+----------+

+-----------------+
|             json|
+-----------------+
|[{"a":1},{"a":2}]|
+-----------------+
```

Also, this PR proposes to rename from `StructToJson` to `StructsToJson ` and `JsonToStruct` to `JsonToStructs`.

## How was this patch tested?

Unit tests in `JsonFunctionsSuite` and `JsonExpressionsSuite` for Scala, doctest for Python and test in `test_sparkSQL.R` for R.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17192 from HyukjinKwon/SPARK-19849.
2017-03-19 22:33:01 -07:00
Tathagata Das 990af630d0 [SPARK-19067][SS] Processing-time-based timeout in MapGroupsWithState
## What changes were proposed in this pull request?

When a key does not get any new data in `mapGroupsWithState`, the mapping function is never called on it. So we need a timeout feature that calls the function again in such cases, so that the user can decide whether to continue waiting or clean up (remove state, save stuff externally, etc.).
Timeouts can be either based on processing time or event time. This JIRA is for processing time, but defines the high level API design for both. The usage would look like this.
```
def stateFunction(key: K, value: Iterator[V], state: KeyedState[S]): U = {
  ...
  state.setTimeoutDuration(10000)
  ...
}

dataset					// type is Dataset[T]
  .groupByKey[K](keyingFunc)   // generates KeyValueGroupedDataset[K, T]
  .mapGroupsWithState[S, U](
     func = stateFunction,
     timeout = KeyedStateTimeout.withProcessingTime)	// returns Dataset[U]
```

Note the following design aspects.

- The timeout type is provided as a param in mapGroupsWithState as a parameter global to all the keys. This is so that the planner knows this at planning time, and accordingly optimize the execution based on whether to saves extra info in state or not (e.g. timeout durations or timestamps).

- The exact timeout duration is provided inside the function call so that it can be customized on a per key basis.

- When the timeout occurs for a key, the function is called with no values, and KeyedState.isTimingOut() set to true.

- The timeout is reset for key every time the function is called on the key, that is, when the key has new data, or the key has timed out. So the user has to set the timeout duration everytime the function is called, otherwise there will not be any timeout set.

Guarantees provided on timeout of key, when timeout duration is D ms:
- Timeout will never be called before real clock time has advanced by D ms
- Timeout will be called eventually when there is a trigger with any data in it (i.e. after D ms). So there is a no strict upper bound on when the timeout would occur. For example, if there is no data in the stream (for any key) for a while, then the timeout will not be hit.

Implementation details:
- Added new param to `mapGroupsWithState` for timeout
- Added new method to `StateStore` to filter data based on timeout timestamp
- Changed the internal map type of `HDFSBackedStateStore` from Java's `HashMap` to `ConcurrentHashMap` as the latter allows weakly-consistent fail-safe iterators on the map data. See comments in code for more details.
- Refactored logic of `MapGroupsWithStateExec` to
  - Save timeout info to state store for each key that has data.
  - Then, filter states that should be timed out based on the current batch processing timestamp.
- Moved KeyedState for `o.a.s.sql` to `o.a.s.sql.streaming`. I remember that this was a feedback in the MapGroupsWithState PR that I had forgotten to address.

## How was this patch tested?
New unit tests in
- MapGroupsWithStateSuite for timeouts.
- StateStoreSuite for new APIs in StateStore.

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

Closes #17179 from tdas/mapgroupwithstate-timeout.
2017-03-19 14:07:49 -07:00
Xiao Li 0ee9fbf51a [SPARK-19990][TEST] Use the database after Hive's current Database is dropped
### What changes were proposed in this pull request?
This PR is to fix the following test failure in maven and the PR https://github.com/apache/spark/pull/15363.

> org.apache.spark.sql.hive.orc.OrcSourceSuite SPARK-19459/SPARK-18220: read char/varchar column written by Hive

The[ test history](https://spark-tests.appspot.com/test-details?suite_name=org.apache.spark.sql.hive.orc.OrcSourceSuite&test_name=SPARK-19459%2FSPARK-18220%3A+read+char%2Fvarchar+column+written+by+Hive) shows all the maven builds failed this test case with the same error message.

```
FAILED: SemanticException [Error 10072]: Database does not exist: db2

      org.apache.spark.sql.execution.QueryExecutionException: FAILED: SemanticException [Error 10072]: Database does not exist: db2
      at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$runHive$1.apply(HiveClientImpl.scala:637)
      at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$runHive$1.apply(HiveClientImpl.scala:621)
      at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$withHiveState$1.apply(HiveClientImpl.scala:288)
      at org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:229)
      at org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:228)
      at org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:271)
      at org.apache.spark.sql.hive.client.HiveClientImpl.runHive(HiveClientImpl.scala:621)
      at org.apache.spark.sql.hive.client.HiveClientImpl.runSqlHive(HiveClientImpl.scala:611)
      at org.apache.spark.sql.hive.orc.OrcSuite$$anonfun$7.apply$mcV$sp(OrcSourceSuite.scala:160)
      at org.apache.spark.sql.hive.orc.OrcSuite$$anonfun$7.apply(OrcSourceSuite.scala:155)
      at org.apache.spark.sql.hive.orc.OrcSuite$$anonfun$7.apply(OrcSourceSuite.scala:155)
      at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22)
      at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85)
      at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
      at org.scalatest.Transformer.apply(Transformer.scala:22)
      at org.scalatest.Transformer.apply(Transformer.scala:20)
      at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166)
      at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:68)
      at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163)
      at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
```

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17344 from gatorsmile/testtest.
2017-03-19 13:52:22 -07:00
Takeshi Yamamuro ccba622e35 [SPARK-19896][SQL] Throw an exception if case classes have circular references in toDS
## What changes were proposed in this pull request?
If case classes have circular references below, it throws StackOverflowError;
```
scala> :pasge
case class classA(i: Int, cls: classB)
case class classB(cls: classA)

scala> Seq(classA(0, null)).toDS()
java.lang.StackOverflowError
  at scala.reflect.internal.Symbols$Symbol.info(Symbols.scala:1494)
  at scala.reflect.runtime.JavaMirrors$JavaMirror$$anon$1.scala$reflect$runtime$SynchronizedSymbols$SynchronizedSymbol$$super$info(JavaMirrors.scala:66)
  at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol$$anonfun$info$1.apply(SynchronizedSymbols.scala:127)
  at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol$$anonfun$info$1.apply(SynchronizedSymbols.scala:127)
  at scala.reflect.runtime.Gil$class.gilSynchronized(Gil.scala:19)
  at scala.reflect.runtime.JavaUniverse.gilSynchronized(JavaUniverse.scala:16)
  at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol$class.gilSynchronizedIfNotThreadsafe(SynchronizedSymbols.scala:123)
  at scala.reflect.runtime.JavaMirrors$JavaMirror$$anon$1.gilSynchronizedIfNotThreadsafe(JavaMirrors.scala:66)
  at scala.reflect.runtime.SynchronizedSymbols$SynchronizedSymbol$class.info(SynchronizedSymbols.scala:127)
  at scala.reflect.runtime.JavaMirrors$JavaMirror$$anon$1.info(JavaMirrors.scala:66)
  at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:48)
  at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:45)
  at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:45)
  at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:45)
  at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:45)
```
This pr added code to throw UnsupportedOperationException in that case as follows;
```
scala> :paste
case class A(cls: B)
case class B(cls: A)

scala> Seq(A(null)).toDS()
java.lang.UnsupportedOperationException: cannot have circular references in class, but got the circular reference of class B
  at org.apache.spark.sql.catalyst.ScalaReflection$.org$apache$spark$sql$catalyst$ScalaReflection$$serializerFor(ScalaReflection.scala:627)
  at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$9.apply(ScalaReflection.scala:644)
  at org.apache.spark.sql.catalyst.ScalaReflection$$anonfun$9.apply(ScalaReflection.scala:632)
  at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
  at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
  at scala.collection.immutable.List.foreach(List.scala:381)
  at scala.collection.TraversableLike$class.flatMap(TraversableLike.scala:241)
```

## How was this patch tested?
Added tests in `DatasetSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17318 from maropu/SPARK-19896.
2017-03-18 14:40:16 +08:00
wangzhenhua c083b6b7de [SPARK-19915][SQL] Exclude cartesian product candidates to reduce the search space
## What changes were proposed in this pull request?

We have some concerns about removing size in the cost model [in the previous pr](https://github.com/apache/spark/pull/17240). It's a tradeoff between code structure and algorithm completeness. I tend to keep the size and thus create this new pr without changing cost model.

What this pr does:
1. We only consider consecutive inner joinable items, thus excluding cartesian products in reordering procedure. This significantly reduces the search space and memory overhead of memo. Otherwise every combination of items will exist in the memo.
2. This pr also includes a bug fix: if a leaf item is a project(_, child), current solution will miss the project.

## How was this patch tested?

Added test cases.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17286 from wzhfy/joinReorder3.
2017-03-18 14:07:25 +08:00
Jacek Laskowski 6326d406b9 [SQL][MINOR] Fix scaladoc for UDFRegistration
## What changes were proposed in this pull request?

Fix scaladoc for UDFRegistration

## How was this patch tested?

local build

Author: Jacek Laskowski <jacek@japila.pl>

Closes #17337 from jaceklaskowski/udfregistration-scaladoc.
2017-03-17 21:55:10 -07:00
Kunal Khamar 3783539d7a [SPARK-19873][SS] Record num shuffle partitions in offset log and enforce in next batch.
## What changes were proposed in this pull request?

If the user changes the shuffle partition number between batches, Streaming aggregation will fail.

Here are some possible cases:

- Change "spark.sql.shuffle.partitions"
- Use "repartition" and change the partition number in codes
- RangePartitioner doesn't generate deterministic partitions. Right now it's safe as we disallow sort before aggregation. Not sure if we will add some operators using RangePartitioner in future.

## How was this patch tested?

- Unit tests
- Manual tests
  - forward compatibility tested by using the new `OffsetSeqMetadata` json with Spark v2.1.0

Author: Kunal Khamar <kkhamar@outlook.com>

Closes #17216 from kunalkhamar/num-partitions.
2017-03-17 16:16:22 -07:00
Takeshi Yamamuro 7de66bae58 [SPARK-19967][SQL] Add from_json in FunctionRegistry
## What changes were proposed in this pull request?
This pr added entries in `FunctionRegistry` and supported `from_json` in SQL.

## How was this patch tested?
Added tests in `JsonFunctionsSuite` and `SQLQueryTestSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17320 from maropu/SPARK-19967.
2017-03-17 14:51:59 -07:00
Andrew Ray 13538cf3dd [SPARK-19882][SQL] Pivot with null as a distinct pivot value throws NPE
## What changes were proposed in this pull request?

Allows null values of the pivot column to be included in the pivot values list without throwing NPE

Note this PR was made as an alternative to #17224 but preserves the two phase aggregate operation that is needed for good performance.

## How was this patch tested?

Additional unit test

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

Closes #17226 from aray/pivot-null.
2017-03-17 16:43:42 +08:00
Reynold Xin 8537c00e0a [SPARK-19987][SQL] Pass all filters into FileIndex
## What changes were proposed in this pull request?
This is a tiny teeny refactoring to pass data filters also to the FileIndex, so FileIndex can have a more global view on predicates.

## How was this patch tested?
Change should be covered by existing test cases.

Author: Reynold Xin <rxin@databricks.com>

Closes #17322 from rxin/SPARK-19987.
2017-03-16 18:31:57 -07:00
Liwei Lin 2ea214dd05 [SPARK-19721][SS] Good error message for version mismatch in log files
## Problem

There are several places where we write out version identifiers in various logs for structured streaming (usually `v1`). However, in the places where we check for this, we throw a confusing error message.

## What changes were proposed in this pull request?

This patch made two major changes:
1. added a `parseVersion(...)` method, and based on this method, fixed the following places the way they did version checking (no other place needed to do this checking):
```
HDFSMetadataLog
  - CompactibleFileStreamLog  ------------> fixed with this patch
    - FileStreamSourceLog  ---------------> inherited the fix of `CompactibleFileStreamLog`
    - FileStreamSinkLog  -----------------> inherited the fix of `CompactibleFileStreamLog`
  - OffsetSeqLog  ------------------------> fixed with this patch
  - anonymous subclass in KafkaSource  ---> fixed with this patch
```

2. changed the type of `FileStreamSinkLog.VERSION`, `FileStreamSourceLog.VERSION` etc. from `String` to `Int`, so that we can identify newer versions via `version > 1` instead of `version != "v1"`
    - note this didn't break any backwards compatibility -- we are still writing out `"v1"` and reading back `"v1"`

## Exception message with this patch
```
java.lang.IllegalStateException: Failed to read log file /private/var/folders/nn/82rmvkk568sd8p3p8tb33trw0000gn/T/spark-86867b65-0069-4ef1-b0eb-d8bd258ff5b8/0. UnsupportedLogVersion: maximum supported log version is v1, but encountered v99. The log file was produced by a newer version of Spark and cannot be read by this version. Please upgrade.
	at org.apache.spark.sql.execution.streaming.HDFSMetadataLog.get(HDFSMetadataLog.scala:202)
	at org.apache.spark.sql.execution.streaming.OffsetSeqLogSuite$$anonfun$3$$anonfun$apply$mcV$sp$2.apply(OffsetSeqLogSuite.scala:78)
	at org.apache.spark.sql.execution.streaming.OffsetSeqLogSuite$$anonfun$3$$anonfun$apply$mcV$sp$2.apply(OffsetSeqLogSuite.scala:75)
	at org.apache.spark.sql.test.SQLTestUtils$class.withTempDir(SQLTestUtils.scala:133)
	at org.apache.spark.sql.execution.streaming.OffsetSeqLogSuite.withTempDir(OffsetSeqLogSuite.scala:26)
	at org.apache.spark.sql.execution.streaming.OffsetSeqLogSuite$$anonfun$3.apply$mcV$sp(OffsetSeqLogSuite.scala:75)
	at org.apache.spark.sql.execution.streaming.OffsetSeqLogSuite$$anonfun$3.apply(OffsetSeqLogSuite.scala:75)
	at org.apache.spark.sql.execution.streaming.OffsetSeqLogSuite$$anonfun$3.apply(OffsetSeqLogSuite.scala:75)
	at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22)
	at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85)
```

## How was this patch tested?

unit tests

Author: Liwei Lin <lwlin7@gmail.com>

Closes #17070 from lw-lin/better-msg.
2017-03-16 13:05:36 -07:00
windpiger 8e8f898335 [SPARK-19945][SQL] add test suite for SessionCatalog with HiveExternalCatalog
## What changes were proposed in this pull request?

Currently `SessionCatalogSuite` is only for `InMemoryCatalog`, there is no suite for `HiveExternalCatalog`.
And there are some ddl function is not proper to test in `ExternalCatalogSuite`, because some logic are not full implement in `ExternalCatalog`, these ddl functions are full implement in `SessionCatalog`(e.g. merge the same logic from `ExternalCatalog` up to `SessionCatalog` ).
It is better to test it in `SessionCatalogSuite` for this situation.

So we should add a test suite for `SessionCatalog` with `HiveExternalCatalog`

The main change is that in `SessionCatalogSuite` add two functions:
`withBasicCatalog` and `withEmptyCatalog`
And replace the code like  `val catalog = new SessionCatalog(newBasicCatalog)` with above two functions

## How was this patch tested?
add `HiveExternalSessionCatalogSuite`

Author: windpiger <songjun@outlook.com>

Closes #17287 from windpiger/sessioncatalogsuit.
2017-03-16 11:34:13 -07:00
Xiao Li 1472cac4bb [SPARK-19830][SQL] Add parseTableSchema API to ParserInterface
### What changes were proposed in this pull request?

Specifying the table schema in DDL formats is needed for different scenarios. For example,
- [specifying the schema in SQL function `from_json` using DDL formats](https://issues.apache.org/jira/browse/SPARK-19637), which is suggested by marmbrus ,
- [specifying the customized JDBC data types](https://github.com/apache/spark/pull/16209).

These two PRs need users to use the JSON format to specify the table schema. This is not user friendly.

This PR is to provide a `parseTableSchema` API in `ParserInterface`.

### How was this patch tested?
Added a test suite `TableSchemaParserSuite`

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17171 from gatorsmile/parseDDLStmt.
2017-03-16 12:06:20 +08:00
Takeshi Yamamuro 21f333c635 [SPARK-19751][SQL] Throw an exception if bean class has one's own class in fields
## What changes were proposed in this pull request?
The current master throws `StackOverflowError` in `createDataFrame`/`createDataset` if bean has one's own class in fields;
```
public class SelfClassInFieldBean implements Serializable {
  private SelfClassInFieldBean child;
  ...
}
```
This pr added code to throw `UnsupportedOperationException` in that case as soon as possible.

## How was this patch tested?
Added tests in `JavaDataFrameSuite` and `JavaDatasetSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17188 from maropu/SPARK-19751.
2017-03-16 08:50:01 +08:00
windpiger fc9314671c [SPARK-19961][SQL][MINOR] unify a erro msg when drop databse for HiveExternalCatalog and InMemoryCatalog
## What changes were proposed in this pull request?

unify a exception erro msg for dropdatabase when the database still have some tables for HiveExternalCatalog and InMemoryCatalog
## How was this patch tested?
N/A

Author: windpiger <songjun@outlook.com>

Closes #17305 from windpiger/unifyErromsg.
2017-03-16 08:44:57 +08:00
Juliusz Sompolski 339b237dc1 [SPARK-19948] Document that saveAsTable uses catalog as source of truth for table existence.
It is quirky behaviour that saveAsTable to e.g. a JDBC source with SaveMode other
than Overwrite will nevertheless overwrite the table in the external source,
if that table was not a catalog table.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #17289 from juliuszsompolski/saveAsTableDoc.
2017-03-16 08:20:47 +08:00
Liang-Chi Hsieh 7d734a6583 [SPARK-19931][SQL] InMemoryTableScanExec should rewrite output partitioning and ordering when aliasing output attributes
## What changes were proposed in this pull request?

Now `InMemoryTableScanExec` simply takes the `outputPartitioning` and `outputOrdering` from the associated `InMemoryRelation`'s `child.outputPartitioning` and `outputOrdering`.

However, `InMemoryTableScanExec` can alias the output attributes. In this case, its `outputPartitioning` and `outputOrdering` are not correct and its parent operators can't correctly determine its data distribution.

## 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 #17175 from viirya/ensure-no-unnecessary-shuffle.
2017-03-16 08:18:36 +08:00
Dongjoon Hyun 54a3697f1f [MINOR][CORE] Fix a info message of prunePartitions
## What changes were proposed in this pull request?

`PrunedInMemoryFileIndex.prunePartitions` shows `pruned NaN% partitions` for the following case.

```scala
scala> Seq.empty[(String, String)].toDF("a", "p").write.partitionBy("p").saveAsTable("t1")

scala> sc.setLogLevel("INFO")

scala> spark.table("t1").filter($"p" === "1").select($"a").show
...
17/03/13 00:33:04 INFO PrunedInMemoryFileIndex: Selected 0 partitions out of 0, pruned NaN% partitions.
```

After this PR, the message looks like this.
```scala
17/03/15 10:39:48 INFO PrunedInMemoryFileIndex: Selected 0 partitions out of 0, pruned 0 partitions.
```

## How was this patch tested?

Pass the Jenkins with the existing tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #17273 from dongjoon-hyun/SPARK-EMPTY-PARTITION.
2017-03-15 15:01:16 -07:00
Tejas Patil 02c274eaba [SPARK-13450] Introduce ExternalAppendOnlyUnsafeRowArray. Change CartesianProductExec, SortMergeJoin, WindowExec to use it
## What issue does this PR address ?

Jira: https://issues.apache.org/jira/browse/SPARK-13450

In `SortMergeJoinExec`, rows of the right relation having the same value for a join key are buffered in-memory. In case of skew, this causes OOMs (see comments in SPARK-13450 for more details). Heap dump from a failed job confirms this : https://issues.apache.org/jira/secure/attachment/12846382/heap-dump-analysis.png . While its possible to increase the heap size to workaround, Spark should be resilient to such issues as skews can happen arbitrarily.

## Change proposed in this pull request

- Introduces `ExternalAppendOnlyUnsafeRowArray`
  - It holds `UnsafeRow`s in-memory upto a certain threshold.
  - After the threshold is hit, it switches to `UnsafeExternalSorter` which enables spilling of the rows to disk. It does NOT sort the data.
  - Allows iterating the array multiple times. However, any alteration to the array (using `add` or `clear`) will invalidate the existing iterator(s)
- `WindowExec` was already using `UnsafeExternalSorter` to support spilling. Changed it to use the new array
- Changed `SortMergeJoinExec` to use the new array implementation
  - NOTE: I have not changed FULL OUTER JOIN to use this new array implementation. Changing that will need more surgery and I will rather put up a separate PR for that once this gets in.
- Changed `CartesianProductExec` to use the new array implementation

#### Note for reviewers

The diff can be divided into 3 parts. My motive behind having all the changes in a single PR was to demonstrate that the API is sane and supports 2 use cases. If reviewing as 3 separate PRs would help, I am happy to make the split.

## How was this patch tested ?

#### Unit testing
- Added unit tests `ExternalAppendOnlyUnsafeRowArray` to validate all its APIs and access patterns
- Added unit test for `SortMergeExec`
  - with and without spill for inner join, left outer join, right outer join to confirm that the spill threshold config behaves as expected and output is as expected.
  - This PR touches the scanning logic in `SortMergeExec` for _all_ joins (except FULL OUTER JOIN). However, I expect existing test cases to cover that there is no regression in correctness.
- Added unit test for `WindowExec` to check behavior of spilling and correctness of results.

#### Stress testing
- Confirmed that OOM is gone by running against a production job which used to OOM
- Since I cannot share details about prod workload externally, created synthetic data to mimic the issue. Ran before and after the fix to demonstrate the issue and query success with this PR

Generating the synthetic data

```
./bin/spark-shell --driver-memory=6G

import org.apache.spark.sql._
val hc = SparkSession.builder.master("local").getOrCreate()

hc.sql("DROP TABLE IF EXISTS spark_13450_large_table").collect
hc.sql("DROP TABLE IF EXISTS spark_13450_one_row_table").collect

val df1 = (0 until 1).map(i => ("10", "100", i.toString, (i * 2).toString)).toDF("i", "j", "str1", "str2")
df1.write.format("org.apache.spark.sql.hive.orc.OrcFileFormat").bucketBy(100, "i", "j").sortBy("i", "j").saveAsTable("spark_13450_one_row_table")

val df2 = (0 until 3000000).map(i => ("10", "100", i.toString, (i * 2).toString)).toDF("i", "j", "str1", "str2")
df2.write.format("org.apache.spark.sql.hive.orc.OrcFileFormat").bucketBy(100, "i", "j").sortBy("i", "j").saveAsTable("spark_13450_large_table")
```

Ran this against trunk VS local build with this PR. OOM repros with trunk and with the fix this query runs fine.

```
./bin/spark-shell --driver-java-options="-XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=/tmp/spark.driver.heapdump.hprof"

import org.apache.spark.sql._
val hc = SparkSession.builder.master("local").getOrCreate()
hc.sql("SET spark.sql.autoBroadcastJoinThreshold=1")
hc.sql("SET spark.sql.sortMergeJoinExec.buffer.spill.threshold=10000")

hc.sql("DROP TABLE IF EXISTS spark_13450_result").collect
hc.sql("""
  CREATE TABLE spark_13450_result
  AS
  SELECT
    a.i AS a_i, a.j AS a_j, a.str1 AS a_str1, a.str2 AS a_str2,
    b.i AS b_i, b.j AS b_j, b.str1 AS b_str1, b.str2 AS b_str2
  FROM
    spark_13450_one_row_table a
  JOIN
    spark_13450_large_table b
  ON
    a.i=b.i AND
    a.j=b.j
""")
```

## Performance comparison

### Macro-benchmark

I ran a SMB join query over two real world tables (2 trillion rows (40 TB) and 6 million rows (120 GB)). Note that this dataset does not have skew so no spill happened. I saw improvement in CPU time by 2-4% over version without this PR. This did not add up as I was expected some regression. I think allocating array of capacity of 128 at the start (instead of starting with default size 16) is the sole reason for the perf. gain : https://github.com/tejasapatil/spark/blob/SPARK-13450_smb_buffer_oom/sql/core/src/main/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArray.scala#L43 . I could remove that and rerun, but effectively the change will be deployed in this form and I wanted to see the effect of it over large workload.

### Micro-benchmark

Two types of benchmarking can be found in `ExternalAppendOnlyUnsafeRowArrayBenchmark`:

[A] Comparing `ExternalAppendOnlyUnsafeRowArray` against raw `ArrayBuffer` when all rows fit in-memory and there is no spill

```
Array with 1000 rows:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
ArrayBuffer                                   7821 / 7941         33.5          29.8       1.0X
ExternalAppendOnlyUnsafeRowArray              8798 / 8819         29.8          33.6       0.9X

Array with 30000 rows:                   Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
ArrayBuffer                                 19200 / 19206         25.6          39.1       1.0X
ExternalAppendOnlyUnsafeRowArray            19558 / 19562         25.1          39.8       1.0X

Array with 100000 rows:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
ArrayBuffer                                   5949 / 6028         17.2          58.1       1.0X
ExternalAppendOnlyUnsafeRowArray              6078 / 6138         16.8          59.4       1.0X
```

[B] Comparing `ExternalAppendOnlyUnsafeRowArray` against raw `UnsafeExternalSorter` when there is spilling of data

```
Spilling with 1000 rows:                 Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
UnsafeExternalSorter                          9239 / 9470         28.4          35.2       1.0X
ExternalAppendOnlyUnsafeRowArray              8857 / 8909         29.6          33.8       1.0X

Spilling with 10000 rows:                Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
UnsafeExternalSorter                             4 /    5         39.3          25.5       1.0X
ExternalAppendOnlyUnsafeRowArray                 5 /    6         29.8          33.5       0.8X
```

Author: Tejas Patil <tejasp@fb.com>

Closes #16909 from tejasapatil/SPARK-13450_smb_buffer_oom.
2017-03-15 20:18:39 +01:00
jiangxingbo ee36bc1c90 [SPARK-19877][SQL] Restrict the nested level of a view
## What changes were proposed in this pull request?

We should restrict the nested level of a view, to avoid stack overflow exception during the view resolution.

## How was this patch tested?

Add new test case in `SQLViewSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #17241 from jiangxb1987/view-depth.
2017-03-14 23:57:54 -07:00