Commit graph

7170 commits

Author SHA1 Message Date
Liang-Chi Hsieh ff9ede0929 [SPARK-25627][TEST] Reduce test time for ContinuousStressSuite
## What changes were proposed in this pull request?

This goes to reduce test time for ContinuousStressSuite - from 8 mins 13 sec to 43 seconds.

The approach taken by this is to reduce the triggers and epochs to wait and to reduce the expected rows accordingly.

## How was this patch tested?

Existing tests.

Closes #22662 from viirya/SPARK-25627.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-22 13:18:29 -05:00
hyukjinkwon b8c6ba9e64
[SPARK-25779][SQL][TESTS] Remove SQL query tests for function documentation by DESCRIBE FUNCTION at SQLQueryTestSuite
Currently, there are some tests testing function descriptions:

```bash
$ grep -ir "describe function" sql/core/src/test/resources/sql-tests/inputs
sql/core/src/test/resources/sql-tests/inputs/json-functions.sql:describe function to_json;
sql/core/src/test/resources/sql-tests/inputs/json-functions.sql:describe function extended to_json;
sql/core/src/test/resources/sql-tests/inputs/json-functions.sql:describe function from_json;
sql/core/src/test/resources/sql-tests/inputs/json-functions.sql:describe function extended from_json;
```

Looks there are not quite good points about testing them since we're not going to test documentation itself.
For `DESCRIBE FCUNTION` functionality itself, they are already being tested here and there.
See the test failures in https://github.com/apache/spark/pull/18749 (where I added examples to function descriptions)

We better remove those tests so that people don't add such tests in the SQL tests.

## How was this patch tested?

Manual.

Closes #22776 from HyukjinKwon/SPARK-25779.

Authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-20 18:02:38 -07:00
Wenchen Fan ab5752cb95
[SPARK-25747][SQL] remove ColumnarBatchScan.needsUnsafeRowConversion
## What changes were proposed in this pull request?

`needsUnsafeRowConversion` is used in 2 places:
1. `ColumnarBatchScan.produceRows`
2. `FileSourceScanExec.doExecute`

When we hit `ColumnarBatchScan.produceRows`, it means whole stage codegen is on but the vectorized reader is off. The vectorized reader can be off for several reasons:
1. the file format doesn't have a vectorized reader(json, csv, etc.)
2. the vectorized reader config is off
3. the schema is not supported

Anyway when the vectorized reader is off, file format reader will always return unsafe rows, and other `ColumnarBatchScan` implementations also always return unsafe rows, so `ColumnarBatchScan.needsUnsafeRowConversion` is not needed.

When we hit `FileSourceScanExec.doExecute`, it means whole stage codegen is off. For this case, we need the `needsUnsafeRowConversion` to convert `ColumnarRow` to `UnsafeRow`, if the file format reader returns batch.

This PR removes `ColumnarBatchScan.needsUnsafeRowConversion`, and keep this flag only in `FileSourceScanExec`

## How was this patch tested?

existing tests

Closes #22750 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-20 17:45:04 -07:00
Yuming Wang 62551cceeb
[SPARK-25492][TEST] Refactor WideSchemaBenchmark to use main method
## What changes were proposed in this pull request?

Refactor `WideSchemaBenchmark` to use main method.
1. use `spark-submit`:
```console
bin/spark-submit --class  org.apache.spark.sql.execution.benchmark.WideSchemaBenchmark --jars ./core/target/spark-core_2.11-3.0.0-SNAPSHOT-tests.jar ./sql/core/target/spark-sql_2.11-3.0.0-SNAPSHOT-tests.jar
```

2. Generate benchmark result:
```console
SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.benchmark.WideSchemaBenchmark"
```

## How was this patch tested?

manual tests

Closes #22501 from wangyum/SPARK-25492.

Lead-authored-by: Yuming Wang <yumwang@ebay.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-20 17:31:13 -07:00
hyukjinkwon 3370865b0e [SPARK-25785][SQL] Add prettyNames for from_json, to_json, from_csv, and schema_of_json
## What changes were proposed in this pull request?

This PR adds `prettyNames` for `from_json`, `to_json`, `from_csv`, and `schema_of_json` so that appropriate names are used.

## How was this patch tested?

Unit tests

Closes #22773 from HyukjinKwon/minor-prettyNames.

Authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-20 10:15:53 +08:00
Yuming Wang 9ad0f6ea89
[SPARK-25269][SQL] SQL interface support specify StorageLevel when cache table
## What changes were proposed in this pull request?

SQL interface support specify `StorageLevel` when cache table. The semantic is:
```sql
CACHE TABLE tableName OPTIONS('storageLevel' 'DISK_ONLY');
```
All supported `StorageLevel` are:
eefdf9f9dd/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala (L172-L183)

## How was this patch tested?

unit tests and manual tests.

manual tests configuration:
```
--executor-memory 15G --executor-cores 5 --num-executors 50
```
Data:
Input Size / Records: 1037.7 GB / 11732805788

Result:
![image](https://user-images.githubusercontent.com/5399861/47213362-56a1c980-d3cd-11e8-82e7-28d7abc5923e.png)

Closes #22263 from wangyum/SPARK-25269.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-19 09:15:55 -07:00
Peter Toth f38594fc56 [SPARK-25768][SQL] fix constant argument expecting UDAFs
## What changes were proposed in this pull request?

Without this PR some UDAFs like `GenericUDAFPercentileApprox` can throw an exception because expecting a constant parameter (object inspector) as a particular argument.

The exception is thrown because `toPrettySQL` call in `ResolveAliases` analyzer rule transforms a `Literal` parameter to a `PrettyAttribute` which is then transformed to an `ObjectInspector` instead of a `ConstantObjectInspector`.
The exception comes from `getEvaluator` method of `GenericUDAFPercentileApprox` that actually shouldn't be called during `toPrettySQL` transformation. The reason why it is called are the non lazy fields in `HiveUDAFFunction`.

This PR makes all fields of `HiveUDAFFunction` lazy.

## How was this patch tested?

added new UT

Closes #22766 from peter-toth/SPARK-25768.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-19 21:17:14 +08:00
maryannxue e8167768cf [SPARK-25044][FOLLOW-UP] Change ScalaUDF constructor signature
## What changes were proposed in this pull request?

This is a follow-up PR for #22259. The extra field added in `ScalaUDF` with the original PR was declared optional, but should be indeed required, otherwise callers of `ScalaUDF`'s constructor could ignore this new field and cause the result to be incorrect. This PR makes the new field required and changes its name to `handleNullForInputs`.

#22259 breaks the previous behavior for null-handling of primitive-type input parameters. For example, for `val f = udf({(x: Int, y: Any) => x})`, `f(null, "str")` should return `null` but would return `0` after #22259. In this PR, all UDF methods except `def udf(f: AnyRef, dataType: DataType): UserDefinedFunction` have been restored with the original behavior. The only exception is documented in the Spark SQL migration guide.

In addition, now that we have this extra field indicating if a null-test should be applied on the corresponding input value, we can also make use of this flag to avoid the rule `HandleNullInputsForUDF` being applied infinitely.

## How was this patch tested?
Added UT in UDFSuite

Passed affected existing UTs:
AnalysisSuite
UDFSuite

Closes #22732 from maryannxue/spark-25044-followup.

Lead-authored-by: maryannxue <maryannxue@apache.org>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-19 21:03:59 +08:00
Russell Spitzer 6e0fc8b0fc [SPARK-25560][SQL] Allow FunctionInjection in SparkExtensions
This allows an implementer of Spark Session Extensions to utilize a
method "injectFunction" which will add a new function to the default
Spark Session Catalogue.

## What changes were proposed in this pull request?

Adds a new function to SparkSessionExtensions

    def injectFunction(functionDescription: FunctionDescription)

Where function description is a new type

  type FunctionDescription = (FunctionIdentifier, FunctionBuilder)

The functions are loaded in BaseSessionBuilder when the function registry does not have a parent
function registry to get loaded from.

## How was this patch tested?

New unit tests are added for the extension in SparkSessionExtensionSuite

Closes #22576 from RussellSpitzer/SPARK-25560.

Authored-by: Russell Spitzer <Russell.Spitzer@gmail.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
2018-10-19 10:40:56 +02:00
Justin Uang 1e6c1d8bfb [SPARK-25493][SQL] Use auto-detection for CRLF in CSV datasource multiline mode
## What changes were proposed in this pull request?

CSVs with windows style crlf ('\r\n') don't work in multiline mode. They work fine in single line mode because the line separation is done by Hadoop, which can handle all the different types of line separators. This PR fixes it by enabling Univocity's line separator detection in multiline mode, which will detect '\r\n', '\r', or '\n' automatically as it is done by hadoop in single line mode.

## How was this patch tested?

Unit test with a file with crlf line endings.

Closes #22503 from justinuang/fix-clrf-multiline.

Authored-by: Justin Uang <juang@palantir.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-19 11:13:02 +08:00
Yuming Wang 1117fc35ff
[SPARK-25760][SQL] Set AddJarCommand return empty
## What changes were proposed in this pull request?

Only `AddJarCommand` return `0`, the user will be confused about what it means. This PR sets it to empty.

```sql
spark-sql> add jar /Users/yumwang/spark/sql/hive/src/test/resources/TestUDTF.jar;
ADD JAR /Users/yumwang/spark/sql/hive/src/test/resources/TestUDTF.jar
0
spark-sql>
```

## How was this patch tested?

manual tests
```sql
spark-sql> add jar /Users/yumwang/spark/sql/hive/src/test/resources/TestUDTF.jar;
ADD JAR /Users/yumwang/spark/sql/hive/src/test/resources/TestUDTF.jar
spark-sql>
```

Closes #22747 from wangyum/AddJarCommand.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-18 09:19:42 -07:00
Russell Spitzer c3eaee7765 [SPARK-25003][PYSPARK] Use SessionExtensions in Pyspark
Master

## What changes were proposed in this pull request?

Previously Pyspark used the private constructor for SparkSession when
building that object. This resulted in a SparkSession without checking
the sql.extensions parameter for additional session extensions. To fix
this we instead use the Session.builder() path as SparkR uses, this
loads the extensions and allows their use in PySpark.

## How was this patch tested?

An integration test was added which mimics the Scala test for the same feature.

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

Closes #21990 from RussellSpitzer/SPARK-25003-master.

Authored-by: Russell Spitzer <Russell.Spitzer@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-18 12:29:09 +08:00
Gengliang Wang 24f5bbd770 [SPARK-25735][CORE][MINOR] Improve start-thriftserver.sh: print clean usage and exit with code 1
## What changes were proposed in this pull request?
Currently if we run
```
sh start-thriftserver.sh -h
```

we get
```
...
Thrift server options:
2018-10-15 21:45:39 INFO HiveThriftServer2:54 - Starting SparkContext
2018-10-15 21:45:40 INFO SparkContext:54 - Running Spark version 2.3.2
2018-10-15 21:45:40 WARN NativeCodeLoader:62 - Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
2018-10-15 21:45:40 ERROR SparkContext:91 - Error initializing SparkContext.
org.apache.spark.SparkException: A master URL must be set in your configuration
at org.apache.spark.SparkContext.<init>(SparkContext.scala:367)
at org.apache.spark.SparkContext$.getOrCreate(SparkContext.scala:2493)
at org.apache.spark.sql.SparkSession$Builder$$anonfun$7.apply(SparkSession.scala:934)
at org.apache.spark.sql.SparkSession$Builder$$anonfun$7.apply(SparkSession.scala:925)
at scala.Option.getOrElse(Option.scala:121)
at org.apache.spark.sql.SparkSession$Builder.getOrCreate(SparkSession.scala:925)
at org.apache.spark.sql.hive.thriftserver.SparkSQLEnv$.init(SparkSQLEnv.scala:48)
at org.apache.spark.sql.hive.thriftserver.HiveThriftServer2$.main(HiveThriftServer2.scala:79)
at org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.main(HiveThriftServer2.scala)
2018-10-15 21:45:40 ERROR Utils:91 - Uncaught exception in thread main
```

After fix, the usage output is clean:
```
...
Thrift server options:
--hiveconf <property=value> Use value for given property
```

Also exit with code 1, to follow other scripts(this is the behavior of parsing option `-h` for other linux commands as well).

## How was this patch tested?

Manual test.

Closes #22727 from gengliangwang/stsUsage.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-17 09:56:17 -05:00
Vladimir Kuriatkov e5b8136f47 [SPARK-21402][SQL] Fix java array of structs deserialization
When deserializing values of ArrayType with struct elements in java beans, fields of structs get mixed up.
I suggest using struct data types retrieved from resolved input data instead of inferring them from java beans.

## What changes were proposed in this pull request?

MapObjects expression is used to map array elements to java beans. Struct type of elements is inferred from java bean structure and ends up with mixed up field order.
I used UnresolvedMapObjects instead of MapObjects, which allows to provide element type for MapObjects during analysis based on the resolved input data, not on the java bean.

## How was this patch tested?

Added a test case.
Built complete project on travis.

michalsenkyr cloud-fan marmbrus liancheng

Closes #22708 from vofque/SPARK-21402.

Lead-authored-by: Vladimir Kuriatkov <vofque@gmail.com>
Co-authored-by: Vladimir Kuriatkov <Vladimir_Kuriatkov@epam.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-17 22:13:05 +08:00
Wenchen Fan 9690eba16e [SPARK-25680][SQL] SQL execution listener shouldn't happen on execution thread
## What changes were proposed in this pull request?

The SQL execution listener framework was created from scratch(see https://github.com/apache/spark/pull/9078). It didn't leverage what we already have in the spark listener framework, and one major problem is, the listener runs on the spark execution thread, which means a bad listener can block spark's query processing.

This PR re-implements the SQL execution listener framework. Now `ExecutionListenerManager` is just a normal spark listener, which watches the `SparkListenerSQLExecutionEnd` events and post events to the
user-provided SQL execution listeners.

## How was this patch tested?

existing tests.

Closes #22674 from cloud-fan/listener.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-17 16:06:07 +08:00
彭灿00244106 e9332f600e [SQL][CATALYST][MINOR] update some error comments
## What changes were proposed in this pull request?

this PR correct some comment error:
1. change from "as low a possible" to "as low as possible" in RewriteDistinctAggregates.scala
2. delete redundant word “with” in HiveTableScanExec’s  doExecute()  method

## How was this patch tested?

Existing unit tests.

Closes #22694 from CarolinePeng/update_comment.

Authored-by: 彭灿00244106 <00244106@zte.intra>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-17 12:45:13 +08:00
Takeshi Yamamuro a9f685bb70 [SPARK-25734][SQL] Literal should have a value corresponding to dataType
## What changes were proposed in this pull request?
`Literal.value` should have a value a value corresponding to `dataType`. This pr added code to verify it and fixed the existing tests to do so.

## How was this patch tested?
Modified the existing tests.

Closes #22724 from maropu/SPARK-25734.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-17 11:02:39 +08:00
Maxim Gekk e9af9460bc [SPARK-25393][SQL] Adding new function from_csv()
## What changes were proposed in this pull request?

The PR adds new function `from_csv()` similar to `from_json()` to parse columns with CSV strings. I added the following methods:
```Scala
def from_csv(e: Column, schema: StructType, options: Map[String, String]): Column
```
and this signature to call it from Python, R and Java:
```Scala
def from_csv(e: Column, schema: String, options: java.util.Map[String, String]): Column
```

## How was this patch tested?

Added new test suites `CsvExpressionsSuite`, `CsvFunctionsSuite` and sql tests.

Closes #22379 from MaxGekk/from_csv.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Co-authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-17 09:32:05 +08:00
Dongjoon Hyun 2c664edc06 [SPARK-25579][SQL] Use quoted attribute names if needed in pushed ORC predicates
## What changes were proposed in this pull request?

This PR aims to fix an ORC performance regression at Spark 2.4.0 RCs from Spark 2.3.2. Currently, for column names with `.`, the pushed predicates are ignored.

**Test Data**
```scala
scala> val df = spark.range(Int.MaxValue).sample(0.2).toDF("col.with.dot")
scala> df.write.mode("overwrite").orc("/tmp/orc")
```

**Spark 2.3.2**
```scala
scala> spark.sql("set spark.sql.orc.impl=native")
scala> spark.sql("set spark.sql.orc.filterPushdown=true")
scala> spark.time(spark.read.orc("/tmp/orc").where("`col.with.dot` < 10").show)
+------------+
|col.with.dot|
+------------+
|           5|
|           7|
|           8|
+------------+

Time taken: 1542 ms

scala> spark.time(spark.read.orc("/tmp/orc").where("`col.with.dot` < 10").show)
+------------+
|col.with.dot|
+------------+
|           5|
|           7|
|           8|
+------------+

Time taken: 152 ms
```

**Spark 2.4.0 RC3**
```scala
scala> spark.time(spark.read.orc("/tmp/orc").where("`col.with.dot` < 10").show)
+------------+
|col.with.dot|
+------------+
|           5|
|           7|
|           8|
+------------+

Time taken: 4074 ms

scala> spark.time(spark.read.orc("/tmp/orc").where("`col.with.dot` < 10").show)
+------------+
|col.with.dot|
+------------+
|           5|
|           7|
|           8|
+------------+

Time taken: 1771 ms
```

## How was this patch tested?

Pass the Jenkins with a newly added test case.

Closes #22597 from dongjoon-hyun/SPARK-25579.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-16 20:30:23 +08:00
Wenchen Fan e028fd3aed [SPARK-25736][SQL][TEST] add tests to verify the behavior of multi-column count
## What changes were proposed in this pull request?

AFAIK multi-column count is not widely supported by the mainstream databases(postgres doesn't support), and the SQL standard doesn't define it clearly, as near as I can tell.

Since Spark supports it, we should clearly document the current behavior and add tests to verify it.

## How was this patch tested?

N/A

Closes #22728 from cloud-fan/doc.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-16 15:13:01 +08:00
Yuming Wang 5c7f6b6636 [SPARK-25629][TEST] Reduce ParquetFilterSuite: filter pushdown test time costs in Jenkins
## What changes were proposed in this pull request?

Only test these 4 cases is enough:
be2238fb50/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala (L269-L279)

## How was this patch tested?

Manual tests on my local machine.
before:
```
- filter pushdown - decimal (13 seconds, 683 milliseconds)
```
after:
```
- filter pushdown - decimal (9 seconds, 713 milliseconds)
```

Closes #22636 from wangyum/SPARK-25629.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-16 12:30:02 +08:00
Imran Rashid fdaa99897a [SPARK-25738][SQL] Fix LOAD DATA INPATH for hdfs port
## What changes were proposed in this pull request?

LOAD DATA INPATH didn't work if the defaultFS included a port for hdfs.
Handling this just requires a small change to use the correct URI
constructor.

## How was this patch tested?

Added a unit test, ran all tests via jenkins

Closes #22733 from squito/SPARK-25738.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-10-15 18:34:30 -07:00
gatorsmile 4cee191c04 [SPARK-25674][FOLLOW-UP] Update the stats for each ColumnarBatch
## What changes were proposed in this pull request?
This PR is a follow-up of https://github.com/apache/spark/pull/22594 . This alternative can avoid the unneeded computation in the hot code path.

- For row-based scan, we keep the original way.
- For the columnar scan, we just need to update the stats after each batch.

## How was this patch tested?
N/A

Closes #22731 from gatorsmile/udpateStatsFileScanRDD.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-16 08:58:29 +08:00
SongYadong 0820484ba1 [SPARK-25716][SQL][MINOR] remove unnecessary collection operation in valid constraints generation
## What changes were proposed in this pull request?

Project logical operator generates valid constraints using two opposite operations. It substracts child constraints from all constraints, than union child constraints again. I think it may be not necessary.
Aggregate operator has the same problem with Project.

This PR try to remove these two opposite collection operations.

## How was this patch tested?

Related unit tests:
ProjectEstimationSuite
CollapseProjectSuite
PushProjectThroughUnionSuite
UnsafeProjectionBenchmark
GeneratedProjectionSuite
CodeGeneratorWithInterpretedFallbackSuite
TakeOrderedAndProjectSuite
GenerateUnsafeProjectionSuite
BucketedRandomProjectionLSHSuite
RemoveRedundantAliasAndProjectSuite
AggregateBenchmark
AggregateOptimizeSuite
AggregateEstimationSuite
DecimalAggregatesSuite
DateFrameAggregateSuite
ObjectHashAggregateSuite
TwoLevelAggregateHashMapSuite
ObjectHashAggregateExecBenchmark
SingleLevelAggregateHaspMapSuite
TypedImperativeAggregateSuite
RewriteDistinctAggregatesSuite
HashAggregationQuerySuite
HashAggregationQueryWithControlledFallbackSuite
TypedImperativeAggregateSuite
TwoLevelAggregateHashMapWithVectorizedMapSuite

Closes #22706 from SongYadong/generate_constraints.

Authored-by: SongYadong <song.yadong1@zte.com.cn>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-10-15 15:45:40 -07:00
Marco Gaido 56247c1d17 [SPARK-25727][FOLLOWUP] Move outputOrdering to case class field for InMemoryRelation
## What changes were proposed in this pull request?

The PR addresses [the comment](https://github.com/apache/spark/pull/22715#discussion_r225024084) in the previous one. `outputOrdering` becomes a field of `InMemoryRelation`.

## How was this patch tested?

existing UTs

Closes #22726 from mgaido91/SPARK-25727_followup.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-10-15 10:12:45 -07:00
gatorsmile 6c3f2c6a6a
[SPARK-25727][SQL] Add outputOrdering to otherCopyArgs in InMemoryRelation
## What changes were proposed in this pull request?
Add `outputOrdering ` to `otherCopyArgs` in InMemoryRelation so that this field will be copied when we doing the tree transformation.

```
    val data = Seq(100).toDF("count").cache()
    data.queryExecution.optimizedPlan.toJSON
```

The above code can generate the following error:

```
assertion failed: InMemoryRelation fields: output, cacheBuilder, statsOfPlanToCache, outputOrdering, values: List(count#178), CachedRDDBuilder(true,10000,StorageLevel(disk, memory, deserialized, 1 replicas),*(1) Project [value#176 AS count#178]
+- LocalTableScan [value#176]
,None), Statistics(sizeInBytes=12.0 B, hints=none)
java.lang.AssertionError: assertion failed: InMemoryRelation fields: output, cacheBuilder, statsOfPlanToCache, outputOrdering, values: List(count#178), CachedRDDBuilder(true,10000,StorageLevel(disk, memory, deserialized, 1 replicas),*(1) Project [value#176 AS count#178]
+- LocalTableScan [value#176]
,None), Statistics(sizeInBytes=12.0 B, hints=none)
	at scala.Predef$.assert(Predef.scala:170)
	at org.apache.spark.sql.catalyst.trees.TreeNode.jsonFields(TreeNode.scala:611)
	at org.apache.spark.sql.catalyst.trees.TreeNode.org$apache$spark$sql$catalyst$trees$TreeNode$$collectJsonValue$1(TreeNode.scala:599)
	at org.apache.spark.sql.catalyst.trees.TreeNode.jsonValue(TreeNode.scala:604)
	at org.apache.spark.sql.catalyst.trees.TreeNode.toJSON(TreeNode.scala:590)
```

## How was this patch tested?

Added a test

Closes #22715 from gatorsmile/copyArgs1.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-13 22:10:17 -07:00
Dongjoon Hyun 6bbceb9fef
[SPARK-25726][SQL][TEST] Fix flaky test in SaveIntoDataSourceCommandSuite
## What changes were proposed in this pull request?

[SPARK-22479](https://github.com/apache/spark/pull/19708/files#diff-5c22ac5160d3c9d81225c5dd86265d27R31) adds a test case which sometimes fails because the used password string `123` matches `41230802`. This PR aims to fix the flakiness.

- https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97343/consoleFull

```scala
SaveIntoDataSourceCommandSuite:
- simpleString is redacted *** FAILED ***
"SaveIntoDataSourceCommand .org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider41230802, Map(password -> *********(redacted), url -> *********(redacted), driver -> mydriver), ErrorIfExists
+- Range (0, 1, step=1, splits=Some(2))
" contained "123" (SaveIntoDataSourceCommandSuite.scala:42)
```

## How was this patch tested?

Pass the Jenkins with the updated test case

Closes #22716 from dongjoon-hyun/SPARK-25726.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-13 18:01:28 -07:00
Wenchen Fan b73f76beb3 [SPARK-25714][SQL][FOLLOWUP] improve the comment inside BooleanSimplification rule
## What changes were proposed in this pull request?

improve the code comment added in https://github.com/apache/spark/pull/22702/files

## How was this patch tested?

N/A

Closes #22711 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-10-13 16:43:10 -07:00
gatorsmile 8812746d4f [MINOR] Fix code comment in BooleanSimplification. 2018-10-12 23:01:06 -07:00
Wenchen Fan 34f229bc21 [SPARK-25710][SQL] range should report metrics correctly
## What changes were proposed in this pull request?

Currently `Range` reports metrics in batch granularity. This is acceptable, but it's better if we can make it row granularity without performance penalty.

Before this PR,  the metrics are updated when preparing the batch, which is before we actually consume data. In this PR, the metrics are updated after the data are consumed. There are 2 different cases:
1. The data processing loop has a stop check. The metrics are updated when we need to stop.
2. no stop check. The metrics are updated after the loop.

## How was this patch tested?

existing tests and a new benchmark

Closes #22698 from cloud-fan/range.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-13 13:55:28 +08:00
gatorsmile c9ba59d38e [SPARK-25714] Fix Null Handling in the Optimizer rule BooleanSimplification
## What changes were proposed in this pull request?
```Scala
    val df1 = Seq(("abc", 1), (null, 3)).toDF("col1", "col2")
    df1.write.mode(SaveMode.Overwrite).parquet("/tmp/test1")
    val df2 = spark.read.parquet("/tmp/test1")
    df2.filter("col1 = 'abc' OR (col1 != 'abc' AND col2 == 3)").show()
```

Before the PR, it returns both rows. After the fix, it returns `Row ("abc", 1))`. This is to fix the bug in NULL handling in BooleanSimplification. This is a bug introduced in Spark 1.6 release.

## How was this patch tested?
Added test cases

Closes #22702 from gatorsmile/fixBooleanSimplify2.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-10-12 21:02:38 -07:00
Yuming Wang e965fb55ac
[SPARK-25664][SQL][TEST] Refactor JoinBenchmark to use main method
## What changes were proposed in this pull request?

Refactor `JoinBenchmark` to use main method.
1. use `spark-submit`:
```console
bin/spark-submit --class  org.apache.spark.sql.execution.benchmark.JoinBenchmark --jars ./core/target/spark-core_2.11-3.0.0-SNAPSHOT-tests.jar ./sql/catalyst/target/spark-sql_2.11-3.0.0-SNAPSHOT-tests.jar
```

2. Generate benchmark result:
```console
SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.benchmark.JoinBenchmark"
```

## How was this patch tested?

manual tests

Closes #22661 from wangyum/SPARK-25664.

Lead-authored-by: Yuming Wang <yumwang@ebay.com>
Co-authored-by: Yuming Wang <wgyumg@gmail.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-12 16:08:12 -07:00
Mathieu St-Louis 4e141a4160 [STREAMING][DOC] Fix typo & formatting for JavaDoc
## What changes were proposed in this pull request?
- Fixed typo for function outputMode
      - OutputMode.Complete(), changed `these is some updates` to `there are some updates`
- Replaced hyphenized list by HTML unordered list tags in comments to fix the Javadoc documentation.

Current render from most recent [Spark API Docs](https://spark.apache.org/docs/2.3.1/api/java/org/apache/spark/sql/streaming/DataStreamWriter.html):

#### outputMode(OutputMode) - List formatted as a prose.

![image](https://user-images.githubusercontent.com/2295469/46250648-11086700-c3f4-11e8-8a5a-d88b079c165d.png)

#### outputMode(String) - List formatted as a prose.
![image](https://user-images.githubusercontent.com/2295469/46250651-24b3cd80-c3f4-11e8-9dac-ae37599afbce.png)

#### partitionBy(String*) - List formatted as a prose.
![image](https://user-images.githubusercontent.com/2295469/46250655-36957080-c3f4-11e8-990b-47bd612d3c51.png)

## How was this patch tested?
This PR contains a document patch ergo no functional testing is required.

Closes #22593 from niofire/fix-typo-datastreamwriter.

Authored-by: Mathieu St-Louis <mastloui@microsoft.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-12 14:09:10 -05:00
Maxim Gekk c7eadb5e66 [SPARK-25660][SQL] Fix for the backward slash as CSV fields delimiter
## What changes were proposed in this pull request?

The PR addresses the exception raised on accessing chars out of delimiter string. In particular, the backward slash `\` as the CSV fields delimiter causes the following exception on reading `abc\1`:
```Scala
String index out of range: 1
java.lang.StringIndexOutOfBoundsException: String index out of range: 1
	at java.lang.String.charAt(String.java:658)
```
because `str.charAt(1)` tries to access a char out of `str` in `CSVUtils.toChar`

## How was this patch tested?

Added tests for empty string and string containing the backward slash to `CSVUtilsSuite`. Besides of that I added an end-to-end test to check how the backward slash is handled in reading CSV string with it.

Closes #22654 from MaxGekk/csv-slash-delim.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-10-12 12:04:00 -07:00
Maxim Gekk d47a25f681 [SPARK-25670][TEST] Reduce number of tested timezones in JsonExpressionsSuite
## What changes were proposed in this pull request?

After the changes, total execution time of `JsonExpressionsSuite.scala` dropped from 12.5 seconds to 3 seconds.

Closes #22657 from MaxGekk/json-timezone-test.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-12 12:38:45 -05:00
Shahid 3494b12281 [SPARK-25566][SPARK-25567][WEBUI][SQL] Support pagination for SQL tab to avoid OOM
## What changes were proposed in this pull request?
Currently SQL tab in the WEBUI doesn't support pagination. Because of that following issues are happening.
1) For large number of executions, SQL page is throwing OOM exception (around 40,000)
2) For large number of executions, loading SQL page is taking time.
3) Difficult to analyse the execution table for large number of execution.
[Note: spark.sql.ui.retainedExecutions = 50000]

All the tabs, Jobs, Stages etc. supports pagination. So, to make it consistent with other tabs
SQL tab also should support pagination.

I have followed the similar flow of the pagination code in the Jobs and Stages page for SQL page.
Also, this patch doesn't make any behavior change for the SQL tab except the pagination support.

## How was this patch tested?
bin/spark-shell --conf spark.sql.ui.retainedExecutions=50000
Run 50,000 sql queries.
**Before this PR**
![screenshot from 2018-10-05 23-48-27](https://user-images.githubusercontent.com/23054875/46552750-4ed82480-c8f9-11e8-8b05-d60bedddd1b8.png)

![screenshot from 2018-10-05 22-58-11](https://user-images.githubusercontent.com/23054875/46550276-33b5e680-c8f2-11e8-9e32-9ae9c5b181e0.png)

**After this PR**

Loading of the page is faster, and OOM issue doesn't happen.
![screenshot from 2018-10-05 23-50-32](https://user-images.githubusercontent.com/23054875/46552814-8050f000-c8f9-11e8-96e9-42502d2cfaea.png)

Closes #22645 from shahidki31/SPARK-25566.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-12 12:36:35 -05:00
Wenchen Fan 78e133141c [SPARK-25708][SQL] HAVING without GROUP BY means global aggregate
## What changes were proposed in this pull request?

According to the SQL standard, when a query contains `HAVING`, it indicates an aggregate operator. For more details please refer to https://blog.jooq.org/2014/12/04/do-you-really-understand-sqls-group-by-and-having-clauses/

However, in Spark SQL parser, we treat HAVING as a normal filter when there is no GROUP BY, which breaks SQL semantic and lead to wrong result. This PR fixes the parser.

## How was this patch tested?

new test

Closes #22696 from cloud-fan/having.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-10-12 00:24:06 -07:00
maryannxue 3685130481 [SPARK-25690][SQL] Analyzer rule HandleNullInputsForUDF does not stabilize and can be applied infinitely
## What changes were proposed in this pull request?

The HandleNullInputsForUDF rule can generate new If node infinitely, thus causing problems like match of SQL cache missed.
This was fixed in SPARK-24891 and was then broken by SPARK-25044.
The unit test in `AnalysisSuite` added in SPARK-24891 should have failed but didn't because it wasn't properly updated after the `ScalaUDF` constructor signature change. So this PR also updates the test accordingly based on the new `ScalaUDF` constructor.

## How was this patch tested?

Updated the original UT. This should be justified as the original UT became invalid after SPARK-25044.

Closes #22701 from maryannxue/spark-25690.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-10-11 20:45:08 -07:00
Kazuaki Ishizaki c9d7d83ed5 [SPARK-25388][TEST][SQL] Detect incorrect nullable of DataType in the result
## What changes were proposed in this pull request?

This PR can correctly cause assertion failure when incorrect nullable of DataType in the result is generated by a target function to be tested.

Let us think the following example. In the future, a developer would write incorrect code that returns unexpected result. We have to correctly cause fail in this test since `valueContainsNull=false` while `expr` includes `null`. However, without this PR, this test passes. This PR can correctly cause fail.

```
test("test TARGETFUNCTON") {
  val expr = TARGETMAPFUNCTON()
  // expr = UnsafeMap(3 -> 6, 7 -> null)
  // expr.dataType = (IntegerType, IntegerType, false)

  expected = Map(3 -> 6, 7 -> null)
  checkEvaluation(expr, expected)
```

In [`checkEvaluationWithUnsafeProjection`](https://github.com/apache/spark/blob/master/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala#L208-L235), the results are compared using `UnsafeRow`. When the given `expected` is [converted](https://github.com/apache/spark/blob/master/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala#L226-L227)) to `UnsafeRow` using the `DataType` of `expr`.
```
val expectedRow = UnsafeProjection.create(Array(expression.dataType, expression.dataType)).apply(lit)
```

In summary, `expr` is `[0,1800000038,5000000038,18,2,0,700000003,2,0,6,18,2,0,700000003,2,0,6]` with and w/o this PR. `expected` is converted to

* w/o  this PR, `[0,1800000038,5000000038,18,2,0,700000003,2,0,6,18,2,0,700000003,2,0,6]`
* with this PR, `[0,1800000038,5000000038,18,2,0,700000003,2,2,6,18,2,0,700000003,2,2,6]`

As a result, w/o this PR, the test unexpectedly passes.

This is because, w/o this PR, based on given `dataType`, generated code of projection for `expected` avoids to set nullbit.
```
                    // tmpInput_2 is expected
/* 155 */           for (int index_1 = 0; index_1 < numElements_1; index_1++) {
/* 156 */             mutableStateArray_1[1].write(index_1, tmpInput_2.getInt(index_1));
/* 157 */           }
```

With this PR, generated code of projection for `expected` always checks whether nullbit should be set by `isNullAt`
```
                    // tmpInput_2 is expected
/* 161 */           for (int index_1 = 0; index_1 < numElements_1; index_1++) {
/* 162 */
/* 163 */             if (tmpInput_2.isNullAt(index_1)) {
/* 164 */               mutableStateArray_1[1].setNull4Bytes(index_1);
/* 165 */             } else {
/* 166 */               mutableStateArray_1[1].write(index_1, tmpInput_2.getInt(index_1));
/* 167 */             }
/* 168 */
/* 169 */           }
```

## How was this patch tested?

Existing UTs

Closes #22375 from kiszk/SPARK-25388.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-12 11:14:35 +08:00
hyukjinkwon 39872af882 [SPARK-25684][SQL] Organize header related codes in CSV datasource
## What changes were proposed in this pull request?

1. Move `CSVDataSource.makeSafeHeader` to `CSVUtils.makeSafeHeader` (as is).

    - Historically and at the first place of refactoring (which I did), I intended to put all CSV specific handling (like options), filtering, extracting header, etc.

    - See `JsonDataSource`. Now `CSVDataSource` is quite consistent with `JsonDataSource`. Since CSV's code path is quite complicated, we might better match them as possible as we can.

2. Create `CSVHeaderChecker` and put `enforceSchema` logics into that.

    - The checking header and column pruning stuff were added (per https://github.com/apache/spark/pull/20894 and https://github.com/apache/spark/pull/21296) but some of codes such as https://github.com/apache/spark/pull/22123 are duplicated

    - Also, checking header code is basically here and there. We better put them in a single place, which was quite error-prone. See (https://github.com/apache/spark/pull/22656).

3. Move `CSVDataSource.checkHeaderColumnNames` to `CSVHeaderChecker.checkHeaderColumnNames` (as is).

    - Similar reasons above with 1.

## How was this patch tested?

Existing tests should cover this.

Closes #22676 from HyukjinKwon/refactoring-csv.

Authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-12 09:16:41 +08:00
liuxian 69f5e9cce1 [SPARK-25674][SQL] If the records are incremented by more than 1 at a time,the number of bytes might rarely ever get updated
## What changes were proposed in this pull request?
If the records are incremented by more than 1 at a time,the number of bytes might rarely ever get updated,because it might skip over the count that is an exact multiple of UPDATE_INPUT_METRICS_INTERVAL_RECORDS.

This PR just checks whether the increment causes the value to exceed a higher multiple of UPDATE_INPUT_METRICS_INTERVAL_RECORDS.

## How was this patch tested?
existed unit tests

Closes #22594 from 10110346/inputMetrics.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-11 14:24:15 -07:00
caoxuewen 65f75db611 [MINOR][SQL] remove Redundant semicolons
## What changes were proposed in this pull request?

remove Redundant semicolons in SortMergeJoinExec, thanks.

## How was this patch tested?

N/A

Closes #22695 from heary-cao/RedundantSemicolons.

Authored-by: caoxuewen <cao.xuewen@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-11 14:03:41 -07:00
Peter Toth 8115e6b269
[SPARK-25662][SQL][TEST] Refactor DataSourceReadBenchmark to use main method
## What changes were proposed in this pull request?

1. Refactor DataSourceReadBenchmark

## How was this patch tested?

Manually tested and regenerated results.
```
SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.benchmark.DataSourceReadBenchmark"
```

Closes #22664 from peter-toth/SPARK-25662.

Lead-authored-by: Peter Toth <peter.toth@gmail.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2018-10-11 20:27:07 +00:00
hyukjinkwon 83e19d5b80
[SPARK-25700][SQL] Creates ReadSupport in only Append Mode in Data Source V2 write path
## What changes were proposed in this pull request?

This PR proposes to avoid to make a readsupport and read schema when it writes in other save modes.

5fef6e3513 happened to create a readsupport in write path, which ended up with reading schema from readsupport at write path.

This breaks `spark.range(1).format("source").write.save("non-existent-path")` case since there's no way to read the schema from "non-existent-path".

See also https://github.com/apache/spark/pull/22009#discussion_r223982672
See also https://github.com/apache/spark/pull/22697
See also http://apache-spark-developers-list.1001551.n3.nabble.com/Possible-bug-in-DatasourceV2-td25343.html

## How was this patch tested?

Unit test and manual tests.

Closes #22688 from HyukjinKwon/append-revert-2.

Authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-11 09:35:49 -07:00
Sean Owen 80813e1980 [SPARK-25016][BUILD][CORE] Remove support for Hadoop 2.6
## What changes were proposed in this pull request?

Remove Hadoop 2.6 references and make 2.7 the default.
Obviously, this is for master/3.0.0 only.
After this we can also get rid of the separate test jobs for Hadoop 2.6.

## How was this patch tested?

Existing tests

Closes #22615 from srowen/SPARK-25016.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-10 12:07:53 -07:00
Gengliang Wang 6df2345794
[SPARK-25699][SQL] Partially push down conjunctive predicated in ORC
## What changes were proposed in this pull request?

Inspired by https://github.com/apache/spark/pull/22574 .
We can partially push down top level conjunctive predicates to Orc.
This PR improves Orc predicate push down in both SQL and Hive module.

## How was this patch tested?

New unit test.

Closes #22684 from gengliangwang/pushOrcFilters.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2018-10-10 18:18:56 +00:00
Dilip Biswal 3528c08beb [SPARK-25611][SPARK-25612][SQL][TESTS] Improve test run time of CompressionCodecSuite
## What changes were proposed in this pull request?
Reduced the combination of codecs from 9 to 3 to improve the test runtime.

## How was this patch tested?
This is a test fix.

Closes #22641 from dilipbiswal/SPARK-25611.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-10 08:51:16 -07:00
Sean Owen eaafcd8a22 [SPARK-25605][TESTS] Alternate take. Run cast string to timestamp tests for a subset of timezones
## What changes were proposed in this pull request?

Try testing timezones in parallel instead in CastSuite, instead of random sampling.
See also #22631

## How was this patch tested?

Existing test.

Closes #22672 from srowen/SPARK-25605.2.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-10 08:25:12 -07:00
gatorsmile faf73dcd33
[SPARK-25559][FOLLOW-UP] Add comments for partial pushdown of conjuncts in Parquet
## What changes were proposed in this pull request?
This is a follow up of https://github.com/apache/spark/pull/22574. Renamed the parameter and added comments.

## How was this patch tested?
N/A

Closes #22679 from gatorsmile/followupSPARK-25559.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2018-10-09 21:10:33 +00:00
Wenchen Fan e3133f4abf [SPARK-25497][SQL] Limit operation within whole stage codegen should not consume all the inputs
## What changes were proposed in this pull request?

This PR is inspired by https://github.com/apache/spark/pull/22524, but proposes a safer fix.

The current limit whole stage codegen has 2 problems:
1. It's only applied to `InputAdapter`, many leaf nodes can't stop earlier w.r.t. limit.
2. It needs to override a method, which will break if we have more than one limit in the whole-stage.

The first problem is easy to fix, just figure out which nodes can stop earlier w.r.t. limit, and update them. This PR updates `RangeExec`, `ColumnarBatchScan`, `SortExec`, `HashAggregateExec`.

The second problem is hard to fix. This PR proposes to propagate the limit counter variable name upstream, so that the upstream leaf/blocking nodes can check the limit counter and quit the loop earlier.

For better performance, the implementation here follows `CodegenSupport.needStopCheck`, so that we only codegen the check only if there is limit in the query. For columnar node like range, we check the limit counter per-batch instead of per-row, to make the inner loop tight and fast.

Why this is safer?
1. the leaf/blocking nodes don't have to check the limit counter and stop earlier. It's only for performance. (this is same as before)
2. The blocking operators can stop propagating the limit counter name, because the counter of limit after blocking operators will never increase, before blocking operators consume all the data from upstream operators. So the upstream operators don't care about limit after blocking operators. This is also for performance only, it's OK if we forget to do it for some new blocking operators.

## How was this patch tested?

a new test

Closes #22630 from cloud-fan/limit.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
2018-10-09 16:46:23 +09:00