Commit graph

21943 commits

Author SHA1 Message Date
Kazuaki Ishizaki d3c426a5b0 [SPARK-10878][CORE] Fix race condition when multiple clients resolves artifacts at the same time
## What changes were proposed in this pull request?

When multiple clients attempt to resolve artifacts via the `--packages` parameter, they could run into race condition when they each attempt to modify the dummy `org.apache.spark-spark-submit-parent-default.xml` file created in the default ivy cache dir.
This PR changes the behavior to encode UUID in the dummy module descriptor so each client will operate on a different resolution file in the ivy cache dir. In addition, this patch changes the behavior of when and which resolution files are cleaned to prevent accumulation of resolution files in the default ivy cache dir.

Since this PR is a successor of #18801, close #18801. Many codes were ported from #18801. **Many efforts were put here. I think this PR should credit to Victsm .**

## How was this patch tested?

added UT into `SparkSubmitUtilsSuite`

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

Closes #21251 from kiszk/SPARK-10878.
2018-05-10 14:41:55 -07:00
“attilapiros” 3e2600538e [SPARK-19181][CORE] Fixing flaky "SparkListenerSuite.local metrics"
## What changes were proposed in this pull request?

Sometimes "SparkListenerSuite.local metrics" test fails because the average of executorDeserializeTime is too short. As squito suggested to avoid these situations in one of the task a reference introduced to an object implementing a custom Externalizable.readExternal which sleeps 1ms before returning.

## How was this patch tested?

With unit tests (and checking the effect of this change to the average with a much larger sleep time).

Author: “attilapiros” <piros.attila.zsolt@gmail.com>
Author: Attila Zsolt Piros <2017933+attilapiros@users.noreply.github.com>

Closes #21280 from attilapiros/SPARK-19181.
2018-05-10 14:26:38 -07:00
mcheah 6282fc64e3 [SPARK-24137][K8S] Mount local directories as empty dir volumes.
## What changes were proposed in this pull request?

Drastically improves performance and won't cause Spark applications to fail because they write too much data to the Docker image's specific file system. The file system's directories that back emptydir volumes are generally larger and more performant.

## How was this patch tested?

Has been in use via the prototype version of Kubernetes support, but lost in the transition to here.

Author: mcheah <mcheah@palantir.com>

Closes #21238 from mccheah/mount-local-dirs.
2018-05-10 11:36:41 -07:00
Maxim Gekk f4fed05121 [SPARK-24171] Adding a note for non-deterministic functions
## What changes were proposed in this pull request?

I propose to add a clear statement for functions like `collect_list()` about non-deterministic behavior of such functions. The behavior must be taken into account by user while creating and running queries.

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

Closes #21228 from MaxGekk/deterministic-comments.
2018-05-10 09:44:49 -07:00
Marco Gaido 94d6714482 [SPARK-23907][SQL] Add regr_* functions
## What changes were proposed in this pull request?

The PR introduces regr_slope, regr_intercept, regr_r2, regr_sxx, regr_syy, regr_sxy, regr_avgx, regr_avgy, regr_count.

The implementation of this functions mirrors Hive's one in HIVE-15978.

## How was this patch tested?

added UT (values compared with Hive)

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21054 from mgaido91/SPARK-23907.
2018-05-10 20:38:52 +09:00
Dongjoon Hyun e3d4349947 [SPARK-22279][SQL] Enable convertMetastoreOrc by default
## What changes were proposed in this pull request?

We reverted `spark.sql.hive.convertMetastoreOrc` at https://github.com/apache/spark/pull/20536 because we should not ignore the table-specific compression conf. Now, it's resolved via [SPARK-23355](8aa1d7b0ed).

## How was this patch tested?

Pass the Jenkins.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #21186 from dongjoon-hyun/SPARK-24112.
2018-05-10 13:36:52 +08:00
Ryan Blue 62d01391fe [SPARK-24073][SQL] Rename DataReaderFactory to InputPartition.
## What changes were proposed in this pull request?

Renames:
* `DataReaderFactory` to `InputPartition`
* `DataReader` to `InputPartitionReader`
* `createDataReaderFactories` to `planInputPartitions`
* `createUnsafeDataReaderFactories` to `planUnsafeInputPartitions`
* `createBatchDataReaderFactories` to `planBatchInputPartitions`

This fixes the changes in SPARK-23219, which renamed ReadTask to
DataReaderFactory. The intent of that change was to make the read and
write API match (write side uses DataWriterFactory), but the underlying
problem is that the two classes are not equivalent.

ReadTask/DataReader function as Iterable/Iterator. One InputPartition is
a specific partition of the data to be read, in contrast to
DataWriterFactory where the same factory instance is used in all write
tasks. InputPartition's purpose is to manage the lifecycle of the
associated reader, which is now called InputPartitionReader, with an
explicit create operation to mirror the close operation. This was no
longer clear from the API because DataReaderFactory appeared to be more
generic than it is and it isn't clear why a set of them is produced for
a read.

## How was this patch tested?

Existing tests, which have been updated to use the new name.

Author: Ryan Blue <blue@apache.org>

Closes #21145 from rdblue/SPARK-24073-revert-data-reader-factory-rename.
2018-05-09 21:48:54 -07:00
Henry Robinson 9341c951e8 [SPARK-23852][SQL] Add test that fails if PARQUET-1217 is not fixed
## What changes were proposed in this pull request?

Add a new test that triggers if PARQUET-1217 - a predicate pushdown bug - is not fixed in Spark's Parquet dependency.

## How was this patch tested?

New unit test passes.

Author: Henry Robinson <henry@apache.org>

Closes #21284 from henryr/spark-23852.
2018-05-09 19:56:03 -07:00
wuyi 9e3bb31368 [SPARK-24141][CORE] Fix bug in CoarseGrainedSchedulerBackend.killExecutors
## What changes were proposed in this pull request?

In method *CoarseGrainedSchedulerBackend.killExecutors()*, `numPendingExecutors` should add
`executorsToKill.size` rather than `knownExecutors.size` if we do not adjust target number of executors.

## How was this patch tested?

N/A

Author: wuyi <ngone_5451@163.com>

Closes #21209 from Ngone51/SPARK-24141.
2018-05-09 15:44:36 -07:00
Shixiong Zhu fd1179c172 [SPARK-24214][SS] Fix toJSON for StreamingRelationV2/StreamingExecutionRelation/ContinuousExecutionRelation
## What changes were proposed in this pull request?

We should overwrite "otherCopyArgs" to provide the SparkSession parameter otherwise TreeNode.toJSON cannot get the full constructor parameter list.

## How was this patch tested?

The new unit test.

Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #21275 from zsxwing/SPARK-24214.
2018-05-09 11:32:17 -07:00
WeichenXu 7aaa148f59 [SPARK-14682][ML] Provide evaluateEachIteration method or equivalent for spark.ml GBTs
## What changes were proposed in this pull request?

Provide evaluateEachIteration method or equivalent for spark.ml GBTs.

## How was this patch tested?

UT.

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

Author: WeichenXu <weichen.xu@databricks.com>

Closes #21097 from WeichenXu123/GBTeval.
2018-05-09 11:09:19 -07:00
Marcelo Vanzin 628c7b5179 [SPARKR] Match pyspark features in SparkR communication protocol. 2018-05-09 10:47:35 -07:00
Marcelo Vanzin cc613b552e [PYSPARK] Update py4j to version 0.10.7. 2018-05-09 10:47:35 -07:00
Joseph K. Bradley 94155d0395 [MINOR][ML][DOC] Improved Naive Bayes user guide explanation
## What changes were proposed in this pull request?

This copies the material from the spark.mllib user guide page for Naive Bayes to the spark.ml user guide page.  I also improved the wording and organization slightly.

## How was this patch tested?

Built docs locally.

Author: Joseph K. Bradley <joseph@databricks.com>

Closes #21272 from jkbradley/nb-doc-update.
2018-05-09 10:34:57 -07:00
DB Tsai 6ea582e36a
[SPARK-24181][SQL] Better error message for writing sorted data
## What changes were proposed in this pull request?

The exception message should clearly distinguish sorting and bucketing in `save` and `jdbc` write.

When a user tries to write a sorted data using save or insertInto, it will throw an exception with message that `s"'$operation' does not support bucketing right now""`.

We should throw `s"'$operation' does not support sortBy right now""` instead.

## How was this patch tested?

More tests in `DataFrameReaderWriterSuite.scala`

Author: DB Tsai <d_tsai@apple.com>

Closes #21235 from dbtsai/fixException.
2018-05-09 09:15:16 -07:00
Ryan Blue cac9b1dea1 [SPARK-23972][BUILD][SQL] Update Parquet to 1.10.0.
## What changes were proposed in this pull request?

This updates Parquet to 1.10.0 and updates the vectorized path for buffer management changes. Parquet 1.10.0 uses ByteBufferInputStream instead of byte arrays in encoders. This allows Parquet to break allocations into smaller chunks that are better for garbage collection.

## How was this patch tested?

Existing Parquet tests. Running in production at Netflix for about 3 months.

Author: Ryan Blue <blue@apache.org>

Closes #21070 from rdblue/SPARK-23972-update-parquet-to-1.10.0.
2018-05-09 12:27:32 +08:00
Lu WANG 7e7350285d [SPARK-24132][ML] Instrumentation improvement for classification
## What changes were proposed in this pull request?

- Add OptionalInstrumentation as argument for getNumClasses in ml.classification.Classifier

- Change the function call for getNumClasses in train() in ml.classification.DecisionTreeClassifier, ml.classification.RandomForestClassifier, and ml.classification.NaiveBayes

- Modify the instrumentation creation in ml.classification.LinearSVC

- Change the log call in ml.classification.OneVsRest and ml.classification.LinearSVC

## How was this patch tested?

Manual.

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

Author: Lu WANG <lu.wang@databricks.com>

Closes #21204 from ludatabricks/SPARK-23686.
2018-05-08 21:20:58 -07:00
Dongjoon Hyun 9498e528d2 [SPARK-23355][SQL][DOC][FOLLOWUP] Add migration doc for TBLPROPERTIES
## What changes were proposed in this pull request?

In Apache Spark 2.4, [SPARK-23355](https://issues.apache.org/jira/browse/SPARK-23355) fixes a bug which ignores table properties during convertMetastore for tables created by STORED AS ORC/PARQUET.

For some Parquet tables having table properties like TBLPROPERTIES (parquet.compression 'NONE'), it was ignored by default before Apache Spark 2.4. After upgrading cluster, Spark will write uncompressed file which is different from Apache Spark 2.3 and old.

This PR adds a migration note for that.

## How was this patch tested?

N/A

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #21269 from dongjoon-hyun/SPARK-23355-DOC.
2018-05-09 08:39:46 +08:00
Maxim Gekk e3de6ab30d [SPARK-24068] Propagating DataFrameReader's options to Text datasource on schema inferring
## What changes were proposed in this pull request?

While reading CSV or JSON files, DataFrameReader's options are converted to Hadoop's parameters, for example there:
https://github.com/apache/spark/blob/branch-2.3/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala#L302

but the options are not propagated to Text datasource on schema inferring, for instance:
https://github.com/apache/spark/blob/branch-2.3/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala#L184-L188

The PR proposes propagation of user's options to Text datasource on scheme inferring in similar way as user's options are converted to Hadoop parameters if schema is specified.

## How was this patch tested?
The changes were tested manually by using https://github.com/twitter/hadoop-lzo:

```
hadoop-lzo> mvn clean package
hadoop-lzo> ln -s ./target/hadoop-lzo-0.4.21-SNAPSHOT.jar ./hadoop-lzo.jar
```
Create 2 test files in JSON and CSV format and compress them:
```shell
$ cat test.csv
col1|col2
a|1
$ lzop test.csv
$ cat test.json
{"col1":"a","col2":1}
$ lzop test.json
```
Run `spark-shell` with hadoop-lzo:
```
bin/spark-shell --jars ~/hadoop-lzo/hadoop-lzo.jar
```
reading compressed CSV and JSON without schema:
```scala
spark.read.option("io.compression.codecs", "com.hadoop.compression.lzo.LzopCodec").option("inferSchema",true).option("header",true).option("sep","|").csv("test.csv.lzo").show()
+----+----+
|col1|col2|
+----+----+
|   a|   1|
+----+----+
```
```scala
spark.read.option("io.compression.codecs", "com.hadoop.compression.lzo.LzopCodec").option("multiLine", true).json("test.json.lzo").printSchema
root
 |-- col1: string (nullable = true)
 |-- col2: long (nullable = true)
```

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

Closes #21182 from MaxGekk/text-options.
2018-05-09 08:32:20 +08:00
Yuming Wang 487faf17ab [SPARK-24117][SQL] Unified the getSizePerRow
## What changes were proposed in this pull request?

This pr unified the `getSizePerRow` because `getSizePerRow` is used in many places. For example:

1. [LocalRelation.scala#L80](f70f46d1e5/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala (L80))
2. [SizeInBytesOnlyStatsPlanVisitor.scala#L36](76b8b840dd/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala (L36))

## How was this patch tested?
Exist tests

Author: Yuming Wang <yumwang@ebay.com>

Closes #21189 from wangyum/SPARK-24117.
2018-05-08 23:43:02 +08:00
gatorsmile 2f6fe7d679 [SPARK-23094][SPARK-23723][SPARK-23724][SQL][FOLLOW-UP] Support custom encoding for json files
## What changes were proposed in this pull request?
This is to add a test case to check the behaviors when users write json in the specified UTF-16/UTF-32 encoding with multiline off.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #21254 from gatorsmile/followupSPARK-23094.
2018-05-08 21:24:35 +08:00
Liang-Chi Hsieh b54bbe57b3 [SPARK-24131][PYSPARK][FOLLOWUP] Add majorMinorVersion API to PySpark for determining Spark versions
## What changes were proposed in this pull request?

More close to Scala API behavior when can't parse input by throwing exception. Add tests.

## How was this patch tested?

Added tests.

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

Closes #21211 from viirya/SPARK-24131-followup.
2018-05-08 21:22:54 +08:00
yucai e17567ca78 [SPARK-24076][SQL] Use different seed in HashAggregate to avoid hash conflict
## What changes were proposed in this pull request?

HashAggregate uses the same hash algorithm and seed as ShuffleExchange, it may lead to bad hash conflict when shuffle.partitions=8192*n.

Considering below example:
```
SET spark.sql.shuffle.partitions=8192;
INSERT OVERWRITE TABLE target_xxx
SELECT
 item_id,
 auct_end_dt
FROM
 from source_xxx
GROUP BY
 item_id,
 auct_end_dt;
```

In the shuffle stage, if user sets the shuffle.partition = 8192, all tuples in the same partition will meet the following relationship:
```
hash(tuple x) = hash(tuple y) + n * 8192
```
Then in the next HashAggregate stage, all tuples from the same partition need be put into a 16K BytesToBytesMap (unsafeRowAggBuffer).

Here, the HashAggregate uses the same hash algorithm on the same expression as shuffle, and uses the same seed, and 16K = 8192 * 2, so actually, all tuples in the same parititon will only be hashed to 2 different places in the BytesToBytesMap. It is bad hash conflict. With BytesToBytesMap growing, the conflict will always exist.

Before change:
<img width="334" alt="hash_conflict" src="https://user-images.githubusercontent.com/2989575/39250210-ed032d46-48d2-11e8-855a-c1afc2a0ceb5.png">

After change:
<img width="334" alt="no_hash_conflict" src="https://user-images.githubusercontent.com/2989575/39250218-f1cb89e0-48d2-11e8-9244-5a93c1e8b60d.png">

## How was this patch tested?

Unit tests and production cases.

Author: yucai <yyu1@ebay.com>

Closes #21149 from yucai/SPARK-24076.
2018-05-08 11:34:27 +02:00
Marcelo Vanzin 05eb19b6e0 [SPARK-24188][CORE] Restore "/version" API endpoint.
It was missing the jax-rs annotation.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #21245 from vanzin/SPARK-24188.

Change-Id: Ib338e34b363d7c729cc92202df020dc51033b719
2018-05-08 14:32:04 +08:00
Henry Robinson cd12c5c3ec [SPARK-24128][SQL] Mention configuration option in implicit CROSS JOIN error
## What changes were proposed in this pull request?

Mention `spark.sql.crossJoin.enabled` in error message when an implicit `CROSS JOIN` is detected.

## How was this patch tested?

`CartesianProductSuite` and `JoinSuite`.

Author: Henry Robinson <henry@apache.org>

Closes #21201 from henryr/spark-24128.
2018-05-08 12:21:33 +08:00
Lu WANG 0d63eb8888 [SPARK-23975][ML] Add support of array input for all clustering methods
## What changes were proposed in this pull request?

Add support for all of the clustering methods

## How was this patch tested?

unit tests added

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

Author: Lu WANG <lu.wang@databricks.com>

Closes #21195 from ludatabricks/SPARK-23975-1.
2018-05-07 20:08:41 -07:00
WeichenXu 76ecd09502 [SPARK-20114][ML] spark.ml parity for sequential pattern mining - PrefixSpan
## What changes were proposed in this pull request?

PrefixSpan API for spark.ml. New implementation instead of #20810

## How was this patch tested?

TestSuite added.

Author: WeichenXu <weichen.xu@databricks.com>

Closes #20973 from WeichenXu123/prefixSpan2.
2018-05-07 14:57:14 -07:00
WeichenXu f48bd6bdc5 [SPARK-22885][ML][TEST] ML test for StructuredStreaming: spark.ml.tuning
## What changes were proposed in this pull request?

ML test for StructuredStreaming: spark.ml.tuning

## How was this patch tested?

N/A

Author: WeichenXu <weichen.xu@databricks.com>

Closes #20261 from WeichenXu123/ml_stream_tuning_test.
2018-05-07 14:55:41 -07:00
hyukjinkwon 1c9c5de951 [SPARK-23291][SPARK-23291][R][FOLLOWUP] Update SparkR migration note for
## What changes were proposed in this pull request?

This PR fixes the migration note for SPARK-23291 since it's going to backport to 2.3.1. See the discussion in https://issues.apache.org/jira/browse/SPARK-23291

## How was this patch tested?

N/A

Author: hyukjinkwon <gurwls223@apache.org>

Closes #21249 from HyukjinKwon/SPARK-23291.
2018-05-07 14:52:14 -07:00
Jeff Zhang 56a52e0a58 [SPARK-15750][MLLIB][PYSPARK] Constructing FPGrowth fails when no numPartitions specified in pyspark
## What changes were proposed in this pull request?

Change FPGrowth from private to private[spark]. If no numPartitions is specified, then default value -1 is used. But -1 is only valid in the construction function of FPGrowth, but not in setNumPartitions. So I make this change and use the constructor directly rather than using set method.
## How was this patch tested?

Unit test is added

Author: Jeff Zhang <zjffdu@apache.org>

Closes #13493 from zjffdu/SPARK-15750.
2018-05-07 14:47:58 -07:00
Bruce Robbins d83e963724 [SPARK-24043][SQL] Interpreted Predicate should initialize nondeterministic expressions
## What changes were proposed in this pull request?

When creating an InterpretedPredicate instance, initialize any Nondeterministic expressions in the expression tree to avoid java.lang.IllegalArgumentException on later call to eval().

## How was this patch tested?

- sbt SQL tests
- python SQL tests
- new unit test

Author: Bruce Robbins <bersprockets@gmail.com>

Closes #21144 from bersprockets/interpretedpredicate.
2018-05-07 17:54:39 +02:00
Herman van Hovell 4e861db5f1 [SPARK-16406][SQL] Improve performance of LogicalPlan.resolve
## What changes were proposed in this pull request?

`LogicalPlan.resolve(...)` uses linear searches to find an attribute matching a name. This is fine in normal cases, but gets problematic when you try to resolve a large number of columns on a plan with a large number of attributes.

This PR adds an indexing structure to `resolve(...)` in order to find potential matches quicker. This PR improves the reference resolution time for the following code by 4x (11.8s -> 2.4s):

``` scala
val n = 4000
val values = (1 to n).map(_.toString).mkString(", ")
val columns = (1 to n).map("column" + _).mkString(", ")
val query =
  s"""
     |SELECT $columns
     |FROM VALUES ($values) T($columns)
     |WHERE 1=2 AND 1 IN ($columns)
     |GROUP BY $columns
     |ORDER BY $columns
     |""".stripMargin

spark.time(sql(query))
```
## How was this patch tested?

Existing tests.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #14083 from hvanhovell/SPARK-16406.
2018-05-07 11:21:22 +02:00
Marco Gaido e35ad3cadd [SPARK-23930][SQL] Add slice function
## What changes were proposed in this pull request?

The PR add the `slice` function. The behavior of the function is based on Presto's one.

The function slices an array according to the requested start index and length.

## How was this patch tested?

added UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21040 from mgaido91/SPARK-23930.
2018-05-07 16:57:37 +09:00
Marco Gaido f06528015d [SPARK-24160][FOLLOWUP] Fix compilation failure
## What changes were proposed in this pull request?

SPARK-24160 is causing a compilation failure (after SPARK-24143 was merged). This fixes the issue.

## How was this patch tested?

building successfully

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21256 from mgaido91/SPARK-24160_FOLLOWUP.
2018-05-07 15:42:10 +08:00
Gabor Somogyi c5981976f1 [SPARK-23775][TEST] Make DataFrameRangeSuite not flaky
## What changes were proposed in this pull request?

DataFrameRangeSuite.test("Cancelling stage in a query with Range.") stays sometimes in an infinite loop and times out the build.

There were multiple issues with the test:

1. The first valid stageId is zero when the test started alone and not in a suite and the following code waits until timeout:

```
eventually(timeout(10.seconds), interval(1.millis)) {
  assert(DataFrameRangeSuite.stageToKill > 0)
}
```

2. The `DataFrameRangeSuite.stageToKill` was overwritten by the task's thread after the reset which ended up in canceling the same stage 2 times. This caused the infinite wait.

This PR solves this mentioned flakyness by removing the shared `DataFrameRangeSuite.stageToKill` and using `onTaskStart` where stage ID is provided. In order to make sure cancelStage called for all stages `waitUntilEmpty` is called on `ListenerBus`.

In [PR20888](https://github.com/apache/spark/pull/20888) this tried to get solved by:
* Stopping the executor thread with `wait`
* Wait for all `cancelStage` called
* Kill the executor thread by setting `SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL`

but the thread killing left the shared `SparkContext` sometimes in a state where further jobs can't be submitted. As a result DataFrameRangeSuite.test("Cancelling stage in a query with Range.") test passed properly but the next test inside the suite was hanging.

## How was this patch tested?

Existing unit test executed 10k times.

Author: Gabor Somogyi <gabor.g.somogyi@gmail.com>

Closes #21214 from gaborgsomogyi/SPARK-23775_1.
2018-05-07 14:45:14 +08:00
Josh Rosen d2aa859b4f [SPARK-24160] ShuffleBlockFetcherIterator should fail if it receives zero-size blocks
## What changes were proposed in this pull request?

This patch modifies `ShuffleBlockFetcherIterator` so that the receipt of zero-size blocks is treated as an error. This is done as a preventative measure to guard against a potential source of data loss bugs.

In the shuffle layer, we guarantee that zero-size blocks will never be requested (a block containing zero records is always 0 bytes in size and is marked as empty such that it will never be legitimately requested by executors). However, the existing code does not fully take advantage of this invariant in the shuffle-read path: the existing code did not explicitly check whether blocks are non-zero-size.

Additionally, our decompression and deserialization streams treat zero-size inputs as empty streams rather than errors (EOF might actually be treated as "end-of-stream" in certain layers (longstanding behavior dating to earliest versions of Spark) and decompressors like Snappy may be tolerant to zero-size inputs).

As a result, if some other bug causes legitimate buffers to be replaced with zero-sized buffers (due to corruption on either the send or receive sides) then this would translate into silent data loss rather than an explicit fail-fast error.

This patch addresses this problem by adding a `buf.size != 0` check. See code comments for pointers to tests which guarantee the invariants relied on here.

## How was this patch tested?

Existing tests (which required modifications, since some were creating empty buffers in mocks). I also added a test to make sure we fail on zero-size blocks.

To test that the zero-size blocks are indeed a potential corruption source, I manually ran a workload in `spark-shell` with a modified build which replaces all buffers with zero-size buffers in the receive path.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #21219 from JoshRosen/SPARK-24160.
2018-05-07 14:34:03 +08:00
Kazuaki Ishizaki 7564a9a706 [SPARK-23921][SQL] Add array_sort function
## What changes were proposed in this pull request?

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

The function sorts the input array in ascending order. The elements of the input array must be orderable. Null elements will be placed at the end of the returned array.

## How was this patch tested?

Added UTs

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

Closes #21021 from kiszk/SPARK-23921.
2018-05-07 15:22:23 +09:00
jinxing 889f6cc10c [SPARK-24143] filter empty blocks when convert mapstatus to (blockId, size) pair
## What changes were proposed in this pull request?

In current code(`MapOutputTracker.convertMapStatuses`), mapstatus are converted to (blockId, size) pair for all blocks – no matter the block is empty or not, which result in OOM when there are lots of consecutive empty blocks, especially when adaptive execution is enabled.

(blockId, size) pair is only used in `ShuffleBlockFetcherIterator` to control shuffle-read and only non-empty block request is sent. Can we just filter out the empty blocks in MapOutputTracker.convertMapStatuses and save memory?

## How was this patch tested?

not added yet.

Author: jinxing <jinxing6042@126.com>

Closes #21212 from jinxing64/SPARK-24143.
2018-05-07 14:16:27 +08:00
Marcelo Vanzin a634d66ce7 [SPARK-24126][PYSPARK] Use build-specific temp directory for pyspark tests.
This avoids polluting and leaving garbage behind in /tmp, and allows the
usual build tools to clean up any leftover files.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #21198 from vanzin/SPARK-24126.
2018-05-07 13:00:18 +08:00
gatorsmile f38ea00e83 [SPARK-24017][SQL] Refactor ExternalCatalog to be an interface
## What changes were proposed in this pull request?
This refactors the external catalog to be an interface. It can be easier for the future work in the catalog federation. After the refactoring, `ExternalCatalog` is much cleaner without mixing the listener event generation logic.

## How was this patch tested?
The existing tests

Author: gatorsmile <gatorsmile@gmail.com>

Closes #21122 from gatorsmile/refactorExternalCatalog.
2018-05-06 20:41:32 -07:00
Huaxin Gao dd4b1b9c7c [SPARK-24185][SPARKR][SQL] add flatten function to SparkR
## What changes were proposed in this pull request?

add array flatten function to SparkR

## How was this patch tested?

Unit tests were added in R/pkg/tests/fulltests/test_sparkSQL.R

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

Closes #21244 from huaxingao/spark-24185.
2018-05-06 10:25:01 +08:00
Tathagata Das 47b5b68528 [SPARK-24157][SS] Enabled no-data batches in MicroBatchExecution for streaming aggregation and deduplication.
## What changes were proposed in this pull request?

This PR enables the MicroBatchExecution to run no-data batches if some SparkPlan requires running another batch to output results based on updated watermark / processing time. In this PR, I have enabled streaming aggregations and streaming deduplicates to automatically run addition batch even if new data is available. See https://issues.apache.org/jira/browse/SPARK-24156 for more context.

Major changes/refactoring done in this PR.
- Refactoring MicroBatchExecution - A major point of confusion in MicroBatchExecution control flow was always (at least to me) was that `populateStartOffsets` internally called `constructNextBatch` which was not obvious from just the name "populateStartOffsets" and made the control flow from the main trigger execution loop very confusing (main loop in `runActivatedStream` called `constructNextBatch` but only if `populateStartOffsets` hadn't already called it). Instead, the refactoring makes it cleaner.
    - `populateStartOffsets` only the updates `availableOffsets` and `committedOffsets`. Does not call `constructNextBatch`.
    - Main loop in `runActivatedStream` calls `constructNextBatch` which returns true or false reflecting whether the next batch is ready for executing. This method is now idempotent; if a batch has already been constructed, then it will always return true until the batch has been executed.
    - If next batch is ready then we call `runBatch` or sleep.
    - That's it.

- Refactoring watermark management logic - This has been refactored out from `MicroBatchExecution` in a separate class to simplify `MicroBatchExecution`.

- New method `shouldRunAnotherBatch` in `IncrementalExecution` - This returns true if there is any stateful operation in the last execution plan that requires another batch for state cleanup, etc. This is used to decide whether to construct a batch or not in `constructNextBatch`.

- Changes to stream testing framework - Many tests used CheckLastBatch to validate answers. This assumed that there will be no more batches after the last set of input has been processed, so the last batch is the one that has output corresponding to the last input. This is not true anymore. To account for that, I made two changes.
    - `CheckNewAnswer` is a new test action that verifies the new rows generated since the last time the answer was checked by `CheckAnswer`, `CheckNewAnswer` or `CheckLastBatch`. This is agnostic to how many batches occurred between the last check and now. To do make this easier, I added a common trait between MemorySink and MemorySinkV2 to abstract out some common methods.
    - `assertNumStateRows` has been updated in the same way to be agnostic to batches while checking what the total rows and how many state rows were updated (sums up updates since the last check).

## How was this patch tested?
- Changes made to existing tests - Tests have been changed in one of the following patterns.
    - Tests where the last input was given again to force another batch to be executed and state cleaned up / output generated, they were simplified by removing the extra input.
    - Tests using aggregation+watermark where CheckLastBatch were replaced with CheckNewAnswer to make them batch agnostic.
- New tests added to check whether the flag works for streaming aggregation and deduplication

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

Closes #21220 from tdas/SPARK-24157.
2018-05-04 16:35:24 -07:00
Jose Torres af4dc50280 [SPARK-24039][SS] Do continuous processing writes with multiple compute() calls
## What changes were proposed in this pull request?

Do continuous processing writes with multiple compute() calls.

The current strategy (before this PR) is hacky; we just call next() on an iterator which has already returned hasNext = false, knowing that all the nodes we whitelist handle this properly. This will have to be changed before we can support more complex query plans. (In particular, I have a WIP https://github.com/jose-torres/spark/pull/13 which should be able to support aggregates in a single partition with minimal additional work.)

Most of the changes here are just refactoring to accommodate the new model. The behavioral changes are:

* The writer now calls prev.compute(split, context) once per epoch within the epoch loop.
* ContinuousDataSourceRDD now spawns a ContinuousQueuedDataReader which is shared across multiple calls to compute() for the same partition.

## How was this patch tested?

existing unit tests

Author: Jose Torres <torres.joseph.f+github@gmail.com>

Closes #21200 from jose-torres/noAggr.
2018-05-04 14:14:40 -07:00
Thomas Graves d04806a23c [SPARK-24124] Spark history server should create spark.history.store.…
…path and set permissions properly

## What changes were proposed in this pull request?

Spark history server should create spark.history.store.path and set permissions properly. Note createdDirectories doesn't do anything if the directories are already created.  This does not stomp on the permissions if the user had manually created the directory before the history server could.

## How was this patch tested?

Manually tested in a 100 node cluster. Ensured directories created with proper permissions. Ensured restarted worked apps/temp directories worked as apps were read.

Author: Thomas Graves <tgraves@thirteenroutine.corp.gq1.yahoo.com>

Closes #21234 from tgravescs/SPARK-24124.
2018-05-04 13:29:47 -07:00
Wenchen Fan 4d5de4d303 [SPARK-23697][CORE] LegacyAccumulatorWrapper should define isZero correctly
## What changes were proposed in this pull request?

It's possible that Accumulators of Spark 1.x may no longer work with Spark 2.x. This is because `LegacyAccumulatorWrapper.isZero` may return wrong answer if `AccumulableParam` doesn't define equals/hashCode.

This PR fixes this by using reference equality check in `LegacyAccumulatorWrapper.isZero`.

## How was this patch tested?

a new test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21229 from cloud-fan/accumulator.
2018-05-04 19:20:15 +08:00
Arun Mahadevan 7f1b6b182e [SPARK-24136][SS] Fix MemoryStreamDataReader.next to skip sleeping if record is available
## What changes were proposed in this pull request?

Avoid unnecessary sleep (10 ms) in each invocation of MemoryStreamDataReader.next.

## How was this patch tested?

Ran ContinuousSuite from IDE.

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

Author: Arun Mahadevan <arunm@apache.org>

Closes #21207 from arunmahadevan/memorystream.
2018-05-04 16:02:21 +08:00
Wenchen Fan 0c23e254c3 [SPARK-24167][SQL] ParquetFilters should not access SQLConf at executor side
## What changes were proposed in this pull request?

This PR is extracted from #21190 , to make it easier to backport.

`ParquetFilters` is used in the file scan function, which is executed in executor side, so we can't call `conf.parquetFilterPushDownDate` there.

## How was this patch tested?

it's tested in #21190

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21224 from cloud-fan/minor2.
2018-05-04 09:27:14 +08:00
Wenchen Fan e646ae67f2 [SPARK-24168][SQL] WindowExec should not access SQLConf at executor side
## What changes were proposed in this pull request?

This PR is extracted from #21190 , to make it easier to backport.

`WindowExec#createBoundOrdering` is called on executor side, so we can't use `conf.sessionLocalTimezone` there.

## How was this patch tested?

tested in #21190

Author: Wenchen Fan <wenchen@databricks.com>

Closes #21225 from cloud-fan/minor3.
2018-05-03 17:27:13 -07:00
maryannxue e3201e165e [SPARK-24035][SQL] SQL syntax for Pivot
## What changes were proposed in this pull request?

Add SQL support for Pivot according to Pivot grammar defined by Oracle (https://docs.oracle.com/database/121/SQLRF/img_text/pivot_clause.htm) with some simplifications, based on our existing functionality and limitations for Pivot at the backend:
1. For pivot_for_clause (https://docs.oracle.com/database/121/SQLRF/img_text/pivot_for_clause.htm), the column list form is not supported, which means the pivot column can only be one single column.
2. For pivot_in_clause (https://docs.oracle.com/database/121/SQLRF/img_text/pivot_in_clause.htm), the sub-query form and "ANY" is not supported (this is only supported by Oracle for XML anyway).
3. For pivot_in_clause, aliases for the constant values are not supported.

The code changes are:
1. Add parser support for Pivot. Note that according to https://docs.oracle.com/database/121/SQLRF/statements_10002.htm#i2076542, Pivot cannot be used together with lateral views in the from clause. This restriction has been implemented in the Parser rule.
2. Infer group-by expressions: group-by expressions are not explicitly specified in SQL Pivot clause and need to be deduced based on this rule: https://docs.oracle.com/database/121/SQLRF/statements_10002.htm#CHDFAFIE, so we have to post-fix it at query analysis stage.
3. Override Pivot.resolved as "false": for the reason mentioned in [2] and the fact that output attributes change after Pivot being replaced by Project or Aggregate, we avoid resolving parent references until after Pivot has been resolved and replaced.
4. Verify aggregate expressions: only aggregate expressions with or without aliases can appear in the first part of the Pivot clause, and this check is performed as analysis stage.

## How was this patch tested?

A new test suite PivotSuite is added.

Author: maryannxue <maryann.xue@gmail.com>

Closes #21187 from maryannxue/spark-24035.
2018-05-03 17:05:02 -07:00
Imran Rashid 94641fe6cc [SPARK-23433][CORE] Late zombie task completions update all tasksets
Fetch failure lead to multiple tasksets which are active for a given
stage.  While there is only one "active" version of the taskset, the
earlier attempts can still have running tasks, which can complete
successfully.  So a task completion needs to update every taskset
so that it knows the partition is completed.  That way the final active
taskset does not try to submit another task for the same partition,
and so that it knows when it is completed and when it should be
marked as a "zombie".

Added a regression test.

Author: Imran Rashid <irashid@cloudera.com>

Closes #21131 from squito/SPARK-23433.
2018-05-03 10:59:18 -05:00