Commit graph

7188 commits

Author SHA1 Message Date
Xingbo Jiang a7ab7f2348 [SPARK-25845][SQL] Fix MatchError for calendar interval type in range frame left boundary
## What changes were proposed in this pull request?

WindowSpecDefinition checks start < last, but CalendarIntervalType is not comparable, so it would throw the following exception at runtime:

```
 scala.MatchError: CalendarIntervalType (of class org.apache.spark.sql.types.CalendarIntervalType$)      at
 org.apache.spark.sql.catalyst.util.TypeUtils$.getInterpretedOrdering(TypeUtils.scala:58) at
 org.apache.spark.sql.catalyst.expressions.BinaryComparison.ordering$lzycompute(predicates.scala:592) at
 org.apache.spark.sql.catalyst.expressions.BinaryComparison.ordering(predicates.scala:592) at
 org.apache.spark.sql.catalyst.expressions.GreaterThan.nullSafeEval(predicates.scala:797) at org.apache.spark.sql.catalyst.expressions.BinaryExpression.eval(Expression.scala:496) at org.apache.spark.sql.catalyst.expressions.SpecifiedWindowFrame.isGreaterThan(windowExpressions.scala:245) at
 org.apache.spark.sql.catalyst.expressions.SpecifiedWindowFrame.checkInputDataTypes(windowExpressions.scala:216) at
 org.apache.spark.sql.catalyst.expressions.Expression.resolved$lzycompute(Expression.scala:171) at
 org.apache.spark.sql.catalyst.expressions.Expression.resolved(Expression.scala:171) at
 org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$childrenResolved$1.apply(Expression.scala:183) at
 org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$childrenResolved$1.apply(Expression.scala:183) at
 scala.collection.IndexedSeqOptimized$class.prefixLengthImpl(IndexedSeqOptimized.scala:38) at scala.collection.IndexedSeqOptimized$class.forall(IndexedSeqOptimized.scala:43) at scala.collection.mutable.ArrayBuffer.forall(ArrayBuffer.scala:48) at
 org.apache.spark.sql.catalyst.expressions.Expression.childrenResolved(Expression.scala:183) at
 org.apache.spark.sql.catalyst.expressions.WindowSpecDefinition.resolved$lzycompute(windowExpressions.scala:48) at
 org.apache.spark.sql.catalyst.expressions.WindowSpecDefinition.resolved(windowExpressions.scala:48) at
 org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$childrenResolved$1.apply(Expression.scala:183) at
 org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$childrenResolved$1.apply(Expression.scala:183) at
 scala.collection.LinearSeqOptimized$class.forall(LinearSeqOptimized.scala:83)
```

We fix the issue by only perform the check on boundary expressions that are AtomicType.

## How was this patch tested?

Add new test case in `DataFrameWindowFramesSuite`

Closes #22853 from jiangxb1987/windowBoundary.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2018-10-28 18:15:47 +08:00
Wenchen Fan ff4bb836aa [SPARK-25817][SQL] Dataset encoder should support combination of map and product type
## What changes were proposed in this pull request?

After https://github.com/apache/spark/pull/22745 , Dataset encoder supports the combination of java bean and map type. This PR is to fix the Scala side.

The reason why it didn't work before is, `CatalystToExternalMap` tries to get the data type of the input map expression, while it can be unresolved and its data type is known. To fix it, we can follow `UnresolvedMapObjects`, to create a `UnresolvedCatalystToExternalMap`, and only create `CatalystToExternalMap` when the input map expression is resolved and the data type is known.

## How was this patch tested?

enable a old test case

Closes #22812 from cloud-fan/map.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-28 13:33:26 +08:00
Dilip Biswal e545811346 [SPARK-19851][SQL] Add support for EVERY and ANY (SOME) aggregates
## What changes were proposed in this pull request?

Implements Every, Some, Any aggregates in SQL. These new aggregate expressions are analyzed in normal way and rewritten to equivalent existing aggregate expressions in the optimizer.

Every(x) => Min(x)  where x is boolean.
Some(x) => Max(x) where x is boolean.

Any is a synonym for Some.
SQL
```
explain extended select every(v) from test_agg group by k;
```
Plan :
```
== Parsed Logical Plan ==
'Aggregate ['k], [unresolvedalias('every('v), None)]
+- 'UnresolvedRelation `test_agg`

== Analyzed Logical Plan ==
every(v): boolean
Aggregate [k#0], [every(v#1) AS every(v)#5]
+- SubqueryAlias `test_agg`
   +- Project [k#0, v#1]
      +- SubqueryAlias `test_agg`
         +- LocalRelation [k#0, v#1]

== Optimized Logical Plan ==
Aggregate [k#0], [min(v#1) AS every(v)#5]
+- LocalRelation [k#0, v#1]

== Physical Plan ==
*(2) HashAggregate(keys=[k#0], functions=[min(v#1)], output=[every(v)#5])
+- Exchange hashpartitioning(k#0, 200)
   +- *(1) HashAggregate(keys=[k#0], functions=[partial_min(v#1)], output=[k#0, min#7])
      +- LocalTableScan [k#0, v#1]
Time taken: 0.512 seconds, Fetched 1 row(s)
```

## How was this patch tested?
Added tests in SQLQueryTestSuite, DataframeAggregateSuite

Closes #22809 from dilipbiswal/SPARK-19851-specific-rewrite.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-28 09:38:38 +08:00
laskfla 6f05669e4e [MINOR][DOC] Fix comment error of HiveUtils
## What changes were proposed in this pull request?
Change the version number in comment of `HiveUtils.newClientForExecution` from `13` to `1.2.1` .

## How was this patch tested?

N/A

Closes #22850 from laskfla/HiveUtils-Comment.

Authored-by: laskfla <wwlsax11@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-27 08:09:59 -05:00
Sean Owen ca545f7941 [SPARK-25821][SQL] Remove SQLContext methods deprecated in 1.4
## What changes were proposed in this pull request?

Remove SQLContext methods deprecated in 1.4

## How was this patch tested?

Existing tests.

Closes #22815 from srowen/SPARK-25821.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-26 16:49:48 -05:00
Gengliang Wang d325ffbf3a [SPARK-25851][SQL][MINOR] Fix deprecated API warning in SQLListener
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/21596, Jackson is upgraded to 2.9.6.
There are some deprecated API warnings in SQLListener.
Create a trivial PR to fix them.

```
[warn] SQLListener.scala:92: method uncheckedSimpleType in class TypeFactory is deprecated: see corresponding Javadoc for more information.
[warn] val objectType = typeFactory.uncheckedSimpleType(classOf[Object])
[warn]
[warn] SQLListener.scala:93: method constructSimpleType in class TypeFactory is deprecated: see corresponding Javadoc for more information.
[warn] typeFactory.constructSimpleType(classOf[(_, _)], classOf[(_, _)], Array(objectType, objectType))
[warn]
[warn] SQLListener.scala:97: method uncheckedSimpleType in class TypeFactory is deprecated: see corresponding Javadoc for more information.
[warn] val longType = typeFactory.uncheckedSimpleType(classOf[Long])
[warn]
[warn] SQLListener.scala:98: method constructSimpleType in class TypeFactory is deprecated: see corresponding Javadoc for more information.
[warn] typeFactory.constructSimpleType(classOf[(_, _)], classOf[(_, _)], Array(longType, longType))
```

## How was this patch tested?

Existing unit tests.

Closes #22848 from gengliangwang/fixSQLListenerWarning.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-26 16:45:56 -05:00
hyukjinkwon 33e337c118 [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's input json as literal only
## What changes were proposed in this pull request?

The main purpose of `schema_of_json` is the usage of combination with `from_json` (to make up the leak of schema inference) which takes its schema only as literal; however, currently `schema_of_json` allows JSON input as non-literal expressions (e.g, column).

This was mistakenly allowed - we don't have to take other usages rather then the main purpose into account for now.

This PR makes a followup to only allow literals for `schema_of_json`'s JSON input. We can allow non literal expressions later when it's needed or there are some usecase for it.

## How was this patch tested?

Unit tests were added.

Closes #22775 from HyukjinKwon/SPARK-25447-followup.

Lead-authored-by: hyukjinkwon <gurwls223@apache.org>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-26 22:14:43 +08:00
Reynold Xin 89d748b33c [SPARK-25842][SQL] Deprecate rangeBetween APIs introduced in SPARK-21608
## What changes were proposed in this pull request?
See the detailed information at https://issues.apache.org/jira/browse/SPARK-25841 on why these APIs should be deprecated and redesigned.

This patch also reverts 8acb51f08b which applies to 2.4.

## How was this patch tested?
Only deprecation and doc changes.

Closes #22841 from rxin/SPARK-25842.

Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-26 13:17:24 +08:00
Shixiong Zhu 86d469aeaa [SPARK-25822][PYSPARK] Fix a race condition when releasing a Python worker
## What changes were proposed in this pull request?

There is a race condition when releasing a Python worker. If `ReaderIterator.handleEndOfDataSection` is not running in the task thread, when a task is early terminated (such as `take(N)`), the task completion listener may close the worker but "handleEndOfDataSection" can still put the worker into the worker pool to reuse.

0e07b483d2 is a patch to reproduce this issue.

I also found a user reported this in the mail list: http://mail-archives.apache.org/mod_mbox/spark-user/201610.mbox/%3CCAAUq=H+YLUEpd23nwvq13Ms5hOStkhX3ao4f4zQV6sgO5zM-xAmail.gmail.com%3E

This PR fixes the issue by using `compareAndSet` to make sure we will never return a closed worker to the work pool.

## How was this patch tested?

Jenkins.

Closes #22816 from zsxwing/fix-socket-closed.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2018-10-26 13:53:51 +09:00
Wenchen Fan 72a23a6c43 [SPARK-25772][SQL][FOLLOWUP] remove GetArrayFromMap
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/22745 we introduced the `GetArrayFromMap` expression. Later on I realized this is duplicated as we already have `MapKeys` and `MapValues`.

This PR removes `GetArrayFromMap`

## How was this patch tested?

existing tests

Closes #22825 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-26 10:19:35 +08:00
Peter Toth ccd07b7366
[SPARK-25665][SQL][TEST] Refactor ObjectHashAggregateExecBenchmark to…
## What changes were proposed in this pull request?

Refactor ObjectHashAggregateExecBenchmark to use main method

## How was this patch tested?

Manually tested:
```
bin/spark-submit --class org.apache.spark.sql.execution.benchmark.ObjectHashAggregateExecBenchmark --jars sql/catalyst/target/spark-catalyst_2.11-3.0.0-SNAPSHOT-tests.jar,core/target/spark-core_2.11-3.0.0-SNAPSHOT-tests.jar,sql/hive/target/spark-hive_2.11-3.0.0-SNAPSHOT.jar --packages org.spark-project.hive:hive-exec:1.2.1.spark2 sql/hive/target/spark-hive_2.11-3.0.0-SNAPSHOT-tests.jar
```
Generated results with:
```
SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "hive/test:runMain org.apache.spark.sql.execution.benchmark.ObjectHashAggregateExecBenchmark"
```

Closes #22804 from peter-toth/SPARK-25665.

Lead-authored-by: Peter Toth <peter.toth@gmail.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-25 12:42:31 -07:00
Liang-Chi Hsieh cb5ea201df [SPARK-25746][SQL] Refactoring ExpressionEncoder to get rid of flat flag
## What changes were proposed in this pull request?

This is inspired during implementing #21732. For now `ScalaReflection` needs to consider how `ExpressionEncoder` uses generated serializers and deserializers. And `ExpressionEncoder` has a weird `flat` flag. After discussion with cloud-fan, it seems to be better to refactor `ExpressionEncoder`. It should make SPARK-24762 easier to do.

To summarize the proposed changes:

1. `serializerFor` and `deserializerFor` return expressions for serializing/deserializing an input expression for a given type. They are private and should not be called directly.
2. `serializerForType` and `deserializerForType` returns an expression for serializing/deserializing for an object of type T to/from Spark SQL representation. It assumes the input object/Spark SQL representation is located at ordinal 0 of a row.

So in other words, `serializerForType` and `deserializerForType` return expressions for atomically serializing/deserializing JVM object to/from Spark SQL value.

A serializer returned by `serializerForType` will serialize an object at `row(0)` to a corresponding Spark SQL representation, e.g. primitive type, array, map, struct.

A deserializer returned by `deserializerForType` will deserialize an input field at `row(0)` to an object with given type.

3. The construction of `ExpressionEncoder` takes a pair of serializer and deserializer for type `T`. It uses them to create serializer and deserializer for T <-> row serialization. Now `ExpressionEncoder` dones't need to remember if serializer is flat or not. When we need to construct new `ExpressionEncoder` based on existing ones, we only need to change input location in the atomic serializer and deserializer.

## How was this patch tested?

Existing tests.

Closes #22749 from viirya/SPARK-24762-refactor.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-25 19:27:45 +08:00
adrian555 ddd1b1e8ae [SPARK-24572][SPARKR] "eager execution" for R shell, IDE
## What changes were proposed in this pull request?

Check the `spark.sql.repl.eagerEval.enabled` configuration property in SparkDataFrame `show()` method. If the `SparkSession` has eager execution enabled, the data will be returned to the R client when the data frame is created. So instead of seeing this
```
> df <- createDataFrame(faithful)
> df
SparkDataFrame[eruptions:double, waiting:double]
```
you will see
```
> df <- createDataFrame(faithful)
> df
+---------+-------+
|eruptions|waiting|
+---------+-------+
|      3.6|   79.0|
|      1.8|   54.0|
|    3.333|   74.0|
|    2.283|   62.0|
|    4.533|   85.0|
|    2.883|   55.0|
|      4.7|   88.0|
|      3.6|   85.0|
|     1.95|   51.0|
|     4.35|   85.0|
|    1.833|   54.0|
|    3.917|   84.0|
|      4.2|   78.0|
|     1.75|   47.0|
|      4.7|   83.0|
|    2.167|   52.0|
|     1.75|   62.0|
|      4.8|   84.0|
|      1.6|   52.0|
|     4.25|   79.0|
+---------+-------+
only showing top 20 rows
```

## How was this patch tested?
Manual tests as well as unit tests (one new test case is added).

Author: adrian555 <v2ave10p>

Closes #22455 from adrian555/eager_execution.
2018-10-24 23:42:06 -07:00
Maxim Gekk 4d6704db4d [SPARK-25243][SQL] Use FailureSafeParser in from_json
## What changes were proposed in this pull request?

In the PR, I propose to switch `from_json` on `FailureSafeParser`, and to make the function compatible to `PERMISSIVE` mode by default, and to support the `FAILFAST` mode as well. The `DROPMALFORMED` mode is not supported by `from_json`.

## How was this patch tested?

It was tested by existing `JsonSuite`/`CSVSuite`, `JsonFunctionsSuite` and `JsonExpressionsSuite` as well as new tests for `from_json` which checks different modes.

Closes #22237 from MaxGekk/from_json-failuresafe.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-24 19:09:15 +08:00
Vladimir Kuriatkov 584e767d37 [SPARK-25772][SQL] Fix java map of structs deserialization
This is a follow-up PR for #22708. It considers another case of java beans deserialization: java maps with struct keys/values.

When deserializing values of MapType with struct keys/values 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?

Invocations of "keyArray" and "valueArray" functions are used to extract arrays of keys and values. Struct type of keys or values is also inferred from java bean structure and ends up with mixed up field order.
I created a new UnresolvedInvoke expression as a temporary substitution of Invoke expression while no actual data is available. It allows to provide the resulting data type during analysis based on the resolved input data, not on the java bean (similar to UnresolvedMapObjects).

Key and value arrays are then fed to MapObjects expression which I replaced with UnresolvedMapObjects, just like in case of ArrayType.

Finally I added resolution of UnresolvedInvoke expressions in Analyzer.resolveExpression method as an additional pattern matching case.

## How was this patch tested?

Added a test case.
Built complete project on travis.

viirya kiszk cloud-fan michalsenkyr marmbrus liancheng

Closes #22745 from vofque/SPARK-21402-FOLLOWUP.

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-24 09:29:40 +08:00
Liang-Chi Hsieh 736fc03930 [SPARK-25791][SQL] Datatype of serializers in RowEncoder should be accessible
## What changes were proposed in this pull request?

The serializers of `RowEncoder` use few `If` Catalyst expression which inherits `ComplexTypeMergingExpression` that will check input data types.

It is possible to generate serializers which fail the check and can't to access the data type of serializers. When producing If expression, we should use the same data type at its input expressions.

## How was this patch tested?

Added test.

Closes #22785 from viirya/SPARK-25791.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-23 22:02:14 +08:00
Imran Rashid 78c8bd2e68 [SPARK-25805][SQL][TEST] Fix test for SPARK-25159
The original test would sometimes fail if the listener bus did not keep
up, so just wait till the listener bus is empty.  Tested by adding a
sleep in the listener, which made the test consistently fail without the
fix, but pass consistently after the fix.

Closes #22799 from squito/SPARK-25805.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-23 14:20:29 +08:00
Liang-Chi Hsieh 03e82e3689 [SPARK-25040][SQL] Empty string for non string types should be disallowed
## What changes were proposed in this pull request?

This takes over original PR at #22019. The original proposal is to have null for float and double types. Later a more reasonable proposal is to disallow empty strings. This patch adds logic to throw exception when finding empty strings for non string types.

## How was this patch tested?

Added test.

Closes #22787 from viirya/SPARK-25040.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-23 13:43:53 +08:00
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