Commit graph

3088 commits

Author SHA1 Message Date
Shixiong Zhu d2923f1732 [SPARK-18143][SQL] Ignore Structured Streaming event logs to avoid breaking history server
## What changes were proposed in this pull request?

Because of the refactoring work in Structured Streaming, the event logs generated by Strucutred Streaming in Spark 2.0.0 and 2.0.1 cannot be parsed.

This PR just ignores these logs in ReplayListenerBus because no places use them.
## How was this patch tested?
- Generated events logs using Spark 2.0.0 and 2.0.1, and saved them as `structured-streaming-query-event-logs-2.0.0.txt` and `structured-streaming-query-event-logs-2.0.1.txt`
- The new added test makes sure ReplayListenerBus will skip these bad jsons.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15663 from zsxwing/fix-event-log.
2016-10-31 00:11:33 -07:00
Dongjoon Hyun 8ae2da0b25 [SPARK-18106][SQL] ANALYZE TABLE should raise a ParseException for invalid option
## What changes were proposed in this pull request?

Currently, `ANALYZE TABLE` command accepts `identifier` for option `NOSCAN`. This PR raises a ParseException for unknown option.

**Before**
```scala
scala> sql("create table test(a int)")
res0: org.apache.spark.sql.DataFrame = []

scala> sql("analyze table test compute statistics blah")
res1: org.apache.spark.sql.DataFrame = []
```

**After**
```scala
scala> sql("create table test(a int)")
res0: org.apache.spark.sql.DataFrame = []

scala> sql("analyze table test compute statistics blah")
org.apache.spark.sql.catalyst.parser.ParseException:
Expected `NOSCAN` instead of `blah`(line 1, pos 0)
```

## How was this patch tested?

Pass the Jenkins test with a new test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15640 from dongjoon-hyun/SPARK-18106.
2016-10-30 23:24:30 +01:00
Eric Liang 90d3b91f4c [SPARK-18103][SQL] Rename *FileCatalog to *FileIndex
## What changes were proposed in this pull request?

To reduce the number of components in SQL named *Catalog, rename *FileCatalog to *FileIndex. A FileIndex is responsible for returning the list of partitions / files to scan given a filtering expression.

```
TableFileCatalog => CatalogFileIndex
FileCatalog => FileIndex
ListingFileCatalog => InMemoryFileIndex
MetadataLogFileCatalog => MetadataLogFileIndex
PrunedTableFileCatalog => PrunedInMemoryFileIndex
```

cc yhuai marmbrus

## How was this patch tested?

N/A

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

Closes #15634 from ericl/rename-file-provider.
2016-10-30 13:14:45 -07:00
Eric Liang 3ad99f1664 [SPARK-18146][SQL] Avoid using Union to chain together create table and repair partition commands
## What changes were proposed in this pull request?

The behavior of union is not well defined here. It is safer to explicitly execute these commands in order. The other use of `Union` in this way will be removed by https://github.com/apache/spark/pull/15633

## How was this patch tested?

Existing tests.

cc yhuai cloud-fan

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

Closes #15665 from ericl/spark-18146.
2016-10-30 20:27:38 +08:00
Shixiong Zhu 59cccbda48 [SPARK-18164][SQL] ForeachSink should fail the Spark job if process throws exception
## What changes were proposed in this pull request?

Fixed the issue that ForeachSink didn't rethrow the exception.

## How was this patch tested?

The fixed unit test.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15674 from zsxwing/foreach-sink-error.
2016-10-28 20:14:38 -07:00
Eric Liang ccb1154304 [SPARK-17970][SQL] store partition spec in metastore for data source table
## What changes were proposed in this pull request?

We should follow hive table and also store partition spec in metastore for data source table.
This brings 2 benefits:

1. It's more flexible to manage the table data files, as users can use `ADD PARTITION`, `DROP PARTITION` and `RENAME PARTITION`
2. We don't need to cache all file status for data source table anymore.

## How was this patch tested?

existing tests.

Author: Eric Liang <ekl@databricks.com>
Author: Michael Allman <michael@videoamp.com>
Author: Eric Liang <ekhliang@gmail.com>
Author: Wenchen Fan <wenchen@databricks.com>

Closes #15515 from cloud-fan/partition.
2016-10-27 14:22:30 -07:00
Shixiong Zhu 79fd0cc058 [SPARK-16963][SQL] Fix test "StreamExecution metadata garbage collection"
## What changes were proposed in this pull request?

A follow up PR for #14553 to fix the flaky test. It's flaky because the file list API doesn't guarantee any order of the return list.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15661 from zsxwing/fix-StreamingQuerySuite.
2016-10-27 12:32:58 -07:00
VinceShieh 0b076d4cb6 [SPARK-17219][ML] enhanced NaN value handling in Bucketizer
## What changes were proposed in this pull request?

This PR is an enhancement of PR with commit ID:57dc326bd00cf0a49da971e9c573c48ae28acaa2.
NaN is a special type of value which is commonly seen as invalid. But We find that there are certain cases where NaN are also valuable, thus need special handling. We provided user when dealing NaN values with 3 options, to either reserve an extra bucket for NaN values, or remove the NaN values, or report an error, by setting handleNaN "keep", "skip", or "error"(default) respectively.

'''Before:
val bucketizer: Bucketizer = new Bucketizer()
          .setInputCol("feature")
          .setOutputCol("result")
          .setSplits(splits)
'''After:
val bucketizer: Bucketizer = new Bucketizer()
          .setInputCol("feature")
          .setOutputCol("result")
          .setSplits(splits)
          .setHandleNaN("keep")

## How was this patch tested?
Tests added in QuantileDiscretizerSuite, BucketizerSuite and DataFrameStatSuite

Signed-off-by: VinceShieh <vincent.xieintel.com>

Author: VinceShieh <vincent.xie@intel.com>
Author: Vincent Xie <vincent.xie@intel.com>
Author: Joseph K. Bradley <joseph@databricks.com>

Closes #15428 from VinceShieh/spark-17219_followup.
2016-10-27 11:52:15 -07:00
Felix Cheung 44c8bfda79 [SQL][DOC] updating doc for JSON source to link to jsonlines.org
## What changes were proposed in this pull request?

API and programming guide doc changes for Scala, Python and R.

## How was this patch tested?

manual test

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #15629 from felixcheung/jsondoc.
2016-10-26 23:06:11 -07:00
Dilip Biswal dd4f088c1d [SPARK-18009][SQL] Fix ClassCastException while calling toLocalIterator() on dataframe produced by RunnableCommand
## What changes were proposed in this pull request?
A short code snippet that uses toLocalIterator() on a dataframe produced by a RunnableCommand
reproduces the problem. toLocalIterator() is called by thriftserver when
`spark.sql.thriftServer.incrementalCollect`is set to handle queries producing large result
set.

**Before**
```SQL
scala> spark.sql("show databases")
res0: org.apache.spark.sql.DataFrame = [databaseName: string]

scala> res0.toLocalIterator()
16/10/26 03:00:24 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0)
java.lang.ClassCastException: org.apache.spark.sql.catalyst.expressions.GenericInternalRow cannot be cast to org.apache.spark.sql.catalyst.expressions.UnsafeRow
```

**After**
```SQL
scala> spark.sql("drop database databases")
res30: org.apache.spark.sql.DataFrame = []

scala> spark.sql("show databases")
res31: org.apache.spark.sql.DataFrame = [databaseName: string]

scala> res31.toLocalIterator().asScala foreach println
[default]
[parquet]
```
## How was this patch tested?
Added a test in DDLSuite

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

Closes #15642 from dilipbiswal/SPARK-18009.
2016-10-27 13:12:14 +08:00
frreiss 5b27598ff5 [SPARK-16963][STREAMING][SQL] Changes to Source trait and related implementation classes
## What changes were proposed in this pull request?

This PR contains changes to the Source trait such that the scheduler can notify data sources when it is safe to discard buffered data. Summary of changes:
* Added a method `commit(end: Offset)` that tells the Source that is OK to discard all offsets up `end`, inclusive.
* Changed the semantics of a `None` value for the `getBatch` method to mean "from the very beginning of the stream"; as opposed to "all data present in the Source's buffer".
* Added notes that the upper layers of the system will never call `getBatch` with a start value less than the last value passed to `commit`.
* Added a `lastCommittedOffset` method to allow the scheduler to query the status of each Source on restart. This addition is not strictly necessary, but it seemed like a good idea -- Sources will be maintaining their own persistent state, and there may be bugs in the checkpointing code.
* The scheduler in `StreamExecution.scala` now calls `commit` on its stream sources after marking each batch as complete in its checkpoint.
* `MemoryStream` now cleans committed batches out of its internal buffer.
* `TextSocketSource` now cleans committed batches from its internal buffer.

## How was this patch tested?
Existing regression tests already exercise the new code.

Author: frreiss <frreiss@us.ibm.com>

Closes #14553 from frreiss/fred-16963.
2016-10-26 17:33:08 -07:00
jiangxingbo 5b7d403c18 [SPARK-18094][SQL][TESTS] Move group analytics test cases from SQLQuerySuite into a query file test.
## What changes were proposed in this pull request?

Currently we have several test cases for group analytics(ROLLUP/CUBE/GROUPING SETS) in `SQLQuerySuite`, should better move them into a query file test.
The following test cases are moved to `group-analytics.sql`:
```
test("rollup")
test("grouping sets when aggregate functions containing groupBy columns")
test("cube")
test("grouping sets")
test("grouping and grouping_id")
test("grouping and grouping_id in having")
test("grouping and grouping_id in sort")
```

This is followup work of #15582

## How was this patch tested?

Modified query file `group-analytics.sql`, which will be tested by `SQLQueryTestSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15624 from jiangxb1987/group-analytics-test.
2016-10-26 23:51:16 +02:00
Shixiong Zhu 7ac70e7ba8 [SPARK-13747][SQL] Fix concurrent executions in ForkJoinPool for SQL
## What changes were proposed in this pull request?

Calling `Await.result` will allow other tasks to be run on the same thread when using ForkJoinPool. However, SQL uses a `ThreadLocal` execution id to trace Spark jobs launched by a query, which doesn't work perfectly in ForkJoinPool.

This PR just uses `Awaitable.result` instead to  prevent ForkJoinPool from running other tasks in the current waiting thread.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15520 from zsxwing/SPARK-13747.
2016-10-26 10:36:36 -07:00
Mark Grover 4bee954079 [SPARK-18093][SQL] Fix default value test in SQLConfSuite to work rega…
…rdless of warehouse dir's existence

## What changes were proposed in this pull request?
Appending a trailing slash, if there already isn't one for the
sake comparison of the two paths. It doesn't take away from
the essence of the check, but removes any potential mismatch
due to lack of trailing slash.

## How was this patch tested?
Ran unit tests and they passed.

Author: Mark Grover <mark@apache.org>

Closes #15623 from markgrover/spark-18093.
2016-10-26 09:07:30 -07:00
jiangxingbo 3c023570b2 [SPARK-17733][SQL] InferFiltersFromConstraints rule never terminates for query
## What changes were proposed in this pull request?

The function `QueryPlan.inferAdditionalConstraints` and `UnaryNode.getAliasedConstraints` can produce a non-converging set of constraints for recursive functions. For instance, if we have two constraints of the form(where a is an alias):
`a = b, a = f(b, c)`
Applying both these rules in the next iteration would infer:
`f(b, c) = f(f(b, c), c)`
This process repeated, the iteration won't converge and the set of constraints will grow larger and larger until OOM.

~~To fix this problem, we collect alias from expressions and skip infer constraints if we are to transform an `Expression` to another which contains it.~~
To fix this problem, we apply additional check in `inferAdditionalConstraints`, when it's possible to generate recursive constraints, we skip generate that.

## How was this patch tested?

Add new testcase in `SQLQuerySuite`/`InferFiltersFromConstraintsSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15319 from jiangxb1987/constraints.
2016-10-26 17:09:48 +02:00
Sean Owen 6c7d094ec4
[SPARK-18022][SQL] java.lang.NullPointerException instead of real exception when saving DF to MySQL
## What changes were proposed in this pull request?

On null next exception in JDBC, don't init it as cause or suppressed

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #15599 from srowen/SPARK-18022.
2016-10-26 14:19:40 +02:00
gatorsmile 93b8ad184a [SPARK-17693][SQL] Fixed Insert Failure To Data Source Tables when the Schema has the Comment Field
### What changes were proposed in this pull request?
```SQL
CREATE TABLE tab1(col1 int COMMENT 'a', col2 int) USING parquet
INSERT INTO TABLE tab1 SELECT 1, 2
```
The insert attempt will fail if the target table has a column with comments. The error is strange to the external users:
```
assertion failed: No plan for InsertIntoTable Relation[col1#15,col2#16] parquet, false, false
+- Project [1 AS col1#19, 2 AS col2#20]
   +- OneRowRelation$
```

This PR is to fix the above bug by checking the metadata when comparing the schema between the table and the query. If not matched, we also copy the metadata. This is an alternative to https://github.com/apache/spark/pull/15266

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15615 from gatorsmile/insertDataSourceTableWithCommentSolution2.
2016-10-26 00:38:34 -07:00
Wenchen Fan a21791e316 [SPARK-18070][SQL] binary operator should not consider nullability when comparing input types
## What changes were proposed in this pull request?

Binary operator requires its inputs to be of same type, but it should not consider nullability, e.g. `EqualTo` should be able to compare an element-nullable array and an element-non-nullable array.

## How was this patch tested?

a regression test in `DataFrameSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15606 from cloud-fan/type-bug.
2016-10-25 12:08:17 -07:00
Wenchen Fan 6f31833dbe [SPARK-18026][SQL] should not always lowercase partition columns of partition spec in parser
## What changes were proposed in this pull request?

Currently we always lowercase the partition columns of partition spec in parser, with the assumption that table partition columns are always lowercased.

However, this is not true for data source tables, which are case preserving. It's safe for now because data source tables don't store partition spec in metastore and don't support `ADD PARTITION`, `DROP PARTITION`, `RENAME PARTITION`, but we should make our code future-proof.

This PR makes partition spec case preserving at parser, and improve the `PreprocessTableInsertion` analyzer rule to normalize the partition columns in partition spec, w.r.t. the table partition columns.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15566 from cloud-fan/partition-spec.
2016-10-25 15:00:33 +08:00
gatorsmile d479c52622 [SPARK-17409][SQL][FOLLOW-UP] Do Not Optimize Query in CTAS More Than Once
### What changes were proposed in this pull request?
This follow-up PR is for addressing the [comment](https://github.com/apache/spark/pull/15048).

We added two test cases based on the suggestion from yhuai . One is a new test case using the `saveAsTable` API to create a data source table. Another is for CTAS on Hive serde table.

Note: No need to backport this PR to 2.0. Will submit a new PR to backport the whole fix with new test cases to Spark 2.0

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15459 from gatorsmile/ctasOptimizedTestCases.
2016-10-25 10:47:11 +08:00
Wenchen Fan 84a3399908 [SPARK-18028][SQL] simplify TableFileCatalog
## What changes were proposed in this pull request?

Simplify/cleanup TableFileCatalog:

1. pass a `CatalogTable` instead of `databaseName` and `tableName` into `TableFileCatalog`, so that we don't need to fetch table metadata from metastore again
2. In `TableFileCatalog.filterPartitions0`, DO NOT set `PartitioningAwareFileCatalog.BASE_PATH_PARAM`. According to the [classdoc](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala#L189-L209), the default value of `basePath` already satisfies our need. What's more, if we set this parameter, we may break the case 2 which is metioned in the classdoc.
3. add `equals` and `hashCode` to `TableFileCatalog`
4. add `SessionCatalog.listPartitionsByFilter` which handles case sensitivity.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15568 from cloud-fan/table-file-catalog.
2016-10-25 08:42:21 +08:00
Tathagata Das 407c3cedf2 [SPARK-17624][SQL][STREAMING][TEST] Fixed flaky StateStoreSuite.maintenance
## What changes were proposed in this pull request?

The reason for the flakiness was follows. The test starts the maintenance background thread, and then writes 20 versions of the state store. The maintenance thread is expected to create snapshots in the middle, and clean up old files that are not needed any more. The earliest delta file (1.delta) is expected to be deleted as snapshots will ensure that the earliest delta would not be needed.

However, the default configuration for the maintenance thread is to retain files such that last 2 versions can be recovered, and delete the rest. Now while generating the versions, the maintenance thread can kick in and create snapshots anywhere between version 10 and 20 (at least 10 deltas needed for snapshot). Then later it will choose to retain only version 20 and 19 (last 2). There are two cases.

- Common case: One of the version between 10 and 19 gets snapshotted. Then recovering versions 19 and 20 just needs 19.snapshot and 20.delta, so 1.delta gets deleted.

- Uncommon case (reason for flakiness): Only version 20 gets snapshotted. Then recovering versoin 20 requires 20.snapshot, and recovering version 19 all the previous 19...1.delta. So 1.delta does not get deleted.

This PR rearranges the checks such that it create 20 versions, and then waits that there is at least one snapshot, then creates another 20. This will ensure that the latest 2 versions cannot require anything older than the first snapshot generated, and therefore will 1.delta will be deleted.

In addition, I have added more logs, and comments that I felt would help future debugging and understanding what is going on.

## How was this patch tested?

Ran the StateStoreSuite > 6K times in a heavily loaded machine (10 instances of tests running in parallel). No failures.

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

Closes #15592 from tdas/SPARK-17624.
2016-10-24 17:21:16 -07:00
Sean Owen 4ecbe1b92f
[SPARK-17810][SQL] Default spark.sql.warehouse.dir is relative to local FS but can resolve as HDFS path
## What changes were proposed in this pull request?

Always resolve spark.sql.warehouse.dir as a local path, and as relative to working dir not home dir

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #15382 from srowen/SPARK-17810.
2016-10-24 10:44:45 +01:00
jiangxingbo b158256c2e [SPARK-18045][SQL][TESTS] Move HiveDataFrameAnalyticsSuite to package sql
## What changes were proposed in this pull request?

The testsuite `HiveDataFrameAnalyticsSuite` has nothing to do with HIVE, we should move it to package `sql`.
The original test cases in that suite are splited into two existing testsuites: `DataFrameAggregateSuite` tests for the functions and ~~`SQLQuerySuite`~~`SQLQueryTestSuite` tests for the SQL statements.

## How was this patch tested?
~~Modified `SQLQuerySuite` in package `sql`.~~
Add query file for `SQLQueryTestSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15582 from jiangxb1987/group-analytics-test.
2016-10-23 13:28:35 +02:00
Tejas Patil 21c7539a52 [SPARK-18038][SQL] Move output partitioning definition from UnaryNodeExec to its children
## What changes were proposed in this pull request?

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

This was a suggestion by rxin over one of the dev list discussion : http://apache-spark-developers-list.1001551.n3.nabble.com/Project-not-preserving-child-partitioning-td19417.html

His words:

>> It would be better (safer) to move the output partitioning definition into each of the operator and remove it from UnaryExecNode.

With this PR, following is the output partitioning and ordering for all the impls of `UnaryExecNode`.

UnaryExecNode's impl | outputPartitioning | outputOrdering | comment
------------ | ------------- | ------------ | ------------
AppendColumnsExec | child's | Nil | child's ordering can be used
AppendColumnsWithObjectExec | child's | Nil | child's ordering can be used
BroadcastExchangeExec | BroadcastPartitioning | Nil | -
CoalesceExec | UnknownPartitioning | Nil | -
CollectLimitExec | SinglePartition | Nil | -
DebugExec | child's | Nil | child's ordering can be used
DeserializeToObjectExec | child's | Nil | child's ordering can be used
ExpandExec | UnknownPartitioning | Nil | -
FilterExec | child's | child's | -
FlatMapGroupsInRExec | child's | Nil | child's ordering can be used
GenerateExec | child's | Nil | need to dig more
GlobalLimitExec | child's | child's | -
HashAggregateExec | child's | Nil | -
InputAdapter | child's | child's | -
InsertIntoHiveTable | child's | Nil | terminal node, doesn't need partitioning
LocalLimitExec | child's | child's | -
MapElementsExec | child's | child's | -
MapGroupsExec | child's | Nil | child's ordering can be used
MapPartitionsExec | child's | Nil | child's ordering can be used
ProjectExec | child's | child's | -
SampleExec | child's | Nil | child's ordering can be used
ScriptTransformation | child's | Nil | child's ordering can be used
SerializeFromObjectExec | child's | Nil | child's ordering can be used
ShuffleExchange | custom | Nil | -
SortAggregateExec | child's | sort over grouped exprs | -
SortExec | child's | custom | -
StateStoreRestoreExec  | child's | Nil | child's ordering can be used
StateStoreSaveExec | child's | Nil | child's ordering can be used
SubqueryExec | child's | child's | -
TakeOrderedAndProjectExec | SinglePartition | custom | -
WholeStageCodegenExec | child's | child's | -
WindowExec | child's | child's | -

## How was this patch tested?

This does NOT change any existing functionality so relying on existing tests

Author: Tejas Patil <tejasp@fb.com>

Closes #15575 from tejasapatil/SPARK-18038_UnaryNodeExec_output_partitioning.
2016-10-23 13:25:47 +02:00
Tejas Patil eff4aed1ac [SPARK-18035][SQL] Introduce performant and memory efficient APIs to create ArrayBasedMapData
## What changes were proposed in this pull request?

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

In HiveInspectors, I saw that converting Java map to Spark's `ArrayBasedMapData` spent quite sometime in buffer copying : https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala#L658

The reason being `map.toSeq` allocates a new buffer and copies the map entries to it: https://github.com/scala/scala/blob/2.11.x/src/library/scala/collection/MapLike.scala#L323

This copy is not needed as we get rid of it once we extract the key and value arrays.

Here is the call trace:

```
org.apache.spark.sql.hive.HiveInspectors$$anonfun$unwrapperFor$41.apply(HiveInspectors.scala:664)
scala.collection.AbstractMap.toSeq(Map.scala:59)
scala.collection.MapLike$class.toSeq(MapLike.scala:323)
scala.collection.AbstractMap.toBuffer(Map.scala:59)
scala.collection.MapLike$class.toBuffer(MapLike.scala:326)
scala.collection.AbstractTraversable.copyToBuffer(Traversable.scala:104)
scala.collection.TraversableOnce$class.copyToBuffer(TraversableOnce.scala:275)
scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:48)
scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:104)
scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:59)
scala.collection.AbstractIterable.foreach(Iterable.scala:54)
scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
scala.collection.AbstractIterator.foreach(Iterator.scala:1336)
scala.collection.Iterator$class.foreach(Iterator.scala:893)
scala.collection.generic.Growable$$anonfun$$plus$plus$eq$1.apply(Growable.scala:59)
scala.collection.generic.Growable$$anonfun$$plus$plus$eq$1.apply(Growable.scala:59)
```

Also, earlier code was populating keys and values arrays separately by iterating twice. The PR avoids double iteration of the map and does it in one iteration.

EDIT: During code review, there were several more places in the code which were found to do similar thing. The PR dedupes those instances and introduces convenient APIs which are performant and memory efficient

## Performance gains

The number is subjective and depends on how many map columns are accessed in the query and average entries per map. For one the queries that I tried out, I saw 3% CPU savings (end-to-end) for the query.

## How was this patch tested?

This does not change the end result produced so relying on existing tests.

Author: Tejas Patil <tejasp@fb.com>

Closes #15573 from tejasapatil/SPARK-18035_avoid_toSeq.
2016-10-22 20:43:43 -07:00
hyukjinkwon 5fa9f8795a [SPARK-17123][SQL] Use type-widened encoder for DataFrame rather than existing encoder to allow type-widening from set operations
# What changes were proposed in this pull request?

This PR fixes set operations in `DataFrame` to be performed fine without exceptions when the types are non-scala native types. (e.g, `TimestampType`, `DateType` and `DecimalType`).

The problem is, it seems set operations such as `union`, `intersect` and `except` uses the encoder belonging to the `Dataset` in caller.

So, `Dataset` of the caller holds `ExpressionEncoder[Row]` as it is when the set operations are performed. However, the return types can be actually widen. So, we should use `ExpressionEncoder[Row]` constructed from executed plan rather than using existing one. Otherwise, this will generate some codes wrongly via `StaticInvoke`.

Running the codes below:

```scala
val dates = Seq(
  (new Date(0), BigDecimal.valueOf(1), new Timestamp(2)),
  (new Date(3), BigDecimal.valueOf(4), new Timestamp(5))
).toDF("date", "timestamp", "decimal")

val widenTypedRows = Seq(
  (new Timestamp(2), 10.5D, "string")
).toDF("date", "timestamp", "decimal")

val results = dates.union(widenTypedRows).collect()
results.foreach(println)
```

prints below:

**Before**

```java
23:08:54.490 ERROR org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 28, Column 107: No applicable constructor/method found for actual parameters "long"; candidates are: "public static java.sql.Date org.apache.spark.sql.catalyst.util.DateTimeUtils.toJavaDate(int)"
/* 001 */ public java.lang.Object generate(Object[] references) {
/* 002 */   return new SpecificSafeProjection(references);
/* 003 */ }
/* 004 */
/* 005 */ class SpecificSafeProjection extends org.apache.spark.sql.catalyst.expressions.codegen.BaseProjection {
/* 006 */
/* 007 */   private Object[] references;
/* 008 */   private MutableRow mutableRow;
/* 009 */   private Object[] values;
/* 010 */   private org.apache.spark.sql.types.StructType schema;
/* 011 */
/* 012 */
/* 013 */   public SpecificSafeProjection(Object[] references) {
/* 014 */     this.references = references;
/* 015 */     mutableRow = (MutableRow) references[references.length - 1];
/* 016 */
/* 017 */     this.schema = (org.apache.spark.sql.types.StructType) references[0];
/* 018 */   }
/* 019 */
/* 020 */   public java.lang.Object apply(java.lang.Object _i) {
/* 021 */     InternalRow i = (InternalRow) _i;
/* 022 */
/* 023 */     values = new Object[3];
/* 024 */
/* 025 */     boolean isNull2 = i.isNullAt(0);
/* 026 */     long value2 = isNull2 ? -1L : (i.getLong(0));
/* 027 */     boolean isNull1 = isNull2;
/* 028 */     final java.sql.Date value1 = isNull1 ? null : org.apache.spark.sql.catalyst.util.DateTimeUtils.toJavaDate(value2);
/* 029 */     isNull1 = value1 == null;
/* 030 */     if (isNull1) {
/* 031 */       values[0] = null;
/* 032 */     } else {
/* 033 */       values[0] = value1;
/* 034 */     }
/* 035 */
/* 036 */     boolean isNull4 = i.isNullAt(1);
/* 037 */     double value4 = isNull4 ? -1.0 : (i.getDouble(1));
/* 038 */
/* 039 */     boolean isNull3 = isNull4;
/* 040 */     java.math.BigDecimal value3 = null;
/* 041 */     if (!isNull3) {
/* 042 */
/* 043 */       Object funcResult = null;
/* 044 */       funcResult = value4.toJavaBigDecimal();
/* 045 */       if (funcResult == null) {
/* 046 */         isNull3 = true;
/* 047 */       } else {
/* 048 */         value3 = (java.math.BigDecimal) funcResult;
/* 049 */       }
/* 050 */
/* 051 */     }
/* 052 */     isNull3 = value3 == null;
/* 053 */     if (isNull3) {
/* 054 */       values[1] = null;
/* 055 */     } else {
/* 056 */       values[1] = value3;
/* 057 */     }
/* 058 */
/* 059 */     boolean isNull6 = i.isNullAt(2);
/* 060 */     UTF8String value6 = isNull6 ? null : (i.getUTF8String(2));
/* 061 */     boolean isNull5 = isNull6;
/* 062 */     final java.sql.Timestamp value5 = isNull5 ? null : org.apache.spark.sql.catalyst.util.DateTimeUtils.toJavaTimestamp(value6);
/* 063 */     isNull5 = value5 == null;
/* 064 */     if (isNull5) {
/* 065 */       values[2] = null;
/* 066 */     } else {
/* 067 */       values[2] = value5;
/* 068 */     }
/* 069 */
/* 070 */     final org.apache.spark.sql.Row value = new org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema(values, schema);
/* 071 */     if (false) {
/* 072 */       mutableRow.setNullAt(0);
/* 073 */     } else {
/* 074 */
/* 075 */       mutableRow.update(0, value);
/* 076 */     }
/* 077 */
/* 078 */     return mutableRow;
/* 079 */   }
/* 080 */ }
```

**After**

```bash
[1969-12-31 00:00:00.0,1.0,1969-12-31 16:00:00.002]
[1969-12-31 00:00:00.0,4.0,1969-12-31 16:00:00.005]
[1969-12-31 16:00:00.002,10.5,string]
```

## How was this patch tested?

Unit tests in `DataFrameSuite`

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15072 from HyukjinKwon/SPARK-17123.
2016-10-22 20:09:04 +02:00
Eric Liang 3eca283aca [SPARK-17994][SQL] Add back a file status cache for catalog tables
## What changes were proposed in this pull request?

In SPARK-16980, we removed the full in-memory cache of table partitions in favor of loading only needed partitions from the metastore. This greatly improves the initial latency of queries that only read a small fraction of table partitions.

However, since the metastore does not store file statistics, we need to discover those from remote storage. With the loss of the in-memory file status cache this has to happen on each query, increasing the latency of repeated queries over the same partitions.

The proposal is to add back a per-table cache of partition contents, i.e. Map[Path, Array[FileStatus]]. This cache would be retained per-table, and can be invalidated through refreshTable() and refreshByPath(). Unlike the prior cache, it can be incrementally updated as new partitions are read.

## How was this patch tested?

Existing tests and new tests in `HiveTablePerfStatsSuite`.

cc mallman

Author: Eric Liang <ekl@databricks.com>
Author: Michael Allman <michael@videoamp.com>
Author: Eric Liang <ekhliang@gmail.com>

Closes #15539 from ericl/meta-cache.
2016-10-22 22:08:28 +08:00
Sean Owen 7178c56433 [SPARK-16606][MINOR] Tiny follow-up to , to correct more instances of the same log message typo
## What changes were proposed in this pull request?

Tiny follow-up to SPARK-16606 / https://github.com/apache/spark/pull/14533 , to correct more instances of the same log message typo

## How was this patch tested?

Existing tests (no functional change anyway)

Author: Sean Owen <sowen@cloudera.com>

Closes #15586 from srowen/SPARK-16606.2.
2016-10-21 22:20:52 -07:00
Reynold Xin 3fbf5a58c2 [SPARK-18042][SQL] OutputWriter should expose file path written
## What changes were proposed in this pull request?
This patch adds a new "path" method on OutputWriter that returns the path of the file written by the OutputWriter. This is part of the necessary work to consolidate structured streaming and batch write paths.

The batch write path has a nice feature that each data source can define the extension of the files, and allow Spark to specify the staging directory and the prefix for the files. However, in the streaming path we need to collect the list of files written, and there is no interface right now to do that.

## How was this patch tested?
N/A - there is no behavior change and this should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #15580 from rxin/SPARK-18042.
2016-10-21 17:27:18 -07:00
Wenchen Fan 140570252f [SPARK-18044][STREAMING] FileStreamSource should not infer partitions in every batch
## What changes were proposed in this pull request?

In `FileStreamSource.getBatch`, we will create a `DataSource` with specified schema, to avoid inferring the schema again and again. However, we don't pass the partition columns, and will infer the partition again and again.

This PR fixes it by keeping the partition columns in `FileStreamSource`, like schema.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15581 from cloud-fan/stream.
2016-10-21 15:28:16 -07:00
Tathagata Das 7a531e3054 [SPARK-17926][SQL][STREAMING] Added json for statuses
## What changes were proposed in this pull request?

StreamingQueryStatus exposed through StreamingQueryListener often needs to be recorded (similar to SparkListener events). This PR adds `.json` and `.prettyJson` to `StreamingQueryStatus`, `SourceStatus` and `SinkStatus`.

## How was this patch tested?
New unit tests

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

Closes #15476 from tdas/SPARK-17926.
2016-10-21 13:07:29 -07:00
Zheng RuiFeng a8ea4da8d0
[SPARK-17331][FOLLOWUP][ML][CORE] Avoid allocating 0-length arrays
## What changes were proposed in this pull request?

`Array[T]()` -> `Array.empty[T]` to avoid allocating 0-length arrays.
Use regex `find . -name '*.scala' | xargs -i bash -c 'egrep "Array\[[A-Za-z]+\]\(\)" -n {} && echo {}'` to find modification candidates.

cc srowen

## How was this patch tested?
existing tests

Author: Zheng RuiFeng <ruifengz@foxmail.com>

Closes #15564 from zhengruifeng/avoid_0_length_array.
2016-10-21 09:49:37 +01:00
Wenchen Fan 57e97fcbd6 [SPARK-18029][SQL] PruneFileSourcePartitions should not change the output of LogicalRelation
## What changes were proposed in this pull request?

In `PruneFileSourcePartitions`, we will replace the `LogicalRelation` with a pruned one. However, this replacement may change the output of the `LogicalRelation` if it doesn't have `expectedOutputAttributes`. This PR fixes it.

## How was this patch tested?

the new `PruneFileSourcePartitionsSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15569 from cloud-fan/partition-bug.
2016-10-21 12:27:53 +08:00
Shixiong Zhu 1bb99c4887 [SPARK-18030][TESTS] Adds more checks to collect more info about FileStreamSourceSuite failure
## What changes were proposed in this pull request?

My hunch is `mkdirs` fails. Just add more checks to collect more info.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15577 from zsxwing/SPARK-18030-debug.
2016-10-20 20:44:32 -07:00
Reynold Xin 7f9ec19eae [SPARK-18021][SQL] Refactor file name specification for data sources
## What changes were proposed in this pull request?
Currently each data source OutputWriter is responsible for specifying the entire file name for each file output. This, however, does not make any sense because we rely on file naming schemes for certain behaviors in Spark SQL, e.g. bucket id. The current approach allows individual data sources to break the implementation of bucketing.

On the flip side, we also don't want to move file naming entirely out of data sources, because different data sources do want to specify different extensions.

This patch divides file name specification into two parts: the first part is a prefix specified by the caller of OutputWriter (in WriteOutput), and the second part is the suffix that can be specified by the OutputWriter itself. Note that a side effect of this change is that now all file based data sources also support bucketing automatically.

There are also some other minor cleanups:

- Removed the UUID passed through generic Configuration string
- Some minor rewrites for better clarity
- Renamed "path" in multiple places to "stagingDir", to more accurately reflect its meaning

## How was this patch tested?
This should be covered by existing data source tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #15562 from rxin/SPARK-18021.
2016-10-20 12:18:56 -07:00
Koert Kuipers 84b245f2dd [SPARK-15780][SQL] Support mapValues on KeyValueGroupedDataset
## What changes were proposed in this pull request?

Add mapValues to KeyValueGroupedDataset

## How was this patch tested?

New test in DatasetSuite for groupBy function, mapValues, flatMap

Author: Koert Kuipers <koert@tresata.com>

Closes #13526 from koertkuipers/feat-keyvaluegroupeddataset-mapvalues.
2016-10-20 10:08:12 -07:00
Dilip Biswal e895bc2548 [SPARK-17860][SQL] SHOW COLUMN's database conflict check should respect case sensitivity configuration
## What changes were proposed in this pull request?
SHOW COLUMNS command validates the user supplied database
name with database name from qualified table name name to make
sure both of them are consistent. This comparison should respect
case sensitivity.

## How was this patch tested?
Added tests in DDLSuite and existing tests were moved to use new sql based test infrastructure.

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

Closes #15423 from dilipbiswal/dkb_show_column_fix.
2016-10-20 19:39:25 +08:00
Dongjoon Hyun 986a3b8b5b
[SPARK-17796][SQL] Support wildcard character in filename for LOAD DATA LOCAL INPATH
## What changes were proposed in this pull request?

Currently, Spark 2.0 raises an `input path does not exist` AnalysisException if the file name contains '*'. It is misleading since it occurs when there exist some matched files. Also, it was a supported feature in Spark 1.6.2. This PR aims to support wildcard characters in filename for `LOAD DATA LOCAL INPATH` SQL command like Spark 1.6.2.

**Reported Error Scenario**
```scala
scala> sql("CREATE TABLE t(a string)")
res0: org.apache.spark.sql.DataFrame = []

scala> sql("LOAD DATA LOCAL INPATH '/tmp/x*' INTO TABLE t")
org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: /tmp/x*;
```

## How was this patch tested?

Pass the Jenkins test with a new test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15376 from dongjoon-hyun/SPARK-17796.
2016-10-20 09:53:12 +01:00
Eric Liang 4bd17c4606 [SPARK-17991][SQL] Enable metastore partition pruning by default.
## What changes were proposed in this pull request?

This should apply to non-converted metastore relations. WIP to see if this causes any test failures.

## How was this patch tested?

Existing tests.

Author: Eric Liang <ekl@databricks.com>

Closes #15475 from ericl/try-enabling-pruning.
2016-10-19 23:55:05 -07:00
Reynold Xin f313117bc9 [SPARK-18012][SQL] Simplify WriterContainer
## What changes were proposed in this pull request?
This patch refactors WriterContainer to simplify the logic and make control flow more obvious.The previous code setup made it pretty difficult to track the actual dependencies on variables and setups because the driver side and the executor side were using the same set of variables.

## How was this patch tested?
N/A - this should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #15551 from rxin/writercontainer-refactor.
2016-10-19 22:22:35 -07:00
hyukjinkwon 4b2011ec9d [SPARK-17989][SQL] Check ascendingOrder type in sort_array function rather than throwing ClassCastException
## What changes were proposed in this pull request?

This PR proposes to check the second argument, `ascendingOrder`  rather than throwing `ClassCastException` exception message.

```sql
select sort_array(array('b', 'd'), '1');
```

**Before**

```
16/10/19 13:16:08 ERROR SparkSQLDriver: Failed in [select sort_array(array('b', 'd'), '1')]
java.lang.ClassCastException: org.apache.spark.unsafe.types.UTF8String cannot be cast to java.lang.Boolean
	at scala.runtime.BoxesRunTime.unboxToBoolean(BoxesRunTime.java:85)
	at org.apache.spark.sql.catalyst.expressions.SortArray.nullSafeEval(collectionOperations.scala:185)
	at org.apache.spark.sql.catalyst.expressions.BinaryExpression.eval(Expression.scala:416)
	at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:50)
	at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:43)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:292)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:292)
	at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:74)
	at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:291)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:297)
```

**After**

```
Error in query: cannot resolve 'sort_array(array('b', 'd'), '1')' due to data type mismatch: Sort order in second argument requires a boolean literal.; line 1 pos 7;
```

## How was this patch tested?

Unit test in `DataFrameFunctionsSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15532 from HyukjinKwon/SPARK-17989.
2016-10-19 19:36:21 -07:00
Wenchen Fan 4329c5cea4 [SPARK-17873][SQL] ALTER TABLE RENAME TO should allow users to specify database in destination table name(but have to be same as source table)
## What changes were proposed in this pull request?

Unlike Hive, in Spark SQL, ALTER TABLE RENAME TO cannot move a table from one database to another(e.g. `ALTER TABLE db1.tbl RENAME TO db2.tbl2`), and will report error if the database in source table and destination table is different. So in #14955 , we forbid users to specify database of destination table in ALTER TABLE RENAME TO, to be consistent with other database systems and also make it easier to rename tables in non-current database, e.g. users can write `ALTER TABLE db1.tbl RENAME TO tbl2`, instead of `ALTER TABLE db1.tbl RENAME TO db1.tbl2`.

However, this is a breaking change. Users may already have queries that specify database of destination table in ALTER TABLE RENAME TO.

This PR reverts most of #14955 , and simplify the usage of ALTER TABLE RENAME TO by making database of source table the default database of destination table, instead of current database, so that users can still write `ALTER TABLE db1.tbl RENAME TO tbl2`, which is consistent with other databases like MySQL, Postgres, etc.

## How was this patch tested?

The added back tests and some new tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15434 from cloud-fan/revert.
2016-10-18 20:23:13 -07:00
Eric Liang 5f20ae0394 [SPARK-17980][SQL] Fix refreshByPath for converted Hive tables
## What changes were proposed in this pull request?

There was a bug introduced in https://github.com/apache/spark/pull/14690 which broke refreshByPath with converted hive tables (though, it turns out it was very difficult to refresh converted hive tables anyways, since you had to specify the exact path of one of the partitions).

This changes refreshByPath to invalidate by prefix instead of exact match, and fixes the issue.

cc sameeragarwal for refreshByPath changes
mallman

## How was this patch tested?

Extended unit test.

Author: Eric Liang <ekl@databricks.com>

Closes #15521 from ericl/fix-caching.
2016-10-19 10:20:12 +08:00
Tathagata Das 941b3f9aca [SPARK-17731][SQL][STREAMING][FOLLOWUP] Refactored StreamingQueryListener APIs
## What changes were proposed in this pull request?

As per rxin request, here are further API changes
- Changed `Stream(Started/Progress/Terminated)` events to `Stream*Event`
- Changed the fields in `StreamingQueryListener.on***` from `query*` to `event`

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

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

Closes #15530 from tdas/SPARK-17731-1.
2016-10-18 17:32:16 -07:00
hyukjinkwon b3130c7b6a [SPARK-17955][SQL] Make DataFrameReader.jdbc call DataFrameReader.format("jdbc").load
## What changes were proposed in this pull request?

This PR proposes to make `DataFrameReader.jdbc` call `DataFrameReader.format("jdbc").load` consistently with other APIs in `DataFrameReader`/`DataFrameWriter` and avoid calling `sparkSession.baseRelationToDataFrame(..)` here and there.

The changes were mostly copied from `DataFrameWriter.jdbc()` which was recently updated.

```diff
-    val params = extraOptions.toMap ++ connectionProperties.asScala.toMap
-    val options = new JDBCOptions(url, table, params)
-    val relation = JDBCRelation(parts, options)(sparkSession)
-    sparkSession.baseRelationToDataFrame(relation)
+    this.extraOptions = this.extraOptions ++ connectionProperties.asScala
+    // explicit url and dbtable should override all
+    this.extraOptions += ("url" -> url, "dbtable" -> table)
+    format("jdbc").load()
```

## How was this patch tested?

Existing tests should cover this.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15499 from HyukjinKwon/SPARK-17955.
2016-10-18 13:49:02 -07:00
Eric Liang 4ef39c2f44 [SPARK-17974] try 2) Refactor FileCatalog classes to simplify the inheritance tree
## What changes were proposed in this pull request?

This renames `BasicFileCatalog => FileCatalog`, combines  `SessionFileCatalog` with `PartitioningAwareFileCatalog`, and removes the old `FileCatalog` trait.

In summary,
```
MetadataLogFileCatalog extends PartitioningAwareFileCatalog
ListingFileCatalog extends PartitioningAwareFileCatalog
PartitioningAwareFileCatalog extends FileCatalog
TableFileCatalog extends FileCatalog
```

(note that this is a re-submission of https://github.com/apache/spark/pull/15518 which got reverted)

## How was this patch tested?

Existing tests

Author: Eric Liang <ekl@databricks.com>

Closes #15533 from ericl/fix-scalastyle-revert.
2016-10-18 13:33:46 -07:00
hyukjinkwon 37686539f5 [SPARK-17388] [SQL] Support for inferring type date/timestamp/decimal for partition column
## What changes were proposed in this pull request?

Currently, Spark only supports to infer `IntegerType`, `LongType`, `DoubleType` and `StringType`.

`DecimalType` is being tried but it seems it never infers type as `DecimalType` as `DoubleType` is being tried first. Also, it seems `DateType` and `TimestampType` could be inferred.

As far as I know, it is pretty common to use both for a partition column.

This PR fixes the incorrect `DecimalType` try and also adds the support for both `DateType` and `TimestampType` for inferring partition column type.

## How was this patch tested?

Unit tests in `ParquetPartitionDiscoverySuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #14947 from HyukjinKwon/SPARK-17388.
2016-10-18 13:20:42 -07:00
Tathagata Das a9e79a41ee [SQL][STREAMING][TEST] Follow up to remove Option.contains for Scala 2.10 compatibility
## What changes were proposed in this pull request?

Scala 2.10 does not have Option.contains, which broke Scala 2.10 build.

## How was this patch tested?
Locally compiled and ran sql/core unit tests in 2.10

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

Closes #15531 from tdas/metrics-flaky-test-fix-1.
2016-10-18 02:29:55 -07:00
Liwei Lin 7d878cf2da [SQL][STREAMING][TEST] Fix flaky tests in StreamingQueryListenerSuite
This work has largely been done by lw-lin in his PR #15497. This is a slight refactoring of it.

## What changes were proposed in this pull request?
There were two sources of flakiness in StreamingQueryListener test.

- When testing with manual clock, consecutive attempts to advance the clock can occur without the stream execution thread being unblocked and doing some work between the two attempts. Hence the following can happen with the current ManualClock.
```
+-----------------------------------+--------------------------------+
|      StreamExecution thread       |         testing thread         |
+-----------------------------------+--------------------------------+
|  ManualClock.waitTillTime(100) {  |                                |
|        _isWaiting = true          |                                |
|            wait(10)               |                                |
|        still in wait(10)          |  if (_isWaiting) advance(100)  |
|        still in wait(10)          |  if (_isWaiting) advance(200)  | <- this should be disallowed !
|        still in wait(10)          |  if (_isWaiting) advance(300)  | <- this should be disallowed !
|      wake up from wait(10)        |                                |
|       current time is 600         |                                |
|       _isWaiting = false          |                                |
|  }                                |                                |
+-----------------------------------+--------------------------------+
```

- Second source of flakiness is that the adding data to memory stream may get processing in any trigger, not just the first trigger.

My fix is to make the manual clock wait for the other stream execution thread to start waiting for the clock at the right wait start time. That is, `advance(200)` (see above) will wait for stream execution thread to complete the wait that started at time 0, and start a new wait at time 200 (i.e. time stamp after the previous `advance(100)`).

In addition, since this is a feature that is solely used by StreamExecution, I removed all the non-generic code from ManualClock and put them in StreamManualClock inside StreamTest.

## How was this patch tested?
Ran existing unit test MANY TIME in Jenkins

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

Closes #15519 from tdas/metrics-flaky-test-fix.
2016-10-18 00:49:57 -07:00