GitHub user tanshai opened a pull request:
https://github.com/apache/spark/pull/13923
Add range partitioner on DataFrame
## What changes were proposed in this pull request?
Currently, a dataframe can be only hash partitioned using "Repartition",
please add a new function that partition a dataframe by range
## How was this patch tested?
Existing tests
(If this patch involves UI changes, please attach a screenshot; otherwise,
remove this)
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/apache/spark master
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/spark/pull/13923.patch
To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:
This closes #13923
----
commit 5c16ad0d522e5124a6977533077afb7b38fc42a1
Author: Kay Ousterhout <[email protected]>
Date: 2016-06-10T19:50:27Z
Revert [SPARK-14485][CORE] ignore task finished for executor lost
This reverts commit 695dbc816a6d70289abeb145cb62ff4e62b3f49b.
This change is being reverted because it hurts performance of some jobs, and
only helps in a narrow set of cases. For more discussion, refer to the
JIRA.
Author: Kay Ousterhout <[email protected]>
Closes #13580 from kayousterhout/revert-SPARK-14485.
commit fb219029dd1b8d2783c3e202361401048296595c
Author: Liwei Lin <[email protected]>
Date: 2016-06-10T20:01:29Z
[SPARK-15871][SQL] Add `assertNotPartitioned` check in `DataFrameWriter`
## What changes were proposed in this pull request?
It doesn't make sense to specify partitioning parameters, when we write
data out from Datasets/DataFrames into `jdbc` tables or streaming
`ForeachWriter`s.
This patch adds `assertNotPartitioned` check in `DataFrameWriter`.
<table>
<tr>
<td align="center"><strong>operation</strong></td>
<td align="center"><strong>should check not partitioned?</strong></td>
</tr>
<tr>
<td align="center">mode</td>
<td align="center"></td>
</tr>
<tr>
<td align="center">outputMode</td>
<td align="center"></td>
</tr>
<tr>
<td align="center">trigger</td>
<td align="center"></td>
</tr>
<tr>
<td align="center">format</td>
<td align="center"></td>
</tr>
<tr>
<td align="center">option/options</td>
<td align="center"></td>
</tr>
<tr>
<td align="center">partitionBy</td>
<td align="center"></td>
</tr>
<tr>
<td align="center">bucketBy</td>
<td align="center"></td>
</tr>
<tr>
<td align="center">sortBy</td>
<td align="center"></td>
</tr>
<tr>
<td align="center">save</td>
<td align="center"></td>
</tr>
<tr>
<td align="center">queryName</td>
<td align="center"></td>
</tr>
<tr>
<td align="center">startStream</td>
<td align="center"></td>
</tr>
<tr>
<td align="center">foreach</td>
<td align="center">yes</td>
</tr>
<tr>
<td align="center">insertInto</td>
<td align="center"></td>
</tr>
<tr>
<td align="center">saveAsTable</td>
<td align="center"></td>
</tr>
<tr>
<td align="center">jdbc</td>
<td align="center">yes</td>
</tr>
<tr>
<td align="center">json</td>
<td align="center"></td>
</tr>
<tr>
<td align="center">parquet</td>
<td align="center"></td>
</tr>
<tr>
<td align="center">orc</td>
<td align="center"></td>
</tr>
<tr>
<td align="center">text</td>
<td align="center"></td>
</tr>
<tr>
<td align="center">csv</td>
<td align="center"></td>
</tr>
</table>
## How was this patch tested?
New dedicated tests.
Author: Liwei Lin <[email protected]>
Closes #13597 from lw-lin/add-assertNotPartitioned.
commit 667d4ea7b35f285954ea7cb719b7c80581e31f4d
Author: Takuya UESHIN <[email protected]>
Date: 2016-06-10T20:06:18Z
[SPARK-6320][SQL] Move planLater method into GenericStrategy.
## What changes were proposed in this pull request?
This PR moves `QueryPlanner.planLater()` method into `GenericStrategy` for
extra strategies to be able to use `planLater` in its strategy.
## How was this patch tested?
Existing tests.
Author: Takuya UESHIN <[email protected]>
Closes #13147 from ueshin/issues/SPARK-6320.
commit 865ec32dd997e63aea01a871d1c7b4947f43c111
Author: Sandeep Singh <[email protected]>
Date: 2016-06-10T20:06:51Z
[MINOR][X][X] Replace all occurrences of None: Option with Option.empty
## What changes were proposed in this pull request?
Replace all occurrences of `None: Option[X]` with `Option.empty[X]`
## How was this patch tested?
Exisiting Tests
Author: Sandeep Singh <[email protected]>
Closes #13591 from techaddict/minor-7.
commit 026eb90644be7685971dacaabae67a293edd0133
Author: wangyang <[email protected]>
Date: 2016-06-10T20:10:03Z
[SPARK-15875] Try to use Seq.isEmpty and Seq.nonEmpty instead of Seq.length
== 0 and Seq.length > 0
## What changes were proposed in this pull request?
In scala, immutable.List.length is an expensive operation so we should
avoid using Seq.length == 0 or Seq.lenth > 0, and use Seq.isEmpty and
Seq.nonEmpty instead.
## How was this patch tested?
existing tests
Author: wangyang <[email protected]>
Closes #13601 from yangw1234/isEmpty.
commit e05a2feebe928df691d5a8f42f22e088c6263dcf
Author: Herman van Hovell <[email protected]>
Date: 2016-06-10T21:29:05Z
[SPARK-15825] [SQL] Fix SMJ invalid results
## What changes were proposed in this pull request?
Code generated `SortMergeJoin` failed with wrong results when using structs
as keys. This could (eventually) be traced back to the use of a wrong row
reference when comparing structs.
## How was this patch tested?
TBD
Author: Herman van Hovell <[email protected]>
Closes #13589 from hvanhovell/SPARK-15822.
commit aec502d9114ad8e18bfbbd63f38780e076d326d1
Author: Davies Liu <[email protected]>
Date: 2016-06-10T21:32:43Z
[SPARK-15654] [SQL] fix non-splitable files for text based file formats
## What changes were proposed in this pull request?
Currently, we always split the files when it's bigger than maxSplitBytes,
but Hadoop LineRecordReader does not respect the splits for compressed files
correctly, we should have a API for FileFormat to check whether the file could
be splitted or not.
This PR is based on #13442, closes #13442
## How was this patch tested?
add regression tests.
Author: Davies Liu <[email protected]>
Closes #13531 from davies/fix_split.
commit 127a6678d7af6b5164a115be7c64525bb80001fe
Author: Sela <[email protected]>
Date: 2016-06-10T21:36:51Z
[SPARK-15489][SQL] Dataset kryo encoder won't load custom user settings
## What changes were proposed in this pull request?
Serializer instantiation will consider existing SparkConf
## How was this patch tested?
manual test with `ImmutableList` (Guava) and `kryo-serializers`'s
`Immutable*Serializer` implementations.
Added Test Suite.
(If this patch involves UI changes, please attach a screenshot; otherwise,
remove this)
Author: Sela <[email protected]>
Closes #13424 from amitsela/SPARK-15489.
commit 2022afe57dbf8cb0c9909399962c4a3649e0601c
Author: Dongjoon Hyun <[email protected]>
Date: 2016-06-10T22:40:29Z
[SPARK-15773][CORE][EXAMPLE] Avoid creating local variable `sc` in examples
if possible
## What changes were proposed in this pull request?
Instead of using local variable `sc` like the following example, this PR
uses `spark.sparkContext`. This makes examples more concise, and also fixes
some misleading, i.e., creating SparkContext from SparkSession.
```
- println("Creating SparkContext")
- val sc = spark.sparkContext
-
println("Writing local file to DFS")
val dfsFilename = dfsDirPath + "/dfs_read_write_test"
- val fileRDD = sc.parallelize(fileContents)
+ val fileRDD = spark.sparkContext.parallelize(fileContents)
```
This will change 12 files (+30 lines, -52 lines).
## How was this patch tested?
Manual.
Author: Dongjoon Hyun <[email protected]>
Closes #13520 from dongjoon-hyun/SPARK-15773.
commit 54f758b5fc60ecb0da6b191939a72ef5829be38c
Author: Narine Kokhlikyan <[email protected]>
Date: 2016-06-11T00:17:47Z
[SPARK-15884][SPARKR][SQL] Overriding stringArgs in MapPartitionsInR
## What changes were proposed in this pull request?
As discussed in https://github.com/apache/spark/pull/12836
we need to override stringArgs method in MapPartitionsInR in order to avoid
too large strings generated by "stringArgs" method based on the input arguments.
In this case exclude some of the input arguments: serialized R objects.
## How was this patch tested?
Existing test cases
Author: Narine Kokhlikyan <[email protected]>
Closes #13610 from NarineK/dapply_MapPartitionsInR_stringArgs.
commit bba5d7999f7b3ae9d816ea552ba9378fea1615a6
Author: Liang-Chi Hsieh <[email protected]>
Date: 2016-06-11T01:23:59Z
[SPARK-15639][SQL] Try to push down filter at RowGroups level for parquet
reader
## What changes were proposed in this pull request?
The base class `SpecificParquetRecordReaderBase` used for vectorized
parquet reader will try to get pushed-down filters from the given
configuration. This pushed-down filters are used for RowGroups-level filtering.
However, we don't set up the filters to push down into the configuration. In
other words, the filters are not actually pushed down to do RowGroups-level
filtering. This patch is to fix this and tries to set up the filters for
pushing down to configuration for the reader.
## How was this patch tested?
Existing tests should be passed.
Author: Liang-Chi Hsieh <[email protected]>
Closes #13371 from viirya/vectorized-reader-push-down-filter.
commit 99f3c82776fe5ea4f89a9965a288c7447585dc2c
Author: hyukjinkwon <[email protected]>
Date: 2016-06-11T01:29:26Z
[SPARK-14615][ML][FOLLOWUP] Fix Python examples to use the new ML Vector
and Matrix APIs in the ML pipeline based algorithms
## What changes were proposed in this pull request?
This PR fixes Python examples to use the new ML Vector and Matrix APIs in
the ML pipeline based algorithms.
I firstly executed this shell command, `grep -r "from pyspark.mllib" .` and
then executed them all.
Some of tests in `ml` produced the error messages as below:
```
pyspark.sql.utils.IllegalArgumentException: u'requirement failed: Input
type must be VectorUDT but got org.apache.spark.mllib.linalg.VectorUDTf71b0bce.'
```
So, I fixed them to use new ones just identically with some Python tests
fixed in https://github.com/apache/spark/pull/12627
## How was this patch tested?
Manually tested for all the examples listed by `grep -r "from
pyspark.mllib" .`.
Author: hyukjinkwon <[email protected]>
Closes #13393 from HyukjinKwon/SPARK-14615.
commit 8e7b56f3d4917692d3ff44d91aa264738a6fc2ed
Author: Cheng Lian <[email protected]>
Date: 2016-06-11T03:41:48Z
Revert "[SPARK-15639][SQL] Try to push down filter at RowGroups level for
parquet reader"
This reverts commit bba5d7999f7b3ae9d816ea552ba9378fea1615a6.
commit 468da03e23a01e02718608f05d778386cbb8416b
Author: Sameer Agarwal <[email protected]>
Date: 2016-06-11T03:43:18Z
[SPARK-15678] Add support to REFRESH data source paths
## What changes were proposed in this pull request?
Spark currently incorrectly continues to use cached data even if the
underlying data is overwritten.
Current behavior:
```scala
val dir = "/tmp/test"
sqlContext.range(1000).write.mode("overwrite").parquet(dir)
val df = sqlContext.read.parquet(dir).cache()
df.count() // outputs 1000
sqlContext.range(10).write.mode("overwrite").parquet(dir)
sqlContext.read.parquet(dir).count() // outputs 1000 <---- We are still
using the cached dataset
```
This patch fixes this bug by adding support for `REFRESH path` that
invalidates and refreshes all the cached data (and the associated metadata) for
any dataframe that contains the given data source path.
Expected behavior:
```scala
val dir = "/tmp/test"
sqlContext.range(1000).write.mode("overwrite").parquet(dir)
val df = sqlContext.read.parquet(dir).cache()
df.count() // outputs 1000
sqlContext.range(10).write.mode("overwrite").parquet(dir)
spark.catalog.refreshResource(dir)
sqlContext.read.parquet(dir).count() // outputs 10 <---- We are not using
the cached dataset
```
## How was this patch tested?
Unit tests for overwrites and appends in `ParquetQuerySuite` and
`CachedTableSuite`.
Author: Sameer Agarwal <[email protected]>
Closes #13566 from sameeragarwal/refresh-path-2.
commit 7504bc73f20fe0e6546a019ed91c3fd3804287ba
Author: Davies Liu <[email protected]>
Date: 2016-06-11T04:12:06Z
[SPARK-15759] [SQL] Fallback to non-codegen when fail to compile generated
code
## What changes were proposed in this pull request?
In case of any bugs in whole-stage codegen, the generated code can't be
compiled, we should fallback to non-codegen to make sure that query could run.
The batch mode of new parquet reader depends on codegen, can't be easily
switched to non-batch mode, so we still use codegen for batched scan (for
parquet). Because it only support primitive types and the number of columns is
less than spark.sql.codegen.maxFields (100), it should not fail.
This could be configurable by `spark.sql.codegen.fallback`
## How was this patch tested?
Manual test it with buggy operator, it worked well.
Author: Davies Liu <[email protected]>
Closes #13501 from davies/codegen_fallback.
commit 3761330dd0151d7369d7fba4d4c344e9863990ef
Author: Sean Owen <[email protected]>
Date: 2016-06-11T11:46:07Z
[SPARK-15879][DOCS][UI] Update logo in UI and docs to add "Apache"
## What changes were proposed in this pull request?
Use new Spark logo including "Apache" (now, with crushed PNGs). Remove old
unreferenced logo files.
## How was this patch tested?
Manual check of generated HTML site and Spark UI. I searched for references
to the deleted files to make sure they were not used.
Author: Sean Owen <[email protected]>
Closes #13609 from srowen/SPARK-15879.
commit ad102af169c7344b30d3b84aa16452fcdc22542c
Author: Dongjoon Hyun <[email protected]>
Date: 2016-06-11T11:55:38Z
[SPARK-15883][MLLIB][DOCS] Fix broken links in mllib documents
## What changes were proposed in this pull request?
This issue fixes all broken links on Spark 2.0 preview MLLib documents.
Also, this contains some editorial change.
**Fix broken links**
* mllib-data-types.md
* mllib-decision-tree.md
* mllib-ensembles.md
* mllib-feature-extraction.md
* mllib-pmml-model-export.md
* mllib-statistics.md
**Fix malformed section header and scala coding style**
* mllib-linear-methods.md
**Replace indirect forward links with direct one**
* ml-classification-regression.md
## How was this patch tested?
Manual tests (with `cd docs; jekyll build`.)
Author: Dongjoon Hyun <[email protected]>
Closes #13608 from dongjoon-hyun/SPARK-15883.
commit cb5d933d86ac4afd947874f1f1c31c7154cb8249
Author: Takeshi YAMAMURO <[email protected]>
Date: 2016-06-11T22:12:21Z
[SPARK-15585][SQL] Add doc for turning off quotations
## What changes were proposed in this pull request?
This pr is to add doc for turning off quotations because this behavior is
different from `com.databricks.spark.csv`.
## How was this patch tested?
Check behavior to put an empty string in csv options.
Author: Takeshi YAMAMURO <[email protected]>
Closes #13616 from maropu/SPARK-15585-2.
commit 5bb4564cd47c8bf06409287e0de4ec45609970b2
Author: Eric Liang <[email protected]>
Date: 2016-06-11T22:26:08Z
[SPARK-15881] Update microbenchmark results for WideSchemaBenchmark
## What changes were proposed in this pull request?
These were not updated after performance improvements. To make updating
them easier, I also moved the results from inline comments out into a file,
which is auto-generated when the benchmark is re-run.
Author: Eric Liang <[email protected]>
Closes #13607 from ericl/sc-3538.
commit 75705e8dbb51ac91ffc7012fa67f072494c13832
Author: Wenchen Fan <[email protected]>
Date: 2016-06-11T22:28:40Z
[SPARK-15856][SQL] Revert API breaking changes made in SQLContext.range
## What changes were proposed in this pull request?
It's easy for users to call `range(...).as[Long]` to get typed Dataset, and
don't worth an API breaking change. This PR reverts it.
## How was this patch tested?
N/A
Author: Wenchen Fan <[email protected]>
Closes #13605 from cloud-fan/range.
commit c06c58bbbb2de0c22cfc70c486d23a94c3079ba4
Author: Eric Liang <[email protected]>
Date: 2016-06-11T22:42:58Z
[SPARK-14851][CORE] Support radix sort with nullable longs
## What changes were proposed in this pull request?
This adds support for radix sort of nullable long fields. When a sort field
is null and radix sort is enabled, we keep nulls in a separate region of the
sort buffer so that radix sort does not need to deal with them. This also has
performance benefits when sorting smaller integer types, since the current
representation of nulls in two's complement (Long.MIN_VALUE) otherwise forces a
full-width radix sort.
This strategy for nulls does mean the sort is no longer stable. cc davies
## How was this patch tested?
Existing randomized sort tests for correctness. I also tested some TPCDS
queries and there does not seem to be any significant regression for non-null
sorts.
Some test queries (best of 5 runs each).
Before change:
scala> val start = System.nanoTime; spark.range(5000000).selectExpr("if(id
> 5, cast(hash(id) as long), NULL) as h").coalesce(1).orderBy("h").collect();
(System.nanoTime - start) / 1e6
start: Long = 3190437233227987
res3: Double = 4716.471091
After change:
scala> val start = System.nanoTime; spark.range(5000000).selectExpr("if(id
> 5, cast(hash(id) as long), NULL) as h").coalesce(1).orderBy("h").collect();
(System.nanoTime - start) / 1e6
start: Long = 3190367870952791
res4: Double = 2981.143045
Author: Eric Liang <[email protected]>
Closes #13161 from ericl/sc-2998.
commit 3fd2ff4dd85633af49865456a52bf0c09c99708b
Author: Dongjoon Hyun <[email protected]>
Date: 2016-06-11T22:47:51Z
[SPARK-15807][SQL] Support varargs for dropDuplicates in Dataset/DataFrame
## What changes were proposed in this pull request?
This PR adds `varargs`-types `dropDuplicates` functions in
`Dataset/DataFrame`. Currently, `dropDuplicates` supports only `Seq` or `Array`.
**Before**
```scala
scala> val ds = spark.createDataFrame(Seq(("a", 1), ("b", 2), ("a", 2)))
ds: org.apache.spark.sql.DataFrame = [_1: string, _2: int]
scala> ds.dropDuplicates(Seq("_1", "_2"))
res0: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [_1: string,
_2: int]
scala> ds.dropDuplicates("_1", "_2")
<console>:26: error: overloaded method value dropDuplicates with
alternatives:
(colNames:
Array[String])org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] <and>
(colNames:
Seq[String])org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] <and>
()org.apache.spark.sql.Dataset[org.apache.spark.sql.Row]
cannot be applied to (String, String)
ds.dropDuplicates("_1", "_2")
^
```
**After**
```scala
scala> val ds = spark.createDataFrame(Seq(("a", 1), ("b", 2), ("a", 2)))
ds: org.apache.spark.sql.DataFrame = [_1: string, _2: int]
scala> ds.dropDuplicates("_1", "_2")
res0: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [_1: string,
_2: int]
```
## How was this patch tested?
Pass the Jenkins tests with new testcases.
Author: Dongjoon Hyun <[email protected]>
Closes #13545 from dongjoon-hyun/SPARK-15807.
commit e1f986c7a3fcc3864d53ef99ef7f14fa4d262ac3
Author: Eric Liang <[email protected]>
Date: 2016-06-12T06:16:21Z
[SPARK-15860] Metrics for codegen size and perf
## What changes were proposed in this pull request?
Adds codahale metrics for the codegen source text size and how long it
takes to compile. The size is particularly interesting, since the JVM does have
hard limits on how large methods can get.
To simplify, I added the metrics under a statically-initialized source that
is always registered with SparkEnv.
## How was this patch tested?
Unit tests
Author: Eric Liang <[email protected]>
Closes #13586 from ericl/spark-15860.
commit 9e204c62c6800e03759e04ef68268105d4b86bf2
Author: hyukjinkwon <[email protected]>
Date: 2016-06-12T06:20:40Z
[SPARK-15840][SQL] Add two missing options in documentation and some option
related changes
## What changes were proposed in this pull request?
This PR
1. Adds the documentations for some missing options, `inferSchema` and
`mergeSchema` for Python and Scala.
2. Fiixes `[[DataFrame]]` to ```:class:`DataFrame` ``` so that this can be
shown
- from

- to (with class link)

(Please refer [the latest
documentation](https://people.apache.org/~pwendell/spark-nightly/spark-master-docs/latest/api/python/pyspark.sql.html))
3. Moves `mergeSchema` option to `ParquetOptions` with removing unused
options, `metastoreSchema` and `metastoreTableName`.
They are not used anymore. They were removed in
https://github.com/apache/spark/commit/e720dda42e806229ccfd970055c7b8a93eb447bf
and there are no use cases as below:
```bash
grep -r -e METASTORE_SCHEMA -e \"metastoreSchema\" -e
\"metastoreTableName\" -e METASTORE_TABLE_NAME .
```
```
./sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala:
private[sql] val METASTORE_SCHEMA = "metastoreSchema"
./sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala:
private[sql] val METASTORE_TABLE_NAME = "metastoreTableName"
./sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala:
ParquetFileFormat.METASTORE_TABLE_NAME -> TableIdentifier(
```
It only sets `metastoreTableName` in the last case but does not use the
table name.
4. Sets the correct default values (in the documentation) for `compression`
option for ORC(`snappy`, see
[OrcOptions.scala#L33-L42](https://github.com/apache/spark/blob/3ded5bc4db2badc9ff49554e73421021d854306b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcOptions.scala#L33-L42))
and Parquet(`the value specified in SQLConf`, see
[ParquetOptions.scala#L38-L47](https://github.com/apache/spark/blob/3ded5bc4db2badc9ff49554e73421021d854306b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala#L38-L47))
and `columnNameOfCorruptRecord` for JSON(`the value specified in SQLConf`, see
[JsonFileFormat.scala#L53-L55](https://github.com/apache/spark/blob/4538443e276597530a27c6922e48503677b13956/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala#L53-L55)
and
[JsonFileFormat.scala#L105-L106](https://github.com/apache/spark/blob/4538443e276597530a27c6922e48503677b13956/sql/core/src/main/scala/org/apach
e/spark/sql/execution/datasources/json/JsonFileFormat.scala#L105-L106)).
## How was this patch tested?
Existing tests should cover this.
Author: hyukjinkwon <[email protected]>
Author: Hyukjin Kwon <[email protected]>
Closes #13576 from HyukjinKwon/SPARK-15840.
commit 8cc22b0085475a188f229536b4f83988ae889a8e
Author: Imran Rashid <[email protected]>
Date: 2016-06-12T11:54:57Z
[SPARK-15878][CORE][TEST] fix cleanup in EventLoggingListenerSuite and
ReplayListenerSuite
## What changes were proposed in this pull request?
These tests weren't properly using `LocalSparkContext` so weren't cleaning
up correctly when tests failed.
## How was this patch tested?
Jenkins.
Author: Imran Rashid <[email protected]>
Closes #13602 from squito/SPARK-15878_cleanup_replaylistener.
commit 3fd3ee038b89821f51f30a4ecd4452b5b3bc6568
Author: bomeng <[email protected]>
Date: 2016-06-12T11:58:34Z
[SPARK-15781][DOCUMENTATION] remove deprecated environment variable doc
## What changes were proposed in this pull request?
Like `SPARK_JAVA_OPTS` and `SPARK_CLASSPATH`, we will remove the document
for `SPARK_WORKER_INSTANCES` to discourage user not to use them. If they are
actually used, SparkConf will show a warning message as before.
## How was this patch tested?
Manually tested.
Author: bomeng <[email protected]>
Closes #13533 from bomeng/SPARK-15781.
commit 50248dcfff3ba79b73323f3a804c1e19a8be6097
Author: bomeng <[email protected]>
Date: 2016-06-12T13:25:48Z
[SPARK-15806][DOCUMENTATION] update doc for SPARK_MASTER_IP
## What changes were proposed in this pull request?
SPARK_MASTER_IP is a deprecated environment variable. It is replaced by
SPARK_MASTER_HOST according to MasterArguments.scala.
## How was this patch tested?
Manually verified.
Author: bomeng <[email protected]>
Closes #13543 from bomeng/SPARK-15806.
commit f51dfe616b24b4234199c98ea857a586a93a889f
Author: Sean Owen <[email protected]>
Date: 2016-06-12T18:44:33Z
[SPARK-15086][CORE][STREAMING] Deprecate old Java accumulator API
## What changes were proposed in this pull request?
- Deprecate old Java accumulator API; should use Scala now
- Update Java tests and examples
- Don't bother testing old accumulator API in Java 8 (too)
- (fix a misspelling too)
## How was this patch tested?
Jenkins tests
Author: Sean Owen <[email protected]>
Closes #13606 from srowen/SPARK-15086.
commit 0a6f090837d8d5f6efa809fa976f09b3f0067602
Author: Sean Owen <[email protected]>
Date: 2016-06-12T18:46:33Z
[SPARK-15876][CORE] Remove support for "zk://" master URL
## What changes were proposed in this pull request?
Remove deprecated support for `zk://` master (`mesos://zk//` remains
supported)
## How was this patch tested?
Jenkins
Author: Sean Owen <[email protected]>
Closes #13625 from srowen/SPARK-15876.
commit 9770f6ee60f6834e4e1200234109120427a5cc0d
Author: frreiss <[email protected]>
Date: 2016-06-12T21:21:10Z
[SPARK-15370][SQL] Update RewriteCorrelatedScalarSubquery rule to fix COUNT
bug
## What changes were proposed in this pull request?
This pull request fixes the COUNT bug in the
`RewriteCorrelatedScalarSubquery` rule.
After this change, the rule tests the expression at the root of the
correlated subquery to determine whether the expression returns NULL on empty
input. If the expression does not return NULL, the rule generates additional
logic in the Project operator above the rewritten subquery. This additional
logic intercepts NULL values coming from the outer join and replaces them with
the value that the subquery's expression would return on empty input.
## How was this patch tested?
Added regression tests to cover all branches of the updated rule (see
changes to `SubquerySuite.scala`).
Ran all existing automated regression tests after merging with latest trunk.
Author: frreiss <[email protected]>
Closes #13155 from frreiss/master.
----
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]