Commit graph

4880 commits

Author SHA1 Message Date
Marcelo Vanzin c8e909cd49 [SPARK-24589][CORE] Correctly identify tasks in output commit coordinator.
When an output stage is retried, it's possible that tasks from the previous
attempt are still running. In that case, there would be a new task for the
same partition in the new attempt, and the coordinator would allow both
tasks to commit their output since it did not keep track of stage attempts.

The change adds more information to the stage state tracked by the coordinator,
so that only one task is allowed to commit the output in the above case.
The stage state in the coordinator is also maintained across stage retries,
so that a stray speculative task from a previous stage attempt is not allowed
to commit.

This also removes some code added in SPARK-18113 that allowed for duplicate
commit requests; with the RPC code used in Spark 2, that situation cannot
happen, so there is no need to handle it.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #21577 from vanzin/SPARK-24552.
2018-06-21 13:25:15 -05:00
Chongguang LIU 7236e759c9 [SPARK-24574][SQL] array_contains, array_position, array_remove and element_at functions deal with Column type
## What changes were proposed in this pull request?

For the function ```def array_contains(column: Column, value: Any): Column ``` , if we pass the `value` parameter as a Column type, it will yield a runtime exception.

This PR proposes a pattern matching to detect if `value` is of type Column. If yes, it will use the .expr of the column, otherwise it will work as it used to.

Same thing for ```array_position, array_remove and element_at``` functions

## How was this patch tested?

Unit test modified to cover this code change.

Ping ueshin

Author: Chongguang LIU <chong@Chongguangs-MacBook-Pro.local>

Closes #21581 from chongguang/SPARK-24574.
2018-06-21 14:58:57 +08:00
Maxim Gekk 54fcaafb09 [SPARK-24571][SQL] Support Char literals
## What changes were proposed in this pull request?

In the PR, I propose to automatically convert a `Literal` with `Char` type to a `Literal` of `String` type. Currently, the following code:
```scala
val df = Seq("Amsterdam", "San Francisco", "London").toDF("city")
df.where($"city".contains('o')).show(false)
```
fails with the exception:
```
Unsupported literal type class java.lang.Character o
java.lang.RuntimeException: Unsupported literal type class java.lang.Character o
at org.apache.spark.sql.catalyst.expressions.Literal$.apply(literals.scala:78)
```
The PR fixes this issue by converting `char` to `string` of length `1`. I believe it makes sense to does not differentiate `char` and `string(1)` in _a unified, multi-language data platform_ like Spark which supports languages like Python/R.

Author: Maxim Gekk <maxim.gekk@databricks.com>
Author: Maxim Gekk <max.gekk@gmail.com>

Closes #21578 from MaxGekk/support-char-literals.
2018-06-20 23:38:37 -07:00
Huaxin Gao 9de11d3f90 [SPARK-23912][SQL] add array_distinct
## What changes were proposed in this pull request?

Add array_distinct to remove duplicate value from the array.

## How was this patch tested?

Add unit tests

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

Closes #21050 from huaxingao/spark-23912.
2018-06-21 12:24:53 +09:00
aokolnychyi c5a0d1132a [SPARK-24575][SQL] Prohibit window expressions inside WHERE and HAVING clauses
## What changes were proposed in this pull request?

As discussed [before](https://github.com/apache/spark/pull/19193#issuecomment-393726964), this PR prohibits window expressions inside WHERE and HAVING clauses.

## How was this patch tested?

This PR comes with a dedicated unit test.

Author: aokolnychyi <anton.okolnychyi@sap.com>

Closes #21580 from aokolnychyi/spark-24575.
2018-06-20 18:57:13 +02:00
Jungtaek Lim c8ef9232cf [MINOR][SQL] Remove invalid comment from SparkStrategies
## What changes were proposed in this pull request?

This patch is removing invalid comment from SparkStrategies, given that TODO-like comment is no longer preferred one as the comment: https://github.com/apache/spark/pull/21388#issuecomment-396856235

Removing invalid comment will prevent contributors to spend their times which is not going to be merged.

## How was this patch tested?

N/A

Author: Jungtaek Lim <kabhwan@gmail.com>

Closes #21595 from HeartSaVioR/MINOR-remove-invalid-comment-on-spark-strategies.
2018-06-20 18:38:42 +02:00
Maryann Xue bc0498d582 [SPARK-24583][SQL] Wrong schema type in InsertIntoDataSourceCommand
## What changes were proposed in this pull request?

Change insert input schema type: "insertRelationType" -> "insertRelationType.asNullable", in order to avoid nullable being overridden.

## How was this patch tested?

Added one test in InsertSuite.

Author: Maryann Xue <maryannxue@apache.org>

Closes #21585 from maryannxue/spark-24583.
2018-06-19 15:27:20 -07:00
Tathagata Das 2cb976355c [SPARK-24565][SS] Add API for in Structured Streaming for exposing output rows of each microbatch as a DataFrame
## What changes were proposed in this pull request?

Currently, the micro-batches in the MicroBatchExecution is not exposed to the user through any public API. This was because we did not want to expose the micro-batches, so that all the APIs we expose, we can eventually support them in the Continuous engine. But now that we have better sense of buiding a ContinuousExecution, I am considering adding APIs which will run only the MicroBatchExecution. I have quite a few use cases where exposing the microbatch output as a dataframe is useful.
- Pass the output rows of each batch to a library that is designed only the batch jobs (example, uses many ML libraries need to collect() while learning).
- Reuse batch data sources for output whose streaming version does not exists (e.g. redshift data source).
- Writer the output rows to multiple places by writing twice for each batch. This is not the most elegant thing to do for multiple-output streaming queries but is likely to be better than running two streaming queries processing the same data twice.

The proposal is to add a method `foreachBatch(f: Dataset[T] => Unit)` to Scala/Java/Python `DataStreamWriter`.

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

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

Closes #21571 from tdas/foreachBatch.
2018-06-19 13:56:51 -07:00
yucai 9dbe53eb6b [SPARK-24556][SQL] Always rewrite output partitioning in ReusedExchangeExec and InMemoryTableScanExec
## What changes were proposed in this pull request?

Currently, ReusedExchange and InMemoryTableScanExec only rewrite output partitioning if child's partitioning is HashPartitioning and do nothing for other partitioning, e.g., RangePartitioning. We should always rewrite it, otherwise, unnecessary shuffle could be introduced like https://issues.apache.org/jira/browse/SPARK-24556.

## How was this patch tested?

Add new tests.

Author: yucai <yyu1@ebay.com>

Closes #21564 from yucai/SPARK-24556.
2018-06-19 10:52:51 -07:00
Li Jin a78a904641 [SPARK-24521][SQL][TEST] Fix ineffective test in CachedTableSuite
## What changes were proposed in this pull request?

test("withColumn doesn't invalidate cached dataframe") in CachedTableSuite doesn't not work because:

The UDF is executed and test count incremented when "df.cache()" is called and the subsequent "df.collect()" has no effect on the test result.

This PR fixed this test and add another test for caching UDF.

## How was this patch tested?

Add new tests.

Author: Li Jin <ice.xelloss@gmail.com>

Closes #21531 from icexelloss/fix-cache-test.
2018-06-19 10:42:08 -07:00
Wenchen Fan 1737d45e08 [SPARK-24478][SQL][FOLLOWUP] Move projection and filter push down to physical conversion
## What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/21503, to completely move operator pushdown to the planner rule.

The code are mostly from https://github.com/apache/spark/pull/21319

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21574 from cloud-fan/followup.
2018-06-18 20:15:01 -07:00
Liang-Chi Hsieh 8f225e055c [SPARK-24548][SQL] Fix incorrect schema of Dataset with tuple encoders
## What changes were proposed in this pull request?

When creating tuple expression encoders, we should give the serializer expressions of tuple items correct names, so we can have correct output schema when we use such tuple encoders.

## How was this patch tested?

Added test.

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

Closes #21576 from viirya/SPARK-24548.
2018-06-18 11:01:17 -07:00
Takeshi Yamamuro e219e692ef [SPARK-23772][SQL] Provide an option to ignore column of all null values or empty array during JSON schema inference
## What changes were proposed in this pull request?
This pr added a new JSON option `dropFieldIfAllNull ` to ignore column of all null values or empty array/struct during JSON schema inference.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>
Author: Xiangrui Meng <meng@databricks.com>

Closes #20929 from maropu/SPARK-23772.
2018-06-19 00:24:54 +08:00
Mukul Murthy e4fee395ec [SPARK-24525][SS] Provide an option to limit number of rows in a MemorySink
## What changes were proposed in this pull request?

Provide an option to limit number of rows in a MemorySink. Currently, MemorySink and MemorySinkV2 have unbounded size, meaning that if they're used on big data, they can OOM the stream. This change adds a maxMemorySinkRows option to limit how many rows MemorySink and MemorySinkV2 can hold. By default, they are still unbounded.

## How was this patch tested?

Added new unit tests.

Author: Mukul Murthy <mukul.murthy@databricks.com>

Closes #21559 from mukulmurthy/SPARK-24525.
2018-06-15 13:56:48 -07:00
Kazuaki Ishizaki 90da7dc241 [SPARK-24452][SQL][CORE] Avoid possible overflow in int add or multiple
## What changes were proposed in this pull request?

This PR fixes possible overflow in int add or multiply. In particular, their overflows in multiply are detected by [Spotbugs](https://spotbugs.github.io/)

The following assignments may cause overflow in right hand side. As a result, the result may be negative.
```
long = int * int
long = int + int
```

To avoid this problem, this PR performs cast from int to long in right hand side.

## How was this patch tested?

Existing UTs.

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

Closes #21481 from kiszk/SPARK-24452.
2018-06-15 13:47:48 -07:00
Tathagata Das b5ccf0d395 [SPARK-24396][SS][PYSPARK] Add Structured Streaming ForeachWriter for python
## What changes were proposed in this pull request?

This PR adds `foreach` for streaming queries in Python. Users will be able to specify their processing logic in two different ways.
- As a function that takes a row as input.
- As an object that has methods `open`, `process`, and `close` methods.

See the python docs in this PR for more details.

## How was this patch tested?
Added java and python unit tests

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

Closes #21477 from tdas/SPARK-24396.
2018-06-15 12:56:39 -07:00
Ryan Blue 22daeba59b [SPARK-24478][SQL] Move projection and filter push down to physical conversion
## What changes were proposed in this pull request?

This removes the v2 optimizer rule for push-down and instead pushes filters and required columns when converting to a physical plan, as suggested by marmbrus. This makes the v2 relation cleaner because the output and filters do not change in the logical plan.

A side-effect of this change is that the stats from the logical (optimized) plan no longer reflect pushed filters and projection. This is a temporary state, until the planner gathers stats from the physical plan instead. An alternative to this approach is 9d3a11e68b.

The first commit was proposed in #21262. This PR replaces #21262.

## How was this patch tested?

Existing tests.

Author: Ryan Blue <blue@apache.org>

Closes #21503 from rdblue/SPARK-24478-move-push-down-to-physical-conversion.
2018-06-14 20:59:42 -07:00
Maxim Gekk b8f27ae3b3 [SPARK-24543][SQL] Support any type as DDL string for from_json's schema
## What changes were proposed in this pull request?

In the PR, I propose to support any DataType represented as DDL string for the from_json function. After the changes, it will be possible to specify `MapType` in SQL like:
```sql
select from_json('{"a":1, "b":2}', 'map<string, int>')
```
and in Scala (similar in other languages)
```scala
val in = Seq("""{"a": {"b": 1}}""").toDS()
val schema = "map<string, map<string, int>>"
val out = in.select(from_json($"value", schema, Map.empty[String, String]))
```

## How was this patch tested?

Added a couple sql tests and modified existing tests for Python and Scala. The former tests were modified because it is not imported for them in which format schema for `from_json` is provided.

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #21550 from MaxGekk/from_json-ddl-schema.
2018-06-14 13:27:27 -07:00
Marco Gaido fdadc4be08 [SPARK-24495][SQL] EnsureRequirement returns wrong plan when reordering equal keys
## What changes were proposed in this pull request?

`EnsureRequirement` in its `reorder` method currently assumes that the same key appears only once in the join condition. This of course might not be the case, and when it is not satisfied, it returns a wrong plan which produces a wrong result of the query.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21529 from mgaido91/SPARK-24495.
2018-06-14 09:20:41 -07:00
Jose Torres 1b46f41c55 [SPARK-24235][SS] Implement continuous shuffle writer for single reader partition.
## What changes were proposed in this pull request?

https://docs.google.com/document/d/1IL4kJoKrZWeyIhklKUJqsW-yEN7V7aL05MmM65AYOfE/edit

Implement continuous shuffle write RDD for a single reader partition. (I don't believe any implementation changes are actually required for multiple reader partitions, but this PR is already very large, so I want to exclude those for now to keep the size down.)

## How was this patch tested?

new unit tests

Author: Jose Torres <torres.joseph.f+github@gmail.com>

Closes #21428 from jose-torres/writerTask.
2018-06-13 13:13:01 -07:00
Herman van Hovell 299d297e25 [SPARK-24500][SQL] Make sure streams are materialized during Tree transforms.
## What changes were proposed in this pull request?
If you construct catalyst trees using `scala.collection.immutable.Stream` you can run into situations where valid transformations do not seem to have any effect. There are two causes for this behavior:
- `Stream` is evaluated lazily. Note that default implementation will generally only evaluate a function for the first element (this makes testing a bit tricky).
- `TreeNode` and `QueryPlan` use side effects to detect if a tree has changed. Mapping over a stream is lazy and does not need to trigger this side effect. If this happens the node will invalidly assume that it did not change and return itself instead if the newly created node (this is for GC reasons).

This PR fixes this issue by forcing materialization on streams in `TreeNode` and `QueryPlan`.

## How was this patch tested?
Unit tests were added to `TreeNodeSuite` and `LogicalPlanSuite`. An integration test was added to the `PlannerSuite`

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

Closes #21539 from hvanhovell/SPARK-24500.
2018-06-13 07:09:48 -07:00
Arun Mahadevan 7703b46d28 [SPARK-24479][SS] Added config for registering streamingQueryListeners
## What changes were proposed in this pull request?

Currently a "StreamingQueryListener" can only be registered programatically. We could have a new config "spark.sql.streamingQueryListeners" similar to  "spark.sql.queryExecutionListeners" and "spark.extraListeners" for users to register custom streaming listeners.

## How was this patch tested?

New unit test and running example programs.

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

Author: Arun Mahadevan <arunm@apache.org>

Closes #21504 from arunmahadevan/SPARK-24480.
2018-06-13 20:43:16 +08:00
Jungtaek Lim 4c388bccf1 [SPARK-24485][SS] Measure and log elapsed time for filesystem operations in HDFSBackedStateStoreProvider
## What changes were proposed in this pull request?

This patch measures and logs elapsed time for each operation which communicate with file system (mostly remote HDFS in production) in HDFSBackedStateStoreProvider to help investigating any latency issue.

## How was this patch tested?

Manually tested.

Author: Jungtaek Lim <kabhwan@gmail.com>

Closes #21506 from HeartSaVioR/SPARK-24485.
2018-06-13 12:36:20 +08:00
Jungtaek Lim 3352d6fe9a [SPARK-24466][SS] Fix TextSocketMicroBatchReader to be compatible with netcat again
## What changes were proposed in this pull request?

TextSocketMicroBatchReader was no longer be compatible with netcat due to launching temporary reader for reading schema, and closing reader, and re-opening reader. While reliable socket server should be able to handle this without any issue, nc command normally can't handle multiple connections and simply exits when closing temporary reader.

This patch fixes TextSocketMicroBatchReader to be compatible with netcat again, via deferring opening socket to the first call of planInputPartitions() instead of constructor.

## How was this patch tested?

Added unit test which fails on current and succeeds with the patch. And also manually tested.

Author: Jungtaek Lim <kabhwan@gmail.com>

Closes #21497 from HeartSaVioR/SPARK-24466.
2018-06-13 12:34:46 +08:00
Li Jin 9786ce66c5 [SPARK-22239][SQL][PYTHON] Enable grouped aggregate pandas UDFs as window functions with unbounded window frames
## What changes were proposed in this pull request?
This PR enables using a grouped aggregate pandas UDFs as window functions. The semantics is the same as using SQL aggregation function as window functions.

```
       >>> from pyspark.sql.functions import pandas_udf, PandasUDFType
       >>> from pyspark.sql import Window
       >>> df = spark.createDataFrame(
       ...     [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)],
       ...     ("id", "v"))
       >>> pandas_udf("double", PandasUDFType.GROUPED_AGG)
       ... def mean_udf(v):
       ...     return v.mean()
       >>> w = Window.partitionBy('id')
       >>> df.withColumn('mean_v', mean_udf(df['v']).over(w)).show()
       +---+----+------+
       | id|   v|mean_v|
       +---+----+------+
       |  1| 1.0|   1.5|
       |  1| 2.0|   1.5|
       |  2| 3.0|   6.0|
       |  2| 5.0|   6.0|
       |  2|10.0|   6.0|
       +---+----+------+
```

The scope of this PR is somewhat limited in terms of:
(1) Only supports unbounded window, which acts essentially as group by.
(2) Only supports aggregation functions, not "transform" like window functions (n -> n mapping)

Both of these are left as future work. Especially, (1) needs careful thinking w.r.t. how to pass rolling window data to python efficiently. (2) is a bit easier but does require more changes therefore I think it's better to leave it as a separate PR.

## How was this patch tested?

WindowPandasUDFTests

Author: Li Jin <ice.xelloss@gmail.com>

Closes #21082 from icexelloss/SPARK-22239-window-udf.
2018-06-13 09:10:52 +08:00
Kazuaki Ishizaki ada28f2595 [SPARK-23933][SQL] Add map_from_arrays function
## What changes were proposed in this pull request?

The PR adds the SQL function `map_from_arrays`. The behavior of the function is based on Presto's `map`. Since SparkSQL already had a `map` function, we prepared the different name for this behavior.

This function returns returns a map from a pair of arrays for keys and values.

## How was this patch tested?

Added UTs

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

Closes #21258 from kiszk/SPARK-23933.
2018-06-12 12:31:22 -07:00
Fangshi Li cc88d7fad1 [SPARK-24216][SQL] Spark TypedAggregateExpression uses getSimpleName that is not safe in scala
## What changes were proposed in this pull request?

When user create a aggregator object in scala and pass the aggregator to Spark Dataset's agg() method, Spark's will initialize TypedAggregateExpression with the nodeName field as aggregator.getClass.getSimpleName. However, getSimpleName is not safe in scala environment, depending on how user creates the aggregator object. For example, if the aggregator class full qualified name is "com.my.company.MyUtils$myAgg$2$", the getSimpleName will throw java.lang.InternalError "Malformed class name". This has been reported in scalatest https://github.com/scalatest/scalatest/pull/1044 and discussed in many scala upstream jiras such as SI-8110, SI-5425.

To fix this issue, we follow the solution in https://github.com/scalatest/scalatest/pull/1044 to add safer version of getSimpleName as a util method, and TypedAggregateExpression will invoke this util method rather than getClass.getSimpleName.

## How was this patch tested?
added unit test

Author: Fangshi Li <fli@linkedin.com>

Closes #21276 from fangshil/SPARK-24216.
2018-06-12 12:10:08 -07:00
DylanGuedes f0ef1b311d [SPARK-23931][SQL] Adds arrays_zip function to sparksql
Signed-off-by: DylanGuedes <djmgguedesgmail.com>

## What changes were proposed in this pull request?

Addition of arrays_zip function to spark sql functions.

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
Unit tests that checks if the results are correct.

Author: DylanGuedes <djmgguedes@gmail.com>

Closes #21045 from DylanGuedes/SPARK-23931.
2018-06-12 11:57:25 -07:00
Tom Saleeba 1d7db65e96 docs: fix typo
no => no[t]

## What changes were proposed in this pull request?

Fixing a typo.

## How was this patch tested?

Visual check of the docs.

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

Author: Tom Saleeba <tom.saleeba@gmail.com>

Closes #21496 from tomsaleeba/patch-1.
2018-06-12 09:22:52 -05:00
Wenchen Fan 01452ea9c7 [SPARK-24502][SQL] flaky test: UnsafeRowSerializerSuite
## What changes were proposed in this pull request?

`UnsafeRowSerializerSuite` calls `UnsafeProjection.create` which accesses `SQLConf.get`, while the current active SparkSession may already be stopped, and we may hit exception like this

```
sbt.ForkMain$ForkError: java.lang.IllegalStateException: LiveListenerBus is stopped.
	at org.apache.spark.scheduler.LiveListenerBus.addToQueue(LiveListenerBus.scala:97)
	at org.apache.spark.scheduler.LiveListenerBus.addToStatusQueue(LiveListenerBus.scala:80)
	at org.apache.spark.sql.internal.SharedState.<init>(SharedState.scala:93)
	at org.apache.spark.sql.SparkSession$$anonfun$sharedState$1.apply(SparkSession.scala:120)
	at org.apache.spark.sql.SparkSession$$anonfun$sharedState$1.apply(SparkSession.scala:120)
	at scala.Option.getOrElse(Option.scala:121)
	at org.apache.spark.sql.SparkSession.sharedState$lzycompute(SparkSession.scala:120)
	at org.apache.spark.sql.SparkSession.sharedState(SparkSession.scala:119)
	at org.apache.spark.sql.internal.BaseSessionStateBuilder.build(BaseSessionStateBuilder.scala:286)
	at org.apache.spark.sql.test.TestSparkSession.sessionState$lzycompute(TestSQLContext.scala:42)
	at org.apache.spark.sql.test.TestSparkSession.sessionState(TestSQLContext.scala:41)
	at org.apache.spark.sql.SparkSession$$anonfun$1$$anonfun$apply$1.apply(SparkSession.scala:95)
	at org.apache.spark.sql.SparkSession$$anonfun$1$$anonfun$apply$1.apply(SparkSession.scala:95)
	at scala.Option.map(Option.scala:146)
	at org.apache.spark.sql.SparkSession$$anonfun$1.apply(SparkSession.scala:95)
	at org.apache.spark.sql.SparkSession$$anonfun$1.apply(SparkSession.scala:94)
	at org.apache.spark.sql.internal.SQLConf$.get(SQLConf.scala:126)
	at org.apache.spark.sql.catalyst.expressions.CodeGeneratorWithInterpretedFallback.createObject(CodeGeneratorWithInterpretedFallback.scala:54)
	at org.apache.spark.sql.catalyst.expressions.UnsafeProjection$.create(Projection.scala:157)
	at org.apache.spark.sql.catalyst.expressions.UnsafeProjection$.create(Projection.scala:150)
	at org.apache.spark.sql.execution.UnsafeRowSerializerSuite.org$apache$spark$sql$execution$UnsafeRowSerializerSuite$$unsafeRowConverter(UnsafeRowSerializerSuite.scala:54)
	at org.apache.spark.sql.execution.UnsafeRowSerializerSuite.org$apache$spark$sql$execution$UnsafeRowSerializerSuite$$toUnsafeRow(UnsafeRowSerializerSuite.scala:49)
	at org.apache.spark.sql.execution.UnsafeRowSerializerSuite$$anonfun$2.apply(UnsafeRowSerializerSuite.scala:63)
	at org.apache.spark.sql.execution.UnsafeRowSerializerSuite$$anonfun$2.apply(UnsafeRowSerializerSuite.scala:60)
...
```

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21518 from cloud-fan/test.
2018-06-11 22:08:44 -07:00
liutang123 048197749e [SPARK-22144][SQL] ExchangeCoordinator combine the partitions of an 0 sized pre-shuffle to 0
## What changes were proposed in this pull request?
when the length of pre-shuffle's partitions is 0, the length of post-shuffle's partitions should be 0 instead of spark.sql.shuffle.partitions.

## How was this patch tested?
ExchangeCoordinator converted a  pre-shuffle that partitions is 0 to a post-shuffle that partitions is 0 instead of one that partitions is spark.sql.shuffle.partitions.

Author: liutang123 <liutang123@yeah.net>

Closes #19364 from liutang123/SPARK-22144.
2018-06-11 17:48:07 -07:00
Marco Gaido f07c5064a3 [SPARK-24468][SQL] Handle negative scale when adjusting precision for decimal operations
## What changes were proposed in this pull request?

In SPARK-22036 we introduced the possibility to allow precision loss in arithmetic operations (according to the SQL standard). The implementation was drawn from Hive's one, where Decimals with a negative scale are not allowed in the operations.

The PR handles the case when the scale is negative, removing the assertion that it is not.

## How was this patch tested?

added UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21499 from mgaido91/SPARK-24468.
2018-06-08 18:51:56 -07:00
Thiruvasakan Paramasivan 36a3409134
[SPARK-24412][SQL] Adding docs about automagical type casting in isin and isInCollection APIs
## What changes were proposed in this pull request?
Update documentation for `isInCollection` API to clealy explain the "auto-casting" of elements if their types are different.

## How was this patch tested?
No-Op

Author: Thiruvasakan Paramasivan <thiru@apple.com>

Closes #21519 from trvskn/sql-doc-update.
2018-06-08 17:17:43 -07:00
Asher Saban e76b0124fb [SPARK-23803][SQL] Support bucket pruning
## What changes were proposed in this pull request?
support bucket pruning when filtering on a single bucketed column on the following predicates -
EqualTo, EqualNullSafe, In, And/Or predicates

## How was this patch tested?
refactored unit tests to test the above.

based on gatorsmile work in e3c75c6398

Author: Asher Saban <asaban@palantir.com>
Author: asaban <asaban@palantir.com>

Closes #20915 from sabanas/filter-prune-buckets.
2018-06-06 07:14:08 -07:00
Tathagata Das 2c2a86b5d5 [SPARK-24453][SS] Fix error recovering from the failure in a no-data batch
## What changes were proposed in this pull request?

The error occurs when we are recovering from a failure in a no-data batch (say X) that has been planned (i.e. written to offset log) but not executed (i.e. not written to commit log). Upon recovery the following sequence of events happen.

1. `MicroBatchExecution.populateStartOffsets` sets `currentBatchId` to X. Since there was no data in the batch, the `availableOffsets` is same as `committedOffsets`, so `isNewDataAvailable` is `false`.
2. When `MicroBatchExecution.constructNextBatch` is called, ideally it should immediately return true because the next batch has already been constructed. However, the check of whether the batch has been constructed was `if (isNewDataAvailable) return true`. Since the planned batch is a no-data batch, it escaped this check and proceeded to plan the same batch X *once again*.

The solution is to have an explicit flag that signifies whether a batch has already been constructed or not. `populateStartOffsets` is going to set the flag appropriately.

## How was this patch tested?

new unit test

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

Closes #21491 from tdas/SPARK-24453.
2018-06-05 01:08:55 -07:00
Yuanjian Li dbb4d83829 [SPARK-24215][PYSPARK] Implement _repr_html_ for dataframes in PySpark
## What changes were proposed in this pull request?

Implement `_repr_html_` for PySpark while in notebook and add config named "spark.sql.repl.eagerEval.enabled" to control this.

The dev list thread for context: http://apache-spark-developers-list.1001551.n3.nabble.com/eager-execution-and-debuggability-td23928.html

## How was this patch tested?

New ut in DataFrameSuite and manual test in jupyter. Some screenshot below.

**After:**
![image](https://user-images.githubusercontent.com/4833765/40268422-8db5bef0-5b9f-11e8-80f1-04bc654a4f2c.png)

**Before:**
![image](https://user-images.githubusercontent.com/4833765/40268431-9f92c1b8-5b9f-11e8-9db9-0611f0940b26.png)

Author: Yuanjian Li <xyliyuanjian@gmail.com>

Closes #21370 from xuanyuanking/SPARK-24215.
2018-06-05 08:23:08 +07:00
aokolnychyi 7297ae04d8 [SPARK-21896][SQL] Fix StackOverflow caused by window functions inside aggregate functions
## What changes were proposed in this pull request?

This PR explicitly prohibits window functions inside aggregates. Currently, this will cause StackOverflow during analysis. See PR #19193 for previous discussion.

## How was this patch tested?

This PR comes with a dedicated unit test.

Author: aokolnychyi <anton.okolnychyi@sap.com>

Closes #21473 from aokolnychyi/fix-stackoverflow-window-funcs.
2018-06-04 13:28:16 -07:00
Yuming Wang 0be5aa2746 [SPARK-23903][SQL] Add support for date extract
## What changes were proposed in this pull request?

Add support for date `extract` function:
```sql
spark-sql> SELECT EXTRACT(YEAR FROM TIMESTAMP '2000-12-16 12:21:13');
2000
```
Supported field same as [Hive](https://github.com/apache/hive/blob/rel/release-2.3.3/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g#L308-L316): `YEAR`, `QUARTER`, `MONTH`, `WEEK`, `DAY`, `DAYOFWEEK`, `HOUR`, `MINUTE`, `SECOND`.

## How was this patch tested?

unit tests

Author: Yuming Wang <yumwang@ebay.com>

Closes #21479 from wangyum/SPARK-23903.
2018-06-04 10:16:13 -07:00
Maxim Gekk 1d9338bb10 [SPARK-23786][SQL] Checking column names of csv headers
## What changes were proposed in this pull request?

Currently column names of headers in CSV files are not checked against provided schema of CSV data. It could cause errors like showed in the [SPARK-23786](https://issues.apache.org/jira/browse/SPARK-23786) and https://github.com/apache/spark/pull/20894#issuecomment-375957777. I introduced new CSV option - `enforceSchema`. If it is enabled (by default `true`), Spark forcibly applies provided or inferred schema to CSV files. In that case, CSV headers are ignored and not checked against the schema. If `enforceSchema` is set to `false`, additional checks can be performed. For example, if column in CSV header and in the schema have different ordering, the following exception is thrown:

```
java.lang.IllegalArgumentException: CSV file header does not contain the expected fields
 Header: depth, temperature
 Schema: temperature, depth
CSV file: marina.csv
```

## How was this patch tested?

The changes were tested by existing tests of CSVSuite and by 2 new tests.

Author: Maxim Gekk <maxim.gekk@databricks.com>
Author: Maxim Gekk <max.gekk@gmail.com>

Closes #20894 from MaxGekk/check-column-names.
2018-06-03 22:02:21 -07:00
Wenchen Fan 416cd1fd96 [SPARK-24369][SQL] Correct handling for multiple distinct aggregations having the same argument set
## What changes were proposed in this pull request?

bring back https://github.com/apache/spark/pull/21443

This is a different approach: just change the check to count distinct columns with `toSet`

## How was this patch tested?

a new test to verify the planner behavior.

Author: Wenchen Fan <wenchen@databricks.com>
Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21487 from cloud-fan/back.
2018-06-03 21:57:42 -07:00
Xiao Li d2c3de7efc Revert "[SPARK-24369][SQL] Correct handling for multiple distinct aggregations having the same argument set"
This reverts commit 1e46f92f95.
2018-06-01 11:51:10 -07:00
Huang Tengfei 6039b13230 [SPARK-24351][SS] offsetLog/commitLog purge thresholdBatchId should be computed with current committed epoch but not currentBatchId in CP mode
## What changes were proposed in this pull request?
Compute the thresholdBatchId to purge metadata based on current committed epoch instead of currentBatchId in CP mode to avoid cleaning all the committed metadata in some case as described in the jira [SPARK-24351](https://issues.apache.org/jira/browse/SPARK-24351).

## How was this patch tested?
Add new unit test.

Author: Huang Tengfei <tengfei.h@gmail.com>

Closes #21400 from ivoson/branch-cp-meta.
2018-06-01 10:47:53 -07:00
Huaxin Gao 98909c398d [SPARK-23920][SQL] add array_remove to remove all elements that equal element from array
## What changes were proposed in this pull request?

add array_remove to remove all elements that equal element from array

## How was this patch tested?

add unit tests

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

Closes #21069 from huaxingao/spark-23920.
2018-05-31 22:04:26 -07:00
Gengliang Wang cbaa729132 [SPARK-24330][SQL] Refactor ExecuteWriteTask and Use while in writing files
## What changes were proposed in this pull request?
1. Refactor ExecuteWriteTask in FileFormatWriter to reduce common logic and improve readability.
After the change, callers only need to call `commit()` or `abort` at the end of task.
Also there is less code in `SingleDirectoryWriteTask` and `DynamicPartitionWriteTask`.
Definitions of related classes are moved to a new file, and `ExecuteWriteTask` is renamed to `FileFormatDataWriter`.

2. As per code style guide: https://github.com/databricks/scala-style-guide#traversal-and-zipwithindex , we avoid using `for` for looping in [FileFormatWriter](https://github.com/apache/spark/pull/21381/files#diff-3b69eb0963b68c65cfe8075f8a42e850L536) , or `foreach` in [WriteToDataSourceV2Exec](https://github.com/apache/spark/pull/21381/files#diff-6fbe10db766049a395bae2e785e9d56eL119).
In such critical code path, using `while` is good for performance.

## How was this patch tested?
Existing unit test.
I tried the microbenchmark in https://github.com/apache/spark/pull/21409

| Workload | Before changes(Best/Avg Time(ms)) | After changes(Best/Avg Time(ms)) |
| --- | --- | -- |
|Output Single Int Column|    2018 / 2043   |    2096 / 2236 |
|Output Single Double Column| 1978 / 2043 | 2013 / 2018 |
|Output Int and String Column| 6332 / 6706 | 6162 / 6298 |
|Output Partitions| 4458 / 5094 |  3792 / 4008  |
|Output Buckets|           5695 / 6102 |    5120 / 5154 |

Also a microbenchmark on my laptop for general comparison among while/foreach/for :
```
class Writer {
  var sum = 0L
  def write(l: Long): Unit = sum += l
}

def testWhile(iterator: Iterator[Long]): Long = {
  val w = new Writer
  while (iterator.hasNext) {
    w.write(iterator.next())
  }
  w.sum
}

def testForeach(iterator: Iterator[Long]): Long = {
  val w = new Writer
  iterator.foreach(w.write)
  w.sum
}

def testFor(iterator: Iterator[Long]): Long = {
  val w = new Writer
  for (x <- iterator) {
    w.write(x)
  }
  w.sum
}

val data = 0L to 100000000L
val start = System.nanoTime
(0 to 10).foreach(_ => testWhile(data.iterator))
println("benchmark while: " + (System.nanoTime - start)/1000000)

val start2 = System.nanoTime
(0 to 10).foreach(_ => testForeach(data.iterator))
println("benchmark foreach: " + (System.nanoTime - start2)/1000000)

val start3 = System.nanoTime
(0 to 10).foreach(_ => testForeach(data.iterator))
println("benchmark for: " + (System.nanoTime - start3)/1000000)
```
Benchmark result:
`while`: 15401 ms
`foreach`: 43034 ms
`for`: 41279 ms

Author: Gengliang Wang <gengliang.wang@databricks.com>

Closes #21381 from gengliangwang/refactorExecuteWriteTask.
2018-06-01 10:01:15 +08:00
Marco Gaido 1b36f14889 [SPARK-23901][SQL] Add masking functions
## What changes were proposed in this pull request?

The PR adds the masking function as they are described in Hive's documentation: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF#LanguageManualUDF-DataMaskingFunctions.
This means that only `string`s are accepted as parameter for the masking functions.

## How was this patch tested?

added UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21246 from mgaido91/SPARK-23901.
2018-05-30 11:18:04 -07:00
Takeshi Yamamuro 1e46f92f95 [SPARK-24369][SQL] Correct handling for multiple distinct aggregations having the same argument set
## What changes were proposed in this pull request?
This pr fixed an issue when having multiple distinct aggregations having the same argument set, e.g.,
```
scala>: paste
val df = sql(
  s"""SELECT corr(DISTINCT x, y), corr(DISTINCT y, x), count(*)
     | FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y)
   """.stripMargin)

java.lang.RuntimeException
You hit a query analyzer bug. Please report your query to Spark user mailing list.
```
The root cause is that `RewriteDistinctAggregates` can't detect multiple distinct aggregations if they have the same argument set. This pr modified code so that `RewriteDistinctAggregates` could count the number of aggregate expressions with `isDistinct=true`.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21443 from maropu/SPARK-24369.
2018-05-31 00:23:25 +08:00
Gengliang Wang f48938800e [SPARK-24365][SQL] Add Data Source write benchmark
## What changes were proposed in this pull request?

Add Data Source write benchmark. So that it would be easier to measure the writer performance.

Author: Gengliang Wang <gengliang.wang@databricks.com>

Closes #21409 from gengliangwang/parquetWriteBenchmark.
2018-05-30 09:32:33 +08:00
DB Tsai 900bc1f7dc
[SPARK-24371][SQL] Added isInCollection in DataFrame API for Scala and Java.
## What changes were proposed in this pull request?

Implemented **`isInCollection `** in DataFrame API for both Scala and Java, so users can do

```scala
val profileDF = Seq(
  Some(1), Some(2), Some(3), Some(4),
  Some(5), Some(6), Some(7), None
).toDF("profileID")

val validUsers: Seq[Any] = Seq(6, 7.toShort, 8L, "3")

val result = profileDF.withColumn("isValid", $"profileID". isInCollection(validUsers))

result.show(10)
"""
+---------+-------+
|profileID|isValid|
+---------+-------+
|        1|  false|
|        2|  false|
|        3|   true|
|        4|  false|
|        5|  false|
|        6|   true|
|        7|   true|
|     null|   null|
+---------+-------+
 """.stripMargin
```
## How was this patch tested?

Several unit tests are added.

Author: DB Tsai <d_tsai@apple.com>

Closes #21416 from dbtsai/optimize-set.
2018-05-29 10:22:18 -07:00
Xiao Li 23db600c95 [SPARK-24250][SQL][FOLLOW-UP] support accessing SQLConf inside tasks
## What changes were proposed in this pull request?
We should not stop users from calling `getActiveSession` and `getDefaultSession` in executors. To not break the existing behaviors, we should simply return None.

## How was this patch tested?
N/A

Author: Xiao Li <gatorsmile@gmail.com>

Closes #21436 from gatorsmile/followUpSPARK-24250.
2018-05-28 23:23:22 -07:00
Dongjoon Hyun b31b587cd0 [SPARK-19613][SS][TEST] Random.nextString is not safe for directory namePrefix
## What changes were proposed in this pull request?

`Random.nextString` is good for generating random string data, but it's not proper for directory name prefix in `Utils.createDirectory(tempDir, Random.nextString(10))`. This PR uses more safe directory namePrefix.

```scala
scala> scala.util.Random.nextString(10)
res0: String = 馨쭔ᎰႻ穚䃈兩㻞藑並
```

```scala
StateStoreRDDSuite:
- versioning and immutability
- recovering from files
- usage with iterators - only gets and only puts
- preferred locations using StateStoreCoordinator *** FAILED ***
  java.io.IOException: Failed to create a temp directory (under /.../spark/sql/core/target/tmp/StateStoreRDDSuite8712796397908632676) after 10 attempts!
  at org.apache.spark.util.Utils$.createDirectory(Utils.scala:295)
  at org.apache.spark.sql.execution.streaming.state.StateStoreRDDSuite$$anonfun$13$$anonfun$apply$6.apply(StateStoreRDDSuite.scala:152)
  at org.apache.spark.sql.execution.streaming.state.StateStoreRDDSuite$$anonfun$13$$anonfun$apply$6.apply(StateStoreRDDSuite.scala:149)
  at org.apache.spark.sql.catalyst.util.package$.quietly(package.scala:42)
  at org.apache.spark.sql.execution.streaming.state.StateStoreRDDSuite$$anonfun$13.apply(StateStoreRDDSuite.scala:149)
  at org.apache.spark.sql.execution.streaming.state.StateStoreRDDSuite$$anonfun$13.apply(StateStoreRDDSuite.scala:149)
...
- distributed test *** FAILED ***
  java.io.IOException: Failed to create a temp directory (under /.../spark/sql/core/target/tmp/StateStoreRDDSuite8712796397908632676) after 10 attempts!
  at org.apache.spark.util.Utils$.createDirectory(Utils.scala:295)
```

## How was this patch tested?

Pass the existing tests.StateStoreRDDSuite:

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #21446 from dongjoon-hyun/SPARK-19613.
2018-05-29 10:35:30 +08:00
Marco Gaido de01a8d50c [SPARK-24373][SQL] Add AnalysisBarrier to RelationalGroupedDataset's and KeyValueGroupedDataset's child
## What changes were proposed in this pull request?

When we create a `RelationalGroupedDataset` or a `KeyValueGroupedDataset` we set its child to the `logicalPlan` of the `DataFrame` we need to aggregate. Since the `logicalPlan` is already analyzed, we should not analyze it again. But this happens when the new plan of the aggregate is analyzed.

The current behavior in most of the cases is likely to produce no harm, but in other cases re-analyzing an analyzed plan can change it, since the analysis is not idempotent. This can cause issues like the one described in the JIRA (missing to find a cached plan).

The PR adds an `AnalysisBarrier` to the `logicalPlan` which is used as child of `RelationalGroupedDataset` or a `KeyValueGroupedDataset`.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21432 from mgaido91/SPARK-24373.
2018-05-28 12:09:44 +08:00
Li Jin 672209f290 [SPARK-24334] Fix race condition in ArrowPythonRunner causes unclean shutdown of Arrow memory allocator
## What changes were proposed in this pull request?

There is a race condition of closing Arrow VectorSchemaRoot and Allocator in the writer thread of ArrowPythonRunner.

The race results in memory leak exception when closing the allocator. This patch removes the closing routine from the TaskCompletionListener and make the writer thread responsible for cleaning up the Arrow memory.

This issue be reproduced by this test:

```
def test_memory_leak(self):
    from pyspark.sql.functions import pandas_udf, col, PandasUDFType, array, lit, explode

   # Have all data in a single executor thread so it can trigger the race condition easier
    with self.sql_conf({'spark.sql.shuffle.partitions': 1}):
        df = self.spark.range(0, 1000)
        df = df.withColumn('id', array([lit(i) for i in range(0, 300)])) \
                   .withColumn('id', explode(col('id'))) \
                   .withColumn('v',  array([lit(i) for i in range(0, 1000)]))

       pandas_udf(df.schema, PandasUDFType.GROUPED_MAP)
       def foo(pdf):
           xxx
           return pdf

       result = df.groupby('id').apply(foo)

       with QuietTest(self.sc):
           with self.assertRaises(py4j.protocol.Py4JJavaError) as context:
               result.count()
           self.assertTrue('Memory leaked' not in str(context.exception))
```

Note: Because of the race condition, the test case cannot reproduce the issue reliably so it's not added to test cases.

## How was this patch tested?

Because of the race condition, the bug cannot be unit test easily. So far it has only happens on large amount of data. This is currently tested manually.

Author: Li Jin <ice.xelloss@gmail.com>

Closes #21397 from icexelloss/SPARK-24334-arrow-memory-leak.
2018-05-28 10:50:17 +08:00
Miles Yucht d440699192 [SPARK-24381][TESTING] Add unit tests for NOT IN subquery around null values
## What changes were proposed in this pull request?
This PR adds several unit tests along the `cols NOT IN (subquery)` pathway. There are a scattering of tests here and there which cover this codepath, but there doesn't seem to be a unified unit test of the correctness of null-aware anti joins anywhere. I have also added a brief explanation of how this expression behaves in SubquerySuite. Lastly, I made some clarifying changes in the NOT IN pathway in RewritePredicateSubquery.

## How was this patch tested?
Added unit tests! There should be no behavioral change in this PR.

Author: Miles Yucht <miles@databricks.com>

Closes #21425 from mgyucht/spark-24381.
2018-05-26 20:42:23 -07:00
Maxim Gekk 64fad0b519 [SPARK-24244][SPARK-24368][SQL] Passing only required columns to the CSV parser
## What changes were proposed in this pull request?

uniVocity parser allows to specify only required column names or indexes for [parsing](https://www.univocity.com/pages/parsers-tutorial) like:

```
// Here we select only the columns by their indexes.
// The parser just skips the values in other columns
parserSettings.selectIndexes(4, 0, 1);
CsvParser parser = new CsvParser(parserSettings);
```
In this PR, I propose to extract indexes from required schema and pass them into the CSV parser. Benchmarks show the following improvements in parsing of 1000 columns:

```
Select 100 columns out of 1000: x1.76
Select 1 column out of 1000: x2
```

**Note**: Comparing to current implementation, the changes can return different result for malformed rows in the `DROPMALFORMED` and `FAILFAST` modes if only subset of all columns is requested. To have previous behavior, set `spark.sql.csv.parser.columnPruning.enabled` to `false`.

## How was this patch tested?

It was tested by new test which selects 3 columns out of 15, by existing tests and by new benchmarks.

Author: Maxim Gekk <maxim.gekk@databricks.com>
Author: Maxim Gekk <max.gekk@gmail.com>

Closes #21415 from MaxGekk/csv-column-pruning2.
2018-05-24 21:38:04 -07:00
Gengliang Wang 3b20b34ab7 [SPARK-24367][SQL] Parquet: use JOB_SUMMARY_LEVEL instead of deprecated flag ENABLE_JOB_SUMMARY
## What changes were proposed in this pull request?

In current parquet version,the conf ENABLE_JOB_SUMMARY is deprecated.

When writing to Parquet files, the warning message
```WARN org.apache.parquet.hadoop.ParquetOutputFormat: Setting parquet.enable.summary-metadata is deprecated, please use parquet.summary.metadata.level```
keeps showing up.

From https://github.com/apache/parquet-mr/blame/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java#L164 we can see that we should use JOB_SUMMARY_LEVEL.

## How was this patch tested?

Unit test

Author: Gengliang Wang <gengliang.wang@databricks.com>

Closes #21411 from gengliangwang/summaryLevel.
2018-05-25 11:16:35 +08:00
Jose Torres 0fd68cb727 [SPARK-24234][SS] Support multiple row writers in continuous processing shuffle reader.
## What changes were proposed in this pull request?

https://docs.google.com/document/d/1IL4kJoKrZWeyIhklKUJqsW-yEN7V7aL05MmM65AYOfE/edit#heading=h.8t3ci57f7uii

Support multiple different row writers in continuous processing shuffle reader.

Note that having multiple read-side buffers ended up being the natural way to do this. Otherwise it's hard to express the constraint of sending an epoch marker only when all writers have sent one.

## How was this patch tested?

new unit tests

Author: Jose Torres <torres.joseph.f+github@gmail.com>

Closes #21385 from jose-torres/multipleWrite.
2018-05-24 17:08:52 -07:00
Maxim Gekk 13bedc05c2 [SPARK-24329][SQL] Test for skipping multi-space lines
## What changes were proposed in this pull request?

The PR is a continue of https://github.com/apache/spark/pull/21380 . It checks cases that are handled by the code:
e3de6ab30d/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala (L303-L304)

Basically the code skips lines with one or many whitespaces, and lines with comments (see [filterCommentAndEmpty](e3de6ab30d/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtils.scala (L47)))

```scala
   iter.filter { line =>
      line.trim.nonEmpty && !line.startsWith(options.comment.toString)
    }
```

Closes #21380

## How was this patch tested?

Added a test for the case described above.

Author: Maxim Gekk <maxim.gekk@databricks.com>
Author: Maxim Gekk <max.gekk@gmail.com>

Closes #21394 from MaxGekk/test-for-multi-space-lines.
2018-05-24 22:18:58 +08:00
Ryan Blue 3469f5c989 [SPARK-24230][SQL] Fix SpecificParquetRecordReaderBase with dictionary filters.
## What changes were proposed in this pull request?

I missed this commit when preparing #21070.

When Parquet is able to filter blocks with dictionary filtering, the expected total value count to be too high in Spark, leading to an error when there were fewer than expected row groups to process. Spark should get the row groups from Parquet to pick up new filter schemes in Parquet like dictionary filtering.

## How was this patch tested?

Using in production at Netflix. Added test case for dictionary-filtered blocks.

Author: Ryan Blue <blue@apache.org>

Closes #21295 from rdblue/SPARK-24230-fix-parquet-block-tracking.
2018-05-24 20:55:26 +08:00
hyukjinkwon 8a545822d0 [SPARK-24364][SS] Prevent InMemoryFileIndex from failing if file path doesn't exist
## What changes were proposed in this pull request?

This PR proposes to follow up https://github.com/apache/spark/pull/15153 and complete SPARK-17599.

`FileSystem` operation (`fs.getFileBlockLocations`) can still fail if the file path does not exist. For example see the exception message below:

```
Error occurred while processing: File does not exist: /rel/00171151/input/PJ/part-00136-b6403bac-a240-44f8-a792-fc2e174682b7-c000.csv
...
java.io.FileNotFoundException: File does not exist: /rel/00171151/input/PJ/part-00136-b6403bac-a240-44f8-a792-fc2e174682b7-c000.csv
...
org.apache.hadoop.hdfs.DistributedFileSystem.getFileBlockLocations(DistributedFileSystem.java:249)
	at org.apache.hadoop.hdfs.DistributedFileSystem.getFileBlockLocations(DistributedFileSystem.java:229)
	at org.apache.spark.sql.execution.datasources.InMemoryFileIndex$$anonfun$org$apache$spark$sql$execution$datasources$InMemoryFileIndex$$listLeafFiles$3.apply(InMemoryFileIndex.scala:314)
	at org.apache.spark.sql.execution.datasources.InMemoryFileIndex$$anonfun$org$apache$spark$sql$execution$datasources$InMemoryFileIndex$$listLeafFiles$3.apply(InMemoryFileIndex.scala:297)
	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
	at scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
	at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186)
	at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
	at scala.collection.mutable.ArrayOps$ofRef.map(ArrayOps.scala:186)
	at org.apache.spark.sql.execution.datasources.InMemoryFileIndex$.org$apache$spark$sql$execution$datasources$InMemoryFileIndex$$listLeafFiles(InMemoryFileIndex.scala:297)
	at org.apache.spark.sql.execution.datasources.InMemoryFileIndex$$anonfun$org$apache$spark$sql$execution$datasources$InMemoryFileIndex$$bulkListLeafFiles$1.apply(InMemoryFileIndex.scala:174)
	at org.apache.spark.sql.execution.datasources.InMemoryFileIndex$$anonfun$org$apache$spark$sql$execution$datasources$InMemoryFileIndex$$bulkListLeafFiles$1.apply(InMemoryFileIndex.scala:173)
	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
	at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
	at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
	at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
	at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
	at scala.collection.AbstractTraversable.map(Traversable.scala:104)
	at org.apache.spark.sql.execution.datasources.InMemoryFileIndex$.org$apache$spark$sql$execution$datasources$InMemoryFileIndex$$bulkListLeafFiles(InMemoryFileIndex.scala:173)
	at org.apache.spark.sql.execution.datasources.InMemoryFileIndex.listLeafFiles(InMemoryFileIndex.scala:126)
	at org.apache.spark.sql.execution.datasources.InMemoryFileIndex.refresh0(InMemoryFileIndex.scala:91)
	at org.apache.spark.sql.execution.datasources.InMemoryFileIndex.<init>(InMemoryFileIndex.scala:67)
	at org.apache.spark.sql.execution.datasources.DataSource.tempFileIndex$lzycompute$1(DataSource.scala:161)
	at org.apache.spark.sql.execution.datasources.DataSource.org$apache$spark$sql$execution$datasources$DataSource$$tempFileIndex$1(DataSource.scala:152)
	at org.apache.spark.sql.execution.datasources.DataSource.getOrInferFileFormatSchema(DataSource.scala:166)
	at org.apache.spark.sql.execution.datasources.DataSource.sourceSchema(DataSource.scala:261)
	at org.apache.spark.sql.execution.datasources.DataSource.sourceInfo$lzycompute(DataSource.scala:94)
	at org.apache.spark.sql.execution.datasources.DataSource.sourceInfo(DataSource.scala:94)
	at org.apache.spark.sql.execution.streaming.StreamingRelation$.apply(StreamingRelation.scala:33)
	at org.apache.spark.sql.streaming.DataStreamReader.load(DataStreamReader.scala:196)
	at org.apache.spark.sql.streaming.DataStreamReader.load(DataStreamReader.scala:206)
	at com.hwx.StreamTest$.main(StreamTest.scala:97)
	at com.hwx.StreamTest.main(StreamTest.scala)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
	at org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:906)
	at org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:197)
	at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:227)
	at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:136)
	at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
Caused by: org.apache.hadoop.ipc.RemoteException(java.io.FileNotFoundException): File does not exist: /rel/00171151/input/PJ/part-00136-b6403bac-a240-44f8-a792-fc2e174682b7-c000.csv
...
```

So, it fixes it to make a warning instead.

## How was this patch tested?

It's hard to write a test. Manually tested multiple times.

Author: hyukjinkwon <gurwls223@apache.org>

Closes #21408 from HyukjinKwon/missing-files.
2018-05-24 13:21:02 +08:00
Dongjoon Hyun 486ecc680e [SPARK-24322][BUILD] Upgrade Apache ORC to 1.4.4
## What changes were proposed in this pull request?

ORC 1.4.4 includes [nine fixes](https://issues.apache.org/jira/issues/?filter=12342568&jql=project%20%3D%20ORC%20AND%20resolution%20%3D%20Fixed%20AND%20fixVersion%20%3D%201.4.4). One of the issues is about `Timestamp` bug (ORC-306) which occurs when `native` ORC vectorized reader reads ORC column vector's sub-vector `times` and `nanos`. ORC-306 fixes this according to the [original definition](https://github.com/apache/hive/blob/master/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/TimestampColumnVector.java#L45-L46) and this PR includes the updated interpretation on ORC column vectors. Note that `hive` ORC reader and ORC MR reader is not affected.

```scala
scala> spark.version
res0: String = 2.3.0
scala> spark.sql("set spark.sql.orc.impl=native")
scala> Seq(java.sql.Timestamp.valueOf("1900-05-05 12:34:56.000789")).toDF().write.orc("/tmp/orc")
scala> spark.read.orc("/tmp/orc").show(false)
+--------------------------+
|value                     |
+--------------------------+
|1900-05-05 12:34:55.000789|
+--------------------------+
```

This PR aims to update Apache Spark to use it.

**FULL LIST**

ID | TITLE
-- | --
ORC-281 | Fix compiler warnings from clang 5.0
ORC-301 | `extractFileTail` should open a file in `try` statement
ORC-304 | Fix TestRecordReaderImpl to not fail with new storage-api
ORC-306 | Fix incorrect workaround for bug in java.sql.Timestamp
ORC-324 | Add support for ARM and PPC arch
ORC-330 | Remove unnecessary Hive artifacts from root pom
ORC-332 | Add syntax version to orc_proto.proto
ORC-336 | Remove avro and parquet dependency management entries
ORC-360 | Implement error checking on subtype fields in Java

## How was this patch tested?

Pass the Jenkins.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #21372 from dongjoon-hyun/SPARK_ORC144.
2018-05-24 11:34:13 +08:00
sychen 888340151f [SPARK-24257][SQL] LongToUnsafeRowMap calculate the new size may be wrong
LongToUnsafeRowMap has a mistake when growing its page array: it blindly grows to `oldSize * 2`, while the new record may be larger than `oldSize * 2`. Then we may have a malformed UnsafeRow when querying this map, whose actual data is smaller than its declared size, and the data is corrupted.

Author: sychen <sychen@ctrip.com>

Closes #21311 from cxzl25/fix_LongToUnsafeRowMap_page_size.
2018-05-24 11:18:07 +08:00
Vayda, Oleksandr: IT (PRG) 230f144197 [SPARK-24350][SQL] Fixes ClassCastException in the "array_position" function
## What changes were proposed in this pull request?

### Fixes `ClassCastException` in the `array_position` function - [SPARK-24350](https://issues.apache.org/jira/browse/SPARK-24350)
When calling `array_position` function with a wrong type of the 1st argument an `AnalysisException` should be thrown instead of `ClassCastException`

Example:

```sql
select array_position('foo', 'bar')
```

```
java.lang.ClassCastException: org.apache.spark.sql.types.StringType$ cannot be cast to org.apache.spark.sql.types.ArrayType
	at org.apache.spark.sql.catalyst.expressions.ArrayPosition.inputTypes(collectionOperations.scala:1398)
	at org.apache.spark.sql.catalyst.expressions.ExpectsInputTypes$class.checkInputDataTypes(ExpectsInputTypes.scala:44)
	at org.apache.spark.sql.catalyst.expressions.ArrayPosition.checkInputDataTypes(collectionOperations.scala:1401)
	at org.apache.spark.sql.catalyst.expressions.Expression.resolved$lzycompute(Expression.scala:168)
	at org.apache.spark.sql.catalyst.expressions.Expression.resolved(Expression.scala:168)
	at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAliases$$anonfun$org$apache$spark$sql$catalyst$analysis$Analyzer$ResolveAliases$$assignAliases$1$$anonfun$apply$3.applyOrElse(Analyzer.scala:256)
	at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAliases$$anonfun$org$apache$spark$sql$catalyst$analysis$Analyzer$ResolveAliases$$assignAliases$1$$anonfun$apply$3.applyOrElse(Analyzer.scala:252)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:289)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:289)
	at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70)
	at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:288)
```

## How was this patch tested?

unit test

Author: Vayda, Oleksandr: IT (PRG) <Oleksandr.Vayda@barclayscapital.com>

Closes #21401 from wajda/SPARK-24350-array_position-error-fix.
2018-05-23 17:22:52 -07:00
Jose Torres f457933293 [SPARK-23416][SS] Add a specific stop method for ContinuousExecution.
## What changes were proposed in this pull request?

Add a specific stop method for ContinuousExecution. The previous StreamExecution.stop() method had a race condition as applied to continuous processing: if the cancellation was round-tripped to the driver too quickly, the generic SparkException it caused would be reported as the query death cause. We earlier decided that SparkException should not be added to the StreamExecution.isInterruptionException() whitelist, so we need to ensure this never happens instead.

## How was this patch tested?

Existing tests. I could consistently reproduce the previous flakiness by putting Thread.sleep(1000) between the first job cancellation and thread interruption in StreamExecution.stop().

Author: Jose Torres <torres.joseph.f+github@gmail.com>

Closes #21384 from jose-torres/fixKafka.
2018-05-23 17:21:29 -07:00
jinxing b7a036b75b [SPARK-24294] Throw SparkException when OOM in BroadcastExchangeExec
## What changes were proposed in this pull request?

When OutOfMemoryError thrown from BroadcastExchangeExec, scala.concurrent.Future will hit scala bug – https://github.com/scala/bug/issues/9554, and hang until future timeout:

We could wrap the OOM inside SparkException to resolve this issue.

## How was this patch tested?

Manually tested.

Author: jinxing <jinxing6042@126.com>

Closes #21342 from jinxing64/SPARK-24294.
2018-05-23 13:12:05 -07:00
Takeshi Yamamuro 84557bc9f8 [SPARK-24206][SQL] Improve DataSource read benchmark code
## What changes were proposed in this pull request?
This pr added benchmark code `DataSourceReadBenchmark` for `orc`, `paruqet`, `csv`, and `json` based on the existing `ParquetReadBenchmark` and `OrcReadBenchmark`.

## How was this patch tested?
N/A

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21266 from maropu/DataSourceReadBenchmark.
2018-05-23 13:02:32 -07:00
Xiao Li 5a5a868dc4 Revert "[SPARK-24244][SQL] Passing only required columns to the CSV parser"
This reverts commit 8086acc2f6.
2018-05-23 11:51:13 -07:00
Liang-Chi Hsieh a40ffc656d [SPARK-23711][SQL] Add fallback generator for UnsafeProjection
## What changes were proposed in this pull request?

Add fallback logic for `UnsafeProjection`. In production we can try to create unsafe projection using codegen implementation. Once any compile error happens, it fallbacks to interpreted implementation.

## How was this patch tested?

Added test.

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

Closes #21106 from viirya/SPARK-23711.
2018-05-23 22:40:52 +08:00
Seth Fitzsimmons 00c13cfad7 Correct reference to Offset class
This is a documentation-only correction; `org.apache.spark.sql.sources.v2.reader.Offset` is actually `org.apache.spark.sql.sources.v2.reader.streaming.Offset`.

Author: Seth Fitzsimmons <seth@mojodna.net>

Closes #21387 from mojodna/patch-1.
2018-05-23 09:14:03 +08:00
Vayda, Oleksandr: IT (PRG) bc6ea614ad [SPARK-24348][SQL] "element_at" error fix
## What changes were proposed in this pull request?

### Fixes a `scala.MatchError` in the `element_at` operation - [SPARK-24348](https://issues.apache.org/jira/browse/SPARK-24348)

When calling `element_at` with a wrong first operand type an `AnalysisException` should be thrown instead of `scala.MatchError`

*Example:*
```sql
select element_at('foo', 1)
```

results in:
```
scala.MatchError: StringType (of class org.apache.spark.sql.types.StringType$)
	at org.apache.spark.sql.catalyst.expressions.ElementAt.inputTypes(collectionOperations.scala:1469)
	at org.apache.spark.sql.catalyst.expressions.ExpectsInputTypes$class.checkInputDataTypes(ExpectsInputTypes.scala:44)
	at org.apache.spark.sql.catalyst.expressions.ElementAt.checkInputDataTypes(collectionOperations.scala:1478)
	at org.apache.spark.sql.catalyst.expressions.Expression.resolved$lzycompute(Expression.scala:168)
	at org.apache.spark.sql.catalyst.expressions.Expression.resolved(Expression.scala:168)
	at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAliases$$anonfun$org$apache$spark$sql$catalyst$analysis$Analyzer$ResolveAliases$$assignAliases$1$$anonfun$apply$3.applyOrElse(Analyzer.scala:256)
	at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveAliases$$anonfun$org$apache$spark$sql$catalyst$analysis$Analyzer$ResolveAliases$$assignAliases$1$$anonfun$apply$3.applyOrElse(Analyzer.scala:252)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:289)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:289)
	at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70)
	at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:288)
```

## How was this patch tested?

unit tests

Author: Vayda, Oleksandr: IT (PRG) <Oleksandr.Vayda@barclayscapital.com>

Closes #21395 from wajda/SPARK-24348-element_at-error-fix.
2018-05-22 13:01:07 -07:00
Liang-Chi Hsieh f9f055afa4 [SPARK-24121][SQL] Add API for handling expression code generation
## What changes were proposed in this pull request?

This patch tries to implement this [proposal](https://github.com/apache/spark/pull/19813#issuecomment-354045400) to add an API for handling expression code generation. It should allow us to manipulate how to generate codes for expressions.

In details, this adds an new abstraction `CodeBlock` to `JavaCode`. `CodeBlock` holds the code snippet and inputs for generating actual java code.

For example, in following java code:

```java
  int ${variable} = 1;
  boolean ${isNull} = ${CodeGenerator.defaultValue(BooleanType)};
```

`variable`, `isNull` are two `VariableValue` and `CodeGenerator.defaultValue(BooleanType)` is a string. They are all inputs to this code block and held by `CodeBlock` representing this code.

For codegen, we provide a specified string interpolator `code`, so you can define a code like this:
```scala
  val codeBlock =
    code"""
         |int ${variable} = 1;
         |boolean ${isNull} = ${CodeGenerator.defaultValue(BooleanType)};
        """.stripMargin
  // Generates actual java code.
  codeBlock.toString
```

Because those inputs are held separately in `CodeBlock` before generating code, we can safely manipulate them, e.g., replacing statements to aliased variables, etc..

## How was this patch tested?

Added tests.

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

Closes #21193 from viirya/SPARK-24121.
2018-05-23 01:50:22 +08:00
Maxim Gekk 8086acc2f6 [SPARK-24244][SQL] Passing only required columns to the CSV parser
## What changes were proposed in this pull request?

uniVocity parser allows to specify only required column names or indexes for [parsing](https://www.univocity.com/pages/parsers-tutorial) like:

```
// Here we select only the columns by their indexes.
// The parser just skips the values in other columns
parserSettings.selectIndexes(4, 0, 1);
CsvParser parser = new CsvParser(parserSettings);
```
In this PR, I propose to extract indexes from required schema and pass them into the CSV parser. Benchmarks show the following improvements in parsing of 1000 columns:

```
Select 100 columns out of 1000: x1.76
Select 1 column out of 1000: x2
```

**Note**: Comparing to current implementation, the changes can return different result for malformed rows in the `DROPMALFORMED` and `FAILFAST` modes if only subset of all columns is requested. To have previous behavior, set `spark.sql.csv.parser.columnPruning.enabled` to `false`.

## How was this patch tested?

It was tested by new test which selects 3 columns out of 15, by existing tests and by new benchmarks.

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #21296 from MaxGekk/csv-column-pruning.
2018-05-22 22:07:32 +08:00
Marco Gaido d3d1807315 [SPARK-24313][SQL] Fix collection operations' interpreted evaluation for complex types
## What changes were proposed in this pull request?

The interpreted evaluation of several collection operations works only for simple datatypes. For complex data types, for instance, `array_contains` it returns always `false`. The list of the affected functions is `array_contains`, `array_position`, `element_at` and `GetMapValue`.

The PR fixes the behavior for all the datatypes.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21361 from mgaido91/SPARK-24313.
2018-05-22 21:08:49 +08:00
Marco Gaido 84d31aa5d4 [SPARK-24209][SHS] Automatic retrieve proxyBase from Knox headers
## What changes were proposed in this pull request?

The PR retrieves the proxyBase automatically from the header `X-Forwarded-Context` (if available). This is the header used by Knox to inform the proxied service about the base path.

This provides 0-configuration support for Knox gateway (instead of having to properly set `spark.ui.proxyBase`) and it allows to access directly SHS when it is proxied by Knox. In the previous scenario, indeed, after setting `spark.ui.proxyBase`, direct access to SHS was not working fine (due to bad link generated).

## How was this patch tested?

added UT + manual tests

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21268 from mgaido91/SPARK-24209.
2018-05-21 18:11:05 -07:00
Maxim Gekk b550b2a1a1 [SPARK-24325] Tests for Hadoop's LinesReader
## What changes were proposed in this pull request?

The tests cover basic functionality of [Hadoop LinesReader](8d79113b81/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala (L42)). In particular, the added tests check:

- A split slices a line or delimiter
- A split slices two consecutive lines and cover a delimiter between the lines
- Two splits slice a line and there are no duplicates
- Internal buffer size (`io.file.buffer.size`) is less than line length
- Constrain of maximum line length - `mapreduce.input.linerecordreader.line.maxlength`

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #21377 from MaxGekk/line-reader-tests.
2018-05-21 14:21:05 -07:00
Jose Torres a33dcf4a0b [SPARK-24234][SS] Reader for continuous processing shuffle
## What changes were proposed in this pull request?

Read RDD for continuous processing shuffle, as well as the initial RPC-based row receiver.

https://docs.google.com/document/d/1IL4kJoKrZWeyIhklKUJqsW-yEN7V7aL05MmM65AYOfE/edit#heading=h.8t3ci57f7uii

## How was this patch tested?

new unit tests

Author: Jose Torres <torres.joseph.f+github@gmail.com>

Closes #21337 from jose-torres/readerRddMaster.
2018-05-21 12:58:05 -07:00
Wenchen Fan 03e90f65bf [SPARK-24250][SQL] support accessing SQLConf inside tasks
re-submit https://github.com/apache/spark/pull/21299 which broke build.

A few new commits are added to fix the SQLConf problem in `JsonSchemaInference.infer`, and prevent us to access `SQLConf` in DAGScheduler event loop thread.

## What changes were proposed in this pull request?

Previously in #20136 we decided to forbid tasks to access `SQLConf`, because it doesn't work and always give you the default conf value. In #21190 we fixed the check and all the places that violate it.

Currently the pattern of accessing configs at the executor side is: read the configs at the driver side, then access the variables holding the config values in the RDD closure, so that they will be serialized to the executor side. Something like
```
val someConf = conf.getXXX
child.execute().mapPartitions {
  if (someConf == ...) ...
  ...
}
```

However, this pattern is hard to apply if the config needs to be propagated via a long call stack. An example is `DataType.sameType`, and see how many changes were made in #21190 .

When it comes to code generation, it's even worse. I tried it locally and we need to change a ton of files to propagate configs to code generators.

This PR proposes to allow tasks to access `SQLConf`. The idea is, we can save all the SQL configs to job properties when an SQL execution is triggered. At executor side we rebuild the `SQLConf` from job properties.

## How was this patch tested?

a new test suite

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21376 from cloud-fan/config.
2018-05-22 00:19:18 +08:00
Marek Novotny a6e883feb3 [SPARK-23935][SQL] Adding map_entries function
## What changes were proposed in this pull request?

This PR adds `map_entries` function that returns an unordered array of all entries in the given map.

## How was this patch tested?

New tests added into:
- `CollectionExpressionSuite`
- `DataFrameFunctionsSuite`

## CodeGen examples
### Primitive types
```
val df = Seq(Map(1 -> 5, 2 -> 6)).toDF("m")
df.filter('m.isNotNull).select(map_entries('m)).debugCodegen
```
Result:
```
/* 042 */         boolean project_isNull_0 = false;
/* 043 */
/* 044 */         ArrayData project_value_0 = null;
/* 045 */
/* 046 */         final int project_numElements_0 = inputadapter_value_0.numElements();
/* 047 */         final ArrayData project_keys_0 = inputadapter_value_0.keyArray();
/* 048 */         final ArrayData project_values_0 = inputadapter_value_0.valueArray();
/* 049 */
/* 050 */         final long project_size_0 = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(
/* 051 */           project_numElements_0,
/* 052 */           32);
/* 053 */         if (project_size_0 > 2147483632) {
/* 054 */           final Object[] project_internalRowArray_0 = new Object[project_numElements_0];
/* 055 */           for (int z = 0; z < project_numElements_0; z++) {
/* 056 */             project_internalRowArray_0[z] = new org.apache.spark.sql.catalyst.expressions.GenericInternalRow(new Object[]{project_keys_0.getInt(z), project_values_0.getInt(z)});
/* 057 */           }
/* 058 */           project_value_0 = new org.apache.spark.sql.catalyst.util.GenericArrayData(project_internalRowArray_0);
/* 059 */
/* 060 */         } else {
/* 061 */           final byte[] project_arrayBytes_0 = new byte[(int)project_size_0];
/* 062 */           UnsafeArrayData project_unsafeArrayData_0 = new UnsafeArrayData();
/* 063 */           Platform.putLong(project_arrayBytes_0, 16, project_numElements_0);
/* 064 */           project_unsafeArrayData_0.pointTo(project_arrayBytes_0, 16, (int)project_size_0);
/* 065 */
/* 066 */           final int project_structsOffset_0 = UnsafeArrayData.calculateHeaderPortionInBytes(project_numElements_0) + project_numElements_0 * 8;
/* 067 */           UnsafeRow project_unsafeRow_0 = new UnsafeRow(2);
/* 068 */           for (int z = 0; z < project_numElements_0; z++) {
/* 069 */             long offset = project_structsOffset_0 + z * 24L;
/* 070 */             project_unsafeArrayData_0.setLong(z, (offset << 32) + 24L);
/* 071 */             project_unsafeRow_0.pointTo(project_arrayBytes_0, 16 + offset, 24);
/* 072 */             project_unsafeRow_0.setInt(0, project_keys_0.getInt(z));
/* 073 */             project_unsafeRow_0.setInt(1, project_values_0.getInt(z));
/* 074 */           }
/* 075 */           project_value_0 = project_unsafeArrayData_0;
/* 076 */
/* 077 */         }
```
### Non-primitive types
```
val df = Seq(Map("a" -> "foo", "b" -> null)).toDF("m")
df.filter('m.isNotNull).select(map_entries('m)).debugCodegen
```
Result:
```
/* 042 */         boolean project_isNull_0 = false;
/* 043 */
/* 044 */         ArrayData project_value_0 = null;
/* 045 */
/* 046 */         final int project_numElements_0 = inputadapter_value_0.numElements();
/* 047 */         final ArrayData project_keys_0 = inputadapter_value_0.keyArray();
/* 048 */         final ArrayData project_values_0 = inputadapter_value_0.valueArray();
/* 049 */
/* 050 */         final Object[] project_internalRowArray_0 = new Object[project_numElements_0];
/* 051 */         for (int z = 0; z < project_numElements_0; z++) {
/* 052 */           project_internalRowArray_0[z] = new org.apache.spark.sql.catalyst.expressions.GenericInternalRow(new Object[]{project_keys_0.getUTF8String(z), project_values_0.getUTF8String(z)});
/* 053 */         }
/* 054 */         project_value_0 = new org.apache.spark.sql.catalyst.util.GenericArrayData(project_internalRowArray_0);
```

Author: Marek Novotny <mn.mikke@gmail.com>

Closes #21236 from mn-mikke/feature/array-api-map_entries-to-master.
2018-05-21 23:14:03 +09:00
Kazuaki Ishizaki e480eccd97 [SPARK-24323][SQL] Fix lint-java errors
## What changes were proposed in this pull request?

This PR fixes the following errors reported by `lint-java`
```
% dev/lint-java
Using `mvn` from path: /usr/bin/mvn
Checkstyle checks failed at following occurrences:
[ERROR] src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartition.java:[39] (sizes) LineLength: Line is longer than 100 characters (found 104).
[ERROR] src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartitionReader.java:[26] (sizes) LineLength: Line is longer than 100 characters (found 110).
[ERROR] src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartitionReader.java:[30] (sizes) LineLength: Line is longer than 100 characters (found 104).
```

## How was this patch tested?

Run `lint-java` manually.

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

Closes #21374 from kiszk/SPARK-24323.
2018-05-21 15:42:04 +08:00
Liang-Chi Hsieh 6d7d45a1af [SPARK-24242][SQL] RangeExec should have correct outputOrdering and outputPartitioning
## What changes were proposed in this pull request?

Logical `Range` node has been added with `outputOrdering` recently. It's used to eliminate redundant `Sort` during optimization. However, this `outputOrdering` doesn't not propagate to physical `RangeExec` node.

We also add correct `outputPartitioning` to `RangeExec` node.

## How was this patch tested?

Added test.

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

Closes #21291 from viirya/SPARK-24242.
2018-05-21 15:39:35 +08:00
Wenchen Fan 000e25ae79 Revert "[SPARK-24250][SQL] support accessing SQLConf inside tasks"
This reverts commit dd37529a8d.
2018-05-20 16:13:42 +08:00
Wenchen Fan dd37529a8d [SPARK-24250][SQL] support accessing SQLConf inside tasks
## What changes were proposed in this pull request?

Previously in #20136 we decided to forbid tasks to access `SQLConf`, because it doesn't work and always give you the default conf value. In #21190 we fixed the check and all the places that violate it.

Currently the pattern of accessing configs at the executor side is: read the configs at the driver side, then access the variables holding the config values in the RDD closure, so that they will be serialized to the executor side. Something like
```
val someConf = conf.getXXX
child.execute().mapPartitions {
  if (someConf == ...) ...
  ...
}
```

However, this pattern is hard to apply if the config needs to be propagated via a long call stack. An example is `DataType.sameType`, and see how many changes were made in #21190 .

When it comes to code generation, it's even worse. I tried it locally and we need to change a ton of files to propagate configs to code generators.

This PR proposes to allow tasks to access `SQLConf`. The idea is, we can save all the SQL configs to job properties when an SQL execution is triggered. At executor side we rebuild the `SQLConf` from job properties.

## How was this patch tested?

a new test suite

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21299 from cloud-fan/config.
2018-05-19 18:51:02 +08:00
Efim Poberezkin 434d74e337 [SPARK-23503][SS] Enforce sequencing of committed epochs for Continuous Execution
## What changes were proposed in this pull request?

Made changes to EpochCoordinator so that it enforces a commit order. In case a message for epoch n is lost and epoch (n + 1) is ready for commit before epoch n is, epoch (n + 1) will wait for epoch n to be committed first.

## How was this patch tested?

Existing tests in ContinuousSuite and EpochCoordinatorSuite.

Author: Efim Poberezkin <efim@poberezkin.ru>

Closes #20936 from efimpoberezkin/pr/sequence-commited-epochs.
2018-05-18 16:54:39 -07:00
Arun Mahadevan 710e4e81a8 [SPARK-24308][SQL] Handle DataReaderFactory to InputPartition rename in left over classes
## What changes were proposed in this pull request?

SPARK-24073 renames DataReaderFactory -> InputPartition and DataReader -> InputPartitionReader. Some classes still reflects the old name and causes confusion. This patch renames the left over classes to reflect the new interface and fixes a few comments.

## How was this patch tested?

Existing unit tests.

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

Author: Arun Mahadevan <arunm@apache.org>

Closes #21355 from arunmahadevan/SPARK-24308.
2018-05-18 14:37:01 -07:00
Takeshi Yamamuro a53ea70c1d [SPARK-23856][SQL] Add an option queryTimeout in JDBCOptions
## What changes were proposed in this pull request?
This pr added an option `queryTimeout` for the number of seconds the  the driver will wait for a Statement object to execute.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21173 from maropu/SPARK-23856.
2018-05-18 13:38:36 -07:00
gatorsmile 1c4553d67d Revert "[SPARK-24277][SQL] Code clean up in SQL module: HadoopMapReduceCommitProtocol"
This reverts commit 7b2dca5b12.
2018-05-18 12:51:09 -07:00
Marcelo Vanzin ed7ba7db8f [SPARK-23850][SQL] Add separate config for SQL options redaction.
The old code was relying on a core configuration and extended its
default value to include things that redact desired things in the
app's environment. Instead, add a SQL-specific option for which
options to redact, and apply both the core and SQL-specific rules
when redacting the options in the save command.

This is a little sub-optimal since it adds another config, but it
retains the current default behavior.

While there I also fixed a typo and a couple of minor config API
usage issues in the related redaction option that SQL already had.

Tested with existing unit tests, plus checking the env page on
a shell UI.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #21158 from vanzin/SPARK-23850.
2018-05-18 11:14:22 -07:00
Tathagata Das 807ba44cb7 [SPARK-24159][SS] Enable no-data micro batches for streaming mapGroupswithState
## What changes were proposed in this pull request?

Enabled no-data batches in flatMapGroupsWithState in following two cases.
- When ProcessingTime timeout is used, then we always run a batch every trigger interval.
- When event-time watermark is defined, then the user may be doing arbitrary logic against the watermark value even if timeouts are not set. In such cases, it's best to run batches whenever the watermark has changed, irrespective of whether timeouts (i.e. event-time timeout) have been explicitly enabled.

## How was this patch tested?
updated tests

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

Closes #21345 from tdas/SPARK-24159.
2018-05-18 10:35:43 -07:00
Soham Aurangabadkar 7696b9de0d [SPARK-20538][SQL] Wrap Dataset.reduce with withNewRddExecutionId.
## What changes were proposed in this pull request?

Wrap Dataset.reduce with `withNewExecutionId`.

Author: Soham Aurangabadkar <sohama4@gmail.com>

Closes #21316 from sohama4/dataset_reduce_withexecutionid.
2018-05-18 10:29:34 -07:00
Gengliang Wang 7b2dca5b12 [SPARK-24277][SQL] Code clean up in SQL module: HadoopMapReduceCommitProtocol
## What changes were proposed in this pull request?

In HadoopMapReduceCommitProtocol and FileFormatWriter, there are unnecessary settings in hadoop configuration.

Also clean up some code in SQL module.

## How was this patch tested?

Unit test

Author: Gengliang Wang <gengliang.wang@databricks.com>

Closes #21329 from gengliangwang/codeCleanWrite.
2018-05-18 15:32:29 +08:00
jinxing 8a837bf4f3 [SPARK-24193] create TakeOrderedAndProjectExec only when the limit number is below spark.sql.execution.topKSortFallbackThreshold.
## What changes were proposed in this pull request?

Physical plan of `select colA from t order by colB limit M` is `TakeOrderedAndProject`;
Currently `TakeOrderedAndProject` sorts data in memory, see https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala#L158
We can add a config – if the number of limit (M) is too big, we can sort by disk. Thus memory issue can be resolved.

## How was this patch tested?

Test added

Author: jinxing <jinxing6042@126.com>

Closes #21252 from jinxing64/SPARK-24193.
2018-05-17 22:29:18 +08:00
Marco Gaido 69350aa2f0 [SPARK-23922][SQL] Add arrays_overlap function
## What changes were proposed in this pull request?

The PR adds the function `arrays_overlap`. This function returns `true` if the input arrays contain a non-null common element; if not, it returns `null` if any of the arrays contains a `null` element, `false` otherwise.

## How was this patch tested?

added UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21028 from mgaido91/SPARK-23922.
2018-05-17 20:45:32 +08:00
Florent Pépin 3e66350c24 [SPARK-23925][SQL] Add array_repeat collection function
## What changes were proposed in this pull request?

The PR adds a new collection function, array_repeat. As there already was a function repeat with the same signature, with the only difference being the expected return type (String instead of Array), the new function is called array_repeat to distinguish.
The behaviour of the function is based on Presto's one.

The function creates an array containing a given element repeated the requested number of times.

## How was this patch tested?

New unit tests added into:
- CollectionExpressionsSuite
- DataFrameFunctionsSuite

Author: Florent Pépin <florentpepin.92@gmail.com>
Author: Florent Pépin <florent.pepin14@imperial.ac.uk>

Closes #21208 from pepinoflo/SPARK-23925.
2018-05-17 13:31:14 +09:00
Tathagata Das 991726f31a [SPARK-24158][SS] Enable no-data batches for streaming joins
## What changes were proposed in this pull request?

This is a continuation of the larger task of enabling zero-data batches for more eager state cleanup. This PR enables it for stream-stream joins.

## How was this patch tested?
- Updated join tests. Additionally, updated them to not use `CheckLastBatch` anywhere to set good precedence for future.

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

Closes #21253 from tdas/SPARK-24158.
2018-05-16 14:55:02 -07:00
Gengliang Wang 6fb7d6c4f7 [SPARK-24275][SQL] Revise doc comments in InputPartition
## What changes were proposed in this pull request?

In #21145,  DataReaderFactory is renamed to InputPartition.

This PR is to revise wording in the comments to make it more clear.

## How was this patch tested?

None

Author: Gengliang Wang <gengliang.wang@databricks.com>

Closes #21326 from gengliangwang/revise_reader_comments.
2018-05-17 00:40:39 +08:00
Wenchen Fan 943493b165 Revert "[SPARK-22938][SQL][FOLLOWUP] Assert that SQLConf.get is acces…
…sed only on the driver"

This reverts commit a4206d58e0.

This is from https://github.com/apache/spark/pull/21299 and to ease the review of it.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21341 from cloud-fan/revert.
2018-05-16 22:01:24 +08:00
Jose Torres 3fabbc5762 [SPARK-24040][SS] Support single partition aggregates in continuous processing.
## What changes were proposed in this pull request?

Support aggregates with exactly 1 partition in continuous processing.

A few small tweaks are needed to make this work:

* Replace currentEpoch tracking with an ThreadLocal. This means that current epoch is scoped to a task rather than a node, but I think that's sustainable even once we add shuffle.
* Add a new testing-only flag to disable the UnsupportedOperationChecker whitelist of allowed continuous processing nodes. I think this is preferable to writing a pile of custom logic to enforce that there is in fact only 1 partition; we plan to support multi-partition aggregates before the next Spark release, so we'd just have to tear that logic back out.
* Restart continuous processing queries from the first available uncommitted epoch, rather than one that's guaranteed to be unused. This is required for stateful operators to overwrite partial state from the previous attempt at the epoch, and there was no specific motivation for the original strategy. In another PR before stabilizing the StreamWriter API, we'll need to narrow down and document more precise semantic guarantees for the epoch IDs.
* We need a single-partition ContinuousMemoryStream. The way MemoryStream is constructed means it can't be a text option like it is for rate source, unfortunately.

## How was this patch tested?

new unit tests

Author: Jose Torres <torres.joseph.f+github@gmail.com>

Closes #21239 from jose-torres/withAggr.
2018-05-15 10:25:29 -07:00
Liang-Chi Hsieh d610d2a3f5 [SPARK-24259][SQL] ArrayWriter for Arrow produces wrong output
## What changes were proposed in this pull request?

Right now `ArrayWriter` used to output Arrow data for array type, doesn't do `clear` or `reset` after each batch. It produces wrong output.

## How was this patch tested?

Added test.

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

Closes #21312 from viirya/SPARK-24259.
2018-05-15 22:06:58 +08:00
Goun Na e29176fd7d [SPARK-23627][SQL] Provide isEmpty in Dataset
## What changes were proposed in this pull request?

This PR adds isEmpty() in DataSet

## How was this patch tested?

Unit tests added

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

Author: Goun Na <gounna@gmail.com>
Author: goungoun <gounna@gmail.com>

Closes #20800 from goungoun/SPARK-23627.
2018-05-15 14:11:20 +08:00
Henry Robinson 061e0084ce [SPARK-23852][SQL] Add withSQLConf(...) to test case
## What changes were proposed in this pull request?

Add a `withSQLConf(...)` wrapper to force Parquet filter pushdown for a test that relies on it.

## How was this patch tested?

Test passes

Author: Henry Robinson <henry@apache.org>

Closes #21323 from henryr/spark-23582.
2018-05-14 14:35:08 -07:00
Maxim Gekk 8cd83acf40 [SPARK-24027][SQL] Support MapType with StringType for keys as the root type by from_json
## What changes were proposed in this pull request?

Currently, the from_json function support StructType or ArrayType as the root type. The PR allows to specify MapType(StringType, DataType) as the root type additionally to mentioned types. For example:

```scala
import org.apache.spark.sql.types._
val schema = MapType(StringType, IntegerType)
val in = Seq("""{"a": 1, "b": 2, "c": 3}""").toDS()
in.select(from_json($"value", schema, Map[String, String]())).collect()
```
```
res1: Array[org.apache.spark.sql.Row] = Array([Map(a -> 1, b -> 2, c -> 3)])
```

## How was this patch tested?

It was checked by new tests for the map type with integer type and struct type as value types. Also roundtrip tests like from_json(to_json) and to_json(from_json) for MapType are added.

Author: Maxim Gekk <maxim.gekk@databricks.com>
Author: Maxim Gekk <max.gekk@gmail.com>

Closes #21108 from MaxGekk/from_json-map-type.
2018-05-14 14:05:42 -07:00
Shixiong Zhu c26f673252 [SPARK-24246][SQL] Improve AnalysisException by setting the cause when it's available
## What changes were proposed in this pull request?

If there is an exception, it's better to set it as the cause of AnalysisException since the exception may contain useful debug information.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #21297 from zsxwing/SPARK-24246.
2018-05-14 11:37:57 -07:00
Kazuaki Ishizaki b6c50d7820 [SPARK-24228][SQL] Fix Java lint errors
## What changes were proposed in this pull request?
This PR fixes the following Java lint errors due to importing unimport classes

```
$ dev/lint-java
Using `mvn` from path: /usr/bin/mvn
Checkstyle checks failed at following occurrences:
[ERROR] src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Distribution.java:[25] (sizes) LineLength: Line is longer than 100 characters (found 109).
[ERROR] src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousReader.java:[38] (sizes) LineLength: Line is longer than 100 characters (found 102).
[ERROR] src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java:[21,8] (imports) UnusedImports: Unused import - java.io.ByteArrayInputStream.
[ERROR] src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedPlainValuesReader.java:[29,8] (imports) UnusedImports: Unused import - org.apache.spark.unsafe.Platform.
[ERROR] src/test/java/test/org/apache/spark/sql/sources/v2/JavaAdvancedDataSourceV2.java:[110] (sizes) LineLength: Line is longer than 100 characters (found 101).
```

With this PR
```
$ dev/lint-java
Using `mvn` from path: /usr/bin/mvn
Checkstyle checks passed.
```

## How was this patch tested?

Existing UTs. Also manually run checkstyles against these two files.

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

Closes #21301 from kiszk/SPARK-24228.
2018-05-14 10:57:10 +08:00
Maxim Gekk 7a2d4895c7 [SPARK-17916][SQL] Fix empty string being parsed as null when nullValue is set.
## What changes were proposed in this pull request?

I propose to bump version of uniVocity parser up to 2.6.3 where quoted empty strings are replaced by the empty value (passed to `setEmptyValue`) instead of `null` values as in the current version 2.5.9:
https://github.com/uniVocity/univocity-parsers/blob/v2.6.3/src/main/java/com/univocity/parsers/csv/CsvParser.java#L125

Empty value for writer is set to `""`. So, empty string in dataframe/dataset is stored as empty quoted string `""`. Empty value for reader is set to empty string (zero size). In this way, saved empty quoted string will be read as just empty string. Please, look at the tests for more details.

Here are main changes made in [2.6.0](https://github.com/uniVocity/univocity-parsers/releases/tag/v2.6.0), [2.6.1](https://github.com/uniVocity/univocity-parsers/releases/tag/v2.6.1), [2.6.2](https://github.com/uniVocity/univocity-parsers/releases/tag/v2.6.2), [2.6.3](https://github.com/uniVocity/univocity-parsers/releases/tag/v2.6.3):

- CSV parser now parses quoted values ~30% faster
- CSV format detection process has option provide a list of possible delimiters, in order of priority ( i.e. settings.detectFormatAutomatically( '-', '.');) - https://github.com/uniVocity/univocity-parsers/issues/214
- Implemented trim quoted values support - https://github.com/uniVocity/univocity-parsers/issues/230
- NullPointer when stopping parser when nothing is parsed - https://github.com/uniVocity/univocity-parsers/issues/219
- Concurrency issue when calling stopParsing() - https://github.com/uniVocity/univocity-parsers/issues/231

Closes #20068

## How was this patch tested?

Added tests from the PR https://github.com/apache/spark/pull/20068

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #21273 from MaxGekk/univocity-2.6.
2018-05-14 10:01:06 +08:00
Reynold Xin e3dabdf6ef [SPARK-23907] Removes regr_* functions in functions.scala
## What changes were proposed in this pull request?
This patch removes the various regr_* functions in functions.scala. They are so uncommon that I don't think they deserve real estate in functions.scala. We can consider adding them later if more users need them.

## How was this patch tested?
Removed the associated test case as well.

Author: Reynold Xin <rxin@databricks.com>

Closes #21309 from rxin/SPARK-23907.
2018-05-12 12:15:36 +08:00
aditkumar 92f6f52ff0 [MINOR][DOCS] Documenting months_between direction
## What changes were proposed in this pull request?

It's useful to know what relationship between date1 and date2 results in a positive number.

Author: aditkumar <aditkumar@gmail.com>
Author: Adit Kumar <aditkumar@gmail.com>

Closes #20787 from aditkumar/master.
2018-05-11 14:42:23 -05:00
Wenchen Fan 928845a422 [SPARK-24172][SQL] we should not apply operator pushdown to data source v2 many times
## What changes were proposed in this pull request?

In `PushDownOperatorsToDataSource`, we use `transformUp` to match `PhysicalOperation` and apply pushdown. This is problematic if we have multiple `Filter` and `Project` above the data source v2 relation.

e.g. for a query
```
Project
  Filter
    DataSourceV2Relation
```

The pattern match will be triggered twice and we will do operator pushdown twice. This is unnecessary, we can use `mapChildren` to only apply pushdown once.

## How was this patch tested?

existing test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21230 from cloud-fan/step2.
2018-05-11 10:00:28 -07:00
Wenchen Fan a4206d58e0 [SPARK-22938][SQL][FOLLOWUP] Assert that SQLConf.get is accessed only on the driver
## What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/20136 . #20136 didn't really work because in the test, we are using local backend, which shares the driver side `SparkEnv`, so `SparkEnv.get.executorId == SparkContext.DRIVER_IDENTIFIER` doesn't work.

This PR changes the check to `TaskContext.get != null`, and move the check to `SQLConf.get`, and fix all the places that violate this check:
* `InMemoryTableScanExec#createAndDecompressColumn` is executed inside `rdd.map`, we can't access `conf.offHeapColumnVectorEnabled` there. https://github.com/apache/spark/pull/21223 merged
* `DataType#sameType` may be executed in executor side, for things like json schema inference, so we can't call `conf.caseSensitiveAnalysis` there. This contributes to most of the code changes, as we need to add `caseSensitive` parameter to a lot of methods.
* `ParquetFilters` is used in the file scan function, which is executed in executor side, so we can't can't call `conf.parquetFilterPushDownDate` there. https://github.com/apache/spark/pull/21224 merged
* `WindowExec#createBoundOrdering` is called on executor side, so we can't use `conf.sessionLocalTimezone` there. https://github.com/apache/spark/pull/21225 merged
* `JsonToStructs` can be serialized to executors and evaluate, we should not call `SQLConf.get.getConf(SQLConf.FROM_JSON_FORCE_NULLABLE_SCHEMA)` in the body. https://github.com/apache/spark/pull/21226 merged

## How was this patch tested?

existing test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21190 from cloud-fan/minor.
2018-05-11 09:01:40 +08:00
Maxim Gekk f4fed05121 [SPARK-24171] Adding a note for non-deterministic functions
## What changes were proposed in this pull request?

I propose to add a clear statement for functions like `collect_list()` about non-deterministic behavior of such functions. The behavior must be taken into account by user while creating and running queries.

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #21228 from MaxGekk/deterministic-comments.
2018-05-10 09:44:49 -07:00
Marco Gaido 94d6714482 [SPARK-23907][SQL] Add regr_* functions
## What changes were proposed in this pull request?

The PR introduces regr_slope, regr_intercept, regr_r2, regr_sxx, regr_syy, regr_sxy, regr_avgx, regr_avgy, regr_count.

The implementation of this functions mirrors Hive's one in HIVE-15978.

## How was this patch tested?

added UT (values compared with Hive)

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21054 from mgaido91/SPARK-23907.
2018-05-10 20:38:52 +09:00
Ryan Blue 62d01391fe [SPARK-24073][SQL] Rename DataReaderFactory to InputPartition.
## What changes were proposed in this pull request?

Renames:
* `DataReaderFactory` to `InputPartition`
* `DataReader` to `InputPartitionReader`
* `createDataReaderFactories` to `planInputPartitions`
* `createUnsafeDataReaderFactories` to `planUnsafeInputPartitions`
* `createBatchDataReaderFactories` to `planBatchInputPartitions`

This fixes the changes in SPARK-23219, which renamed ReadTask to
DataReaderFactory. The intent of that change was to make the read and
write API match (write side uses DataWriterFactory), but the underlying
problem is that the two classes are not equivalent.

ReadTask/DataReader function as Iterable/Iterator. One InputPartition is
a specific partition of the data to be read, in contrast to
DataWriterFactory where the same factory instance is used in all write
tasks. InputPartition's purpose is to manage the lifecycle of the
associated reader, which is now called InputPartitionReader, with an
explicit create operation to mirror the close operation. This was no
longer clear from the API because DataReaderFactory appeared to be more
generic than it is and it isn't clear why a set of them is produced for
a read.

## How was this patch tested?

Existing tests, which have been updated to use the new name.

Author: Ryan Blue <blue@apache.org>

Closes #21145 from rdblue/SPARK-24073-revert-data-reader-factory-rename.
2018-05-09 21:48:54 -07:00
Henry Robinson 9341c951e8 [SPARK-23852][SQL] Add test that fails if PARQUET-1217 is not fixed
## What changes were proposed in this pull request?

Add a new test that triggers if PARQUET-1217 - a predicate pushdown bug - is not fixed in Spark's Parquet dependency.

## How was this patch tested?

New unit test passes.

Author: Henry Robinson <henry@apache.org>

Closes #21284 from henryr/spark-23852.
2018-05-09 19:56:03 -07:00
Shixiong Zhu fd1179c172 [SPARK-24214][SS] Fix toJSON for StreamingRelationV2/StreamingExecutionRelation/ContinuousExecutionRelation
## What changes were proposed in this pull request?

We should overwrite "otherCopyArgs" to provide the SparkSession parameter otherwise TreeNode.toJSON cannot get the full constructor parameter list.

## How was this patch tested?

The new unit test.

Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #21275 from zsxwing/SPARK-24214.
2018-05-09 11:32:17 -07:00
Marcelo Vanzin cc613b552e [PYSPARK] Update py4j to version 0.10.7. 2018-05-09 10:47:35 -07:00
DB Tsai 6ea582e36a
[SPARK-24181][SQL] Better error message for writing sorted data
## What changes were proposed in this pull request?

The exception message should clearly distinguish sorting and bucketing in `save` and `jdbc` write.

When a user tries to write a sorted data using save or insertInto, it will throw an exception with message that `s"'$operation' does not support bucketing right now""`.

We should throw `s"'$operation' does not support sortBy right now""` instead.

## How was this patch tested?

More tests in `DataFrameReaderWriterSuite.scala`

Author: DB Tsai <d_tsai@apple.com>

Closes #21235 from dbtsai/fixException.
2018-05-09 09:15:16 -07:00
Ryan Blue cac9b1dea1 [SPARK-23972][BUILD][SQL] Update Parquet to 1.10.0.
## What changes were proposed in this pull request?

This updates Parquet to 1.10.0 and updates the vectorized path for buffer management changes. Parquet 1.10.0 uses ByteBufferInputStream instead of byte arrays in encoders. This allows Parquet to break allocations into smaller chunks that are better for garbage collection.

## How was this patch tested?

Existing Parquet tests. Running in production at Netflix for about 3 months.

Author: Ryan Blue <blue@apache.org>

Closes #21070 from rdblue/SPARK-23972-update-parquet-to-1.10.0.
2018-05-09 12:27:32 +08:00
Maxim Gekk e3de6ab30d [SPARK-24068] Propagating DataFrameReader's options to Text datasource on schema inferring
## What changes were proposed in this pull request?

While reading CSV or JSON files, DataFrameReader's options are converted to Hadoop's parameters, for example there:
https://github.com/apache/spark/blob/branch-2.3/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala#L302

but the options are not propagated to Text datasource on schema inferring, for instance:
https://github.com/apache/spark/blob/branch-2.3/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala#L184-L188

The PR proposes propagation of user's options to Text datasource on scheme inferring in similar way as user's options are converted to Hadoop parameters if schema is specified.

## How was this patch tested?
The changes were tested manually by using https://github.com/twitter/hadoop-lzo:

```
hadoop-lzo> mvn clean package
hadoop-lzo> ln -s ./target/hadoop-lzo-0.4.21-SNAPSHOT.jar ./hadoop-lzo.jar
```
Create 2 test files in JSON and CSV format and compress them:
```shell
$ cat test.csv
col1|col2
a|1
$ lzop test.csv
$ cat test.json
{"col1":"a","col2":1}
$ lzop test.json
```
Run `spark-shell` with hadoop-lzo:
```
bin/spark-shell --jars ~/hadoop-lzo/hadoop-lzo.jar
```
reading compressed CSV and JSON without schema:
```scala
spark.read.option("io.compression.codecs", "com.hadoop.compression.lzo.LzopCodec").option("inferSchema",true).option("header",true).option("sep","|").csv("test.csv.lzo").show()
+----+----+
|col1|col2|
+----+----+
|   a|   1|
+----+----+
```
```scala
spark.read.option("io.compression.codecs", "com.hadoop.compression.lzo.LzopCodec").option("multiLine", true).json("test.json.lzo").printSchema
root
 |-- col1: string (nullable = true)
 |-- col2: long (nullable = true)
```

Author: Maxim Gekk <maxim.gekk@databricks.com>
Author: Maxim Gekk <max.gekk@gmail.com>

Closes #21182 from MaxGekk/text-options.
2018-05-09 08:32:20 +08:00
Yuming Wang 487faf17ab [SPARK-24117][SQL] Unified the getSizePerRow
## What changes were proposed in this pull request?

This pr unified the `getSizePerRow` because `getSizePerRow` is used in many places. For example:

1. [LocalRelation.scala#L80](f70f46d1e5/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala (L80))
2. [SizeInBytesOnlyStatsPlanVisitor.scala#L36](76b8b840dd/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala (L36))

## How was this patch tested?
Exist tests

Author: Yuming Wang <yumwang@ebay.com>

Closes #21189 from wangyum/SPARK-24117.
2018-05-08 23:43:02 +08:00
gatorsmile 2f6fe7d679 [SPARK-23094][SPARK-23723][SPARK-23724][SQL][FOLLOW-UP] Support custom encoding for json files
## What changes were proposed in this pull request?
This is to add a test case to check the behaviors when users write json in the specified UTF-16/UTF-32 encoding with multiline off.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #21254 from gatorsmile/followupSPARK-23094.
2018-05-08 21:24:35 +08:00
yucai e17567ca78 [SPARK-24076][SQL] Use different seed in HashAggregate to avoid hash conflict
## What changes were proposed in this pull request?

HashAggregate uses the same hash algorithm and seed as ShuffleExchange, it may lead to bad hash conflict when shuffle.partitions=8192*n.

Considering below example:
```
SET spark.sql.shuffle.partitions=8192;
INSERT OVERWRITE TABLE target_xxx
SELECT
 item_id,
 auct_end_dt
FROM
 from source_xxx
GROUP BY
 item_id,
 auct_end_dt;
```

In the shuffle stage, if user sets the shuffle.partition = 8192, all tuples in the same partition will meet the following relationship:
```
hash(tuple x) = hash(tuple y) + n * 8192
```
Then in the next HashAggregate stage, all tuples from the same partition need be put into a 16K BytesToBytesMap (unsafeRowAggBuffer).

Here, the HashAggregate uses the same hash algorithm on the same expression as shuffle, and uses the same seed, and 16K = 8192 * 2, so actually, all tuples in the same parititon will only be hashed to 2 different places in the BytesToBytesMap. It is bad hash conflict. With BytesToBytesMap growing, the conflict will always exist.

Before change:
<img width="334" alt="hash_conflict" src="https://user-images.githubusercontent.com/2989575/39250210-ed032d46-48d2-11e8-855a-c1afc2a0ceb5.png">

After change:
<img width="334" alt="no_hash_conflict" src="https://user-images.githubusercontent.com/2989575/39250218-f1cb89e0-48d2-11e8-9244-5a93c1e8b60d.png">

## How was this patch tested?

Unit tests and production cases.

Author: yucai <yyu1@ebay.com>

Closes #21149 from yucai/SPARK-24076.
2018-05-08 11:34:27 +02:00
Henry Robinson cd12c5c3ec [SPARK-24128][SQL] Mention configuration option in implicit CROSS JOIN error
## What changes were proposed in this pull request?

Mention `spark.sql.crossJoin.enabled` in error message when an implicit `CROSS JOIN` is detected.

## How was this patch tested?

`CartesianProductSuite` and `JoinSuite`.

Author: Henry Robinson <henry@apache.org>

Closes #21201 from henryr/spark-24128.
2018-05-08 12:21:33 +08:00
Marco Gaido e35ad3cadd [SPARK-23930][SQL] Add slice function
## What changes were proposed in this pull request?

The PR add the `slice` function. The behavior of the function is based on Presto's one.

The function slices an array according to the requested start index and length.

## How was this patch tested?

added UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21040 from mgaido91/SPARK-23930.
2018-05-07 16:57:37 +09:00
Gabor Somogyi c5981976f1 [SPARK-23775][TEST] Make DataFrameRangeSuite not flaky
## What changes were proposed in this pull request?

DataFrameRangeSuite.test("Cancelling stage in a query with Range.") stays sometimes in an infinite loop and times out the build.

There were multiple issues with the test:

1. The first valid stageId is zero when the test started alone and not in a suite and the following code waits until timeout:

```
eventually(timeout(10.seconds), interval(1.millis)) {
  assert(DataFrameRangeSuite.stageToKill > 0)
}
```

2. The `DataFrameRangeSuite.stageToKill` was overwritten by the task's thread after the reset which ended up in canceling the same stage 2 times. This caused the infinite wait.

This PR solves this mentioned flakyness by removing the shared `DataFrameRangeSuite.stageToKill` and using `onTaskStart` where stage ID is provided. In order to make sure cancelStage called for all stages `waitUntilEmpty` is called on `ListenerBus`.

In [PR20888](https://github.com/apache/spark/pull/20888) this tried to get solved by:
* Stopping the executor thread with `wait`
* Wait for all `cancelStage` called
* Kill the executor thread by setting `SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL`

but the thread killing left the shared `SparkContext` sometimes in a state where further jobs can't be submitted. As a result DataFrameRangeSuite.test("Cancelling stage in a query with Range.") test passed properly but the next test inside the suite was hanging.

## How was this patch tested?

Existing unit test executed 10k times.

Author: Gabor Somogyi <gabor.g.somogyi@gmail.com>

Closes #21214 from gaborgsomogyi/SPARK-23775_1.
2018-05-07 14:45:14 +08:00
Kazuaki Ishizaki 7564a9a706 [SPARK-23921][SQL] Add array_sort function
## What changes were proposed in this pull request?

The PR adds the SQL function `array_sort`. The behavior of the function is based on Presto's one.

The function sorts the input array in ascending order. The elements of the input array must be orderable. Null elements will be placed at the end of the returned array.

## How was this patch tested?

Added UTs

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

Closes #21021 from kiszk/SPARK-23921.
2018-05-07 15:22:23 +09:00
gatorsmile f38ea00e83 [SPARK-24017][SQL] Refactor ExternalCatalog to be an interface
## What changes were proposed in this pull request?
This refactors the external catalog to be an interface. It can be easier for the future work in the catalog federation. After the refactoring, `ExternalCatalog` is much cleaner without mixing the listener event generation logic.

## How was this patch tested?
The existing tests

Author: gatorsmile <gatorsmile@gmail.com>

Closes #21122 from gatorsmile/refactorExternalCatalog.
2018-05-06 20:41:32 -07:00
Tathagata Das 47b5b68528 [SPARK-24157][SS] Enabled no-data batches in MicroBatchExecution for streaming aggregation and deduplication.
## What changes were proposed in this pull request?

This PR enables the MicroBatchExecution to run no-data batches if some SparkPlan requires running another batch to output results based on updated watermark / processing time. In this PR, I have enabled streaming aggregations and streaming deduplicates to automatically run addition batch even if new data is available. See https://issues.apache.org/jira/browse/SPARK-24156 for more context.

Major changes/refactoring done in this PR.
- Refactoring MicroBatchExecution - A major point of confusion in MicroBatchExecution control flow was always (at least to me) was that `populateStartOffsets` internally called `constructNextBatch` which was not obvious from just the name "populateStartOffsets" and made the control flow from the main trigger execution loop very confusing (main loop in `runActivatedStream` called `constructNextBatch` but only if `populateStartOffsets` hadn't already called it). Instead, the refactoring makes it cleaner.
    - `populateStartOffsets` only the updates `availableOffsets` and `committedOffsets`. Does not call `constructNextBatch`.
    - Main loop in `runActivatedStream` calls `constructNextBatch` which returns true or false reflecting whether the next batch is ready for executing. This method is now idempotent; if a batch has already been constructed, then it will always return true until the batch has been executed.
    - If next batch is ready then we call `runBatch` or sleep.
    - That's it.

- Refactoring watermark management logic - This has been refactored out from `MicroBatchExecution` in a separate class to simplify `MicroBatchExecution`.

- New method `shouldRunAnotherBatch` in `IncrementalExecution` - This returns true if there is any stateful operation in the last execution plan that requires another batch for state cleanup, etc. This is used to decide whether to construct a batch or not in `constructNextBatch`.

- Changes to stream testing framework - Many tests used CheckLastBatch to validate answers. This assumed that there will be no more batches after the last set of input has been processed, so the last batch is the one that has output corresponding to the last input. This is not true anymore. To account for that, I made two changes.
    - `CheckNewAnswer` is a new test action that verifies the new rows generated since the last time the answer was checked by `CheckAnswer`, `CheckNewAnswer` or `CheckLastBatch`. This is agnostic to how many batches occurred between the last check and now. To do make this easier, I added a common trait between MemorySink and MemorySinkV2 to abstract out some common methods.
    - `assertNumStateRows` has been updated in the same way to be agnostic to batches while checking what the total rows and how many state rows were updated (sums up updates since the last check).

## How was this patch tested?
- Changes made to existing tests - Tests have been changed in one of the following patterns.
    - Tests where the last input was given again to force another batch to be executed and state cleaned up / output generated, they were simplified by removing the extra input.
    - Tests using aggregation+watermark where CheckLastBatch were replaced with CheckNewAnswer to make them batch agnostic.
- New tests added to check whether the flag works for streaming aggregation and deduplication

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

Closes #21220 from tdas/SPARK-24157.
2018-05-04 16:35:24 -07:00
Jose Torres af4dc50280 [SPARK-24039][SS] Do continuous processing writes with multiple compute() calls
## What changes were proposed in this pull request?

Do continuous processing writes with multiple compute() calls.

The current strategy (before this PR) is hacky; we just call next() on an iterator which has already returned hasNext = false, knowing that all the nodes we whitelist handle this properly. This will have to be changed before we can support more complex query plans. (In particular, I have a WIP https://github.com/jose-torres/spark/pull/13 which should be able to support aggregates in a single partition with minimal additional work.)

Most of the changes here are just refactoring to accommodate the new model. The behavioral changes are:

* The writer now calls prev.compute(split, context) once per epoch within the epoch loop.
* ContinuousDataSourceRDD now spawns a ContinuousQueuedDataReader which is shared across multiple calls to compute() for the same partition.

## How was this patch tested?

existing unit tests

Author: Jose Torres <torres.joseph.f+github@gmail.com>

Closes #21200 from jose-torres/noAggr.
2018-05-04 14:14:40 -07:00
Arun Mahadevan 7f1b6b182e [SPARK-24136][SS] Fix MemoryStreamDataReader.next to skip sleeping if record is available
## What changes were proposed in this pull request?

Avoid unnecessary sleep (10 ms) in each invocation of MemoryStreamDataReader.next.

## How was this patch tested?

Ran ContinuousSuite from IDE.

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

Author: Arun Mahadevan <arunm@apache.org>

Closes #21207 from arunmahadevan/memorystream.
2018-05-04 16:02:21 +08:00
Wenchen Fan 0c23e254c3 [SPARK-24167][SQL] ParquetFilters should not access SQLConf at executor side
## What changes were proposed in this pull request?

This PR is extracted from #21190 , to make it easier to backport.

`ParquetFilters` is used in the file scan function, which is executed in executor side, so we can't call `conf.parquetFilterPushDownDate` there.

## How was this patch tested?

it's tested in #21190

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21224 from cloud-fan/minor2.
2018-05-04 09:27:14 +08:00
Wenchen Fan e646ae67f2 [SPARK-24168][SQL] WindowExec should not access SQLConf at executor side
## What changes were proposed in this pull request?

This PR is extracted from #21190 , to make it easier to backport.

`WindowExec#createBoundOrdering` is called on executor side, so we can't use `conf.sessionLocalTimezone` there.

## How was this patch tested?

tested in #21190

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21225 from cloud-fan/minor3.
2018-05-03 17:27:13 -07:00
maryannxue e3201e165e [SPARK-24035][SQL] SQL syntax for Pivot
## What changes were proposed in this pull request?

Add SQL support for Pivot according to Pivot grammar defined by Oracle (https://docs.oracle.com/database/121/SQLRF/img_text/pivot_clause.htm) with some simplifications, based on our existing functionality and limitations for Pivot at the backend:
1. For pivot_for_clause (https://docs.oracle.com/database/121/SQLRF/img_text/pivot_for_clause.htm), the column list form is not supported, which means the pivot column can only be one single column.
2. For pivot_in_clause (https://docs.oracle.com/database/121/SQLRF/img_text/pivot_in_clause.htm), the sub-query form and "ANY" is not supported (this is only supported by Oracle for XML anyway).
3. For pivot_in_clause, aliases for the constant values are not supported.

The code changes are:
1. Add parser support for Pivot. Note that according to https://docs.oracle.com/database/121/SQLRF/statements_10002.htm#i2076542, Pivot cannot be used together with lateral views in the from clause. This restriction has been implemented in the Parser rule.
2. Infer group-by expressions: group-by expressions are not explicitly specified in SQL Pivot clause and need to be deduced based on this rule: https://docs.oracle.com/database/121/SQLRF/statements_10002.htm#CHDFAFIE, so we have to post-fix it at query analysis stage.
3. Override Pivot.resolved as "false": for the reason mentioned in [2] and the fact that output attributes change after Pivot being replaced by Project or Aggregate, we avoid resolving parent references until after Pivot has been resolved and replaced.
4. Verify aggregate expressions: only aggregate expressions with or without aliases can appear in the first part of the Pivot clause, and this check is performed as analysis stage.

## How was this patch tested?

A new test suite PivotSuite is added.

Author: maryannxue <maryann.xue@gmail.com>

Closes #21187 from maryannxue/spark-24035.
2018-05-03 17:05:02 -07:00
Wenchen Fan 96a50016bb [SPARK-24169][SQL] JsonToStructs should not access SQLConf at executor side
## What changes were proposed in this pull request?

This PR is extracted from #21190 , to make it easier to backport.

`JsonToStructs` can be serialized to executors and evaluate, we should not call `SQLConf.get.getConf(SQLConf.FROM_JSON_FORCE_NULLABLE_SCHEMA)` in the body.

## How was this patch tested?

tested in #21190

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21226 from cloud-fan/minor4.
2018-05-03 23:36:09 +08:00
Wenchen Fan 991b526992 [SPARK-24166][SQL] InMemoryTableScanExec should not access SQLConf at executor side
## What changes were proposed in this pull request?

This PR is extracted from https://github.com/apache/spark/pull/21190 , to make it easier to backport.

`InMemoryTableScanExec#createAndDecompressColumn` is executed inside `rdd.map`, we can't access `conf.offHeapColumnVectorEnabled` there.

## How was this patch tested?

it's tested in #21190

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21223 from cloud-fan/minor1.
2018-05-03 19:56:30 +08:00
Wenchen Fan 417ad92502 [SPARK-23715][SQL] the input of to/from_utc_timestamp can not have timezone
## What changes were proposed in this pull request?

`from_utc_timestamp` assumes its input is in UTC timezone and shifts it to the specified timezone. When the timestamp contains timezone(e.g. `2018-03-13T06:18:23+00:00`), Spark breaks the semantic and respect the timezone in the string. This is not what user expects and the result is different from Hive/Impala. `to_utc_timestamp` has the same problem.

More details please refer to the JIRA ticket.

This PR fixes this by returning null if the input timestamp contains timezone.

## How was this patch tested?

new tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21169 from cloud-fan/from_utc_timezone.
2018-05-03 19:27:01 +08:00
Takeshi Yamamuro e4c91c089a [SPARK-24111][SQL] Add the TPCDS v2.7 (latest) queries in TPCDSQueryBenchmark
## What changes were proposed in this pull request?
This pr added  the TPCDS v2.7 (latest) queries in `TPCDSQueryBenchmark`.
These query files have been added in `SPARK-23167`.

## How was this patch tested?
Manually checked.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21177 from maropu/AddTpcdsV2_7InBenchmark.
2018-05-02 16:12:21 -07:00
Kazuaki Ishizaki 5be8aab144 [SPARK-23923][SQL] Add cardinality function
## What changes were proposed in this pull request?

The PR adds the SQL function `cardinality`. The behavior of the function is based on Presto's one.

The function returns the length of the array or map stored in the column as `int` while the Presto version returns the value as `BigInt` (`long` in Spark). The discussions regarding the difference of return type are [here](https://github.com/apache/spark/pull/21031#issuecomment-381284638) and [there](https://github.com/apache/spark/pull/21031#discussion_r181622107).

## How was this patch tested?

Added UTs

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

Closes #21031 from kiszk/SPARK-23923.
2018-05-02 13:53:10 -07:00
Ala Luszczak 8bd27025b7 [SPARK-24133][SQL] Check for integer overflows when resizing WritableColumnVectors
## What changes were proposed in this pull request?

`ColumnVector`s store string data in one big byte array. Since the array size is capped at just under Integer.MAX_VALUE, a single `ColumnVector` cannot store more than 2GB of string data.
But since the Parquet files commonly contain large blobs stored as strings, and `ColumnVector`s by default carry 4096 values, it's entirely possible to go past that limit. In such cases a negative capacity is requested from `WritableColumnVector.reserve()`. The call succeeds (requested capacity is smaller than already allocated capacity), and consequently `java.lang.ArrayIndexOutOfBoundsException` is thrown when the reader actually attempts to put the data into the array.

This change introduces a simple check for integer overflow to `WritableColumnVector.reserve()` which should help catch the error earlier and provide more informative exception. Additionally, the error message in `WritableColumnVector.throwUnsupportedException()` was corrected, as it previously encouraged users to increase rather than reduce the batch size.

## How was this patch tested?

New units tests were added.

Author: Ala Luszczak <ala@databricks.com>

Closes #21206 from ala/overflow-reserve.
2018-05-02 12:43:19 -07:00
Marco Gaido 8dbf56c055 [SPARK-24013][SQL] Remove unneeded compress in ApproximatePercentile
## What changes were proposed in this pull request?

`ApproximatePercentile` contains a workaround logic to compress the samples since at the beginning `QuantileSummaries` was ignoring the compression threshold. This problem was fixed in SPARK-17439, but the workaround logic was not removed. So we are compressing the samples many more times than needed: this could lead to critical performance degradation.

This can create serious performance issues in queries like:
```
select approx_percentile(id, array(0.1)) from range(10000000)
```

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21133 from mgaido91/SPARK-24013.
2018-05-02 11:58:55 -07:00
Wenchen Fan b42ad165bb [SPARK-24072][SQL] clearly define pushed filters
## What changes were proposed in this pull request?

filters like parquet row group filter, which is actually pushed to the data source but still to be evaluated by Spark, should also count as `pushedFilters`.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21143 from cloud-fan/step1.
2018-04-30 09:13:32 -07:00
Maxim Gekk 3121b411f7 [SPARK-23846][SQL] The samplingRatio option for CSV datasource
## What changes were proposed in this pull request?

I propose to support the `samplingRatio` option for schema inferring of CSV datasource similar to the same option of JSON datasource:
b14993e1fc/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala (L49-L50)

## How was this patch tested?

Added 2 tests for json and 2 tests for csv datasources. The tests checks that only subset of input dataset is used for schema inferring.

Author: Maxim Gekk <maxim.gekk@databricks.com>
Author: Maxim Gekk <max.gekk@gmail.com>

Closes #20959 from MaxGekk/csv-sampling.
2018-04-30 09:45:22 +08:00
Maxim Gekk bd14da6fd5 [SPARK-23094][SPARK-23723][SPARK-23724][SQL] Support custom encoding for json files
## What changes were proposed in this pull request?

I propose new option for JSON datasource which allows to specify encoding (charset) of input and output files. Here is an example of using of the option:

```
spark.read.schema(schema)
  .option("multiline", "true")
  .option("encoding", "UTF-16LE")
  .json(fileName)
```

If the option is not specified, charset auto-detection mechanism is used by default.

The option can be used for saving datasets to jsons. Currently Spark is able to save datasets into json files in `UTF-8` charset only. The changes allow to save data in any supported charset. Here is the approximate list of supported charsets by Oracle Java SE: https://docs.oracle.com/javase/8/docs/technotes/guides/intl/encoding.doc.html . An user can specify the charset of output jsons via the charset option like `.option("charset", "UTF-16BE")`. By default the output charset is still `UTF-8` to keep backward compatibility.

The solution has the following restrictions for per-line mode (`multiline = false`):

- If charset is different from UTF-8, the lineSep option must be specified. The option required because Hadoop LineReader cannot detect the line separator correctly. Here is the ticket for solving the issue: https://issues.apache.org/jira/browse/SPARK-23725

- Encoding with [BOM](https://en.wikipedia.org/wiki/Byte_order_mark) are not supported. For example, the `UTF-16` and `UTF-32` encodings are blacklisted. The problem can be solved by https://github.com/MaxGekk/spark-1/pull/2

## How was this patch tested?

I added the following tests:
- reads an json file in `UTF-16LE` encoding with BOM in `multiline` mode
- read json file by using charset auto detection (`UTF-32BE` with BOM)
- read json file using of user's charset (`UTF-16LE`)
- saving in `UTF-32BE` and read the result by standard library (not by Spark)
- checking that default charset is `UTF-8`
- handling wrong (unsupported) charset

Author: Maxim Gekk <maxim.gekk@databricks.com>
Author: Maxim Gekk <max.gekk@gmail.com>

Closes #20937 from MaxGekk/json-encoding-line-sep.
2018-04-29 11:25:31 +08:00
Yuming Wang 4df51361a5 [SPARK-22732][SS][FOLLOW-UP] Fix MemorySinkV2 toString error
## What changes were proposed in this pull request?

Fix `MemorySinkV2` toString() error

## How was this patch tested?

N/A

Author: Yuming Wang <yumwang@ebay.com>

Closes #21170 from wangyum/SPARK-22732.
2018-04-28 16:57:41 +08:00
Marco Gaido ad94e8592b [SPARK-23736][SQL][FOLLOWUP] Error message should contains SQL types
## What changes were proposed in this pull request?

In the error messages we should return the SQL types (like `string` rather than the internal types like `StringType`).

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21181 from mgaido91/SPARK-23736_followup.
2018-04-28 10:47:43 +08:00
Jungtaek Lim 1fb46f30f8 [SPARK-23688][SS] Refactor tests away from rate source
## What changes were proposed in this pull request?

Replace rate source with memory source in continuous mode test suite. Keep using "rate" source if the tests intend to put data periodically in background, or need to put short source name to load, since "memory" doesn't have provider for source.

## How was this patch tested?

Ran relevant test suite from IDE.

Author: Jungtaek Lim <kabhwan@gmail.com>

Closes #21152 from HeartSaVioR/SPARK-23688.
2018-04-28 09:55:56 +08:00
Juliusz Sompolski 8614edd445 [SPARK-24104] SQLAppStatusListener overwrites metrics onDriverAccumUpdates instead of updating them
## What changes were proposed in this pull request?

Event `SparkListenerDriverAccumUpdates` may happen multiple times in a query - e.g. every `FileSourceScanExec` and `BroadcastExchangeExec` call `postDriverMetricUpdates`.
In Spark 2.2 `SQLListener` updated the map with new values. `SQLAppStatusListener` overwrites it.
Unless `update` preserved it in the KV store (dependant on `exec.lastWriteTime`), only the metrics from the last operator that does `postDriverMetricUpdates` are preserved.

## How was this patch tested?

Unit test added.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #21171 from juliuszsompolski/SPARK-24104.
2018-04-27 14:14:28 -07:00
Dilip Biswal 3fd297af6d [SPARK-24085][SQL] Query returns UnsupportedOperationException when scalar subquery is present in partitioning expression
## What changes were proposed in this pull request?
In this case, the partition pruning happens before the planning phase of scalar subquery expressions.
For scalar subquery expressions, the planning occurs late in the cycle (after the physical planning)  in "PlanSubqueries" just before execution. Currently we try to execute the scalar subquery expression as part of partition pruning and fail as it implements Unevaluable.

The fix attempts to ignore the Subquery expressions from partition pruning computation. Another option can be to somehow plan the subqueries before the partition pruning. Since this may not be a commonly occuring expression, i am opting for a simpler fix.

Repro
``` SQL
CREATE TABLE test_prc_bug (
id_value string
)
partitioned by (id_type string)
location '/tmp/test_prc_bug'
stored as parquet;

insert into test_prc_bug values ('1','a');
insert into test_prc_bug values ('2','a');
insert into test_prc_bug values ('3','b');
insert into test_prc_bug values ('4','b');

select * from test_prc_bug
where id_type = (select 'b');
```
## How was this patch tested?
Added test in SubquerySuite and hive/SQLQuerySuite

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

Closes #21174 from dilipbiswal/spark-24085.
2018-04-27 11:43:29 -07:00
Patrick McGloin 2824f12b8b [SPARK-23565][SS] New error message for structured streaming sources assertion
## What changes were proposed in this pull request?

A more informative message to tell you why a structured streaming query cannot continue if you have added more sources, than there are in the existing checkpoint offsets.

## How was this patch tested?

I added a Unit Test.

Author: Patrick McGloin <mcgloin.patrick@gmail.com>

Closes #20946 from patrickmcgloin/master.
2018-04-27 23:04:14 +08:00
Tathagata Das d1eb8d3ddc [SPARK-24094][SS][MINOR] Change description strings of v2 streaming sources to reflect the change
## What changes were proposed in this pull request?

This makes it easy to understand at runtime which version is running. Great for debugging production issues.

## How was this patch tested?
Not necessary.

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

Closes #21160 from tdas/SPARK-24094.
2018-04-25 23:24:05 -07:00
Marco Gaido cd10f9df82 [SPARK-23916][SQL] Add array_join function
## What changes were proposed in this pull request?

The PR adds the SQL function `array_join`. The behavior of the function is based on Presto's one.

The function accepts an `array` of `string` which is to be joined, a `string` which is the delimiter to use between the items of the first argument and optionally a `string` which is used to replace `null` values.

## How was this patch tested?

added UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21011 from mgaido91/SPARK-23916.
2018-04-26 13:37:13 +09:00
Marco Gaido 58c55cb4a6 [SPARK-23902][SQL] Add roundOff flag to months_between
## What changes were proposed in this pull request?

HIVE-15511 introduced the `roundOff` flag in order to disable the rounding to 8 digits which is performed in `months_between`. Since this can be a computational intensive operation, skipping it may improve performances when the rounding is not needed.

## How was this patch tested?

modified existing UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21008 from mgaido91/SPARK-23902.
2018-04-26 12:19:20 +09:00
Maxim Gekk 3f1e999d3d [SPARK-23849][SQL] Tests for samplingRatio of json datasource
## What changes were proposed in this pull request?

Added the `samplingRatio` option to the `json()` method of PySpark DataFrame Reader. Improving existing tests for Scala API according to review of the PR: https://github.com/apache/spark/pull/20959

## How was this patch tested?

Added new test for PySpark, updated 2 existing tests according to reviews of https://github.com/apache/spark/pull/20959 and added new negative test

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #21056 from MaxGekk/json-sampling.
2018-04-26 09:14:24 +08:00
Tathagata Das 396938ef02 [SPARK-24050][SS] Calculate input / processing rates correctly for DataSourceV2 streaming sources
## What changes were proposed in this pull request?

In some streaming queries, the input and processing rates are not calculated at all (shows up as zero) because MicroBatchExecution fails to associated metrics from the executed plan of a trigger with the sources in the logical plan of the trigger. The way this executed-plan-leaf-to-logical-source attribution works is as follows. With V1 sources, there was no way to identify which execution plan leaves were generated by a streaming source. So did a best-effort attempt to match logical and execution plan leaves when the number of leaves were same. In cases where the number of leaves is different, we just give up and report zero rates. An example where this may happen is as follows.

```
val cachedStaticDF = someStaticDF.union(anotherStaticDF).cache()
val streamingInputDF = ...

val query = streamingInputDF.join(cachedStaticDF).writeStream....
```
In this case, the `cachedStaticDF` has multiple logical leaves, but in the trigger's execution plan it only has leaf because a cached subplan is represented as a single InMemoryTableScanExec leaf. This leads to a mismatch in the number of leaves causing the input rates to be computed as zero.

With DataSourceV2, all inputs are represented in the executed plan using `DataSourceV2ScanExec`, each of which has a reference to the associated logical `DataSource` and `DataSourceReader`. So its easy to associate the metrics to the original streaming sources.

In this PR, the solution is as follows. If all the streaming sources in a streaming query as v2 sources, then use a new code path where the execution-metrics-to-source mapping is done directly. Otherwise we fall back to existing mapping logic.

## How was this patch tested?
- New unit tests using V2 memory source
- Existing unit tests using V1 source

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

Closes #21126 from tdas/SPARK-24050.
2018-04-25 12:21:55 -07:00
Takeshi Yamamuro 20ca208bcd [SPARK-23880][SQL] Do not trigger any jobs for caching data
## What changes were proposed in this pull request?
This pr fixed code so that `cache` could prevent any jobs from being triggered.
For example, in the current master, an operation below triggers a actual job;
```
val df = spark.range(10000000000L)
  .filter('id > 1000)
  .orderBy('id.desc)
  .cache()
```
This triggers a job while the cache should be lazy. The problem is that, when creating `InMemoryRelation`, we build the RDD, which calls `SparkPlan.execute` and may trigger jobs, like sampling job for range partitioner, or broadcast job.

This pr removed the code to build a cached `RDD` in the constructor of `InMemoryRelation` and added `CachedRDDBuilder` to lazily build the `RDD` in `InMemoryRelation`. Then, the first call of `CachedRDDBuilder.cachedColumnBuffers` triggers a job to materialize the cache in  `InMemoryTableScanExec` .

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21018 from maropu/SPARK-23880.
2018-04-25 19:06:18 +08:00
liutang123 64e8408e6f [SPARK-24012][SQL] Union of map and other compatible column
## What changes were proposed in this pull request?
Union of map and other compatible column result in unresolved operator 'Union; exception

Reproduction
`spark-sql>select map(1,2), 'str' union all select map(1,2,3,null), 1`
Output:
```
Error in query: unresolved operator 'Union;;
'Union
:- Project [map(1, 2) AS map(1, 2)#106, str AS str#107]
:  +- OneRowRelation$
+- Project [map(1, cast(2 as int), 3, cast(null as int)) AS map(1, CAST(2 AS INT), 3, CAST(NULL AS INT))#109, 1 AS 1#108]
   +- OneRowRelation$
```
So, we should cast part of columns to be compatible when appropriate.

## How was this patch tested?
Added a test (query union of map and other columns) to SQLQueryTestSuite's union.sql.

Author: liutang123 <liutang123@yeah.net>

Closes #21100 from liutang123/SPARK-24012.
2018-04-25 18:10:51 +08:00
mn-mikke 5fea17b3be [SPARK-23821][SQL] Collection function: flatten
## What changes were proposed in this pull request?

This PR adds a new collection function that transforms an array of arrays into a single array. The PR comprises:
- An expression for flattening array structure
- Flatten function
- A wrapper for PySpark

## How was this patch tested?

New tests added into:
- CollectionExpressionsSuite
- DataFrameFunctionsSuite

## Codegen examples
### Primitive type
```
val df = Seq(
  Seq(Seq(1, 2), Seq(4, 5)),
  Seq(null, Seq(1))
).toDF("i")
df.filter($"i".isNotNull || $"i".isNull).select(flatten($"i")).debugCodegen
```
Result:
```
/* 033 */         boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 034 */         ArrayData inputadapter_value = inputadapter_isNull ?
/* 035 */         null : (inputadapter_row.getArray(0));
/* 036 */
/* 037 */         boolean filter_value = true;
/* 038 */
/* 039 */         if (!(!inputadapter_isNull)) {
/* 040 */           filter_value = inputadapter_isNull;
/* 041 */         }
/* 042 */         if (!filter_value) continue;
/* 043 */
/* 044 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 045 */
/* 046 */         boolean project_isNull = inputadapter_isNull;
/* 047 */         ArrayData project_value = null;
/* 048 */
/* 049 */         if (!inputadapter_isNull) {
/* 050 */           for (int z = 0; !project_isNull && z < inputadapter_value.numElements(); z++) {
/* 051 */             project_isNull |= inputadapter_value.isNullAt(z);
/* 052 */           }
/* 053 */           if (!project_isNull) {
/* 054 */             long project_numElements = 0;
/* 055 */             for (int z = 0; z < inputadapter_value.numElements(); z++) {
/* 056 */               project_numElements += inputadapter_value.getArray(z).numElements();
/* 057 */             }
/* 058 */             if (project_numElements > 2147483632) {
/* 059 */               throw new RuntimeException("Unsuccessful try to flatten an array of arrays with " +
/* 060 */                 project_numElements + " elements due to exceeding the array size limit 2147483632.");
/* 061 */             }
/* 062 */
/* 063 */             long project_size = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(
/* 064 */               project_numElements,
/* 065 */               4);
/* 066 */             if (project_size > 2147483632) {
/* 067 */               throw new RuntimeException("Unsuccessful try to flatten an array of arrays with " +
/* 068 */                 project_size + " bytes of data due to exceeding the limit 2147483632" +
/* 069 */                 " bytes for UnsafeArrayData.");
/* 070 */             }
/* 071 */
/* 072 */             byte[] project_array = new byte[(int)project_size];
/* 073 */             UnsafeArrayData project_tempArrayData = new UnsafeArrayData();
/* 074 */             Platform.putLong(project_array, 16, project_numElements);
/* 075 */             project_tempArrayData.pointTo(project_array, 16, (int)project_size);
/* 076 */             int project_counter = 0;
/* 077 */             for (int k = 0; k < inputadapter_value.numElements(); k++) {
/* 078 */               ArrayData arr = inputadapter_value.getArray(k);
/* 079 */               for (int l = 0; l < arr.numElements(); l++) {
/* 080 */                 if (arr.isNullAt(l)) {
/* 081 */                   project_tempArrayData.setNullAt(project_counter);
/* 082 */                 } else {
/* 083 */                   project_tempArrayData.setInt(
/* 084 */                     project_counter,
/* 085 */                     arr.getInt(l)
/* 086 */                   );
/* 087 */                 }
/* 088 */                 project_counter++;
/* 089 */               }
/* 090 */             }
/* 091 */             project_value = project_tempArrayData;
/* 092 */
/* 093 */           }
/* 094 */
/* 095 */         }
```
### Non-primitive type
```
val df = Seq(
  Seq(Seq("a", "b"), Seq(null, "d")),
  Seq(null, Seq("a"))
).toDF("s")
df.filter($"s".isNotNull || $"s".isNull).select(flatten($"s")).debugCodegen
```
Result:
```
/* 033 */         boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 034 */         ArrayData inputadapter_value = inputadapter_isNull ?
/* 035 */         null : (inputadapter_row.getArray(0));
/* 036 */
/* 037 */         boolean filter_value = true;
/* 038 */
/* 039 */         if (!(!inputadapter_isNull)) {
/* 040 */           filter_value = inputadapter_isNull;
/* 041 */         }
/* 042 */         if (!filter_value) continue;
/* 043 */
/* 044 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 045 */
/* 046 */         boolean project_isNull = inputadapter_isNull;
/* 047 */         ArrayData project_value = null;
/* 048 */
/* 049 */         if (!inputadapter_isNull) {
/* 050 */           for (int z = 0; !project_isNull && z < inputadapter_value.numElements(); z++) {
/* 051 */             project_isNull |= inputadapter_value.isNullAt(z);
/* 052 */           }
/* 053 */           if (!project_isNull) {
/* 054 */             long project_numElements = 0;
/* 055 */             for (int z = 0; z < inputadapter_value.numElements(); z++) {
/* 056 */               project_numElements += inputadapter_value.getArray(z).numElements();
/* 057 */             }
/* 058 */             if (project_numElements > 2147483632) {
/* 059 */               throw new RuntimeException("Unsuccessful try to flatten an array of arrays with " +
/* 060 */                 project_numElements + " elements due to exceeding the array size limit 2147483632.");
/* 061 */             }
/* 062 */
/* 063 */             Object[] project_arrayObject = new Object[(int)project_numElements];
/* 064 */             int project_counter = 0;
/* 065 */             for (int k = 0; k < inputadapter_value.numElements(); k++) {
/* 066 */               ArrayData arr = inputadapter_value.getArray(k);
/* 067 */               for (int l = 0; l < arr.numElements(); l++) {
/* 068 */                 project_arrayObject[project_counter] = arr.getUTF8String(l);
/* 069 */                 project_counter++;
/* 070 */               }
/* 071 */             }
/* 072 */             project_value = new org.apache.spark.sql.catalyst.util.GenericArrayData(project_arrayObject);
/* 073 */
/* 074 */           }
/* 075 */
/* 076 */         }
```

Author: mn-mikke <mrkAha12346github>

Closes #20938 from mn-mikke/feature/array-api-flatten-to-master.
2018-04-25 11:19:08 +09:00
Jose Torres d6c26d1c9a [SPARK-24038][SS] Refactor continuous writing to its own class
## What changes were proposed in this pull request?

Refactor continuous writing to its own class.

See WIP https://github.com/jose-torres/spark/pull/13 for the overall direction this is going, but I think this PR is very isolated and necessary anyway.

## How was this patch tested?

existing unit tests - refactoring only

Author: Jose Torres <torres.joseph.f+github@gmail.com>

Closes #21116 from jose-torres/SPARK-24038.
2018-04-24 17:06:03 -07:00
seancxmao c303b1b676 [MINOR][DOCS] Fix comments of SQLExecution#withExecutionId
## What changes were proposed in this pull request?
Fix comment. Change `BroadcastHashJoin.broadcastFuture` to `BroadcastExchangeExec.relationFuture`: d28d5732ae/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala (L66)

## How was this patch tested?
N/A

Author: seancxmao <seancxmao@gmail.com>

Closes #21113 from seancxmao/SPARK-13136.
2018-04-24 16:16:07 +08:00
Tathagata Das 770add81c3 [SPARK-23004][SS] Ensure StateStore.commit is called only once in a streaming aggregation task
## What changes were proposed in this pull request?

A structured streaming query with a streaming aggregation can throw the following error in rare cases. 

```
java.lang.IllegalStateException: Cannot commit after already committed or aborted
	at org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider.org$apache$spark$sql$execution$streaming$state$HDFSBackedStateStoreProvider$$verify(HDFSBackedStateStoreProvider.scala:643)
	at org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider$HDFSBackedStateStore.commit(HDFSBackedStateStoreProvider.scala:135)
	at org.apache.spark.sql.execution.streaming.StateStoreSaveExec$$anonfun$doExecute$3$$anon$2$$anonfun$hasNext$2.apply$mcV$sp(statefulOperators.scala:359)
	at org.apache.spark.sql.execution.streaming.StateStoreWriter$class.timeTakenMs(statefulOperators.scala:102)
	at org.apache.spark.sql.execution.streaming.StateStoreSaveExec.timeTakenMs(statefulOperators.scala:251)
	at org.apache.spark.sql.execution.streaming.StateStoreSaveExec$$anonfun$doExecute$3$$anon$2.hasNext(statefulOperators.scala:359)
	at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.processInputs(ObjectAggregationIterator.scala:188)
	at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.<init>(ObjectAggregationIterator.scala:78)
	at org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec$$anonfun$doExecute$1$$anonfun$2.apply(ObjectHashAggregateExec.scala:114)
	at org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec$$anonfun$doExecute$1$$anonfun$2.apply(ObjectHashAggregateExec.scala:105)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsWithIndexInternal$1$$anonfun$apply$24.apply(RDD.scala:830)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsWithIndexInternal$1$$anonfun$apply$24.apply(RDD.scala:830)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:42)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:336)
```

This can happen when the following conditions are accidentally hit. 
 - Streaming aggregation with aggregation function that is a subset of [`TypedImperativeAggregation`](76b8b840dd/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala (L473)) (for example, `collect_set`, `collect_list`, `percentile`, etc.). 
 - Query running in `update}` mode
 - After the shuffle, a partition has exactly 128 records. 

This causes StateStore.commit to be called twice. See the [JIRA](https://issues.apache.org/jira/browse/SPARK-23004) for a more detailed explanation. The solution is to use `NextIterator` or `CompletionIterator`, each of which has a flag to prevent the "onCompletion" task from being called more than once. In this PR, I chose to implement using `NextIterator`.

## How was this patch tested?

Added unit test that I have confirm will fail without the fix.

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

Closes #21124 from tdas/SPARK-23004.
2018-04-23 13:20:32 -07:00
Wenchen Fan f70f46d1e5 [SPARK-23877][SQL][FOLLOWUP] use PhysicalOperation to simplify the handling of Project and Filter over partitioned relation
## What changes were proposed in this pull request?

A followup of https://github.com/apache/spark/pull/20988

`PhysicalOperation` can collect Project and Filters over a certain plan and substitute the alias with the original attributes in the bottom plan. We can use it in `OptimizeMetadataOnlyQuery` rule to handle the Project and Filter over partitioned relation.

## How was this patch tested?

existing test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21111 from cloud-fan/refactor.
2018-04-23 20:18:50 +08:00
Mykhailo Shtelma c48085aa91 [SPARK-23799][SQL] FilterEstimation.evaluateInSet produces devision by zero in a case of empty table with analyzed statistics
>What changes were proposed in this pull request?

During evaluation of IN conditions, if the source data frame, is represented by a plan, that uses hive table with columns, which were previously analysed, and the plan has conditions for these fields, that cannot be satisfied (which leads us to an empty data frame), FilterEstimation.evaluateInSet method produces NumberFormatException and ClassCastException.
In order to fix this bug, method FilterEstimation.evaluateInSet at first checks, if distinct count is not zero, and also checks if colStat.min and colStat.max  are defined, and only in this case proceeds with the calculation. If at least one of the conditions is not satisfied, zero is returned.

>How was this patch tested?

In order to test the PR two tests were implemented: one in FilterEstimationSuite, that tests the plan with the statistics that violates the conditions mentioned above,  and another one in StatisticsCollectionSuite, that test the whole process of analysis/optimisation of the query, that leads to the problems, mentioned in the first section.

Author: Mykhailo Shtelma <mykhailo.shtelma@bearingpoint.com>
Author: smikesh <mshtelma@gmail.com>

Closes #21052 from mshtelma/filter_estimation_evaluateInSet_Bugs.
2018-04-21 23:33:57 -07:00
gatorsmile 7bc853d089 [SPARK-24033][SQL] Fix Mismatched of Window Frame specifiedwindowframe(RowFrame, -1, -1)
## What changes were proposed in this pull request?

When the OffsetWindowFunction's frame is `UnaryMinus(Literal(1))` but the specified window frame has been simplified to `Literal(-1)` by some optimizer rules e.g., `ConstantFolding`. Thus, they do not match and cause the following error:
```
org.apache.spark.sql.AnalysisException: Window Frame specifiedwindowframe(RowFrame, -1, -1) must match the required frame specifiedwindowframe(RowFrame, -1, -1);
at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.failAnalysis(CheckAnalysis.scala:41)
at org.apache.spark.sql.catalyst.analysis.Analyzer.failAnalysis(Analyzer.scala:91)
at
```
## How was this patch tested?
Added a test

Author: gatorsmile <gatorsmile@gmail.com>

Closes #21115 from gatorsmile/fixLag.
2018-04-21 10:45:12 -07:00
Marcelo Vanzin 1d758dc73b Revert "[SPARK-23775][TEST] Make DataFrameRangeSuite not flaky"
This reverts commit 0c94e48bc5.
2018-04-20 10:23:01 -07:00
mn-mikke e6b466084c [SPARK-23736][SQL] Extending the concat function to support array columns
## What changes were proposed in this pull request?
The PR adds a logic for easy concatenation of multiple array columns and covers:
- Concat expression has been extended to support array columns
- A Python wrapper

## How was this patch tested?
New tests added into:
- CollectionExpressionsSuite
- DataFrameFunctionsSuite
- typeCoercion/native/concat.sql

## Codegen examples
### Primitive-type elements
```
val df = Seq(
  (Seq(1 ,2), Seq(3, 4)),
  (Seq(1, 2, 3), null)
).toDF("a", "b")
df.filter('a.isNotNull).select(concat('a, 'b)).debugCodegen()
```
Result:
```
/* 033 */         boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 034 */         ArrayData inputadapter_value = inputadapter_isNull ?
/* 035 */         null : (inputadapter_row.getArray(0));
/* 036 */
/* 037 */         if (!(!inputadapter_isNull)) continue;
/* 038 */
/* 039 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 040 */
/* 041 */         ArrayData[] project_args = new ArrayData[2];
/* 042 */
/* 043 */         if (!false) {
/* 044 */           project_args[0] = inputadapter_value;
/* 045 */         }
/* 046 */
/* 047 */         boolean inputadapter_isNull1 = inputadapter_row.isNullAt(1);
/* 048 */         ArrayData inputadapter_value1 = inputadapter_isNull1 ?
/* 049 */         null : (inputadapter_row.getArray(1));
/* 050 */         if (!inputadapter_isNull1) {
/* 051 */           project_args[1] = inputadapter_value1;
/* 052 */         }
/* 053 */
/* 054 */         ArrayData project_value = new Object() {
/* 055 */           public ArrayData concat(ArrayData[] args) {
/* 056 */             for (int z = 0; z < 2; z++) {
/* 057 */               if (args[z] == null) return null;
/* 058 */             }
/* 059 */
/* 060 */             long project_numElements = 0L;
/* 061 */             for (int z = 0; z < 2; z++) {
/* 062 */               project_numElements += args[z].numElements();
/* 063 */             }
/* 064 */             if (project_numElements > 2147483632) {
/* 065 */               throw new RuntimeException("Unsuccessful try to concat arrays with " + project_numElements +
/* 066 */                 " elements due to exceeding the array size limit 2147483632.");
/* 067 */             }
/* 068 */
/* 069 */             long project_size = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(
/* 070 */               project_numElements,
/* 071 */               4);
/* 072 */             if (project_size > 2147483632) {
/* 073 */               throw new RuntimeException("Unsuccessful try to concat arrays with " + project_size +
/* 074 */                 " bytes of data due to exceeding the limit 2147483632 bytes" +
/* 075 */                 " for UnsafeArrayData.");
/* 076 */             }
/* 077 */
/* 078 */             byte[] project_array = new byte[(int)project_size];
/* 079 */             UnsafeArrayData project_arrayData = new UnsafeArrayData();
/* 080 */             Platform.putLong(project_array, 16, project_numElements);
/* 081 */             project_arrayData.pointTo(project_array, 16, (int)project_size);
/* 082 */             int project_counter = 0;
/* 083 */             for (int y = 0; y < 2; y++) {
/* 084 */               for (int z = 0; z < args[y].numElements(); z++) {
/* 085 */                 if (args[y].isNullAt(z)) {
/* 086 */                   project_arrayData.setNullAt(project_counter);
/* 087 */                 } else {
/* 088 */                   project_arrayData.setInt(
/* 089 */                     project_counter,
/* 090 */                     args[y].getInt(z)
/* 091 */                   );
/* 092 */                 }
/* 093 */                 project_counter++;
/* 094 */               }
/* 095 */             }
/* 096 */             return project_arrayData;
/* 097 */           }
/* 098 */         }.concat(project_args);
/* 099 */         boolean project_isNull = project_value == null;
```

### Non-primitive-type elements
```
val df = Seq(
  (Seq("aa" ,"bb"), Seq("ccc", "ddd")),
  (Seq("x", "y"), null)
).toDF("a", "b")
df.filter('a.isNotNull).select(concat('a, 'b)).debugCodegen()
```
Result:
```
/* 033 */         boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 034 */         ArrayData inputadapter_value = inputadapter_isNull ?
/* 035 */         null : (inputadapter_row.getArray(0));
/* 036 */
/* 037 */         if (!(!inputadapter_isNull)) continue;
/* 038 */
/* 039 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 040 */
/* 041 */         ArrayData[] project_args = new ArrayData[2];
/* 042 */
/* 043 */         if (!false) {
/* 044 */           project_args[0] = inputadapter_value;
/* 045 */         }
/* 046 */
/* 047 */         boolean inputadapter_isNull1 = inputadapter_row.isNullAt(1);
/* 048 */         ArrayData inputadapter_value1 = inputadapter_isNull1 ?
/* 049 */         null : (inputadapter_row.getArray(1));
/* 050 */         if (!inputadapter_isNull1) {
/* 051 */           project_args[1] = inputadapter_value1;
/* 052 */         }
/* 053 */
/* 054 */         ArrayData project_value = new Object() {
/* 055 */           public ArrayData concat(ArrayData[] args) {
/* 056 */             for (int z = 0; z < 2; z++) {
/* 057 */               if (args[z] == null) return null;
/* 058 */             }
/* 059 */
/* 060 */             long project_numElements = 0L;
/* 061 */             for (int z = 0; z < 2; z++) {
/* 062 */               project_numElements += args[z].numElements();
/* 063 */             }
/* 064 */             if (project_numElements > 2147483632) {
/* 065 */               throw new RuntimeException("Unsuccessful try to concat arrays with " + project_numElements +
/* 066 */                 " elements due to exceeding the array size limit 2147483632.");
/* 067 */             }
/* 068 */
/* 069 */             Object[] project_arrayObjects = new Object[(int)project_numElements];
/* 070 */             int project_counter = 0;
/* 071 */             for (int y = 0; y < 2; y++) {
/* 072 */               for (int z = 0; z < args[y].numElements(); z++) {
/* 073 */                 project_arrayObjects[project_counter] = args[y].getUTF8String(z);
/* 074 */                 project_counter++;
/* 075 */               }
/* 076 */             }
/* 077 */             return new org.apache.spark.sql.catalyst.util.GenericArrayData(project_arrayObjects);
/* 078 */           }
/* 079 */         }.concat(project_args);
/* 080 */         boolean project_isNull = project_value == null;
```

Author: mn-mikke <mrkAha12346github>

Closes #20858 from mn-mikke/feature/array-api-concat_arrays-to-master.
2018-04-20 14:58:11 +09:00
Ryan Blue b3fde5a41e [SPARK-23877][SQL] Use filter predicates to prune partitions in metadata-only queries
## What changes were proposed in this pull request?

This updates the OptimizeMetadataOnlyQuery rule to use filter expressions when listing partitions, if there are filter nodes in the logical plan. This avoids listing all partitions for large tables on the driver.

This also fixes a minor bug where the partitions returned from fsRelation cannot be serialized without hitting a stack level too deep error. This is caused by serializing a stream to executors, where the stream is a recursive structure. If the stream is too long, the serialization stack reaches the maximum level of depth. The fix is to create a LocalRelation using an Array instead of the incoming Seq.

## How was this patch tested?

Existing tests for metadata-only queries.

Author: Ryan Blue <blue@apache.org>

Closes #20988 from rdblue/SPARK-23877-metadata-only-push-filters.
2018-04-20 12:06:41 +08:00
“attilapiros” 9ea8d3d31b [SPARK-22362][SQL] Add unit test for Window Aggregate Functions
## What changes were proposed in this pull request?

Improving the test coverage of window functions focusing on missing test for window aggregate functions. No new UDAF test is added as it has been tested already.

## How was this patch tested?

Only new tests were added, automated tests were executed.

Author: “attilapiros” <piros.attila.zsolt@gmail.com>
Author: Attila Zsolt Piros <2017933+attilapiros@users.noreply.github.com>

Closes #20046 from attilapiros/SPARK-22362.
2018-04-19 18:55:59 +02:00
Wenchen Fan 6e19f7683f [SPARK-23989][SQL] exchange should copy data before non-serialized shuffle
## What changes were proposed in this pull request?

In Spark SQL, we usually reuse the `UnsafeRow` instance and need to copy the data when a place buffers non-serialized objects.

Shuffle may buffer objects if we don't make it to the bypass merge shuffle or unsafe shuffle.

`ShuffleExchangeExec.needToCopyObjectsBeforeShuffle` misses the case that, if `spark.sql.shuffle.partitions` is large enough, we could fail to run unsafe shuffle and go with the non-serialized shuffle.

This bug is very hard to hit since users wouldn't set such a large number of partitions(16 million) for Spark SQL exchange.

TODO: test

## How was this patch tested?

todo.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21101 from cloud-fan/shuffle.
2018-04-19 17:54:53 +02:00
Kazuaki Ishizaki 46bb2b5129 [SPARK-23924][SQL] Add element_at function
## What changes were proposed in this pull request?

The PR adds the SQL function `element_at`. The behavior of the function is based on Presto's one.

This function returns element of array at given index in value if column is array, or returns value for the given key in value if column is map.

## How was this patch tested?

Added UTs

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

Closes #21053 from kiszk/SPARK-23924.
2018-04-19 21:00:10 +09:00
Kazuaki Ishizaki d5bec48b9c [SPARK-23919][SQL] Add array_position function
## What changes were proposed in this pull request?

The PR adds the SQL function `array_position`. The behavior of the function is based on Presto's one.

The function returns the position of the first occurrence of the element in array x (or 0 if not found) using 1-based index as BigInt.

## How was this patch tested?

Added UTs

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

Closes #21037 from kiszk/SPARK-23919.
2018-04-19 11:59:17 +09:00
Gabor Somogyi 0c94e48bc5 [SPARK-23775][TEST] Make DataFrameRangeSuite not flaky
## What changes were proposed in this pull request?

DataFrameRangeSuite.test("Cancelling stage in a query with Range.") stays sometimes in an infinite loop and times out the build.

There were multiple issues with the test:

1. The first valid stageId is zero when the test started alone and not in a suite and the following code waits until timeout:

```
eventually(timeout(10.seconds), interval(1.millis)) {
  assert(DataFrameRangeSuite.stageToKill > 0)
}
```

2. The `DataFrameRangeSuite.stageToKill` was overwritten by the task's thread after the reset which ended up in canceling the same stage 2 times. This caused the infinite wait.

This PR solves this mentioned flakyness by removing the shared `DataFrameRangeSuite.stageToKill` and using `wait` and `CountDownLatch` for synhronization.

## How was this patch tested?

Existing unit test.

Author: Gabor Somogyi <gabor.g.somogyi@gmail.com>

Closes #20888 from gaborgsomogyi/SPARK-23775.
2018-04-18 16:37:41 -07:00
mn-mikke f81fa478ff [SPARK-23926][SQL] Extending reverse function to support ArrayType arguments
## What changes were proposed in this pull request?

This PR extends `reverse` functions to be able to operate over array columns and covers:
- Introduction of `Reverse` expression that represents logic for reversing arrays and also strings
- Removal of `StringReverse` expression
- A wrapper for PySpark

## How was this patch tested?

New tests added into:
- CollectionExpressionsSuite
- DataFrameFunctionsSuite

## Codegen examples
### Primitive type
```
val df = Seq(
  Seq(1, 3, 4, 2),
  null
).toDF("i")
df.filter($"i".isNotNull || $"i".isNull).select(reverse($"i")).debugCodegen
```
Result:
```
/* 032 */         boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 033 */         ArrayData inputadapter_value = inputadapter_isNull ?
/* 034 */         null : (inputadapter_row.getArray(0));
/* 035 */
/* 036 */         boolean filter_value = true;
/* 037 */
/* 038 */         if (!(!inputadapter_isNull)) {
/* 039 */           filter_value = inputadapter_isNull;
/* 040 */         }
/* 041 */         if (!filter_value) continue;
/* 042 */
/* 043 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 044 */
/* 045 */         boolean project_isNull = inputadapter_isNull;
/* 046 */         ArrayData project_value = null;
/* 047 */
/* 048 */         if (!inputadapter_isNull) {
/* 049 */           final int project_length = inputadapter_value.numElements();
/* 050 */           project_value = inputadapter_value.copy();
/* 051 */           for(int k = 0; k < project_length / 2; k++) {
/* 052 */             int l = project_length - k - 1;
/* 053 */             boolean isNullAtK = project_value.isNullAt(k);
/* 054 */             boolean isNullAtL = project_value.isNullAt(l);
/* 055 */             if(!isNullAtK) {
/* 056 */               int el = project_value.getInt(k);
/* 057 */               if(!isNullAtL) {
/* 058 */                 project_value.setInt(k, project_value.getInt(l));
/* 059 */               } else {
/* 060 */                 project_value.setNullAt(k);
/* 061 */               }
/* 062 */               project_value.setInt(l, el);
/* 063 */             } else if (!isNullAtL) {
/* 064 */               project_value.setInt(k, project_value.getInt(l));
/* 065 */               project_value.setNullAt(l);
/* 066 */             }
/* 067 */           }
/* 068 */
/* 069 */         }
```
### Non-primitive type
```
val df = Seq(
  Seq("a", "c", "d", "b"),
  null
).toDF("s")
df.filter($"s".isNotNull || $"s".isNull).select(reverse($"s")).debugCodegen
```
Result:
```
/* 032 */         boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 033 */         ArrayData inputadapter_value = inputadapter_isNull ?
/* 034 */         null : (inputadapter_row.getArray(0));
/* 035 */
/* 036 */         boolean filter_value = true;
/* 037 */
/* 038 */         if (!(!inputadapter_isNull)) {
/* 039 */           filter_value = inputadapter_isNull;
/* 040 */         }
/* 041 */         if (!filter_value) continue;
/* 042 */
/* 043 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 044 */
/* 045 */         boolean project_isNull = inputadapter_isNull;
/* 046 */         ArrayData project_value = null;
/* 047 */
/* 048 */         if (!inputadapter_isNull) {
/* 049 */           final int project_length = inputadapter_value.numElements();
/* 050 */           project_value = new org.apache.spark.sql.catalyst.util.GenericArrayData(new Object[project_length]);
/* 051 */           for(int k = 0; k < project_length; k++) {
/* 052 */             int l = project_length - k - 1;
/* 053 */             project_value.update(k, inputadapter_value.getUTF8String(l));
/* 054 */           }
/* 055 */
/* 056 */         }
```

Author: mn-mikke <mrkAha12346github>

Closes #21034 from mn-mikke/feature/array-api-reverse-to-master.
2018-04-18 18:41:55 +09:00
gatorsmile cce469435d [SPARK-24002][SQL] Task not serializable caused by org.apache.parquet.io.api.Binary$ByteBufferBackedBinary.getBytes
## What changes were proposed in this pull request?
```
Py4JJavaError: An error occurred while calling o153.sql.
: org.apache.spark.SparkException: Job aborted.
	at org.apache.spark.sql.execution.datasources.FileFormatWriter$.write(FileFormatWriter.scala:223)
	at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.run(InsertIntoHadoopFsRelationCommand.scala:189)
	at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:70)
	at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:68)
	at org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:79)
	at org.apache.spark.sql.Dataset$$anonfun$6.apply(Dataset.scala:190)
	at org.apache.spark.sql.Dataset$$anonfun$6.apply(Dataset.scala:190)
	at org.apache.spark.sql.Dataset$$anonfun$59.apply(Dataset.scala:3021)
	at org.apache.spark.sql.execution.SQLExecution$.withCustomExecutionEnv(SQLExecution.scala:89)
	at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:127)
	at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3020)
	at org.apache.spark.sql.Dataset.<init>(Dataset.scala:190)
	at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:74)
	at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:646)
	at sun.reflect.GeneratedMethodAccessor153.invoke(Unknown Source)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)
	at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:380)
	at py4j.Gateway.invoke(Gateway.java:293)
	at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
	at py4j.commands.CallCommand.execute(CallCommand.java:79)
	at py4j.GatewayConnection.run(GatewayConnection.java:226)
	at java.lang.Thread.run(Thread.java:748)
Caused by: org.apache.spark.SparkException: Exception thrown in Future.get:
	at org.apache.spark.sql.execution.exchange.BroadcastExchangeExec.doExecuteBroadcast(BroadcastExchangeExec.scala:190)
	at org.apache.spark.sql.execution.InputAdapter.doExecuteBroadcast(WholeStageCodegenExec.scala:267)
	at org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec.doConsume(BroadcastNestedLoopJoinExec.scala:530)
	at org.apache.spark.sql.execution.CodegenSupport$class.consume(WholeStageCodegenExec.scala:155)
	at org.apache.spark.sql.execution.ProjectExec.consume(basicPhysicalOperators.scala:37)
	at org.apache.spark.sql.execution.ProjectExec.doConsume(basicPhysicalOperators.scala:69)
	at org.apache.spark.sql.execution.CodegenSupport$class.consume(WholeStageCodegenExec.scala:155)
	at org.apache.spark.sql.execution.FilterExec.consume(basicPhysicalOperators.scala:144)
	...
	at org.apache.spark.sql.execution.datasources.FileFormatWriter$.write(FileFormatWriter.scala:190)
	... 23 more
Caused by: java.util.concurrent.ExecutionException: org.apache.spark.SparkException: Task not serializable
	at java.util.concurrent.FutureTask.report(FutureTask.java:122)
	at java.util.concurrent.FutureTask.get(FutureTask.java:206)
	at org.apache.spark.sql.execution.exchange.BroadcastExchangeExec.doExecuteBroadcast(BroadcastExchangeExec.scala:179)
	... 276 more
Caused by: org.apache.spark.SparkException: Task not serializable
	at org.apache.spark.util.ClosureCleaner$.ensureSerializable(ClosureCleaner.scala:340)
	at org.apache.spark.util.ClosureCleaner$.org$apache$spark$util$ClosureCleaner$$clean(ClosureCleaner.scala:330)
	at org.apache.spark.util.ClosureCleaner$.clean(ClosureCleaner.scala:156)
	at org.apache.spark.SparkContext.clean(SparkContext.scala:2380)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsWithIndex$1.apply(RDD.scala:850)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsWithIndex$1.apply(RDD.scala:849)
	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
	at org.apache.spark.rdd.RDD.withScope(RDD.scala:371)
	at org.apache.spark.rdd.RDD.mapPartitionsWithIndex(RDD.scala:849)
	at org.apache.spark.sql.execution.WholeStageCodegenExec.doExecute(WholeStageCodegenExec.scala:417)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:123)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:118)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$3.apply(SparkPlan.scala:152)
	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
	at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:149)
	at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:118)
	at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.prepareShuffleDependency(ShuffleExchangeExec.scala:89)
	at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec$$anonfun$doExecute$1.apply(ShuffleExchangeExec.scala:125)
	at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec$$anonfun$doExecute$1.apply(ShuffleExchangeExec.scala:116)
	at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:52)
	at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec.doExecute(ShuffleExchangeExec.scala:116)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:123)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:118)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$3.apply(SparkPlan.scala:152)
	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
	at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:149)
	at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:118)
	at org.apache.spark.sql.execution.InputAdapter.inputRDDs(WholeStageCodegenExec.scala:271)
	at org.apache.spark.sql.execution.aggregate.HashAggregateExec.inputRDDs(HashAggregateExec.scala:181)
	at org.apache.spark.sql.execution.WholeStageCodegenExec.doExecute(WholeStageCodegenExec.scala:414)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:123)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:118)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$3.apply(SparkPlan.scala:152)
	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
	at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:149)
	at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:118)
	at org.apache.spark.sql.execution.collect.Collector$.collect(Collector.scala:61)
	at org.apache.spark.sql.execution.collect.Collector$.collect(Collector.scala:70)
	at org.apache.spark.sql.execution.SparkPlan.executeCollectResult(SparkPlan.scala:264)
	at org.apache.spark.sql.execution.exchange.BroadcastExchangeExec$$anon$1$$anonfun$call$1.apply(BroadcastExchangeExec.scala:93)
	at org.apache.spark.sql.execution.exchange.BroadcastExchangeExec$$anon$1$$anonfun$call$1.apply(BroadcastExchangeExec.scala:81)
	at org.apache.spark.sql.execution.SQLExecution$.withExecutionId(SQLExecution.scala:150)
	at org.apache.spark.sql.execution.exchange.BroadcastExchangeExec$$anon$1.call(BroadcastExchangeExec.scala:80)
	at org.apache.spark.sql.execution.exchange.BroadcastExchangeExec$$anon$1.call(BroadcastExchangeExec.scala:76)
	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	... 1 more
Caused by: java.nio.BufferUnderflowException
	at java.nio.HeapByteBuffer.get(HeapByteBuffer.java:151)
	at java.nio.ByteBuffer.get(ByteBuffer.java:715)
	at org.apache.parquet.io.api.Binary$ByteBufferBackedBinary.getBytes(Binary.java:405)
	at org.apache.parquet.io.api.Binary$ByteBufferBackedBinary.getBytesUnsafe(Binary.java:414)
	at org.apache.parquet.io.api.Binary$ByteBufferBackedBinary.writeObject(Binary.java:484)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at java.io.ObjectStreamClass.invokeWriteObject(ObjectStreamClass.java:1128)
	at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1496)
```

The Parquet filters are serializable but not thread safe. SparkPlan.prepare() could be called in different threads (BroadcastExchange will call it in a thread pool). Thus, we could serialize the same Parquet filter at the same time. This is not easily reproduced. The fix is to avoid serializing these Parquet filters in the driver. This PR is to avoid serializing these Parquet filters by moving the parquet filter generation from the driver to executors.

## How was this patch tested?
Having two queries one is a 1000-line SQL query and a 3000-line SQL query. Need to run at least one hour with a heavy write workload to reproduce once.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #21086 from gatorsmile/taskNotSerializable.
2018-04-17 21:03:57 -07:00
Wenchen Fan 310a8cd062 [SPARK-23341][SQL] define some standard options for data source v2
## What changes were proposed in this pull request?

Each data source implementation can define its own options and teach its users how to set them. Spark doesn't have any restrictions about what options a data source should or should not have. It's possible that some options are very common and many data sources use them. However different data sources may define the common options(key and meaning) differently, which is quite confusing to end users.

This PR defines some standard options that data sources can optionally adopt: path, table and database.

## How was this patch tested?

a new test case.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20535 from cloud-fan/options.
2018-04-18 11:51:10 +08:00
Marco Gaido 0a9172a05e [SPARK-23835][SQL] Add not-null check to Tuples' arguments deserialization
## What changes were proposed in this pull request?

There was no check on nullability for arguments of `Tuple`s. This could lead to have weird behavior when a null value had to be deserialized into a non-nullable Scala object: in those cases, the `null` got silently transformed in a valid value (like `-1` for `Int`), corresponding to the default value we are using in the SQL codebase. This situation was very likely to happen when deserializing to a Tuple of primitive Scala types (like Double, Int, ...).

The PR adds the `AssertNotNull` to arguments of tuples which have been asked to be converted to non-nullable types.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20976 from mgaido91/SPARK-23835.
2018-04-17 21:45:20 +08:00
Efim Poberezkin 05ae74778a [SPARK-23747][STRUCTURED STREAMING] Add EpochCoordinator unit tests
## What changes were proposed in this pull request?

Unit tests for EpochCoordinator that test correct sequencing of committed epochs. Several tests are ignored since they test functionality implemented in SPARK-23503 which is not yet merged, otherwise they fail.

Author: Efim Poberezkin <efim@poberezkin.ru>

Closes #20983 from efimpoberezkin/pr/EpochCoordinator-tests.
2018-04-17 04:13:17 -07:00
Jose Torres 1cc66a072b [SPARK-23687][SS] Add a memory source for continuous processing.
## What changes were proposed in this pull request?

Add a memory source for continuous processing.

Note that only one of the ContinuousSuite tests is migrated to minimize the diff here. I'll submit a second PR for SPARK-23688 to change the rest and get rid of waitForRateSourceTriggers.

## How was this patch tested?

unit test

Author: Jose Torres <torres.joseph.f+github@gmail.com>

Closes #20828 from jose-torres/continuousMemory.
2018-04-17 01:59:38 -07:00
Marco Gaido 14844a62c0 [SPARK-23918][SQL] Add array_min function
## What changes were proposed in this pull request?

The PR adds the SQL function `array_min`. It takes an array as argument and returns the minimum value in it.

## How was this patch tested?

added UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21025 from mgaido91/SPARK-23918.
2018-04-17 17:55:35 +09:00
Marco Gaido 6931022031 [SPARK-23917][SQL] Add array_max function
## What changes were proposed in this pull request?

The PR adds the SQL function `array_max`. It takes an array as argument and returns the maximum value in it.

## How was this patch tested?

added UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21024 from mgaido91/SPARK-23917.
2018-04-15 21:45:55 -07:00
Tathagata Das cbb41a0c5b [SPARK-23966][SS] Refactoring all checkpoint file writing logic in a common CheckpointFileManager interface
## What changes were proposed in this pull request?

Checkpoint files (offset log files, state store files) in Structured Streaming must be written atomically such that no partial files are generated (would break fault-tolerance guarantees). Currently, there are 3 locations which try to do this individually, and in some cases, incorrectly.

1. HDFSOffsetMetadataLog - This uses a FileManager interface to use any implementation of `FileSystem` or `FileContext` APIs. It preferably loads `FileContext` implementation as FileContext of HDFS has atomic renames.
1. HDFSBackedStateStore (aka in-memory state store)
  - Writing a version.delta file - This uses FileSystem APIs only to perform a rename. This is incorrect as rename is not atomic in HDFS FileSystem implementation.
  - Writing a snapshot file - Same as above.

#### Current problems:
1. State Store behavior is incorrect - HDFS FileSystem implementation does not have atomic rename.
1. Inflexible - Some file systems provide mechanisms other than write-to-temp-file-and-rename for writing atomically and more efficiently. For example, with S3 you can write directly to the final file and it will be made visible only when the entire file is written and closed correctly. Any failure can be made to terminate the writing without making any partial files visible in S3. The current code does not abstract out this mechanism enough that it can be customized.

#### Solution:

1. Introduce a common interface that all 3 cases above can use to write checkpoint files atomically.
2. This interface must provide the necessary interfaces that allow customization of the write-and-rename mechanism.

This PR does that by introducing the interface `CheckpointFileManager` and modifying `HDFSMetadataLog` and `HDFSBackedStateStore` to use the interface. Similar to earlier `FileManager`, there are implementations based on `FileSystem` and `FileContext` APIs, and the latter implementation is preferred to make it work correctly with HDFS.

The key method this interface has is `createAtomic(path, overwrite)` which returns a `CancellableFSDataOutputStream` that has the method `cancel()`. All users of this method need to either call `close()` to successfully write the file, or `cancel()` in case of an error.

## How was this patch tested?
New tests in `CheckpointFileManagerSuite` and slightly modified existing tests.

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

Closes #21048 from tdas/SPARK-23966.
2018-04-13 16:31:39 -07:00
Marco Gaido 25892f3cc9 [SPARK-23375][SQL] Eliminate unneeded Sort in Optimizer
## What changes were proposed in this pull request?

Added a new rule to remove Sort operation when its child is already sorted.
For instance, this simple code:
```
spark.sparkContext.parallelize(Seq(("a", "b"))).toDF("a", "b").registerTempTable("table1")
val df = sql(s"""SELECT b
                | FROM (
                |     SELECT a, b
                |     FROM table1
                |     ORDER BY a
                | ) t
                | ORDER BY a""".stripMargin)
df.explain(true)
```
before the PR produces this plan:
```
== Parsed Logical Plan ==
'Sort ['a ASC NULLS FIRST], true
+- 'Project ['b]
   +- 'SubqueryAlias t
      +- 'Sort ['a ASC NULLS FIRST], true
         +- 'Project ['a, 'b]
            +- 'UnresolvedRelation `table1`

== Analyzed Logical Plan ==
b: string
Project [b#7]
+- Sort [a#6 ASC NULLS FIRST], true
   +- Project [b#7, a#6]
      +- SubqueryAlias t
         +- Sort [a#6 ASC NULLS FIRST], true
            +- Project [a#6, b#7]
               +- SubqueryAlias table1
                  +- Project [_1#3 AS a#6, _2#4 AS b#7]
                     +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1, true, false) AS _1#3, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2, true, false) AS _2#4]
                        +- ExternalRDD [obj#2]

== Optimized Logical Plan ==
Project [b#7]
+- Sort [a#6 ASC NULLS FIRST], true
   +- Project [b#7, a#6]
      +- Sort [a#6 ASC NULLS FIRST], true
         +- Project [_1#3 AS a#6, _2#4 AS b#7]
            +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, scala.Tuple2, true])._1, true, false) AS _1#3, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, scala.Tuple2, true])._2, true, false) AS _2#4]
               +- ExternalRDD [obj#2]

== Physical Plan ==
*(3) Project [b#7]
+- *(3) Sort [a#6 ASC NULLS FIRST], true, 0
   +- Exchange rangepartitioning(a#6 ASC NULLS FIRST, 200)
      +- *(2) Project [b#7, a#6]
         +- *(2) Sort [a#6 ASC NULLS FIRST], true, 0
            +- Exchange rangepartitioning(a#6 ASC NULLS FIRST, 200)
               +- *(1) Project [_1#3 AS a#6, _2#4 AS b#7]
                  +- *(1) SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, scala.Tuple2, true])._1, true, false) AS _1#3, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, scala.Tuple2, true])._2, true, false) AS _2#4]
                     +- Scan ExternalRDDScan[obj#2]
```

while after the PR produces:

```
== Parsed Logical Plan ==
'Sort ['a ASC NULLS FIRST], true
+- 'Project ['b]
   +- 'SubqueryAlias t
      +- 'Sort ['a ASC NULLS FIRST], true
         +- 'Project ['a, 'b]
            +- 'UnresolvedRelation `table1`

== Analyzed Logical Plan ==
b: string
Project [b#7]
+- Sort [a#6 ASC NULLS FIRST], true
   +- Project [b#7, a#6]
      +- SubqueryAlias t
         +- Sort [a#6 ASC NULLS FIRST], true
            +- Project [a#6, b#7]
               +- SubqueryAlias table1
                  +- Project [_1#3 AS a#6, _2#4 AS b#7]
                     +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, scala.Tuple2, true]))._1, true, false) AS _1#3, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(assertnotnull(input[0, scala.Tuple2, true]))._2, true, false) AS _2#4]
                        +- ExternalRDD [obj#2]

== Optimized Logical Plan ==
Project [b#7]
+- Sort [a#6 ASC NULLS FIRST], true
   +- Project [_1#3 AS a#6, _2#4 AS b#7]
      +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, scala.Tuple2, true])._1, true, false) AS _1#3, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, scala.Tuple2, true])._2, true, false) AS _2#4]
         +- ExternalRDD [obj#2]

== Physical Plan ==
*(2) Project [b#7]
+- *(2) Sort [a#6 ASC NULLS FIRST], true, 0
   +- Exchange rangepartitioning(a#6 ASC NULLS FIRST, 5)
      +- *(1) Project [_1#3 AS a#6, _2#4 AS b#7]
         +- *(1) SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, scala.Tuple2, true])._1, true, false) AS _1#3, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(input[0, scala.Tuple2, true])._2, true, false) AS _2#4]
            +- Scan ExternalRDDScan[obj#2]
```

this means that an unnecessary sort operation is not performed after the PR.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20560 from mgaido91/SPARK-23375.
2018-04-14 01:01:00 +08:00
Gengliang Wang 4dfd746de3 [SPARK-23896][SQL] Improve PartitioningAwareFileIndex
## What changes were proposed in this pull request?

Currently `PartitioningAwareFileIndex` accepts an optional parameter `userPartitionSchema`. If provided, it will combine the inferred partition schema with the parameter.

However,
1. to get `userPartitionSchema`, we need to  combine inferred partition schema with `userSpecifiedSchema`
2. to get the inferred partition schema, we have to create a temporary file index.

Only after that, a final version of `PartitioningAwareFileIndex` can be created.

This can be improved by passing `userSpecifiedSchema` to `PartitioningAwareFileIndex`.

With the improvement, we can reduce redundant code and avoid parsing the file partition twice.
## How was this patch tested?
Unit test

Author: Gengliang Wang <gengliang.wang@databricks.com>

Closes #21004 from gengliangwang/PartitioningAwareFileIndex.
2018-04-14 00:22:38 +08:00
yucai 0323e61465 [SPARK-23905][SQL] Add UDF weekday
## What changes were proposed in this pull request?

Add UDF weekday

## How was this patch tested?

A new test

Author: yucai <yyu1@ebay.com>

Closes #21009 from yucai/SPARK-23905.
2018-04-13 00:00:04 -07:00
Eric Liang 1018be44d6 [SPARK-23971] Should not leak Spark sessions across test suites
## What changes were proposed in this pull request?

Many suites currently leak Spark sessions (sometimes with stopped SparkContexts) via the thread-local active Spark session and default Spark session. We should attempt to clean these up and detect when this happens to improve the reproducibility of tests.

## How was this patch tested?

Existing tests

Author: Eric Liang <ekl@databricks.com>

Closes #21058 from ericl/clear-session.
2018-04-12 22:30:59 -07:00
hyukjinkwon ab7b961a4f [SPARK-23942][PYTHON][SQL] Makes collect in PySpark as action for a query executor listener
## What changes were proposed in this pull request?

This PR proposes to add `collect` to  a query executor as an action.

Seems `collect` / `collect` with Arrow are not recognised via `QueryExecutionListener` as an action. For example, if we have a custom listener as below:

```scala
package org.apache.spark.sql

import org.apache.spark.internal.Logging
import org.apache.spark.sql.execution.QueryExecution
import org.apache.spark.sql.util.QueryExecutionListener

class TestQueryExecutionListener extends QueryExecutionListener with Logging {
  override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = {
    logError("Look at me! I'm 'onSuccess'")
  }

  override def onFailure(funcName: String, qe: QueryExecution, exception: Exception): Unit = { }
}
```
and set `spark.sql.queryExecutionListeners` to `org.apache.spark.sql.TestQueryExecutionListener`

Other operations in PySpark or Scala side seems fine:

```python
>>> sql("SELECT * FROM range(1)").show()
```
```
18/04/09 17:02:04 ERROR TestQueryExecutionListener: Look at me! I'm 'onSuccess'
+---+
| id|
+---+
|  0|
+---+
```

```scala
scala> sql("SELECT * FROM range(1)").collect()
```
```
18/04/09 16:58:41 ERROR TestQueryExecutionListener: Look at me! I'm 'onSuccess'
res1: Array[org.apache.spark.sql.Row] = Array([0])
```

but ..

**Before**

```python
>>> sql("SELECT * FROM range(1)").collect()
```
```
[Row(id=0)]
```

```python
>>> spark.conf.set("spark.sql.execution.arrow.enabled", "true")
>>> sql("SELECT * FROM range(1)").toPandas()
```
```
   id
0   0
```

**After**

```python
>>> sql("SELECT * FROM range(1)").collect()
```
```
18/04/09 16:57:58 ERROR TestQueryExecutionListener: Look at me! I'm 'onSuccess'
[Row(id=0)]
```

```python
>>> spark.conf.set("spark.sql.execution.arrow.enabled", "true")
>>> sql("SELECT * FROM range(1)").toPandas()
```
```
18/04/09 17:53:26 ERROR TestQueryExecutionListener: Look at me! I'm 'onSuccess'
   id
0   0
```

## How was this patch tested?

I have manually tested as described above and unit test was added.

Author: hyukjinkwon <gurwls223@apache.org>

Closes #21007 from HyukjinKwon/SPARK-23942.
2018-04-13 11:28:13 +08:00
jerryshao 14291b061b [SPARK-23748][SS] Fix SS continuous process doesn't support SubqueryAlias issue
## What changes were proposed in this pull request?

Current SS continuous doesn't support processing on temp table or `df.as("xxx")`, SS will throw an exception as LogicalPlan not supported, details described in [here](https://issues.apache.org/jira/browse/SPARK-23748).

So here propose to add this support.

## How was this patch tested?

new UT.

Author: jerryshao <sshao@hortonworks.com>

Closes #21017 from jerryshao/SPARK-23748.
2018-04-12 20:00:25 -07:00
Imran Rashid 6a2289ecf0 [SPARK-23962][SQL][TEST] Fix race in currentExecutionIds().
SQLMetricsTestUtils.currentExecutionIds() was racing with the listener
bus, which lead to some flaky tests.  We should wait till the listener bus is
empty.

I tested by adding some Thread.sleep()s in SQLAppStatusListener, which
reproduced the exceptions I saw on Jenkins.  With this change, they went
away.

Author: Imran Rashid <irashid@cloudera.com>

Closes #21041 from squito/SPARK-23962.
2018-04-12 15:58:04 +08:00
gatorsmile e904dfaf0d Revert "[SPARK-23960][SQL][MINOR] Mark HashAggregateExec.bufVars as transient"
This reverts commit 271c891b91.
2018-04-11 17:04:34 -07:00
Kris Mok 271c891b91 [SPARK-23960][SQL][MINOR] Mark HashAggregateExec.bufVars as transient
## What changes were proposed in this pull request?

Mark `HashAggregateExec.bufVars` as transient to avoid it from being serialized.
Also manually null out this field at the end of `doProduceWithoutKeys()` to shorten its lifecycle, because it'll no longer be used after that.

## How was this patch tested?

Existing tests.

Author: Kris Mok <kris.mok@databricks.com>

Closes #21039 from rednaxelafx/codegen-improve.
2018-04-11 21:52:48 +08:00
Herman van Hovell c604d659e1 [SPARK-23951][SQL] Use actual java class instead of string representation.
## What changes were proposed in this pull request?
This PR slightly refactors the newly added `ExprValue` API by quite a bit. The following changes are introduced:

1. `ExprValue` now uses the actual class instead of the class name as its type. This should give some more flexibility with generating code in the future.
2. Renamed `StatementValue` to `SimpleExprValue`. The statement concept is broader then an expression (untyped and it cannot be on the right hand side of an assignment), and this was not really what we were using it for. I have added a top level `JavaCode` trait that can be used in the future to reinstate (no pun intended) a statement a-like code fragment.
3. Added factory methods to the `JavaCode` companion object to make it slightly less verbose to create `JavaCode`/`ExprValue` objects. This is also what makes the diff quite large.
4. Added one more factory method to `ExprCode` to make it easier to create code-less expressions.

## How was this patch tested?
Existing tests.

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

Closes #21026 from hvanhovell/SPARK-23951.
2018-04-11 20:11:03 +08:00
Gengliang Wang e179658914 [SPARK-19724][SQL][FOLLOW-UP] Check location of managed table when ignoreIfExists is true
## What changes were proposed in this pull request?

In the PR #20886, I mistakenly check the table location only when `ignoreIfExists` is false, which was following the original deprecated PR.
That was wrong. When `ignoreIfExists` is true and the target table doesn't exist, we should also check the table location. In other word, **`ignoreIfExists` has nothing to do with table location validation**.
This is a follow-up PR to fix the mistake.

## How was this patch tested?

Add one unit test.

Author: Gengliang Wang <gengliang.wang@databricks.com>

Closes #21001 from gengliangwang/SPARK-19724-followup.
2018-04-10 09:33:09 -07:00
Liang-Chi Hsieh 7c1654e215 [SPARK-22856][SQL] Add wrappers for codegen output and nullability
## What changes were proposed in this pull request?

The codegen output of `Expression`, aka `ExprCode`, now encapsulates only strings of output value (`value`) and nullability (`isNull`). It makes difficulty for us to know what the output really is. I think it is better if we can add wrappers for the value and nullability that let us to easily know that.

## How was this patch tested?

Existing tests.

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

Closes #20043 from viirya/SPARK-22856.
2018-04-09 11:54:35 -07:00
Kazuaki Ishizaki 8d40a79a07 [SPARK-23893][CORE][SQL] Avoid possible integer overflow in multiplication
## What changes were proposed in this pull request?

This PR avoids possible overflow at an operation `long = (long)(int * int)`. The multiplication of large positive integer values may set one to MSB. This leads to a negative value in long while we expected a positive value (e.g. `0111_0000_0000_0000 * 0000_0000_0000_0010`).

This PR performs long cast before the multiplication to avoid this situation.

## How was this patch tested?

Existing UTs

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

Closes #21002 from kiszk/SPARK-23893.
2018-04-08 20:40:27 +02:00
Maxim Gekk 6a734575a8 [SPARK-23849][SQL] Tests for the samplingRatio option of JSON datasource
## What changes were proposed in this pull request?

Proposed tests checks that only subset of input dataset is touched during schema inferring.

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #20963 from MaxGekk/json-sampling-tests.
2018-04-07 21:44:32 -07:00
Huaxin Gao 2c1fe64757 [SPARK-23847][PYTHON][SQL] Add asc_nulls_first, asc_nulls_last to PySpark
## What changes were proposed in this pull request?

Column.scala and Functions.scala have asc_nulls_first, asc_nulls_last,  desc_nulls_first and desc_nulls_last. Add the corresponding python APIs in column.py and functions.py

## How was this patch tested?
Add doctest

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

Closes #20962 from huaxingao/spark-23847.
2018-04-08 12:09:06 +08:00
Li Jin d766ea2ff2 [SPARK-23861][SQL][DOC] Clarify default window frame with and without orderBy clause
## What changes were proposed in this pull request?

Add docstring to clarify default window frame boundaries with and without orderBy clause

## How was this patch tested?

Manually generate doc and check.

Author: Li Jin <ice.xelloss@gmail.com>

Closes #20978 from icexelloss/SPARK-23861-window-doc.
2018-04-07 00:15:54 +08:00
Yuchen Huo 9452401931 [SPARK-23822][SQL] Improve error message for Parquet schema mismatches
## What changes were proposed in this pull request?

This pull request tries to improve the error message for spark while reading parquet files with different schemas, e.g. One with a STRING column and the other with a INT column. A new ParquetSchemaColumnConvertNotSupportedException is added to replace the old UnsupportedOperationException. The Exception is again wrapped in FileScanRdd.scala to throw a more a general QueryExecutionException with the actual parquet file name which trigger the exception.

## How was this patch tested?

Unit tests added to check the new exception and verify the error messages.

Also manually tested with two parquet with different schema to check the error message.

<img width="1125" alt="screen shot 2018-03-30 at 4 03 04 pm" src="https://user-images.githubusercontent.com/37087310/38156580-dd58a140-3433-11e8-973a-b816d859fbe1.png">

Author: Yuchen Huo <yuchen.huo@databricks.com>

Closes #20953 from yuchenhuo/SPARK-23822.
2018-04-06 08:35:20 -07:00
Gengliang Wang 249007e37f [SPARK-19724][SQL] create a managed table with an existed default table should throw an exception
## What changes were proposed in this pull request?
This PR is to finish https://github.com/apache/spark/pull/17272

This JIRA is a follow up work after SPARK-19583

As we discussed in that PR

The following DDL for a managed table with an existed default location should throw an exception:

CREATE TABLE ... (PARTITIONED BY ...) AS SELECT ...
CREATE TABLE ... (PARTITIONED BY ...)
Currently there are some situations which are not consist with above logic:

CREATE TABLE ... (PARTITIONED BY ...) succeed with an existed default location
situation: for both hive/datasource(with HiveExternalCatalog/InMemoryCatalog)

CREATE TABLE ... (PARTITIONED BY ...) AS SELECT ...
situation: hive table succeed with an existed default location

This PR is going to make above two situations consist with the logic that it should throw an exception
with an existed default location.
## How was this patch tested?

unit test added

Author: Gengliang Wang <gengliang.wang@databricks.com>

Closes #20886 from gengliangwang/pr-17272.
2018-04-05 20:19:25 -07:00
Kazuaki Ishizaki 4807d381bb [SPARK-10399][CORE][SQL] Introduce multiple MemoryBlocks to choose several types of memory block
## What changes were proposed in this pull request?

This PR allows us to use one of several types of `MemoryBlock`, such as byte array, int array, long array, or `java.nio.DirectByteBuffer`. To use `java.nio.DirectByteBuffer` allows to have off heap memory which is automatically deallocated by JVM. `MemoryBlock`  class has primitive accessors like `Platform.getInt()`, `Platform.putint()`, or `Platform.copyMemory()`.

This PR uses `MemoryBlock` for `OffHeapColumnVector`, `UTF8String`, and other places. This PR can improve performance of operations involving memory accesses (e.g. `UTF8String.trim`) by 1.8x.

For now, this PR does not use `MemoryBlock` for `BufferHolder` based on cloud-fan's [suggestion](https://github.com/apache/spark/pull/11494#issuecomment-309694290).

Since this PR is a successor of #11494, close #11494. Many codes were ported from #11494. Many efforts were put here. **I think this PR should credit to yzotov.**

This PR can achieve **1.1-1.4x performance improvements** for  operations in `UTF8String` or `Murmur3_x86_32`. Other operations are almost comparable performances.

Without this PR
```
OpenJDK 64-Bit Server VM 1.8.0_121-8u121-b13-0ubuntu1.16.04.2-b13 on Linux 4.4.0-22-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
OpenJDK 64-Bit Server VM 1.8.0_121-8u121-b13-0ubuntu1.16.04.2-b13 on Linux 4.4.0-22-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
Hash byte arrays with length 268435487:  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Murmur3_x86_32                                 526 /  536          0.0   131399881.5       1.0X

UTF8String benchmark:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
hashCode                                       525 /  552       1022.6           1.0       1.0X
substring                                      414 /  423       1298.0           0.8       1.3X
```

With this PR
```
OpenJDK 64-Bit Server VM 1.8.0_121-8u121-b13-0ubuntu1.16.04.2-b13 on Linux 4.4.0-22-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
Hash byte arrays with length 268435487:  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Murmur3_x86_32                                 474 /  488          0.0   118552232.0       1.0X

UTF8String benchmark:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
hashCode                                       476 /  480       1127.3           0.9       1.0X
substring                                      287 /  291       1869.9           0.5       1.7X
```

Benchmark program
```
test("benchmark Murmur3_x86_32") {
  val length = 8192 * 32768 + 31
  val seed = 42L
  val iters = 1 << 2
  val random = new Random(seed)
  val arrays = Array.fill[MemoryBlock](numArrays) {
    val bytes = new Array[Byte](length)
    random.nextBytes(bytes)
    new ByteArrayMemoryBlock(bytes, Platform.BYTE_ARRAY_OFFSET, length)
  }

  val benchmark = new Benchmark("Hash byte arrays with length " + length,
    iters * numArrays, minNumIters = 20)
  benchmark.addCase("HiveHasher") { _: Int =>
    var sum = 0L
    for (_ <- 0L until iters) {
      sum += HiveHasher.hashUnsafeBytesBlock(
        arrays(i), Platform.BYTE_ARRAY_OFFSET, length)
    }
  }
  benchmark.run()
}

test("benchmark UTF8String") {
  val N = 512 * 1024 * 1024
  val iters = 2
  val benchmark = new Benchmark("UTF8String benchmark", N, minNumIters = 20)
  val str0 = new java.io.StringWriter() { { for (i <- 0 until N) { write(" ") } } }.toString
  val s0 = UTF8String.fromString(str0)
  benchmark.addCase("hashCode") { _: Int =>
    var h: Int = 0
    for (_ <- 0L until iters) { h += s0.hashCode }
  }
  benchmark.addCase("substring") { _: Int =>
    var s: UTF8String = null
    for (_ <- 0L until iters) { s = s0.substring(N / 2 - 5, N / 2 + 5) }
  }
  benchmark.run()
}
```

I run [this benchmark program](https://gist.github.com/kiszk/94f75b506c93a663bbbc372ffe8f05de) using [the commit](ee5a79861c). I got the following results:

```
OpenJDK 64-Bit Server VM 1.8.0_151-8u151-b12-0ubuntu0.16.04.2-b12 on Linux 4.4.0-66-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
Memory access benchmarks:                Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
ByteArrayMemoryBlock get/putInt()              220 /  221        609.3           1.6       1.0X
Platform get/putInt(byte[])                    220 /  236        610.9           1.6       1.0X
Platform get/putInt(Object)                    492 /  494        272.8           3.7       0.4X
OnHeapMemoryBlock get/putLong()                322 /  323        416.5           2.4       0.7X
long[]                                         221 /  221        608.0           1.6       1.0X
Platform get/putLong(long[])                   321 /  321        418.7           2.4       0.7X
Platform get/putLong(Object)                   561 /  563        239.2           4.2       0.4X
```

I also run [this benchmark program](https://gist.github.com/kiszk/5fdb4e03733a5d110421177e289d1fb5) for comparing performance of `Platform.copyMemory()`.
```
OpenJDK 64-Bit Server VM 1.8.0_151-8u151-b12-0ubuntu0.16.04.2-b12 on Linux 4.4.0-66-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
Platform copyMemory:                     Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Object to Object                              1961 / 1967          8.6         116.9       1.0X
System.arraycopy Object to Object             1917 / 1921          8.8         114.3       1.0X
byte array to byte array                      1961 / 1968          8.6         116.9       1.0X
System.arraycopy byte array to byte array      1909 / 1937          8.8         113.8       1.0X
int array to int array                        1921 / 1990          8.7         114.5       1.0X
double array to double array                  1918 / 1923          8.7         114.3       1.0X
Object to byte array                          1961 / 1967          8.6         116.9       1.0X
Object to short array                         1965 / 1972          8.5         117.1       1.0X
Object to int array                           1910 / 1915          8.8         113.9       1.0X
Object to float array                         1971 / 1978          8.5         117.5       1.0X
Object to double array                        1919 / 1944          8.7         114.4       1.0X
byte array to Object                          1959 / 1967          8.6         116.8       1.0X
int array to Object                           1961 / 1970          8.6         116.9       1.0X
double array to Object                        1917 / 1924          8.8         114.3       1.0X
```

These results show three facts:
1. According to the second/third or sixth/seventh results in the first experiment, if we use `Platform.get/putInt(Object)`, we achieve more than 2x worse performance than `Platform.get/putInt(byte[])` with concrete type (i.e. `byte[]`).
2. According to the second/third or fourth/fifth/sixth results in the first experiment, the fastest way to access an array element on Java heap is `array[]`. **Cons of `array[]` is that it is not possible to support unaligned-8byte access.**
3. According to the first/second/third or fourth/sixth/seventh results in the first experiment, `getInt()/putInt() or getLong()/putLong()` in subclasses of `MemoryBlock` can achieve comparable performance to `Platform.get/putInt()` or `Platform.get/putLong()` with concrete type (second or sixth result). There is no overhead regarding virtual call.
4. According to results in the second experiment, for `Platform.copy()`, to pass `Object` can achieve the same performance as to pass any type of primitive array as source or destination.
5. According to second/fourth results in the second experiment, `Platform.copy()` can achieve the same performance as `System.arrayCopy`. **It would be good to use `Platform.copy()` since `Platform.copy()` can take any types for src and dst.**

We are incrementally replace `Platform.get/putXXX` with `MemoryBlock.get/putXXX`. This is because we have two advantages.
1) Achieve better performance due to having a concrete type for an array.
2) Use simple OO design instead of passing `Object`
It is easy to use `MemoryBlock` in `InternalRow`, `BufferHolder`, `TaskMemoryManager`, and others that are already abstracted. It is not easy to use `MemoryBlock` in utility classes related to hashing or others.

Other candidates are
- UnsafeRow, UnsafeArrayData, UnsafeMapData, SpecificUnsafeRowJoiner
- UTF8StringBuffer
- BufferHolder
- TaskMemoryManager
- OnHeapColumnVector
- BytesToBytesMap
- CachedBatch
- classes for hash
- others.

## How was this patch tested?

Added `UnsafeMemoryAllocator`

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

Closes #19222 from kiszk/SPARK-10399.
2018-04-06 10:13:59 +08:00
Gengliang Wang d8379e5bc3 [SPARK-23838][WEBUI] Running SQL query is displayed as "completed" in SQL tab
## What changes were proposed in this pull request?

A running SQL query would appear as completed in the Spark UI:
![image1](https://user-images.githubusercontent.com/1097932/38170733-3d7cb00c-35bf-11e8-994c-43f2d4fa285d.png)

We can see the query in "Completed queries", while in in the job page we see it's still running Job 132.
![image2](https://user-images.githubusercontent.com/1097932/38170735-48f2c714-35bf-11e8-8a41-6fae23543c46.png)

After some time in the query still appears in "Completed queries" (while it's still running), but the "Duration" gets increased.
![image3](https://user-images.githubusercontent.com/1097932/38170737-50f87ea4-35bf-11e8-8b60-000f6f918964.png)

To reproduce, we can run a query with multiple jobs. E.g. Run TPCDS q6.

The reason is that updates from executions are written into kvstore periodically, and the job start event may be missed.

## How was this patch tested?
Manually run the job again and check the SQL Tab. The fix is pretty simple.

Author: Gengliang Wang <gengliang.wang@databricks.com>

Closes #20955 from gengliangwang/jobCompleted.
2018-04-04 15:43:58 -07:00
Takeshi Yamamuro 5197562afe [SPARK-21351][SQL] Update nullability based on children's output
## What changes were proposed in this pull request?
This pr added a new optimizer rule `UpdateNullabilityInAttributeReferences ` to update the nullability that `Filter` changes when having `IsNotNull`. In the master, optimized plans do not respect the nullability when `Filter` has `IsNotNull`. This wrongly generates unnecessary code. For example:

```
scala> val df = Seq((Some(1), Some(2))).toDF("a", "b")
scala> val bIsNotNull = df.where($"b" =!= 2).select($"b")
scala> val targetQuery = bIsNotNull.distinct
scala> val targetQuery.queryExecution.optimizedPlan.output(0).nullable
res5: Boolean = true

scala> targetQuery.debugCodegen
Found 2 WholeStageCodegen subtrees.
== Subtree 1 / 2 ==
*HashAggregate(keys=[b#19], functions=[], output=[b#19])
+- Exchange hashpartitioning(b#19, 200)
   +- *HashAggregate(keys=[b#19], functions=[], output=[b#19])
      +- *Project [_2#16 AS b#19]
         +- *Filter isnotnull(_2#16)
            +- LocalTableScan [_1#15, _2#16]

Generated code:
...
/* 124 */   protected void processNext() throws java.io.IOException {
...
/* 132 */     // output the result
/* 133 */
/* 134 */     while (agg_mapIter.next()) {
/* 135 */       wholestagecodegen_numOutputRows.add(1);
/* 136 */       UnsafeRow agg_aggKey = (UnsafeRow) agg_mapIter.getKey();
/* 137 */       UnsafeRow agg_aggBuffer = (UnsafeRow) agg_mapIter.getValue();
/* 138 */
/* 139 */       boolean agg_isNull4 = agg_aggKey.isNullAt(0);
/* 140 */       int agg_value4 = agg_isNull4 ? -1 : (agg_aggKey.getInt(0));
/* 141 */       agg_rowWriter1.zeroOutNullBytes();
/* 142 */
                // We don't need this NULL check because NULL is filtered out in `$"b" =!=2`
/* 143 */       if (agg_isNull4) {
/* 144 */         agg_rowWriter1.setNullAt(0);
/* 145 */       } else {
/* 146 */         agg_rowWriter1.write(0, agg_value4);
/* 147 */       }
/* 148 */       append(agg_result1);
/* 149 */
/* 150 */       if (shouldStop()) return;
/* 151 */     }
/* 152 */
/* 153 */     agg_mapIter.close();
/* 154 */     if (agg_sorter == null) {
/* 155 */       agg_hashMap.free();
/* 156 */     }
/* 157 */   }
/* 158 */
/* 159 */ }
```

In the line 143, we don't need this NULL check because NULL is filtered out in `$"b" =!=2`.
This pr could remove this NULL check;

```
scala> val targetQuery.queryExecution.optimizedPlan.output(0).nullable
res5: Boolean = false

scala> targetQuery.debugCodegen
...
Generated code:
...
/* 144 */   protected void processNext() throws java.io.IOException {
...
/* 152 */     // output the result
/* 153 */
/* 154 */     while (agg_mapIter.next()) {
/* 155 */       wholestagecodegen_numOutputRows.add(1);
/* 156 */       UnsafeRow agg_aggKey = (UnsafeRow) agg_mapIter.getKey();
/* 157 */       UnsafeRow agg_aggBuffer = (UnsafeRow) agg_mapIter.getValue();
/* 158 */
/* 159 */       int agg_value4 = agg_aggKey.getInt(0);
/* 160 */       agg_rowWriter1.write(0, agg_value4);
/* 161 */       append(agg_result1);
/* 162 */
/* 163 */       if (shouldStop()) return;
/* 164 */     }
/* 165 */
/* 166 */     agg_mapIter.close();
/* 167 */     if (agg_sorter == null) {
/* 168 */       agg_hashMap.free();
/* 169 */     }
/* 170 */   }
```

## How was this patch tested?
Added `UpdateNullabilityInAttributeReferencesSuite` for unit tests.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #18576 from maropu/SPARK-21351.
2018-04-04 14:39:19 +08:00
Eric Liang 359375eff7 [SPARK-23809][SQL] Active SparkSession should be set by getOrCreate
## What changes were proposed in this pull request?

Currently, the active spark session is set inconsistently (e.g., in createDataFrame, prior to query execution). Many places in spark also incorrectly query active session when they should be calling activeSession.getOrElse(defaultSession) and so might get None even if a Spark session exists.

The semantics here can be cleaned up if we also set the active session when the default session is set.

Related: https://github.com/apache/spark/pull/20926/files

## How was this patch tested?

Unit test, existing test. Note that if https://github.com/apache/spark/pull/20926 merges first we should also update the tests there.

Author: Eric Liang <ekl@databricks.com>

Closes #20927 from ericl/active-session-cleanup.
2018-04-03 17:09:12 -07:00
Jose Torres 66a3a5a2dc [SPARK-23099][SS] Migrate foreach sink to DataSourceV2
## What changes were proposed in this pull request?

Migrate foreach sink to DataSourceV2.

Since the previous attempt at this PR #20552, we've changed and strictly defined the lifecycle of writer components. This means we no longer need the complicated lifecycle shim from that PR; it just naturally works.

## How was this patch tested?

existing tests

Author: Jose Torres <torres.joseph.f+github@gmail.com>

Closes #20951 from jose-torres/foreach.
2018-04-03 11:05:29 -07:00