Commit graph

2163 commits

Author SHA1 Message Date
Tathagata Das 607a1e63db [SPARK-18894][SS] Fix event time watermark delay threshold specified in months or years
## What changes were proposed in this pull request?

Two changes
- Fix how delays specified in months and years are translated to milliseconds
- Following up on #16258, not show watermark when there is no watermarking in the query

## How was this patch tested?
Updated and new unit tests

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

Closes #16304 from tdas/SPARK-18834-1.
2016-12-21 10:44:20 -08:00
Burak Yavuz b2dd8ec6b2 [SPARK-18900][FLAKY-TEST] StateStoreSuite.maintenance
## What changes were proposed in this pull request?

It was pretty flaky before 10 days ago.
https://spark-tests.appspot.com/test-details?suite_name=org.apache.spark.sql.execution.streaming.state.StateStoreSuite&test_name=maintenance

Since no code changes went into this code path to not be so flaky, I'm just increasing the timeouts such that load related flakiness shouldn't be a problem. As you may see from the testing, I haven't been able to reproduce it.

## How was this patch tested?

2000 retries 5 times

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #16314 from brkyvz/maint-flaky.
2016-12-20 19:28:18 -08:00
Burak Yavuz caed89321f [SPARK-18927][SS] MemorySink for StructuredStreaming can't recover from checkpoint if location is provided in SessionConf
## What changes were proposed in this pull request?

Checkpoint Location can be defined for a StructuredStreaming on a per-query basis by the `DataStreamWriter` options, but it can also be provided through SparkSession configurations. It should be able to recover in both cases when the OutputMode is Complete for MemorySinks.

## How was this patch tested?

Unit tests

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #16342 from brkyvz/chk-rec.
2016-12-20 14:19:35 -08:00
Wenchen Fan f923c849e5 [SPARK-18899][SPARK-18912][SPARK-18913][SQL] refactor the error checking when append data to an existing table
## What changes were proposed in this pull request?

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

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

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

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

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16313 from cloud-fan/bug1.
2016-12-19 20:03:33 -08:00
Shixiong Zhu 4faa8a3ec0 [SPARK-18904][SS][TESTS] Merge two FileStreamSourceSuite files
## What changes were proposed in this pull request?

Merge two FileStreamSourceSuite files into one file.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16315 from zsxwing/FileStreamSourceSuite.
2016-12-16 15:04:11 -08:00
Takeshi YAMAMURO dc2a4d4ad4 [SPARK-18108][SQL] Fix a schema inconsistent bug that makes a parquet reader fail to read data
## What changes were proposed in this pull request?
A vectorized parquet reader fails to read column data if data schema and partition schema overlap with each other and inferred types in the partition schema differ from ones in the data schema. An example code to reproduce this bug is as follows;

```
scala> case class A(a: Long, b: Int)
scala> val as = Seq(A(1, 2))
scala> spark.createDataFrame(as).write.parquet("/data/a=1/")
scala> val df = spark.read.parquet("/data/")
scala> df.printSchema
root
 |-- a: long (nullable = true)
 |-- b: integer (nullable = true)
scala> df.collect
java.lang.NullPointerException
        at org.apache.spark.sql.execution.vectorized.OnHeapColumnVector.getLong(OnHeapColumnVector.java:283)
        at org.apache.spark.sql.execution.vectorized.ColumnarBatch$Row.getLong(ColumnarBatch.java:191)
        at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source)
        at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source)
        at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
        at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
```
The root cause is that a logical layer (`HadoopFsRelation`) and a physical layer (`VectorizedParquetRecordReader`) have a different assumption on partition schema; the logical layer trusts the data schema to infer the type the overlapped partition columns, and, on the other hand, the physical layer trusts partition schema which is inferred from path string. To fix this bug, this pr simply updates `HadoopFsRelation.schema` to respect the partition columns position in data schema and respect the partition columns type in partition schema.

## How was this patch tested?
Add tests in `ParquetPartitionDiscoverySuite`

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

Closes #16030 from maropu/SPARK-18108.
2016-12-16 22:44:42 +08:00
Shixiong Zhu d7f3058e17 [SPARK-18850][SS] Make StreamExecution and progress classes serializable
## What changes were proposed in this pull request?

This PR adds StreamingQueryWrapper to make StreamExecution and progress classes serializable because it is too easy for it to get captured with normal usage. If StreamingQueryWrapper gets captured in a closure but no place calls its methods, it should not fail the Spark tasks. However if its methods are called, then this PR will throw a better message.

## How was this patch tested?

`test("StreamingQuery should be Serializable but cannot be used in executors")`
`test("progress classes should be Serializable")`

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16272 from zsxwing/SPARK-18850.
2016-12-16 00:42:39 -08:00
Burak Yavuz 9c7f83b028 [SPARK-18868][FLAKY-TEST] Deflake StreamingQueryListenerSuite: single listener, check trigger...
## What changes were proposed in this pull request?

Use `recentProgress` instead of `lastProgress` and filter out last non-zero value. Also add eventually to the latest assertQuery similar to first `assertQuery`

## How was this patch tested?

Ran test 1000 times

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #16287 from brkyvz/SPARK-18868.
2016-12-15 15:46:03 -08:00
Shixiong Zhu 68a6dc974b [SPARK-18826][SS] Add 'latestFirst' option to FileStreamSource
## What changes were proposed in this pull request?

When starting a stream with a lot of backfill and maxFilesPerTrigger, the user could often want to start with most recent files first. This would let you keep low latency for recent data and slowly backfill historical data.

This PR adds a new option `latestFirst` to control this behavior. When it's true, `FileStreamSource` will sort the files by the modified time from latest to oldest, and take the first `maxFilesPerTrigger` files as a new batch.

## How was this patch tested?

The added test.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16251 from zsxwing/newest-first.
2016-12-15 13:17:51 -08:00
jiangxingbo 01e14bf303 [SPARK-17910][SQL] Allow users to update the comment of a column
## What changes were proposed in this pull request?

Right now, once a user set the comment of a column with create table command, he/she cannot update the comment. It will be useful to provide a public interface (e.g. SQL) to do that.

This PR implements the following SQL statement:
```
ALTER TABLE table [PARTITION partition_spec]
CHANGE [COLUMN] column_old_name column_new_name column_dataType
[COMMENT column_comment]
[FIRST | AFTER column_name];
```

For further expansion, we could support alter `name`/`dataType`/`index` of a column too.

## How was this patch tested?

Add new test cases in `ExternalCatalogSuite` and `SessionCatalogSuite`.
Add sql file test for `ALTER TABLE CHANGE COLUMN` statement.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15717 from jiangxb1987/change-column.
2016-12-15 10:09:42 -08:00
Wenchen Fan d6f11a12a1 [SPARK-18856][SQL] non-empty partitioned table should not report zero size
## What changes were proposed in this pull request?

In `DataSource`, if the table is not analyzed, we will use 0 as the default value for table size. This is dangerous, we may broadcast a large table and cause OOM. We should use `defaultSizeInBytes` instead.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16280 from cloud-fan/bug.
2016-12-14 21:03:56 -08:00
Reynold Xin ffdd1fcd1e [SPARK-18854][SQL] numberedTreeString and apply(i) inconsistent for subqueries
## What changes were proposed in this pull request?
This is a bug introduced by subquery handling. numberedTreeString (which uses generateTreeString under the hood) numbers trees including innerChildren (used to print subqueries), but apply (which uses getNodeNumbered) ignores innerChildren. As a result, apply(i) would return the wrong plan node if there are subqueries.

This patch fixes the bug.

## How was this patch tested?
Added a test case in SubquerySuite.scala to test both the depth-first traversal of numbering as well as making sure the two methods are consistent.

Author: Reynold Xin <rxin@databricks.com>

Closes #16277 from rxin/SPARK-18854.
2016-12-14 16:12:14 -08:00
Shixiong Zhu 1ac6567bdb [SPARK-18852][SS] StreamingQuery.lastProgress should be null when recentProgress is empty
## What changes were proposed in this pull request?

Right now `StreamingQuery.lastProgress` throws NoSuchElementException and it's hard to be used in Python since Python user will just see Py4jError.

This PR just makes it return null instead.

## How was this patch tested?

`test("lastProgress should be null when recentProgress is empty")`

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16273 from zsxwing/SPARK-18852.
2016-12-14 13:36:41 -08:00
hyukjinkwon 89ae26dcdb [SPARK-18753][SQL] Keep pushed-down null literal as a filter in Spark-side post-filter for FileFormat datasources
## What changes were proposed in this pull request?

Currently, `FileSourceStrategy` does not handle the case when the pushed-down filter is `Literal(null)` and removes it at the post-filter in Spark-side.

For example, the codes below:

```scala
val df = Seq(Tuple1(Some(true)), Tuple1(None), Tuple1(Some(false))).toDF()
df.filter($"_1" === "true").explain(true)
```

shows it keeps `null` properly.

```
== Parsed Logical Plan ==
'Filter ('_1 = true)
+- LocalRelation [_1#17]

== Analyzed Logical Plan ==
_1: boolean
Filter (cast(_1#17 as double) = cast(true as double))
+- LocalRelation [_1#17]

== Optimized Logical Plan ==
Filter (isnotnull(_1#17) && null)
+- LocalRelation [_1#17]

== Physical Plan ==
*Filter (isnotnull(_1#17) && null)       << Here `null` is there
+- LocalTableScan [_1#17]
```

However, when we read it back from Parquet,

```scala
val path = "/tmp/testfile"
df.write.parquet(path)
spark.read.parquet(path).filter($"_1" === "true").explain(true)
```

`null` is removed at the post-filter.

```
== Parsed Logical Plan ==
'Filter ('_1 = true)
+- Relation[_1#11] parquet

== Analyzed Logical Plan ==
_1: boolean
Filter (cast(_1#11 as double) = cast(true as double))
+- Relation[_1#11] parquet

== Optimized Logical Plan ==
Filter (isnotnull(_1#11) && null)
+- Relation[_1#11] parquet

== Physical Plan ==
*Project [_1#11]
+- *Filter isnotnull(_1#11)       << Here `null` is missing
   +- *FileScan parquet [_1#11] Batched: true, Format: ParquetFormat, Location: InMemoryFileIndex[file:/tmp/testfile], PartitionFilters: [null], PushedFilters: [IsNotNull(_1)], ReadSchema: struct<_1:boolean>
```

This PR fixes it to keep it properly. In more details,

```scala
val partitionKeyFilters =
  ExpressionSet(normalizedFilters.filter(_.references.subsetOf(partitionSet)))
```

This keeps this `null` in `partitionKeyFilters` as `Literal` always don't have `children` and `references` is being empty  which is always the subset of `partitionSet`.

And then in

```scala
val afterScanFilters = filterSet -- partitionKeyFilters
```

`null` is always removed from the post filter. So, if the referenced fields are empty, it should be applied into data columns too.

After this PR, it becomes as below:

```
== Parsed Logical Plan ==
'Filter ('_1 = true)
+- Relation[_1#276] parquet

== Analyzed Logical Plan ==
_1: boolean
Filter (cast(_1#276 as double) = cast(true as double))
+- Relation[_1#276] parquet

== Optimized Logical Plan ==
Filter (isnotnull(_1#276) && null)
+- Relation[_1#276] parquet

== Physical Plan ==
*Project [_1#276]
+- *Filter (isnotnull(_1#276) && null)
   +- *FileScan parquet [_1#276] Batched: true, Format: ParquetFormat, Location: InMemoryFileIndex[file:/private/var/folders/9j/gf_c342d7d150mwrxvkqnc180000gn/T/spark-a5d59bdb-5b..., PartitionFilters: [null], PushedFilters: [IsNotNull(_1)], ReadSchema: struct<_1:boolean>
```

## How was this patch tested?

Unit test in `FileSourceStrategySuite`

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16184 from HyukjinKwon/SPARK-18753.
2016-12-14 11:29:11 -08:00
hyukjinkwon c6b8eb71a9
[SPARK-18842][TESTS][LAUNCHER] De-duplicate paths in classpaths in commands for local-cluster mode to work around the path length limitation on Windows
## What changes were proposed in this pull request?

Currently, some tests are being failed and hanging on Windows due to this problem. For the reason in SPARK-18718, some tests using `local-cluster` mode were disabled on Windows due to the length limitation by paths given to classpaths.

The limitation seems roughly 32K (see the [blog in MS](https://blogs.msdn.microsoft.com/oldnewthing/20031210-00/?p=41553/) and [another reference](https://support.thoughtworks.com/hc/en-us/articles/213248526-Getting-around-maximum-command-line-length-is-32767-characters-on-Windows)) but in `local-cluster` mode, executors were being launched as processes with the command such as [here](https://gist.github.com/HyukjinKwon/5bc81061c250d4af5a180869b59d42ea) in (only) tests.

This length is roughly 40K due to the classpaths given to `java` command. However, it seems duplicates are almost half of them. So, if we deduplicate the paths, it seems reduced to roughly 20K with the command, [here](https://gist.github.com/HyukjinKwon/dad0c8db897e5e094684a2dc6a417790).

Maybe, we should consider as some more paths are added in the future but it seems better than disabling all the tests for now with minimised changes.

Therefore, this PR proposes to deduplicate the paths in classpaths in case of launching executors as processes in `local-cluster` mode.

## How was this patch tested?

Existing tests in `ShuffleSuite` and `BroadcastJoinSuite` manually via AppVeyor

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16266 from HyukjinKwon/disable-local-cluster-tests.
2016-12-14 19:24:24 +00:00
Nattavut Sutyanyong cccd64393e [SPARK-18814][SQL] CheckAnalysis rejects TPCDS query 32
## What changes were proposed in this pull request?
Move the checking of GROUP BY column in correlated scalar subquery from CheckAnalysis
to Analysis to fix a regression caused by SPARK-18504.

This problem can be reproduced with a simple script now.

Seq((1,1)).toDF("pk","pv").createOrReplaceTempView("p")
Seq((1,1)).toDF("ck","cv").createOrReplaceTempView("c")
sql("select * from p,c where p.pk=c.ck and c.cv = (select avg(c1.cv) from c c1 where c1.ck = p.pk)").show

The requirements are:
1. We need to reference the same table twice in both the parent and the subquery. Here is the table c.
2. We need to have a correlated predicate but to a different table. Here is from c (as c1) in the subquery to p in the parent.
3. We will then "deduplicate" c1.ck in the subquery to `ck#<n1>#<n2>` at `Project` above `Aggregate` of `avg`. Then when we compare `ck#<n1>#<n2>` and the original group by column `ck#<n1>` by their canonicalized form, which is #<n2> != #<n1>. That's how we trigger the exception added in SPARK-18504.

## How was this patch tested?

SubquerySuite and a simplified version of TPCDS-Q32

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

Closes #16246 from nsyca/18814.
2016-12-14 11:09:31 +01:00
Weiqing Yang ae5b2d3e46 [SPARK-18746][SQL] Add implicit encoder for BigDecimal, timestamp and date
## What changes were proposed in this pull request?
Add implicit encoders for BigDecimal, timestamp and date.

## How was this patch tested?
Add an unit test.  Pass build, unit tests, and some tests below .
Before:
```
scala> spark.createDataset(Seq(new java.math.BigDecimal(10)))
<console>:24: error: Unable to find encoder for type stored in a Dataset.  Primitive types (Int, String, etc) and Product types (case classes) are supported by importing spark.implicits._  Support for serializing other types will be added in future releases.
       spark.createDataset(Seq(new java.math.BigDecimal(10)))
                          ^

scala>
```
After:
```
scala> spark.createDataset(Seq(new java.math.BigDecimal(10)))
res0: org.apache.spark.sql.Dataset[java.math.BigDecimal] = [value: decimal(38,18)]
```

Author: Weiqing Yang <yangweiqing001@gmail.com>

Closes #16176 from weiqingy/SPARK-18746.
2016-12-14 09:48:38 +08:00
Tathagata Das c68fb426d4 [SPARK-18834][SS] Expose event time stats through StreamingQueryProgress
## What changes were proposed in this pull request?

- Changed `StreamingQueryProgress.watermark` to `StreamingQueryProgress.queryTimestamps` which is a `Map[String, String]` containing the following keys: "eventTime.max", "eventTime.min", "eventTime.avg", "processingTime", "watermark". All of them UTC formatted strings.

- Renamed `StreamingQuery.timestamp` to `StreamingQueryProgress.triggerTimestamp` to differentiate from `queryTimestamps`. It has the timestamp of when the trigger was started.

## How was this patch tested?

Updated tests

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

Closes #16258 from tdas/SPARK-18834.
2016-12-13 14:14:25 -08:00
jiangxingbo 5572ccf86b [SPARK-17932][SQL][FOLLOWUP] Change statement SHOW TABLES EXTENDED to SHOW TABLE EXTENDED
## What changes were proposed in this pull request?

Change the statement `SHOW TABLES [EXTENDED] [(IN|FROM) database_name] [[LIKE] 'identifier_with_wildcards'] [PARTITION(partition_spec)]` to the following statements:

- SHOW TABLES [(IN|FROM) database_name] [[LIKE] 'identifier_with_wildcards']
- SHOW TABLE EXTENDED [(IN|FROM) database_name] LIKE 'identifier_with_wildcards' [PARTITION(partition_spec)]

After this change, the statements `SHOW TABLE/SHOW TABLES` have the same syntax with that HIVE has.

## How was this patch tested?
Modified the test sql file `show-tables.sql`;
Modified the test suite `DDLSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #16262 from jiangxb1987/show-table-extended.
2016-12-13 19:04:34 +01:00
Andrew Ray 46d30ac484 [SPARK-18717][SQL] Make code generation for Scala Map work with immutable.Map also
## What changes were proposed in this pull request?

Fixes compile errors in generated code when user has case class with a `scala.collections.immutable.Map` instead of a `scala.collections.Map`. Since ArrayBasedMapData.toScalaMap returns the immutable version we can make it work with both.

## How was this patch tested?

Additional unit tests.

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

Closes #16161 from aray/fix-map-codegen.
2016-12-13 15:49:22 +08:00
Shixiong Zhu 417e45c584 [SPARK-18796][SS] StreamingQueryManager should not block when starting a query
## What changes were proposed in this pull request?

Major change in this PR:
- Add `pendingQueryNames` and `pendingQueryIds` to track that are going to start but not yet put into `activeQueries` so that we don't need to hold a lock when starting a query.

Minor changes:
- Fix a potential NPE when the user sets `checkpointLocation` using SQLConf but doesn't specify a query name.
- Add missing docs in `StreamingQueryListener`

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16220 from zsxwing/SPARK-18796.
2016-12-12 22:31:22 -08:00
Tyson Condie 83a42897ae [SPARK-18790][SS] Keep a general offset history of stream batches
## What changes were proposed in this pull request?

Instead of only keeping the minimum number of offsets around, we should keep enough information to allow us to roll back n batches and reexecute the stream starting from a given point. In particular, we should create a config in SQLConf, spark.sql.streaming.retainedBatches that defaults to 100 and ensure that we keep enough log files in the following places to roll back the specified number of batches:
the offsets that are present in each batch
versions of the state store
the files lists stored for the FileStreamSource
the metadata log stored by the FileStreamSink

marmbrus zsxwing

## How was this patch tested?

The following tests were added.

### StreamExecution offset metadata
Test added to StreamingQuerySuite that ensures offset metadata is garbage collected according to minBatchesRetain

### CompactibleFileStreamLog
Tests added in CompactibleFileStreamLogSuite to ensure that logs are purged starting before the first compaction file that proceeds the current batch id - minBatchesToRetain.

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

Author: Tyson Condie <tcondie@gmail.com>

Closes #16219 from tcondie/offset_hist.
2016-12-11 23:38:31 -08:00
wangzhenhua a29ee55aaa [SPARK-18815][SQL] Fix NPE when collecting column stats for string/binary column having only null values
## What changes were proposed in this pull request?

During column stats collection, average and max length will be null if a column of string/binary type has only null values. To fix this, I use default size when avg/max length is null.

## How was this patch tested?

Add a test for handling null columns

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #16243 from wzhfy/nullStats.
2016-12-10 21:25:29 -08:00
hyukjinkwon e094d01156
[SPARK-18803][TESTS] Fix JarEntry-related & path-related test failures and skip some tests by path length limitation on Windows
## What changes were proposed in this pull request?

This PR proposes to fix some tests being failed on Windows as below for several problems.

### Incorrect path handling

- FileSuite
  ```
  [info] - binary file input as byte array *** FAILED *** (500 milliseconds)
  [info]   "file:/C:/projects/spark/target/tmp/spark-e7c3a3b8-0a4b-4a7f-9ebe-7c4883e48624/record-bytestream-00000.bin" did not contain "C:\projects\spark\target\tmp\spark-e7c3a3b8-0a4b-4a7f-9ebe-7c4883e48624\record-bytestream-00000.bin" (FileSuite.scala:258)
  [info]   org.scalatest.exceptions.TestFailedException:
  [info]   at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500)
  ...
  ```
  ```
  [info] - Get input files via old Hadoop API *** FAILED *** (1 second, 94 milliseconds)
  [info]   Set("/C:/projects/spark/target/tmp/spark-cf5b1f8b-c5ed-43e0-8d17-546ebbfa8200/output/part-00000", "/C:/projects/spark/target/tmp/spark-cf5b1f8b-c5ed-43e0-8d17-546ebbfa8200/output/part-00001") did not equal Set("C:\projects\spark\target\tmp\spark-cf5b1f8b-c5ed-43e0-8d17-546ebbfa8200\output/part-00000", "C:\projects\spark\target\tmp\spark-cf5b1f8b-c5ed-43e0-8d17-546ebbfa8200\output/part-00001") (FileSuite.scala:535)
  [info]   org.scalatest.exceptions.TestFailedException:
  [info]   at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500)
  ...
  ```

  ```
  [info] - Get input files via new Hadoop API *** FAILED *** (313 milliseconds)
  [info]   Set("/C:/projects/spark/target/tmp/spark-12bc1540-1111-4df6-9c4d-79e0e614407c/output/part-00000", "/C:/projects/spark/target/tmp/spark-12bc1540-1111-4df6-9c4d-79e0e614407c/output/part-00001") did not equal Set("C:\projects\spark\target\tmp\spark-12bc1540-1111-4df6-9c4d-79e0e614407c\output/part-00000", "C:\projects\spark\target\tmp\spark-12bc1540-1111-4df6-9c4d-79e0e614407c\output/part-00001") (FileSuite.scala:549)
  [info]   org.scalatest.exceptions.TestFailedException:
  ...
  ```

- TaskResultGetterSuite

  ```
  [info] - handling results larger than max RPC message size *** FAILED *** (1 second, 579 milliseconds)
  [info]   1 did not equal 0 Expect result to be removed from the block manager. (TaskResultGetterSuite.scala:129)
  [info]   org.scalatest.exceptions.TestFailedException:
  [info]   ...
  [info]   Cause: java.net.URISyntaxException: Illegal character in path at index 12: string:///C:\projects\spark\target\tmp\spark-93c485af-68da-440f-a907-aac7acd5fc25\repro\MyException.java
  [info]   at java.net.URI$Parser.fail(URI.java:2848)
  [info]   at java.net.URI$Parser.checkChars(URI.java:3021)
  ...
  ```
  ```
  [info] - failed task deserialized with the correct classloader (SPARK-11195) *** FAILED *** (0 milliseconds)
  [info]   java.lang.IllegalArgumentException: Illegal character in path at index 12: string:///C:\projects\spark\target\tmp\spark-93c485af-68da-440f-a907-aac7acd5fc25\repro\MyException.java
  [info]   at java.net.URI.create(URI.java:852)
  ...
  ```

- SparkSubmitSuite

  ```
  [info]   java.lang.IllegalArgumentException: Illegal character in path at index 12: string:///C:\projects\spark\target\tmp\1481210831381-0\870903339\MyLib.java
  [info]   at java.net.URI.create(URI.java:852)
  [info]   at org.apache.spark.TestUtils$.org$apache$spark$TestUtils$$createURI(TestUtils.scala:112)
  ...
  ```

### Incorrect separate for JarEntry

After the path fix from above, then `TaskResultGetterSuite` throws another exception as below:

```
[info] - failed task deserialized with the correct classloader (SPARK-11195) *** FAILED *** (907 milliseconds)
[info]   java.lang.ClassNotFoundException: repro.MyException
[info]   at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
...
```

This is because `Paths.get` concatenates the given paths to an OS-specific path (Windows `\` and Linux `/`). However, for `JarEntry` we should comply ZIP specification meaning it should be always `/` according to ZIP specification.

See `4.4.17 file name: (Variable)` in https://pkware.cachefly.net/webdocs/casestudies/APPNOTE.TXT

### Long path problem on Windows

Some tests in `ShuffleSuite` via `ShuffleNettySuite` were skipped due to the same reason with SPARK-18718

## How was this patch tested?

Manually via AppVeyor.

**Before**

- `FileSuite`, `TaskResultGetterSuite`,`SparkSubmitSuite`
  https://ci.appveyor.com/project/spark-test/spark/build/164-tmp-windows-base (please grep each to check each)
- `ShuffleSuite`
  https://ci.appveyor.com/project/spark-test/spark/build/157-tmp-windows-base

**After**

- `FileSuite`
  https://ci.appveyor.com/project/spark-test/spark/build/166-FileSuite
- `TaskResultGetterSuite`
  https://ci.appveyor.com/project/spark-test/spark/build/173-TaskResultGetterSuite
- `SparkSubmitSuite`
  https://ci.appveyor.com/project/spark-test/spark/build/167-SparkSubmitSuite
- `ShuffleSuite`
  https://ci.appveyor.com/project/spark-test/spark/build/176-ShuffleSuite

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16234 from HyukjinKwon/test-errors-windows.
2016-12-10 19:55:22 +00:00
gatorsmile 422a45cf04 [SPARK-18766][SQL] Push Down Filter Through BatchEvalPython (Python UDF)
### What changes were proposed in this pull request?
Currently, when users use Python UDF in Filter, BatchEvalPython is always generated below FilterExec. However, not all the predicates need to be evaluated after Python UDF execution. Thus, this PR is to push down the determinisitc predicates through `BatchEvalPython`.
```Python
>>> df = spark.createDataFrame([(1, "1"), (2, "2"), (1, "2"), (1, "2")], ["key", "value"])
>>> from pyspark.sql.functions import udf, col
>>> from pyspark.sql.types import BooleanType
>>> my_filter = udf(lambda a: a < 2, BooleanType())
>>> sel = df.select(col("key"), col("value")).filter((my_filter(col("key"))) & (df.value < "2"))
>>> sel.explain(True)
```
Before the fix, the plan looks like
```
== Optimized Logical Plan ==
Filter ((isnotnull(value#1) && <lambda>(key#0L)) && (value#1 < 2))
+- LogicalRDD [key#0L, value#1]

== Physical Plan ==
*Project [key#0L, value#1]
+- *Filter ((isnotnull(value#1) && pythonUDF0#9) && (value#1 < 2))
   +- BatchEvalPython [<lambda>(key#0L)], [key#0L, value#1, pythonUDF0#9]
      +- Scan ExistingRDD[key#0L,value#1]
```

After the fix, the plan looks like
```
== Optimized Logical Plan ==
Filter ((isnotnull(value#1) && <lambda>(key#0L)) && (value#1 < 2))
+- LogicalRDD [key#0L, value#1]

== Physical Plan ==
*Project [key#0L, value#1]
+- *Filter pythonUDF0#9: boolean
   +- BatchEvalPython [<lambda>(key#0L)], [key#0L, value#1, pythonUDF0#9]
      +- *Filter (isnotnull(value#1) && (value#1 < 2))
         +- Scan ExistingRDD[key#0L,value#1]
```

### How was this patch tested?
Added both unit test cases for `BatchEvalPythonExec` and also add an end-to-end test case in Python test suite.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16193 from gatorsmile/pythonUDFPredicatePushDown.
2016-12-10 08:47:45 -08:00
Huaxin Gao c5172568b5 [SPARK-17460][SQL] Make sure sizeInBytes in Statistics will not overflow
## What changes were proposed in this pull request?

1. In SparkStrategies.canBroadcast, I will add the check   plan.statistics.sizeInBytes >= 0
2. In LocalRelations.statistics, when calculate the statistics, I will change the size to BigInt so it won't overflow.

## How was this patch tested?

I will add a test case to make sure the statistics.sizeInBytes won't overflow.

Author: Huaxin Gao <huaxing@us.ibm.com>

Closes #16175 from huaxingao/spark-17460.
2016-12-10 22:41:40 +08:00
Burak Yavuz 63c9159870 [SPARK-18811] StreamSource resolution should happen in stream execution thread
## What changes were proposed in this pull request?

When you start a stream, if we are trying to resolve the source of the stream, for example if we need to resolve partition columns, this could take a long time. This long execution time should not block the main thread where `query.start()` was called on. It should happen in the stream execution thread possibly before starting any triggers.

## How was this patch tested?

Unit test added. Made sure test fails with no code changes.

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #16238 from brkyvz/SPARK-18811.
2016-12-09 22:49:51 -08:00
Tathagata Das 458fa3325e [SPARK-18776][SS] Make Offset for FileStreamSource corrected formatted in json
## What changes were proposed in this pull request?

- Changed FileStreamSource to use new FileStreamSourceOffset rather than LongOffset. The field is named as `logOffset` to make it more clear that this is a offset in the file stream log.
- Fixed bug in FileStreamSourceLog, the field endId in the FileStreamSourceLog.get(startId, endId) was not being used at all. No test caught it earlier. Only my updated tests caught it.

Other minor changes
- Dont use batchId in the FileStreamSource, as calling it batch id is extremely miss leading. With multiple sources, it may happen that a new batch has no new data from a file source. So offset of FileStreamSource != batchId after that batch.

## How was this patch tested?

Updated unit test.

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

Closes #16205 from tdas/SPARK-18776.
2016-12-08 17:53:34 -08:00
Reynold Xin 5f894d23a5 [SPARK-18760][SQL] Consistent format specification for FileFormats
## What changes were proposed in this pull request?
This patch fixes the format specification in explain for file sources (Parquet and Text formats are the only two that are different from the rest):

Before:
```
scala> spark.read.text("test.text").explain()
== Physical Plan ==
*FileScan text [value#15] Batched: false, Format: org.apache.spark.sql.execution.datasources.text.TextFileFormatxyz, Location: InMemoryFileIndex[file:/scratch/rxin/spark/test.text], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<value:string>
```

After:
```
scala> spark.read.text("test.text").explain()
== Physical Plan ==
*FileScan text [value#15] Batched: false, Format: Text, Location: InMemoryFileIndex[file:/scratch/rxin/spark/test.text], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<value:string>
```

Also closes #14680.

## How was this patch tested?
Verified in spark-shell.

Author: Reynold Xin <rxin@databricks.com>

Closes #16187 from rxin/SPARK-18760.
2016-12-08 12:52:05 -08:00
hyukjinkwon 7f3c778fd0
[SPARK-18718][TESTS] Skip some test failures due to path length limitation and fix tests to pass on Windows
## What changes were proposed in this pull request?

There are some tests failed on Windows due to the wrong format of path and the limitation of path length as below:

This PR proposes both to fix the failed tests by fixing the path for the tests below:

- `InsertSuite`
  ```
  Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.sources.InsertSuite *** ABORTED *** (12 seconds, 547 milliseconds)
      org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-177945ef-9128-42b4-8c07-de31f78bbbd6;
      at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$14.apply(DataSource.scala:382)
      at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$14.apply(DataSource.scala:370)
      at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
  ```

- `PathOptionSuite`
  ```
  - path option also exist for write path *** FAILED *** (1 second, 93 milliseconds)
    "C:[projectsspark	arget	mp]spark-5ab34a58-df8d-..." did not equal "C:[\projects\spark\target\tmp\]spark-5ab34a58-df8d-..." (PathOptionSuite.scala:93)
    org.scalatest.exceptions.TestFailedException:
        at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500)
        at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555)
    ...
  ```

- `UDFSuite`
  ```
  - SPARK-8005 input_file_name *** FAILED *** (2 seconds, 234 milliseconds)
    "file:///C:/projects/spark/target/tmp/spark-e4e5720a-2006-48f9-8b11-797bf59794bf/part-00001-26fb05e4-603d-471d-ae9d-b9549e0c7765.snappy.parquet" did not contain "C:\projects\spark\target\tmp\spark-e4e5720a-2006-48f9-8b11-797bf59794bf" (UDFSuite.scala:67)
    org.scalatest.exceptions.TestFailedException:
      at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500)
      at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555)
    ...
  ```

and to skip the tests belows which are being failed on Windows due to path length limitation.

- `SparkLauncherSuite`
  ```
  Test org.apache.spark.launcher.SparkLauncherSuite.testChildProcLauncher failed: java.lang.AssertionError: expected:<0> but was:<1>, took 0.062 sec
    at org.apache.spark.launcher.SparkLauncherSuite.testChildProcLauncher(SparkLauncherSuite.java:177)
      ...
  ```

  The stderr from the process is `The filename or extension is too long` which is equivalent to the one below.

- `BroadcastJoinSuite`
  ```
  04:09:40.882 ERROR org.apache.spark.deploy.worker.ExecutorRunner: Error running executor
  java.io.IOException: Cannot run program "C:\Progra~1\Java\jdk1.8.0\bin\java" (in directory "C:\projects\spark\work\app-20161205040542-0000\51658"): CreateProcess error=206, The filename or extension is too long
      at java.lang.ProcessBuilder.start(ProcessBuilder.java:1048)
      at org.apache.spark.deploy.worker.ExecutorRunner.org$apache$spark$deploy$worker$ExecutorRunner$$fetchAndRunExecutor(ExecutorRunner.scala:167)
      at org.apache.spark.deploy.worker.ExecutorRunner$$anon$1.run(ExecutorRunner.scala:73)
  Caused by: java.io.IOException: CreateProcess error=206, The filename or extension is too long
      at java.lang.ProcessImpl.create(Native Method)
      at java.lang.ProcessImpl.<init>(ProcessImpl.java:386)
      at java.lang.ProcessImpl.start(ProcessImpl.java:137)
      at java.lang.ProcessBuilder.start(ProcessBuilder.java:1029)
      ... 2 more
  04:09:40.929 ERROR org.apache.spark.deploy.worker.ExecutorRunner: Error running executor

    (appearently infinite same error messages)

  ...
  ```

## How was this patch tested?

Manually tested via AppVeyor.

**Before**

`InsertSuite`: https://ci.appveyor.com/project/spark-test/spark/build/148-InsertSuite-pr
`PathOptionSuite`: https://ci.appveyor.com/project/spark-test/spark/build/139-PathOptionSuite-pr
`UDFSuite`: https://ci.appveyor.com/project/spark-test/spark/build/143-UDFSuite-pr
`SparkLauncherSuite`: https://ci.appveyor.com/project/spark-test/spark/build/141-SparkLauncherSuite-pr
`BroadcastJoinSuite`: https://ci.appveyor.com/project/spark-test/spark/build/145-BroadcastJoinSuite-pr

**After**

`PathOptionSuite`: https://ci.appveyor.com/project/spark-test/spark/build/140-PathOptionSuite-pr
`SparkLauncherSuite`: https://ci.appveyor.com/project/spark-test/spark/build/142-SparkLauncherSuite-pr
`UDFSuite`: https://ci.appveyor.com/project/spark-test/spark/build/144-UDFSuite-pr
`InsertSuite`: https://ci.appveyor.com/project/spark-test/spark/build/147-InsertSuite-pr
`BroadcastJoinSuite`: https://ci.appveyor.com/project/spark-test/spark/build/149-BroadcastJoinSuite-pr

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16147 from HyukjinKwon/fix-tests.
2016-12-08 23:02:05 +08:00
Tathagata Das 9ab725eabb [SPARK-18758][SS] StreamingQueryListener events from a StreamingQuery should be sent only to the listeners in the same session as the query
## What changes were proposed in this pull request?

Listeners added with `sparkSession.streams.addListener(l)` are added to a SparkSession. So events only from queries in the same session as a listener should be posted to the listener. Currently, all the events gets rerouted through the Spark's main listener bus, that is,
- StreamingQuery posts event to StreamingQueryListenerBus. Only the queries associated with the same session as the bus posts events to it.
- StreamingQueryListenerBus posts event to Spark's main LiveListenerBus as a SparkEvent.
- StreamingQueryListenerBus also subscribes to LiveListenerBus events thus getting back the posted event in a different thread.
- The received is posted to the registered listeners.

The problem is that *all StreamingQueryListenerBuses in all sessions* gets the events and posts them to their listeners. This is wrong.

In this PR, I solve it by making StreamingQueryListenerBus track active queries (by their runIds) when a query posts the QueryStarted event to the bus. This allows the rerouted events to be filtered using the tracked queries.

Note that this list needs to be maintained separately
from the `StreamingQueryManager.activeQueries` because a terminated query is cleared from
`StreamingQueryManager.activeQueries` as soon as it is stopped, but the this ListenerBus must
clear a query only after the termination event of that query has been posted lazily, much after the query has been terminated.

Credit goes to zsxwing for coming up with the initial idea.

## How was this patch tested?
Updated test harness code to use the correct session, and added new unit test.

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

Closes #16186 from tdas/SPARK-18758.
2016-12-07 19:23:27 -08:00
Nathan Howell bec0a9217b [SPARK-18654][SQL] Remove unreachable patterns in makeRootConverter
## What changes were proposed in this pull request?

`makeRootConverter` is only called with a `StructType` value. By making this method less general we can remove pattern matches, which are never actually hit outside of the test suite.

## How was this patch tested?

The existing tests.

Author: Nathan Howell <nhowell@godaddy.com>

Closes #16084 from NathanHowell/SPARK-18654.
2016-12-07 16:52:05 -08:00
Michael Armbrust 70b2bf717d [SPARK-18754][SS] Rename recentProgresses to recentProgress
Based on an informal survey, users find this option easier to understand / remember.

Author: Michael Armbrust <michael@databricks.com>

Closes #16182 from marmbrus/renameRecentProgress.
2016-12-07 15:36:29 -08:00
Shixiong Zhu edc87e1892 [SPARK-18588][TESTS] Fix flaky test: KafkaSourceStressForDontFailOnDataLossSuite
## What changes were proposed in this pull request?

Fixed the following failures:

```
org.scalatest.exceptions.TestFailedDueToTimeoutException: The code passed to eventually never returned normally. Attempted 3745 times over 1.0000790851666665 minutes. Last failure message: assertion failed: failOnDataLoss-0 not deleted after timeout.
```

```
sbt.ForkMain$ForkError: org.apache.spark.sql.streaming.StreamingQueryException: Query query-66 terminated with exception: null
	at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches(StreamExecution.scala:252)
	at org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:146)
Caused by: sbt.ForkMain$ForkError: java.lang.NullPointerException: null
	at java.util.ArrayList.addAll(ArrayList.java:577)
	at org.apache.kafka.clients.Metadata.getClusterForCurrentTopics(Metadata.java:257)
	at org.apache.kafka.clients.Metadata.update(Metadata.java:177)
	at org.apache.kafka.clients.NetworkClient$DefaultMetadataUpdater.handleResponse(NetworkClient.java:605)
	at org.apache.kafka.clients.NetworkClient$DefaultMetadataUpdater.maybeHandleCompletedReceive(NetworkClient.java:582)
	at org.apache.kafka.clients.NetworkClient.handleCompletedReceives(NetworkClient.java:450)
	at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:269)
	at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.clientPoll(ConsumerNetworkClient.java:360)
	at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:224)
	at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:192)
	at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.awaitPendingRequests(ConsumerNetworkClient.java:260)
	at org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureActiveGroup(AbstractCoordinator.java:222)
	at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.ensurePartitionAssignment(ConsumerCoordinator.java:366)
	at org.apache.kafka.clients.consumer.KafkaConsumer.pollOnce(KafkaConsumer.java:978)
	at org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:938)
	at
...
```

## How was this patch tested?

Tested in #16048 by running many times.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16109 from zsxwing/fix-kafka-flaky-test.
2016-12-07 13:47:44 -08:00
Andrew Ray f1fca81b16 [SPARK-17760][SQL] AnalysisException with dataframe pivot when groupBy column is not attribute
## What changes were proposed in this pull request?

Fixes AnalysisException for pivot queries that have group by columns that are expressions and not attributes by substituting the expressions output attribute in the second aggregation and final projection.

## How was this patch tested?

existing and additional unit tests

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

Closes #16177 from aray/SPARK-17760.
2016-12-07 04:44:14 -08:00
Tathagata Das 5c6bcdbda4 [SPARK-18671][SS][TEST-MAVEN] Follow up PR to fix test for Maven
## What changes were proposed in this pull request?

Maven compilation seem to not allow resource is sql/test to be easily referred to in kafka-0-10-sql tests. So moved the kafka-source-offset-version-2.1.0 from sql test resources to kafka-0-10-sql test resources.

## How was this patch tested?

Manually ran maven test

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

Closes #16183 from tdas/SPARK-18671-1.
2016-12-06 21:51:38 -08:00
Tathagata Das 539bb3cf95 [SPARK-18734][SS] Represent timestamp in StreamingQueryProgress as formatted string instead of millis
## What changes were proposed in this pull request?

Easier to read while debugging as a formatted string (in ISO8601 format) than in millis

## How was this patch tested?
Updated unit tests

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

Closes #16166 from tdas/SPARK-18734.
2016-12-06 17:04:26 -08:00
Tathagata Das 1ef6b296d7 [SPARK-18671][SS][TEST] Added tests to ensure stability of that all Structured Streaming log formats
## What changes were proposed in this pull request?

To be able to restart StreamingQueries across Spark version, we have already made the logs (offset log, file source log, file sink log) use json. We should added tests with actual json files in the Spark such that any incompatible changes in reading the logs is immediately caught. This PR add tests for FileStreamSourceLog, FileStreamSinkLog, and OffsetSeqLog.

## How was this patch tested?
new unit tests

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

Closes #16128 from tdas/SPARK-18671.
2016-12-06 13:05:22 -08:00
Shixiong Zhu 7863c62379 [SPARK-18721][SS] Fix ForeachSink with watermark + append
## What changes were proposed in this pull request?

Right now ForeachSink creates a new physical plan, so StreamExecution cannot retrieval metrics and watermark.

This PR changes ForeachSink to manually convert InternalRows to objects without creating a new plan.

## How was this patch tested?

`test("foreach with watermark: append")`.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16160 from zsxwing/SPARK-18721.
2016-12-05 20:35:24 -08:00
Shixiong Zhu 4af142f557 [SPARK-18722][SS] Move no data rate limit from StreamExecution to ProgressReporter
## What changes were proposed in this pull request?

Move no data rate limit from StreamExecution to ProgressReporter to make `recentProgresses` and listener events consistent.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16155 from zsxwing/SPARK-18722.
2016-12-05 18:51:07 -08:00
root 508de38c99 [SPARK-18555][SQL] DataFrameNaFunctions.fill miss up original values in long integers
## What changes were proposed in this pull request?

   DataSet.na.fill(0) used on a DataSet which has a long value column, it will change the original long value.

   The reason is that the type of the function fill's param is Double, and the numeric columns are always cast to double(`fillCol[Double](f, value)`) .
```
  def fill(value: Double, cols: Seq[String]): DataFrame = {
    val columnEquals = df.sparkSession.sessionState.analyzer.resolver
    val projections = df.schema.fields.map { f =>
      // Only fill if the column is part of the cols list.
      if (f.dataType.isInstanceOf[NumericType] && cols.exists(col => columnEquals(f.name, col))) {
        fillCol[Double](f, value)
      } else {
        df.col(f.name)
      }
    }
    df.select(projections : _*)
  }
```

 For example:
```
scala> val df = Seq[(Long, Long)]((1, 2), (-1, -2), (9123146099426677101L, 9123146560113991650L)).toDF("a", "b")
df: org.apache.spark.sql.DataFrame = [a: bigint, b: bigint]

scala> df.show
+-------------------+-------------------+
|                  a|                  b|
+-------------------+-------------------+
|                  1|                  2|
|                 -1|                 -2|
|9123146099426677101|9123146560113991650|
+-------------------+-------------------+

scala> df.na.fill(0).show
+-------------------+-------------------+
|                  a|                  b|
+-------------------+-------------------+
|                  1|                  2|
|                 -1|                 -2|
|9123146099426676736|9123146560113991680|
+-------------------+-------------------+
 ```

the original values changed [which is not we expected result]:
```
 9123146099426677101 -> 9123146099426676736
 9123146560113991650 -> 9123146560113991680
```

## How was this patch tested?

unit test added.

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

Closes #15994 from windpiger/nafillMissupOriginalValue.
2016-12-05 18:39:56 -08:00
Tathagata Das bb57bfe97d [SPARK-18657][SPARK-18668] Make StreamingQuery.id persists across restart and not auto-generate StreamingQuery.name
## What changes were proposed in this pull request?
Here are the major changes in this PR.
- Added the ability to recover `StreamingQuery.id` from checkpoint location, by writing the id to `checkpointLoc/metadata`.
- Added `StreamingQuery.runId` which is unique for every query started and does not persist across restarts. This is to identify each restart of a query separately (same as earlier behavior of `id`).
- Removed auto-generation of `StreamingQuery.name`. The purpose of name was to have the ability to define an identifier across restarts, but since id is precisely that, there is no need for a auto-generated name. This means name becomes purely cosmetic, and is null by default.
- Added `runId` to `StreamingQueryListener` events and `StreamingQueryProgress`.

Implementation details
- Renamed existing `StreamExecutionMetadata` to `OffsetSeqMetadata`, and moved it to the file `OffsetSeq.scala`, because that is what this metadata is tied to. Also did some refactoring to make the code cleaner (got rid of a lot of `.json` and `.getOrElse("{}")`).
- Added the `id` as the new `StreamMetadata`.
- When a StreamingQuery is created it gets or writes the `StreamMetadata` from `checkpointLoc/metadata`.
- All internal logging in `StreamExecution` uses `(name, id, runId)` instead of just `name`

TODO
- [x] Test handling of name=null in json generation of StreamingQueryProgress
- [x] Test handling of name=null in json generation of StreamingQueryListener events
- [x] Test python API of runId

## How was this patch tested?
Updated unit tests and new unit tests

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

Closes #16113 from tdas/SPARK-18657.
2016-12-05 18:17:38 -08:00
Wenchen Fan 01a7d33d08 [SPARK-18711][SQL] should disable subexpression elimination for LambdaVariable
## What changes were proposed in this pull request?

This is kind of a long-standing bug, it's hidden until https://github.com/apache/spark/pull/15780 , which may add `AssertNotNull` on top of `LambdaVariable` and thus enables subexpression elimination.

However, subexpression elimination will evaluate the common expressions at the beginning, which is invalid for `LambdaVariable`. `LambdaVariable` usually represents loop variable, which can't be evaluated ahead of the loop.

This PR skips expressions containing `LambdaVariable` when doing subexpression elimination.

## How was this patch tested?

updated test in `DatasetAggregatorSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16143 from cloud-fan/aggregator.
2016-12-05 11:37:13 -08:00
Shixiong Zhu 246012859f [SPARK-18694][SS] Add StreamingQuery.explain and exception to Python and fix StreamingQueryException
## What changes were proposed in this pull request?

- Add StreamingQuery.explain and exception to Python.
- Fix StreamingQueryException to not expose `OffsetSeq`.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16125 from zsxwing/py-streaming-explain.
2016-12-05 11:36:11 -08:00
Reynold Xin e9730b707d [SPARK-18702][SQL] input_file_block_start and input_file_block_length
## What changes were proposed in this pull request?
We currently have function input_file_name to get the path of the input file, but don't have functions to get the block start offset and length. This patch introduces two functions:

1. input_file_block_start: returns the file block start offset, or -1 if not available.

2. input_file_block_length: returns the file block length, or -1 if not available.

## How was this patch tested?
Updated existing test cases in ColumnExpressionSuite that covered input_file_name to also cover the two new functions.

Author: Reynold Xin <rxin@databricks.com>

Closes #16133 from rxin/SPARK-18702.
2016-12-04 21:51:10 -08:00
Eric Liang d9eb4c7215 [SPARK-18661][SQL] Creating a partitioned datasource table should not scan all files for table
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

Perf stat tests.

Author: Eric Liang <ekl@databricks.com>

Closes #16090 from ericl/spark-18661.
2016-12-04 20:44:04 +08:00
Nattavut Sutyanyong 4a3c09601b [SPARK-18582][SQL] Whitelist LogicalPlan operators allowed in correlated subqueries
## What changes were proposed in this pull request?

This fix puts an explicit list of operators that Spark supports for correlated subqueries.

## How was this patch tested?

Run sql/test, catalyst/test and add a new test case on Generate.

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

Closes #16046 from nsyca/spark18455.0.
2016-12-03 11:36:26 -08:00
Shixiong Zhu 56a503df5c [SPARK-18670][SS] Limit the number of StreamingQueryListener.StreamProgressEvent when there is no data
## What changes were proposed in this pull request?

This PR adds a sql conf `spark.sql.streaming.noDataReportInterval` to control how long to wait before outputing the next StreamProgressEvent when there is no data.

## How was this patch tested?

The added unit test.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16108 from zsxwing/SPARK-18670.
2016-12-02 12:42:47 -08:00
Eric Liang 7935c8470c [SPARK-18659][SQL] Incorrect behaviors in overwrite table for datasource tables
## What changes were proposed in this pull request?

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

cc yhuai cloud-fan

## How was this patch tested?

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

Author: Eric Liang <ekl@databricks.com>

Closes #16088 from ericl/spark-18659.
2016-12-02 21:59:02 +08:00
Dongjoon Hyun 55d528f2ba [SPARK-18419][SQL] JDBCRelation.insert should not remove Spark options
## What changes were proposed in this pull request?

Currently, `JDBCRelation.insert` removes Spark options too early by mistakenly using `asConnectionProperties`. Spark options like `numPartitions` should be passed into `DataFrameWriter.jdbc` correctly. This bug have been **hidden** because `JDBCOptions.asConnectionProperties` fails to filter out the mixed-case options. This PR aims to fix both.

**JDBCRelation.insert**
```scala
override def insert(data: DataFrame, overwrite: Boolean): Unit = {
  val url = jdbcOptions.url
  val table = jdbcOptions.table
- val properties = jdbcOptions.asConnectionProperties
+ val properties = jdbcOptions.asProperties
  data.write
    .mode(if (overwrite) SaveMode.Overwrite else SaveMode.Append)
    .jdbc(url, table, properties)
```

**JDBCOptions.asConnectionProperties**
```scala
scala> import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions
scala> import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
scala> new JDBCOptions(Map("url" -> "jdbc:mysql://localhost:3306/temp", "dbtable" -> "t1", "numPartitions" -> "10")).asConnectionProperties
res0: java.util.Properties = {numpartitions=10}
scala> new JDBCOptions(new CaseInsensitiveMap(Map("url" -> "jdbc:mysql://localhost:3306/temp", "dbtable" -> "t1", "numPartitions" -> "10"))).asConnectionProperties
res1: java.util.Properties = {numpartitions=10}
```

## How was this patch tested?

Pass the Jenkins with a new testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15863 from dongjoon-hyun/SPARK-18419.
2016-12-02 21:48:22 +08:00
Eric Liang 294163ee93 [SPARK-18679][SQL] Fix regression in file listing performance for non-catalog tables
## What changes were proposed in this pull request?

In Spark 2.1 ListingFileCatalog was significantly refactored (and renamed to InMemoryFileIndex). This introduced a regression where parallelism could only be introduced at the very top of the tree. However, in many cases (e.g. `spark.read.parquet(topLevelDir)`), the top of the tree is only a single directory.

This PR simplifies and fixes the parallel recursive listing code to allow parallelism to be introduced at any level during recursive descent (though note that once we decide to list a sub-tree in parallel, the sub-tree is listed in serial on executors).

cc mallman  cloud-fan

## How was this patch tested?

Checked metrics in unit tests.

Author: Eric Liang <ekl@databricks.com>

Closes #16112 from ericl/spark-18679.
2016-12-02 20:59:39 +08:00
Weiqing Yang 2159bf8b2c
[SPARK-18629][SQL] Fix numPartition of JDBCSuite Testcase
## What changes were proposed in this pull request?
Fix numPartition of JDBCSuite Testcase.

## How was this patch tested?
Before:
Run any one of the test cases in JDBCSuite, you will get the following warning.
```
10:34:26.389 WARN org.apache.spark.sql.execution.datasources.jdbc.JDBCRelation: The number of partitions is reduced because the specified number of partitions is less than the difference between upper bound and lower bound. Updated number of partitions: 3; Input number of partitions: 4; Lower bound: 1; Upper bound: 4.
```
After: Pass tests without the warning.

Author: Weiqing Yang <yangweiqing001@gmail.com>

Closes #16062 from weiqingy/SPARK-18629.
2016-12-02 11:53:15 +00:00
Cheng Lian ca63916372 [SPARK-17213][SQL] Disable Parquet filter push-down for string and binary columns due to PARQUET-686
This PR targets to both master and branch-2.1.

## What changes were proposed in this pull request?

Due to PARQUET-686, Parquet doesn't do string comparison correctly while doing filter push-down for string columns. This PR disables filter push-down for both string and binary columns to work around this issue. Binary columns are also affected because some Parquet data models (like Hive) may store string columns as a plain Parquet `binary` instead of a `binary (UTF8)`.

## How was this patch tested?

New test case added in `ParquetFilterSuite`.

Author: Cheng Lian <lian@databricks.com>

Closes #16106 from liancheng/spark-17213-bad-string-ppd.
2016-12-01 22:02:45 -08:00
Reynold Xin d3c90b74ed [SPARK-18663][SQL] Simplify CountMinSketch aggregate implementation
## What changes were proposed in this pull request?
SPARK-18429 introduced count-min sketch aggregate function for SQL, but the implementation and testing is more complicated than needed. This simplifies the test cases and removes support for data types that don't have clear equality semantics:

1. Removed support for floating point and decimal types.

2. Removed the heavy randomized tests. The underlying CountMinSketch implementation already had pretty good test coverage through randomized tests, and the SPARK-18429 implementation is just to add an aggregate function wrapper around CountMinSketch. There is no need for randomized tests at three different levels of the implementations.

## How was this patch tested?
A lot of the change is to simplify test cases.

Author: Reynold Xin <rxin@databricks.com>

Closes #16093 from rxin/SPARK-18663.
2016-12-01 21:38:52 -08:00
Kazuaki Ishizaki 38b9e69623 [SPARK-18284][SQL] Make ExpressionEncoder.serializer.nullable precise
## What changes were proposed in this pull request?

This PR makes `ExpressionEncoder.serializer.nullable` for flat encoder for a primitive type `false`. Since it is `true` for now, it is too conservative.
While `ExpressionEncoder.schema` has correct information (e.g. `<IntegerType, false>`), `serializer.head.nullable` of `ExpressionEncoder`, which got from `encoderFor[T]`, is always false. It is too conservative.

This is accomplished by checking whether a type is one of primitive types. If it is `true`, `nullable` should be `false`.

## How was this patch tested?

Added new tests for encoder and dataframe

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

Closes #15780 from kiszk/SPARK-18284.
2016-12-02 12:30:13 +08:00
sureshthalamati 70c5549ee9 [SPARK-18141][SQL] Fix to quote column names in the predicate clause of the JDBC RDD generated sql statement
## What changes were proposed in this pull request?

SQL query generated for the JDBC data source is not quoting columns in the predicate clause. When the source table has quoted column names,  spark jdbc read fails with column not found error incorrectly.

Error:
org.h2.jdbc.JdbcSQLException: Column "ID" not found;
Source SQL statement:
SELECT "Name","Id" FROM TEST."mixedCaseCols" WHERE (Id < 1)

This PR fixes by quoting column names in the generated  SQL for predicate clause  when filters are pushed down to the data source.

Source SQL statement after the fix:
SELECT "Name","Id" FROM TEST."mixedCaseCols" WHERE ("Id" < 1)

## How was this patch tested?

Added new test case to the JdbcSuite

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

Closes #15662 from sureshthalamati/filter_quoted_cols-SPARK-18141.
2016-12-01 19:13:38 -08:00
gatorsmile b28fe4a4a9 [SPARK-18538][SQL] Fix Concurrent Table Fetching Using DataFrameReader JDBC APIs
### What changes were proposed in this pull request?
The following two `DataFrameReader` JDBC APIs ignore the user-specified parameters of parallelism degree.

```Scala
  def jdbc(
      url: String,
      table: String,
      columnName: String,
      lowerBound: Long,
      upperBound: Long,
      numPartitions: Int,
      connectionProperties: Properties): DataFrame
```

```Scala
  def jdbc(
      url: String,
      table: String,
      predicates: Array[String],
      connectionProperties: Properties): DataFrame
```

This PR is to fix the issues. To verify the behavior correctness, we improve the plan output of `EXPLAIN` command by adding `numPartitions` in the `JDBCRelation` node.

Before the fix,
```
== Physical Plan ==
*Scan JDBCRelation(TEST.PEOPLE) [NAME#1896,THEID#1897] ReadSchema: struct<NAME:string,THEID:int>
```

After the fix,
```
== Physical Plan ==
*Scan JDBCRelation(TEST.PEOPLE) [numPartitions=3] [NAME#1896,THEID#1897] ReadSchema: struct<NAME:string,THEID:int>
```
### How was this patch tested?
Added the verification logics on all the test cases for JDBC concurrent fetching.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15975 from gatorsmile/jdbc.
2016-12-01 15:42:30 +08:00
Shixiong Zhu c4979f6ea8 [SPARK-18655][SS] Ignore Structured Streaming 2.0.2 logs in history server
## What changes were proposed in this pull request?

As `queryStatus` in StreamingQueryListener events was removed in #15954, parsing 2.0.2 structured streaming logs will throw the following errror:

```
[info]   com.fasterxml.jackson.databind.exc.UnrecognizedPropertyException: Unrecognized field "queryStatus" (class org.apache.spark.sql.streaming.StreamingQueryListener$QueryTerminatedEvent), not marked as ignorable (2 known properties: "id", "exception"])
[info]  at [Source: {"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryTerminatedEvent","queryStatus":{"name":"query-1","id":1,"timestamp":1480491532753,"inputRate":0.0,"processingRate":0.0,"latency":null,"sourceStatuses":[{"description":"FileStreamSource[file:/Users/zsx/stream]","offsetDesc":"#0","inputRate":0.0,"processingRate":0.0,"triggerDetails":{"latency.getOffset.source":"1","triggerId":"1"}}],"sinkStatus":{"description":"FileSink[/Users/zsx/stream2]","offsetDesc":"[#0]"},"triggerDetails":{}},"exception":null}; line: 1, column: 521] (through reference chain: org.apache.spark.sql.streaming.QueryTerminatedEvent["queryStatus"])
[info]   at com.fasterxml.jackson.databind.exc.UnrecognizedPropertyException.from(UnrecognizedPropertyException.java:51)
[info]   at com.fasterxml.jackson.databind.DeserializationContext.reportUnknownProperty(DeserializationContext.java:839)
[info]   at com.fasterxml.jackson.databind.deser.std.StdDeserializer.handleUnknownProperty(StdDeserializer.java:1045)
[info]   at com.fasterxml.jackson.databind.deser.BeanDeserializerBase.handleUnknownProperty(BeanDeserializerBase.java:1352)
[info]   at com.fasterxml.jackson.databind.deser.BeanDeserializerBase.handleUnknownProperties(BeanDeserializerBase.java:1306)
[info]   at com.fasterxml.jackson.databind.deser.BeanDeserializer._deserializeUsingPropertyBased(BeanDeserializer.java:453)
[info]   at com.fasterxml.jackson.databind.deser.BeanDeserializerBase.deserializeFromObjectUsingNonDefault(BeanDeserializerBase.java:1099)
...
```

This PR just ignores such errors and adds a test to make sure we can read 2.0.2 logs.

## How was this patch tested?

`query-event-logs-version-2.0.2.txt` has all types of events generated by Structured Streaming in Spark 2.0.2. `testQuietly("ReplayListenerBus should ignore broken event jsons generated in 2.0.2")` verified we can load them without any error.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16085 from zsxwing/SPARK-18655.
2016-11-30 16:18:53 -08:00
Wenchen Fan f135b70fd5 [SPARK-18251][SQL] the type of Dataset can't be Option of non-flat type
## What changes were proposed in this pull request?

For input object of non-flat type, we can't encode it to row if it's null, as Spark SQL doesn't allow the entire row to be null, only its columns can be null. That's the reason we forbid users to use top level null objects in https://github.com/apache/spark/pull/13469

However, if users wrap non-flat type with `Option`, then we may still encoder top level null object to row, which is not allowed.

This PR fixes this case, and suggests users to wrap their type with `Tuple1` if they do wanna top level null objects.

## How was this patch tested?

new test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15979 from cloud-fan/option.
2016-11-30 13:36:17 -08:00
jiangxingbo c24076dcf8 [SPARK-17932][SQL] Support SHOW TABLES EXTENDED LIKE 'identifier_with_wildcards' statement
## What changes were proposed in this pull request?

Currently we haven't implemented `SHOW TABLE EXTENDED` in Spark 2.0. This PR is to implement the statement.
Goals:
1. Support `SHOW TABLES EXTENDED LIKE 'identifier_with_wildcards'`;
2. Explicitly output an unsupported error message for `SHOW TABLES [EXTENDED] ... PARTITION` statement;
3. Improve test cases for `SHOW TABLES` statement.

## How was this patch tested?
1. Add new test cases in file `show-tables.sql`.
2. Modify tests for `SHOW TABLES` in `DDLSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15958 from jiangxb1987/show-table-extended.
2016-11-30 03:59:25 -08:00
gatorsmile 2eb093decb [SPARK-17897][SQL] Fixed IsNotNull Constraint Inference Rule
### What changes were proposed in this pull request?
The `constraints` of an operator is the expressions that evaluate to `true` for all the rows produced. That means, the expression result should be neither `false` nor `unknown` (NULL). Thus, we can conclude that `IsNotNull` on all the constraints, which are generated by its own predicates or propagated from the children. The constraint can be a complex expression. For better usage of these constraints, we try to push down `IsNotNull` to the lowest-level expressions (i.e., `Attribute`). `IsNotNull` can be pushed through an expression when it is null intolerant. (When the input is NULL, the null-intolerant expression always evaluates to NULL.)

Below is the existing code we have for `IsNotNull` pushdown.
```Scala
  private def scanNullIntolerantExpr(expr: Expression): Seq[Attribute] = expr match {
    case a: Attribute => Seq(a)
    case _: NullIntolerant | IsNotNull(_: NullIntolerant) =>
      expr.children.flatMap(scanNullIntolerantExpr)
    case _ => Seq.empty[Attribute]
  }
```

**`IsNotNull` itself is not null-intolerant.** It converts `null` to `false`. If the expression does not include any `Not`-like expression, it works; otherwise, it could generate a wrong result. This PR is to fix the above function by removing the `IsNotNull` from the inference. After the fix, when a constraint has a `IsNotNull` expression, we infer new attribute-specific `IsNotNull` constraints if and only if `IsNotNull` appears in the root.

Without the fix, the following test case will return empty.
```Scala
val data = Seq[java.lang.Integer](1, null).toDF("key")
data.filter("not key is not null").show()
```
Before the fix, the optimized plan is like
```
== Optimized Logical Plan ==
Project [value#1 AS key#3]
+- Filter (isnotnull(value#1) && NOT isnotnull(value#1))
   +- LocalRelation [value#1]
```

After the fix, the optimized plan is like
```
== Optimized Logical Plan ==
Project [value#1 AS key#3]
+- Filter NOT isnotnull(value#1)
   +- LocalRelation [value#1]
```

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16067 from gatorsmile/isNotNull2.
2016-11-30 19:40:58 +08:00
Herman van Hovell 879ba71110 [SPARK-18622][SQL] Fix the datatype of the Sum aggregate function
## What changes were proposed in this pull request?
The result of a `sum` aggregate function is typically a Decimal, Double or a Long. Currently the output dataType is based on input's dataType.

The `FunctionArgumentConversion` rule will make sure that the input is promoted to the largest type, and that also ensures that the output uses a (hopefully) sufficiently large output dataType. The issue is that sum is in a resolved state when we cast the input type, this means that rules assuming that the dataType of the expression does not change anymore could have been applied in the mean time. This is what happens if we apply `WidenSetOperationTypes` before applying the casts, and this breaks analysis.

The most straight forward and future proof solution is to make `sum` always output the widest dataType in its class (Long for IntegralTypes, Decimal for DecimalTypes & Double for FloatType and DoubleType). This PR implements that solution.

We should move expression specific type casting rules into the given Expression at some point.

## How was this patch tested?
Added (regression) tests to SQLQueryTestSuite's `union.sql`.

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

Closes #16063 from hvanhovell/SPARK-18622.
2016-11-30 15:25:33 +08:00
Tathagata Das bc09a2b8c3 [SPARK-18516][STRUCTURED STREAMING] Follow up PR to add StreamingQuery.status to Python
## What changes were proposed in this pull request?
- Add StreamingQueryStatus.json
- Make it not case class (to avoid unnecessarily exposing implicit object StreamingQueryStatus, consistent with StreamingQueryProgress)
- Add StreamingQuery.status to Python
- Fix post-termination status

## How was this patch tested?
New unit tests

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

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

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

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

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

Closes #16066 from hvanhovell/SPARK-18632.
2016-11-29 20:05:15 -08:00
Tathagata Das c3d08e2f29 [SPARK-18516][SQL] Split state and progress in streaming
This PR separates the status of a `StreamingQuery` into two separate APIs:
 - `status` - describes the status of a `StreamingQuery` at this moment, including what phase of processing is currently happening and if data is available.
 - `recentProgress` - an array of statistics about the most recent microbatches that have executed.

A recent progress contains the following information:
```
{
  "id" : "2be8670a-fce1-4859-a530-748f29553bb6",
  "name" : "query-29",
  "timestamp" : 1479705392724,
  "inputRowsPerSecond" : 230.76923076923077,
  "processedRowsPerSecond" : 10.869565217391303,
  "durationMs" : {
    "triggerExecution" : 276,
    "queryPlanning" : 3,
    "getBatch" : 5,
    "getOffset" : 3,
    "addBatch" : 234,
    "walCommit" : 30
  },
  "currentWatermark" : 0,
  "stateOperators" : [ ],
  "sources" : [ {
    "description" : "KafkaSource[Subscribe[topic-14]]",
    "startOffset" : {
      "topic-14" : {
        "2" : 0,
        "4" : 1,
        "1" : 0,
        "3" : 0,
        "0" : 0
      }
    },
    "endOffset" : {
      "topic-14" : {
        "2" : 1,
        "4" : 2,
        "1" : 0,
        "3" : 0,
        "0" : 1
      }
    },
    "numRecords" : 3,
    "inputRowsPerSecond" : 230.76923076923077,
    "processedRowsPerSecond" : 10.869565217391303
  } ]
}
```

Additionally, in order to make it possible to correlate progress updates across restarts, we change the `id` field from an integer that is unique with in the JVM to a `UUID` that is globally unique.

Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #15954 from marmbrus/queryProgress.
2016-11-29 17:24:17 -08:00
Nattavut Sutyanyong 3600635215 [SPARK-18614][SQL] Incorrect predicate pushdown from ExistenceJoin
## What changes were proposed in this pull request?

ExistenceJoin should be treated the same as LeftOuter and LeftAnti, not InnerLike and LeftSemi. This is not currently exposed because the rewrite of [NOT] EXISTS OR ... to ExistenceJoin happens in rule RewritePredicateSubquery, which is in a separate rule set and placed after the rule PushPredicateThroughJoin. During the transformation in the rule PushPredicateThroughJoin, an ExistenceJoin never exists.

The semantics of ExistenceJoin says we need to preserve all the rows from the left table through the join operation as if it is a regular LeftOuter join. The ExistenceJoin augments the LeftOuter operation with a new column called exists, set to true when the join condition in the ON clause is true and false otherwise. The filter of any rows will happen in the Filter operation above the ExistenceJoin.

Example:

A(c1, c2): { (1, 1), (1, 2) }
// B can be any value as it is irrelevant in this example
B(c1): { (NULL) }

select A.*
from   A
where  exists (select 1 from B where A.c1 = A.c2)
       or A.c2=2

In this example, the correct result is all the rows from A. If the pattern ExistenceJoin around line 935 in Optimizer.scala is indeed active, the code will push down the predicate A.c1 = A.c2 to be a Filter on relation A, which will incorrectly filter the row (1,2) from A.

## How was this patch tested?

Since this is not an exposed case, no new test cases is added. The scenario is discovered via a code review of another PR and confirmed to be valid with peer.

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

Closes #16044 from nsyca/spark-18614.
2016-11-29 15:27:43 -08:00
Mark Hamstra f8878a4c6f [SPARK-18631][SQL] Changed ExchangeCoordinator re-partitioning to avoid more data skew
## What changes were proposed in this pull request?

Re-partitioning logic in ExchangeCoordinator changed so that adding another pre-shuffle partition to the post-shuffle partition will not be done if doing so would cause the size of the post-shuffle partition to exceed the target partition size.

## How was this patch tested?

Existing tests updated to reflect new expectations.

Author: Mark Hamstra <markhamstra@gmail.com>

Closes #16065 from markhamstra/SPARK-17064.
2016-11-29 15:01:12 -08:00
wangzhenhua d57a594b8b [SPARK-18429][SQL] implement a new Aggregate for CountMinSketch
## What changes were proposed in this pull request?

This PR implements a new Aggregate to generate count min sketch, which is a wrapper of CountMinSketch.

## How was this patch tested?

add test cases

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #15877 from wzhfy/cms.
2016-11-29 13:16:46 -08:00
Tyson Condie 3c0beea475 [SPARK-18339][SPARK-18513][SQL] Don't push down current_timestamp for filters in StructuredStreaming and persist batch and watermark timestamps to offset log.
## What changes were proposed in this pull request?

For the following workflow:
1. I have a column called time which is at minute level precision in a Streaming DataFrame
2. I want to perform groupBy time, count
3. Then I want my MemorySink to only have the last 30 minutes of counts and I perform this by
.where('time >= current_timestamp().cast("long") - 30 * 60)
what happens is that the `filter` gets pushed down before the aggregation, and the filter happens on the source data for the aggregation instead of the result of the aggregation (where I actually want to filter).
I guess the main issue here is that `current_timestamp` is non-deterministic in the streaming context and shouldn't be pushed down the filter.
Does this require us to store the `current_timestamp` for each trigger of the streaming job, that is something to discuss.

Furthermore, we want to persist current batch timestamp and watermark timestamp to the offset log so that these values are consistent across multiple executions of the same batch.

brkyvz zsxwing tdas

## How was this patch tested?

A test was added to StreamingAggregationSuite ensuring the above use case is handled. The test injects a stream of time values (in seconds) to a query that runs in complete mode and only outputs the (count) aggregation results for the past 10 seconds.

Author: Tyson Condie <tcondie@gmail.com>

Closes #15949 from tcondie/SPARK-18339.
2016-11-28 23:07:17 -08:00
Kazuaki Ishizaki ad67993b73 [SPARK-17680][SQL][TEST] Added test cases for InMemoryRelation
## What changes were proposed in this pull request?

This pull request adds test cases for the following cases:
- keep all data types with null or without null
- access `CachedBatch` disabling whole stage codegen
- access only some columns in `CachedBatch`

This PR is a part of https://github.com/apache/spark/pull/15219. Here are motivations to add these tests. When https://github.com/apache/spark/pull/15219 is enabled, the first two cases are handled by specialized (generated) code. The third one is a pitfall.

In general, even for now, it would be helpful to increase test coverage.
## How was this patch tested?

added test suites itself

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

Closes #15462 from kiszk/columnartestsuites.
2016-11-28 14:06:37 -05:00
Wenchen Fan d31ff9b7ca [SPARK-17732][SQL] Revert ALTER TABLE DROP PARTITION should support comparators
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16036 from cloud-fan/revert.
2016-11-28 08:46:00 -08:00
Herman van Hovell 38e29824d9 [SPARK-18597][SQL] Do not push-down join conditions to the right side of a LEFT ANTI join
## What changes were proposed in this pull request?
We currently push down join conditions of a Left Anti join to both sides of the join. This is similar to Inner, Left Semi and Existence (a specialized left semi) join. The problem is that this changes the semantics of the join; a left anti join filters out rows that matches the join condition.

This PR fixes this by only pushing down conditions to the left hand side of the join. This is similar to the behavior of left outer join.

## How was this patch tested?
Added tests to `FilterPushdownSuite.scala` and created a SQLQueryTestSuite file for left anti joins with a regression test.

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

Closes #16026 from hvanhovell/SPARK-18597.
2016-11-28 07:10:52 -08:00
gatorsmile 9f273c5173 [SPARK-17783][SQL] Hide Credentials in CREATE and DESC FORMATTED/EXTENDED a PERSISTENT/TEMP Table for JDBC
### What changes were proposed in this pull request?

We should never expose the Credentials in the EXPLAIN and DESC FORMATTED/EXTENDED command. However, below commands exposed the credentials.

In the related PR: https://github.com/apache/spark/pull/10452

> URL patterns to specify credential seems to be vary between different databases.

Thus, we hide the whole `url` value if it contains the keyword `password`. We also hide the `password` property.

Before the fix, the command outputs look like:

``` SQL
CREATE TABLE tab1
USING org.apache.spark.sql.jdbc
OPTIONS (
 url 'jdbc:h2:mem:testdb0;user=testUser;password=testPass',
 dbtable 'TEST.PEOPLE',
 user 'testUser',
 password '$password')

DESC FORMATTED tab1
DESC EXTENDED tab1
```

Before the fix,
- The output of SQL statement EXPLAIN
```
== Physical Plan ==
ExecutedCommand
   +- CreateDataSourceTableCommand CatalogTable(
	Table: `tab1`
	Created: Wed Nov 16 23:00:10 PST 2016
	Last Access: Wed Dec 31 15:59:59 PST 1969
	Type: MANAGED
	Provider: org.apache.spark.sql.jdbc
	Storage(Properties: [url=jdbc:h2:mem:testdb0;user=testUser;password=testPass, dbtable=TEST.PEOPLE, user=testUser, password=testPass])), false
```

- The output of `DESC FORMATTED`
```
...
|Storage Desc Parameters:    |                                                                  |       |
|  url                       |jdbc:h2:mem:testdb0;user=testUser;password=testPass               |       |
|  dbtable                   |TEST.PEOPLE                                                       |       |
|  user                      |testUser                                                          |       |
|  password                  |testPass                                                          |       |
+----------------------------+------------------------------------------------------------------+-------+
```

- The output of `DESC EXTENDED`
```
|# Detailed Table Information|CatalogTable(
	Table: `default`.`tab1`
	Created: Wed Nov 16 23:00:10 PST 2016
	Last Access: Wed Dec 31 15:59:59 PST 1969
	Type: MANAGED
	Schema: [StructField(NAME,StringType,false), StructField(THEID,IntegerType,false)]
	Provider: org.apache.spark.sql.jdbc
	Storage(Location: file:/Users/xiaoli/IdeaProjects/sparkDelivery/spark-warehouse/tab1, Properties: [url=jdbc:h2:mem:testdb0;user=testUser;password=testPass, dbtable=TEST.PEOPLE, user=testUser, password=testPass]))|       |
```

After the fix,
- The output of SQL statement EXPLAIN
```
== Physical Plan ==
ExecutedCommand
   +- CreateDataSourceTableCommand CatalogTable(
	Table: `tab1`
	Created: Wed Nov 16 22:43:49 PST 2016
	Last Access: Wed Dec 31 15:59:59 PST 1969
	Type: MANAGED
	Provider: org.apache.spark.sql.jdbc
	Storage(Properties: [url=###, dbtable=TEST.PEOPLE, user=testUser, password=###])), false
```
- The output of `DESC FORMATTED`
```
...
|Storage Desc Parameters:    |                                                                  |       |
|  url                       |###                                                               |       |
|  dbtable                   |TEST.PEOPLE                                                       |       |
|  user                      |testUser                                                          |       |
|  password                  |###                                                               |       |
+----------------------------+------------------------------------------------------------------+-------+
```

- The output of `DESC EXTENDED`
```
|# Detailed Table Information|CatalogTable(
	Table: `default`.`tab1`
	Created: Wed Nov 16 22:43:49 PST 2016
	Last Access: Wed Dec 31 15:59:59 PST 1969
	Type: MANAGED
	Schema: [StructField(NAME,StringType,false), StructField(THEID,IntegerType,false)]
	Provider: org.apache.spark.sql.jdbc
	Storage(Location: file:/Users/xiaoli/IdeaProjects/sparkDelivery/spark-warehouse/tab1, Properties: [url=###, dbtable=TEST.PEOPLE, user=testUser, password=###]))|       |
```

### How was this patch tested?

Added test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15358 from gatorsmile/maskCredentials.
2016-11-28 07:04:38 -08:00
Kazuaki Ishizaki f075cd9cb7 [SPARK-18118][SQL] fix a compilation error due to nested JavaBeans
## What changes were proposed in this pull request?

This PR avoids a compilation error due to more than 64KB Java byte code size. This error occur since generated java code `SpecificSafeProjection.apply()` for nested JavaBeans is too big. This PR avoids this compilation error by splitting a big code chunk into multiple methods by calling `CodegenContext.splitExpression` at `InitializeJavaBean.doGenCode`
An object reference for JavaBean is stored to an instance variable `javaBean...`. Then, the instance variable will be referenced in the split methods.

Generated code with this PR
````
/* 22098 */   private void apply130_0(InternalRow i) {
...
/* 22125 */     boolean isNull238 = i.isNullAt(2);
/* 22126 */     InternalRow value238 = isNull238 ? null : (i.getStruct(2, 3));
/* 22127 */     boolean isNull236 = false;
/* 22128 */     test.org.apache.spark.sql.JavaDatasetSuite$Nesting1 value236 = null;
/* 22129 */     if (!false && isNull238) {
/* 22130 */
/* 22131 */       final test.org.apache.spark.sql.JavaDatasetSuite$Nesting1 value239 = null;
/* 22132 */       isNull236 = true;
/* 22133 */       value236 = value239;
/* 22134 */     } else {
/* 22135 */
/* 22136 */       final test.org.apache.spark.sql.JavaDatasetSuite$Nesting1 value241 = false ? null : new test.org.apache.spark.sql.JavaDatasetSuite$Nesting1();
/* 22137 */       this.javaBean14 = value241;
/* 22138 */       if (!false) {
/* 22139 */         apply25_0(i);
/* 22140 */         apply25_1(i);
/* 22141 */         apply25_2(i);
/* 22142 */       }
/* 22143 */       isNull236 = false;
/* 22144 */       value236 = value241;
/* 22145 */     }
/* 22146 */     this.javaBean.setField2(value236);
/* 22147 */
/* 22148 */   }
...
/* 22928 */   public java.lang.Object apply(java.lang.Object _i) {
/* 22929 */     InternalRow i = (InternalRow) _i;
/* 22930 */
/* 22931 */     final test.org.apache.spark.sql.JavaDatasetSuite$NestedComplicatedJavaBean value1 = false ? null : new test.org.apache.spark.sql.JavaDatasetSuite$NestedComplicatedJavaBean();
/* 22932 */     this.javaBean = value1;
/* 22933 */     if (!false) {
/* 22934 */       apply130_0(i);
/* 22935 */       apply130_1(i);
/* 22936 */       apply130_2(i);
/* 22937 */       apply130_3(i);
/* 22938 */       apply130_4(i);
/* 22939 */     }
/* 22940 */     if (false) {
/* 22941 */       mutableRow.setNullAt(0);
/* 22942 */     } else {
/* 22943 */
/* 22944 */       mutableRow.update(0, value1);
/* 22945 */     }
/* 22946 */
/* 22947 */     return mutableRow;
/* 22948 */   }
````

## How was this patch tested?

added a test suite into `JavaDatasetSuite.java`

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

Closes #16032 from kiszk/SPARK-18118.
2016-11-28 04:18:35 -08:00
jiangxingbo e2fb9fd365 [SPARK-18436][SQL] isin causing SQL syntax error with JDBC
## What changes were proposed in this pull request?

The expression `in(empty seq)` is invalid in some data source. Since `in(empty seq)` is always false, we should generate `in(empty seq)` to false literal in optimizer.
The sql `SELECT * FROM t WHERE a IN ()` throws a `ParseException` which is consistent with Hive, don't need to change that behavior.

## How was this patch tested?
Add new test case in `OptimizeInSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15977 from jiangxb1987/isin-empty.
2016-11-25 12:44:34 -08:00
Dongjoon Hyun fb07bbe575 [SPARK-18413][SQL][FOLLOW-UP] Use numPartitions instead of maxConnections
## What changes were proposed in this pull request?

This is a follow-up PR of #15868 to merge `maxConnections` option into `numPartitions` options.

## How was this patch tested?

Pass the existing tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15966 from dongjoon-hyun/SPARK-18413-2.
2016-11-25 10:35:07 -08:00
Nattavut Sutyanyong a367d5ff00 [SPARK-18578][SQL] Full outer join in correlated subquery returns incorrect results
## What changes were proposed in this pull request?

- Raise Analysis exception when correlated predicates exist in the descendant operators of either operand of a Full outer join in a subquery as well as in a FOJ operator itself
- Raise Analysis exception when correlated predicates exists in a Window operator (a side effect inadvertently introduced by SPARK-17348)

## How was this patch tested?

Run sql/test catalyst/test and new test cases, added to SubquerySuite, showing the reported incorrect results.

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

Closes #16005 from nsyca/FOJ-incorrect.1.
2016-11-24 12:07:55 -08:00
Burak Yavuz 0d1bf2b6c8 [SPARK-18510] Fix data corruption from inferred partition column dataTypes
## What changes were proposed in this pull request?

### The Issue

If I specify my schema when doing
```scala
spark.read
  .schema(someSchemaWherePartitionColumnsAreStrings)
```
but if the partition inference can infer it as IntegerType or I assume LongType or DoubleType (basically fixed size types), then once UnsafeRows are generated, your data will be corrupted.

### Proposed solution

The partition handling code path is kind of a mess. In my fix I'm probably adding to the mess, but at least trying to standardize the code path.

The real issue is that a user that uses the `spark.read` code path can never clearly specify what the partition columns are. If you try to specify the fields in `schema`, we practically ignore what the user provides, and fall back to our inferred data types. What happens in the end is data corruption.

My solution tries to fix this by always trying to infer partition columns the first time you specify the table. Once we find what the partition columns are, we try to find them in the user specified schema and use the dataType provided there, or fall back to the smallest common data type.

We will ALWAYS append partition columns to the user's schema, even if they didn't ask for it. We will only use the data type they provided if they specified it. While this is confusing, this has been the behavior since Spark 1.6, and I didn't want to change this behavior in the QA period of Spark 2.1. We may revisit this decision later.

A side effect of this PR is that we won't need https://github.com/apache/spark/pull/15942 if this PR goes in.

## How was this patch tested?

Regression tests

Author: Burak Yavuz <brkyvz@gmail.com>

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

This pull request introduces the following changes:

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

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

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

Author: Reynold Xin <rxin@databricks.com>

Closes #15959 from rxin/SPARK-18522.
2016-11-23 20:48:41 +08:00
Wenchen Fan 84284e8c82 [SPARK-18053][SQL] compare unsafe and safe complex-type values correctly
## What changes were proposed in this pull request?

In Spark SQL, some expression may output safe format values, e.g. `CreateArray`, `CreateStruct`, `Cast`, etc. When we compare 2 values, we should be able to compare safe and unsafe formats.

The `GreaterThan`, `LessThan`, etc. in Spark SQL already handles it, but the `EqualTo` doesn't. This PR fixes it.

## How was this patch tested?

new unit test and regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15929 from cloud-fan/type-aware.
2016-11-23 04:15:19 -08:00
Dilip Biswal 39a1d30636 [SPARK-18533] Raise correct error upon specification of schema for datasource tables created using CTAS
## What changes were proposed in this pull request?
Fixes the inconsistency of error raised between data source and hive serde
tables when schema is specified in CTAS scenario. In the process the grammar for
create table (datasource) is simplified.

**before:**
``` SQL
spark-sql> create table t2 (c1 int, c2 int) using parquet as select * from t1;
Error in query:
mismatched input 'as' expecting {<EOF>, '.', 'OPTIONS', 'CLUSTERED', 'PARTITIONED'}(line 1, pos 64)

== SQL ==
create table t2 (c1 int, c2 int) using parquet as select * from t1
----------------------------------------------------------------^^^
```

**After:**
```SQL
spark-sql> create table t2 (c1 int, c2 int) using parquet as select * from t1
         > ;
Error in query:
Operation not allowed: Schema may not be specified in a Create Table As Select (CTAS) statement(line 1, pos 0)

== SQL ==
create table t2 (c1 int, c2 int) using parquet as select * from t1
^^^
```
## How was this patch tested?
Added a new test in CreateTableAsSelectSuite

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

Closes #15968 from dilipbiswal/ctas.
2016-11-22 15:57:07 -08:00
Nattavut Sutyanyong 45ea46b7b3 [SPARK-18504][SQL] Scalar subquery with extra group by columns returning incorrect result
## What changes were proposed in this pull request?

This PR blocks an incorrect result scenario in scalar subquery where there are GROUP BY column(s)
that are not part of the correlated predicate(s).

Example:
// Incorrect result
Seq(1).toDF("c1").createOrReplaceTempView("t1")
Seq((1,1),(1,2)).toDF("c1","c2").createOrReplaceTempView("t2")
sql("select (select sum(-1) from t2 where t1.c1=t2.c1 group by t2.c2) from t1").show

// How can selecting a scalar subquery from a 1-row table return 2 rows?

## How was this patch tested?
sql/test, catalyst/test
new test case covering the reported problem is added to SubquerySuite.scala

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

Closes #15936 from nsyca/scalarSubqueryIncorrect-1.
2016-11-22 12:06:21 -08:00
Liwei Lin ebeb0830a3 [SPARK-18425][STRUCTURED STREAMING][TESTS] Test CompactibleFileStreamLog directly
## What changes were proposed in this pull request?

Right now we are testing the most of `CompactibleFileStreamLog` in `FileStreamSinkLogSuite` (because `FileStreamSinkLog` once was the only subclass of `CompactibleFileStreamLog`, but now it's not the case any more).

Let's refactor the tests so that `CompactibleFileStreamLog` is directly tested, making future changes (like https://github.com/apache/spark/pull/15828, https://github.com/apache/spark/pull/15827) to `CompactibleFileStreamLog` much easier to test and much easier to review.

## How was this patch tested?

the PR itself is about tests

Author: Liwei Lin <lwlin7@gmail.com>

Closes #15870 from lw-lin/test-compact-1113.
2016-11-21 21:14:13 -08:00
Dongjoon Hyun 07beb5d21c
[SPARK-18413][SQL] Add maxConnections JDBCOption
## What changes were proposed in this pull request?

This PR adds a new JDBCOption `maxConnections` which means the maximum number of simultaneous JDBC connections allowed. This option applies only to writing with coalesce operation if needed. It defaults to the number of partitions of RDD. Previously, SQL users cannot cannot control this while Scala/Java/Python users can use `coalesce` (or `repartition`) API.

**Reported Scenario**

For the following cases, the number of connections becomes 200 and database cannot handle all of them.

```sql
CREATE OR REPLACE TEMPORARY VIEW resultview
USING org.apache.spark.sql.jdbc
OPTIONS (
  url "jdbc:oracle:thin:10.129.10.111:1521:BKDB",
  dbtable "result",
  user "HIVE",
  password "HIVE"
);
-- set spark.sql.shuffle.partitions=200
INSERT OVERWRITE TABLE resultview SELECT g, count(1) AS COUNT FROM tnet.DT_LIVE_INFO GROUP BY g
```

## How was this patch tested?

Manual. Do the followings and see Spark UI.

**Step 1 (MySQL)**
```
CREATE TABLE t1 (a INT);
CREATE TABLE data (a INT);
INSERT INTO data VALUES (1);
INSERT INTO data VALUES (2);
INSERT INTO data VALUES (3);
```

**Step 2 (Spark)**
```scala
SPARK_HOME=$PWD bin/spark-shell --driver-memory 4G --driver-class-path mysql-connector-java-5.1.40-bin.jar
scala> sql("SET spark.sql.shuffle.partitions=3")
scala> sql("CREATE OR REPLACE TEMPORARY VIEW data USING org.apache.spark.sql.jdbc OPTIONS (url 'jdbc:mysql://localhost:3306/t', dbtable 'data', user 'root', password '')")
scala> sql("CREATE OR REPLACE TEMPORARY VIEW t1 USING org.apache.spark.sql.jdbc OPTIONS (url 'jdbc:mysql://localhost:3306/t', dbtable 't1', user 'root', password '', maxConnections '1')")
scala> sql("INSERT OVERWRITE TABLE t1 SELECT a FROM data GROUP BY a")
scala> sql("CREATE OR REPLACE TEMPORARY VIEW t1 USING org.apache.spark.sql.jdbc OPTIONS (url 'jdbc:mysql://localhost:3306/t', dbtable 't1', user 'root', password '', maxConnections '2')")
scala> sql("INSERT OVERWRITE TABLE t1 SELECT a FROM data GROUP BY a")
scala> sql("CREATE OR REPLACE TEMPORARY VIEW t1 USING org.apache.spark.sql.jdbc OPTIONS (url 'jdbc:mysql://localhost:3306/t', dbtable 't1', user 'root', password '', maxConnections '3')")
scala> sql("INSERT OVERWRITE TABLE t1 SELECT a FROM data GROUP BY a")
scala> sql("CREATE OR REPLACE TEMPORARY VIEW t1 USING org.apache.spark.sql.jdbc OPTIONS (url 'jdbc:mysql://localhost:3306/t', dbtable 't1', user 'root', password '', maxConnections '4')")
scala> sql("INSERT OVERWRITE TABLE t1 SELECT a FROM data GROUP BY a")
```

![maxconnections](https://cloud.githubusercontent.com/assets/9700541/20287987/ed8409c2-aa84-11e6-8aab-ae28e63fe54d.png)

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15868 from dongjoon-hyun/SPARK-18413.
2016-11-21 13:57:36 +00:00
Reynold Xin b625a36ebc [HOTFIX][SQL] Fix DDLSuite failure. 2016-11-20 20:00:59 -08:00
Herman van Hovell 7ca7a63524 [SPARK-15214][SQL] Code-generation for Generate
## What changes were proposed in this pull request?

This PR adds code generation to `Generate`. It supports two code paths:
- General `TraversableOnce` based iteration. This used for regular `Generator` (code generation supporting) expressions. This code path expects the expression to return a `TraversableOnce[InternalRow]` and it will iterate over the returned collection. This PR adds code generation for the `stack` generator.
- Specialized `ArrayData/MapData` based iteration. This is used for the `explode`, `posexplode` & `inline` functions and operates directly on the `ArrayData`/`MapData` result that the child of the generator returns.

### Benchmarks
I have added some benchmarks and it seems we can create a nice speedup for explode:
#### Environment
```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6
Intel(R) Core(TM) i7-4980HQ CPU  2.80GHz
```
#### Explode Array
##### Before
```
generate explode array:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate explode array wholestage off         7377 / 7607          2.3         439.7       1.0X
generate explode array wholestage on          6055 / 6086          2.8         360.9       1.2X
```
##### After
```
generate explode array:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate explode array wholestage off         7432 / 7696          2.3         443.0       1.0X
generate explode array wholestage on           631 /  646         26.6          37.6      11.8X
```
#### Explode Map
##### Before
```
generate explode map:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate explode map wholestage off         12792 / 12848          1.3         762.5       1.0X
generate explode map wholestage on          11181 / 11237          1.5         666.5       1.1X
```
##### After
```
generate explode map:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate explode map wholestage off         10949 / 10972          1.5         652.6       1.0X
generate explode map wholestage on             870 /  913         19.3          51.9      12.6X
```
#### Posexplode
##### Before
```
generate posexplode array:               Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate posexplode array wholestage off      7547 / 7580          2.2         449.8       1.0X
generate posexplode array wholestage on       5786 / 5838          2.9         344.9       1.3X
```
##### After
```
generate posexplode array:               Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate posexplode array wholestage off      7535 / 7548          2.2         449.1       1.0X
generate posexplode array wholestage on        620 /  624         27.1          37.0      12.1X
```
#### Inline
##### Before
```
generate inline array:                   Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate inline array wholestage off          6935 / 6978          2.4         413.3       1.0X
generate inline array wholestage on           6360 / 6400          2.6         379.1       1.1X
```
##### After
```
generate inline array:                   Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate inline array wholestage off          6940 / 6966          2.4         413.6       1.0X
generate inline array wholestage on           1002 / 1012         16.7          59.7       6.9X
```
#### Stack
##### Before
```
generate stack:                          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate stack wholestage off               12980 / 13104          1.3         773.7       1.0X
generate stack wholestage on                11566 / 11580          1.5         689.4       1.1X
```
##### After
```
generate stack:                          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate stack wholestage off               12875 / 12949          1.3         767.4       1.0X
generate stack wholestage on                   840 /  845         20.0          50.0      15.3X
```
## How was this patch tested?

Existing tests.

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

Closes #13065 from hvanhovell/SPARK-15214.
2016-11-19 23:55:09 -08:00
hyukjinkwon d5b1d5fc80
[SPARK-18445][BUILD][DOCS] Fix the markdown for Note:/NOTE:/Note that/'''Note:''' across Scala/Java API documentation
## What changes were proposed in this pull request?

It seems in Scala/Java,

- `Note:`
- `NOTE:`
- `Note that`
- `'''Note:'''`
- `note`

This PR proposes to fix those to `note` to be consistent.

**Before**

- Scala
  ![2016-11-17 6 16 39](https://cloud.githubusercontent.com/assets/6477701/20383180/1a7aed8c-acf2-11e6-9611-5eaf6d52c2e0.png)

- Java
  ![2016-11-17 6 14 41](https://cloud.githubusercontent.com/assets/6477701/20383096/c8ffc680-acf1-11e6-914a-33460bf1401d.png)

**After**

- Scala
  ![2016-11-17 6 16 44](https://cloud.githubusercontent.com/assets/6477701/20383167/09940490-acf2-11e6-937a-0d5e1dc2cadf.png)

- Java
  ![2016-11-17 6 13 39](https://cloud.githubusercontent.com/assets/6477701/20383132/e7c2a57e-acf1-11e6-9c47-b849674d4d88.png)

## How was this patch tested?

The notes were found via

```bash
grep -r "NOTE: " . | \ # Note:|NOTE:|Note that|'''Note:'''
grep -v "// NOTE: " | \  # starting with // does not appear in API documentation.
grep -E '.scala|.java' | \ # java/scala files
grep -v Suite | \ # exclude tests
grep -v Test | \ # exclude tests
grep -e 'org.apache.spark.api.java' \ # packages appear in API documenation
-e 'org.apache.spark.api.java.function' \ # note that this is a regular expression. So actual matches were mostly `org/apache/spark/api/java/functions ...`
-e 'org.apache.spark.api.r' \
...
```

```bash
grep -r "Note that " . | \ # Note:|NOTE:|Note that|'''Note:'''
grep -v "// Note that " | \  # starting with // does not appear in API documentation.
grep -E '.scala|.java' | \ # java/scala files
grep -v Suite | \ # exclude tests
grep -v Test | \ # exclude tests
grep -e 'org.apache.spark.api.java' \ # packages appear in API documenation
-e 'org.apache.spark.api.java.function' \
-e 'org.apache.spark.api.r' \
...
```

```bash
grep -r "Note: " . | \ # Note:|NOTE:|Note that|'''Note:'''
grep -v "// Note: " | \  # starting with // does not appear in API documentation.
grep -E '.scala|.java' | \ # java/scala files
grep -v Suite | \ # exclude tests
grep -v Test | \ # exclude tests
grep -e 'org.apache.spark.api.java' \ # packages appear in API documenation
-e 'org.apache.spark.api.java.function' \
-e 'org.apache.spark.api.r' \
...
```

```bash
grep -r "'''Note:'''" . | \ # Note:|NOTE:|Note that|'''Note:'''
grep -v "// '''Note:''' " | \  # starting with // does not appear in API documentation.
grep -E '.scala|.java' | \ # java/scala files
grep -v Suite | \ # exclude tests
grep -v Test | \ # exclude tests
grep -e 'org.apache.spark.api.java' \ # packages appear in API documenation
-e 'org.apache.spark.api.java.function' \
-e 'org.apache.spark.api.r' \
...
```

And then fixed one by one comparing with API documentation/access modifiers.

After that, manually tested via `jekyll build`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15889 from HyukjinKwon/SPARK-18437.
2016-11-19 11:24:15 +00:00
Shixiong Zhu 2a40de408b [SPARK-18497][SS] Make ForeachSink support watermark
## What changes were proposed in this pull request?

The issue in ForeachSink is the new created DataSet still uses the old QueryExecution. When `foreachPartition` is called, `QueryExecution.toString` will be called and then fail because it doesn't know how to plan EventTimeWatermark.

This PR just replaces the QueryExecution with IncrementalExecution to fix the issue.

## How was this patch tested?

`test("foreach with watermark")`.

Author: Shixiong Zhu <shixiong@databricks.com>

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

This is a small pull request to clean up AnalyzeColumnCommand:

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

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

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

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

Author: Reynold Xin <rxin@databricks.com>

Closes #15933 from rxin/SPARK-18505.
2016-11-18 16:34:11 -08:00
Tyson Condie 51baca2219 [SPARK-18187][SQL] CompactibleFileStreamLog should not use "compactInterval" direcly with user setting.
## What changes were proposed in this pull request?
CompactibleFileStreamLog relys on "compactInterval" to detect a compaction batch. If the "compactInterval" is reset by user, CompactibleFileStreamLog will return wrong answer, resulting data loss. This PR procides a way to check the validity of 'compactInterval', and calculate an appropriate value.

## How was this patch tested?
When restart a stream, we change the 'spark.sql.streaming.fileSource.log.compactInterval' different with the former one.

The primary solution to this issue was given by uncleGen
Added extensions include an additional metadata field in OffsetSeq and CompactibleFileStreamLog APIs. zsxwing

Author: Tyson Condie <tcondie@gmail.com>
Author: genmao.ygm <genmao.ygm@genmaoygmdeMacBook-Air.local>

Closes #15852 from tcondie/spark-18187.
2016-11-18 11:11:24 -08:00
Josh Rosen d9dd979d17 [SPARK-18462] Fix ClassCastException in SparkListenerDriverAccumUpdates event
## What changes were proposed in this pull request?

This patch fixes a `ClassCastException: java.lang.Integer cannot be cast to java.lang.Long` error which could occur in the HistoryServer while trying to process a deserialized `SparkListenerDriverAccumUpdates` event.

The problem stems from how `jackson-module-scala` handles primitive type parameters (see https://github.com/FasterXML/jackson-module-scala/wiki/FAQ#deserializing-optionint-and-other-primitive-challenges for more details). This was causing a problem where our code expected a field to be deserialized as a `(Long, Long)` tuple but we got an `(Int, Int)` tuple instead.

This patch hacks around this issue by registering a custom `Converter` with Jackson in order to deserialize the tuples as `(Object, Object)` and perform the appropriate casting.

## How was this patch tested?

New regression tests in `SQLListenerSuite`.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #15922 from JoshRosen/SPARK-18462.
2016-11-17 18:45:15 -08:00
Tathagata Das 0048ce7ce6 [SPARK-18459][SPARK-18460][STRUCTUREDSTREAMING] Rename triggerId to batchId and add triggerDetails to json in StreamingQueryStatus
## What changes were proposed in this pull request?

SPARK-18459: triggerId seems like a number that should be increasing with each trigger, whether or not there is data in it. However, actually, triggerId increases only where there is a batch of data in a trigger. So its better to rename it to batchId.

SPARK-18460: triggerDetails was missing from json representation. Fixed it.

## How was this patch tested?
Updated existing unit tests.

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

Closes #15895 from tdas/SPARK-18459.
2016-11-16 10:00:59 -08:00
Dongjoon Hyun 74f5c2176d [SPARK-18433][SQL] Improve DataSource option keys to be more case-insensitive
## What changes were proposed in this pull request?

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

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

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

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

## How was this patch tested?

Pass the Jenkins test with newly added test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

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

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

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

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

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

**Spark 1.6**

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

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

**Spark 2.0**

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

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

After this PR, it's supported.

## How was this patch tested?

Pass the Jenkins test with a newly added testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15704 from dongjoon-hyun/SPARK-17732-2.
2016-11-15 15:59:04 -08:00
Tathagata Das 1ae4652b7e [SPARK-18440][STRUCTURED STREAMING] Pass correct query execution to FileFormatWriter
## What changes were proposed in this pull request?

SPARK-18012 refactored the file write path in FileStreamSink using FileFormatWriter which always uses the default non-streaming QueryExecution to perform the writes. This is wrong for FileStreamSink, because the streaming QueryExecution (i.e. IncrementalExecution) should be used for correctly incrementalizing aggregation. The addition of watermarks in SPARK-18124, file stream sink should logically supports aggregation + watermark + append mode. But actually it fails with
```
16:23:07.389 ERROR org.apache.spark.sql.execution.streaming.StreamExecution: Query query-0 terminated with error
java.lang.AssertionError: assertion failed: No plan for EventTimeWatermark timestamp#7: timestamp, interval 10 seconds
+- LocalRelation [timestamp#7]

	at scala.Predef$.assert(Predef.scala:170)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:92)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$2$$anonfun$apply$2.apply(QueryPlanner.scala:77)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$2$$anonfun$apply$2.apply(QueryPlanner.scala:74)
	at scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:157)
	at scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:157)
	at scala.collection.Iterator$class.foreach(Iterator.scala:893)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1336)
	at scala.collection.TraversableOnce$class.foldLeft(TraversableOnce.scala:157)
	at scala.collection.AbstractIterator.foldLeft(Iterator.scala:1336)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$2.apply(QueryPlanner.scala:74)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$2.apply(QueryPlanner.scala:66)
	at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
	at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:92)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$2$$anonfun$apply$2.apply(QueryPlanner.scala:77)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$2$$anonfun$apply$2.apply(QueryPlanner.scala:74)
```

This PR fixes it by passing the correct query execution.

## How was this patch tested?
New unit test

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

Closes #15885 from tdas/SPARK-18440.
2016-11-15 15:12:30 -08:00
Burak Yavuz 2afdaa9805 [SPARK-18337] Complete mode memory sinks should be able to recover from checkpoints
## What changes were proposed in this pull request?

It would be nice if memory sinks can also recover from checkpoints. For correctness reasons, the only time we should support it is in `Complete` OutputMode. We can support this in CompleteMode, because the output of the StateStore is already persisted in the checkpoint directory.

## How was this patch tested?

Unit test

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #15801 from brkyvz/mem-stream.
2016-11-15 13:09:29 -08:00
Herman van Hovell f14ae4900a [SPARK-18300][SQL] Do not apply foldable propagation with expand as a child.
## What changes were proposed in this pull request?
The `FoldablePropagation` optimizer rule, pulls foldable values out from under an `Expand`. This breaks the `Expand` in two ways:

- It rewrites the output attributes of the `Expand`. We explicitly define output attributes for `Expand`, these are (unfortunately) considered as part of the expressions of the `Expand` and can be rewritten.
- Expand can actually change the column (it will typically re-use the attributes or the underlying plan). This means that we cannot safely propagate the expressions from under an `Expand`.

This PR fixes this and (hopefully) other issues by explicitly whitelisting allowed operators.

## How was this patch tested?
Added tests to `FoldablePropagationSuite` and to `SQLQueryTestSuite`.

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

Closes #15857 from hvanhovell/SPARK-18300.
2016-11-15 06:59:25 -08:00
gatorsmile 86430cc4e8 [SPARK-18430][SQL] Fixed Exception Messages when Hitting an Invocation Exception of Function Lookup
### What changes were proposed in this pull request?
When the exception is an invocation exception during function lookup, we return a useless/confusing error message:

For example,
```Scala
df.selectExpr("concat_ws()")
```
Below is the error message we got:
```
null; line 1 pos 0
org.apache.spark.sql.AnalysisException: null; line 1 pos 0
```

To get the meaningful error message, we need to get the cause. The fix is exactly the same as what we did in https://github.com/apache/spark/pull/12136. After the fix, the message we got is the exception issued in the constuctor of function implementation:
```
requirement failed: concat_ws requires at least one argument.; line 1 pos 0
org.apache.spark.sql.AnalysisException: requirement failed: concat_ws requires at least one argument.; line 1 pos 0
```

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15878 from gatorsmile/functionNotFound.
2016-11-14 21:21:34 -08:00
Michael Armbrust c07187823a [SPARK-18124] Observed delay based Event Time Watermarks
This PR adds a new method `withWatermark` to the `Dataset` API, which can be used specify an _event time watermark_.  An event time watermark allows the streaming engine to reason about the point in time after which we no longer expect to see late data.  This PR also has augmented `StreamExecution` to use this watermark for several purposes:
  - To know when a given time window aggregation is finalized and thus results can be emitted when using output modes that do not allow updates (e.g. `Append` mode).
  - To minimize the amount of state that we need to keep for on-going aggregations, by evicting state for groups that are no longer expected to change.  Although, we do still maintain all state if the query requires (i.e. if the event time is not present in the `groupBy` or when running in `Complete` mode).

An example that emits windowed counts of records, waiting up to 5 minutes for late data to arrive.
```scala
df.withWatermark("eventTime", "5 minutes")
  .groupBy(window($"eventTime", "1 minute") as 'window)
  .count()
  .writeStream
  .format("console")
  .mode("append") // In append mode, we only output finalized aggregations.
  .start()
```

### Calculating the watermark.
The current event time is computed by looking at the `MAX(eventTime)` seen this epoch across all of the partitions in the query minus some user defined _delayThreshold_.  An additional constraint is that the watermark must increase monotonically.

Note that since we must coordinate this value across partitions occasionally, the actual watermark used is only guaranteed to be at least `delay` behind the actual event time.  In some cases we may still process records that arrive more than delay late.

This mechanism was chosen for the initial implementation over processing time for two reasons:
  - it is robust to downtime that could affect processing delay
  - it does not require syncing of time or timezones between the producer and the processing engine.

### Other notable implementation details
 - A new trigger metric `eventTimeWatermark` outputs the current value of the watermark.
 - We mark the event time column in the `Attribute` metadata using the key `spark.watermarkDelay`.  This allows downstream operations to know which column holds the event time.  Operations like `window` propagate this metadata.
 - `explain()` marks the watermark with a suffix of `-T${delayMs}` to ease debugging of how this information is propagated.
 - Currently, we don't filter out late records, but instead rely on the state store to avoid emitting records that are both added and filtered in the same epoch.

### Remaining in this PR
 - [ ] The test for recovery is currently failing as we don't record the watermark used in the offset log.  We will need to do so to ensure determinism, but this is deferred until #15626 is merged.

### Other follow-ups
There are some natural additional features that we should consider for future work:
 - Ability to write records that arrive too late to some external store in case any out-of-band remediation is required.
 - `Update` mode so you can get partial results before a group is evicted.
 - Other mechanisms for calculating the watermark.  In particular a watermark based on quantiles would be more robust to outliers.

Author: Michael Armbrust <michael@databricks.com>

Closes #15702 from marmbrus/watermarks.
2016-11-14 16:46:26 -08:00