Commit graph

5358 commits

Author SHA1 Message Date
Yuming Wang 77c996403d [SPARK-25368][SQL] Incorrect predicate pushdown returns wrong result
## What changes were proposed in this pull request?
How to reproduce:
```scala
val df1 = spark.createDataFrame(Seq(
   (1, 1)
)).toDF("a", "b").withColumn("c", lit(null).cast("int"))
val df2 = df1.union(df1).withColumn("d", spark_partition_id).filter($"c".isNotNull)
df2.show

+---+---+----+---+
|  a|  b|   c|  d|
+---+---+----+---+
|  1|  1|null|  0|
|  1|  1|null|  1|
+---+---+----+---+
```
`filter($"c".isNotNull)` was transformed to `(null <=> c#10)` before https://github.com/apache/spark/pull/19201, but it is transformed to `(c#10 = null)` since https://github.com/apache/spark/pull/20155. This pr revert it to `(null <=> c#10)` to fix this issue.

## How was this patch tested?

unit tests

Closes #22368 from wangyum/SPARK-25368.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-09-09 09:07:31 -07:00
gatorsmile 0b9ccd55c2 Revert [SPARK-10399] [SPARK-23879] [SPARK-23762] [SPARK-25317]
## What changes were proposed in this pull request?

When running TPC-DS benchmarks on 2.4 release, npoggi and winglungngai  saw more than 10% performance regression on the following queries: q67, q24a and q24b. After we applying the PR https://github.com/apache/spark/pull/22338, the performance regression still exists. If we revert the changes in https://github.com/apache/spark/pull/19222, npoggi and winglungngai  found the performance regression was resolved. Thus, this PR is to revert the related changes for unblocking the 2.4 release.

In the future release, we still can continue the investigation and find out the root cause of the regression.

## How was this patch tested?

The existing test cases

Closes #22361 from gatorsmile/revertMemoryBlock.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-09 21:25:19 +08:00
ptkool 78981efc2c [SPARK-20636] Add new optimization rule to transpose adjacent Window expressions.
## What changes were proposed in this pull request?

Add new optimization rule to eliminate unnecessary shuffling by flipping adjacent Window expressions.

## How was this patch tested?

Tested with unit tests, integration tests, and manual tests.

Closes #17899 from ptkool/adjacent_window_optimization.

Authored-by: ptkool <michael.styles@shopify.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-09-08 11:36:55 -07:00
Dilip Biswal 6d7bc5af45 [SPARK-25267][SQL][TEST] Disable ConvertToLocalRelation in the test cases of sql/core and sql/hive
## What changes were proposed in this pull request?
In SharedSparkSession and TestHive, we need to disable the rule ConvertToLocalRelation for better test case coverage.
## How was this patch tested?
Identify the failures after excluding "ConvertToLocalRelation" rule.

Closes #22270 from dilipbiswal/SPARK-25267-final.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-09-06 23:35:02 -07:00
dujunling ed249db9c4 [SPARK-25237][SQL] Remove updateBytesReadWithFileSize in FileScanRDD
## What changes were proposed in this pull request?
This pr removed the method `updateBytesReadWithFileSize` in `FileScanRDD` because it computes input metrics by file size supported in Hadoop 2.5 and earlier. The current Spark does not support the versions, so it causes wrong input metric numbers.

This is rework from #22232.

Closes #22232

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

Closes #22324 from maropu/pr22232-2.

Lead-authored-by: dujunling <dujunling@huawei.com>
Co-authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-09-06 21:44:46 -07:00
xuejianbest f5817d8bb3 [SPARK-25108][SQL] Fix the show method to display the wide character alignment problem
This is not a perfect solution. It is designed to minimize complexity on the basis of solving problems.

It is effective for English, Chinese characters, Japanese, Korean and so on.

```scala
before:
+---+---------------------------+-------------+
|id |中国                         |s2           |
+---+---------------------------+-------------+
|1  |ab                         |[a]          |
|2  |null                       |[中国, abc]    |
|3  |ab1                        |[hello world]|
|4  |か行 きゃ(kya) きゅ(kyu) きょ(kyo) |[“中国]        |
|5  |中国(你好)a                    |[“中(国), 312] |
|6  |中国山(东)服务区                  |[“中(国)]      |
|7  |中国山东服务区                    |[中(国)]       |
|8  |                           |[中国]         |
+---+---------------------------+-------------+

after:
+---+-----------------------------------+----------------+
|id |中国                               |s2              |
+---+-----------------------------------+----------------+
|1  |ab                                 |[a]             |
|2  |null                               |[中国, abc]     |
|3  |ab1                                |[hello world]   |
|4  |か行 きゃ(kya) きゅ(kyu) きょ(kyo) |[“中国]         |
|5  |中国(你好)a                      |[“中(国), 312]|
|6  |中国山(东)服务区                   |[“中(国)]      |
|7  |中国山东服务区                     |[中(国)]        |
|8  |                                   |[中国]          |
+---+-----------------------------------+----------------+
```

## What changes were proposed in this pull request?

When there are wide characters such as Chinese characters or Japanese characters in the data, the show method has a alignment problem.
Try to fix this problem.

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)

![image](https://user-images.githubusercontent.com/13044869/44250564-69f6b400-a227-11e8-88b2-6cf6960377ff.png)

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

Closes #22048 from xuejianbest/master.

Authored-by: xuejianbest <384329882@qq.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-09-06 07:17:37 -07:00
Maxim Gekk d749d034a8 [SPARK-25252][SQL] Support arrays of any types by to_json
## What changes were proposed in this pull request?

In the PR, I propose to extended `to_json` and support any types as element types of input arrays. It should allow converting arrays of primitive types and arrays of arrays. For example:

```
select to_json(array('1','2','3'))
> ["1","2","3"]
select to_json(array(array(1,2,3),array(4)))
> [[1,2,3],[4]]
```

## How was this patch tested?

Added a couple sql tests for arrays of primitive type and of arrays. Also I added round trip test `from_json` -> `to_json`.

Closes #22226 from MaxGekk/to_json-array.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-06 12:35:59 +08:00
Gengliang Wang 3d6b68b030 [SPARK-25313][SQL] Fix regression in FileFormatWriter output names
## What changes were proposed in this pull request?

Let's see the follow example:
```
        val location = "/tmp/t"
        val df = spark.range(10).toDF("id")
        df.write.format("parquet").saveAsTable("tbl")
        spark.sql("CREATE VIEW view1 AS SELECT id FROM tbl")
        spark.sql(s"CREATE TABLE tbl2(ID long) USING parquet location $location")
        spark.sql("INSERT OVERWRITE TABLE tbl2 SELECT ID FROM view1")
        println(spark.read.parquet(location).schema)
        spark.table("tbl2").show()
```
The output column name in schema will be `id` instead of `ID`, thus the last query shows nothing from `tbl2`.
By enabling the debug message we can see that the output naming is changed from `ID` to `id`, and then the `outputColumns` in `InsertIntoHadoopFsRelationCommand` is changed in `RemoveRedundantAliases`.
![wechatimg5](https://user-images.githubusercontent.com/1097932/44947871-6299f200-ae46-11e8-9c96-d45fe368206c.jpeg)

![wechatimg4](https://user-images.githubusercontent.com/1097932/44947866-56ae3000-ae46-11e8-8923-8b3bbe060075.jpeg)

**To guarantee correctness**, we should change the output columns from `Seq[Attribute]` to `Seq[String]` to avoid its names being replaced by optimizer.

I will fix project elimination related rules in https://github.com/apache/spark/pull/22311 after this one.

## How was this patch tested?

Unit test.

Closes #22320 from gengliangwang/fixOutputSchema.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-06 10:37:52 +08:00
Wenchen Fan 71bd796517 [SPARK-23243][CORE] Fix RDD.repartition() data correctness issue
## What changes were proposed in this pull request?

An alternative fix for https://github.com/apache/spark/pull/21698

When Spark rerun tasks for an RDD, there are 3 different behaviors:
1. determinate. Always return the same result with same order when rerun.
2. unordered. Returns same data set in random order when rerun.
3. indeterminate. Returns different result when rerun.

Normally Spark doesn't need to care about it. Spark runs stages one by one, when a task is failed, just rerun it. Although the rerun task may return a different result, users will not be surprised.

However, Spark may rerun a finished stage when seeing fetch failures. When this happens, Spark needs to rerun all the tasks of all the succeeding stages if the RDD output is indeterminate, because the input of the succeeding stages has been changed.

If the RDD output is determinate, we only need to rerun the failed tasks of the succeeding stages, because the input doesn't change.

If the RDD output is unordered, it's same as determinate, because shuffle partitioner is always deterministic(round-robin partitioner is not a shuffle partitioner that extends `org.apache.spark.Partitioner`), so the reducers will still get the same input data set.

This PR fixed the failure handling for `repartition`, to avoid correctness issues.

For `repartition`, it applies a stateful map function to generate a round-robin id, which is order sensitive and makes the RDD's output indeterminate. When the stage contains `repartition` reruns, we must also rerun all the tasks of all the succeeding stages.

**future improvement:**
1. Currently we can't rollback and rerun a shuffle map stage, and just fail. We should fix it later. https://issues.apache.org/jira/browse/SPARK-25341
2. Currently we can't rollback and rerun a result stage, and just fail. We should fix it later. https://issues.apache.org/jira/browse/SPARK-25342
3. We should provide public API to allow users to tag the random level of the RDD's computing function.

## How is this pull request tested?
a new test case

Closes #22112 from cloud-fan/repartition.

Lead-authored-by: Wenchen Fan <wenchen@databricks.com>
Co-authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2018-09-05 15:36:34 -07:00
Dongjoon Hyun c66eef8440 [SPARK-25306][SQL][FOLLOWUP] Change test to ignore in FilterPushdownBenchmark
## What changes were proposed in this pull request?

This is a follow-up of #22313 and aim to ignore the micro benchmark test which takes over 2 minutes in Jenkins.
- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.6/4939/consoleFull

## How was this patch tested?

The test case should be ignored in Jenkins.
```
[info] FilterPushdownBenchmark:
...
[info] - Pushdown benchmark with many filters !!! IGNORED !!!
```

Closes #22336 from dongjoon-hyun/SPARK-25306-2.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2018-09-05 11:29:15 -07:00
Wenchen Fan 341b55a589 [SPARK-25044][SQL][FOLLOWUP] add back UserDefinedFunction.inputTypes
## What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/22259 .

Scala case class has a wide surface: apply, unapply, accessors, copy, etc.

In https://github.com/apache/spark/pull/22259 , we change the type of `UserDefinedFunction.inputTypes` from `Option[Seq[DataType]]` to `Option[Seq[Schema]]`. This breaks backward compatibility.

This PR changes the type back, and use a `var` to keep the new nullable info.

## How was this patch tested?

N/A

Closes #22319 from cloud-fan/revert.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-05 21:13:16 +08:00
Shixiong Zhu 2119e518d3 [SPARK-25336][SS]Revert SPARK-24863 and SPARK-24748
## What changes were proposed in this pull request?

Revert SPARK-24863 (#21819) and SPARK-24748 (#21721) as per discussion in #21721. We will revisit them when the data source v2 APIs are out.

## How was this patch tested?

Jenkins

Closes #22334 from zsxwing/revert-SPARK-24863-SPARK-24748.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-05 13:39:34 +08:00
Dongjoon Hyun 103f513231 [SPARK-25306][SQL] Avoid skewed filter trees to speed up createFilter in ORC
## What changes were proposed in this pull request?

In both ORC data sources, `createFilter` function has exponential time complexity due to its skewed filter tree generation. This PR aims to improve it by using new `buildTree` function.

**REPRODUCE**
```scala
// Create and read 1 row table with 1000 columns
sql("set spark.sql.orc.filterPushdown=true")
val selectExpr = (1 to 1000).map(i => s"id c$i")
spark.range(1).selectExpr(selectExpr: _*).write.mode("overwrite").orc("/tmp/orc")
print(s"With 0 filters, ")
spark.time(spark.read.orc("/tmp/orc").count)

// Increase the number of filters
(20 to 30).foreach { width =>
  val whereExpr = (1 to width).map(i => s"c$i is not null").mkString(" and ")
  print(s"With $width filters, ")
  spark.time(spark.read.orc("/tmp/orc").where(whereExpr).count)
}
```

**RESULT**
```scala
With 0 filters, Time taken: 653 ms
With 20 filters, Time taken: 962 ms
With 21 filters, Time taken: 1282 ms
With 22 filters, Time taken: 1982 ms
With 23 filters, Time taken: 3855 ms
With 24 filters, Time taken: 6719 ms
With 25 filters, Time taken: 12669 ms
With 26 filters, Time taken: 25032 ms
With 27 filters, Time taken: 49585 ms
With 28 filters, Time taken: 98980 ms    // over 1 min 38 seconds
With 29 filters, Time taken: 198368 ms   // over 3 mins
With 30 filters, Time taken: 393744 ms   // over 6 mins
```

**AFTER THIS PR**
```scala
With 0 filters, Time taken: 774 ms
With 20 filters, Time taken: 601 ms
With 21 filters, Time taken: 399 ms
With 22 filters, Time taken: 679 ms
With 23 filters, Time taken: 363 ms
With 24 filters, Time taken: 342 ms
With 25 filters, Time taken: 336 ms
With 26 filters, Time taken: 352 ms
With 27 filters, Time taken: 322 ms
With 28 filters, Time taken: 302 ms
With 29 filters, Time taken: 307 ms
With 30 filters, Time taken: 301 ms
```

## How was this patch tested?

Pass the Jenkins with newly added test cases.

Closes #22313 from dongjoon-hyun/SPARK-25306.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-05 10:24:13 +08:00
Xingbo Jiang 3aa60282cc [SPARK-19355][SQL][FOLLOWUP][TEST] Properly recycle SparkSession on TakeOrderedAndProjectSuite finishes
## What changes were proposed in this pull request?

Previously in `TakeOrderedAndProjectSuite` the SparkSession will not get recycled when the test suite finishes.

## How was this patch tested?

N/A

Closes #22330 from jiangxb1987/SPARK-19355.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2018-09-04 09:44:42 -07:00
Xiao Li 7fc8881b0f [SPARK-25296][SQL][TEST] Create ExplainSuite
## What changes were proposed in this pull request?
Move the output verification of Explain test cases to a new suite ExplainSuite.

## How was this patch tested?
N/A

Closes #22300 from gatorsmile/test3200.

Authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2018-08-31 08:47:20 -07:00
yucai 8d9495a8f1 [SPARK-25207][SQL] Case-insensitve field resolution for filter pushdown when reading Parquet
## What changes were proposed in this pull request?

Currently, filter pushdown will not work if Parquet schema and Hive metastore schema are in different letter cases even spark.sql.caseSensitive is false.

Like the below case:
```scala
spark.sparkContext.hadoopConfiguration.setInt("parquet.block.size", 8 * 1024 * 1024)
spark.range(1, 40 * 1024 * 1024, 1, 1).sortWithinPartitions("id").write.parquet("/tmp/t")
sql("CREATE TABLE t (ID LONG) USING parquet LOCATION '/tmp/t'")
sql("select * from t where id < 100L").write.csv("/tmp/id")
```

Although filter "ID < 100L" is generated by Spark, it fails to pushdown into parquet actually, Spark still does the full table scan when reading.
This PR provides a case-insensitive field resolution to make it work.

Before - "ID < 100L" fail to pushedown:
<img width="273" alt="screen shot 2018-08-23 at 10 08 26 pm" src="https://user-images.githubusercontent.com/2989575/44530558-40ef8b00-a721-11e8-8abc-7f97671590d3.png">
After - "ID < 100L" pushedown sucessfully:
<img width="267" alt="screen shot 2018-08-23 at 10 08 40 pm" src="https://user-images.githubusercontent.com/2989575/44530567-44831200-a721-11e8-8634-e9f664b33d39.png">

## How was this patch tested?

Added UTs.

Closes #22197 from yucai/SPARK-25207.

Authored-by: yucai <yyu1@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-31 19:24:09 +08:00
忍冬 f29c2b5287 [SPARK-25256][SQL][TEST] Plan mismatch errors in Hive tests in Scala 2.12
## What changes were proposed in this pull request?

### For `SPARK-5775 read array from partitioned_parquet_with_key_and_complextypes`:

scala2.12
```
scala> (1 to 10).toString
res4: String = Range 1 to 10
```

scala2.11
```
scala> (1 to 10).toString
res2: String = Range(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)
```
And

```
  def prepareAnswer(answer: Seq[Row], isSorted: Boolean): Seq[Row] = {
    val converted: Seq[Row] = answer.map(prepareRow)
    if (!isSorted) converted.sortBy(_.toString()) else converted
  }
```
sortBy `_.toString` is not a good idea.

### Other failures are caused by

```
Array(Int.box(1)).toSeq == Array(Double.box(1.0)).toSeq
```

It is false in 2.12.2 + and is true in 2.11.x , 2.12.0, 2.12.1

## How was this patch tested?

This is a  patch on a specific unit test.

Closes #22264 from sadhen/SPARK25256.

Authored-by: 忍冬 <rendong@wacai.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-08-30 22:37:40 -05:00
aai95 c685b5f56a
[SPARK-24411][SQL] Adding native Java tests for 'isInCollection'
## What changes were proposed in this pull request?
`JavaColumnExpressionSuite.java` was added and `org.apache.spark.sql.ColumnExpressionSuite#test("isInCollection: Java Collection")` was removed.
It provides native Java tests for the method `org.apache.spark.sql.Column#isInCollection`.

Closes #22253 from aai95/isInCollectionJavaTest.

Authored-by: aai95 <aai95@yandex.ru>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2018-08-30 20:38:03 +00:00
忍冬 56bc70047e [SQL][MINOR] Fix compiling for scala 2.12
## What changes were proposed in this pull request?
Introduced by #21320 and #11744

```
$ sbt
> ++2.12.6
> project sql
> compile
...
[error] [warn] spark/sql/core/src/main/scala/org/apache/spark/sql/execution/ProjectionOverSchema.scala:41: match may not be exhaustive.
[error] It would fail on the following inputs: (_, ArrayType(_, _)), (_, _)
[error] [warn]         getProjection(a.child).map(p => (p, p.dataType)).map {
[error] [warn]
[error] [warn] spark/sql/core/src/main/scala/org/apache/spark/sql/execution/ProjectionOverSchema.scala:52: match may not be exhaustive.
[error] It would fail on the following input: (_, _)
[error] [warn]         getProjection(child).map(p => (p, p.dataType)).map {
[error] [warn]
...
```

And

```
$ sbt
> ++2.12.6
> project hive
> testOnly *ParquetMetastoreSuite
...
[error] /Users/rendong/wdi/spark/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala:22: object tools is not a member of package scala
[error] import scala.tools.nsc.Properties
[error]              ^
[error] /Users/rendong/wdi/spark/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala:146: not found: value Properties
[error]     val version = Properties.versionNumberString match {
[error]                   ^
[error] two errors found
...
```

## How was this patch tested?
Existing tests.

Closes #22260 from sadhen/fix_exhaustive_match.

Authored-by: 忍冬 <rendong@wacai.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-30 15:05:36 +08:00
Juliusz Sompolski 6b1b10ca85 [DOC] Fix comment on SparkPlanGraphEdge
## What changes were proposed in this pull request?

`fromId` is the child, and `toId` is the parent, see line 127 in `buildSparkPlanGraphNode` above.
The edges in Spark UI also go from child to parent.

## How was this patch tested?

Comment change only. Inspected code above. Inspected how the edges in Spark UI look like.

Closes #22268 from juliuszsompolski/sparkplangraphedgedoc.

Authored-by: Juliusz Sompolski <julek@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2018-08-29 12:55:44 -07:00
Sean Owen 1fd59c129a [WIP][SPARK-25044][SQL] (take 2) Address translation of LMF closure primitive args to Object in Scala 2.12
## What changes were proposed in this pull request?

Alternative take on https://github.com/apache/spark/pull/22063 that does not introduce udfInternal.
Resolve issue with inferring func types in 2.12 by instead using info captured when UDF is registered -- capturing which types are nullable (i.e. not primitive)

## How was this patch tested?

Existing tests.

Closes #22259 from srowen/SPARK-25044.2.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-29 15:23:16 +08:00
Bryan Cutler 82c18c240a [SPARK-23030][SQL][PYTHON] Use Arrow stream format for creating from and collecting Pandas DataFrames
## What changes were proposed in this pull request?

This changes the calls of `toPandas()` and `createDataFrame()` to use the Arrow stream format, when Arrow is enabled.  Previously, Arrow data was written to byte arrays where each chunk is an output of the Arrow file format.  This was mainly due to constraints at the time, and caused some overhead by writing the schema/footer on each chunk of data and then having to read multiple Arrow file inputs and concat them together.

Using the Arrow stream format has improved these by increasing performance, lower memory overhead for the average case, and simplified the code.  Here are the details of this change:

**toPandas()**

_Before:_
Spark internal rows are converted to Arrow file format, each group of records is a complete Arrow file which contains the schema and other metadata.  Next a collect is done and an Array of Arrow files is the result.  After that each Arrow file is sent to Python driver which then loads each file and concats them to a single Arrow DataFrame.

_After:_
Spark internal rows are converted to ArrowRecordBatches directly, which is the simplest Arrow component for IPC data transfers.  The driver JVM then immediately starts serving data to Python as an Arrow stream, sending the schema first. It then starts a Spark job with a custom handler that sends Arrow RecordBatches to Python. Partitions arriving in order are sent immediately, and out-of-order partitions are buffered until the ones that precede it come in. This improves performance, simplifies memory usage on executors, and improves the average memory usage on the JVM driver.  Since the order of partitions must be preserved, the worst case is that the first partition will be the last to arrive all data must be buffered in memory until then. This case is no worse that before when doing a full collect.

**createDataFrame()**

_Before:_
A Pandas DataFrame is split into parts and each part is made into an Arrow file.  Then each file is prefixed by the buffer size and written to a temp file.  The temp file is read and each Arrow file is parallelized as a byte array.

_After:_
A Pandas DataFrame is split into parts, then an Arrow stream is written to a temp file where each part is an ArrowRecordBatch.  The temp file is read as a stream and the Arrow messages are examined.  If the message is an ArrowRecordBatch, the data is saved as a byte array.  After reading the file, each ArrowRecordBatch is parallelized as a byte array.  This has slightly more processing than before because we must look each Arrow message to extract the record batches, but performance ends up a litle better.  It is cleaner in the sense that IPC from Python to JVM is done over a single Arrow stream.

## How was this patch tested?

Added new unit tests for the additions to ArrowConverters in Scala, existing tests for Python.

## Performance Tests - toPandas

Tests run on a 4 node standalone cluster with 32 cores total, 14.04.1-Ubuntu and OpenJDK 8
measured wall clock time to execute `toPandas()` and took the average best time of 5 runs/5 loops each.

Test code
```python
df = spark.range(1 << 25, numPartitions=32).toDF("id").withColumn("x1", rand()).withColumn("x2", rand()).withColumn("x3", rand()).withColumn("x4", rand())
for i in range(5):
	start = time.time()
	_ = df.toPandas()
	elapsed = time.time() - start
```

Current Master | This PR
---------------------|------------
5.803557 | 5.16207
5.409119 | 5.133671
5.493509 | 5.147513
5.433107 | 5.105243
5.488757 | 5.018685

Avg Master | Avg This PR
------------------|--------------
5.5256098 | 5.1134364

Speedup of **1.08060595**

## Performance Tests - createDataFrame

Tests run on a 4 node standalone cluster with 32 cores total, 14.04.1-Ubuntu and OpenJDK 8
measured wall clock time to execute `createDataFrame()` and get the first record. Took the average best time of 5 runs/5 loops each.

Test code
```python
def run():
	pdf = pd.DataFrame(np.random.rand(10000000, 10))
	spark.createDataFrame(pdf).first()

for i in range(6):
	start = time.time()
	run()
	elapsed = time.time() - start
	gc.collect()
	print("Run %d: %f" % (i, elapsed))
```

Current Master | This PR
--------------------|----------
6.234608 | 5.665641
6.32144 | 5.3475
6.527859 | 5.370803
6.95089 | 5.479151
6.235046 | 5.529167

Avg Master | Avg This PR
---------------|----------------
6.4539686 | 5.4784524

Speedup of **1.178064192**

## Memory Improvements

**toPandas()**

The most significant improvement is reduction of the upper bound space complexity in the JVM driver.  Before, the entire dataset was collected in the JVM first before sending it to Python.  With this change, as soon as a partition is collected, the result handler immediately sends it to Python, so the upper bound is the size of the largest partition.  Also, using the Arrow stream format is more efficient because the schema is written once per stream, followed by record batches.  The schema is now only send from driver JVM to Python.  Before, multiple Arrow file formats were used that each contained the schema.  This duplicated schema was created in the executors, sent to the driver JVM, and then Python where all but the first one received are discarded.

I verified the upper bound limit by running a test that would collect data that would exceed the amount of driver JVM memory available.  Using these settings on a standalone cluster:
```
spark.driver.memory 1g
spark.executor.memory 5g
spark.sql.execution.arrow.enabled true
spark.sql.execution.arrow.fallback.enabled false
spark.sql.execution.arrow.maxRecordsPerBatch 0
spark.driver.maxResultSize 2g
```

Test code:
```python
from pyspark.sql.functions import rand
df = spark.range(1 << 25, numPartitions=32).toDF("id").withColumn("x1", rand()).withColumn("x2", rand()).withColumn("x3", rand())
df.toPandas()
```

This makes total data size of 33554432×8×4 = 1073741824

With the current master, it fails with OOM but passes using this PR.

**createDataFrame()**

No significant change in memory except that using the stream format instead of separate file formats avoids duplicated the schema, similar to toPandas above.  The process of reading the stream and parallelizing the batches does cause the record batch message metadata to be copied, but it's size is insignificant.

Closes #21546 from BryanCutler/arrow-toPandas-stream-SPARK-23030.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-29 15:01:12 +08:00
Bogdan Raducanu 103854028e [SPARK-25212][SQL] Support Filter in ConvertToLocalRelation
## What changes were proposed in this pull request?
Support Filter in ConvertToLocalRelation, similar to how Project works.
Additionally, in Optimizer, run ConvertToLocalRelation earlier to simplify the plan. This is good for very short queries which often are queries on local relations.

## How was this patch tested?
New test. Manual benchmark.

Author: Bogdan Raducanu <bogdan@databricks.com>
Author: Shixiong Zhu <zsxwing@gmail.com>
Author: Yinan Li <ynli@google.com>
Author: Li Jin <ice.xelloss@gmail.com>
Author: s71955 <sujithchacko.2010@gmail.com>
Author: DB Tsai <d_tsai@apple.com>
Author: jaroslav chládek <mastermism@gmail.com>
Author: Huangweizhe <huangweizhe@bbdservice.com>
Author: Xiangrui Meng <meng@databricks.com>
Author: hyukjinkwon <gurwls223@apache.org>
Author: Kent Yao <yaooqinn@hotmail.com>
Author: caoxuewen <cao.xuewen@zte.com.cn>
Author: liuxian <liu.xian3@zte.com.cn>
Author: Adam Bradbury <abradbury@users.noreply.github.com>
Author: Jose Torres <torres.joseph.f+github@gmail.com>
Author: Yuming Wang <yumwang@ebay.com>
Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #22205 from bogdanrdc/local-relation-filter.
2018-08-28 15:50:25 -07:00
Ryan Blue 7ad18ee9f2 [SPARK-25004][CORE] Add spark.executor.pyspark.memory limit.
## What changes were proposed in this pull request?

This adds `spark.executor.pyspark.memory` to configure Python's address space limit, [`resource.RLIMIT_AS`](https://docs.python.org/3/library/resource.html#resource.RLIMIT_AS). Limiting Python's address space allows Python to participate in memory management. In practice, we see fewer cases of Python taking too much memory because it doesn't know to run garbage collection. This results in YARN killing fewer containers. This also improves error messages so users know that Python is consuming too much memory:

```
  File "build/bdist.linux-x86_64/egg/package/library.py", line 265, in fe_engineer
    fe_eval_rec.update(f(src_rec_prep, mat_rec_prep))
  File "build/bdist.linux-x86_64/egg/package/library.py", line 163, in fe_comp
    comparisons = EvaluationUtils.leven_list_compare(src_rec_prep.get(item, []), mat_rec_prep.get(item, []))
  File "build/bdist.linux-x86_64/egg/package/evaluationutils.py", line 25, in leven_list_compare
    permutations = sorted(permutations, reverse=True)
  MemoryError
```

The new pyspark memory setting is used to increase requested YARN container memory, instead of sharing overhead memory between python and off-heap JVM activity.

## How was this patch tested?

Tested memory limits in our YARN cluster and verified that MemoryError is thrown.

Author: Ryan Blue <blue@apache.org>

Closes #21977 from rdblue/SPARK-25004-add-python-memory-limit.
2018-08-28 12:31:33 -07:00
Maxim Gekk aff8f15c15 [SPARK-25240][SQL] Fix for a deadlock in RECOVER PARTITIONS
## What changes were proposed in this pull request?

In the PR, I propose to not perform recursive parallel listening of files in the `scanPartitions` method because it can cause a deadlock. Instead of that I propose to do `scanPartitions` in parallel for top level partitions only.

## How was this patch tested?

I extended an existing test to trigger the deadlock.

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

Closes #22233 from MaxGekk/fix-recover-partitions.
2018-08-28 11:29:05 -07:00
Fernando Pereira de46df549a [SPARK-23997][SQL] Configurable maximum number of buckets
## What changes were proposed in this pull request?
This PR implements the possibility of the user to override the maximum number of buckets when saving to a table.
Currently the limit is a hard-coded 100k, which might be insufficient for large workloads.
A new configuration entry is proposed: `spark.sql.bucketing.maxBuckets`, which defaults to the previous 100k.

## How was this patch tested?
Added unit tests in the following spark.sql test suites:

- CreateTableAsSelectSuite
- BucketedWriteSuite

Author: Fernando Pereira <fernando.pereira@epfl.ch>

Closes #21087 from ferdonline/enh/configurable_bucket_limit.
2018-08-28 10:31:47 -07:00
Li Jin 8198ea5019 [SPARK-24721][SQL] Exclude Python UDFs filters in FileSourceStrategy
## What changes were proposed in this pull request?
The PR excludes Python UDFs filters in FileSourceStrategy so that they don't ExtractPythonUDF rule to throw exception. It doesn't make sense to pass Python UDF filters in FileSourceStrategy anyway because they cannot be used as push down filters.

## How was this patch tested?
Add a new regression test

Closes #22104 from icexelloss/SPARK-24721-udf-filter.

Authored-by: Li Jin <ice.xelloss@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-28 10:57:13 +08:00
caoxuewen 6193a202aa [SPARK-24978][SQL] Add spark.sql.fast.hash.aggregate.row.max.capacity to configure the capacity of fast aggregation.
## What changes were proposed in this pull request?

this pr add a configuration parameter to configure the capacity of fast aggregation.
Performance comparison:

```
 Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Windows 7 6.1
 Intel64 Family 6 Model 94 Stepping 3, GenuineIntel
 Aggregate w multiple keys:               Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
 ------------------------------------------------------------------------------------------------
 fasthash = default                            5612 / 5882          3.7         267.6       1.0X
 fasthash = config                             3586 / 3595          5.8         171.0       1.6X

```

## How was this patch tested?
the existed test cases.

Closes #21931 from heary-cao/FastHashCapacity.

Authored-by: caoxuewen <cao.xuewen@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-27 15:45:48 +08:00
Liang-Chi Hsieh 5c27b0d4f8 [SPARK-19355][SQL][FOLLOWUP] Remove the child.outputOrdering check in global limit
## What changes were proposed in this pull request?

This is based on the discussion https://github.com/apache/spark/pull/16677/files#r212805327.

As SQL standard doesn't mandate that a nested order by followed by a limit has to respect that ordering clause, this patch removes the `child.outputOrdering` check.

## How was this patch tested?

Unit tests.

Closes #22239 from viirya/improve-global-limit-parallelism-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-27 14:02:50 +08:00
Adam Bradbury ad43e2c1e8 [SPARK-23792][DOCS] Documentation improvements for datetime functions
## What changes were proposed in this pull request?

Improved the documentation for the datetime functions in `org.apache.spark.sql.functions` by adding details about the supported column input types, the column return type, behaviour on invalid input, supporting examples and clarifications.

## How was this patch tested?

Manually testing each of the datetime functions with different input to ensure that the corresponding Javadoc/Scaladoc matches the behaviour of the function. Successfully ran the `unidoc` SBT process.

Closes #20901 from abradbury/SPARK-23792.

Authored-by: Adam Bradbury <abradbury@users.noreply.github.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-08-26 08:37:52 -05:00
Kazuaki Ishizaki ab33028957 [SPARK-25178][SQL] Directly ship the StructType objects of the keySchema / valueSchema for xxxHashMapGenerator
## What changes were proposed in this pull request?

This PR generates the code that to refer a `StructType` generated in the scala code instead of generating `StructType` in Java code.

The original code has two issues.
1. Avoid to used the field name such as `key.name`
1. Support complicated schema (e.g. nested DataType)

At first, [the JIRA entry](https://issues.apache.org/jira/browse/SPARK-25178) proposed to change the generated field name of the keySchema / valueSchema to a dummy name in `RowBasedHashMapGenerator` and `VectorizedHashMapGenerator.scala`. This proposal can addresse issue 1.

Ueshin suggested an approach to refer to a `StructType` generated in the scala code using `ctx.addReferenceObj()`. This approach can address issues 1 and 2. Finally, this PR uses this approach.

## How was this patch tested?

Existing UTs

Closes #22187 from kiszk/SPARK-25178.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2018-08-24 14:58:55 +09:00
Michael Allman f2d35427ee [SPARK-4502][SQL] Parquet nested column pruning - foundation
(Link to Jira: https://issues.apache.org/jira/browse/SPARK-4502)

_N.B. This is a restart of PR #16578 which includes a subset of that code. Relevant review comments from that PR should be considered incorporated by reference. Please avoid duplication in review by reviewing that PR first. The summary below is an edited copy of the summary of the previous PR._

## What changes were proposed in this pull request?

One of the hallmarks of a column-oriented data storage format is the ability to read data from a subset of columns, efficiently skipping reads from other columns. Spark has long had support for pruning unneeded top-level schema fields from the scan of a parquet file. For example, consider a table, `contacts`, backed by parquet with the following Spark SQL schema:

```
root
 |-- name: struct
 |    |-- first: string
 |    |-- last: string
 |-- address: string
```

Parquet stores this table's data in three physical columns: `name.first`, `name.last` and `address`. To answer the query

```SQL
select address from contacts
```

Spark will read only from the `address` column of parquet data. However, to answer the query

```SQL
select name.first from contacts
```

Spark will read `name.first` and `name.last` from parquet.

This PR modifies Spark SQL to support a finer-grain of schema pruning. With this patch, Spark reads only the `name.first` column to answer the previous query.

### Implementation

There are two main components of this patch. First, there is a `ParquetSchemaPruning` optimizer rule for gathering the required schema fields of a `PhysicalOperation` over a parquet file, constructing a new schema based on those required fields and rewriting the plan in terms of that pruned schema. The pruned schema fields are pushed down to the parquet requested read schema. `ParquetSchemaPruning` uses a new `ProjectionOverSchema` extractor for rewriting a catalyst expression in terms of a pruned schema.

Second, the `ParquetRowConverter` has been patched to ensure the ordinals of the parquet columns read are correct for the pruned schema. `ParquetReadSupport` has been patched to address a compatibility mismatch between Spark's built in vectorized reader and the parquet-mr library's reader.

### Limitation

Among the complex Spark SQL data types, this patch supports parquet column pruning of nested sequences of struct fields only.

## How was this patch tested?

Care has been taken to ensure correctness and prevent regressions. A more advanced version of this patch incorporating optimizations for rewriting queries involving aggregations and joins has been running on a production Spark cluster at VideoAmp for several years. In that time, one bug was found and fixed early on, and we added a regression test for that bug.

We forward-ported this patch to Spark master in June 2016 and have been running this patch against Spark 2.x branches on ad-hoc clusters since then.

Closes #21320 from mallman/spark-4502-parquet_column_pruning-foundation.

Lead-authored-by: Michael Allman <msa@allman.ms>
Co-authored-by: Adam Jacques <adam@technowizardry.net>
Co-authored-by: Michael Allman <michael@videoamp.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2018-08-23 21:31:10 -07:00
Bogdan Raducanu cd6dff78be [SPARK-25209][SQL] Avoid deserializer check in Dataset.apply when Dataset is actually DataFrame
## What changes were proposed in this pull request?
Dataset.apply calls dataset.deserializer (to provide an early error) which ends up calling the full Analyzer on the deserializer. This can take tens of milliseconds, depending on how big the plan is.
Since Dataset.apply is called for many Dataset operations such as Dataset.where it can be a significant overhead for short queries.
According to a comment in the PR that introduced this check, we can at least remove this check for DataFrames: https://github.com/apache/spark/pull/20402#discussion_r164338267

## How was this patch tested?
Existing tests + manual benchmark

Author: Bogdan Raducanu <bogdan@databricks.com>

Closes #22201 from bogdanrdc/deserializer-fix.
2018-08-24 04:13:07 +02:00
s71955 b88ddb8a83 [SPARK-23425][SQL] Support wildcard in HDFS path for load table command
## What changes were proposed in this pull request?
**Problem statement**
load data command  with hdfs  file paths consists of  wild card strings like * are not working
eg:
"load data inpath 'hdfs://hacluster/user/ext*  into table t1"
throws Analysis exception while executing this query

![wildcard_issue](https://user-images.githubusercontent.com/12999161/42673744-9f5c0c16-8621-11e8-8d28-cdc41bbe6efe.PNG)

**Analysis -**
Currently fs.exists() API which is used for path validation in load command API cannot resolve the path with wild card pattern, To mitigate this problem i am using globStatus() API  another api  which can resolve the paths with hdfs supported wildcards like *,? etc(inline with hive wildcard support).

**Improvement identified as part of this issue -**
Currently system wont support wildcard character to be used for folder level path in a local file system.  This PR has handled this scenario, the same globStatus API will unify the validation logic of local and non local file systems, this will ensure the behavior consistency between the hdfs and local file path in  load command.

with this improvement user will be able to use a wildcard character in folder level path of a local file system in  load command inline with hive behaviour, in older versions user can use wildcards only in file path of the local file system if they use in folder path system use to give an error by mentioning that not supported.
eg: load data local  inpath '/localfilesystem/folder*  into table t1

## How was this patch tested?
a) Manually tested by executing test-cases in HDFS yarn cluster.  Reports is been attached in below section.
b) Existing test-case can verify the impact and functionality  for local file path scenarios
c) A test-case is been added for verifying the functionality when wild card is been used in folder level path of a local file system
## Test Results
Note: all ip's were updated to localhost for security reasons.
HDFS path details
```
vm1:/opt/ficlient # hadoop fs -ls /user/data/sujith1
Found 2 items
-rw-r--r--   3 shahid hadoop       4802 2018-03-26 15:45 /user/data/sujith1/typeddata60.txt
-rw-r--r--   3 shahid hadoop       4883 2018-03-26 15:45 /user/data/sujith1/typeddata61.txt
vm1:/opt/ficlient # hadoop fs -ls /user/data/sujith2
Found 2 items
-rw-r--r--   3 shahid hadoop       4802 2018-03-26 15:45 /user/data/sujith2/typeddata60.txt
-rw-r--r--   3 shahid hadoop       4883 2018-03-26 15:45 /user/data/sujith2/typeddata61.txt
```
positive scenario by specifying complete file path to know about record size
```
0: jdbc:hive2://localhost:22550/default> create table wild_spark (time timestamp, name string, isright boolean, datetoday date, num binary, height double, score float, decimaler decimal(10,0), id tinyint, age int, license bigint, length smallint) row format delimited fields terminated by ',';
+---------+--+
| Result  |
+---------+--+
+---------+--+
No rows selected (1.217 seconds)
0: jdbc:hive2://localhost:22550/default> load data  inpath '/user/data/sujith1/typeddata60.txt' into table wild_spark;
+---------+--+
| Result  |
+---------+--+
+---------+--+
No rows selected (4.236 seconds)
0: jdbc:hive2://localhost:22550/default> load data  inpath '/user/data/sujith1/typeddata61.txt' into table wild_spark;
+---------+--+
| Result  |
+---------+--+
+---------+--+
No rows selected (0.602 seconds)
0: jdbc:hive2://localhost:22550/default> select count(*) from wild_spark;
+-----------+--+
| count(1)  |
+-----------+--+
| 121       |
+-----------+--+
1 row selected (18.529 seconds)
0: jdbc:hive2://localhost:22550/default>
```
With wild card character in file path
```
0: jdbc:hive2://localhost:22550/default> create table spark_withWildChar (time timestamp, name string, isright boolean, datetoday date, num binary, height double, score float, decimaler decimal(10,0), id tinyint, age int, license bigint, length smallint) row format delimited fields terminated by ',';
+---------+--+
| Result  |
+---------+--+
+---------+--+
No rows selected (0.409 seconds)
0: jdbc:hive2://localhost:22550/default> load data  inpath '/user/data/sujith1/type*' into table spark_withWildChar;
+---------+--+
| Result  |
+---------+--+
+---------+--+
No rows selected (1.502 seconds)
0: jdbc:hive2://localhost:22550/default> select count(*) from spark_withWildChar;
+-----------+--+
| count(1)  |
+-----------+--+
| 121       |
+-----------+--+
```
with ? wild card scenario
```
0: jdbc:hive2://localhost:22550/default> create table spark_withWildChar_DiffChar (time timestamp, name string, isright boolean, datetoday date, num binary, height double, score float, decimaler decimal(10,0), id tinyint, age int, license bigint, length smallint) row format delimited fields terminated by ',';
+---------+--+
| Result  |
+---------+--+
+---------+--+
No rows selected (0.489 seconds)
0: jdbc:hive2://localhost:22550/default> load data  inpath '/user/data/sujith1/?ypeddata60.txt' into table spark_withWildChar_DiffChar;
+---------+--+
| Result  |
+---------+--+
+---------+--+
No rows selected (1.152 seconds)
0: jdbc:hive2://localhost:22550/default> load data  inpath '/user/data/sujith1/?ypeddata61.txt' into table spark_withWildChar_DiffChar;
+---------+--+
| Result  |
+---------+--+
+---------+--+
No rows selected (0.644 seconds)
0: jdbc:hive2://localhost:22550/default> select count(*) from spark_withWildChar_DiffChar;
+-----------+--+
| count(1)  |
+-----------+--+
| 121       |
+-----------+--+
1 row selected (16.078 seconds)
```
with  folder level wild card scenario
```
0: jdbc:hive2://localhost:22550/default> create table spark_withWildChar_folderlevel (time timestamp, name string, isright boolean, datetoday date, num binary, height double, score float, decimaler decimal(10,0), id tinyint, age int, license bigint, length smallint) row format delimited fields terminated by ',';
+---------+--+
| Result  |
+---------+--+
+---------+--+
No rows selected (0.489 seconds)
0: jdbc:hive2://localhost:22550/default> load data  inpath '/user/data/suji*/*' into table spark_withWildChar_folderlevel;
+---------+--+
| Result  |
+---------+--+
+---------+--+
No rows selected (1.152 seconds)

0: jdbc:hive2://localhost:22550/default> select count(*) from spark_withWildChar_folderlevel;
+-----------+--+
| count(1)  |
+-----------+--+
| 242       |
+-----------+--+
1 row selected (16.078 seconds)
```
Negative scenario invalid path
```
0: jdbc:hive2://localhost:22550/default> load data  inpath '/user/data/sujiinvalid*/*' into  table spark_withWildChar_folder;
Error: org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: /user/data/sujiinvalid*/*; (state=,code=0)
0: jdbc:hive2://localhost:22550/default>
```
Hive Test results- file level
```
0: jdbc:hive2://localhost:21066/> create table hive_withWildChar_files (time timestamp, name string, isright boolean, datetoday date, num binary, height double, score float, decimaler decimal(10,0), id tinyint, age int, license bigint, length smallint) stored as TEXTFILE;
No rows affected (0.723 seconds)
0: jdbc:hive2://localhost:21066/> load data  inpath '/user/data/sujith1/type*'  into  table hive_withWildChar_files;
INFO  : Loading data to table default.hive_withwildchar_files from hdfs://hacluster/user/sujith1/type*
No rows affected (0.682 seconds)
0: jdbc:hive2://localhost:21066/> select count(*) from hive_withWildChar_files;
+------+--+
| _c0  |
+------+--+
| 121  |
+------+--+
1 row selected (50.832 seconds)
```
Hive Test results- folder level
```
0: jdbc:hive2://localhost:21066/> create table hive_withWildChar_folder (time timestamp, name string, isright boolean, datetoday date, num binary, height double, score float, decimaler decimal(10,0), id tinyint, age int, license bigint, length smallint) stored as TEXTFILE;
No rows affected (0.459 seconds)
0: jdbc:hive2://localhost:21066/> load data  inpath '/user/data/suji*/*' into table hive_withWildChar_folder;
INFO  : Loading data to table default.hive_withwildchar_folder from hdfs://hacluster/user/data/suji*/*
No rows affected (0.76 seconds)
0: jdbc:hive2://localhost:21066/> select count(*) from hive_withWildChar_folder;
+------+--+
| _c0  |
+------+--+
| 242  |
+------+--+
1 row selected (46.483 seconds)
```

Closes #20611 from sujith71955/master_wldcardsupport.

Lead-authored-by: s71955 <sujithchacko.2010@gmail.com>
Co-authored-by: sujith71955 <sujithchacko.2010@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-24 09:54:30 +08:00
Jose Torres 8ed0449285 [SPARK-25204][SS] Fix race in rate source test.
## What changes were proposed in this pull request?

Fix a race in the rate source tests. We need a better way of testing restart behavior.

## How was this patch tested?

unit test

Closes #22191 from jose-torres/racetest.

Authored-by: Jose Torres <torres.joseph.f+github@gmail.com>
Signed-off-by: Tathagata Das <tathagata.das1565@gmail.com>
2018-08-23 12:14:27 -07:00
Takuya UESHIN a9aacdf1c2 [SPARK-25208][SQL] Loosen Cast.forceNullable for DecimalType.
## What changes were proposed in this pull request?

Casting to `DecimalType` is not always needed to force nullable.
If the decimal type to cast is wider than original type, or only truncating or precision loss, the casted value won't be `null`.

## How was this patch tested?

Added and modified tests.

Closes #22200 from ueshin/issues/SPARK-25208/cast_nullable_decimal.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-23 22:48:26 +08:00
Rao Fu 5d572fc7c3 [SPARK-25126][SQL] Avoid creating Reader for all orc files
## What changes were proposed in this pull request?

[SPARK-25126] (https://issues.apache.org/jira/browse/SPARK-25126)
reports loading a large number of orc files consumes a lot of memory
in both 2.0 and 2.3. The issue is caused by creating a Reader for every
orc file in order to infer the schema.

In OrFileOperator.ReadSchema, a Reader is created for every file
although only the first valid one is used. This uses significant
amount of memory when there `paths` have a lot of files. In 2.3
a different code path (OrcUtils.readSchema) is used for inferring
schema for orc files. This commit changes both functions to create
Reader lazily.

## How was this patch tested?

Pass the Jenkins with a newly added test case by dongjoon-hyun

Closes #22157 from raofu/SPARK-25126.

Lead-authored-by: Rao Fu <rao@coupang.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Co-authored-by: Rao Fu <raofu04@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-23 22:00:20 +08:00
Bruce Robbins 8cc591c91a [SPARK-25164][SQL] Avoid rebuilding column and path list for each column in parquet reader
## What changes were proposed in this pull request?

VectorizedParquetRecordReader::initializeInternal rebuilds the column list and path list once for each column. Therefore, it indirectly iterates 2\*colCount\*colCount times for each parquet file.

This inefficiency impacts jobs that read parquet-backed tables with many columns and many files. Jobs that read tables with few columns or few files are not impacted.

This PR changes initializeInternal so that it builds each list only once.

I ran benchmarks on my laptop with 1 worker thread, running this query:
<pre>
sql("select * from parquet_backed_table where id1 = 1").collect
</pre>
There are roughly one matching row for every 425 rows, and the matching rows are sprinkled pretty evenly throughout the table (that is, every page for column <code>id1</code> has at least one matching row).

6000 columns, 1 million rows, 67 32M files:

master | branch | improvement
-------|---------|-----------
10.87 min | 6.09 min | 44%

6000 columns, 1 million rows, 23 98m files:

master | branch | improvement
-------|---------|-----------
7.39 min | 5.80 min | 21%

600 columns 10 million rows, 67 32M files:

master | branch | improvement
-------|---------|-----------
1.95 min | 1.96 min | -0.5%

60 columns, 100 million rows, 67 32M files:

master | branch | improvement
-------|---------|-----------
0.55 min | 0.55 min | 0%

## How was this patch tested?

- sql unit tests
- pyspark-sql tests

Closes #22188 from bersprockets/SPARK-25164.

Authored-by: Bruce Robbins <bersprockets@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-23 14:52:23 +08:00
Takeshi Yamamuro 2a0a8f753b [SPARK-23034][SQL] Show RDD/relation names in RDD/Hive table scan nodes
## What changes were proposed in this pull request?
This pr proposed to show RDD/relation names in RDD/Hive table scan nodes.
This change made these names show up in the webUI and explain results.
For example;
```
scala> sql("CREATE TABLE t(c1 int) USING hive")
scala> sql("INSERT INTO t VALUES(1)")
scala> spark.table("t").explain()
== Physical Plan ==
Scan hive default.t [c1#8], HiveTableRelation `default`.`t`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [c1#8]
         ^^^^^^^^^^^
```
<img width="212" alt="spark-pr-hive" src="https://user-images.githubusercontent.com/692303/44501013-51264c80-a6c6-11e8-94f8-0704aee83bb6.png">

Closes #20226

## How was this patch tested?
Added tests in `DataFrameSuite`, `DatasetSuite`, and `HiveExplainSuite`

Closes #22153 from maropu/pr20226.

Lead-authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Co-authored-by: Tejas Patil <tejasp@fb.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-23 14:26:10 +08:00
Reynold Xin 0295ad40de [SPARK-25127] DataSourceV2: Remove SupportsPushDownCatalystFilters
## What changes were proposed in this pull request?
They depend on internal Expression APIs. Let's see how far we can get without it.

## How was this patch tested?
Just some code removal. There's no existing tests as far as I can tell so it's easy to remove.

Closes #22185 from rxin/SPARK-25127.

Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-23 08:10:45 +08:00
Tathagata Das 3106324986 [SPARK-25184][SS] Fixed race condition in StreamExecution that caused flaky test in FlatMapGroupsWithState
## What changes were proposed in this pull request?

The race condition that caused test failure is between 2 threads.
- The MicrobatchExecution thread that processes inputs to produce answers and then generates progress events.
- The test thread that generates some input data, checked the answer and then verified the query generated progress event.

The synchronization structure between these threads is as follows
1. MicrobatchExecution thread, in every batch, does the following in order.
   a. Processes batch input to generate answer.
   b. Signals `awaitProgressLockCondition` to wake up threads waiting for progress using `awaitOffset`
   c. Generates progress event

2. Test execution thread
   a. Calls `awaitOffset` to wait for progress, which waits on `awaitProgressLockCondition`.
   b. As soon as `awaitProgressLockCondition` is signaled, it would move on the in the test to check answer.
  c. Finally, it would verify the last generated progress event.

What can happen is the following sequence of events: 2a -> 1a -> 1b -> 2b -> 2c -> 1c.
In other words, the progress event may be generated after the test tries to verify it.

The solution has two steps.
1. Signal the waiting thread after the progress event has been generated, that is, after `finishTrigger()`.
2. Increase the timeout of `awaitProgressLockCondition.await(100 ms)` to a large value.

This latter is to ensure that test thread for keeps waiting on `awaitProgressLockCondition`until the MicroBatchExecution thread explicitly signals it. With the existing small timeout of 100ms the following sequence can occur.
 - MicroBatchExecution thread updates committed offsets
 - Test thread waiting on `awaitProgressLockCondition` accidentally times out after 100 ms, finds that the committed offsets have been updated, therefore returns from `awaitOffset` and moves on to the progress event tests.
 - MicroBatchExecution thread then generates progress event and signals. But the test thread has already attempted to verify the event and failed.

By increasing the timeout to large (e.g., `streamingTimeoutMs = 60 seconds`, similar to `awaitInitialization`), this above type of race condition is also avoided.

## How was this patch tested?
Ran locally many times.

Closes #22182 from tdas/SPARK-25184.

Authored-by: Tathagata Das <tathagata.das1565@gmail.com>
Signed-off-by: Tathagata Das <tathagata.das1565@gmail.com>
2018-08-22 12:22:53 -07:00
Wenchen Fan e754887182 [SPARK-24882][SQL] improve data source v2 API
## What changes were proposed in this pull request?

Improve the data source v2 API according to the [design doc](https://docs.google.com/document/d/1DDXCTCrup4bKWByTalkXWgavcPdvur8a4eEu8x1BzPM/edit?usp=sharing)

summary of the changes
1. rename `ReadSupport` -> `DataSourceReader` -> `InputPartition` -> `InputPartitionReader` to `BatchReadSupportProvider` -> `BatchReadSupport` -> `InputPartition`/`PartitionReaderFactory` -> `PartitionReader`. Similar renaming also happens at streaming and write APIs.
2. create `ScanConfig` to store query specific information like operator pushdown result, streaming offsets, etc. This makes batch and streaming `ReadSupport`(previouslly named `DataSourceReader`) immutable. All other methods take `ScanConfig` as input, which implies applying operator pushdown and getting streaming offsets happen before all other things(get input partitions, report statistics, etc.).
3. separate `InputPartition` to `InputPartition` and `PartitionReaderFactory`. This is a natural separation, data splitting and reading are orthogonal and we should not mix them in one interfaces. This also makes the naming consistent between read and write API: `PartitionReaderFactory` vs `DataWriterFactory`.
4. separate the batch and streaming interfaces. Sometimes it's painful to force the streaming interface to extend batch interface, as we may need to override some batch methods to return false, or even leak the streaming concept to batch API(e.g. `DataWriterFactory#createWriter(partitionId, taskId, epochId)`)

Some follow-ups we should do after this PR (tracked by https://issues.apache.org/jira/browse/SPARK-25186 ):
1. Revisit the life cycle of `ReadSupport` instances. Currently I keep it same as the previous `DataSourceReader`, i.e. the life cycle is bound to the batch/stream query. This fits streaming very well but may not be perfect for batch source. We can also consider to let `ReadSupport.newScanConfigBuilder` take `DataSourceOptions` as parameter, if we decide to change the life cycle.
2. Add `WriteConfig`. This is similar to `ScanConfig` and makes the write API more flexible. But it's only needed when we add the `replaceWhere` support, and it needs to change the streaming execution engine for this new concept, which I think is better to be done in another PR.
3. Refine the document. This PR adds/changes a lot of document and it's very likely that some people may have better ideas.
4. Figure out the life cycle of `CustomMetrics`. It looks to me that it should be bound to a `ScanConfig`, but we need to change `ProgressReporter` to get the `ScanConfig`. Better to be done in another PR.
5. Better operator pushdown API. This PR keeps the pushdown API as it was, i.e. using the `SupportsPushdownXYZ` traits. We can design a better API using build pattern, but this is a complicated design and deserves an individual JIRA ticket and design doc.
6. Improve the continuous streaming engine to only create a new `ScanConfig` when re-configuring.
7. Remove `SupportsPushdownCatalystFilter`. This is actually not a must-have for file source, we can change the hive partition pruning to use the public `Filter`.

## How was this patch tested?

existing tests.

Closes #22009 from cloud-fan/redesign.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2018-08-22 00:10:55 -07:00
Wenchen Fan 4a9c9d8f9a [SPARK-25159][SQL] json schema inference should only trigger one job
## What changes were proposed in this pull request?

This fixes a perf regression caused by https://github.com/apache/spark/pull/21376 .

We should not use `RDD#toLocalIterator`, which triggers one Spark job per RDD partition. This is very bad for RDDs with a lot of small partitions.

To fix it, this PR introduces a way to access SQLConf in the scheduler event loop thread, so that we don't need to use `RDD#toLocalIterator` anymore in `JsonInferSchema`.

## How was this patch tested?

a new test

Closes #22152 from cloud-fan/conf.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2018-08-21 22:21:08 -07:00
Takeshi Yamamuro 07737c87d6 [SPARK-23711][SPARK-25140][SQL] Catch correct exceptions when expr codegen fails
## What changes were proposed in this pull request?
This pr is to fix bugs when expr codegen fails; we need to catch `java.util.concurrent.ExecutionException` instead of `InternalCompilerException` and `CompileException` . This handling is the same with the `WholeStageCodegenExec ` one: 60af2501e1/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala (L585)

## How was this patch tested?
Added tests in `CodeGeneratorWithInterpretedFallbackSuite`

Closes #22154 from maropu/SPARK-25140.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2018-08-21 22:17:44 -07:00
Tathagata Das a998e9d829 [MINOR] Added import to fix compilation
## What changes were proposed in this pull request?

Two back to PRs implicitly conflicted by one PR removing an existing import that the other PR needed. This did not cause explicit conflict as the import already existed, but not used.

https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Compile/job/spark-master-compile-maven-hadoop-2.7/8226/consoleFull

```
[info] Compiling 342 Scala sources and 97 Java sources to /home/jenkins/workspace/spark-master-compile-maven-hadoop-2.7/sql/core/target/scala-2.11/classes...
[warn] /home/jenkins/workspace/spark-master-compile-maven-hadoop-2.7/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala:128: value ENABLE_JOB_SUMMARY in object ParquetOutputFormat is deprecated: see corresponding Javadoc for more information.
[warn]       && conf.get(ParquetOutputFormat.ENABLE_JOB_SUMMARY) == null) {
[warn]                                       ^
[error] /home/jenkins/workspace/spark-master-compile-maven-hadoop-2.7/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala:95: value asJava is not a member of scala.collection.immutable.Map[String,Long]
[error]       new java.util.HashMap(customMetrics.mapValues(long2Long).asJava)
[error]                                                                ^
[warn] one warning found
[error] one error found
[error] Compile failed at Aug 21, 2018 4:04:35 PM [12.827s]
```

## How was this patch tested?
It compiles!

Closes #22175 from tdas/fix-build.

Authored-by: Tathagata Das <tathagata.das1565@gmail.com>
Signed-off-by: Tathagata Das <tathagata.das1565@gmail.com>
2018-08-21 17:08:15 -07:00
Jungtaek Lim 42035a4fec [SPARK-24441][SS] Expose total estimated size of states in HDFSBackedStateStoreProvider
## What changes were proposed in this pull request?

This patch exposes the estimation of size of cache (loadedMaps) in HDFSBackedStateStoreProvider as a custom metric of StateStore.

The rationalize of the patch is that state backed by HDFSBackedStateStoreProvider will consume more memory than the number what we can get from query status due to caching multiple versions of states. The memory footprint to be much larger than query status reports in situations where the state store is getting a lot of updates: while shallow-copying map incurs additional small memory usages due to the size of map entities and references, but row objects will still be shared across the versions. If there're lots of updates between batches, less row objects will be shared and more row objects will exist in memory consuming much memory then what we expect.

While HDFSBackedStateStore refers loadedMaps in HDFSBackedStateStoreProvider directly, there would be only one `StateStoreWriter` which refers a StateStoreProvider, so the value is not exposed as well as being aggregated multiple times. Current state metrics are safe to aggregate for the same reason.

## How was this patch tested?

Tested manually. Below is the snapshot of UI page which is reflected by the patch:

<img width="601" alt="screen shot 2018-06-05 at 10 16 16 pm" src="https://user-images.githubusercontent.com/1317309/40978481-b46ad324-690e-11e8-9b0f-e80528612a62.png">

Please refer "estimated size of states cache in provider total" as well as "count of versions in state cache in provider".

Closes #21469 from HeartSaVioR/SPARK-24441.

Authored-by: Jungtaek Lim <kabhwan@gmail.com>
Signed-off-by: Tathagata Das <tathagata.das1565@gmail.com>
2018-08-21 15:28:31 -07:00
Gengliang Wang ac0174e55a [SPARK-25129][SQL] Make the mapping of com.databricks.spark.avro to built-in module configurable
## What changes were proposed in this pull request?

In https://issues.apache.org/jira/browse/SPARK-24924, the data source provider com.databricks.spark.avro is mapped to the new package org.apache.spark.sql.avro .

As per the discussion in the [Jira](https://issues.apache.org/jira/browse/SPARK-24924) and PR #22119, we should make the mapping configurable.

This PR also improve the error message when data source of Avro/Kafka is not found.

## How was this patch tested?

Unit test

Closes #22133 from gengliangwang/configurable_avro_mapping.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2018-08-21 15:26:24 -07:00
Jungtaek Lim 6c5cb85856 [SPARK-24763][SS] Remove redundant key data from value in streaming aggregation
## What changes were proposed in this pull request?

This patch proposes a new flag option for stateful aggregation: remove redundant key data from value.
Enabling new option runs similar with current, and uses less memory for state according to key/value fields of state operator.

Please refer below link to see detailed perf. test result:
https://issues.apache.org/jira/browse/SPARK-24763?focusedCommentId=16536539&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16536539

Since the state between enabling the option and disabling the option is not compatible, the option is set to 'disable' by default (to ensure backward compatibility), and OffsetSeqMetadata would prevent modifying the option after executing query.

## How was this patch tested?

Modify unit tests to cover both disabling option and enabling option.
Also did manual tests to see whether propose patch improves state memory usage.

Closes #21733 from HeartSaVioR/SPARK-24763.

Authored-by: Jungtaek Lim <kabhwan@gmail.com>
Signed-off-by: Tathagata Das <tathagata.das1565@gmail.com>
2018-08-21 15:22:42 -07:00
Xingbo Jiang 4fb96e5105 [SPARK-25114][CORE] Fix RecordBinaryComparator when subtraction between two words is divisible by Integer.MAX_VALUE.
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/22079#discussion_r209705612 It is possible for two objects to be unequal and yet we consider them as equal with this code, if the long values are separated by Int.MaxValue.
This PR fixes the issue.

## How was this patch tested?
Add new test cases in `RecordBinaryComparatorSuite`.

Closes #22101 from jiangxb1987/fix-rbc.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2018-08-20 23:13:31 -07:00
seancxmao f984ec75ed [SPARK-25132][SQL] Case-insensitive field resolution when reading from Parquet
## What changes were proposed in this pull request?
Spark SQL returns NULL for a column whose Hive metastore schema and Parquet schema are in different letter cases, regardless of spark.sql.caseSensitive set to true or false. This PR aims to add case-insensitive field resolution for ParquetFileFormat.
* Do case-insensitive resolution only if Spark is in case-insensitive mode.
* Field resolution should fail if there is ambiguity, i.e. more than one field is matched.

## How was this patch tested?
Unit tests added.

Closes #22148 from seancxmao/SPARK-25132-Parquet.

Authored-by: seancxmao <seancxmao@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-21 10:34:23 +08:00
Koert Kuipers b461acb2d9 [SPARK-25134][SQL] Csv column pruning with checking of headers throws incorrect error
## What changes were proposed in this pull request?

When column pruning is turned on the checking of headers in the csv should only be for the fields in the requiredSchema, not the dataSchema, because column pruning means only requiredSchema is read.

## How was this patch tested?

Added 2 unit tests where column pruning is turned on/off and csv headers are checked againt schema

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

Closes #22123 from koertkuipers/feat-csv-column-pruning-and-check-header.

Authored-by: Koert Kuipers <koert@tresata.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-21 10:23:55 +08:00
Dongjoon Hyun 883f3aff67 [SPARK-25144][SQL][TEST] Free aggregate map when task ends
## What changes were proposed in this pull request?

[SPARK-25144](https://issues.apache.org/jira/browse/SPARK-25144) reports memory leaks on Apache Spark 2.0.2 ~ 2.3.2-RC5. The bug is already fixed via #21738 as a part of SPARK-21743. This PR only adds a test case to prevent any future regression.

```scala
scala> case class Foo(bar: Option[String])
scala> val ds = List(Foo(Some("bar"))).toDS
scala> val result = ds.flatMap(_.bar).distinct
scala> result.rdd.isEmpty
18/08/19 23:01:54 WARN Executor: Managed memory leak detected; size = 8650752 bytes, TID = 125
res0: Boolean = false
```

## How was this patch tested?

Pass the Jenkins with a new added test case.

Closes #22155 from dongjoon-hyun/SPARK-25144-2.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-21 09:08:36 +08:00
Maxim Gekk a8a1ac01c4 [SPARK-24959][SQL] Speed up count() for JSON and CSV
## What changes were proposed in this pull request?

In the PR, I propose to skip invoking of the CSV/JSON parser per each line in the case if the required schema is empty. Added benchmarks for `count()` shows performance improvement up to **3.5 times**.

Before:

```
Count a dataset with 10 columns:      Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)
--------------------------------------------------------------------------------------
JSON count()                               7676 / 7715          1.3         767.6
CSV count()                                3309 / 3363          3.0         330.9
```

After:

```
Count a dataset with 10 columns:      Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)
--------------------------------------------------------------------------------------
JSON count()                               2104 / 2156          4.8         210.4
CSV count()                                2332 / 2386          4.3         233.2
```

## How was this patch tested?

It was tested by `CSVSuite` and `JSONSuite` as well as on added benchmarks.

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

Closes #21909 from MaxGekk/empty-schema-optimization.
2018-08-18 10:34:49 -07:00
codeatri f16140975d [SPARK-23940][SQL] Add transform_values SQL function
## What changes were proposed in this pull request?
This pr adds `transform_values` function which applies the function to each entry of the map and transforms the values.
```javascript
> SELECT transform_values(map(array(1, 2, 3), array(1, 2, 3)), (k,v) -> v + 1);
       map(1->2, 2->3, 3->4)

> SELECT transform_values(map(array(1, 2, 3), array(1, 2, 3)), (k,v) -> k + v);
       map(1->2, 2->4, 3->6)
```
## How was this patch tested?
New Tests added to
`DataFrameFunctionsSuite`
`HigherOrderFunctionsSuite`
`SQLQueryTestSuite`

Closes #22045 from codeatri/SPARK-23940.

Authored-by: codeatri <nehapatil6@gmail.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2018-08-17 11:50:06 +09:00
Sandeep Singh ea63a7a168 [SPARK-23932][SQL] Higher order function zip_with
## What changes were proposed in this pull request?
Merges the two given arrays, element-wise, into a single array using function. If one array is shorter, nulls are appended at the end to match the length of the longer array, before applying function:
```
    SELECT zip_with(ARRAY[1, 3, 5], ARRAY['a', 'b', 'c'], (x, y) -> (y, x)); -- [ROW('a', 1), ROW('b', 3), ROW('c', 5)]
    SELECT zip_with(ARRAY[1, 2], ARRAY[3, 4], (x, y) -> x + y); -- [4, 6]
    SELECT zip_with(ARRAY['a', 'b', 'c'], ARRAY['d', 'e', 'f'], (x, y) -> concat(x, y)); -- ['ad', 'be', 'cf']
    SELECT zip_with(ARRAY['a'], ARRAY['d', null, 'f'], (x, y) -> coalesce(x, y)); -- ['a', null, 'f']
```
## How was this patch tested?
Added tests

Closes #22031 from techaddict/SPARK-23932.

Authored-by: Sandeep Singh <sandeep@techaddict.me>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2018-08-16 23:02:45 +09:00
codeatri 5b4a38d826 [SPARK-23939][SQL] Add transform_keys function
## What changes were proposed in this pull request?
This pr adds transform_keys function which applies the function to each entry of the map and transforms the keys.
```javascript
> SELECT transform_keys(map(array(1, 2, 3), array(1, 2, 3)), (k,v) -> k + 1);
       map(2->1, 3->2, 4->3)

> SELECT transform_keys(map(array(1, 2, 3), array(1, 2, 3)), (k,v) -> k + v);
       map(2->1, 4->2, 6->3)
```

## How was this patch tested?
Added tests.

Closes #22013 from codeatri/SPARK-23939.

Authored-by: codeatri <nehapatil6@gmail.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2018-08-16 17:07:33 +09:00
Bo Meng 7822c3f8d1 [SPARK-25082][SQL] improve the javadoc for expm1()
## What changes were proposed in this pull request?
Correct the javadoc for expm1() function.

## How was this patch tested?
None. It is a minor issue.

Closes #22115 from bomeng/25082.

Authored-by: Bo Meng <bo.meng@jd.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-16 14:14:42 +08:00
Bryan Cutler ed075e1ff6 [SPARK-23874][SQL][PYTHON] Upgrade Apache Arrow to 0.10.0
## What changes were proposed in this pull request?

Upgrade Apache Arrow to 0.10.0

Version 0.10.0 has a number of bug fixes and improvements with the following pertaining directly to usage in Spark:
 * Allow for adding BinaryType support ARROW-2141
 * Bug fix related to array serialization ARROW-1973
 * Python2 str will be made into an Arrow string instead of bytes ARROW-2101
 * Python bytearrays are supported in as input to pyarrow ARROW-2141
 * Java has common interface for reset to cleanup complex vectors in Spark ArrowWriter ARROW-1962
 * Cleanup pyarrow type equality checks ARROW-2423
 * ArrowStreamWriter should not hold references to ArrowBlocks ARROW-2632, ARROW-2645
 * Improved low level handling of messages for RecordBatch ARROW-2704

## How was this patch tested?

existing tests

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #21939 from BryanCutler/arrow-upgrade-010.
2018-08-14 17:13:38 -07:00
Marek Novotny 42263fd0cb [SPARK-23938][SQL] Add map_zip_with function
## What changes were proposed in this pull request?

This PR adds a new SQL function called ```map_zip_with```. It merges the two given maps into a single map by applying function to the pair of values with the same key.

## How was this patch tested?

Added new tests into:
- DataFrameFunctionsSuite.scala
- HigherOrderFunctionsSuite.scala

Closes #22017 from mn-mikke/SPARK-23938.

Authored-by: Marek Novotny <mn.mikke@gmail.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2018-08-14 21:14:15 +09:00
Dongjoon Hyun e2ab7deae7 [MINOR][SQL][DOC] Fix to_json example in function description and doc
## What changes were proposed in this pull request?

This PR fixes the an example for `to_json` in doc and function description.

- http://spark.apache.org/docs/2.3.0/api/sql/#to_json
- `describe function extended`

## How was this patch tested?

Pass the Jenkins with the updated test.

Closes #22096 from dongjoon-hyun/minor_json.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-14 19:59:39 +08:00
Marco Gaido c220cc42ab [SPARK-25028][SQL] Avoid NPE when analyzing partition with NULL values
## What changes were proposed in this pull request?

`ANALYZE TABLE ... PARTITION(...) COMPUTE STATISTICS` can fail with a NPE if a partition column contains a NULL value.

The PR avoids the NPE, replacing the `NULL` values with the default partition placeholder.

## How was this patch tested?

added UT

Closes #22036 from mgaido91/SPARK-25028.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-14 00:59:18 +08:00
Takuya UESHIN b804ca5771 [SPARK-23908][SQL][FOLLOW-UP] Rename inputs to arguments, and add argument type check.
## What changes were proposed in this pull request?

This is a follow-up pr of #21954 to address comments.

- Rename ambiguous name `inputs` to `arguments`.
- Add argument type check and remove hacky workaround.
- Address other small comments.

## How was this patch tested?

Existing tests and some additional tests.

Closes #22075 from ueshin/issues/SPARK-23908/fup1.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-13 20:58:29 +08:00
Maxim Gekk ab06c25350 [SPARK-24391][SQL] Support arrays of any types by from_json
## What changes were proposed in this pull request?

The PR removes a restriction for element types of array type which exists in `from_json` for the root type. Currently, the function can handle only arrays of structs. Even array of primitive types is disallowed. The PR allows arrays of any types currently supported by JSON datasource. Here is an example of an array of a primitive type:

```
scala> import org.apache.spark.sql.functions._
scala> val df = Seq("[1, 2, 3]").toDF("a")
scala> val schema = new ArrayType(IntegerType, false)
scala> val arr = df.select(from_json($"a", schema))
scala> arr.printSchema
root
 |-- jsontostructs(a): array (nullable = true)
 |    |-- element: integer (containsNull = true)
```
and result of converting of the json string to the `ArrayType`:
```
scala> arr.show
+----------------+
|jsontostructs(a)|
+----------------+
|       [1, 2, 3]|
+----------------+
```

## How was this patch tested?

I added a few positive and negative tests:
- array of primitive types
- array of arrays
- array of structs
- array of maps

Closes #21439 from MaxGekk/from_json-array.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-13 20:13:09 +08:00
Gengliang Wang be2238fb50 [SPARK-24774][SQL] Avro: Support logical decimal type
## What changes were proposed in this pull request?

Support Avro logical date type:
https://avro.apache.org/docs/1.8.2/spec.html#Decimal

## How was this patch tested?
Unit test

Closes #22037 from gengliangwang/avro_decimal.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-13 08:29:07 +08:00
忍冬 a90b1f5d93 [MINOR][DOC] Fix Java example code in Column's comments
## What changes were proposed in this pull request?
Fix scaladoc in Column

## How was this patch tested?
None

Closes #22069 from sadhen/fix_doc_minor.

Authored-by: 忍冬 <rendong@wacai.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2018-08-12 08:26:21 -05:00
Kazuaki Ishizaki d177234792 [SQL][TEST][MINOR] Add missing codes to ParquetCompressionCodecPrecedenceSuite
## What changes were proposed in this pull request?

This PR adds codes to ``"Test `spark.sql.parquet.compression.codec` config"` in `ParquetCompressionCodecPrecedenceSuite`.

## How was this patch tested?

Existing UTs

Closes #22083 from kiszk/ParquetCompressionCodecPrecedenceSuite.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-12 19:43:25 +08:00
Kazuhiro Sera 8ec25cd67e Fix typos detected by github.com/client9/misspell
## What changes were proposed in this pull request?

Fixing typos is sometimes very hard. It's not so easy to visually review them. Recently, I discovered a very useful tool for it, [misspell](https://github.com/client9/misspell).

This pull request fixes minor typos detected by [misspell](https://github.com/client9/misspell) except for the false positives. If you would like me to work on other files as well, let me know.

## How was this patch tested?

### before

```
$ misspell . | grep -v '.js'
R/pkg/R/SQLContext.R:354:43: "definiton" is a misspelling of "definition"
R/pkg/R/SQLContext.R:424:43: "definiton" is a misspelling of "definition"
R/pkg/R/SQLContext.R:445:43: "definiton" is a misspelling of "definition"
R/pkg/R/SQLContext.R:495:43: "definiton" is a misspelling of "definition"
NOTICE-binary:454:16: "containd" is a misspelling of "contained"
R/pkg/R/context.R:46:43: "definiton" is a misspelling of "definition"
R/pkg/R/context.R:74:43: "definiton" is a misspelling of "definition"
R/pkg/R/DataFrame.R:591:48: "persistance" is a misspelling of "persistence"
R/pkg/R/streaming.R:166:44: "occured" is a misspelling of "occurred"
R/pkg/inst/worker/worker.R:65:22: "ouput" is a misspelling of "output"
R/pkg/tests/fulltests/test_utils.R:106:25: "environemnt" is a misspelling of "environment"
common/kvstore/src/test/java/org/apache/spark/util/kvstore/InMemoryStoreSuite.java:38:39: "existant" is a misspelling of "existent"
common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java:83:39: "existant" is a misspelling of "existent"
common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java:243:46: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:234:19: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:238:63: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:244:46: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:276:39: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/util/AbstractFileRegion.java:27:20: "transfered" is a misspelling of "transferred"
common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala:195:15: "orgin" is a misspelling of "origin"
core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala:621:39: "gauranteed" is a misspelling of "guaranteed"
core/src/main/scala/org/apache/spark/status/storeTypes.scala:113:29: "ect" is a misspelling of "etc"
core/src/main/scala/org/apache/spark/storage/DiskStore.scala:282:18: "transfered" is a misspelling of "transferred"
core/src/main/scala/org/apache/spark/util/ListenerBus.scala:64:17: "overriden" is a misspelling of "overridden"
core/src/test/scala/org/apache/spark/ShuffleSuite.scala:211:7: "substracted" is a misspelling of "subtracted"
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:1922:49: "agriculteur" is a misspelling of "agriculture"
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:2468:84: "truely" is a misspelling of "truly"
core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala:25:18: "persistance" is a misspelling of "persistence"
core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala:26:69: "persistance" is a misspelling of "persistence"
data/streaming/AFINN-111.txt:1219:0: "humerous" is a misspelling of "humorous"
dev/run-pip-tests:55:28: "enviroments" is a misspelling of "environments"
dev/run-pip-tests:91:37: "virutal" is a misspelling of "virtual"
dev/merge_spark_pr.py:377:72: "accross" is a misspelling of "across"
dev/merge_spark_pr.py:378:66: "accross" is a misspelling of "across"
dev/run-pip-tests:126:25: "enviroments" is a misspelling of "environments"
docs/configuration.md:1830:82: "overriden" is a misspelling of "overridden"
docs/structured-streaming-programming-guide.md:525:45: "processs" is a misspelling of "processes"
docs/structured-streaming-programming-guide.md:1165:61: "BETWEN" is a misspelling of "BETWEEN"
docs/sql-programming-guide.md:1891:810: "behaivor" is a misspelling of "behavior"
examples/src/main/python/sql/arrow.py:98:8: "substract" is a misspelling of "subtract"
examples/src/main/python/sql/arrow.py:103:27: "substract" is a misspelling of "subtract"
licenses/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING"
licenses/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING"
licenses/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses-binary/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING"
licenses-binary/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses-binary/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING"
licenses-binary/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/hungarian.txt:170:0: "teh" is a misspelling of "the"
mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/portuguese.txt:53:0: "eles" is a misspelling of "eels"
mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:99:20: "Euclidian" is a misspelling of "Euclidean"
mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:539:11: "Euclidian" is a misspelling of "Euclidean"
mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala:77:36: "Teh" is a misspelling of "The"
mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala:230:24: "inital" is a misspelling of "initial"
mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala:276:9: "Euclidian" is a misspelling of "Euclidean"
mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala:237:26: "descripiton" is a misspelling of "descriptions"
python/pyspark/find_spark_home.py:30:13: "enviroment" is a misspelling of "environment"
python/pyspark/context.py:937:12: "supress" is a misspelling of "suppress"
python/pyspark/context.py:938:12: "supress" is a misspelling of "suppress"
python/pyspark/context.py:939:12: "supress" is a misspelling of "suppress"
python/pyspark/context.py:940:12: "supress" is a misspelling of "suppress"
python/pyspark/heapq3.py:6:63: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:7:2: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:263:29: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:263:39: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:270:49: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:270:59: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:275:2: "STICHTING" is a misspelling of "STITCHING"
python/pyspark/heapq3.py:275:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
python/pyspark/heapq3.py:277:29: "STICHTING" is a misspelling of "STITCHING"
python/pyspark/heapq3.py:277:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
python/pyspark/heapq3.py:713:8: "probabilty" is a misspelling of "probability"
python/pyspark/ml/clustering.py:1038:8: "Currenlty" is a misspelling of "Currently"
python/pyspark/ml/stat.py:339:23: "Euclidian" is a misspelling of "Euclidean"
python/pyspark/ml/regression.py:1378:20: "paramter" is a misspelling of "parameter"
python/pyspark/mllib/stat/_statistics.py:262:8: "probabilty" is a misspelling of "probability"
python/pyspark/rdd.py:1363:32: "paramter" is a misspelling of "parameter"
python/pyspark/streaming/tests.py:825:42: "retuns" is a misspelling of "returns"
python/pyspark/sql/tests.py:768:29: "initalization" is a misspelling of "initialization"
python/pyspark/sql/tests.py:3616:31: "initalize" is a misspelling of "initialize"
resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala:120:39: "arbitary" is a misspelling of "arbitrary"
resource-managers/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArgumentsSuite.scala:26:45: "sucessfully" is a misspelling of "successfully"
resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala:358:27: "constaints" is a misspelling of "constraints"
resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala:111:24: "senstive" is a misspelling of "sensitive"
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala:1063:5: "overwirte" is a misspelling of "overwrite"
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala:1348:17: "compatability" is a misspelling of "compatibility"
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:77:36: "paramter" is a misspelling of "parameter"
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:1374:22: "precendence" is a misspelling of "precedence"
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala:238:27: "unnecassary" is a misspelling of "unnecessary"
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala:212:17: "whn" is a misspelling of "when"
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinHelper.scala:147:60: "timestmap" is a misspelling of "timestamp"
sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala:150:45: "precentage" is a misspelling of "percentage"
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala:135:29: "infered" is a misspelling of "inferred"
sql/hive/src/test/resources/golden/udf_instr-1-2e76f819563dbaba4beb51e3a130b922:1:52: "occurance" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_instr-2-32da357fc754badd6e3898dcc8989182:1:52: "occurance" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_locate-1-6e41693c9c6dceea4d7fab4c02884e4e:1:63: "occurance" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_locate-2-d9b5934457931447874d6bb7c13de478:1:63: "occurance" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_translate-2-f7aa38a33ca0df73b7a1e6b6da4b7fe8:9:79: "occurence" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_translate-2-f7aa38a33ca0df73b7a1e6b6da4b7fe8:13:110: "occurence" is a misspelling of "occurrence"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_join.q:46:105: "distint" is a misspelling of "distinct"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q:29:3: "Currenly" is a misspelling of "Currently"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_partitioned.q:72:15: "existant" is a misspelling of "existent"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_udf.q:25:3: "substraction" is a misspelling of "subtraction"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q:16:51: "funtion" is a misspelling of "function"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_8.q:15:30: "issueing" is a misspelling of "issuing"
sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala:669:52: "wiht" is a misspelling of "with"
sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java:474:9: "Refering" is a misspelling of "Referring"
```

### after

```
$ misspell . | grep -v '.js'
common/network-common/src/main/java/org/apache/spark/network/util/AbstractFileRegion.java:27:20: "transfered" is a misspelling of "transferred"
core/src/main/scala/org/apache/spark/status/storeTypes.scala:113:29: "ect" is a misspelling of "etc"
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:1922:49: "agriculteur" is a misspelling of "agriculture"
data/streaming/AFINN-111.txt:1219:0: "humerous" is a misspelling of "humorous"
licenses/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING"
licenses/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING"
licenses/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses-binary/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING"
licenses-binary/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses-binary/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING"
licenses-binary/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/hungarian.txt:170:0: "teh" is a misspelling of "the"
mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/portuguese.txt:53:0: "eles" is a misspelling of "eels"
mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:99:20: "Euclidian" is a misspelling of "Euclidean"
mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:539:11: "Euclidian" is a misspelling of "Euclidean"
mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala:77:36: "Teh" is a misspelling of "The"
mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala:276:9: "Euclidian" is a misspelling of "Euclidean"
python/pyspark/heapq3.py:6:63: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:7:2: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:263:29: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:263:39: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:270:49: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:270:59: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:275:2: "STICHTING" is a misspelling of "STITCHING"
python/pyspark/heapq3.py:275:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
python/pyspark/heapq3.py:277:29: "STICHTING" is a misspelling of "STITCHING"
python/pyspark/heapq3.py:277:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
python/pyspark/ml/stat.py:339:23: "Euclidian" is a misspelling of "Euclidean"
```

Closes #22070 from seratch/fix-typo.

Authored-by: Kazuhiro Sera <seratch@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2018-08-11 21:23:36 -05:00
yucai 41a7de6002 [SPARK-25084][SQL] "distribute by" on multiple columns (wrap in brackets) may lead to codegen issue
## What changes were proposed in this pull request?

"distribute by" on multiple columns (wrap in brackets) may lead to codegen issue.

Simple way to reproduce:
```scala
  val df = spark.range(1000)
  val columns = (0 until 400).map{ i => s"id as id$i" }
  val distributeExprs = (0 until 100).map(c => s"id$c").mkString(",")
  df.selectExpr(columns : _*).createTempView("test")
  spark.sql(s"select * from test distribute by ($distributeExprs)").count()
```

## How was this patch tested?

Add UT.

Closes #22066 from yucai/SPARK-25084.

Authored-by: yucai <yyu1@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-11 21:38:31 +08:00
liuxian 4b11d909fd [MINOR][DOC] Add missing compression codec .
## What changes were proposed in this pull request?

Parquet file provides six codecs: "snappy", "gzip", "lzo", "lz4", "brotli", "zstd".
This pr add missing compression codec :"lz4", "brotli", "zstd" .
## How was this patch tested?
N/A

Closes #22068 from 10110346/nosupportlz4.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-11 20:49:52 +08:00
Liang-Chi Hsieh 4f17585098 [SPARK-19355][SQL] Use map output statistics to improve global limit's parallelism
## What changes were proposed in this pull request?

A logical `Limit` is performed physically by two operations `LocalLimit` and `GlobalLimit`.

Most of time, we gather all data into a single partition in order to run `GlobalLimit`. If we use a very big limit number, shuffling data causes performance issue also reduces parallelism.

We can avoid shuffling into single partition if we don't care data ordering. This patch implements this idea by doing a map stage during global limit. It collects the info of row numbers at each partition. For each partition, we locally retrieves limited data without any shuffling to finish this global limit.

For example, we have three partitions with rows (100, 100, 50) respectively. In global limit of 100 rows, we may take (34, 33, 33) rows for each partition locally. After global limit we still have three partitions.

If the data partition has certain ordering, we can't distribute required rows evenly to each partitions because it could change data ordering. But we still can avoid shuffling.

## How was this patch tested?

Jenkins tests.

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

Closes #16677 from viirya/improve-global-limit-parallelism.
2018-08-10 11:32:15 +02:00
Arun Mahadevan 9abe09bfc1 [SPARK-24127][SS] Continuous text socket source
## What changes were proposed in this pull request?

Support for text socket stream in spark structured streaming "continuous" mode. This is roughly based on the idea of ContinuousMemoryStream where the executor queries the data from driver over an RPC endpoint.

This makes it possible to create Structured streaming continuous pipeline to ingest data via "nc" and run examples.

## How was this patch tested?

Unit test and ran spark examples in structured streaming continuous mode.

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

Closes #21199 from arunmahadevan/SPARK-24127.

Authored-by: Arun Mahadevan <arunm@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-10 15:53:31 +08:00
Takuya UESHIN 9b8521e53e [SPARK-25068][SQL] Add exists function.
## What changes were proposed in this pull request?

This pr adds `exists` function which tests whether a predicate holds for one or more elements in the array.

```sql
> SELECT exists(array(1, 2, 3), x -> x % 2 == 0);
 true
```

## How was this patch tested?

Added tests.

Closes #22052 from ueshin/issues/SPARK-25068/exists.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2018-08-09 14:41:59 -07:00
Wenchen Fan fec67ed7e9 [SPARK-25076][SQL] SQLConf should not be retrieved from a stopped SparkSession
## What changes were proposed in this pull request?

When a `SparkSession` is stopped, `SQLConf.get` should use the fallback conf to avoid weird issues like
```
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)
...
```

## How was this patch tested?

a new test suite

Closes #22056 from cloud-fan/session.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2018-08-09 14:38:58 -07:00
liyuanjian bd6db1505f [SPARK-25077][SQL] Delete unused variable in WindowExec
## What changes were proposed in this pull request?

Just delete the unused variable `inputFields` in WindowExec, avoid making others confused while reading the code.

## How was this patch tested?

Existing UT.

Closes #22057 from xuanyuanking/SPARK-25077.

Authored-by: liyuanjian <liyuanjian@baidu.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2018-08-09 13:43:07 -07:00
Achuth17 d36539741f [SPARK-24626][SQL] Improve location size calculation in Analyze Table command
## What changes were proposed in this pull request?

Currently, Analyze table calculates table size sequentially for each partition. We can parallelize size calculations over partitions.

Results : Tested on a table with 100 partitions and data stored in S3.
With changes :
- 10.429s
- 10.557s
- 10.439s
- 9.893s


Without changes :
- 110.034s
- 99.510s
- 100.743s
- 99.106s

## How was this patch tested?

Simple unit test.

Closes #21608 from Achuth17/improveAnalyze.

Lead-authored-by: Achuth17 <Achuth.narayan@gmail.com>
Co-authored-by: arajagopal17 <arajagopal@qubole.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2018-08-09 08:29:24 -07:00
Wenchen Fan b2950cef3c Revert "[SPARK-24648][SQL] SqlMetrics should be threadsafe"
This reverts commit 5264164a67.
2018-08-09 20:33:59 +08:00
Kazuaki Ishizaki 56e9e97073 [MINOR][DOC] Fix typo
## What changes were proposed in this pull request?

This PR fixes typo regarding `auxiliary verb + verb[s]`. This is a follow-on of #21956.

## How was this patch tested?

N/A

Closes #22040 from kiszk/spellcheck1.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-09 20:10:17 +08:00
Takuya UESHIN 519e03d82e [SPARK-25058][SQL] Use Block.isEmpty/nonEmpty to check whether the code is empty or not.
## What changes were proposed in this pull request?

We should use `Block.isEmpty/nonEmpty` instead of comparing with empty string to check whether the code is empty or not.

```
[error] [warn] /.../sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala:278: org.apache.spark.sql.catalyst.expressions.codegen.Block and String are unrelated: they will most likely always compare unequal
[error] [warn]       if (ev.code != "" && required.contains(attributes(i))) {
[error] [warn]
[error] [warn] /.../sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala:323: org.apache.spark.sql.catalyst.expressions.codegen.Block and String are unrelated: they will most likely never compare equal
[error] [warn]          |  ${buildVars.filter(_.code == "").map(v => s"${v.isNull} = true;").mkString("\n")}
[error] [warn]
```

## How was this patch tested?

Existing tests.

Closes #22041 from ueshin/issues/SPARK-25058/fix_comparison.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2018-08-09 14:06:28 +09:00
Takuya UESHIN f62fe435de [SPARK-25036][SQL][FOLLOW-UP] Avoid match may not be exhaustive in Scala-2.12.
## What changes were proposed in this pull request?

This is a follow-up pr of #22014.

We still have some more compilation errors in scala-2.12 with sbt:

```
[error] [warn] /.../sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala:493: match may not be exhaustive.
[error] It would fail on the following input: (_, _)
[error] [warn]       val typeMatches = (targetType, f.dataType) match {
[error] [warn]
[error] [warn] /.../sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala:393: match may not be exhaustive.
[error] It would fail on the following input: (_, _)
[error] [warn]             prevBatchOff.get.toStreamProgress(sources).foreach {
[error] [warn]
[error] [warn] /.../sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala:173: match may not be exhaustive.
[error] It would fail on the following input: AggregateExpression(_, _, false, _)
[error] [warn]     val rewrittenDistinctFunctions = functionsWithDistinct.map {
[error] [warn]
[error] [warn] /.../sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SymmetricHashJoinStateManager.scala:271: match may not be exhaustive.
[error] It would fail on the following input: (_, _)
[error] [warn]       keyWithIndexToValueMetrics.customMetrics.map {
[error] [warn]
[error] [warn] /.../sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala:959: match may not be exhaustive.
[error] It would fail on the following input: CatalogTableType(_)
[error] [warn]     val tableTypeString = metadata.tableType match {
[error] [warn]
[error] [warn] /.../sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala:923: match may not be exhaustive.
[error] It would fail on the following input: CatalogTableType(_)
[error] [warn]     hiveTable.setTableType(table.tableType match {
[error] [warn]
```

## How was this patch tested?

Manually build with Scala-2.12.

Closes #22039 from ueshin/issues/SPARK-25036/fix_match.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2018-08-08 16:47:22 -05:00
Takuya UESHIN c7a229d655 [SPARK-25010][SQL][FOLLOWUP] Shuffle should also produce different values for each execution in streaming query.
## What changes were proposed in this pull request?

This is a follow-up pr of #21980.

`Shuffle` can also be `ExpressionWithRandomSeed` to produce different values for each execution in streaming query.

## How was this patch tested?

Added a test.

Closes #22027 from ueshin/issues/SPARK-25010/random_seed.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-08 11:05:52 +08:00
Ryan Blue 5fef6e3513 [SPARK-24251][SQL] Add AppendData logical plan.
## What changes were proposed in this pull request?

This adds a new logical plan, AppendData, that was proposed in SPARK-23521: Standardize SQL logical plans.

* DataFrameWriter uses the new AppendData plan for DataSourceV2 appends
* AppendData is resolved if its output columns match the incoming data frame
* A new analyzer rule, ResolveOutputColumns, validates data before it is appended. This rule will add safe casts, rename columns, and checks nullability

## How was this patch tested?

Existing tests for v2 appends. Will add AppendData tests to validate logical plan analysis.

Closes #21305 from rdblue/SPARK-24251-add-append-data.

Lead-authored-by: Ryan Blue <blue@apache.org>
Co-authored-by: Ryan Blue <rdblue@users.noreply.github.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-08 09:55:52 +08:00
Neal Song f6356f9bc0 [SPARK-25046][SQL] Fix Alter View can excute sql like "ALTER VIEW ... AS INSERT INTO"
## What changes were proposed in this pull request?

Alter View  can excute sql  like "ALTER VIEW ... AS INSERT INTO" . We should throw ParseException(s"Operation not allowed: $message", ctx)  as Create View does.
 ```
override def visitCreateView(ctx: CreateViewContext): LogicalPlan = withOrigin(ctx) {
    if (ctx.identifierList != null) {
      operationNotAllowed("CREATE VIEW ... PARTITIONED ON", ctx)
    } else {
      // CREATE VIEW ... AS INSERT INTO is not allowed.
      ctx.query.queryNoWith match {
        case s: SingleInsertQueryContext if s.insertInto != null =>
          operationNotAllowed("CREATE VIEW ... AS INSERT INTO", ctx)
        case _: MultiInsertQueryContext =>
          operationNotAllowed("CREATE VIEW ... AS FROM ... [INSERT INTO ...]+", ctx)
        case _ => // OK
      }
```

```
override def visitAlterViewQuery(ctx: AlterViewQueryContext): LogicalPlan = withOrigin(ctx) {
    // ALTER VIEW ... AS INSERT INTO is not allowed.
    ctx.query.queryNoWith match {
      case s: SingleInsertQueryContext if s.insertInto != null =>
        operationNotAllowed("ALTER VIEW ... AS INSERT INTO", ctx)
      case _: MultiInsertQueryContext =>
        operationNotAllowed("ALTER VIEW ... AS FROM ... [INSERT INTO ...]+", ctx)
      case _ => // OK
    }
    AlterViewAsCommand(
      name = visitTableIdentifier(ctx.tableIdentifier),
      originalText = source(ctx.query),
      query = plan(ctx.query))
  }
```

## How was this patch tested?

UT has been added in SparkSqlParserSuite

Closes #22028 from sddyljsx/SPARK-25046.

Lead-authored-by: Neal Song <neal_song@126.com>
Co-authored-by: neal <neal_song@126.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2018-08-07 14:51:41 -07:00
Marco Gaido cb6cb31363 [SPARK-23937][SQL] Add map_filter SQL function
## What changes were proposed in this pull request?

The PR adds the high order function `map_filter`, which filters the entries of a map and returns a new map which contains only the entries which satisfied the filter function.

## How was this patch tested?

added UTs

Closes #21986 from mgaido91/SPARK-23937.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2018-08-08 02:12:19 +09:00
Wenchen Fan 1a29fec8e2 [SPARK-24979][SQL] add AnalysisHelper#resolveOperatorsUp
## What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/21822

Similar to `TreeNode`, `AnalysisHelper` should also provide 3 versions of transformations: `resolveOperatorsUp`, `resolveOperatorsDown` and `resolveOperators`.

This PR adds the missing `resolveOperatorsUp`, and also fixes some code style which is missed in #21822

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21932 from cloud-fan/follow.
2018-08-07 08:45:20 -07:00
Sunitha Kambhampati b4bf8be549 [SPARK-19602][SQL] Support column resolution of fully qualified column name ( 3 part name)
## What changes were proposed in this pull request?
The design details is attached to the JIRA issue [here](https://drive.google.com/file/d/1zKm3aNZ3DpsqIuoMvRsf0kkDkXsAasxH/view)

High level overview of the changes are:
- Enhance the qualifier to be more than one string
- Add support to store the qualifier. Enhance the lookupRelation to keep the qualifier appropriately.
- Enhance the table matching column resolution algorithm to account for qualifier being more than a string.
- Enhance the table matching algorithm in UnresolvedStar.expand
- Ensure that we continue to support select t1.i1 from db1.t1

## How was this patch tested?
- New tests are added.
- Several test scenarios were added in a separate  [test pr 17067](https://github.com/apache/spark/pull/17067).  The tests that were not supported earlier are marked with TODO markers and those are now supported with the code changes here.
- Existing unit tests ( hive, catalyst and sql) were run successfully.

Closes #17185 from skambha/colResolution.

Authored-by: Sunitha Kambhampati <skambha@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-07 21:11:08 +08:00
Maxim Gekk 131ca146ed [SPARK-24005][CORE] Remove usage of Scala’s parallel collection
## What changes were proposed in this pull request?

In the PR, I propose to replace Scala parallel collections by new methods `parmap()`. The methods use futures to transform a sequential collection by applying a lambda function to each element in parallel. The result of `parmap` is another regular (sequential) collection.

The proposed `parmap` method aims to solve the problem of impossibility to interrupt parallel Scala collection. This possibility is needed for reliable task preemption.

## How was this patch tested?

A test was added to `ThreadUtilsSuite`

Closes #21913 from MaxGekk/par-map.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-07 17:14:30 +08:00
Marco Gaido 88e0c7bbd5 [SPARK-24341][SQL] Support only IN subqueries with the same number of items per row
## What changes were proposed in this pull request?

Using struct types in subqueries with the `IN` clause can generate invalid plans in `RewritePredicateSubquery`. Indeed, we are not handling clearly the cases when the outer value is a struct or the output of the inner subquery is a struct.

The PR aims to make Spark's behavior the same as the one of the other RDBMS - namely Oracle and Postgres behavior were checked. So we consider valid only queries having the same number of fields in the outer value and in the subquery. This means that:

 - `(a, b) IN (select c, d from ...)` is a valid query;
 - `(a, b) IN (select (c, d) from ...)` throws an AnalysisException, as in the subquery we have only one field of type struct while in the outer value we have 2 fields;
 - `a IN (select (c, d) from ...)` - where `a` is a struct - is a valid query.

## How was this patch tested?

Added UT

Closes #21403 from mgaido91/SPARK-24313.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-07 15:43:41 +08:00
Liang-Chi Hsieh 43763629f1 [SPARK-25010][SQL] Rand/Randn should produce different values for each execution in streaming query
## What changes were proposed in this pull request?

Like Uuid in SPARK-24896, Rand and Randn expressions now produce the same results for each execution in streaming query. It doesn't make too much sense for streaming queries. We should make them produce different results as Uuid.

In this change, similar to Uuid, we assign new random seeds to Rand/Randn when returning optimized plan from `IncrementalExecution`.

Note: Different to Uuid, Rand/Randn can be created with initial seed. Because we replace this initial seed at `IncrementalExecution`, it doesn't use the initial seed anymore. For now it seems to me not a big issue for streaming query. But need to confirm with others. cc zsxwing cloud-fan

## How was this patch tested?

Added test.

Closes #21980 from viirya/SPARK-25010.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-07 14:28:14 +08:00
Kazuaki Ishizaki 4446a0b0d9 [SPARK-23914][SQL][FOLLOW-UP] refactor ArrayUnion
## What changes were proposed in this pull request?

This PR refactors `ArrayUnion` based on [this suggestion](https://github.com/apache/spark/pull/21103#discussion_r205668821).
1. Generate optimized code for all of the primitive types except `boolean`
1. Generate code using `ArrayBuilder` or `ArrayBuffer`
1. Leave only a generic path in the interpreted path

## How was this patch tested?

Existing tests

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

Closes #21937 from kiszk/SPARK-23914-follow.
2018-08-07 12:07:56 +09:00
Arun Mahadevan 18b6ec1471 [SPARK-24748][SS] Support for reporting custom metrics via StreamingQuery Progress
## What changes were proposed in this pull request?

Currently the Structured Streaming sources and sinks does not have a way to report custom metrics. Providing an option to report custom metrics and making it available via Streaming Query progress can enable sources and sinks to report custom progress information (E.g. the lag metrics for Kafka source).

Similar metrics can be reported for Sinks as well, but would like to get initial feedback before proceeding further.

## How was this patch tested?

New and existing unit tests.

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

Closes #21721 from arunmahadevan/SPARK-24748.

Authored-by: Arun Mahadevan <arunm@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-07 10:28:26 +08:00
Jungtaek Lim 6afe6f32ca [SPARK-24637][SS] Add metrics regarding state and watermark to dropwizard metrics
## What changes were proposed in this pull request?

The patch adds metrics regarding state and watermark to dropwizard metrics, so that watermark and state rows/size can be tracked via time-series manner.

## How was this patch tested?

Manually tested with CSV metric sink.

Closes #21622 from HeartSaVioR/SPARK-24637.

Authored-by: Jungtaek Lim <kabhwan@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-07 10:12:22 +08:00
Jungtaek Lim 87ca7396c7
[SPARK-24161][SS] Enable debug package feature on structured streaming
## What changes were proposed in this pull request?

Currently, debug package has a implicit class "DebugQuery" which matches Dataset to provide debug features on Dataset class. It doesn't work with structured streaming: it requires query is already started, and the information can be retrieved from StreamingQuery, not Dataset. I guess that's why "explain" had to be placed to StreamingQuery whereas it already exists on Dataset.

This patch adds a new implicit class "DebugStreamQuery" which matches StreamingQuery to provide similar debug features on StreamingQuery class.

## How was this patch tested?

Added relevant unit tests.

Author: Jungtaek Lim <kabhwan@gmail.com>

Closes #21222 from HeartSaVioR/SPARK-24161.
2018-08-06 15:23:47 -07:00
Dongjoon Hyun 278984d5a5 [SPARK-25019][BUILD] Fix orc dependency to use the same exclusion rules
## What changes were proposed in this pull request?

During upgrading Apache ORC to 1.5.2 ([SPARK-24576](https://issues.apache.org/jira/browse/SPARK-24576)), `sql/core` module overrides the exclusion rules of parent pom file and it causes published `spark-sql_2.1X` artifacts have incomplete exclusion rules ([SPARK-25019](https://issues.apache.org/jira/browse/SPARK-25019)). This PR fixes it by moving the newly added exclusion rule to the parent pom. This also fixes the sbt build hack introduced at that time.

## How was this patch tested?

Pass the existing dependency check and the tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #22003 from dongjoon-hyun/SPARK-25019.
2018-08-06 12:00:39 -07:00
Kazuaki Ishizaki 1a5e460762 [SPARK-23913][SQL] Add array_intersect function
## What changes were proposed in this pull request?

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

This function returns returns an array of the elements in the intersection of array1 and array2.

Note: The order of elements in the result is not defined.

## How was this patch tested?

Added UTs

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

Closes #21102 from kiszk/SPARK-23913.
2018-08-06 23:27:57 +09:00
Dilip Biswal c1760da5dd [SPARK-25025][SQL] Remove the default value of isAll in INTERSECT/EXCEPT
## What changes were proposed in this pull request?

Having the default value of isAll in the logical plan nodes INTERSECT/EXCEPT could introduce bugs when the callers are not aware of it. This PR removes the default value and makes caller explicitly specify them.

## How was this patch tested?
This is a refactoring change. Existing tests test the functionality already.

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

Closes #22000 from dilipbiswal/SPARK-25025.
2018-08-06 06:56:36 -04:00
Wenchen Fan ac527b5205 [SPARK-24991][SQL] use InternalRow in DataSourceWriter
## What changes were proposed in this pull request?

A follow up of #21118

Since we use `InternalRow` in the read API of data source v2, we should do the same thing for the write API.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21948 from cloud-fan/row-write.
2018-08-06 15:52:01 +08:00
Takuya UESHIN 327bb30075 [SPARK-23911][SQL] Add aggregate function.
## What changes were proposed in this pull request?

This pr adds `aggregate` function which applies a binary operator to an initial state and all elements in the array, and reduces this to a single state. The final state is converted into the final result by applying a finish function.

```sql
> SELECT aggregate(array(1, 2, 3), (acc, x) -> acc + x);
 6
> SELECT aggregate(array(1, 2, 3), (acc, x) -> acc + x, acc -> acc * 10);
 60
```

## How was this patch tested?

Added tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #21982 from ueshin/issues/SPARK-23911/aggregate.
2018-08-05 08:58:35 +09:00
hyukjinkwon 55e3ae6930 [SPARK-25001][BUILD] Fix miscellaneous build warnings
## What changes were proposed in this pull request?

There are many warnings in the current build (for instance see https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.7/4734/console).

**common**:

```
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java:237: warning: [rawtypes] found raw type: LevelDBIterator
[warn]   void closeIterator(LevelDBIterator it) throws IOException {
[warn]                      ^

[warn]   missing type arguments for generic class LevelDBIterator<T>
[warn]   where T is a type-variable:
[warn]     T extends Object declared in class LevelDBIterator
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java:151: warning: [deprecation] group() in AbstractBootstrap has been deprecated
[warn]     if (bootstrap != null && bootstrap.group() != null) {
[warn]                                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java:152: warning: [deprecation] group() in AbstractBootstrap has been deprecated
[warn]       bootstrap.group().shutdownGracefully();
[warn]                ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java:154: warning: [deprecation] childGroup() in ServerBootstrap has been deprecated
[warn]     if (bootstrap != null && bootstrap.childGroup() != null) {
[warn]                                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java:155: warning: [deprecation] childGroup() in ServerBootstrap has been deprecated
[warn]       bootstrap.childGroup().shutdownGracefully();
[warn]                ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/util/NettyUtils.java:112: warning: [deprecation] PooledByteBufAllocator(boolean,int,int,int,int,int,int,int) in PooledByteBufAllocator has been deprecated
[warn]     return new PooledByteBufAllocator(
[warn]            ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java:321: warning: [rawtypes] found raw type: Future
[warn]     public void operationComplete(Future future) throws Exception {
[warn]                                   ^

[warn]   missing type arguments for generic class Future<V>
[warn]   where V is a type-variable:
[warn]     V extends Object declared in interface Future
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java:215: warning: [rawtypes] found raw type: StreamInterceptor
[warn]           StreamInterceptor interceptor = new StreamInterceptor(this, resp.streamId, resp.byteCount,
[warn]           ^

[warn]   missing type arguments for generic class StreamInterceptor<T>
[warn]   where T is a type-variable:
[warn]     T extends Message declared in class StreamInterceptor
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java:215: warning: [rawtypes] found raw type: StreamInterceptor
[warn]           StreamInterceptor interceptor = new StreamInterceptor(this, resp.streamId, resp.byteCount,
[warn]                                               ^

[warn]   missing type arguments for generic class StreamInterceptor<T>
[warn]   where T is a type-variable:
[warn]     T extends Message declared in class StreamInterceptor
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java:215: warning: [unchecked] unchecked call to StreamInterceptor(MessageHandler<T>,String,long,StreamCallback) as a member of the raw type StreamInterceptor
[warn]           StreamInterceptor interceptor = new StreamInterceptor(this, resp.streamId, resp.byteCount,
[warn]                                           ^

[warn]   where T is a type-variable:
[warn]     T extends Message declared in class StreamInterceptor
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java:255: warning: [rawtypes] found raw type: StreamInterceptor
[warn]         StreamInterceptor interceptor = new StreamInterceptor(this, wrappedCallback.getID(),
[warn]         ^

[warn]   missing type arguments for generic class StreamInterceptor<T>
[warn]   where T is a type-variable:
[warn]     T extends Message declared in class StreamInterceptor
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java:255: warning: [rawtypes] found raw type: StreamInterceptor
[warn]         StreamInterceptor interceptor = new StreamInterceptor(this, wrappedCallback.getID(),
[warn]                                             ^

[warn]   missing type arguments for generic class StreamInterceptor<T>
[warn]   where T is a type-variable:
[warn]     T extends Message declared in class StreamInterceptor
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java:255: warning: [unchecked] unchecked call to StreamInterceptor(MessageHandler<T>,String,long,StreamCallback) as a member of the raw type StreamInterceptor
[warn]         StreamInterceptor interceptor = new StreamInterceptor(this, wrappedCallback.getID(),
[warn]                                         ^

[warn]   where T is a type-variable:
[warn]     T extends Message declared in class StreamInterceptor
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java:270: warning: [deprecation] transfered() in FileRegion has been deprecated
[warn]         region.transferTo(byteRawChannel, region.transfered());
[warn]                                                 ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:304: warning: [deprecation] transfered() in FileRegion has been deprecated
[warn]         region.transferTo(byteChannel, region.transfered());
[warn]                                              ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/test/java/org/apache/spark/network/ProtocolSuite.java:119: warning: [deprecation] transfered() in FileRegion has been deprecated
[warn]       while (in.transfered() < in.count()) {
[warn]                ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/test/java/org/apache/spark/network/ProtocolSuite.java:120: warning: [deprecation] transfered() in FileRegion has been deprecated
[warn]         in.transferTo(channel, in.transfered());
[warn]                                  ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java:80: warning: [static] static method should be qualified by type name, Murmur3_x86_32, instead of by an expression
[warn]     Assert.assertEquals(-300363099, hasher.hashUnsafeWords(bytes, offset, 16, 42));
[warn]                                           ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java:84: warning: [static] static method should be qualified by type name, Murmur3_x86_32, instead of by an expression
[warn]     Assert.assertEquals(-1210324667, hasher.hashUnsafeWords(bytes, offset, 16, 42));
[warn]                                            ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java:88: warning: [static] static method should be qualified by type name, Murmur3_x86_32, instead of by an expression
[warn]     Assert.assertEquals(-634919701, hasher.hashUnsafeWords(bytes, offset, 16, 42));
[warn]                                           ^
```

**launcher**:

```
[warn] Pruning sources from previous analysis, due to incompatible CompileSetup.
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java:31: warning: [rawtypes] found raw type: AbstractLauncher
[warn] public abstract class AbstractLauncher<T extends AbstractLauncher> {
[warn]                                                  ^
[warn]   missing type arguments for generic class AbstractLauncher<T>
[warn]   where T is a type-variable:
[warn]     T extends AbstractLauncher declared in class AbstractLauncher
```

**core**:

```
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/main/scala/org/apache/spark/api/r/RBackend.scala:99: method group in class AbstractBootstrap is deprecated: see corresponding Javadoc for more information.
[warn]     if (bootstrap != null && bootstrap.group() != null) {
[warn]                                        ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/main/scala/org/apache/spark/api/r/RBackend.scala💯 method group in class AbstractBootstrap is deprecated: see corresponding Javadoc for more information.
[warn]       bootstrap.group().shutdownGracefully()
[warn]                 ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/main/scala/org/apache/spark/api/r/RBackend.scala:102: method childGroup in class ServerBootstrap is deprecated: see corresponding Javadoc for more information.
[warn]     if (bootstrap != null && bootstrap.childGroup() != null) {
[warn]                                        ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/main/scala/org/apache/spark/api/r/RBackend.scala:103: method childGroup in class ServerBootstrap is deprecated: see corresponding Javadoc for more information.
[warn]       bootstrap.childGroup().shutdownGracefully()
[warn]                 ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala:151: reflective access of structural type member method getData should be enabled
[warn] by making the implicit value scala.language.reflectiveCalls visible.
[warn] This can be achieved by adding the import clause 'import scala.language.reflectiveCalls'
[warn] or by setting the compiler option -language:reflectiveCalls.
[warn] See the Scaladoc for value scala.language.reflectiveCalls for a discussion
[warn] why the feature should be explicitly enabled.
[warn]       val rdd = sc.parallelize(1 to 1).map(concreteObject.getData)
[warn]                                                           ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala:175: reflective access of structural type member value innerObject2 should be enabled
[warn] by making the implicit value scala.language.reflectiveCalls visible.
[warn]       val rdd = sc.parallelize(1 to 1).map(concreteObject.innerObject2.getData)
[warn]                                                           ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala:175: reflective access of structural type member method getData should be enabled
[warn] by making the implicit value scala.language.reflectiveCalls visible.
[warn]       val rdd = sc.parallelize(1 to 1).map(concreteObject.innerObject2.getData)
[warn]                                                                        ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/LocalSparkContext.scala:32: constructor Slf4JLoggerFactory in class Slf4JLoggerFactory is deprecated: see corresponding Javadoc for more information.
[warn]     InternalLoggerFactory.setDefaultFactory(new Slf4JLoggerFactory())
[warn]                                             ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:218: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]         assert(wrapper.stageAttemptId === stages.head.attemptId)
[warn]                                                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:261: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]       stageAttemptId = stages.head.attemptId))
[warn]                                    ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:287: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]       stageAttemptId = stages.head.attemptId))
[warn]                                    ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:471: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]       stageAttemptId = stages.last.attemptId))
[warn]                                    ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:966: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]     listener.onTaskStart(SparkListenerTaskStart(dropped.stageId, dropped.attemptId, task))
[warn]                                                                          ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:972: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]     listener.onTaskEnd(SparkListenerTaskEnd(dropped.stageId, dropped.attemptId,
[warn]                                                                      ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:976: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]       .taskSummary(dropped.stageId, dropped.attemptId, Array(0.25d, 0.50d, 0.75d))
[warn]                                             ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:1146: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]       SparkListenerTaskEnd(stage1.stageId, stage1.attemptId, "taskType", Success, tasks(1), null))
[warn]                                                   ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:1150: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]       SparkListenerTaskEnd(stage1.stageId, stage1.attemptId, "taskType", Success, tasks(0), null))
[warn]                                                   ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala:197: method transfered in trait FileRegion is deprecated: see corresponding Javadoc for more information.
[warn]     while (region.transfered() < region.count()) {
[warn]                   ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala:198: method transfered in trait FileRegion is deprecated: see corresponding Javadoc for more information.
[warn]       region.transferTo(byteChannel, region.transfered())
[warn]                                             ^
```

**sql**:

```
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala:534: abstract type T is unchecked since it is eliminated by erasure
[warn]       assert(partitioning.isInstanceOf[T])
[warn]                                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala:534: abstract type T is unchecked since it is eliminated by erasure
[warn]       assert(partitioning.isInstanceOf[T])
[warn]             ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala:323: inferred existential type Option[Class[_$1]]( forSome { type _$1 }), which cannot be expressed by wildcards,  should be enabled
[warn] by making the implicit value scala.language.existentials visible.
[warn] This can be achieved by adding the import clause 'import scala.language.existentials'
[warn] or by setting the compiler option -language:existentials.
[warn] See the Scaladoc for value scala.language.existentials for a discussion
[warn] why the feature should be explicitly enabled.
[warn]       val optClass = Option(collectionCls)
[warn]                            ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java:226: warning: [deprecation] ParquetFileReader(Configuration,FileMetaData,Path,List<BlockMetaData>,List<ColumnDescriptor>) in ParquetFileReader has been deprecated
[warn]     this.reader = new ParquetFileReader(
[warn]                   ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:178: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]             (descriptor.getType() == PrimitiveType.PrimitiveTypeName.INT32 ||
[warn]                        ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:179: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]             (descriptor.getType() == PrimitiveType.PrimitiveTypeName.INT64  &&
[warn]                        ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:181: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]             descriptor.getType() == PrimitiveType.PrimitiveTypeName.FLOAT ||
[warn]                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:182: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]             descriptor.getType() == PrimitiveType.PrimitiveTypeName.DOUBLE ||
[warn]                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:183: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]             descriptor.getType() == PrimitiveType.PrimitiveTypeName.BINARY))) {
[warn]                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:198: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]         switch (descriptor.getType()) {
[warn]                           ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:221: warning: [deprecation] getTypeLength() in ColumnDescriptor has been deprecated
[warn]             readFixedLenByteArrayBatch(rowId, num, column, descriptor.getTypeLength());
[warn]                                                                      ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:224: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]             throw new IOException("Unsupported type: " + descriptor.getType());
[warn]                                                                    ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:246: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]       descriptor.getType().toString(),
[warn]                 ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:258: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]     switch (descriptor.getType()) {
[warn]                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:384: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]         throw new UnsupportedOperationException("Unsupported type: " + descriptor.getType());
[warn]                                                                                  ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java:458: warning: [static] static variable should be qualified by type name, BaseRepeatedValueVector, instead of by an expression
[warn]       int index = rowId * accessor.OFFSET_WIDTH;
[warn]                                   ^
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java:460: warning: [static] static variable should be qualified by type name, BaseRepeatedValueVector, instead of by an expression
[warn]       int end = offsets.getInt(index + accessor.OFFSET_WIDTH);
[warn]                                                ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/test/scala/org/apache/spark/sql/BenchmarkQueryTest.scala:57: a pure expression does nothing in statement position; you may be omitting necessary parentheses
[warn]       case s => s
[warn]                 ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala:182: inferred existential type org.apache.parquet.column.statistics.Statistics[?0]( forSome { type ?0 <: Comparable[?0] }), which cannot be expressed by wildcards,  should be enabled
[warn] by making the implicit value scala.language.existentials visible.
[warn] This can be achieved by adding the import clause 'import scala.language.existentials'
[warn] or by setting the compiler option -language:existentials.
[warn] See the Scaladoc for value scala.language.existentials for a discussion
[warn] why the feature should be explicitly enabled.
[warn]                 val columnStats = oneBlockColumnMeta.getStatistics
[warn]                                                      ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSinkSuite.scala:146: implicit conversion method conv should be enabled
[warn] by making the implicit value scala.language.implicitConversions visible.
[warn] This can be achieved by adding the import clause 'import scala.language.implicitConversions'
[warn] or by setting the compiler option -language:implicitConversions.
[warn] See the Scaladoc for value scala.language.implicitConversions for a discussion
[warn] why the feature should be explicitly enabled.
[warn]     implicit def conv(x: (Int, Long)): KV = KV(x._1, x._2)
[warn]                  ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/shuffle/ContinuousShuffleSuite.scala:48: implicit conversion method unsafeRow should be enabled
[warn] by making the implicit value scala.language.implicitConversions visible.
[warn]   private implicit def unsafeRow(value: Int) = {
[warn]                        ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala:178: method getType in class ColumnDescriptor is deprecated: see corresponding Javadoc for more information.
[warn]                 assert(oneFooter.getFileMetaData.getSchema.getColumns.get(0).getType() ===
[warn]                                                                              ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala:154: method readAllFootersInParallel in object ParquetFileReader is deprecated: see corresponding Javadoc for more information.
[warn]     ParquetFileReader.readAllFootersInParallel(configuration, fs.getFileStatus(path)).asScala.toSeq
[warn]                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/hive/src/test/java/org/apache/spark/sql/hive/test/Complex.java:679: warning: [cast] redundant cast to Complex
[warn]     Complex typedOther = (Complex)other;
[warn]                          ^
```

**mllib**:

```
[warn] Pruning sources from previous analysis, due to incompatible CompileSetup.
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala:597: match may not be exhaustive.
[warn] It would fail on the following inputs: None, Some((x: Tuple2[?, ?] forSome x not in (?, ?)))
[warn]     val df = dfs.find {
[warn]                       ^
```

This PR does not target fix all of them since some look pretty tricky to fix and there look too many warnings including false positive (like deprecated API but it's used in its test, etc.)

## How was this patch tested?

Existing tests should cover this.

Author: hyukjinkwon <gurwls223@apache.org>

Closes #21975 from HyukjinKwon/remove-build-warnings.
2018-08-04 11:52:49 -05:00
Takuya UESHIN 0ecc132d6b [SPARK-23909][SQL] Add filter function.
## What changes were proposed in this pull request?

This pr adds `filter` function which filters the input array using the given predicate.

```sql
> SELECT filter(array(1, 2, 3), x -> x % 2 == 1);
 array(1, 3)
```

## How was this patch tested?

Added tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #21965 from ueshin/issues/SPARK-23909/filter.
2018-08-04 16:08:53 +09:00
John Zhuge 36ea55e97e [SPARK-24940][SQL] Coalesce and Repartition Hint for SQL Queries
## What changes were proposed in this pull request?

Many Spark SQL users in my company have asked for a way to control the number of output files in Spark SQL. The users prefer not to use function repartition(n) or coalesce(n, shuffle) that require them to write and deploy Scala/Java/Python code. We propose adding the following Hive-style Coalesce and Repartition Hint to Spark SQL:
```
... SELECT /*+ COALESCE(numPartitions) */ ...
... SELECT /*+ REPARTITION(numPartitions) */ ...
```
Multiple such hints are allowed. Multiple nodes are inserted into the logical plan, and the optimizer will pick the leftmost hint.
```
INSERT INTO s SELECT /*+ REPARTITION(100), COALESCE(500), COALESCE(10) */ * FROM t

== Logical Plan ==
'InsertIntoTable 'UnresolvedRelation `s`, false, false
+- 'UnresolvedHint REPARTITION, [100]
   +- 'UnresolvedHint COALESCE, [500]
      +- 'UnresolvedHint COALESCE, [10]
         +- 'Project [*]
            +- 'UnresolvedRelation `t`

== Optimized Logical Plan ==
InsertIntoHadoopFsRelationCommand ...
+- Repartition 100, true
   +- HiveTableRelation ...
```

## How was this patch tested?

All unit tests. Manual tests using explain.

Author: John Zhuge <jzhuge@apache.org>

Closes #21911 from jzhuge/SPARK-24940.
2018-08-04 02:27:15 -04:00
Maxim Gekk 41c2227a23 [SPARK-24722][SQL] pivot() with Column type argument
## What changes were proposed in this pull request?

In the PR, I propose column-based API for the `pivot()` function. It allows using of any column expressions as the pivot column. Also this makes it consistent with how groupBy() works.

## How was this patch tested?

I added new tests to `DataFramePivotSuite` and updated PySpark examples for the `pivot()` function.

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

Closes #21699 from MaxGekk/pivot-column.
2018-08-04 14:17:32 +08:00
Dilip Biswal 19a4531913 [SPARK-24997][SQL] Enable support of MINUS ALL
## What changes were proposed in this pull request?
Enable support for MINUS ALL which was gated at AstBuilder.

## How was this patch tested?
Added tests in SQLQueryTestSuite and modify PlanParserSuite.

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

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

Closes #21963 from dilipbiswal/minus-all.
2018-08-02 22:45:10 -07:00
Chris Horn b0d6967d45 [SPARK-24788][SQL] RelationalGroupedDataset.toString with unresolved exprs should not fail
## What changes were proposed in this pull request?
In the current master, `toString` throws an exception when `RelationalGroupedDataset` has unresolved expressions;
```
scala> spark.range(0, 10).groupBy("id")
res4: org.apache.spark.sql.RelationalGroupedDataset = RelationalGroupedDataset: [grouping expressions: [id: bigint], value: [id: bigint], type: GroupBy]

scala> spark.range(0, 10).groupBy('id)
org.apache.spark.sql.catalyst.analysis.UnresolvedException: Invalid call to dataType on unresolved object, tree: 'id
  at org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute.dataType(unresolved.scala:105)
  at org.apache.spark.sql.RelationalGroupedDataset$$anonfun$12.apply(RelationalGroupedDataset.scala:474)
  at org.apache.spark.sql.RelationalGroupedDataset$$anonfun$12.apply(RelationalGroupedDataset.scala:473)
  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.RelationalGroupedDataset.toString(RelationalGroupedDataset.scala:473)
  at scala.runtime.ScalaRunTime$.scala$runtime$ScalaRunTime$$inner$1(ScalaRunTime.scala:332)
  at scala.runtime.ScalaRunTime$.stringOf(ScalaRunTime.scala:337)
  at scala.runtime.ScalaRunTime$.replStringOf(ScalaRunTime.scala:345)
```
This pr fixed code to handle the unresolved case in `RelationalGroupedDataset.toString`.

Closes #21752

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

Author: Chris Horn <chorn4033@gmail.com>
Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21964 from maropu/SPARK-24788.
2018-08-02 22:40:58 -07:00
Dilip Biswal 73dd6cf9b5 [SPARK-24966][SQL] Implement precedence rules for set operations.
## What changes were proposed in this pull request?

Currently the set operations INTERSECT, UNION and EXCEPT are assigned the same precedence. This PR fixes the problem by giving INTERSECT  higher precedence than UNION and EXCEPT. UNION and EXCEPT operators are evaluated in the order in which they appear in the query from left to right.

This results in change in behavior because of the change in order of evaluations of set operators in a query. The old behavior is still preserved under a newly added config parameter.

Query `:`
```
SELECT * FROM t1
UNION
SELECT * FROM t2
EXCEPT
SELECT * FROM t3
INTERSECT
SELECT * FROM t4
```
Parsed plan before the change `:`
```
== Parsed Logical Plan ==
'Intersect false
:- 'Except false
:  :- 'Distinct
:  :  +- 'Union
:  :     :- 'Project [*]
:  :     :  +- 'UnresolvedRelation `t1`
:  :     +- 'Project [*]
:  :        +- 'UnresolvedRelation `t2`
:  +- 'Project [*]
:     +- 'UnresolvedRelation `t3`
+- 'Project [*]
   +- 'UnresolvedRelation `t4`
```
Parsed plan after the change `:`
```
== Parsed Logical Plan ==
'Except false
:- 'Distinct
:  +- 'Union
:     :- 'Project [*]
:     :  +- 'UnresolvedRelation `t1`
:     +- 'Project [*]
:        +- 'UnresolvedRelation `t2`
+- 'Intersect false
   :- 'Project [*]
   :  +- 'UnresolvedRelation `t3`
   +- 'Project [*]
      +- 'UnresolvedRelation `t4`
```
## How was this patch tested?
Added tests in PlanParserSuite, SQLQueryTestSuite.

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

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

Closes #21941 from dilipbiswal/SPARK-24966.
2018-08-02 22:04:17 -07:00
Maxim Gekk b3f2911eeb [SPARK-24945][SQL] Switching to uniVocity 2.7.3
## What changes were proposed in this pull request?

In the PR, I propose to upgrade uniVocity parser from **2.6.3** to **2.7.3**. The recent version includes a fix for the SPARK-24645 issue and has better performance.

Before changes:
```
Parsing quoted values:                   Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
One quoted string                           33336 / 34122          0.0      666727.0       1.0X

Wide rows with 1000 columns:             Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Select 1000 columns                         90287 / 91713          0.0       90286.9       1.0X
Select 100 columns                          31826 / 36589          0.0       31826.4       2.8X
Select one column                           25738 / 25872          0.0       25737.9       3.5X
count()                                       6931 / 7269          0.1        6931.5      13.0X
```
after:
```
Parsing quoted values:                   Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
One quoted string                           33411 / 33510          0.0      668211.4       1.0X

Wide rows with 1000 columns:             Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Select 1000 columns                         88028 / 89311          0.0       88028.1       1.0X
Select 100 columns                          29010 / 32755          0.0       29010.1       3.0X
Select one column                           22936 / 22953          0.0       22936.5       3.8X
count()                                       6657 / 6740          0.2        6656.6      13.5X
```
Closes #21892

## How was this patch tested?

It was tested by `CSVSuite` and `CSVBenchmarks`

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

Closes #21969 from MaxGekk/univocity-2_7_3.
2018-08-03 08:33:28 +08:00
Kazuaki Ishizaki bbdcc3bf61 [SPARK-22219][SQL] Refactor code to get a value for "spark.sql.codegen.comments"
## What changes were proposed in this pull request?

This PR refactors code to get a value for "spark.sql.codegen.comments" by avoiding `SparkEnv.get.conf`. This PR uses `SQLConf.get.codegenComments` since `SQLConf.get` always returns an instance of `SQLConf`.

## How was this patch tested?

Added test case to `DebuggingSuite`

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

Closes #19449 from kiszk/SPARK-22219.
2018-08-02 18:19:04 -05:00
Liang-Chi Hsieh d0bc3ed679
[SPARK-24896][SQL] Uuid should produce different values for each execution in streaming query
## What changes were proposed in this pull request?

`Uuid`'s results depend on random seed given during analysis. Thus under streaming query, we will have the same uuids in each execution. This seems to be incorrect for streaming query execution.

## How was this patch tested?

Added test.

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

Closes #21854 from viirya/uuid_in_streaming.
2018-08-02 15:35:46 -07:00
Takeshi Yamamuro efef55388f [SPARK-24705][SQL] ExchangeCoordinator broken when duplicate exchanges reused
## What changes were proposed in this pull request?
In the current master, `EnsureRequirements` sets the number of exchanges in `ExchangeCoordinator` before `ReuseExchange`. Then, `ReuseExchange` removes some duplicate exchange and the actual number of registered exchanges changes. Finally, the assertion in `ExchangeCoordinator` fails because the logical number of exchanges and the actual number of registered exchanges become different;
https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala#L201

This pr fixed the issue and the code to reproduce this is as follows;
```
scala> sql("SET spark.sql.adaptive.enabled=true")
scala> sql("SET spark.sql.autoBroadcastJoinThreshold=-1")
scala> val df = spark.range(1).selectExpr("id AS key", "id AS value")
scala> val resultDf = df.join(df, "key").join(df, "key")
scala> resultDf.show
...
  at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec$$anonfun$doExecute$1.apply(ShuffleExchangeExec.scala:119)
  at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:52)
  ... 101 more
Caused by: java.lang.AssertionError: assertion failed
  at scala.Predef$.assert(Predef.scala:156)
  at org.apache.spark.sql.execution.exchange.ExchangeCoordinator.doEstimationIfNecessary(ExchangeCoordinator.scala:201)
  at org.apache.spark.sql.execution.exchange.ExchangeCoordinator.postShuffleRDD(ExchangeCoordinator.scala:259)
  at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec$$anonfun$doExecute$1.apply(ShuffleExchangeExec.scala:124)
  at org.apache.spark.sql.execution.exchange.ShuffleExchangeExec$$anonfun$doExecute$1.apply(ShuffleExchangeExec.scala:119)
  at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:52)
...
```

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21754 from maropu/SPARK-24705-2.
2018-08-02 13:05:36 -07:00
Takuya UESHIN 02f967795b [SPARK-23908][SQL] Add transform function.
## What changes were proposed in this pull request?

This pr adds `transform` function which transforms elements in an array using the function.
Optionally we can take the index of each element as the second argument.

```sql
> SELECT transform(array(1, 2, 3), x -> x + 1);
 array(2, 3, 4)
> SELECT transform(array(1, 2, 3), (x, i) -> x + i);
 array(1, 3, 5)
```

## How was this patch tested?

Added tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #21954 from ueshin/issues/SPARK-23908/transform.
2018-08-02 13:00:33 -07:00
Xiao Li 46110a589f [SPARK-24865][FOLLOW-UP] Remove AnalysisBarrier LogicalPlan Node
## What changes were proposed in this pull request?
Remove the AnalysisBarrier LogicalPlan node, which is useless now.

## How was this patch tested?
N/A

Author: Xiao Li <gatorsmile@gmail.com>

Closes #21962 from gatorsmile/refactor2.
2018-08-02 22:20:41 +08:00
Stavros Kontopoulos a65736996b [SPARK-14540][CORE] Fix remaining major issues for Scala 2.12 Support
## What changes were proposed in this pull request?
This PR addresses issues 2,3 in this [document](https://docs.google.com/document/d/1fbkjEL878witxVQpOCbjlvOvadHtVjYXeB-2mgzDTvk).

* We modified the closure cleaner to identify closures that are implemented via the LambdaMetaFactory mechanism (serializedLambdas) (issue2).

* We also fix the issue due to scala/bug#11016. There are two options for solving the Unit issue, either add () at the end of the closure or use the trick described in the doc. Otherwise overloading resolution does not work (we are not going to eliminate either of the methods) here. Compiler tries to adapt to Unit and makes these two methods candidates for overloading, when there is polymorphic overloading there is no ambiguity (that is the workaround implemented). This does not look that good but it serves its purpose as we need to support two different uses for method: `addTaskCompletionListener`. One that passes a TaskCompletionListener and one that passes a closure that is wrapped with a TaskCompletionListener later on (issue3).

Note: regarding issue 1 in the doc the plan is:

> Do Nothing. Don’t try to fix this as this is only a problem for Java users who would want to use 2.11 binaries. In that case they can cast to MapFunction to be able to utilize lambdas. In Spark 3.0.0 the API should be simplified so that this issue is removed.

## How was this patch tested?
This was manually tested:
```./dev/change-scala-version.sh 2.12
./build/mvn -DskipTests -Pscala-2.12 clean package
./build/mvn -Pscala-2.12 clean package -DwildcardSuites=org.apache.spark.serializer.ProactiveClosureSerializationSuite -Dtest=None
./build/mvn -Pscala-2.12 clean package -DwildcardSuites=org.apache.spark.util.ClosureCleanerSuite -Dtest=None
./build/mvn -Pscala-2.12 clean package -DwildcardSuites=org.apache.spark.streaming.DStreamClosureSuite -Dtest=None```

Author: Stavros Kontopoulos <stavros.kontopoulos@lightbend.com>

Closes #21930 from skonto/scala2.12-sup.
2018-08-02 09:17:09 -05:00
Wenchen Fan ce084d3e06 [SPARK-24990][SQL] merge ReadSupport and ReadSupportWithSchema
## What changes were proposed in this pull request?

Regarding user-specified schema, data sources may have 3 different behaviors:
1. must have a user-specified schema
2. can't have a user-specified schema
3. can accept the user-specified if it's given, or infer the schema.

I added `ReadSupportWithSchema` to support these behaviors, following data source v1. But it turns out we don't need this extra interface. We can just add a `createReader(schema, options)` to `ReadSupport` and make it call `createReader(options)` by default.

TODO: also fix the streaming API in followup PRs.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21946 from cloud-fan/ds-schema.
2018-08-01 15:57:54 -07:00
Yuming Wang 9f558601e8 [SPARK-24937][SQL] Datasource partition table should load empty static partitions
## What changes were proposed in this pull request?

How to reproduce:
```sql
spark-sql> CREATE TABLE tbl AS SELECT 1;
spark-sql> CREATE TABLE tbl1 (c1 BIGINT, day STRING, hour STRING)
         > USING parquet
         > PARTITIONED BY (day, hour);
spark-sql> INSERT INTO TABLE tbl1 PARTITION (day = '2018-07-25', hour='01') SELECT * FROM tbl where 1=0;
spark-sql> SHOW PARTITIONS tbl1;
spark-sql> CREATE TABLE tbl2 (c1 BIGINT)
         > PARTITIONED BY (day STRING, hour STRING);
spark-sql> INSERT INTO TABLE tbl2 PARTITION (day = '2018-07-25', hour='01') SELECT * FROM tbl where 1=0;
spark-sql> SHOW PARTITIONS tbl2;
day=2018-07-25/hour=01
spark-sql>
```
1. Users will be confused about whether the partition data of `tbl1` is generated.
2. Inconsistent with Hive table behavior.

This pr fix this issues.

## How was this patch tested?

unit tests

Author: Yuming Wang <yumwang@ebay.com>

Closes #21883 from wangyum/SPARK-24937.
2018-08-01 13:58:29 -07:00
Kazuaki Ishizaki 95a9d5e3a5 [SPARK-23915][SQL] Add array_except function
## What changes were proposed in this pull request?

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

This function returns returns an array of the elements in array1 but not in array2.

Note: The order of elements in the result is not defined.

## How was this patch tested?

Added UTs.

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

Closes #21103 from kiszk/SPARK-23915.
2018-08-02 02:52:30 +08:00
Wenchen Fan defc54c69a [SPARK-24971][SQL] remove SupportsDeprecatedScanRow
## What changes were proposed in this pull request?

This is a follow up of https://github.com/apache/spark/pull/21118 .

In https://github.com/apache/spark/pull/21118 we added `SupportsDeprecatedScanRow`. Ideally data source should produce `InternalRow` instead of `Row` for better performance. We should remove `SupportsDeprecatedScanRow` and encourage data sources to produce `InternalRow`, which is also very easy to build.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21921 from cloud-fan/row.
2018-08-01 21:39:35 +08:00
Reynold Xin 1efffb7993 [SPARK-24982][SQL] UDAF resolution should not throw AssertionError
## What changes were proposed in this pull request?
When user calls anUDAF with the wrong number of arguments, Spark previously throws an AssertionError, which is not supposed to be a user-facing exception.  This patch updates it to throw AnalysisException instead, so it is consistent with a regular UDF.

## How was this patch tested?
Updated test case udaf.sql.

Author: Reynold Xin <rxin@databricks.com>

Closes #21938 from rxin/SPARK-24982.
2018-08-01 00:15:31 -07:00
Reynold Xin 1f7e22c72c [SPARK-24951][SQL] Table valued functions should throw AnalysisException
## What changes were proposed in this pull request?
Previously TVF resolution could throw IllegalArgumentException if the data type is null type. This patch replaces that exception with AnalysisException, enriched with positional information, to improve error message reporting and to be more consistent with rest of Spark SQL.

## How was this patch tested?
Updated the test case in table-valued-functions.sql.out, which is how I identified this problem in the first place.

Author: Reynold Xin <rxin@databricks.com>

Closes #21934 from rxin/SPARK-24951.
2018-07-31 22:25:40 -07:00
Mauro Palsgraaf 4ac2126bc6 [SPARK-24536] Validate that an evaluated limit clause cannot be null
## What changes were proposed in this pull request?

It proposes a version in which nullable expressions are not valid in the limit clause

## How was this patch tested?

It was tested with unit and e2e tests.

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

Author: Mauro Palsgraaf <mauropalsgraaf@hotmail.com>

Closes #21807 from mauropalsgraaf/SPARK-24536.
2018-07-31 08:18:08 -07:00
maryannxue b4fd75fb9b [SPARK-24972][SQL] PivotFirst could not handle pivot columns of complex types
## What changes were proposed in this pull request?

When the pivot column is of a complex type, the eval() result will be an UnsafeRow, while the keys of the HashMap for column value matching is a GenericInternalRow. As a result, there will be no match and the result will always be empty.
So for a pivot column of complex-types, we should:
1) If the complex-type is not comparable (orderable), throw an Exception. It cannot be a pivot column.
2) Otherwise, if it goes through the `PivotFirst` code path, `PivotFirst` should use a TreeMap instead of HashMap for such columns.

This PR has also reverted the walk-around in Analyzer that had been introduced to avoid this `PivotFirst` issue.

## How was this patch tested?

Added UT.

Author: maryannxue <maryannxue@apache.org>

Closes #21926 from maryannxue/pivot_followup.
2018-07-30 23:43:53 -07:00
Reynold Xin abbb4ab4d8 [SPARK-24865][SQL] Remove AnalysisBarrier addendum
## What changes were proposed in this pull request?
I didn't want to pollute the diff in the previous PR and left some TODOs. This is a follow-up to address those TODOs.

## How was this patch tested?
Should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #21896 from rxin/SPARK-24865-addendum.
2018-07-30 14:05:45 -07:00
Takeshi Yamamuro 47d84e4d0e [SPARK-22814][SQL] Support Date/Timestamp in a JDBC partition column
## What changes were proposed in this pull request?
This pr supported Date/Timestamp in a JDBC partition column (a numeric column is only supported in the master). This pr also modified code to verify a partition column type;
```
val jdbcTable = spark.read
 .option("partitionColumn", "text")
 .option("lowerBound", "aaa")
 .option("upperBound", "zzz")
 .option("numPartitions", 2)
 .jdbc("jdbc:postgresql:postgres", "t", options)

// with this pr
org.apache.spark.sql.AnalysisException: Partition column type should be numeric, date, or timestamp, but string found.;
  at org.apache.spark.sql.execution.datasources.jdbc.JDBCRelation$.verifyAndGetNormalizedPartitionColumn(JDBCRelation.scala:165)
  at org.apache.spark.sql.execution.datasources.jdbc.JDBCRelation$.columnPartition(JDBCRelation.scala:85)
  at org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider.createRelation(JdbcRelationProvider.scala:36)
  at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:317)

// without this pr
java.lang.NumberFormatException: For input string: "aaa"
  at java.lang.NumberFormatException.forInputString(NumberFormatException.java:65)
  at java.lang.Long.parseLong(Long.java:589)
  at java.lang.Long.parseLong(Long.java:631)
  at scala.collection.immutable.StringLike$class.toLong(StringLike.scala:277)
```

Closes #19999

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21834 from maropu/SPARK-22814.
2018-07-30 07:42:00 -07:00
Gengliang Wang b90bfe3c42 [SPARK-24771][BUILD] Upgrade Apache AVRO to 1.8.2
## What changes were proposed in this pull request?

Upgrade Apache Avro from 1.7.7 to 1.8.2. The major new features:

1. More logical types. From the spec of 1.8.2 https://avro.apache.org/docs/1.8.2/spec.html#Logical+Types we can see comparing to [1.7.7](https://avro.apache.org/docs/1.7.7/spec.html#Logical+Types), the new version support:
    - Date
    - Time (millisecond precision)
    - Time (microsecond precision)
    - Timestamp (millisecond precision)
    - Timestamp (microsecond precision)
    - Duration

2. Single-object encoding: https://avro.apache.org/docs/1.8.2/spec.html#single_object_encoding

This PR aims to update Apache Spark to support these new features.

## How was this patch tested?

Unit test

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

Closes #21761 from gengliangwang/upgrade_avro_1.8.
2018-07-30 07:30:47 -07:00
hyukjinkwon bfe60fcdb4 [SPARK-24934][SQL] Explicitly whitelist supported types in upper/lower bounds for in-memory partition pruning
## What changes were proposed in this pull request?

Looks we intentionally set `null` for upper/lower bounds for complex types and don't use it. However, these look used in in-memory partition pruning, which ends up with incorrect results.

This PR proposes to explicitly whitelist the supported types.

```scala
val df = Seq(Array("a", "b"), Array("c", "d")).toDF("arrayCol")
df.cache().filter("arrayCol > array('a', 'b')").show()
```

```scala
val df = sql("select cast('a' as binary) as a")
df.cache().filter("a == cast('a' as binary)").show()
```

**Before:**

```
+--------+
|arrayCol|
+--------+
+--------+
```

```
+---+
|  a|
+---+
+---+
```

**After:**

```
+--------+
|arrayCol|
+--------+
|  [c, d]|
+--------+
```

```
+----+
|   a|
+----+
|[61]|
+----+
```

## How was this patch tested?

Unit tests were added and manually tested.

Author: hyukjinkwon <gurwls223@apache.org>

Closes #21882 from HyukjinKwon/stats-filter.
2018-07-30 13:20:03 +08:00
Dilip Biswal 65a4bc143a [SPARK-21274][SQL] Implement INTERSECT ALL clause
## What changes were proposed in this pull request?
Implements INTERSECT ALL clause through query rewrites using existing operators in Spark.  Please refer to [Link](https://drive.google.com/open?id=1nyW0T0b_ajUduQoPgZLAsyHK8s3_dko3ulQuxaLpUXE) for the design.

Input Query
``` SQL
SELECT c1 FROM ut1 INTERSECT ALL SELECT c1 FROM ut2
```
Rewritten Query
```SQL
   SELECT c1
    FROM (
         SELECT replicate_row(min_count, c1)
         FROM (
              SELECT c1,
                     IF (vcol1_cnt > vcol2_cnt, vcol2_cnt, vcol1_cnt) AS min_count
              FROM (
                   SELECT   c1, count(vcol1) as vcol1_cnt, count(vcol2) as vcol2_cnt
                   FROM (
                        SELECT c1, true as vcol1, null as vcol2 FROM ut1
                        UNION ALL
                        SELECT c1, null as vcol1, true as vcol2 FROM ut2
                        ) AS union_all
                   GROUP BY c1
                   HAVING vcol1_cnt >= 1 AND vcol2_cnt >= 1
                  )
              )
          )
```

## How was this patch tested?
Added test cases in SQLQueryTestSuite, DataFrameSuite, SetOperationSuite

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

Closes #21886 from dilipbiswal/dkb_intersect_all_final.
2018-07-29 22:11:01 -07:00
hyukjinkwon 6690924c49 [MINOR] Avoid the 'latest' link that might vary per release in functions.scala's comment
## What changes were proposed in this pull request?

This PR propose to address https://github.com/apache/spark/pull/21318#discussion_r187843125 comment.

This is rather a nit but looks we better avoid to update the link for each release since it always points the latest (it doesn't look like worth enough updating release guide on the other hand as well).

## How was this patch tested?

N/A

Author: hyukjinkwon <gurwls223@apache.org>

Closes #21907 from HyukjinKwon/minor-fix.
2018-07-30 10:02:29 +08:00
liulijia 2c54aae1bc [SPARK-24809][SQL] Serializing LongToUnsafeRowMap in executor may result in data error
When join key is long or int in broadcast join, Spark will use `LongToUnsafeRowMap` to store key-values of the table witch will be broadcasted. But, when `LongToUnsafeRowMap` is broadcasted to executors, and it is too big to hold in memory, it will be stored in disk. At that time, because `write` uses a variable `cursor` to determine how many bytes in `page` of `LongToUnsafeRowMap` will be write out and the `cursor` was not restore when deserializing, executor will write out nothing from page into disk.

## What changes were proposed in this pull request?
Restore cursor value when deserializing.

Author: liulijia <liutang123@yeah.net>

Closes #21772 from liutang123/SPARK-24809.
2018-07-29 13:13:00 -07:00
Li Jin e8752095a0 [SPARK-24624][SQL][PYTHON] Support mixture of Python UDF and Scalar Pandas UDF
## What changes were proposed in this pull request?

This PR add supports for using mixed Python UDF and Scalar Pandas UDF, in the following two cases:

(1)
```
from pyspark.sql.functions import udf, pandas_udf

udf('int')
def f1(x):
    return x + 1

pandas_udf('int')
def f2(x):
    return x + 1

df = spark.range(0, 1).toDF('v') \
    .withColumn('foo', f1(col('v'))) \
    .withColumn('bar', f2(col('v')))

```

QueryPlan:
```
>>> df.explain(True)
== Parsed Logical Plan ==
'Project [v#2L, foo#5, f2('v) AS bar#9]
+- AnalysisBarrier
      +- Project [v#2L, f1(v#2L) AS foo#5]
         +- Project [id#0L AS v#2L]
            +- Range (0, 1, step=1, splits=Some(4))

== Analyzed Logical Plan ==
v: bigint, foo: int, bar: int
Project [v#2L, foo#5, f2(v#2L) AS bar#9]
+- Project [v#2L, f1(v#2L) AS foo#5]
   +- Project [id#0L AS v#2L]
      +- Range (0, 1, step=1, splits=Some(4))

== Optimized Logical Plan ==
Project [id#0L AS v#2L, f1(id#0L) AS foo#5, f2(id#0L) AS bar#9]
+- Range (0, 1, step=1, splits=Some(4))

== Physical Plan ==
*(2) Project [id#0L AS v#2L, pythonUDF0#13 AS foo#5, pythonUDF0#14 AS bar#9]
+- ArrowEvalPython [f2(id#0L)], [id#0L, pythonUDF0#13, pythonUDF0#14]
   +- BatchEvalPython [f1(id#0L)], [id#0L, pythonUDF0#13]
      +- *(1) Range (0, 1, step=1, splits=4)
```

(2)
```
from pyspark.sql.functions import udf, pandas_udf
udf('int')
def f1(x):
    return x + 1

pandas_udf('int')
def f2(x):
    return x + 1

df = spark.range(0, 1).toDF('v')
df = df.withColumn('foo', f2(f1(df['v'])))
```

QueryPlan:
```
>>> df.explain(True)
== Parsed Logical Plan ==
Project [v#21L, f2(f1(v#21L)) AS foo#46]
+- AnalysisBarrier
      +- Project [v#21L, f1(f2(v#21L)) AS foo#39]
         +- Project [v#21L, <lambda>(<lambda>(v#21L)) AS foo#32]
            +- Project [v#21L, <lambda>(<lambda>(v#21L)) AS foo#25]
               +- Project [id#19L AS v#21L]
                  +- Range (0, 1, step=1, splits=Some(4))

== Analyzed Logical Plan ==
v: bigint, foo: int
Project [v#21L, f2(f1(v#21L)) AS foo#46]
+- Project [v#21L, f1(f2(v#21L)) AS foo#39]
   +- Project [v#21L, <lambda>(<lambda>(v#21L)) AS foo#32]
      +- Project [v#21L, <lambda>(<lambda>(v#21L)) AS foo#25]
         +- Project [id#19L AS v#21L]
            +- Range (0, 1, step=1, splits=Some(4))

== Optimized Logical Plan ==
Project [id#19L AS v#21L, f2(f1(id#19L)) AS foo#46]
+- Range (0, 1, step=1, splits=Some(4))

== Physical Plan ==
*(2) Project [id#19L AS v#21L, pythonUDF0#50 AS foo#46]
+- ArrowEvalPython [f2(pythonUDF0#49)], [id#19L, pythonUDF0#49, pythonUDF0#50]
   +- BatchEvalPython [f1(id#19L)], [id#19L, pythonUDF0#49]
      +- *(1) Range (0, 1, step=1, splits=4)
```

## How was this patch tested?

New tests are added to BatchEvalPythonExecSuite and ScalarPandasUDFTests

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

Closes #21650 from icexelloss/SPARK-24624-mix-udf.
2018-07-28 13:41:07 +08:00
Reynold Xin 6424b146c9 [MINOR] Update docs for functions.scala to make it clear not all the built-in functions are defined there
The title summarizes the change.

Author: Reynold Xin <rxin@databricks.com>

Closes #21318 from rxin/functions.
2018-07-27 17:24:55 -07:00
Dilip Biswal 10f1f19659 [SPARK-21274][SQL] Implement EXCEPT ALL clause.
## What changes were proposed in this pull request?
Implements EXCEPT ALL clause through query rewrites using existing operators in Spark. In this PR, an internal UDTF (replicate_rows) is added to aid in preserving duplicate rows. Please refer to [Link](https://drive.google.com/open?id=1nyW0T0b_ajUduQoPgZLAsyHK8s3_dko3ulQuxaLpUXE) for the design.

**Note** This proposed UDTF is kept as a internal function that is purely used to aid with this particular rewrite to give us flexibility to change to a more generalized UDTF in future.

Input Query
``` SQL
SELECT c1 FROM ut1 EXCEPT ALL SELECT c1 FROM ut2
```
Rewritten Query
```SQL
SELECT c1
    FROM (
     SELECT replicate_rows(sum_val, c1)
       FROM (
         SELECT c1, sum_val
           FROM (
             SELECT c1, sum(vcol) AS sum_val
               FROM (
                 SELECT 1L as vcol, c1 FROM ut1
                 UNION ALL
                 SELECT -1L as vcol, c1 FROM ut2
              ) AS union_all
            GROUP BY union_all.c1
          )
        WHERE sum_val > 0
       )
   )
```

## How was this patch tested?
Added test cases in SQLQueryTestSuite, DataFrameSuite and SetOperationSuite

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

Closes #21857 from dilipbiswal/dkb_except_all_final.
2018-07-27 13:47:33 -07:00
pkuwm ef6c8395c4 [SPARK-23928][SQL] Add shuffle collection function.
## What changes were proposed in this pull request?

This PR adds a new collection function: shuffle. It generates a random permutation of the given array. This implementation uses the "inside-out" version of Fisher-Yates algorithm.

## How was this patch tested?

New tests are added to CollectionExpressionsSuite.scala and DataFrameFunctionsSuite.scala.

Author: Takuya UESHIN <ueshin@databricks.com>
Author: pkuwm <ihuizhi.lu@gmail.com>

Closes #21802 from ueshin/issues/SPARK-23928/shuffle.
2018-07-27 23:02:48 +09:00
maryannxue 21fcac1645 [SPARK-24288][SQL] Add a JDBC Option to enable preventing predicate pushdown
## What changes were proposed in this pull request?

Add a JDBC Option "pushDownPredicate" (default `true`) to allow/disallow predicate push-down in JDBC data source.

## How was this patch tested?

Add a test in `JDBCSuite`

Author: maryannxue <maryannxue@apache.org>

Closes #21875 from maryannxue/spark-24288.
2018-07-26 23:47:32 -07:00
Reynold Xin e6e9031d7b [SPARK-24865] Remove AnalysisBarrier
## What changes were proposed in this pull request?
AnalysisBarrier was introduced in SPARK-20392 to improve analysis speed (don't re-analyze nodes that have already been analyzed).

Before AnalysisBarrier, we already had some infrastructure in place, with analysis specific functions (resolveOperators and resolveExpressions). These functions do not recursively traverse down subplans that are already analyzed (with a mutable boolean flag _analyzed). The issue with the old system was that developers started using transformDown, which does a top-down traversal of the plan tree, because there was not top-down resolution function, and as a result analyzer performance became pretty bad.

In order to fix the issue in SPARK-20392, AnalysisBarrier was introduced as a special node and for this special node, transform/transformUp/transformDown don't traverse down. However, the introduction of this special node caused a lot more troubles than it solves. This implicit node breaks assumptions and code in a few places, and it's hard to know when analysis barrier would exist, and when it wouldn't. Just a simple search of AnalysisBarrier in PR discussions demonstrates it is a source of bugs and additional complexity.

Instead, this pull request removes AnalysisBarrier and reverts back to the old approach. We added infrastructure in tests that fail explicitly if transform methods are used in the analyzer.

## How was this patch tested?
Added a test suite AnalysisHelperSuite for testing the resolve* methods and transform* methods.

Author: Reynold Xin <rxin@databricks.com>
Author: Xiao Li <gatorsmile@gmail.com>

Closes #21822 from rxin/SPARK-24865.
2018-07-27 14:29:05 +08:00
Gengliang Wang fa09d91925 [SPARK-24919][BUILD] New linter rule for sparkContext.hadoopConfiguration
## What changes were proposed in this pull request?

In most cases, we should use `spark.sessionState.newHadoopConf()` instead of `sparkContext.hadoopConfiguration`, so that the hadoop configurations specified in Spark session
configuration will come into effect.

Add a rule matching `spark.sparkContext.hadoopConfiguration` or `spark.sqlContext.sparkContext.hadoopConfiguration` to prevent the usage.
## How was this patch tested?

Unit test

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

Closes #21873 from gengliangwang/linterRule.
2018-07-26 16:50:59 -07:00
maryannxue 5ed7660d14 [SPARK-24802][SQL][FOLLOW-UP] Add a new config for Optimization Rule Exclusion
## What changes were proposed in this pull request?

This is an extension to the original PR, in which rule exclusion did not work for classes derived from Optimizer, e.g., SparkOptimizer.
To solve this issue, Optimizer and its derived classes will define/override `defaultBatches` and `nonExcludableRules` in order to define its default rule set as well as rules that cannot be excluded by the SQL config. In the meantime, Optimizer's `batches` method is dedicated to the rule exclusion logic and is defined "final".

## How was this patch tested?

Added UT.

Author: maryannxue <maryannxue@apache.org>

Closes #21876 from maryannxue/rule-exclusion.
2018-07-26 11:06:23 -07:00
Dongjoon Hyun 58353d7f4b [SPARK-24924][SQL] Add mapping for built-in Avro data source
## What changes were proposed in this pull request?

This PR aims to the followings.
1. Like `com.databricks.spark.csv` mapping, we had better map `com.databricks.spark.avro` to built-in Avro data source.
2. Remove incorrect error message, `Please find an Avro package at ...`.

## How was this patch tested?

Pass the newly added tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #21878 from dongjoon-hyun/SPARK-24924.
2018-07-26 16:11:03 +08:00
Takuya UESHIN c9b233d414 [SPARK-24878][SQL] Fix reverse function for array type of primitive type containing null.
## What changes were proposed in this pull request?

If we use `reverse` function for array type of primitive type containing `null` and the child array is `UnsafeArrayData`, the function returns a wrong result because `UnsafeArrayData` doesn't define the behavior of re-assignment, especially we can't set a valid value after we set `null`.

## How was this patch tested?

Added some tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #21830 from ueshin/issues/SPARK-24878/fix_reverse.
2018-07-26 15:06:13 +08:00
Xiao Li d2e7deb59f [SPARK-24867][SQL] Add AnalysisBarrier to DataFrameWriter
## What changes were proposed in this pull request?
```Scala
      val udf1 = udf({(x: Int, y: Int) => x + y})
      val df = spark.range(0, 3).toDF("a")
        .withColumn("b", udf1($"a", udf1($"a", lit(10))))
      df.cache()
      df.write.saveAsTable("t")
```
Cache is not being used because the plans do not match with the cached plan. This is a regression caused by the changes we made in AnalysisBarrier, since not all the Analyzer rules are idempotent.

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

Also found a bug in the DSV1 write path. This is not a regression. Thus, opened a separate JIRA https://issues.apache.org/jira/browse/SPARK-24869

Author: Xiao Li <gatorsmile@gmail.com>

Closes #21821 from gatorsmile/testMaster22.
2018-07-25 17:22:37 -07:00
Koert Kuipers 17f469bc80 [SPARK-24860][SQL] Support setting of partitionOverWriteMode in output options for writing DataFrame
## What changes were proposed in this pull request?

Besides spark setting spark.sql.sources.partitionOverwriteMode also allow setting partitionOverWriteMode per write

## How was this patch tested?

Added unit test in InsertSuite

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

Author: Koert Kuipers <koert@tresata.com>

Closes #21818 from koertkuipers/feat-partition-overwrite-mode-per-write.
2018-07-25 13:06:03 -07:00
Maxim Gekk 2f77616e1d [SPARK-24849][SPARK-24911][SQL] Converting a value of StructType to a DDL string
## What changes were proposed in this pull request?

In the PR, I propose to extend the `StructType`/`StructField` classes by new method `toDDL` which converts a value of the `StructType`/`StructField` type to a string formatted in DDL style. The resulted string can be used in a table creation.

The `toDDL` method of `StructField` is reused in `SHOW CREATE TABLE`. In this way the PR fixes the bug of unquoted names of nested fields.

## How was this patch tested?

I add a test for checking the new method and 2 round trip tests: `fromDDL` -> `toDDL` and `toDDL` -> `fromDDL`

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

Closes #21803 from MaxGekk/to-ddl.
2018-07-25 11:09:12 -07:00
Yuming Wang 7a5fd4a91e [SPARK-18874][SQL][FOLLOW-UP] Improvement type mismatched message
## What changes were proposed in this pull request?
Improvement `IN` predicate type mismatched message:
```sql
Mismatched columns:
[(, t, 4, ., `, t, 4, a, `, :, d, o, u, b, l, e, ,,  , t, 5, ., `, t, 5, a, `, :, d, e, c, i, m, a, l, (, 1, 8, ,, 0, ), ), (, t, 4, ., `, t, 4, c, `, :, s, t, r, i, n, g, ,,  , t, 5, ., `, t, 5, c, `, :, b, i, g, i, n, t, )]
```
After this patch:
```sql
Mismatched columns:
[(t4.`t4a`:double, t5.`t5a`:decimal(18,0)), (t4.`t4c`:string, t5.`t5c`:bigint)]
```

## How was this patch tested?

unit tests

Author: Yuming Wang <yumwang@ebay.com>

Closes #21863 from wangyum/SPARK-18874.
2018-07-24 23:59:13 -07:00
crafty-coder 78e0a725e0 [SPARK-19018][SQL] Add support for custom encoding on csv writer
## What changes were proposed in this pull request?

Add support for custom encoding on csv writer, see https://issues.apache.org/jira/browse/SPARK-19018

## How was this patch tested?

Added two unit tests in CSVSuite

Author: crafty-coder <carlospb86@gmail.com>
Author: Carlos <crafty-coder@users.noreply.github.com>

Closes #20949 from crafty-coder/master.
2018-07-25 14:17:20 +08:00
Dilip Biswal afb0627536 [SPARK-23957][SQL] Sorts in subqueries are redundant and can be removed
## What changes were proposed in this pull request?
Thanks to henryr for the original idea at https://github.com/apache/spark/pull/21049

Description from the original PR :
Subqueries (at least in SQL) have 'bag of tuples' semantics. Ordering
them is therefore redundant (unless combined with a limit).

This patch removes the top sort operators from the subquery plans.

This closes https://github.com/apache/spark/pull/21049.

## How was this patch tested?
Added test cases in SubquerySuite to cover in, exists and scalar subqueries.

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

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

Closes #21853 from dilipbiswal/SPARK-23957.
2018-07-24 20:46:27 -07:00
DB Tsai d4c3415894 [SPARK-24890][SQL] Short circuiting the if condition when trueValue and falseValue are the same
## What changes were proposed in this pull request?

When `trueValue` and `falseValue` are semantic equivalence, the condition expression in `if` can be removed to avoid extra computation in runtime.

## How was this patch tested?

Test added.

Author: DB Tsai <d_tsai@apple.com>

Closes #21848 from dbtsai/short-circuit-if.
2018-07-24 20:21:11 -07:00
maryannxue c26b092169 [SPARK-24891][SQL] Fix HandleNullInputsForUDF rule
## What changes were proposed in this pull request?

The HandleNullInputsForUDF would always add a new `If` node every time it is applied. That would cause a difference between the same plan being analyzed once and being analyzed twice (or more), thus raising issues like plan not matched in the cache manager. The solution is to mark the arguments as null-checked, which is to add a "KnownNotNull" node above those arguments, when adding the UDF under an `If` node, because clearly the UDF will not be called when any of those arguments is null.

## How was this patch tested?

Add new tests under sql/UDFSuite and AnalysisSuite.

Author: maryannxue <maryannxue@apache.org>

Closes #21851 from maryannxue/spark-24891.
2018-07-24 19:35:34 -07:00
Ryan Blue 9d27541a85 [SPARK-23325] Use InternalRow when reading with DataSourceV2.
## What changes were proposed in this pull request?

This updates the DataSourceV2 API to use InternalRow instead of Row for the default case with no scan mix-ins.

Support for readers that produce Row is added through SupportsDeprecatedScanRow, which matches the previous API. Readers that used Row now implement this class and should be migrated to InternalRow.

Readers that previously implemented SupportsScanUnsafeRow have been migrated to use no SupportsScan mix-ins and produce InternalRow.

## How was this patch tested?

This uses existing tests.

Author: Ryan Blue <blue@apache.org>

Closes #21118 from rdblue/SPARK-23325-datasource-v2-internal-row.
2018-07-24 10:46:36 -07:00
10129659 13a67b070d [SPARK-24870][SQL] Cache can't work normally if there are case letters in SQL
## What changes were proposed in this pull request?
Modified the canonicalized to not case-insensitive.
Before the PR, cache can't work normally if there are case letters in SQL,
for example:
     sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING) USING hive")

    sql("select key, sum(case when Key > 0 then 1 else 0 end) as positiveNum " +
      "from src group by key").cache().createOrReplaceTempView("src_cache")
    sql(
      s"""select a.key
           from
           (select key from src_cache where positiveNum = 1)a
           left join
           (select key from src_cache )b
           on a.key=b.key
        """).explain

The physical plan of the sql is:
![image](https://user-images.githubusercontent.com/26834091/42979518-3decf0fa-8c05-11e8-9837-d5e4c334cb1f.png)

The subquery "select key from src_cache where positiveNum = 1" on the left of join can use the cache data, but the subquery "select key from src_cache" on the right of join cannot use the cache data.

## How was this patch tested?

new added test

Author: 10129659 <chen.yanshan@zte.com.cn>

Closes #21823 from eatoncys/canonicalized.
2018-07-23 23:05:08 -07:00
Tathagata Das 61f0ca4f1c [SPARK-24699][SS] Make watermarks work with Trigger.Once by saving updated watermark to commit log
## What changes were proposed in this pull request?

Streaming queries with watermarks do not work with Trigger.Once because of the following.
- Watermark is updated in the driver memory after a batch completes, but it is persisted to checkpoint (in the offset log) only when the next batch is planned
- In trigger.once, the query terminated as soon as one batch has completed. Hence, the updated watermark is never persisted anywhere.

The simple solution is to persist the updated watermark value in the commit log when a batch is marked as completed. Then the next batch, in the next trigger.once run can pick it up from the commit log.

## How was this patch tested?
new unit tests

Co-authored-by: Tathagata Das <tathagata.das1565gmail.com>
Co-authored-by: c-horn <chorn4033gmail.com>

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

Closes #21746 from tdas/SPARK-24699.
2018-07-23 13:03:32 -07:00
Onur Satici 2edf17effd [SPARK-24850][SQL] fix str representation of CachedRDDBuilder
## What changes were proposed in this pull request?
As of https://github.com/apache/spark/pull/21018, InMemoryRelation includes its cacheBuilder when logging query plans. This PR changes the string representation of the CachedRDDBuilder to not include the cached spark plan.

## How was this patch tested?

spark-shell, query:
```
var df_cached = spark.read.format("csv").option("header", "true").load("test.csv").cache()
0 to 1 foreach { _ =>
df_cached = df_cached.join(spark.read.format("csv").option("header", "true").load("test.csv"), "A").cache()
}
df_cached.explain
```
as of master results in:
```
== Physical Plan ==
InMemoryTableScan [A#10, B#11, B#35, B#87]
+- InMemoryRelation [A#10, B#11, B#35, B#87], CachedRDDBuilder(true,10000,StorageLevel(disk, memory, deserialized, 1 replicas),*(2) Project [A#10, B#11, B#35, B#87]
+- *(2) BroadcastHashJoin [A#10], [A#86], Inner, BuildRight
:- *(2) Filter isnotnull(A#10)
: +- InMemoryTableScan [A#10, B#11, B#35], [isnotnull(A#10)]
: +- InMemoryRelation [A#10, B#11, B#35], CachedRDDBuilder(true,10000,StorageLevel(disk, memory, deserialized, 1 replicas),*(2) Project [A#10, B#11, B#35]
+- *(2) BroadcastHashJoin [A#10], [A#34], Inner, BuildRight
:- *(2) Filter isnotnull(A#10)
: +- InMemoryTableScan [A#10, B#11], [isnotnull(A#10)]
: +- InMemoryRelation [A#10, B#11], CachedRDDBuilder(true,10000,StorageLevel(disk, memory, deserialized, 1 replicas),*(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<A:string,B:string>
,None)
: +- *(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<A:string,B:string>
+- BroadcastExchange HashedRelationBroadcastMode(List(input[0, string, false]))
+- *(1) Filter isnotnull(A#34)
+- InMemoryTableScan [A#34, B#35], [isnotnull(A#34)]
+- InMemoryRelation [A#34, B#35], CachedRDDBuilder(true,10000,StorageLevel(disk, memory, deserialized, 1 replicas),*(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<A:string,B:string>
,None)
+- *(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<A:string,B:string>
,None)
: +- *(2) Project [A#10, B#11, B#35]
: +- *(2) BroadcastHashJoin [A#10], [A#34], Inner, BuildRight
: :- *(2) Filter isnotnull(A#10)
: : +- InMemoryTableScan [A#10, B#11], [isnotnull(A#10)]
: : +- InMemoryRelation [A#10, B#11], CachedRDDBuilder(true,10000,StorageLevel(disk, memory, deserialized, 1 replicas),*(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<A:string,B:string>
,None)
: : +- *(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<A:string,B:string>
: +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, string, false]))
: +- *(1) Filter isnotnull(A#34)
: +- InMemoryTableScan [A#34, B#35], [isnotnull(A#34)]
: +- InMemoryRelation [A#34, B#35], CachedRDDBuilder(true,10000,StorageLevel(disk, memory, deserialized, 1 replicas),*(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<A:string,B:string>
,None)
: +- *(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<A:string,B:string>
+- BroadcastExchange HashedRelationBroadcastMode(List(input[0, string, false]))
+- *(1) Filter isnotnull(A#86)
+- InMemoryTableScan [A#86, B#87], [isnotnull(A#86)]
+- InMemoryRelation [A#86, B#87], CachedRDDBuilder(true,10000,StorageLevel(disk, memory, deserialized, 1 replicas),*(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<A:string,B:string>
,None)
+- *(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<A:string,B:string>
,None)
+- *(2) Project [A#10, B#11, B#35, B#87]
+- *(2) BroadcastHashJoin [A#10], [A#86], Inner, BuildRight
:- *(2) Filter isnotnull(A#10)
: +- InMemoryTableScan [A#10, B#11, B#35], [isnotnull(A#10)]
: +- InMemoryRelation [A#10, B#11, B#35], CachedRDDBuilder(true,10000,StorageLevel(disk, memory, deserialized, 1 replicas),*(2) Project [A#10, B#11, B#35]
+- *(2) BroadcastHashJoin [A#10], [A#34], Inner, BuildRight
:- *(2) Filter isnotnull(A#10)
: +- InMemoryTableScan [A#10, B#11], [isnotnull(A#10)]
: +- InMemoryRelation [A#10, B#11], CachedRDDBuilder(true,10000,StorageLevel(disk, memory, deserialized, 1 replicas),*(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<A:string,B:string>
,None)
: +- *(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<A:string,B:string>
+- BroadcastExchange HashedRelationBroadcastMode(List(input[0, string, false]))
+- *(1) Filter isnotnull(A#34)
+- InMemoryTableScan [A#34, B#35], [isnotnull(A#34)]
+- InMemoryRelation [A#34, B#35], CachedRDDBuilder(true,10000,StorageLevel(disk, memory, deserialized, 1 replicas),*(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<A:string,B:string>
,None)
+- *(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<A:string,B:string>
,None)
: +- *(2) Project [A#10, B#11, B#35]
: +- *(2) BroadcastHashJoin [A#10], [A#34], Inner, BuildRight
: :- *(2) Filter isnotnull(A#10)
: : +- InMemoryTableScan [A#10, B#11], [isnotnull(A#10)]
: : +- InMemoryRelation [A#10, B#11], CachedRDDBuilder(true,10000,StorageLevel(disk, memory, deserialized, 1 replicas),*(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<A:string,B:string>
,None)
: : +- *(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<A:string,B:string>
: +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, string, false]))
: +- *(1) Filter isnotnull(A#34)
: +- InMemoryTableScan [A#34, B#35], [isnotnull(A#34)]
: +- InMemoryRelation [A#34, B#35], CachedRDDBuilder(true,10000,StorageLevel(disk, memory, deserialized, 1 replicas),*(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<A:string,B:string>
,None)
: +- *(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<A:string,B:string>
+- BroadcastExchange HashedRelationBroadcastMode(List(input[0, string, false]))
+- *(1) Filter isnotnull(A#86)
+- InMemoryTableScan [A#86, B#87], [isnotnull(A#86)]
+- InMemoryRelation [A#86, B#87], CachedRDDBuilder(true,10000,StorageLevel(disk, memory, deserialized, 1 replicas),*(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<A:string,B:string>
,None)
+- *(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<A:string,B:string>
```
with this patch results in:
```
== Physical Plan ==
InMemoryTableScan [A#10, B#11, B#35, B#87]
   +- InMemoryRelation [A#10, B#11, B#35, B#87], CachedRDDBuilder(true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas))
         +- *(2) Project [A#10, B#11, B#35, B#87]
            +- *(2) BroadcastHashJoin [A#10], [A#86], Inner, BuildRight
               :- *(2) Filter isnotnull(A#10)
               :  +- InMemoryTableScan [A#10, B#11, B#35], [isnotnull(A#10)]
               :        +- InMemoryRelation [A#10, B#11, B#35], CachedRDDBuilder(true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas))
               :              +- *(2) Project [A#10, B#11, B#35]
               :                 +- *(2) BroadcastHashJoin [A#10], [A#34], Inner, BuildRight
               :                    :- *(2) Filter isnotnull(A#10)
               :                    :  +- InMemoryTableScan [A#10, B#11], [isnotnull(A#10)]
               :                    :        +- InMemoryRelation [A#10, B#11], CachedRDDBuilder(true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas))
               :                    :              +- *(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<A:string,B:string>
               :                    +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, string, false]))
               :                       +- *(1) Filter isnotnull(A#34)
               :                          +- InMemoryTableScan [A#34, B#35], [isnotnull(A#34)]
               :                                +- InMemoryRelation [A#34, B#35], CachedRDDBuilder(true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas))
               :                                      +- *(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<A:string,B:string>
               +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, string, false]))
                  +- *(1) Filter isnotnull(A#86)
                     +- InMemoryTableScan [A#86, B#87], [isnotnull(A#86)]
                           +- InMemoryRelation [A#86, B#87], CachedRDDBuilder(true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas))
                                 +- *(1) FileScan csv [A#10,B#11] Batched: false, Format: CSV, Location: InMemoryFileIndex[file:test.csv], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<A:string,B:string>
```

Author: Onur Satici <osatici@palantir.com>

Closes #21805 from onursatici/os/inmemoryrelation-str.
2018-07-23 09:52:28 -07:00
Daniel van der Ende 2333a34d39 [SPARK-22880][SQL] Add cascadeTruncate option to JDBC datasource
This commit adds the `cascadeTruncate` option to the JDBC datasource
API, for databases that support this functionality (PostgreSQL and
Oracle at the moment). This allows for applying a cascading truncate
that affects tables that have foreign key constraints on the table
being truncated.

## What changes were proposed in this pull request?

Add `cascadeTruncate` option to JDBC datasource API. Allow this to affect the
`TRUNCATE` query for databases that support this option.

## How was this patch tested?
Existing tests for `truncateQuery` were updated. Also, an additional test was added
to ensure that the correct syntax was applied, and that enabling the config for databases
that do not support this option does not result in invalid queries.

Author: Daniel van der Ende <daniel.vanderende@gmail.com>

Closes #20057 from danielvdende/SPARK-22880.
2018-07-20 13:03:57 -07:00
hyukjinkwon e0b6383218 [SPARK-23731][SQL] Make FileSourceScanExec canonicalizable after being (de)serialized
## What changes were proposed in this pull request?

### What's problem?

In some cases, sub scalar query could throw a NPE, which is caused in execution side.

```
java.lang.NullPointerException
	at org.apache.spark.sql.execution.FileSourceScanExec.<init>(DataSourceScanExec.scala:169)
	at org.apache.spark.sql.execution.FileSourceScanExec.doCanonicalize(DataSourceScanExec.scala:526)
	at org.apache.spark.sql.execution.FileSourceScanExec.doCanonicalize(DataSourceScanExec.scala:159)
	at org.apache.spark.sql.catalyst.plans.QueryPlan.canonicalized$lzycompute(QueryPlan.scala:211)
	at org.apache.spark.sql.catalyst.plans.QueryPlan.canonicalized(QueryPlan.scala:210)
	at org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$3.apply(QueryPlan.scala:225)
	at org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$3.apply(QueryPlan.scala:225)
	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.immutable.List.foreach(List.scala:392)
	at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
	at scala.collection.immutable.List.map(List.scala:296)
	at org.apache.spark.sql.catalyst.plans.QueryPlan.doCanonicalize(QueryPlan.scala:225)
	at org.apache.spark.sql.catalyst.plans.QueryPlan.canonicalized$lzycompute(QueryPlan.scala:211)
	at org.apache.spark.sql.catalyst.plans.QueryPlan.canonicalized(QueryPlan.scala:210)
	at org.apache.spark.sql.catalyst.plans.QueryPlan.sameResult(QueryPlan.scala:258)
	at org.apache.spark.sql.execution.ScalarSubquery.semanticEquals(subquery.scala:58)
	at org.apache.spark.sql.catalyst.expressions.EquivalentExpressions$Expr.equals(EquivalentExpressions.scala:36)
	at scala.collection.mutable.HashTable$class.elemEquals(HashTable.scala:364)
	at scala.collection.mutable.HashMap.elemEquals(HashMap.scala:40)
	at scala.collection.mutable.HashTable$class.scala$collection$mutable$HashTable$$findEntry0(HashTable.scala:139)
	at scala.collection.mutable.HashTable$class.findEntry(HashTable.scala:135)
	at scala.collection.mutable.HashMap.findEntry(HashMap.scala:40)
	at scala.collection.mutable.HashMap.get(HashMap.scala:70)
	at org.apache.spark.sql.catalyst.expressions.EquivalentExpressions.addExpr(EquivalentExpressions.scala:56)
	at org.apache.spark.sql.catalyst.expressions.EquivalentExpressions.addExprTree(EquivalentExpressions.scala:97)
	at org.apache.spark.sql.catalyst.expressions.EquivalentExpressions$$anonfun$addExprTree$1.apply(EquivalentExpressions.scala:98)
	at org.apache.spark.sql.catalyst.expressions.EquivalentExpressions$$anonfun$addExprTree$1.apply(EquivalentExpressions.scala:98)
	at scala.collection.immutable.List.foreach(List.scala:392)
	at org.apache.spark.sql.catalyst.expressions.EquivalentExpressions.addExprTree(EquivalentExpressions.scala:98)
	at org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext$$anonfun$subexpressionElimination$1.apply(CodeGenerator.scala:1102)
	at org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext$$anonfun$subexpressionElimination$1.apply(CodeGenerator.scala:1102)
	at scala.collection.immutable.List.foreach(List.scala:392)
	at org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext.subexpressionElimination(CodeGenerator.scala:1102)
	at org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext.generateExpressions(CodeGenerator.scala:1154)
	at org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.createCode(GenerateUnsafeProjection.scala:270)
	at org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.create(GenerateUnsafeProjection.scala:319)
	at org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.generate(GenerateUnsafeProjection.scala:308)
	at org.apache.spark.sql.catalyst.expressions.UnsafeProjection$.create(Projection.scala:181)
	at org.apache.spark.sql.execution.ProjectExec$$anonfun$9.apply(basicPhysicalOperators.scala:71)
	at org.apache.spark.sql.execution.ProjectExec$$anonfun$9.apply(basicPhysicalOperators.scala:70)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsWithIndexInternal$1$$anonfun$apply$24.apply(RDD.scala:818)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsWithIndexInternal$1$$anonfun$apply$24.apply(RDD.scala:818)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
	at org.apache.spark.scheduler.Task.run(Task.scala:109)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:367)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
```

### How does this happen?

Here looks what happen now:

1. Sub scalar query was made (for instance `SELECT (SELECT id FROM foo)`).

2. Try to extract some common expressions (via `CodeGenerator.subexpressionElimination`) so that it can generates some common codes and can be reused.

3. During this, seems it extracts some expressions that can be reused (via `EquivalentExpressions.addExprTree`)

  b2deef64f6/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala (L1102)

4. During this, if the hash (`EquivalentExpressions.Expr.hashCode`) happened to be the same at `EquivalentExpressions.addExpr` anyhow, `EquivalentExpressions.Expr.equals` is called to identify object in the same hash, which eventually calls `semanticEquals` in `ScalarSubquery`

  087879a77a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala (L54)

  087879a77a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala (L36)

5. `ScalarSubquery`'s `semanticEquals` needs `SubqueryExec`'s `sameResult`

  77a2fc5b52/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala (L58)

6. `SubqueryExec`'s `sameResult` requires a canonicalized plan which calls `FileSourceScanExec`'s `doCanonicalize`

  e008ad1752/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala (L258)

7. In `FileSourceScanExec`'s `doCanonicalize`, `FileSourceScanExec`'s `relation` is required but seems `transient` so it becomes `null`.

  e76b0124fb/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala (L527)

  e76b0124fb/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala (L160)

8. NPE is thrown.

\*1. driver side
\*2., 3., 4., 5., 6., 7., 8. executor side

Note that most of cases, it looks fine because we will usually call:

087879a77a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala (L40)

which make a canonicalized plan via:

b045315e5d/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala (L192)

77a2fc5b52/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala (L52)

### How to reproduce?

This looks what happened now. I can reproduce this by a bit of messy way:

```diff
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala
index 8d06804ce1e..d25fc9a7ba9 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala
 -37,7 +37,9  class EquivalentExpressions {
       case _ => false
     }

-    override def hashCode: Int = e.semanticHash()
+    override def hashCode: Int = {
+      1
+    }
   }
```

```scala
spark.range(1).write.mode("overwrite").parquet("/tmp/foo")
spark.read.parquet("/tmp/foo").createOrReplaceTempView("foo")
spark.conf.set("spark.sql.codegen.wholeStage", false)
sql("SELECT (SELECT id FROM foo) == (SELECT id FROM foo)").collect()
```

### How does this PR fix?

- Make all variables that access to `FileSourceScanExec`'s `relation` as `lazy val` so that we avoid NPE. This is a temporary fix.

- Allow `makeCopy` in `SparkPlan` without Spark session too. This looks still able to be accessed within executor side. For instance:

  ```
	at org.apache.spark.sql.execution.SparkPlan.makeCopy(SparkPlan.scala:70)
	at org.apache.spark.sql.execution.SparkPlan.makeCopy(SparkPlan.scala:47)
	at org.apache.spark.sql.catalyst.trees.TreeNode.withNewChildren(TreeNode.scala:233)
	at org.apache.spark.sql.catalyst.plans.QueryPlan.doCanonicalize(QueryPlan.scala:243)
	at org.apache.spark.sql.catalyst.plans.QueryPlan.canonicalized$lzycompute(QueryPlan.scala:211)
	at org.apache.spark.sql.catalyst.plans.QueryPlan.canonicalized(QueryPlan.scala:210)
	at org.apache.spark.sql.catalyst.plans.QueryPlan.sameResult(QueryPlan.scala:258)
	at org.apache.spark.sql.execution.ScalarSubquery.semanticEquals(subquery.scala:58)
	at org.apache.spark.sql.catalyst.expressions.EquivalentExpressions$Expr.equals(EquivalentExpressions.scala:36)
	at scala.collection.mutable.HashTable$class.elemEquals(HashTable.scala:364)
	at scala.collection.mutable.HashMap.elemEquals(HashMap.scala:40)
	at scala.collection.mutable.HashTable$class.scala$collection$mutable$HashTable$$findEntry0(HashTable.scala:139)
	at scala.collection.mutable.HashTable$class.findEntry(HashTable.scala:135)
	at scala.collection.mutable.HashMap.findEntry(HashMap.scala:40)
	at scala.collection.mutable.HashMap.get(HashMap.scala:70)
	at org.apache.spark.sql.catalyst.expressions.EquivalentExpressions.addExpr(EquivalentExpressions.scala:54)
	at org.apache.spark.sql.catalyst.expressions.EquivalentExpressions.addExprTree(EquivalentExpressions.scala:95)
	at org.apache.spark.sql.catalyst.expressions.EquivalentExpressions$$anonfun$addExprTree$1.apply(EquivalentExpressions.scala:96)
	at org.apache.spark.sql.catalyst.expressions.EquivalentExpressions$$anonfun$addExprTree$1.apply(EquivalentExpressions.scala:96)
	at scala.collection.immutable.List.foreach(List.scala:392)
	at org.apache.spark.sql.catalyst.expressions.EquivalentExpressions.addExprTree(EquivalentExpressions.scala:96)
	at org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext$$anonfun$subexpressionElimination$1.apply(CodeGenerator.scala:1102)
	at org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext$$anonfun$subexpressionElimination$1.apply(CodeGenerator.scala:1102)
	at scala.collection.immutable.List.foreach(List.scala:392)
	at org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext.subexpressionElimination(CodeGenerator.scala:1102)
	at org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext.generateExpressions(CodeGenerator.scala:1154)
	at org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.createCode(GenerateUnsafeProjection.scala:270)
	at org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.create(GenerateUnsafeProjection.scala:319)
	at org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection$.generate(GenerateUnsafeProjection.scala:308)
	at org.apache.spark.sql.catalyst.expressions.UnsafeProjection$.create(Projection.scala:181)
	at org.apache.spark.sql.execution.ProjectExec$$anonfun$9.apply(basicPhysicalOperators.scala:71)
	at org.apache.spark.sql.execution.ProjectExec$$anonfun$9.apply(basicPhysicalOperators.scala:70)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsWithIndexInternal$1$$anonfun$apply$24.apply(RDD.scala:818)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsWithIndexInternal$1$$anonfun$apply$24.apply(RDD.scala:818)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:324)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:288)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
	at org.apache.spark.scheduler.Task.run(Task.scala:109)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:367)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
  ```

This PR takes over https://github.com/apache/spark/pull/20856.

## How was this patch tested?

Manually tested and unit test was added.

Closes #20856

Author: hyukjinkwon <gurwls223@apache.org>

Closes #21815 from HyukjinKwon/SPARK-23731.
2018-07-20 20:59:48 +08:00
Dilip Biswal 2b91d9918c [SPARK-24424][SQL] Support ANSI-SQL compliant syntax for GROUPING SET
## What changes were proposed in this pull request?

Enhances the parser and analyzer to support ANSI compliant syntax for GROUPING SET. As part of this change we derive the grouping expressions from user supplied groupings in the grouping sets clause.

```SQL
SELECT c1, c2, max(c3)
FROM t1
GROUP BY GROUPING SETS ((c1), (c1, c2))
```

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

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

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

Closes #21813 from dilipbiswal/spark-24424.
2018-07-19 23:52:53 -07:00
Marco Gaido a5925c1631 [SPARK-24268][SQL] Use datatype.catalogString in error messages
## What changes were proposed in this pull request?

As stated in https://github.com/apache/spark/pull/21321, in the error messages we should use `catalogString`. This is not the case, as SPARK-22893 used `simpleString` in order to have the same representation everywhere and it missed some places.

The PR unifies the messages using alway the `catalogString` representation of the dataTypes in the messages.

## How was this patch tested?

existing/modified UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21804 from mgaido91/SPARK-24268_catalog.
2018-07-19 23:29:29 -07:00
Wenchen Fan 1462b17666 [SPARK-24861][SS][TEST] create corrected temp directories in RateSourceSuite
## What changes were proposed in this pull request?

`RateSourceSuite` may leave garbage files under `sql/core/dummy`, we should use a corrected temp directory

## How was this patch tested?

test only

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21817 from cloud-fan/minor.
2018-07-20 13:40:26 +08:00
Tathagata Das b3d88ac029 [SPARK-22187][SS] Update unsaferow format for saved state in flatMapGroupsWithState to allow timeouts with deleted state
## What changes were proposed in this pull request?

Currently, the group state of user-defined-type is encoded as top-level columns in the UnsafeRows stores in the state store. The timeout timestamp is also saved as (when needed) as the last top-level column. Since the group state is serialized to top-level columns, you cannot save "null" as a value of state (setting null in all the top-level columns is not equivalent). So we don't let the user set the timeout without initializing the state for a key. Based on user experience, this leads to confusion.

This PR is to change the row format such that the state is saved as nested columns. This would allow the state to be set to null, and avoid these confusing corner cases. However, queries recovering from existing checkpoint will use the previous format to maintain compatibility with existing production queries.

## How was this patch tested?
Refactored existing end-to-end tests and added new tests for explicitly testing obj-to-row conversion for both state formats.

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

Closes #21739 from tdas/SPARK-22187-1.
2018-07-19 13:17:28 -07:00
Gengliang Wang 6a9a058e09 [SPARK-24858][SQL] Avoid unnecessary parquet footer reads
## What changes were proposed in this pull request?

Currently the same Parquet footer is read twice in the function `buildReaderWithPartitionValues` of ParquetFileFormat if filter push down is enabled.

Fix it with simple changes.
## How was this patch tested?

Unit test

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

Closes #21814 from gengliangwang/parquetFooter.
2018-07-19 22:24:53 +08:00
Jungtaek Lim 8b7d4f842f [SPARK-24717][SS] Split out max retain version of state for memory in HDFSBackedStateStoreProvider
## What changes were proposed in this pull request?

This patch proposes breaking down configuration of retaining batch size on state into two pieces: files and in memory (cache). While this patch reuses existing configuration for files, it introduces new configuration, "spark.sql.streaming.maxBatchesToRetainInMemory" to configure max count of batch to retain in memory.

## How was this patch tested?

Apply this patch on top of SPARK-24441 (https://github.com/apache/spark/pull/21469), and manually tested in various workloads to ensure overall size of states in memory is around 2x or less of the size of latest version of state, while it was 10x ~ 80x before applying the patch.

Author: Jungtaek Lim <kabhwan@gmail.com>

Closes #21700 from HeartSaVioR/SPARK-24717.
2018-07-19 00:07:35 -07:00
Wenchen Fan d05a926e78 [SPARK-24840][SQL] do not use dummy filter to switch codegen on/of
## What changes were proposed in this pull request?

It's a little tricky and fragile to use a dummy filter to switch codegen on/off. For now we should use local/cached relation to switch. In the future when we are able to use a config to turn off codegen, we shall use that.

## How was this patch tested?

test only PR.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21795 from cloud-fan/follow.
2018-07-19 11:54:41 +08:00
maryannxue cd203e0dfc [SPARK-24163][SPARK-24164][SQL] Support column list as the pivot column in Pivot
## What changes were proposed in this pull request?

1. Extend the Parser to enable parsing a column list as the pivot column.
2. Extend the Parser and the Pivot node to enable parsing complex expressions with aliases as the pivot value.
3. Add type check and constant check in Analyzer for Pivot node.

## How was this patch tested?

Add tests in pivot.sql

Author: maryannxue <maryannxue@apache.org>

Closes #21720 from maryannxue/spark-24164.
2018-07-18 13:33:26 -07:00
韩田田00222924 002300dd41 [SPARK-24804] There are duplicate words in the test title in the DatasetSuite
## What changes were proposed in this pull request?
In DatasetSuite.scala, in the 1299 line,
test("SPARK-19896: cannot have circular references in in case class") ,
there are  duplicate words "in in".  We can get rid of one.

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

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

Author: 韩田田00222924 <han.tiantian@zte.com.cn>

Closes #21767 from httfighter/inin.
2018-07-18 09:40:36 -05:00
Takuya UESHIN 34cb3b54e9 [SPARK-24386][SPARK-24768][BUILD][FOLLOWUP] Fix lint-java and Scala 2.12 build.
## What changes were proposed in this pull request?

This pr fixes lint-java and Scala 2.12 build.

lint-java:

```
[ERROR] src/test/resources/log4j.properties:[0] (misc) NewlineAtEndOfFile: File does not end with a newline.
```

Scala 2.12 build:

```
[error] /.../sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousCoalesceRDD.scala:121: overloaded method value addTaskCompletionListener with alternatives:
[error]   (f: org.apache.spark.TaskContext => Unit)org.apache.spark.TaskContext <and>
[error]   (listener: org.apache.spark.util.TaskCompletionListener)org.apache.spark.TaskContext
[error]  cannot be applied to (org.apache.spark.TaskContext => java.util.List[Runnable])
[error]       context.addTaskCompletionListener { ctx =>
[error]               ^
```

## How was this patch tested?

Manually executed lint-java and Scala 2.12 build in my local environment.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #21801 from ueshin/issues/SPARK-24386_24768/fix_build.
2018-07-18 19:17:18 +08:00
Dongjoon Hyun 3b59d326c7 [SPARK-24576][BUILD] Upgrade Apache ORC to 1.5.2
## What changes were proposed in this pull request?

This issue aims to upgrade Apache ORC library from 1.4.4 to 1.5.2 in order to bring the following benefits into Apache Spark.

- [ORC-91](https://issues.apache.org/jira/browse/ORC-91) Support for variable length blocks in HDFS (The current space wasted in ORC to padding is known to be 5%.)
- [ORC-344](https://issues.apache.org/jira/browse/ORC-344) Support for using Decimal64ColumnVector

In addition to that, Apache Hive 3.1 and 3.2 will use ORC 1.5.1 ([HIVE-19669](https://issues.apache.org/jira/browse/HIVE-19465)) and 1.5.2 ([HIVE-19792](https://issues.apache.org/jira/browse/HIVE-19792)) respectively. This will improve the compatibility between Apache Spark and Apache Hive by sharing the common library.

## How was this patch tested?

Pass the Jenkins with all existing tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #21582 from dongjoon-hyun/SPARK-24576.
2018-07-17 23:52:17 -07:00
HanShuliang 7688ce88b2 [SPARK-21590][SS] Window start time should support negative values
## What changes were proposed in this pull request?

Remove the non-negative checks of window start time to make window support negative start time, and add a check to guarantee the absolute value of start time is less than slide duration.

## How was this patch tested?

New unit tests.

Author: HanShuliang <kevinzwx1992@gmail.com>

Closes #18903 from KevinZwx/dev.
2018-07-17 11:25:23 -05:00
Marek Novotny b0c95a1d69 [SPARK-23901][SQL] Removing masking functions
The PR reverts #21246.

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

Closes #21786 from mn-mikke/SPARK-23901.
2018-07-16 14:28:35 -07:00
Takuya UESHIN b045315e5d [SPARK-24734][SQL] Fix type coercions and nullabilities of nested data types of some functions.
## What changes were proposed in this pull request?

We have some functions which need to aware the nullabilities of all children, such as `CreateArray`, `CreateMap`, `Concat`, and so on. Currently we add casts to fix the nullabilities, but the casts might be removed during the optimization phase.
After the discussion, we decided to not add extra casts for just fixing the nullabilities of the nested types, but handle them by functions themselves.

## How was this patch tested?

Modified and added some tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #21704 from ueshin/issues/SPARK-24734/concat_containsnull.
2018-07-16 23:16:25 +08:00
Yuming Wang 9549a28149 [SPARK-24549][SQL] Support Decimal type push down to the parquet data sources
## What changes were proposed in this pull request?

Support Decimal type push down to the parquet data sources.
The Decimal comparator used is: [`BINARY_AS_SIGNED_INTEGER_COMPARATOR`](c6764c4a08/parquet-column/src/main/java/org/apache/parquet/schema/PrimitiveComparator.java (L224-L292)).

## How was this patch tested?

unit tests and manual tests.

**manual tests**:
```scala
spark.range(10000000).selectExpr("id", "cast(id as decimal(9)) as d1", "cast(id as decimal(9, 2)) as d2", "cast(id as decimal(18)) as d3", "cast(id as decimal(18, 4)) as d4", "cast(id as decimal(38)) as d5", "cast(id as decimal(38, 18)) as d6").coalesce(1).write.option("parquet.block.size", 1048576).parquet("/tmp/spark/parquet/decimal")
val df = spark.read.parquet("/tmp/spark/parquet/decimal/")
spark.sql("set spark.sql.parquet.filterPushdown.decimal=true")
// Only read about 1 MB data
df.filter("d2 = 10000").show
// Only read about 1 MB data
df.filter("d4 = 10000").show
spark.sql("set spark.sql.parquet.filterPushdown.decimal=false")
// Read 174.3 MB data
df.filter("d2 = 10000").show
// Read 174.3 MB data
df.filter("d4 = 10000").show
```

Author: Yuming Wang <yumwang@ebay.com>

Closes #21556 from wangyum/SPARK-24549.
2018-07-16 15:44:51 +08:00
Maxim Gekk 9f929458fb [SPARK-24810][SQL] Fix paths to test files in AvroSuite
## What changes were proposed in this pull request?

In the PR, I propose to move `testFile()` to the common trait `SQLTestUtilsBase` and wrap test files in `AvroSuite` by the method `testFile()` which returns full paths to test files in the resource folder.

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

Closes #21773 from MaxGekk/test-file.
2018-07-15 23:01:36 -07:00
Takeshi Yamamuro d463533ded [SPARK-24676][SQL] Project required data from CSV parsed data when column pruning disabled
## What changes were proposed in this pull request?
This pr modified code to project required data from CSV parsed data when column pruning disabled.
In the current master, an exception below happens if `spark.sql.csv.parser.columnPruning.enabled` is false. This is because required formats and CSV parsed formats are different from each other;
```
./bin/spark-shell --conf spark.sql.csv.parser.columnPruning.enabled=false
scala> val dir = "/tmp/spark-csv/csv"
scala> spark.range(10).selectExpr("id % 2 AS p", "id").write.mode("overwrite").partitionBy("p").csv(dir)
scala> spark.read.csv(dir).selectExpr("sum(p)").collect()
18/06/25 13:48:46 ERROR Executor: Exception in task 2.0 in stage 2.0 (TID 7)
java.lang.ClassCastException: org.apache.spark.unsafe.types.UTF8String cannot be cast to java.lang.Integer
        at scala.runtime.BoxesRunTime.unboxToInt(BoxesRunTime.java:101)
        at org.apache.spark.sql.catalyst.expressions.BaseGenericInternalRow$class.getInt(rows.scala:41)
        ...
```

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21657 from maropu/SPARK-24676.
2018-07-15 20:22:09 -07:00
Yuming Wang 43e4e851b6 [SPARK-24718][SQL] Timestamp support pushdown to parquet data source
## What changes were proposed in this pull request?

`Timestamp` support pushdown to parquet data source.
Only `TIMESTAMP_MICROS` and `TIMESTAMP_MILLIS` support push down.

## How was this patch tested?

unit tests and benchmark tests

Author: Yuming Wang <yumwang@ebay.com>

Closes #21741 from wangyum/SPARK-24718.
2018-07-15 11:13:49 +08:00
Yuming Wang e1de34113e [SPARK-17091][SQL] Add rule to convert IN predicate to equivalent Parquet filter
## What changes were proposed in this pull request?

The original pr is: https://github.com/apache/spark/pull/18424

Add a new optimizer rule to convert an IN predicate to an equivalent Parquet filter and add `spark.sql.parquet.pushdown.inFilterThreshold` to control limit thresholds. Different data types have different limit thresholds, this is a copy of data for reference:

Type | limit threshold
-- | --
string | 370
int | 210
long | 285
double | 270
float | 220
decimal | Won't provide better performance before [SPARK-24549](https://issues.apache.org/jira/browse/SPARK-24549)

## How was this patch tested?
unit tests and manual tests

Author: Yuming Wang <yumwang@ebay.com>

Closes #21603 from wangyum/SPARK-17091.
2018-07-14 17:50:54 +08:00
Liang-Chi Hsieh dfd7ac9887 [SPARK-24781][SQL] Using a reference from Dataset in Filter/Sort might not work
## What changes were proposed in this pull request?

When we use a reference from Dataset in filter or sort, which was not used in the prior select, an AnalysisException occurs, e.g.,

```scala
val df = Seq(("test1", 0), ("test2", 1)).toDF("name", "id")
df.select(df("name")).filter(df("id") === 0).show()
```

```scala
org.apache.spark.sql.AnalysisException: Resolved attribute(s) id#6 missing from name#5 in operator !Filter (id#6 = 0).;;
!Filter (id#6 = 0)
   +- AnalysisBarrier
      +- Project [name#5]
         +- Project [_1#2 AS name#5, _2#3 AS id#6]
            +- LocalRelation [_1#2, _2#3]
```
This change updates the rule `ResolveMissingReferences` so `Filter` and `Sort` with non-empty `missingInputs` will also be transformed.

## How was this patch tested?

Added tests.

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

Closes #21745 from viirya/SPARK-24781.
2018-07-13 08:25:00 -07:00
maryannxue 75725057b3 [SPARK-24790][SQL] Allow complex aggregate expressions in Pivot
## What changes were proposed in this pull request?

Relax the check to allow complex aggregate expressions, like `ceil(sum(col1))` or `sum(col1) + 1`, which roughly means any aggregate expression that could appear in an Aggregate plan except pandas UDF (due to the fact that it is not supported in pivot yet).

## How was this patch tested?

Added 2 tests in pivot.sql

Author: maryannxue <maryannxue@apache.org>

Closes #21753 from maryannxue/pivot-relax-syntax.
2018-07-12 16:54:03 -07:00
Marco Gaido 11384893b6 [SPARK-24208][SQL][FOLLOWUP] Move test cases to proper locations
## What changes were proposed in this pull request?

The PR is a followup to move the test cases introduced by the original PR in their proper location.

## How was this patch tested?

moved UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21751 from mgaido91/SPARK-24208_followup.
2018-07-12 15:13:26 -07:00
Dongjoon Hyun 07704c971c [SPARK-23007][SQL][TEST] Add read schema suite for file-based data sources
## What changes were proposed in this pull request?

The reader schema is said to be evolved (or projected) when it changed after the data is written. The followings are already supported in file-based data sources. Note that partition columns are not maintained in files. In this PR, `column` means `non-partition column`.

   1. Add a column
   2. Hide a column
   3. Change a column position
   4. Change a column type (upcast)

This issue aims to guarantee users a backward-compatible read-schema test coverage on file-based data sources and to prevent future regressions by *adding read schema tests explicitly*.

Here, we consider safe changes without data loss. For example, data type change should be from small types to larger types like `int`-to-`long`, not vice versa.

As of today, in the master branch, file-based data sources have the following coverage.

File Format | Coverage  | Note
----------- | ---------- | ------------------------------------------------
TEXT          | N/A            | Schema consists of a single string column.
CSV            | 1, 2, 4        |
JSON          | 1, 2, 3, 4    |
ORC            | 1, 2, 3, 4    | Native vectorized ORC reader has the widest coverage among ORC formats.
PARQUET   | 1, 2, 3        |

## How was this patch tested?

Pass the Jenkins with newly added test suites.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20208 from dongjoon-hyun/SPARK-SCHEMA-EVOLUTION.
2018-07-12 14:08:49 -07:00
Gengliang Wang e6c6f90a55 [SPARK-24691][SQL] Dispatch the type support check in FileFormat implementation
## What changes were proposed in this pull request?

With https://github.com/apache/spark/pull/21389,  data source schema is validated on driver side before launching read/write tasks.
However,

1. Putting all the validations together in `DataSourceUtils` is tricky and hard to maintain. On second thought after review, I find that the `OrcFileFormat` in hive package is not matched, so that its validation wrong.
2.  `DataSourceUtils.verifyWriteSchema` and `DataSourceUtils.verifyReadSchema` is not supposed to be called in every file format. We can move them to some upper entry.

So, I propose we can add a new method `validateDataType` in FileFormat. File format implementation can override the method to specify its supported/non-supported data types.
Although we should focus on data source V2 API, `FileFormat` should remain workable for some time. Adding this new method should be helpful.

## How was this patch tested?

Unit test

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

Closes #21667 from gengliangwang/refactorSchemaValidate.
2018-07-13 00:26:49 +08:00
Kazuaki Ishizaki 301bff7063 [SPARK-23914][SQL] Add array_union function
## What changes were proposed in this pull request?

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

This function returns returns an array of the elements in the union of array1 and array2.

Note: The order of elements in the result is not defined.

## How was this patch tested?

Added UTs

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

Closes #21061 from kiszk/SPARK-23914.
2018-07-12 17:42:29 +09:00
Maxim Gekk 3ab48f985c [SPARK-24761][SQL] Adding of isModifiable() to RuntimeConfig
## What changes were proposed in this pull request?

In the PR, I propose to extend `RuntimeConfig` by new method `isModifiable()` which returns `true` if a config parameter can be modified at runtime (for current session state). For static SQL and core parameters, the method returns `false`.

## How was this patch tested?

Added new test to `RuntimeConfigSuite` for checking Spark core and SQL parameters.

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

Closes #21730 from MaxGekk/is-modifiable.
2018-07-11 17:38:43 -07:00
Marco Gaido e008ad1752 [SPARK-24782][SQL] Simplify conf retrieval in SQL expressions
## What changes were proposed in this pull request?

The PR simplifies the retrieval of config in `size`, as we can access them from tasks too thanks to SPARK-24250.

## How was this patch tested?

existing UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21736 from mgaido91/SPARK-24605_followup.
2018-07-11 17:30:43 -07:00
Tathagata Das ff7f6ef75c [SPARK-24697][SS] Fix the reported start offsets in streaming query progress
## What changes were proposed in this pull request?

In ProgressReporter for streams, we use the `committedOffsets` as the startOffset and `availableOffsets` as the end offset when reporting the status of a trigger in `finishTrigger`. This is a bad pattern that has existed since the beginning of ProgressReporter and it is bad because its super hard to reason about when `availableOffsets` and `committedOffsets` are updated, and when they are recorded. Case in point, this bug silently existed in ContinuousExecution, since before MicroBatchExecution was refactored.

The correct fix it to record the offsets explicitly. This PR adds a simple method which is explicitly called from MicroBatch/ContinuousExecition before updating the `committedOffsets`.

## How was this patch tested?
Added new tests

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

Closes #21744 from tdas/SPARK-24697.
2018-07-11 12:44:42 -07:00
Marco Gaido ebf4bfb966 [SPARK-24208][SQL] Fix attribute deduplication for FlatMapGroupsInPandas
## What changes were proposed in this pull request?

A self-join on a dataset which contains a `FlatMapGroupsInPandas` fails because of duplicate attributes. This happens because we are not dealing with this specific case in our `dedupAttr` rules.

The PR fix the issue by adding the management of the specific case

## How was this patch tested?

added UT + manual tests

Author: Marco Gaido <marcogaido91@gmail.com>
Author: Marco Gaido <mgaido@hortonworks.com>

Closes #21737 from mgaido91/SPARK-24208.
2018-07-11 09:29:19 -07:00
Marco Gaido 592cc84583 [SPARK-24562][TESTS] Support different configs for same test in SQLQueryTestSuite
## What changes were proposed in this pull request?

The PR proposes to add support for running the same SQL test input files against different configs leading to the same result.

## How was this patch tested?

Involved UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21568 from mgaido91/SPARK-24562.
2018-07-11 23:43:06 +08:00
Marek Novotny 74a8d6308b [SPARK-24165][SQL] Fixing conditional expressions to handle nullability of nested types
## What changes were proposed in this pull request?
This PR is proposing a fix for the output data type of ```If``` and ```CaseWhen``` expression. Upon till now, the implementation of exprassions has ignored nullability of nested types from different execution branches and returned the type of the first branch.

This could lead to an unwanted ```NullPointerException``` from other expressions depending on a ```If```/```CaseWhen``` expression.

Example:
```
val rows = new util.ArrayList[Row]()
rows.add(Row(true, ("a", 1)))
rows.add(Row(false, (null, 2)))
val schema = StructType(Seq(
  StructField("cond", BooleanType, false),
  StructField("s", StructType(Seq(
    StructField("val1", StringType, true),
    StructField("val2", IntegerType, false)
  )), false)
))

val df = spark.createDataFrame(rows, schema)

df
  .select(when('cond, struct(lit("x").as("val1"), lit(10).as("val2"))).otherwise('s) as "res")
  .select('res.getField("val1"))
  .show()
```
Exception:
```
Exception in thread "main" java.lang.NullPointerException
	at org.apache.spark.sql.catalyst.expressions.codegen.UnsafeWriter.write(UnsafeWriter.java:109)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(Unknown Source)
	at org.apache.spark.sql.execution.LocalTableScanExec$$anonfun$unsafeRows$1.apply(LocalTableScanExec.scala:44)
	at org.apache.spark.sql.execution.LocalTableScanExec$$anonfun$unsafeRows$1.apply(LocalTableScanExec.scala:44)
...
```
Output schema:
```
root
 |-- res.val1: string (nullable = false)
```

## How was this patch tested?
New test cases added into
- DataFrameSuite.scala
- conditionalExpressions.scala

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

Closes #21687 from mn-mikke/SPARK-24165.
2018-07-11 12:21:03 +08:00
Tathagata Das 6078b891da [SPARK-24730][SS] Add policy to choose max as global watermark when streaming query has multiple watermarks
## What changes were proposed in this pull request?

Currently, when a streaming query has multiple watermark, the policy is to choose the min of them as the global watermark. This is safe to do as the global watermark moves with the slowest stream, and is therefore is safe as it does not unexpectedly drop some data as late, etc. While this is indeed the safe thing to do, in some cases, you may want the watermark to advance with the fastest stream, that is, take the max of multiple watermarks. This PR is to add that configuration. It makes the following changes.

- Adds a configuration to specify max as the policy.
- Saves the configuration in OffsetSeqMetadata because changing it in the middle can lead to unpredictable results.
   - For old checkpoints without the configuration, it assumes the default policy as min (irrespective of the policy set at the session where the query is being restarted). This is to ensure that existing queries are affected in any way.

TODO
- [ ] Add a test for recovery from existing checkpoints.

## How was this patch tested?
New unit test

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

Closes #21701 from tdas/SPARK-24730.
2018-07-10 18:03:40 -07:00
Mukul Murthy 32cb50835e [SPARK-24662][SQL][SS] Support limit in structured streaming
## What changes were proposed in this pull request?

Support the LIMIT operator in structured streaming.

For streams in append or complete output mode, a stream with a LIMIT operator will return no more than the specified number of rows. LIMIT is still unsupported for the update output mode.

This change reverts e4fee395ec as part of it because it is a better and more complete implementation.

## How was this patch tested?

New and existing unit tests.

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

Closes #21662 from mukulmurthy/SPARK-24662.
2018-07-10 11:08:04 -07:00
Wenchen Fan e0559f2380 [SPARK-21743][SQL][FOLLOWUP] free aggregate map when task ends
## What changes were proposed in this pull request?

This is the first follow-up of https://github.com/apache/spark/pull/21573 , which was only merged to 2.3.

This PR fixes the memory leak in another way: free the `UnsafeExternalMap` when the task ends. All the data buffers in Spark SQL are using `UnsafeExternalMap` and `UnsafeExternalSorter` under the hood, e.g. sort, aggregate, window, SMJ, etc. `UnsafeExternalSorter` registers a task completion listener to free the resource, we should apply the same thing to `UnsafeExternalMap`.

TODO in the next PR:
do not consume all the inputs when having limit in whole stage codegen.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21738 from cloud-fan/limit.
2018-07-10 23:07:10 +08:00
Yuming Wang a289009567 [SPARK-24706][SQL] ByteType and ShortType support pushdown to parquet
## What changes were proposed in this pull request?

`ByteType` and `ShortType` support pushdown to parquet data source.
[Benchmark result](https://issues.apache.org/jira/browse/SPARK-24706?focusedCommentId=16528878&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16528878).

## How was this patch tested?

unit tests

Author: Yuming Wang <yumwang@ebay.com>

Closes #21682 from wangyum/SPARK-24706.
2018-07-10 15:58:14 +08:00
Xiao Li eb6e988039 [SPARK-24759][SQL] No reordering keys for broadcast hash join
## What changes were proposed in this pull request?

As the implementation of the broadcast hash join is independent of the input hash partitioning, reordering keys is not necessary. Thus, we solve this issue by simply removing the broadcast hash join from the reordering rule in EnsureRequirements.

## How was this patch tested?
N/A

Author: Xiao Li <gatorsmile@gmail.com>

Closes #21728 from gatorsmile/cleanER.
2018-07-09 14:53:14 -07:00
Xiao Li aec966b05e Revert "[SPARK-24268][SQL] Use datatype.simpleString in error messages"
This reverts commit 1bd3d61f41.
2018-07-09 14:24:23 -07:00
Marco Gaido 1bd3d61f41 [SPARK-24268][SQL] Use datatype.simpleString in error messages
## What changes were proposed in this pull request?

SPARK-22893 tried to unify error messages about dataTypes. Unfortunately, still many places were missing the `simpleString` method in other to have the same representation everywhere.

The PR unified the messages using alway the simpleString representation of the dataTypes in the messages.

## How was this patch tested?

existing/modified UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21321 from mgaido91/SPARK-24268.
2018-07-09 22:59:05 +08:00
Bruce Robbins 034913b62b [SPARK-23936][SQL] Implement map_concat
## What changes were proposed in this pull request?

Implement map_concat high order function.

This implementation does not pick a winner when the specified maps have overlapping keys. Therefore, this implementation preserves existing duplicate keys in the maps and potentially introduces new duplicates (After discussion with ueshin, we settled on option 1 from [here](https://issues.apache.org/jira/browse/SPARK-23936?focusedCommentId=16464245&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16464245)).

## How was this patch tested?

New tests
Manual tests
Run all sbt SQL tests
Run all pyspark sql tests

Author: Bruce Robbins <bersprockets@gmail.com>

Closes #21073 from bersprockets/SPARK-23936.
2018-07-09 21:21:38 +09:00
Maxim Gekk 79c6689429 [SPARK-24757][SQL] Improving the error message for broadcast timeouts
## What changes were proposed in this pull request?

In the PR, I propose to provide a tip to user how to resolve the issue of timeout expiration for broadcast joins. In particular, they can increase the timeout via **spark.sql.broadcastTimeout** or disable the broadcast at all by setting **spark.sql.autoBroadcastJoinThreshold** to `-1`.

## How was this patch tested?

It tested manually from `spark-shell`:
```
scala> spark.conf.set("spark.sql.broadcastTimeout", 1)
scala> val df = spark.range(100).join(spark.range(15).as[Long].map { x =>
               Thread.sleep(5000)
               x
            }).where("id = value")
scala> df.count()
```
```
org.apache.spark.SparkException: Could not execute broadcast in 1 secs. You can increase the timeout for broadcasts via spark.sql.broadcastTimeout or disable broadcast join by setting spark.sql.autoBroadcastJoinThreshold to -1
  at org.apache.spark.sql.execution.exchange.BroadcastExchangeExec.doExecuteBroadcast(BroadcastExchangeExec.scala:150)
```

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

Closes #21727 from MaxGekk/broadcast-timeout-error.
2018-07-07 22:16:48 +02:00
Liang-Chi Hsieh 4de0425df8 [SPARK-24569][SQL] Aggregator with output type Option should produce consistent schema
## What changes were proposed in this pull request?

SQL `Aggregator` with output type `Option[Boolean]` creates column of type `StructType`. It's not in consistency with a Dataset of similar java class.

This changes the way `definedByConstructorParams` checks given type. For `Option[_]`, it goes to check its type argument.

## How was this patch tested?

Added test.

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

Closes #21611 from viirya/SPARK-24569.
2018-07-07 10:54:14 +08:00
Yuming Wang bf67f70c48 [SPARK-24692][TESTS] Improvement FilterPushdownBenchmark
## What changes were proposed in this pull request?
Refer to the [`WideSchemaBenchmark`](https://github.com/apache/spark/blob/v2.3.1/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala) update `FilterPushdownBenchmark`:
1. Write the result to `benchmarks/FilterPushdownBenchmark-results.txt` for easy maintenance.
2. Add more benchmark case: `StringStartsWith`, `Decimal`, `InSet -> InFilters` and `tinyint`.

## How was this patch tested?

manual tests

Author: Yuming Wang <yumwang@ebay.com>

Closes #21677 from wangyum/SPARK-24692.
2018-07-06 11:13:57 +08:00
Gengliang Wang 33952cfa81 [SPARK-24675][SQL] Rename table: validate existence of new location
## What changes were proposed in this pull request?
If table is renamed to a existing new location, data won't show up.
```
scala>  Seq("hello").toDF("a").write.format("parquet").saveAsTable("t")

scala> sql("select * from t").show()
+-----+
|    a|
+-----+
|hello|
+-----+

scala> sql("alter table t rename to test")
res2: org.apache.spark.sql.DataFrame = []

scala> sql("select * from test").show()
+---+
|  a|
+---+
+---+
```
The file layout is like
```
$ tree test
test
├── gabage
└── t
    ├── _SUCCESS
    └── part-00000-856b0f10-08f1-42d6-9eb3-7719261f3d5e-c000.snappy.parquet
```

In Hive, if the new location exists, the renaming will fail even the location is empty.

We should have the same validation in Catalog, in case of unexpected bugs.

## How was this patch tested?

New unit test.

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

Closes #21655 from gengliangwang/validate_rename_table.
2018-07-05 09:25:19 -07:00
Antonio Murgia 4be9f0c028 [SPARK-24673][SQL] scala sql function from_utc_timestamp second argument could be Column instead of String
## What changes were proposed in this pull request?

Add an overloaded version to `from_utc_timestamp` and `to_utc_timestamp` having second argument as a `Column` instead of `String`.

## How was this patch tested?

Unit testing, especially adding two tests to org.apache.spark.sql.DateFunctionsSuite.scala

Author: Antonio Murgia <antonio.murgia@agilelab.it>
Author: Antonio Murgia <antonio.murgia2@studio.unibo.it>

Closes #21693 from tmnd1991/feature/SPARK-24673.
2018-07-05 16:10:34 +08:00
Xiao Li 489a5294d1 [SPARK-17213][SPARK-17213][FOLLOW-UP] Improve the test of
## What changes were proposed in this pull request?
This is a minor improvement for the test of SPARK-17213

## How was this patch tested?
N/A

Author: Xiao Li <gatorsmile@gmail.com>

Closes #21716 from gatorsmile/testMaster23.
2018-07-05 09:56:48 +08:00
Yuming Wang 021145f364 [SPARK-24716][SQL] Refactor ParquetFilters
## What changes were proposed in this pull request?

Replace DataFrame schema to Parquet file schema when create `ParquetFilters`.
Thus we can easily implement `Decimal` and `Timestamp` push down. some thing like this:
```scala
// DecimalType: 32BitDecimalType
case ParquetSchemaType(DECIMAL, INT32, decimal)
  if pushDownDecimal =>
  (n: String, v: Any) => FilterApi.eq(
    intColumn(n),
    Option(v).map(_.asInstanceOf[JBigDecimal].unscaledValue().intValue()
      .asInstanceOf[Integer]).orNull)
// DecimalType: 64BitDecimalType
case ParquetSchemaType(DECIMAL, INT64, decimal)
  if pushDownDecimal =>
  (n: String, v: Any) => FilterApi.eq(
    longColumn(n),
    Option(v).map(_.asInstanceOf[JBigDecimal].unscaledValue().longValue()
      .asInstanceOf[java.lang.Long]).orNull)
// DecimalType: LegacyParquetFormat 32BitDecimalType & 64BitDecimalType
case ParquetSchemaType(DECIMAL, FIXED_LEN_BYTE_ARRAY, decimal)
  if pushDownDecimal && decimal.getPrecision <= Decimal.MAX_LONG_DIGITS =>
  (n: String, v: Any) => FilterApi.eq(
    binaryColumn(n),
    Option(v).map(d => decimalToBinaryUsingUnscaledLong(decimal.getPrecision,
      d.asInstanceOf[JBigDecimal])).orNull)
// DecimalType: ByteArrayDecimalType
case ParquetSchemaType(DECIMAL, FIXED_LEN_BYTE_ARRAY, decimal)
  if pushDownDecimal && decimal.getPrecision > Decimal.MAX_LONG_DIGITS =>
  (n: String, v: Any) => FilterApi.eq(
    binaryColumn(n),
    Option(v).map(d => decimalToBinaryUsingUnscaledBytes(decimal.getPrecision,
      d.asInstanceOf[JBigDecimal])).orNull)
```

```scala
// INT96 doesn't support pushdown
case ParquetSchemaType(TIMESTAMP_MICROS, INT64, null) =>
  (n: String, v: Any) => FilterApi.eq(
    longColumn(n),
    Option(v).map(t => DateTimeUtils.fromJavaTimestamp(t.asInstanceOf[Timestamp])
      .asInstanceOf[java.lang.Long]).orNull)
case ParquetSchemaType(TIMESTAMP_MILLIS, INT64, null) =>
  (n: String, v: Any) => FilterApi.eq(
    longColumn(n),
    Option(v).map(_.asInstanceOf[Timestamp].getTime.asInstanceOf[java.lang.Long]).orNull)
```

## How was this patch tested?

unit tests

Author: Yuming Wang <yumwang@ebay.com>

Closes #21696 from wangyum/SPARK-24716.
2018-07-04 20:15:40 +08:00
Takeshi Yamamuro b2deef64f6 [SPARK-24727][SQL] Add a static config to control cache size for generated classes
## What changes were proposed in this pull request?
Since SPARK-24250 has been resolved, executors correctly references user-defined configurations. So, this pr added a static config to control cache size for generated classes in `CodeGenerator`.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21705 from maropu/SPARK-24727.
2018-07-04 20:04:18 +08:00
Maxim Gekk 776f299fc8 [SPARK-24709][SQL] schema_of_json() - schema inference from an example
## What changes were proposed in this pull request?

In the PR, I propose to add new function - *schema_of_json()* which infers schema of JSON string literal. The result of the function is a string containing a schema in DDL format.

One of the use cases is using of *schema_of_json()* in the combination with *from_json()*. Currently, _from_json()_ requires a schema as a mandatory argument. The *schema_of_json()* function will allow to point out an JSON string as an example which has the same schema as the first argument of _from_json()_. For instance:

```sql
select from_json(json_column, schema_of_json('{"c1": [0], "c2": [{"c3":0}]}'))
from json_table;
```

## How was this patch tested?

Added new test to `JsonFunctionsSuite`, `JsonExpressionsSuite` and SQL tests to `json-functions.sql`

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

Closes #21686 from MaxGekk/infer_schema_json.
2018-07-04 09:38:18 +08:00
DB Tsai 5585c5765f
[SPARK-24420][BUILD] Upgrade ASM to 6.1 to support JDK9+
## What changes were proposed in this pull request?

Upgrade ASM to 6.1 to support JDK9+

## How was this patch tested?

Existing tests.

Author: DB Tsai <d_tsai@apple.com>

Closes #21459 from dbtsai/asm.
2018-07-03 10:13:48 -07:00
Marco Gaido a7c8f0c8cb [SPARK-24385][SQL] Resolve self-join condition ambiguity for EqualNullSafe
## What changes were proposed in this pull request?

In Dataset.join we have a small hack for resolving ambiguity in the column name for self-joins. The current code supports only `EqualTo`.

The PR extends the fix to `EqualNullSafe`.

Credit for this PR should be given to daniel-shields.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21605 from mgaido91/SPARK-24385_2.
2018-07-03 12:20:03 +08:00
maryannxue 797971ed42 [SPARK-24696][SQL] ColumnPruning rule fails to remove extra Project
## What changes were proposed in this pull request?

The ColumnPruning rule tries adding an extra Project if an input node produces fields more than needed, but as a post-processing step, it needs to remove the lower Project in the form of "Project - Filter - Project" otherwise it would conflict with PushPredicatesThroughProject and would thus cause a infinite optimization loop. The current post-processing method is defined as:
```
  private def removeProjectBeforeFilter(plan: LogicalPlan): LogicalPlan = plan transform {
    case p1  Project(_, f  Filter(_, p2  Project(_, child)))
      if p2.outputSet.subsetOf(child.outputSet) =>
      p1.copy(child = f.copy(child = child))
  }
```
This method works well when there is only one Filter but would not if there's two or more Filters. In this case, there is a deterministic filter and a non-deterministic filter so they stay as separate filter nodes and cannot be combined together.

An simplified illustration of the optimization process that forms the infinite loop is shown below (F1 stands for the 1st filter, F2 for the 2nd filter, P for project, S for scan of relation, PredicatePushDown as abbrev. of PushPredicatesThroughProject):
```
                             F1 - F2 - P - S
PredicatePushDown      =>    F1 - P - F2 - S
ColumnPruning          =>    F1 - P - F2 - P - S
                       =>    F1 - P - F2 - S        (Project removed)
PredicatePushDown      =>    P - F1 - F2 - S
ColumnPruning          =>    P - F1 - P - F2 - S
                       =>    P - F1 - P - F2 - P - S
                       =>    P - F1 - F2 - P - S    (only one Project removed)
RemoveRedundantProject =>    F1 - F2 - P - S        (goes back to the loop start)
```
So the problem is the ColumnPruning rule adds a Project under a Filter (and fails to remove it in the end), and that new Project triggers PushPredicateThroughProject. Once the filters have been push through the Project, a new Project will be added by the ColumnPruning rule and this goes on and on.
The fix should be when adding Projects, the rule applies top-down, but later when removing extra Projects, the process should go bottom-up to ensure all extra Projects can be matched.

## How was this patch tested?

Added a optimization rule test in ColumnPruningSuite; and a end-to-end test in SQLQuerySuite.

Author: maryannxue <maryannxue@apache.org>

Closes #21674 from maryannxue/spark-24696.
2018-06-29 23:46:12 -07:00
Yuming Wang 03545ce6de [SPARK-24638][SQL] StringStartsWith support push down
## What changes were proposed in this pull request?

`StringStartsWith` support push down. About 50% savings in compute time.

## How was this patch tested?
unit tests, manual tests and performance test:
```scala
cat <<EOF > SPARK-24638.scala
def benchmark(func: () => Unit): Long = {
  val start = System.currentTimeMillis()
  for(i <- 0 until 100) { func() }
  val end = System.currentTimeMillis()
  end - start
}
val path = "/tmp/spark/parquet/string/"
spark.range(10000000).selectExpr("concat(id, 'str', id) as id").coalesce(1).write.mode("overwrite").option("parquet.block.size", 1048576).parquet(path)
val df = spark.read.parquet(path)

spark.sql("set spark.sql.parquet.filterPushdown.string.startsWith=true")
val pushdownEnable = benchmark(() => df.where("id like '999998%'").count())

spark.sql("set spark.sql.parquet.filterPushdown.string.startsWith=false")
val pushdownDisable = benchmark(() => df.where("id like '999998%'").count())

val improvements = pushdownDisable - pushdownEnable
println(s"improvements: $improvements")
EOF

bin/spark-shell -i SPARK-24638.scala
```
result:
```scala
Loading SPARK-24638.scala...
benchmark: (func: () => Unit)Long
path: String = /tmp/spark/parquet/string/
df: org.apache.spark.sql.DataFrame = [id: string]
res1: org.apache.spark.sql.DataFrame = [key: string, value: string]
pushdownEnable: Long = 11608
res2: org.apache.spark.sql.DataFrame = [key: string, value: string]
pushdownDisable: Long = 31981
improvements: Long = 20373
```

Author: Yuming Wang <yumwang@ebay.com>

Closes #21623 from wangyum/SPARK-24638.
2018-06-30 13:58:50 +08:00
Jose Torres f6e6899a8b [SPARK-24386][SS] coalesce(1) aggregates in continuous processing
## What changes were proposed in this pull request?

Provide a continuous processing implementation of coalesce(1), as well as allowing aggregates on top of it.

The changes in ContinuousQueuedDataReader and such are to use split.index (the ID of the partition within the RDD currently being compute()d) rather than context.partitionId() (the partition ID of the scheduled task within the Spark job - that is, the post coalesce writer). In the absence of a narrow dependency, these values were previously always the same, so there was no need to distinguish.

## How was this patch tested?

new unit test

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

Closes #21560 from jose-torres/coalesce.
2018-06-28 16:25:40 -07:00
Jacek Laskowski e1d3f80103 [SPARK-24408][SQL][DOC] Move abs function to math_funcs group
## What changes were proposed in this pull request?

A few math functions (`abs` , `bitwiseNOT`, `isnan`, `nanvl`) are not in **math_funcs** group. They should really be.

## How was this patch tested?

Awaiting Jenkins

Author: Jacek Laskowski <jacek@japila.pl>

Closes #21448 from jaceklaskowski/SPARK-24408-math-funcs-doc.
2018-06-28 13:22:52 -07:00
Xingbo Jiang 5b05966488 [SPARK-24564][TEST] Add test suite for RecordBinaryComparator
## What changes were proposed in this pull request?

Add a new test suite to test RecordBinaryComparator.

## How was this patch tested?

New test suite.

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #21570 from jiangxb1987/rbc-test.
2018-06-28 14:19:50 +08:00
Fokko Driesprong 6a97e8eb31 [SPARK-24603][SQL] Fix findTightestCommonType reference in comments
findTightestCommonTypeOfTwo has been renamed to findTightestCommonType

## What changes were proposed in this pull request?

(Please fill in changes proposed in this fix)

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

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

Author: Fokko Driesprong <fokkodriesprong@godatadriven.com>

Closes #21597 from Fokko/fd-typo.
2018-06-28 09:59:00 +08:00
Takeshi Yamamuro 1c9acc2438 [SPARK-24206][SQL][FOLLOW-UP] Update DataSourceReadBenchmark benchmark results
## What changes were proposed in this pull request?
This pr corrected the default configuration (`spark.master=local[1]`) for benchmarks. Also, this updated performance results on the AWS `r3.xlarge`.

## How was this patch tested?
N/A

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21625 from maropu/FixDataSourceReadBenchmark.
2018-06-28 09:21:10 +08:00
Takeshi Yamamuro bd32b509a1 [SPARK-24645][SQL] Skip parsing when csvColumnPruning enabled and partitions scanned only
## What changes were proposed in this pull request?
In the master, when `csvColumnPruning`(implemented in [this commit](64fad0b519 (diff-d19881aceddcaa5c60620fdcda99b4c4))) enabled and partitions scanned only, it throws an exception below;

```
scala> val dir = "/tmp/spark-csv/csv"
scala> spark.range(10).selectExpr("id % 2 AS p", "id").write.mode("overwrite").partitionBy("p").csv(dir)
scala> spark.read.csv(dir).selectExpr("sum(p)").collect()
18/06/25 13:12:51 ERROR Executor: Exception in task 0.0 in stage 2.0 (TID 5)
java.lang.NullPointerException
        at org.apache.spark.sql.execution.datasources.csv.UnivocityParser.org$apache$spark$sql$execution$datasources$csv$UnivocityParser$$convert(UnivocityParser.scala:197)
        at org.apache.spark.sql.execution.datasources.csv.UnivocityParser.parse(UnivocityParser.scala:190)
        at org.apache.spark.sql.execution.datasources.csv.UnivocityParser$$anonfun$5.apply(UnivocityParser.scala:309)
        at org.apache.spark.sql.execution.datasources.csv.UnivocityParser$$anonfun$5.apply(UnivocityParser.scala:309)
        at org.apache.spark.sql.execution.datasources.FailureSafeParser.parse(FailureSafeParser.scala:61)
        ...
```
This pr modified code to skip CSV parsing in the case.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21631 from maropu/SPARK-24645.
2018-06-28 09:19:25 +08:00
Kallman, Steven c5aa54d54b [SPARK-24553][WEB-UI] http 302 fixes for href redirect
## What changes were proposed in this pull request?

Updated URL/href links to include a '/' before '?id' to make links consistent and avoid http 302 redirect errors within UI port 4040 tabs.

## How was this patch tested?

Built a runnable distribution and executed jobs. Validated that http 302 redirects are no longer encountered when clicking on links within UI port 4040 tabs.

Author: Steven Kallman <SJKallmangmail.com>

Author: Kallman, Steven <Steven.Kallman@CapitalOne.com>

Closes #21600 from SJKallman/{Spark-24553}{WEB-UI}-redirect-href-fixes.
2018-06-27 15:36:59 -07:00
Takeshi Yamamuro 893ea224cc [SPARK-24204][SQL] Verify a schema in Json/Orc/ParquetFileFormat
## What changes were proposed in this pull request?
This pr added code to verify a schema in Json/Orc/ParquetFileFormat along with CSVFileFormat.

## How was this patch tested?
Added verification tests in `FileBasedDataSourceSuite` and  `HiveOrcSourceSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21389 from maropu/SPARK-24204.
2018-06-27 15:25:51 -07:00
debugger87 c04cb2d1b7 [SPARK-21687][SQL] Spark SQL should set createTime for Hive partition
## What changes were proposed in this pull request?

Set createTime for every hive partition created in Spark SQL, which could be used to manage data lifecycle in Hive warehouse. We found  that almost every partition modified by spark sql has not been set createTime.

```
mysql> select * from partitions where create_time=0 limit 1\G;
*************************** 1. row ***************************
         PART_ID: 1028584
     CREATE_TIME: 0
LAST_ACCESS_TIME: 1502203611
       PART_NAME: date=20170130
           SD_ID: 1543605
          TBL_ID: 211605
  LINK_TARGET_ID: NULL
1 row in set (0.27 sec)
```

## How was this patch tested?
 N/A

Author: debugger87 <yangchaozhong.2009@gmail.com>
Author: Chaozhong Yang <yangchaozhong.2009@gmail.com>

Closes #18900 from debugger87/fix/set-create-time-for-hive-partition.
2018-06-27 11:34:28 -07:00
Yuanjian Li 6a0b77a55d [SPARK-24215][PYSPARK][FOLLOW UP] Implement eager evaluation for DataFrame APIs in PySpark
## What changes were proposed in this pull request?

Address comments in #21370 and add more test.

## How was this patch tested?

Enhance test in pyspark/sql/test.py and DataFrameSuite

Author: Yuanjian Li <xyliyuanjian@gmail.com>

Closes #21553 from xuanyuanking/SPARK-24215-follow.
2018-06-27 10:43:06 -07:00
Vayda, Oleksandr: IT (PRG) 2669b4de3b [SPARK-23927][SQL] Add "sequence" expression
## What changes were proposed in this pull request?
The PR adds the SQL function ```sequence```.
https://issues.apache.org/jira/browse/SPARK-23927

The behavior of the function is based on Presto's one.
Ref: https://prestodb.io/docs/current/functions/array.html

- ```sequence(start, stop) → array<bigint>```
Generate a sequence of integers from ```start``` to ```stop```, incrementing by ```1``` if ```start``` is less than or equal to ```stop```, otherwise ```-1```.
- ```sequence(start, stop, step) → array<bigint>```
Generate a sequence of integers from ```start``` to ```stop```, incrementing by ```step```.
- ```sequence(start_date, stop_date) → array<date>```
Generate a sequence of dates from ```start_date``` to ```stop_date```, incrementing by ```interval 1 day``` if ```start_date``` is less than or equal to ```stop_date```, otherwise ```- interval 1 day```.
- ```sequence(start_date, stop_date, step_interval) → array<date>```
Generate a sequence of dates from ```start_date``` to ```stop_date```, incrementing by ```step_interval```. The type of ```step_interval``` is ```CalendarInterval```.
- ```sequence(start_timestemp, stop_timestemp) → array<timestamp>```
Generate a sequence of timestamps from ```start_timestamps``` to ```stop_timestamps```, incrementing by ```interval 1 day``` if ```start_date``` is less than or equal to ```stop_date```, otherwise ```- interval 1 day```.
- ```sequence(start_timestamp, stop_timestamp, step_interval) → array<timestamp>```
Generate a sequence of timestamps from ```start_timestamps``` to ```stop_timestamps```, incrementing by ```step_interval```. The type of ```step_interval``` is ```CalendarInterval```.

## How was this patch tested?

Added unit tests.

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

Closes #21155 from wajda/feature/array-api-sequence.
2018-06-27 11:52:31 +09:00
Maxim Gekk d08f53dc61 [SPARK-24605][SQL] size(null) returns null instead of -1
## What changes were proposed in this pull request?

In PR, I propose new behavior of `size(null)` under the config flag `spark.sql.legacy.sizeOfNull`. If the former one is disabled, the `size()` function returns `null` for `null` input. By default the `spark.sql.legacy.sizeOfNull` is enabled to keep backward compatibility with previous versions. In that case, `size(null)` returns `-1`.

## How was this patch tested?

Modified existing tests for the `size()` function to check new behavior (`null`) and old one (`-1`).

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

Closes #21598 from MaxGekk/legacy-size-of-null.
2018-06-27 10:36:51 +08:00
Dilip Biswal 02f8781fa2 [SPARK-24423][SQL] Add a new option for JDBC sources
## What changes were proposed in this pull request?
Here is the description in the JIRA -

Currently, our JDBC connector provides the option `dbtable` for users to specify the to-be-loaded JDBC source table.

 ```SQL
 val jdbcDf = spark.read
   .format("jdbc")
   .option("dbtable", "dbName.tableName")
   .options(jdbcCredentials: Map)
   .load()
 ```

Normally, users do not fetch the whole JDBC table due to the poor performance/throughput of JDBC. Thus, they normally just fetch a small set of tables. For advanced users, they can pass a subquery as the option.

 ```SQL
 val query = """ (select * from tableName limit 10) as tmp """
 val jdbcDf = spark.read
   .format("jdbc")
   .option("dbtable", query)
   .options(jdbcCredentials: Map)
   .load()
 ```
However, this is straightforward to end users. We should simply allow users to specify the query by a new option `query`. We will handle the complexity for them.

 ```SQL
 val query = """select * from tableName limit 10"""
 val jdbcDf = spark.read
   .format("jdbc")
   .option("query", query)
   .options(jdbcCredentials: Map)
   .load()
```

## How was this patch tested?
Added tests in JDBCSuite and JDBCWriterSuite.
Also tested against MySQL, Postgress, Oracle, DB2 (using docker infrastructure) to make sure there are no syntax issues.

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

Closes #21590 from dilipbiswal/SPARK-24423.
2018-06-26 15:17:00 -07:00
Marek Novotny e07aee2165 [SPARK-24636][SQL] Type coercion of arrays for array_join function
## What changes were proposed in this pull request?
Presto's implementation accepts arbitrary arrays of primitive types as an input:

```
presto> SELECT array_join(ARRAY [1, 2, 3], ', ');
_col0
---------
1, 2, 3
(1 row)
```

This PR proposes to implement a type coercion rule for ```array_join``` function that converts arrays of primitive as well as non-primitive types to arrays of string.

## How was this patch tested?

New test cases add into:
- sql-tests/inputs/typeCoercion/native/arrayJoin.sql
- DataFrameFunctionsSuite.scala

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

Closes #21620 from mn-mikke/SPARK-24636.
2018-06-26 09:51:55 +08:00
Marcelo Vanzin 6d16b9885d [SPARK-24552][CORE][SQL] Use task ID instead of attempt number for writes.
This passes the unique task attempt id instead of attempt number to v2 data sources because attempt number is reused when stages are retried. When attempt numbers are reused, sources that track data by partition id and attempt number may incorrectly clean up data because the same attempt number can be both committed and aborted.

For v1 / Hadoop writes, generate a unique ID based on available attempt numbers to avoid a similar problem.

Closes #21558

Author: Marcelo Vanzin <vanzin@cloudera.com>
Author: Ryan Blue <blue@apache.org>

Closes #21606 from vanzin/SPARK-24552.2.
2018-06-25 16:54:57 -07:00
Stacy Kerkela 5264164a67 [SPARK-24648][SQL] SqlMetrics should be threadsafe
Use LongAdder to make SQLMetrics thread safe.

## What changes were proposed in this pull request?
Replace += with LongAdder.add() for concurrent counting

## How was this patch tested?
Unit tests with local threads

Author: Stacy Kerkela <stacy.kerkela@databricks.com>

Closes #21634 from dbkerkela/sqlmetrics-concurrency-stacy.
2018-06-25 23:41:39 +02:00
Marco Gaido 594ac4f7b8 [SPARK-24633][SQL] Fix codegen when split is required for arrays_zip
## What changes were proposed in this pull request?

In function array_zip, when split is required by the high number of arguments, a codegen error can happen.

The PR fixes codegen for cases when splitting the code is required.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21621 from mgaido91/SPARK-24633.
2018-06-25 23:44:20 +08:00
Maryann Xue bac50aa371 [SPARK-24596][SQL] Non-cascading Cache Invalidation
## What changes were proposed in this pull request?

1. Add parameter 'cascade' in CacheManager.uncacheQuery(). Under 'cascade=false' mode, only invalidate the current cache, and for other dependent caches, rebuild execution plan and reuse cached buffer.
2. Pass true/false from callers in different uncache scenarios:
- Drop tables and regular (persistent) views: regular mode
- Drop temporary views: non-cascading mode
- Modify table contents (INSERT/UPDATE/MERGE/DELETE): regular mode
- Call `DataSet.unpersist()`: non-cascading mode
- Call `Catalog.uncacheTable()`: follow the same convention as drop tables/view, which is, use non-cascading mode for temporary views and regular mode for the rest

Note that a regular (persistent) view is a database object just like a table, so after dropping a regular view (whether cached or not cached), any query referring to that view should no long be valid. Hence if a cached persistent view is dropped, we need to invalidate the all dependent caches so that exceptions will be thrown for any later reference. On the other hand, a temporary view is in fact equivalent to an unnamed DataSet, and dropping a temporary view should have no impact on queries referencing that view. Thus we should do non-cascading uncaching for temporary views, which also guarantees a consistent uncaching behavior between temporary views and unnamed DataSets.

## How was this patch tested?

New tests in CachedTableSuite and DatasetCacheSuite.

Author: Maryann Xue <maryannxue@apache.org>

Closes #21594 from maryannxue/noncascading-cache.
2018-06-25 07:17:30 -07:00
Takuya UESHIN 6e0596e263 [SPARK-23931][SQL][FOLLOW-UP] Make arrays_zip in function.scala @scala.annotation.varargs.
## What changes were proposed in this pull request?

This is a follow-up pr of #21045 which added `arrays_zip`.
The `arrays_zip` in functions.scala should've been `scala.annotation.varargs`.

This pr makes it `scala.annotation.varargs`.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #21630 from ueshin/issues/SPARK-23931/fup1.
2018-06-24 23:56:47 -07:00
Takeshi Yamamuro f596ebe4d3 [SPARK-24327][SQL] Verify and normalize a partition column name based on the JDBC resolved schema
## What changes were proposed in this pull request?
This pr modified JDBC datasource code to verify and normalize a partition column based on the JDBC resolved schema before building `JDBCRelation`.

Closes #20370

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21379 from maropu/SPARK-24327.
2018-06-24 23:14:42 -07:00
Bryan Cutler a5849ad9a3 [SPARK-24324][PYTHON] Pandas Grouped Map UDF should assign result columns by name
## What changes were proposed in this pull request?

Currently, a `pandas_udf` of type `PandasUDFType.GROUPED_MAP` will assign the resulting columns based on index of the return pandas.DataFrame.  If a new DataFrame is returned and constructed using a dict, then the order of the columns could be arbitrary and be different than the defined schema for the UDF.  If the schema types still match, then no error will be raised and the user will see column names and column data mixed up.

This change will first try to assign columns using the return type field names.  If a KeyError occurs, then the column index is checked if it is string based. If so, then the error is raised as it is most likely a naming mistake, else it will fallback to assign columns by position and raise a TypeError if the field types do not match.

## How was this patch tested?

Added a test that returns a new DataFrame with column order different than the schema.

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #21427 from BryanCutler/arrow-grouped-map-mixesup-cols-SPARK-24324.
2018-06-24 09:28:46 +08:00
Takeshi Yamamuro 98f363b774 [SPARK-24206][SQL] Improve FilterPushdownBenchmark benchmark code
## What changes were proposed in this pull request?
This pr added benchmark code `FilterPushdownBenchmark` for string pushdown and updated performance results on the AWS `r3.xlarge`.

## How was this patch tested?
N/A

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #21288 from maropu/UpdateParquetBenchmark.
2018-06-23 17:51:18 -07:00
Maxim Gekk c7e2742f9b [SPARK-24190][SQL] Allow saving of JSON files in UTF-16 and UTF-32
## What changes were proposed in this pull request?

Currently, restrictions in JSONOptions for `encoding` and `lineSep` are the same for read and for write. For example, a requirement for `lineSep` in the code:

```
df.write.option("encoding", "UTF-32BE").json(file)
```
doesn't allow to skip `lineSep` and use its default value `\n` because it throws the exception:
```
equirement failed: The lineSep option must be specified for the UTF-32BE encoding
java.lang.IllegalArgumentException: requirement failed: The lineSep option must be specified for the UTF-32BE encoding
```

In the PR, I propose to separate JSONOptions in read and write, and make JSONOptions in write less restrictive.

## How was this patch tested?

Added new test for blacklisted encodings in read. And the `lineSep` option was removed in write for some tests.

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

Closes #21247 from MaxGekk/json-options-in-write.
2018-06-23 17:40:20 -07:00
Marek Novotny 92c2f00bd2 [SPARK-23934][SQL] Adding map_from_entries function
## What changes were proposed in this pull request?
The PR adds the `map_from_entries` function that returns a map created from the given array of entries.

## How was this patch tested?
New tests added into:
- `CollectionExpressionSuite`
- `DataFrameFunctionSuite`

## CodeGen Examples
### Primitive-type Keys and Values
```
val idf = Seq(
  Seq((1, 10), (2, 20), (3, 10)),
  Seq((1, 10), null, (2, 20))
).toDF("a")
idf.filter('a.isNotNull).select(map_from_entries('a)).debugCodegen
```
Result:
```
/* 042 */         boolean project_isNull_0 = false;
/* 043 */         MapData project_value_0 = null;
/* 044 */
/* 045 */         for (int project_idx_2 = 0; !project_isNull_0 && project_idx_2 < inputadapter_value_0.numElements(); project_idx_2++) {
/* 046 */           project_isNull_0 |= inputadapter_value_0.isNullAt(project_idx_2);
/* 047 */         }
/* 048 */         if (!project_isNull_0) {
/* 049 */           final int project_numEntries_0 = inputadapter_value_0.numElements();
/* 050 */
/* 051 */           final long project_keySectionSize_0 = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(project_numEntries_0, 4);
/* 052 */           final long project_valueSectionSize_0 = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(project_numEntries_0, 4);
/* 053 */           final long project_byteArraySize_0 = 8 + project_keySectionSize_0 + project_valueSectionSize_0;
/* 054 */           if (project_byteArraySize_0 > 2147483632) {
/* 055 */             final Object[] project_keys_0 = new Object[project_numEntries_0];
/* 056 */             final Object[] project_values_0 = new Object[project_numEntries_0];
/* 057 */
/* 058 */             for (int project_idx_1 = 0; project_idx_1 < project_numEntries_0; project_idx_1++) {
/* 059 */               InternalRow project_entry_1 = inputadapter_value_0.getStruct(project_idx_1, 2);
/* 060 */
/* 061 */               project_keys_0[project_idx_1] = project_entry_1.getInt(0);
/* 062 */               project_values_0[project_idx_1] = project_entry_1.getInt(1);
/* 063 */             }
/* 064 */
/* 065 */             project_value_0 = org.apache.spark.sql.catalyst.util.ArrayBasedMapData.apply(project_keys_0, project_values_0);
/* 066 */
/* 067 */           } else {
/* 068 */             final byte[] project_byteArray_0 = new byte[(int)project_byteArraySize_0];
/* 069 */             UnsafeMapData project_unsafeMapData_0 = new UnsafeMapData();
/* 070 */             Platform.putLong(project_byteArray_0, 16, project_keySectionSize_0);
/* 071 */             Platform.putLong(project_byteArray_0, 24, project_numEntries_0);
/* 072 */             Platform.putLong(project_byteArray_0, 24 + project_keySectionSize_0, project_numEntries_0);
/* 073 */             project_unsafeMapData_0.pointTo(project_byteArray_0, 16, (int)project_byteArraySize_0);
/* 074 */             ArrayData project_keyArrayData_0 = project_unsafeMapData_0.keyArray();
/* 075 */             ArrayData project_valueArrayData_0 = project_unsafeMapData_0.valueArray();
/* 076 */
/* 077 */             for (int project_idx_0 = 0; project_idx_0 < project_numEntries_0; project_idx_0++) {
/* 078 */               InternalRow project_entry_0 = inputadapter_value_0.getStruct(project_idx_0, 2);
/* 079 */
/* 080 */               project_keyArrayData_0.setInt(project_idx_0, project_entry_0.getInt(0));
/* 081 */               project_valueArrayData_0.setInt(project_idx_0, project_entry_0.getInt(1));
/* 082 */             }
/* 083 */
/* 084 */             project_value_0 = project_unsafeMapData_0;
/* 085 */           }
/* 086 */
/* 087 */         }
```
### Non-primitive-type Keys and Values
```
val sdf = Seq(
  Seq(("a", null), ("b", "bb"), ("c", "aa")),
  Seq(("a", "aa"), null, (null, "bb"))
).toDF("a")
sdf.filter('a.isNotNull).select(map_from_entries('a)).debugCodegen
```
Result:
```
/* 042 */         boolean project_isNull_0 = false;
/* 043 */         MapData project_value_0 = null;
/* 044 */
/* 045 */         for (int project_idx_1 = 0; !project_isNull_0 && project_idx_1 < inputadapter_value_0.numElements(); project_idx_1++) {
/* 046 */           project_isNull_0 |= inputadapter_value_0.isNullAt(project_idx_1);
/* 047 */         }
/* 048 */         if (!project_isNull_0) {
/* 049 */           final int project_numEntries_0 = inputadapter_value_0.numElements();
/* 050 */
/* 051 */           final Object[] project_keys_0 = new Object[project_numEntries_0];
/* 052 */           final Object[] project_values_0 = new Object[project_numEntries_0];
/* 053 */
/* 054 */           for (int project_idx_0 = 0; project_idx_0 < project_numEntries_0; project_idx_0++) {
/* 055 */             InternalRow project_entry_0 = inputadapter_value_0.getStruct(project_idx_0, 2);
/* 056 */
/* 057 */             if (project_entry_0.isNullAt(0)) {
/* 058 */               throw new RuntimeException("The first field from a struct (key) can't be null.");
/* 059 */             }
/* 060 */
/* 061 */             project_keys_0[project_idx_0] = project_entry_0.getUTF8String(0);
/* 062 */             project_values_0[project_idx_0] = project_entry_0.getUTF8String(1);
/* 063 */           }
/* 064 */
/* 065 */           project_value_0 = org.apache.spark.sql.catalyst.util.ArrayBasedMapData.apply(project_keys_0, project_values_0);
/* 066 */
/* 067 */         }
```

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

Closes #21282 from mn-mikke/feature/array-api-map_from_entries-to-master.
2018-06-22 16:18:22 +09:00
Wenchen Fan dc8a6befa5 [SPARK-24588][SS] streaming join should require HashClusteredPartitioning from children
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/19080 we simplified the distribution/partitioning framework, and make all the join-like operators require `HashClusteredDistribution` from children. Unfortunately streaming join operator was missed.

This can cause wrong result. Think about
```
val input1 = MemoryStream[Int]
val input2 = MemoryStream[Int]

val df1 = input1.toDF.select('value as 'a, 'value * 2 as 'b)
val df2 = input2.toDF.select('value as 'a, 'value * 2 as 'b).repartition('b)
val joined = df1.join(df2, Seq("a", "b")).select('a)
```

The physical plan is
```
*(3) Project [a#5]
+- StreamingSymmetricHashJoin [a#5, b#6], [a#10, b#11], Inner, condition = [ leftOnly = null, rightOnly = null, both = null, full = null ], state info [ checkpoint = <unknown>, runId = 54e31fce-f055-4686-b75d-fcd2b076f8d8, opId = 0, ver = 0, numPartitions = 5], 0, state cleanup [ left = null, right = null ]
   :- Exchange hashpartitioning(a#5, b#6, 5)
   :  +- *(1) Project [value#1 AS a#5, (value#1 * 2) AS b#6]
   :     +- StreamingRelation MemoryStream[value#1], [value#1]
   +- Exchange hashpartitioning(b#11, 5)
      +- *(2) Project [value#3 AS a#10, (value#3 * 2) AS b#11]
         +- StreamingRelation MemoryStream[value#3], [value#3]
```

The left table is hash partitioned by `a, b`, while the right table is hash partitioned by `b`. This means, we may have a matching record that is in different partitions, which should be in the output but not.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21587 from cloud-fan/join.
2018-06-21 15:38:46 -07:00
Maryann Xue b9a6f7499a [SPARK-24613][SQL] Cache with UDF could not be matched with subsequent dependent caches
## What changes were proposed in this pull request?

Wrap the logical plan with a `AnalysisBarrier` for execution plan compilation in CacheManager, in order to avoid the plan being analyzed again.

## How was this patch tested?

Add one test in `DatasetCacheSuite`

Author: Maryann Xue <maryannxue@apache.org>

Closes #21602 from maryannxue/cache-mismatch.
2018-06-21 11:45:30 -07:00
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
Kazuaki Ishizaki a7c19d9c21 [SPARK-23713][SQL] Cleanup UnsafeWriter and BufferHolder classes
## What changes were proposed in this pull request?

This PR implemented the following cleanups related to  `UnsafeWriter` class:
- Remove code duplication between `UnsafeRowWriter` and `UnsafeArrayWriter`
- Make `BufferHolder` class internal by delegating its accessor methods to `UnsafeWriter`
- Replace `UnsafeRow.setTotalSize(...)` with `UnsafeRowWriter.setTotalSize()`

## How was this patch tested?

Tested by existing UTs

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

Closes #20850 from kiszk/SPARK-23713.
2018-04-02 21:48:44 +02:00
Tathagata Das 15298b99ac [SPARK-23827][SS] StreamingJoinExec should ensure that input data is partitioned into specific number of partitions
## What changes were proposed in this pull request?

Currently, the requiredChildDistribution does not specify the partitions. This can cause the weird corner cases where the child's distribution is `SinglePartition` which satisfies the required distribution of `ClusterDistribution(no-num-partition-requirement)`, thus eliminating the shuffle needed to repartition input data into the required number of partitions (i.e. same as state stores). That can lead to "file not found" errors on the state store delta files as the micro-batch-with-no-shuffle will not run certain tasks and therefore not generate the expected state store delta files.

This PR adds the required constraint on the number of partitions.

## How was this patch tested?
Modified test harness to always check that ANY stateful operator should have a constraint on the number of partitions. As part of that, the existing opt-in checks on child output partitioning were removed, as they are redundant.

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

Closes #20941 from tdas/SPARK-23827.
2018-03-30 16:48:26 -07:00
gatorsmile bc8d093117 [SPARK-23500][SQL][FOLLOWUP] Fix complex type simplification rules to apply to entire plan
## What changes were proposed in this pull request?
This PR is to improve the test coverage of the original PR https://github.com/apache/spark/pull/20687

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20911 from gatorsmile/addTests.
2018-03-30 23:21:07 +08:00
Jose Torres 5b5a36ed6d Roll forward "[SPARK-23096][SS] Migrate rate source to V2"
## What changes were proposed in this pull request?

Roll forward c68ec4e (#20688).

There are two minor test changes required:

* An error which used to be TreeNodeException[ArithmeticException] is no longer wrapped and is now just ArithmeticException.
* The test framework simply does not set the active Spark session. (Or rather, it doesn't do so early enough - I think it only happens when a query is analyzed.) I've added the required logic to SQLTestUtils.

## How was this patch tested?

existing tests

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

Closes #20922 from jose-torres/ratefix.
2018-03-30 21:54:26 +08:00
yucai b02e76cbff [SPARK-23727][SQL] Support for pushing down filters for DateType in parquet
## What changes were proposed in this pull request?

This PR supports for pushing down filters for DateType in parquet

## How was this patch tested?

Added UT and tested in local.

Author: yucai <yyu1@ebay.com>

Closes #20851 from yucai/SPARK-23727.
2018-03-30 15:07:38 +08:00
Jose Torres b348901192 [SPARK-23808][SQL] Set default Spark session in test-only spark sessions.
## What changes were proposed in this pull request?

Set default Spark session in the TestSparkSession and TestHiveSparkSession constructors.

## How was this patch tested?

new unit tests

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

Closes #20926 from jose-torres/test3.
2018-03-29 21:36:56 -07:00
gatorsmile 761565a3cc Revert "[SPARK-23096][SS] Migrate rate source to V2"
This reverts commit c68ec4e6a1.
2018-03-28 09:11:52 -07:00
hyukjinkwon 34c4b9c57e [SPARK-23765][SQL] Supports custom line separator for json datasource
## What changes were proposed in this pull request?

This PR proposes to add lineSep option for a configurable line separator in text datasource.
It supports this option by using `LineRecordReader`'s functionality with passing it to the constructor.

The approach is similar with https://github.com/apache/spark/pull/20727; however, one main difference is, it uses text datasource's `lineSep` option to parse line by line in JSON's schema inference.

## How was this patch tested?

Manually tested and unit tests were added.

Author: hyukjinkwon <gurwls223@apache.org>
Author: hyukjinkwon <gurwls223@gmail.com>

Closes #20877 from HyukjinKwon/linesep-json.
2018-03-28 19:49:27 +08:00
jerryshao c68ec4e6a1 [SPARK-23096][SS] Migrate rate source to V2
## What changes were proposed in this pull request?

This PR migrate micro batch rate source to V2 API and rewrite UTs to suite V2 test.

## How was this patch tested?

UTs.

Author: jerryshao <sshao@hortonworks.com>

Closes #20688 from jerryshao/SPARK-23096.
2018-03-27 14:39:05 -07:00
Kazuaki Ishizaki e4bec7cb88 [SPARK-23549][SQL] Cast to timestamp when comparing timestamp with date
## What changes were proposed in this pull request?

This PR fixes an incorrect comparison in SQL between timestamp and date. This is because both of them are casted to `string` and then are compared lexicographically. This implementation shows `false` regarding this query `spark.sql("select cast('2017-03-01 00:00:00' as timestamp) between cast('2017-02-28' as date) and cast('2017-03-01' as date)").show`.

This PR shows `true` for this query by casting `date("2017-03-01")` to `timestamp("2017-03-01 00:00:00")`.

(Please fill in changes proposed in this fix)

## How was this patch tested?

Added new UTs to `TypeCoercionSuite`.

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

Closes #20774 from kiszk/SPARK-23549.
2018-03-25 16:38:49 -07:00
Takeshi Yamamuro 5f653d4f7c [SPARK-23167][SQL] Add TPCDS queries v2.7 in TPCDSQuerySuite
## What changes were proposed in this pull request?
This pr added TPCDS v2.7 (latest) queries in `TPCDSQuerySuite` because the current `TPCDSQuerySuite` tests older one (v1.4) and some queries are different from v1.4 and v2.7. Since the original v2.7 queries have the syntaxes that Spark cannot parse, I changed these queries in a following way:

 - [date] + 14 days -> date + `INTERVAL` 14 days
 - [column name] as "30 days" -> [column name] as \`30 days\`
 - Fix some syntax errors, e.g., missing brackets

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20343 from maropu/TPCDSV2_7.
2018-03-25 09:18:26 -07:00
Jose Torres 816a5496ba [SPARK-23788][SS] Fix race in StreamingQuerySuite
## What changes were proposed in this pull request?

The serializability test uses the same MemoryStream instance for 3 different queries. If any of those queries ask it to commit before the others have run, the rest will see empty dataframes. This can fail the test if q3 is affected.

We should use one instance per query instead.

## How was this patch tested?

Existing unit test. If I move q2.processAllAvailable() before starting q3, the test always fails without the fix.

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

Closes #20896 from jose-torres/fixrace.
2018-03-24 18:21:01 -07:00
Liang-Chi Hsieh b2edc30db1 [SPARK-23614][SQL] Fix incorrect reuse exchange when caching is used
## What changes were proposed in this pull request?

We should provide customized canonicalize plan for `InMemoryRelation` and `InMemoryTableScanExec`. Otherwise, we can wrongly treat two different cached plans as same result. It causes wrongly reused exchange then.

For a test query like this:
```scala
val cached = spark.createDataset(Seq(TestDataUnion(1, 2, 3), TestDataUnion(4, 5, 6))).cache()
val group1 = cached.groupBy("x").agg(min(col("y")) as "value")
val group2 = cached.groupBy("x").agg(min(col("z")) as "value")
group1.union(group2)
```

Canonicalized plans before:

First exchange:
```
Exchange hashpartitioning(none#0, 5)
+- *(1) HashAggregate(keys=[none#0], functions=[partial_min(none#1)], output=[none#0, none#4])
   +- *(1) InMemoryTableScan [none#0, none#1]
         +- InMemoryRelation [x#4253, y#4254, z#4255], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)
               +- LocalTableScan [x#4253, y#4254, z#4255]
```

Second exchange:
```
Exchange hashpartitioning(none#0, 5)
+- *(3) HashAggregate(keys=[none#0], functions=[partial_min(none#1)], output=[none#0, none#4])
   +- *(3) InMemoryTableScan [none#0, none#1]
         +- InMemoryRelation [x#4253, y#4254, z#4255], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)
               +- LocalTableScan [x#4253, y#4254, z#4255]
```

You can find that they have the canonicalized plans are the same, although we use different columns in two `InMemoryTableScan`s.

Canonicalized plan after:

First exchange:
```
Exchange hashpartitioning(none#0, 5)
+- *(1) HashAggregate(keys=[none#0], functions=[partial_min(none#1)], output=[none#0, none#4])
   +- *(1) InMemoryTableScan [none#0, none#1]
         +- InMemoryRelation [none#0, none#1, none#2], true, 10000, StorageLevel(memory, 1 replicas)
               +- LocalTableScan [none#0, none#1, none#2]
```

Second exchange:
```
Exchange hashpartitioning(none#0, 5)
+- *(3) HashAggregate(keys=[none#0], functions=[partial_min(none#1)], output=[none#0, none#4])
   +- *(3) InMemoryTableScan [none#0, none#2]
         +- InMemoryRelation [none#0, none#1, none#2], true, 10000, StorageLevel(memory, 1 replicas)
               +- LocalTableScan [none#0, none#1, none#2]
```

## How was this patch tested?

Added unit test.

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

Closes #20831 from viirya/SPARK-23614.
2018-03-22 21:23:25 -07:00
Liang-Chi Hsieh 4d37008c78 [SPARK-23599][SQL] Use RandomUUIDGenerator in Uuid expression
## What changes were proposed in this pull request?

As stated in Jira, there are problems with current `Uuid` expression which uses `java.util.UUID.randomUUID` for UUID generation.

This patch uses the newly added `RandomUUIDGenerator` for UUID generation. So we can make `Uuid` deterministic between retries.

## How was this patch tested?

Added unit tests.

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

Closes #20861 from viirya/SPARK-23599-2.
2018-03-22 19:57:32 +01:00
Dilip Biswal 5c9eaa6b58 [SPARK-23372][SQL] Writing empty struct in parquet fails during execution. It should fail earlier in the processing.
## What changes were proposed in this pull request?
Currently we allow writing data frames with empty schema into a file based datasource for certain file formats such as JSON, ORC etc. For formats such as Parquet and Text, we raise error at different times of execution. For text format, we return error from the driver early on in processing where as for format such as parquet, the error is raised from executor.

**Example**
spark.emptyDataFrame.write.format("parquet").mode("overwrite").save(path)
**Results in**
``` SQL
org.apache.parquet.schema.InvalidSchemaException: Cannot write a schema with an empty group: message spark_schema {
 }

at org.apache.parquet.schema.TypeUtil$1.visit(TypeUtil.java:27)
 at org.apache.parquet.schema.TypeUtil$1.visit(TypeUtil.java:37)
 at org.apache.parquet.schema.MessageType.accept(MessageType.java:58)
 at org.apache.parquet.schema.TypeUtil.checkValidWriteSchema(TypeUtil.java:23)
 at org.apache.parquet.hadoop.ParquetFileWriter.<init>(ParquetFileWriter.java:225)
 at org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:342)
 at org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:302)
 at org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.<init>(ParquetOutputWriter.scala:37)
 at org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat$$anon$1.newInstance(ParquetFileFormat.scala:151)
 at org.apache.spark.sql.execution.datasources.FileFormatWriter$SingleDirectoryWriteTask.newOutputWriter(FileFormatWriter.scala:376)
 at org.apache.spark.sql.execution.datasources.FileFormatWriter$SingleDirectoryWriteTask.execute(FileFormatWriter.scala:387)
 at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask$3.apply(FileFormatWriter.scala:278)
 at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask$3.apply(FileFormatWriter.scala:276)
 at org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1411)
 at org.apache.spark.sql.execution.datasources.FileFormatWriter$.org$apache$spark$sql$execution$datasources$FileFormatWriter$$executeTask(FileFormatWriter.scala:281)
 at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply(FileFormatWriter.scala:206)
 at org.apache.spark.sql.execution.datasources.FileFormatWriter$$anonfun$write$1.apply(FileFormatWriter.scala:205)
 at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
 at org.apache.spark.scheduler.Task.run(Task.scala:109)
 at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:345)
 at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
 at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
 at java.lang.Thread.run(Thread.
```

In this PR, we unify the error processing and raise error on attempt to write empty schema based dataframes into file based datasource (orc, parquet, text , csv, json etc) early on in the processing.

## How was this patch tested?

Unit tests added in FileBasedDatasourceSuite.

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

Closes #20579 from dilipbiswal/spark-23372.
2018-03-21 21:49:02 -07:00
Gabor Somogyi 918c7e99af [SPARK-23288][SS] Fix output metrics with parquet sink
## What changes were proposed in this pull request?

Output metrics were not filled when parquet sink used.

This PR fixes this problem by passing a `BasicWriteJobStatsTracker` in `FileStreamSink`.

## How was this patch tested?

Additional unit test added.

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

Closes #20745 from gaborgsomogyi/SPARK-23288.
2018-03-21 10:06:26 -07:00
Takeshi Yamamuro 98d0ea3f60 [SPARK-23264][SQL] Fix scala.MatchError in literals.sql.out
## What changes were proposed in this pull request?
To fix `scala.MatchError` in `literals.sql.out`, this pr added an entry for `CalendarIntervalType` in `QueryExecution.toHiveStructString`.

## How was this patch tested?
Existing tests and added tests in `literals.sql`

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20872 from maropu/FixIntervalTests.
2018-03-21 09:52:28 -07:00
hyukjinkwon 8d79113b81 [SPARK-23577][SQL] Supports custom line separator for text datasource
## What changes were proposed in this pull request?

This PR proposes to add `lineSep` option for a configurable line separator in text datasource.

It supports this option by using `LineRecordReader`'s functionality with passing it to the constructor.

## How was this patch tested?

Manual tests and unit tests were added.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #20727 from HyukjinKwon/linesep-text.
2018-03-21 09:46:47 -07:00
Takeshi Yamamuro 983e8d9d64 [SPARK-23666][SQL] Do not display exprIds of Alias in user-facing info.
## What changes were proposed in this pull request?
To drop `exprId`s for `Alias` in user-facing info., this pr added an entry for `Alias` in `NonSQLExpression.sql`

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20827 from maropu/SPARK-23666.
2018-03-20 23:17:49 -07:00
Jose Torres 2c4b9962fd [SPARK-23574][SQL] Report SinglePartition in DataSourceV2ScanExec when there's exactly 1 data reader factory.
## What changes were proposed in this pull request?

Report SinglePartition in DataSourceV2ScanExec when there's exactly 1 data reader factory.

Note that this means reader factories end up being constructed as partitioning is checked; let me know if you think that could be a problem.

## How was this patch tested?

existing unit tests

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

Closes #20726 from jose-torres/SPARK-23574.
2018-03-20 11:46:51 -07:00
Dongjoon Hyun 5414abca4f [SPARK-23553][TESTS] Tests should not assume the default value of spark.sql.sources.default
## What changes were proposed in this pull request?

Currently, some tests have an assumption that `spark.sql.sources.default=parquet`. In fact, that is a correct assumption, but that assumption makes it difficult to test new data source format.

This PR aims to
- Improve test suites more robust and makes it easy to test new data sources in the future.
- Test new native ORC data source with the full existing Apache Spark test coverage.

As an example, the PR uses `spark.sql.sources.default=orc` during reviews. The value should be `parquet` when this PR is accepted.

## How was this patch tested?

Pass the Jenkins with updated tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20705 from dongjoon-hyun/SPARK-23553.
2018-03-16 09:36:30 -07:00
myroslavlisniak c2632edebd [SPARK-23670][SQL] Fix memory leak on SparkPlanGraphWrapper
Clean up SparkPlanGraphWrapper objects from InMemoryStore together with cleaning up SQLExecutionUIData
existing unit test was extended to check also SparkPlanGraphWrapper object count

vanzin

Author: myroslavlisniak <acnipin@gmail.com>

Closes #20813 from myroslavlisniak/master.
2018-03-15 17:20:59 -07:00