Commit graph

22813 commits

Author SHA1 Message Date
Marco Gaido 44a71741d5 [SPARK-25379][SQL] Improve AttributeSet and ColumnPruning performance
## What changes were proposed in this pull request?

This PR contains 3 optimizations:
 1)  it improves significantly the operation `--` on `AttributeSet`. As a benchmark for the `--` operation, the following code has been run
```
test("AttributeSet -- benchmark") {
    val attrSetA = AttributeSet((1 to 100).map { i => AttributeReference(s"c$i", IntegerType)() })
    val attrSetB = AttributeSet(attrSetA.take(80).toSeq)
    val attrSetC = AttributeSet((1 to 100).map { i => AttributeReference(s"c2_$i", IntegerType)() })
    val attrSetD = AttributeSet((attrSetA.take(50) ++ attrSetC.take(50)).toSeq)
    val attrSetE = AttributeSet((attrSetC.take(50) ++ attrSetA.take(50)).toSeq)
    val n_iter = 1000000
    val t0 = System.nanoTime()
    (1 to n_iter) foreach { _ =>
      val r1 = attrSetA -- attrSetB
      val r2 = attrSetA -- attrSetC
      val r3 = attrSetA -- attrSetD
      val r4 = attrSetA -- attrSetE
    }
    val t1 = System.nanoTime()
    val totalTime = t1 - t0
    println(s"Average time: ${totalTime / n_iter} us")
  }
```
The results are:
```
Before PR - Average time: 67674 us (100  %)
After PR -  Average time: 28827 us (42.6 %)
```
2) In `ColumnPruning`, it replaces the occurrences of `(attributeSet1 -- attributeSet2).nonEmpty` with `attributeSet1.subsetOf(attributeSet2)` which is order of magnitudes more efficient (especially where there are many attributes). Running the previous benchmark replacing `--` with `subsetOf` returns:
```
Average time: 67 us (0.1 %)
```

3) Provides a more efficient way of building `AttributeSet`s, which can greatly improve the performance of the methods `references` and `outputSet` of `Expression` and `QueryPlan`. This basically avoids unneeded operations (eg. creating many `AttributeEqual` wrapper classes which could be avoided)

The overall effect of those optimizations has been tested on `ColumnPruning` with the following benchmark:

```
test("ColumnPruning benchmark") {
    val attrSetA = (1 to 100).map { i => AttributeReference(s"c$i", IntegerType)() }
    val attrSetB = attrSetA.take(80)
    val attrSetC = attrSetA.take(20).map(a => Alias(Add(a, Literal(1)), s"${a.name}_1")())

    val input = LocalRelation(attrSetA)
    val query1 = Project(attrSetB, Project(attrSetA, input)).analyze
    val query2 = Project(attrSetC, Project(attrSetA, input)).analyze
    val query3 = Project(attrSetA, Project(attrSetA, input)).analyze
    val nIter = 100000
    val t0 = System.nanoTime()
    (1 to nIter).foreach { _ =>
      ColumnPruning(query1)
      ColumnPruning(query2)
      ColumnPruning(query3)
    }
    val t1 = System.nanoTime()
    val totalTime = t1 - t0
    println(s"Average time: ${totalTime / nIter} us")
}
```

The output of the test is:

```
Before PR - Average time: 733471 us (100  %)
After PR  - Average time: 362455 us (49.4 %)
```

The performance improvement has been evaluated also on the `SQLQueryTestSuite`'s queries:

```
(before) org.apache.spark.sql.catalyst.optimizer.ColumnPruning                                              518413198 / 1377707172                          2756 / 15717
(after)  org.apache.spark.sql.catalyst.optimizer.ColumnPruning                                              415432579 / 1121147950                          2756 / 15717
% Running time                                                                                                  80.1% / 81.3%
```

Also other rules benefit especially from (3), despite the impact is lower, eg:
```
(before) org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences                                  307341442 / 623436806                           2154 / 16480
(after)  org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveReferences                                  290511312 / 560962495                           2154 / 16480
% Running time                                                                                                  94.5% / 90.0%
```

The reason why the impact on the `SQLQueryTestSuite`'s queries is lower compared to the other benchmark is that the optimizations are more significant when the number of attributes involved is higher. Since in the tests we often have very few attributes, the effect there is lower.

## How was this patch tested?

run benchmarks + existing UTs

Closes #22364 from mgaido91/SPARK-25379.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-26 21:34:18 +08:00
Gengliang Wang b39e228ce8 [SPARK-25541][SQL] CaseInsensitiveMap should be serializable after '-' or 'filterKeys'
## What changes were proposed in this pull request?

`CaseInsensitiveMap` is declared as Serializable. However, it is no serializable after `-` operator or `filterKeys` method.

This PR fix the issue by  overriding the operator `-` and method `filterKeys`. So the we can avoid potential `NotSerializableException` on using `CaseInsensitiveMap`.

## How was this patch tested?

New test suite.

Closes #22553 from gengliangwang/fixCaseInsensitiveMap.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-26 19:41:45 +08:00
Dongjoon Hyun 81cbcca600
[SPARK-25534][SQL] Make SQLHelper trait
## What changes were proposed in this pull request?

Currently, Spark has 7 `withTempPath` and 6 `withSQLConf` functions. This PR aims to remove duplicated and inconsistent code and reduce them to the following meaningful implementations.

**withTempPath**
- `SQLHelper.withTempPath`: The one which was used in `SQLTestUtils`.

**withSQLConf**
- `SQLHelper.withSQLConf`: The one which was used in `PlanTest`.
- `ExecutorSideSQLConfSuite.withSQLConf`: The one which doesn't throw `AnalysisException` on StaticConf changes.
- `SQLTestUtils.withSQLConf`: The one which overrides intentionally to change the active session.
```scala
  protected override def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = {
    SparkSession.setActiveSession(spark)
    super.withSQLConf(pairs: _*)(f)
  }
```

## How was this patch tested?

Pass the Jenkins with the existing tests.

Closes #22548 from dongjoon-hyun/SPARK-25534.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-09-25 23:03:54 -07:00
Maxim Gekk 473d0d862d [SPARK-25514][SQL] Generating pretty JSON by to_json
## What changes were proposed in this pull request?

The PR introduces new JSON option `pretty` which allows to turn on `DefaultPrettyPrinter` of `Jackson`'s Json generator. New option is useful in exploring of deep nested columns and in converting of JSON columns in more readable representation (look at the added test).

## How was this patch tested?

Added rount trip test which convert an JSON string to pretty representation via `from_json()` and `to_json()`.

Closes #22534 from MaxGekk/pretty-json.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-26 09:52:15 +08:00
Huaxin Gao cb77a66891 [SPARK-21291][R] add R partitionBy API in DataFrame
## What changes were proposed in this pull request?

add R partitionBy API in write.df
I didn't add bucketBy in write.df. The last line of write.df is
```
write <- handledCallJMethod(write, "save")
```
save doesn't support bucketBy right now.
```
 assertNotBucketed("save")
```

## How was this patch tested?

Add unit test in test_sparkSQL.R

Closes #22537 from huaxingao/spark-21291.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-26 09:37:44 +08:00
gatorsmile 8c2edf46d0 [SPARK-24324][PYTHON][FOLLOW-UP] Rename the Conf to spark.sql.legacy.execution.pandas.groupedMap.assignColumnsByName
## What changes were proposed in this pull request?

Add the legacy prefix for spark.sql.execution.pandas.groupedMap.assignColumnsByPosition and rename it to spark.sql.legacy.execution.pandas.groupedMap.assignColumnsByName

## How was this patch tested?
The existing tests.

Closes #22540 from gatorsmile/renameAssignColumnsByPosition.

Lead-authored-by: gatorsmile <gatorsmile@gmail.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-26 09:32:51 +08:00
Imran Rashid 9bb3a0c67b [SPARK-25422][CORE] Don't memory map blocks streamed to disk.
After data has been streamed to disk, the buffers are inserted into the
memory store in some cases (eg., with broadcast blocks).  But broadcast
code also disposes of those buffers when the data has been read, to
ensure that we don't leave mapped buffers using up memory, which then
leads to garbage data in the memory store.

## How was this patch tested?

Ran the old failing test in a loop. Full tests on jenkins

Closes #22546 from squito/SPARK-25422-master.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-26 08:45:27 +08:00
Shixiong Zhu 66d29870c0
[SPARK-25495][SS] FetchedData.reset should reset all fields
## What changes were proposed in this pull request?

`FetchedData.reset` should reset `_nextOffsetInFetchedData` and `_offsetAfterPoll`. Otherwise it will cause inconsistent cached data and may make Kafka connector return wrong results.

## How was this patch tested?

The new unit test.

Closes #22507 from zsxwing/fix-kafka-reset.

Lead-authored-by: Shixiong Zhu <zsxwing@gmail.com>
Co-authored-by: Shixiong Zhu <shixiong@databricks.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2018-09-25 11:42:27 -07:00
yucai 04db035378
[SPARK-25486][TEST] Refactor SortBenchmark to use main method
## What changes were proposed in this pull request?

Refactor SortBenchmark to use main method.
Generate benchmark result:
```
SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.benchmark.SortBenchmark"
```

## How was this patch tested?

manual tests

Closes #22495 from yucai/SPARK-25486.

Authored-by: yucai <yyu1@ebay.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-09-25 11:13:05 -07:00
Reynold Xin 9cbd001e24 [SPARK-23907][SQL] Revert regr_* functions entirely
## What changes were proposed in this pull request?
This patch reverts entirely all the regr_* functions added in SPARK-23907. These were added by mgaido91 (and proposed by gatorsmile) to improve compatibility with other database systems, without any actual use cases. However, they are very rarely used, and in Spark there are much better ways to compute these functions, due to Spark's flexibility in exposing real programming APIs.

I'm going through all the APIs added in Spark 2.4 and I think we should revert these. If there are strong enough demands and more use cases, we can add them back in the future pretty easily.

## How was this patch tested?
Reverted test cases also.

Closes #22541 from rxin/SPARK-23907.

Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-25 20:13:07 +08:00
Dilip Biswal 7d8f5b62c5 [SPARK-25519][SQL] ArrayRemove function may return incorrect result when right expression is implicitly downcasted.
## What changes were proposed in this pull request?
In ArrayRemove, we currently cast the right hand side expression to match the element type of the left hand side Array. This may result in down casting and may return wrong result or questionable result.

Example :
```SQL
spark-sql> select array_remove(array(1,2,3), 1.23D);
       [2,3]
```
```SQL
spark-sql> select array_remove(array(1,2,3), 'foo');
        NULL
```
We should safely coerce both left and right hand side expressions.
## How was this patch tested?
Added tests in DataFrameFunctionsSuite

Closes #22542 from dilipbiswal/SPARK-25519.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-25 12:05:04 +08:00
Shahid 615792da42
[SPARK-25503][CORE][WEBUI] Total task message in stage page is ambiguous
## What changes were proposed in this pull request?
Test steps :
   1) bin/spark-shell --conf spark.ui.retainedTasks=10
   2) val rdd = sc.parallelize(1 to 1000, 1000)
  3)  rdd.count

Stage page tab in the UI will display 10 tasks, but display message is wrong. It should reverse.

**Before fix :**
![webui_1](https://user-images.githubusercontent.com/23054875/45917921-8926d800-be9c-11e8-8da5-3998d07e3ccc.jpg)

**After fix**
![spark_web_ui2](https://user-images.githubusercontent.com/23054875/45917935-b4112c00-be9c-11e8-9d10-4fcc8e88568f.jpg)

## How was this patch tested?

Manually tested

Closes #22525 from shahidki31/SparkUI.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-09-24 20:03:52 -07:00
hyukjinkwon 2c9ffda1b5 [BUILD] Closes stale PR
Closes #22517
2018-09-25 07:38:40 +08:00
Shahid 3ce2e008ec [SPARK-25502][CORE][WEBUI] Empty Page when page number exceeds the reatinedTask size.
## What changes were proposed in this pull request?
Test steps :
1)  bin/spark-shell --conf spark.ui.retainedTasks=200
```
val rdd = sc.parallelize(1 to 1000, 1000)
rdd.count
```

Stage tab in the UI will display 10 pages with 100 tasks per page. But number of retained tasks is only 200. So, from the 3rd page onwards will display nothing.
 We have to calculate total pages based on the number of tasks need display in the UI.

**Before fix:**
![empty_4](https://user-images.githubusercontent.com/23054875/45918251-b1650580-bea1-11e8-90d3-7e0d491981a2.jpg)

**After fix:**
![empty_3](https://user-images.githubusercontent.com/23054875/45918257-c2ae1200-bea1-11e8-960f-dfbdb4a90ae7.jpg)

## How was this patch tested?

Manually tested

Closes #22526 from shahidki31/SPARK-25502.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-09-24 14:17:42 -07:00
Dilip Biswal bb49661e19 [SPARK-25416][SQL] ArrayPosition function may return incorrect result when right expression is implicitly down casted
## What changes were proposed in this pull request?
In ArrayPosition, we currently cast the right hand side expression to match the element type of the left hand side Array. This may result in down casting and may return wrong result or questionable result.

Example :
```SQL
spark-sql> select array_position(array(1), 1.34);
1
```
```SQL
spark-sql> select array_position(array(1), 'foo');
null
```

We should safely coerce both left and right hand side expressions.
## How was this patch tested?
Added tests in DataFrameFunctionsSuite

Closes #22407 from dilipbiswal/SPARK-25416.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-24 21:37:51 +08:00
Stan Zhai 804515f821 [SPARK-21318][SQL] Improve exception message thrown by lookupFunction
## What changes were proposed in this pull request?

The function actually exists in current selected database, and it's failed to init during `lookupFunciton`, but the exception message is:
```
This function is neither a registered temporary function nor a permanent function registered in the database 'default'.
```

This is not conducive to positioning problems. This PR fix the problem.

## How was this patch tested?

new test case + manual tests

Closes #18544 from stanzhai/fix-udf-error-message.

Authored-by: Stan Zhai <mail@stanzhai.site>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-24 21:33:12 +08:00
hyukjinkwon c3b4a94a91 [SPARKR] Match pyspark features in SparkR communication protocol 2018-09-24 19:25:02 +08:00
Yuming Wang c79072aafa
[SPARK-25478][SQL][TEST] Refactor CompressionSchemeBenchmark to use main method
## What changes were proposed in this pull request?

Refactor `CompressionSchemeBenchmark` to use main method.
Generate benchmark result:
```sh
SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.columnar.compression.CompressionSchemeBenchmark"
```

## How was this patch tested?

manual tests

Closes #22486 from wangyum/SPARK-25478.

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-09-23 20:46:40 -07:00
Yuming Wang d522a563ad [SPARK-25415][SQL][FOLLOW-UP] Add Locale.ROOT when toUpperCase
## What changes were proposed in this pull request?

Add `Locale.ROOT` when `toUpperCase`.

## How was this patch tested?

manual tests

Closes #22531 from wangyum/SPARK-25415.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-24 09:30:07 +08:00
seancxmao 9bf04d8543
[SPARK-25489][ML][TEST] Refactor UDTSerializationBenchmark
## What changes were proposed in this pull request?
Refactor `UDTSerializationBenchmark` to use main method and print the output as a separate file.

Run blow command to generate benchmark results:

```
SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "mllib/test:runMain org.apache.spark.mllib.linalg.UDTSerializationBenchmark"
```

## How was this patch tested?
Manual tests.

Closes #22499 from seancxmao/SPARK-25489.

Authored-by: seancxmao <seancxmao@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-09-23 13:34:06 -07:00
hyukjinkwon a72d118cd9 [SPARK-25473][PYTHON][SS][TEST] ForeachWriter tests failed on Python 3.6 and macOS High Sierra
## What changes were proposed in this pull request?

This PR does not fix the problem itself but just target to add few comments to run PySpark tests on Python 3.6 and macOS High Serria since it actually blocks to run tests on this enviornment.

it does not target to fix the problem yet.

The problem here looks because we fork python workers and the forked workers somehow call Objective-C libraries in some codes at CPython's implementation. After debugging a while, I suspect `pickle` in Python 3.6 has some changes:

58419b9267/python/pyspark/serializers.py (L577)

in particular, it looks also related to which objects are serialized or not as well.

This link (http://sealiesoftware.com/blog/archive/2017/6/5/Objective-C_and_fork_in_macOS_1013.html) and this link (https://blog.phusion.nl/2017/10/13/why-ruby-app-servers-break-on-macos-high-sierra-and-what-can-be-done-about-it/) were helpful for me to understand this.

I am still debugging this but my guts say it's difficult to fix or workaround within Spark side.

## How was this patch tested?

Manually tested:

Before `OBJC_DISABLE_INITIALIZE_FORK_SAFETY=YES`:

```
/usr/local/Cellar/python/3.6.5/Frameworks/Python.framework/Versions/3.6/lib/python3.6/subprocess.py:766: ResourceWarning: subprocess 27563 is still running
  ResourceWarning, source=self)
[Stage 0:>                                                          (0 + 1) / 1]objc[27586]: +[__NSPlaceholderDictionary initialize] may have been in progress in another thread when fork() was called.
objc[27586]: +[__NSPlaceholderDictionary initialize] may have been in progress in another thread when fork() was called. We cannot safely call it or ignore it in the fork() child process. Crashing instead. Set a breakpoint on objc_initializeAfterForkError to debug.
ERROR

======================================================================
ERROR: test_streaming_foreach_with_simple_function (pyspark.sql.tests.SQLTests)
----------------------------------------------------------------------
Traceback (most recent call last):
  File "/.../spark/python/pyspark/sql/utils.py", line 63, in deco
    return f(*a, **kw)
  File "/.../spark/python/lib/py4j-0.10.7-src.zip/py4j/protocol.py", line 328, in get_return_value
    format(target_id, ".", name), value)
py4j.protocol.Py4JJavaError: An error occurred while calling o54.processAllAvailable.
: org.apache.spark.sql.streaming.StreamingQueryException: Writing job aborted.
=== Streaming Query ===
Identifier: [id = f508d634-407c-4232-806b-70e54b055c42, runId = 08d1435b-5358-4fb6-b167-811584a3163e]
Current Committed Offsets: {}
Current Available Offsets: {FileStreamSource[file:/var/folders/71/484zt4z10ks1vydt03bhp6hr0000gp/T/tmpolebys1s]: {"logOffset":0}}

Current State: ACTIVE
Thread State: RUNNABLE

Logical Plan:
FileStreamSource[file:/var/folders/71/484zt4z10ks1vydt03bhp6hr0000gp/T/tmpolebys1s]
	at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runStream(StreamExecution.scala:295)
	at org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:189)
Caused by: org.apache.spark.SparkException: Writing job aborted.
	at org.apache.spark.sql.execution.datasources.v2.WriteToDataSourceV2Exec.doExecute(WriteToDataSourceV2Exec.scala:91)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
```

After `OBJC_DISABLE_INITIALIZE_FORK_SAFETY=YES`:

```
test_streaming_foreach_with_simple_function (pyspark.sql.tests.SQLTests) ...
ok
```

Closes #22480 from HyukjinKwon/SPARK-25473.

Authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-23 11:14:27 +08:00
gatorsmile 0fbba76faa [MINOR][PYSPARK] Always Close the tempFile in _serialize_to_jvm
## What changes were proposed in this pull request?

Always close the tempFile after `serializer.dump_stream(data, tempFile)` in _serialize_to_jvm

## How was this patch tested?

N/A

Closes #22523 from gatorsmile/fixMinor.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-23 10:16:33 +08:00
Gengliang Wang 6ca87eb2e0 [SPARK-25465][TEST] Refactor Parquet test suites in project Hive
## What changes were proposed in this pull request?

Current the file [parquetSuites.scala](f29c2b5287/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala) is not recognizable.
When I tried to find test suites for built-in Parquet conversions for Hive serde, I can only find [HiveParquetSuite](f29c2b5287/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala) in the first few minutes.

This PR is to:
1. Rename `ParquetMetastoreSuite` to `HiveParquetMetastoreSuite`, and create a single file for it.
2. Rename `ParquetSourceSuite` to `HiveParquetSourceSuite`, and create a single file for it.
3. Create a single file for `ParquetPartitioningTest`.
4. Delete `parquetSuites.scala` .

## How was this patch tested?

Unit test

Closes #22467 from gengliangwang/refactor_parquet_suites.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-09-22 09:44:46 -07:00
WeichenXu 40edab209b [SPARK-25321][ML] Fix local LDA model constructor
## What changes were proposed in this pull request?

change back the constructor to:
```
class LocalLDAModel private[ml] (
    uid: String,
    vocabSize: Int,
    private[clustering] val oldLocalModel : OldLocalLDAModel,
    sparkSession: SparkSession)
```

Although it is marked `private[ml]`, it is used in `mleap` and the master change breaks `mleap` building.
See mleap code [here](c7860af328/mleap-spark/src/main/scala/org/apache/spark/ml/bundle/ops/clustering/LDAModelOp.scala (L57))
## How was this patch tested?

Manual.

Closes #22510 from WeichenXu123/LDA_fix.

Authored-by: WeichenXu <weichen.xu@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2018-09-21 13:08:01 -07:00
Reynold Xin 4a11209539 [SPARK-19724][SQL] allowCreatingManagedTableUsingNonemptyLocation should have legacy prefix
One more legacy config to go ...

Closes #22515 from rxin/allowCreatingManagedTableUsingNonemptyLocation.

Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-09-21 09:45:41 -07:00
Gengliang Wang d25f425c96 [SPARK-25499][TEST] Refactor BenchmarkBase and Benchmark
## What changes were proposed in this pull request?

Currently there are two classes with the same naming BenchmarkBase:
1. `org.apache.spark.util.BenchmarkBase`
2. `org.apache.spark.sql.execution.benchmark.BenchmarkBase`

This is very confusing. And the benchmark object `org.apache.spark.sql.execution.benchmark.FilterPushdownBenchmark` is using the one in `org.apache.spark.util.BenchmarkBase`, while there is another class `BenchmarkBase` in the same package of it...

Here I propose:
1. the package `org.apache.spark.util.BenchmarkBase` should be in test package of core module. Move it to package `org.apache.spark.benchmark` .
2. Move `org.apache.spark.util.Benchmark` to test package of core module. Move it to package `org.apache.spark.benchmark` .
3. Rename the class `org.apache.spark.sql.execution.benchmark.BenchmarkBase` as `BenchmarkWithCodegen`

## How was this patch tested?

Unit test

Closes #22513 from gengliangwang/refactorBenchmarkBase.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-21 22:20:55 +08:00
Sanket Chintapalli ff601cf71d [SPARK-24355] Spark external shuffle server improvement to better handle block fetch requests.
## What changes were proposed in this pull request?

Description:
Right now, the default server side netty handler threads is 2 * # cores, and can be further configured with parameter spark.shuffle.io.serverThreads.
In order to process a client request, it would require one available server netty handler thread.
However, when the server netty handler threads start to process ChunkFetchRequests, they will be blocked on disk I/O, mostly due to disk contentions from the random read operations initiated by all the ChunkFetchRequests received from clients.
As a result, when the shuffle server is serving many concurrent ChunkFetchRequests, the server side netty handler threads could all be blocked on reading shuffle files, thus leaving no handler thread available to process other types of requests which should all be very quick to process.

This issue could potentially be fixed by limiting the number of netty handler threads that could get blocked when processing ChunkFetchRequest. We have a patch to do this by using a separate EventLoopGroup with a dedicated ChannelHandler to process ChunkFetchRequest. This enables shuffle server to reserve netty handler threads for non-ChunkFetchRequest, thus enabling consistent processing time for these requests which are fast to process. After deploying the patch in our infrastructure, we no longer see timeout issues with either executor registration with local shuffle server or shuffle client establishing connection with remote shuffle server.
(Please fill in changes proposed in this fix)

For Original PR please refer here
https://github.com/apache/spark/pull/21402

## How was this patch tested?

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

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

Closes #22173 from redsanket/SPARK-24335.

Authored-by: Sanket Chintapalli <schintap@yahoo-inc.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2018-09-21 09:05:56 -05:00
Marek Novotny 2c9d8f56c7 [SPARK-25469][SQL] Eval methods of Concat, Reverse and ElementAt should use pattern matching only once
## What changes were proposed in this pull request?

The PR proposes to avoid usage of pattern matching for each call of ```eval``` method within:
- ```Concat```
- ```Reverse```
- ```ElementAt```

## How was this patch tested?

Run the existing tests for ```Concat```, ```Reverse``` and  ```ElementAt``` expression classes.

Closes #22471 from mn-mikke/SPARK-25470.

Authored-by: Marek Novotny <mn.mikke@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2018-09-21 18:16:54 +09:00
Reynold Xin 411ecc365e [SPARK-23549][SQL] Rename config spark.sql.legacy.compareDateTimestampInTimestamp
## What changes were proposed in this pull request?
See title. Makes our legacy backward compatibility configs more consistent.

## How was this patch tested?
Make sure all references have been updated:
```
> git grep compareDateTimestampInTimestamp
docs/sql-programming-guide.md:  - Since Spark 2.4, Spark compares a DATE type with a TIMESTAMP type after promotes both sides to TIMESTAMP. To set `false` to `spark.sql.legacy.compareDateTimestampInTimestamp` restores the previous behavior. This option will be removed in Spark 3.0.
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:    // if conf.compareDateTimestampInTimestamp is true
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:      => if (conf.compareDateTimestampInTimestamp) Some(TimestampType) else Some(StringType)
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala:      => if (conf.compareDateTimestampInTimestamp) Some(TimestampType) else Some(StringType)
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:    buildConf("spark.sql.legacy.compareDateTimestampInTimestamp")
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:  def compareDateTimestampInTimestamp : Boolean = getConf(COMPARE_DATE_TIMESTAMP_IN_TIMESTAMP)
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala:        "spark.sql.legacy.compareDateTimestampInTimestamp" -> convertToTS.toString) {
```

Closes #22508 from rxin/SPARK-23549.

Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-21 14:27:14 +08:00
Reynold Xin fb3276a54a [SPARK-25384][SQL] Clarify fromJsonForceNullableSchema will be removed in Spark 3.0
See above. This should go into the 2.4 release.

Closes #22509 from rxin/SPARK-25384.

Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-21 14:17:34 +08:00
seancxmao 1f4ca6f5c5 [SPARK-25487][SQL][TEST] Refactor PrimitiveArrayBenchmark
## What changes were proposed in this pull request?
Refactor PrimitiveArrayBenchmark to use main method and print the output as a separate file.

Run blow command to generate benchmark results:

```
SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.benchmark.PrimitiveArrayBenchmark"
```

## How was this patch tested?
Manual tests.

Closes #22497 from seancxmao/SPARK-25487.

Authored-by: seancxmao <seancxmao@gmail.com>
Signed-off-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
2018-09-21 15:04:47 +09:00
Kris Mok 596af211a5 [SPARK-25494][SQL] Upgrade Spark's use of Janino to 3.0.10
## What changes were proposed in this pull request?

This PR upgrades Spark's use of Janino from 3.0.9 to 3.0.10.
Note that 3.0.10 is a out-of-band release specifically for fixing an integer overflow issue in Janino's `ClassFile` reader. It is otherwise exactly the same as 3.0.9, so it's a low risk and compatible upgrade.

The integer overflow issue affects Spark SQL's codegen stats collection: when a generated Class file is huge, especially when the constant pool size is above `Short.MAX_VALUE`, Janino's `ClassFile reader` will throw an exception when Spark wants to parse the generated Class file to collect stats. So we'll miss the stats of some huge Class files.

The related Janino issue is: https://github.com/janino-compiler/janino/issues/58

## How was this patch tested?

Existing codegen tests.

Closes #22506 from rednaxelafx/upgrade-janino.

Authored-by: Kris Mok <kris.mok@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-09-20 22:15:52 -07:00
gatorsmile 5d25e15440 Revert "[SPARK-23715][SQL] the input of to/from_utc_timestamp can not have timezone
## What changes were proposed in this pull request?

This reverts commit 417ad92502.

We decided to keep the current behaviors unchanged and will consider whether we will deprecate the  these functions in 3.0. For more details, see the discussion in https://issues.apache.org/jira/browse/SPARK-23715

## How was this patch tested?

The existing tests.

Closes #22505 from gatorsmile/revertSpark-23715.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-21 10:39:45 +08:00
Gengliang Wang 950ab79957 [SPARK-24777][SQL] Add write benchmark for AVRO
## What changes were proposed in this pull request?

Refactor `DataSourceWriteBenchmark` and add write benchmark for AVRO.

## How was this patch tested?

Build and run the benchmark.

Closes #22451 from gengliangwang/avroWriteBenchmark.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-09-20 17:41:24 -07:00
Burak Yavuz 77e52448e7 [SPARK-25472][SS] Don't have legitimate stops of streams cause stream exceptions
## What changes were proposed in this pull request?

Legitimate stops of streams may actually cause an exception to be captured by stream execution, because the job throws a SparkException regarding job cancellation during a stop. This PR makes the stop more graceful by swallowing this cancellation error.

## How was this patch tested?

This is pretty hard to test. The existing tests should make sure that we're not swallowing other specific SparkExceptions. I've also run the `KafkaSourceStressForDontFailOnDataLossSuite`100 times, and it didn't fail, whereas it used to be flaky.

Closes #22478 from brkyvz/SPARK-25472.

Authored-by: Burak Yavuz <brkyvz@gmail.com>
Signed-off-by: Burak Yavuz <brkyvz@gmail.com>
2018-09-20 15:46:33 -07:00
liuxian 4d114fc9a2 [SPARK-25366][SQL] Zstd and brotli CompressionCodec are not supported for parquet files
## What changes were proposed in this pull request?
Hadoop2.6  and  hadoop2.7 do not contain zstd and brotli compressioncodec ,hadoop 3.1 also contains only zstd  compressioncodec .
 So I think we should remove zstd and brotil  for the time being.

**set  `spark.sql.parquet.compression.codec=brotli`:**
Caused by: org.apache.parquet.hadoop.BadConfigurationException: Class org.apache.hadoop.io.compress.BrotliCodec was not found
        at org.apache.parquet.hadoop.CodecFactory.getCodec(CodecFactory.java:235)
        at org.apache.parquet.hadoop.CodecFactory$HeapBytesCompressor.<init>(CodecFactory.java:142)
        at org.apache.parquet.hadoop.CodecFactory.createCompressor(CodecFactory.java:206)
        at org.apache.parquet.hadoop.CodecFactory.getCompressor(CodecFactory.java:189)
        at org.apache.parquet.hadoop.ParquetRecordWriter.<init>(ParquetRecordWriter.java:153)
        at org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:411)
        at org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:349)
        at org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.<init>(ParquetOutputWriter.scala:37)
        at org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat$$anon$1.newInstance(ParquetFileFormat.scala:161)

**set  `spark.sql.parquet.compression.codec=zstd`:**
Caused by: org.apache.parquet.hadoop.BadConfigurationException: Class org.apache.hadoop.io.compress.ZStandardCodec was not found
        at org.apache.parquet.hadoop.CodecFactory.getCodec(CodecFactory.java:235)
        at org.apache.parquet.hadoop.CodecFactory$HeapBytesCompressor.<init>(CodecFactory.java:142)
        at org.apache.parquet.hadoop.CodecFactory.createCompressor(CodecFactory.java:206)
        at org.apache.parquet.hadoop.CodecFactory.getCompressor(CodecFactory.java:189)
        at org.apache.parquet.hadoop.ParquetRecordWriter.<init>(ParquetRecordWriter.java:153)
        at org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:411)
        at org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:349)
        at org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.<init>(ParquetOutputWriter.scala:37)
        at org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat$$anon$1.newInstance(ParquetFileFormat.scala:161)

## How was this patch tested?
Exist unit test

Closes #22358 from 10110346/notsupportzstdandbrotil.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-09-20 16:53:48 -05:00
Nihar Sheth 2f51e72356 [SPARK-24918][CORE] Executor Plugin API
## What changes were proposed in this pull request?

A continuation of squito's executor plugin task. By his request I took his code and added testing and moved the plugin initialization to a separate thread.

Executor plugins now run on one separate thread, so the executor does not wait on them. Added testing.

## How was this patch tested?

Added test cases that test using a sample plugin.

Closes #22192 from NiharS/executorPlugin.

Lead-authored-by: Nihar Sheth <niharrsheth@gmail.com>
Co-authored-by: NiharS <niharrsheth@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-09-20 11:52:20 -07:00
Maxim Gekk a86f84102e [SPARK-25381][SQL] Stratified sampling by Column argument
## What changes were proposed in this pull request?

In the PR, I propose to add an overloaded method for `sampleBy` which accepts the first argument of the `Column` type. This will allow to sample by any complex columns as well as sampling by multiple columns. For example:

```Scala
spark.createDataFrame(Seq(("Bob", 17), ("Alice", 10), ("Nico", 8), ("Bob", 17),
  ("Alice", 10))).toDF("name", "age")
  .stat
  .sampleBy(struct($"name", $"age"), Map(Row("Alice", 10) -> 0.3, Row("Nico", 8) -> 1.0), 36L)
  .show()

+-----+---+
| name|age|
+-----+---+
| Nico|  8|
|Alice| 10|
+-----+---+
```

## How was this patch tested?

Added new test for sampling by multiple columns for Scala and test for Java, Python to check that `sampleBy` is able to sample by `Column` type argument.

Closes #22365 from MaxGekk/sample-by-column.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-21 01:11:40 +08:00
maryannxue 88446b6ad1 [SPARK-25450][SQL] PushProjectThroughUnion rule uses the same exprId for project expressions in each Union child, causing mistakes in constant propagation
## What changes were proposed in this pull request?

The problem was cause by the PushProjectThroughUnion rule, which, when creating new Project for each child of Union, uses the same exprId for expressions of the same position. This is wrong because, for each child of Union, the expressions are all independent, and it can lead to a wrong result if other rules like FoldablePropagation kicks in, taking two different expressions as the same.

This fix is to create new expressions in the new Project for each child of Union.

## How was this patch tested?

Added UT.

Closes #22447 from maryannxue/push-project-thru-union-bug.

Authored-by: maryannxue <maryannxue@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-09-20 10:00:28 -07:00
hyukjinkwon 88e7e87bd5 [MINOR][PYTHON] Use a helper in PythonUtils instead of direct accessing Scala package
## What changes were proposed in this pull request?

This PR proposes to use add a helper in `PythonUtils` instead of direct accessing Scala package.

## How was this patch tested?

Jenkins tests.

Closes #22483 from HyukjinKwon/minor-refactoring.

Authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-21 00:41:42 +08:00
Dilip Biswal 67f2c6a554 [SPARK-25417][SQL] ArrayContains function may return incorrect result when right expression is implicitly down casted
## What changes were proposed in this pull request?
In ArrayContains, we currently cast the right hand side expression to match the element type of the left hand side Array. This may result in down casting and may return wrong result or questionable result.

Example :
```SQL
spark-sql> select array_contains(array(1), 1.34);
true
```
```SQL
spark-sql> select array_contains(array(1), 'foo');
null
```

We should safely coerce both left and right hand side expressions.
## How was this patch tested?
Added tests in DataFrameFunctionsSuite

Closes #22408 from dilipbiswal/SPARK-25417.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-20 20:33:44 +08:00
hyukjinkwon edf5cc64e4 [SPARK-25460][SS] DataSourceV2: SS sources do not respect SessionConfigSupport
## What changes were proposed in this pull request?

This PR proposes to respect `SessionConfigSupport` in SS datasources as well. Currently these are only respected in batch sources:

e06da95cd9/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala (L198-L203)

e06da95cd9/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala (L244-L249)

If a developer makes a datasource V2 that supports both structured streaming and batch jobs, batch jobs respect a specific configuration, let's say, URL to connect and fetch data (which end users might not be aware of); however, structured streaming ends up with not supporting this (and should explicitly be set into options).

## How was this patch tested?

Unit tests were added.

Closes #22462 from HyukjinKwon/SPARK-25460.

Authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-20 20:22:55 +08:00
Liang-Chi Hsieh 89671a27e7 Revert [SPARK-19355][SPARK-25352]
## What changes were proposed in this pull request?

This goes to revert sequential PRs based on some discussion and comments at https://github.com/apache/spark/pull/16677#issuecomment-422650759.

#22344
#22330
#22239
#16677

## How was this patch tested?

Existing tests.

Closes #22481 from viirya/revert-SPARK-19355-1.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-20 20:18:31 +08:00
hyukjinkwon 7ff5386ed9 [MINOR][PYTHON][TEST] Use collect() instead of show() to make the output silent
## What changes were proposed in this pull request?

This PR replace an effective `show()` to `collect()` to make the output silent.

**Before:**

```
test_simple_udt_in_df (pyspark.sql.tests.SQLTests) ... +---+----------+
|key|       val|
+---+----------+
|  0|[0.0, 0.0]|
|  1|[1.0, 1.0]|
|  2|[2.0, 2.0]|
|  0|[3.0, 3.0]|
|  1|[4.0, 4.0]|
|  2|[5.0, 5.0]|
|  0|[6.0, 6.0]|
|  1|[7.0, 7.0]|
|  2|[8.0, 8.0]|
|  0|[9.0, 9.0]|
+---+----------+
```

**After:**

```
test_simple_udt_in_df (pyspark.sql.tests.SQLTests) ... ok
```

## How was this patch tested?

Manually tested.

Closes #22479 from HyukjinKwon/minor-udf-test.

Authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-20 15:03:16 +08:00
Yuming Wang 0e31a6f25e [SPARK-25339][TEST] Refactor FilterPushdownBenchmark
## What changes were proposed in this pull request?

Refactor `FilterPushdownBenchmark` use `main` method. we can use 3 ways to run this test now:

1. bin/spark-submit --class org.apache.spark.sql.execution.benchmark.FilterPushdownBenchmark spark-sql_2.11-2.5.0-SNAPSHOT-tests.jar
2. build/sbt "sql/test:runMain org.apache.spark.sql.execution.benchmark.FilterPushdownBenchmark"
3. SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.benchmark.FilterPushdownBenchmark"

The method 2 and the method 3 do not need to compile the `spark-sql_*-tests.jar` package. So these two methods are mainly for developers to quickly do benchmark.

## How was this patch tested?

manual tests

Closes #22443 from wangyum/SPARK-25339.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-20 12:34:39 +08:00
Huaxin Gao 95b177c8f0 [SPARK-23648][R][SQL] Adds more types for hint in SparkR
## What changes were proposed in this pull request?

Addition of numeric and list hints for  SparkR.

## How was this patch tested?
Add test in test_sparkSQL.R

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

Closes #21649 from huaxingao/spark-23648.
2018-09-19 21:27:30 -07:00
Reynold Xin 76399d75e2 [SPARK-4502][SQL] Rename to spark.sql.optimizer.nestedSchemaPruning.enabled
## What changes were proposed in this pull request?
This patch adds an "optimizer" prefix to nested schema pruning.

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

Closes #22475 from rxin/SPARK-4502.

Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-09-19 21:23:35 -07:00
Marco Gaido 47d6e80a2e [SPARK-25457][SQL] IntegralDivide returns data type of the operands
## What changes were proposed in this pull request?

The PR proposes to return the data type of the operands as a result for the `div` operator. Before the PR, `bigint` is always returned. It introduces also a `spark.sql.legacy.integralDivide.returnBigint` config in order to let the users restore the legacy behavior.

## How was this patch tested?

added UTs

Closes #22465 from mgaido91/SPARK-25457.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-20 10:23:37 +08:00
Marco Gaido 8aae49afc7 [SPARK-24341][FOLLOWUP][DOCS] Add migration note for IN subqueries behavior
## What changes were proposed in this pull request?

The PR updates the migration guide in order to explain the changes introduced in the behavior of the IN operator with subqueries, in particular, the improved handling of struct attributes in these situations.

## How was this patch tested?

NA

Closes #22469 from mgaido91/SPARK-24341_followup.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-20 10:10:20 +08:00
Reynold Xin 936c920347 [SPARK-24157][SS][FOLLOWUP] Rename to spark.sql.streaming.noDataMicroBatches.enabled
## What changes were proposed in this pull request?
This patch changes the config option `spark.sql.streaming.noDataMicroBatchesEnabled` to `spark.sql.streaming.noDataMicroBatches.enabled` to be more consistent with rest of the configs. Unfortunately there is one streaming config called `spark.sql.streaming.metricsEnabled`. For that one we should just use a fallback config and change it in a separate patch.

## How was this patch tested?
Made sure no other references to this config are in the code base:
```
> git grep "noDataMicro"
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:    buildConf("spark.sql.streaming.noDataMicroBatches.enabled")
```

Closes #22476 from rxin/SPARK-24157.

Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: Reynold Xin <rxin@databricks.com>
2018-09-19 18:51:20 -07:00