Commit graph

7051 commits

Author SHA1 Message Date
lichaoqun f69998ace6 [MINOR][DOC] update the condition description of BypassMergeSortShuffle
## What changes were proposed in this pull request?
These three condition descriptions should be updated, follow #23228  :
<li>no Ordering is specified,</li>
<li>no Aggregator is specified, and</li>
<li>the number of partitions is less than
<code>spark.shuffle.sort.bypassMergeThreshold</code>.
</li>
1、If the shuffle dependency specifies aggregation, but it only aggregates at the reduce-side, BypassMergeSortShuffle can still be used.
2、If the number of output partitions is spark.shuffle.sort.bypassMergeThreshold(eg.200), we can use BypassMergeSortShuffle.

## How was this patch tested?
N/A

Closes #23281 from lcqzte10192193/wid-lcq-1211.

Authored-by: lichaoqun <li.chaoqun@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-12-13 07:42:17 -06:00
Gabor Somogyi 6daa783094 [SPARK-26322][SS] Add spark.kafka.sasl.token.mechanism to ease delegation token configuration.
## What changes were proposed in this pull request?

When Kafka delegation token obtained, SCRAM `sasl.mechanism` has to be configured for authentication. This can be configured on the related source/sink which is inconvenient from user perspective. Such granularity is not required and this configuration can be implemented with one central parameter.

In this PR `spark.kafka.sasl.token.mechanism` added to configure this centrally (default: `SCRAM-SHA-512`).

## How was this patch tested?

Existing unit tests + on cluster.

Closes #23274 from gaborgsomogyi/SPARK-26322.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-12-12 16:45:50 -08:00
Luca Canali 2920438c43 [SPARK-25277][YARN] YARN applicationMaster metrics should not register static metrics
## What changes were proposed in this pull request?

YARN applicationMaster metrics registration introduced in SPARK-24594 causes further registration of static metrics (Codegenerator and HiveExternalCatalog) and of JVM metrics, which I believe do not belong in this context.
This looks like an unintended side effect of using the start method of [[MetricsSystem]].
A possible solution proposed here, is to introduce startNoRegisterSources to avoid these additional registrations of static sources and of JVM sources in the case of YARN applicationMaster metrics (this could be useful for other metrics that may be added in the future).

## How was this patch tested?

Manually tested on a YARN cluster,

Closes #22279 from LucaCanali/YarnMetricsRemoveExtraSourceRegistration.

Lead-authored-by: Luca Canali <luca.canali@cern.ch>
Co-authored-by: LucaCanali <luca.canali@cern.ch>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-12-12 16:18:22 -08:00
Yuanjian Li bd8da3799d [SPARK-26193][SQL][FOLLOW UP] Read metrics rename and display text changes
## What changes were proposed in this pull request?
Follow up pr for #23207, include following changes:

- Rename `SQLShuffleMetricsReporter` to `SQLShuffleReadMetricsReporter` to make it match with write side naming.
- Display text changes for read side for naming consistent.
- Rename function in `ShuffleWriteProcessor`.
- Delete `private[spark]` in execution package.

## How was this patch tested?

Existing tests.

Closes #23286 from xuanyuanking/SPARK-26193-follow.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-12-12 10:03:50 +08:00
mcheah 57d6fbfa8c [SPARK-26239] File-based secret key loading for SASL.
This proposes an alternative way to load secret keys into a Spark application that is running on Kubernetes. Instead of automatically generating the secret, the secret key can reside in a file that is shared between both the driver and executor containers.

Unit tests.

Closes #23252 from mccheah/auth-secret-with-file.

Authored-by: mcheah <mcheah@palantir.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-12-11 13:50:16 -08:00
Liang-Chi Hsieh a3bbca98d7 [SPARK-26265][CORE] Fix deadlock in BytesToBytesMap.MapIterator when locking both BytesToBytesMap.MapIterator and TaskMemoryManager
## What changes were proposed in this pull request?

In `BytesToBytesMap.MapIterator.advanceToNextPage`, We will first lock this `MapIterator` and then `TaskMemoryManager` when going to free a memory page by calling `freePage`. At the same time, it is possibly that another memory consumer first locks `TaskMemoryManager` and then this `MapIterator` when it acquires memory and causes spilling on this `MapIterator`.

So it ends with the `MapIterator` object holds lock to the `MapIterator` object and waits for lock on `TaskMemoryManager`, and the other consumer holds lock to `TaskMemoryManager` and waits for lock on the `MapIterator` object.

To avoid deadlock here, this patch proposes to keep reference to the page to free and free it after releasing the lock of `MapIterator`.

## How was this patch tested?

Added test and manually test by running the test 100 times to make sure there is no deadlock.

Closes #23272 from viirya/SPARK-26265.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-12-11 21:08:39 +08:00
韩田田00222924 82c1ac48a3 [SPARK-25696] The storage memory displayed on spark Application UI is…
… incorrect.

## What changes were proposed in this pull request?
In the reported heartbeat information, the unit of the memory data is bytes, which is converted by the formatBytes() function in the utils.js file before being displayed in the interface. The cardinality of the unit conversion in the formatBytes function is 1000, which should be 1024.
Change the cardinality of the unit conversion in the formatBytes function to 1024.

## How was this patch tested?
 manual tests

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

Closes #22683 from httfighter/SPARK-25696.

Lead-authored-by: 韩田田00222924 <han.tiantian@zte.com.cn>
Co-authored-by: han.tiantian@zte.com.cn <han.tiantian@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-12-10 18:27:01 -06:00
Reza Safi 90c77ea313 [SPARK-24958][CORE] Add memory from procfs to executor metrics.
This adds the entire memory used by spark’s executor (as measured by procfs) to the executor metrics.  The memory usage is collected from the entire process tree under the executor.  The metrics are subdivided into memory used by java, by python, and by other processes, to aid users in diagnosing the source of high memory usage.
The additional metrics are sent to the driver in heartbeats, using the mechanism introduced by SPARK-23429.  This also slightly extends that approach to allow one ExecutorMetricType to collect multiple metrics.

Added unit tests and also tested on a live cluster.

Closes #22612 from rezasafi/ptreememory2.

Authored-by: Reza Safi <rezasafi@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2018-12-10 11:14:11 -06:00
liuxian 9794923272 [MINOR][DOC] Update the condition description of serialized shuffle
## What changes were proposed in this pull request?
`1. The shuffle dependency specifies no aggregation or output ordering.`
If the shuffle dependency specifies aggregation, but it only aggregates at the reduce-side, serialized shuffle can still be used.
`3. The shuffle produces fewer than 16777216 output partitions.`
If the number of output partitions is 16777216 , we can use serialized shuffle.

We can see this mothod: `canUseSerializedShuffle`
## How was this patch tested?
N/A

Closes #23228 from 10110346/SerializedShuffle_doc.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-12-10 22:37:17 +08:00
10087686 42e8c381b1 [SPARK-26286][TEST] Add MAXIMUM_PAGE_SIZE_BYTES exception bound unit test
## What changes were proposed in this pull request?
Add MAXIMUM_PAGE_SIZE_BYTES Exception test

## How was this patch tested?
Existing tests

(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 #23226 from wangjiaochun/BytesToBytesMapSuite.

Authored-by: 10087686 <wang.jiaochun@zte.com.cn>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-10 22:28:26 +08:00
10087686 403c8d5a60
[SPARK-26287][CORE] Don't need to create an empty spill file when memory has no records
## What changes were proposed in this pull request?
 If there are no records in memory, then we don't need to create an empty temp spill file.

## How was this patch tested?
Existing tests

(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 #23225 from wangjiaochun/ShufflSorter.

Authored-by: 10087686 <wang.jiaochun@zte.com.cn>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-12-09 22:44:41 -08:00
Shahid ec506bd30c [SPARK-26283][CORE] Enable reading from open frames of zstd, when reading zstd compressed eventLog
## What changes were proposed in this pull request?
Root cause: Prior to Spark2.4, When we enable zst for eventLog compression, for inprogress application, It always throws exception in the Application UI, when we open from the history server. But after 2.4 it will display the UI information based on the completed frames in the zstd compressed eventLog. But doesn't read incomplete frames for inprogress application.
In this PR, we have added 'setContinous(true)' for reading input stream from eventLog, so that it can read from open frames also. (By default 'isContinous=false' for zstd inputStream and when we try to read an open frame, it throws truncated error)

## How was this patch tested?
Test steps:
1) Add the configurations in the spark-defaults.conf
   (i) spark.eventLog.compress true
   (ii) spark.io.compression.codec zstd
2) Restart history server
3) bin/spark-shell
4) sc.parallelize(1 to 1000, 1000).count
5) Open app UI from the history server UI

**Before fix**
![screenshot from 2018-12-06 00-01-38](https://user-images.githubusercontent.com/23054875/49537340-bfe28b00-f8ee-11e8-9fca-6d42fdc89e1a.png)

**After fix:**
![screenshot from 2018-12-06 00-34-39](https://user-images.githubusercontent.com/23054875/49537353-ca9d2000-f8ee-11e8-803d-645897b9153b.png)

Closes #23241 from shahidki31/zstdEventLog.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-12-09 11:44:16 -06:00
Yuanjian Li 877f82cb30 [SPARK-26193][SQL] Implement shuffle write metrics in SQL
## What changes were proposed in this pull request?

1. Implement `SQLShuffleWriteMetricsReporter` on the SQL side as the customized `ShuffleWriteMetricsReporter`.
2. Add shuffle write metrics to `ShuffleExchangeExec`, and use these metrics to create corresponding `SQLShuffleWriteMetricsReporter` in shuffle dependency.
3. Rework on `ShuffleMapTask` to add new class named `ShuffleWriteProcessor` which control shuffle write process, we use sql shuffle write metrics by customizing a ShuffleWriteProcessor on SQL side.

## How was this patch tested?
Add UT in SQLMetricsSuite.
Manually test locally, update screen shot to document attached in JIRA.

Closes #23207 from xuanyuanking/SPARK-26193.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-12-09 10:49:15 +08:00
Gabor Somogyi 9b1f6c8bab [SPARK-26304][SS] Add default value to spark.kafka.sasl.kerberos.service.name parameter
## What changes were proposed in this pull request?

spark.kafka.sasl.kerberos.service.name is an optional parameter but most of the time value `kafka` has to be set. As I've written in the jira the following reasoning is behind:
* Kafka's configuration guide suggest the same value: https://kafka.apache.org/documentation/#security_sasl_kerberos_brokerconfig
* It would be easier for spark users by providing less configuration
* Other streaming engines are doing the same

In this PR I've changed the parameter from optional to `WithDefault` and set `kafka` as default value.

## How was this patch tested?

Available unit tests + on cluster.

Closes #23254 from gaborgsomogyi/SPARK-26304.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-12-07 13:58:02 -08:00
Shahid 3b8ae23735 [SPARK-26196][SPARK-26281][WEBUI] Total tasks title in the stage page is incorrect when there are failed or killed tasks and update duration metrics
## What changes were proposed in this pull request?
This PR fixes 3 issues
1) Total tasks message in the tasks table is incorrect, when there are failed or killed tasks
2) Sorting of the "Duration" column is not correct
3) Duration in the aggregated tasks summary table and the tasks table and not matching.

Total tasks  = numCompleteTasks +  numActiveTasks + numKilledTasks + numFailedTasks;

Corrected the duration metrics in the tasks table as executorRunTime based on the PR https://github.com/apache/spark/pull/23081

## How was this patch tested?
test step:
1)
```
bin/spark-shell
scala > sc.parallelize(1 to 100, 10).map{ x => throw new RuntimeException("Bad executor")}.collect()
```
![screenshot from 2018-11-28 07-26-00](https://user-images.githubusercontent.com/23054875/49123523-e2691880-f2de-11e8-9c16-60d1865e6e77.png)

After patch:
![screenshot from 2018-11-28 07-24-31](https://user-images.githubusercontent.com/23054875/49123525-e432dc00-f2de-11e8-89ca-4a53e19c9c18.png)

2)  Duration metrics:
Before patch:
![screenshot from 2018-12-06 03-25-14](https://user-images.githubusercontent.com/23054875/49546591-9e8d9900-f906-11e8-8a0b-157742c47655.png)

After patch:
![screenshot from 2018-12-06 03-23-14](https://user-images.githubusercontent.com/23054875/49546589-9cc3d580-f906-11e8-827f-52ef8ffdeaec.png)

Closes #23160 from shahidki31/totalTasks.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-12-07 14:31:35 -06:00
10087686 9b7679a97e [SPARK-26294][CORE] Delete Unnecessary If statement
## What changes were proposed in this pull request?
Delete unnecessary If statement, because it Impossible execution when
records less than or equal to zero.it is only execution when records begin zero.
...................
if (inMemSorter == null || inMemSorter.numRecords() <= 0) {
       return 0L;
 }
....................
if (inMemSorter.numRecords() > 0) {
.....................
}
## How was this patch tested?
Existing tests

(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 #23247 from wangjiaochun/inMemSorter.

Authored-by: 10087686 <wang.jiaochun@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-12-07 14:11:25 -06:00
Sahil Takiar 543577a1e8 [SPARK-24243][CORE] Expose exceptions from InProcessAppHandle
Adds a new method to SparkAppHandle called getError which returns
the exception (if present) that caused the underlying Spark app to
fail.

New tests added to SparkLauncherSuite for the new method.

Closes #21849

Closes #23221 from vanzin/SPARK-24243.

Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-12-07 10:34:33 -08:00
caoxuewen bfc5569a53 [SPARK-26289][CORE] cleanup enablePerfMetrics parameter from BytesToBytesMap
## What changes were proposed in this pull request?

`enablePerfMetrics `was originally designed in `BytesToBytesMap `to control `getNumHashCollisions  getTimeSpentResizingNs  getAverageProbesPerLookup`.

However, as the Spark version gradual progress.  this parameter is only used for `getAverageProbesPerLookup ` and always given to true when using `BytesToBytesMap`.

 it is also dangerous to determine whether `getAverageProbesPerLookup `opens and throws an `IllegalStateException `exception.
So this pr will be remove `enablePerfMetrics `parameter from `BytesToBytesMap`. thanks.

## How was this patch tested?

the existed test cases.

Closes #23244 from heary-cao/enablePerfMetrics.

Authored-by: caoxuewen <cao.xuewen@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-12-07 09:57:35 +08:00
Marcelo Vanzin dbd90e5440 [SPARK-26194][K8S] Auto generate auth secret for k8s apps.
This change modifies the logic in the SecurityManager to do two
things:

- generate unique app secrets also when k8s is being used
- only store the secret in the user's UGI on YARN

The latter is needed so that k8s won't unnecessarily create
k8s secrets for the UGI credentials when only the auth token
is stored there.

On the k8s side, the secret is propagated to executors using
an environment variable instead. This ensures it works in both
client and cluster mode.

Security doc was updated to mention the feature and clarify that
proper access control in k8s should be enabled for it to be secure.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #23174 from vanzin/SPARK-26194.
2018-12-06 14:17:13 -08:00
Imran Rashid 180f969c97 [SPARK-26094][CORE][STREAMING] createNonEcFile creates parent dirs.
## What changes were proposed in this pull request?

We explicitly avoid files with hdfs erasure coding for the streaming WAL
and for event logs, as hdfs EC does not support all relevant apis.
However, the new builder api used has different semantics -- it does not
create parent dirs, and it does not resolve relative paths.  This
updates createNonEcFile to have similar semantics to the old api.

## How was this patch tested?

Ran tests with the WAL pointed at a non-existent dir, which failed before this change.  Manually tested the new function with a relative path as well.
Unit tests via jenkins.

Closes #23092 from squito/SPARK-26094.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-12-04 14:35:04 -08:00
Shahid 35f9163adf [SPARK-26119][CORE][WEBUI] Task summary table should contain only successful tasks' metrics
## What changes were proposed in this pull request?

Task summary table in the stage page currently displays the summary of all the tasks. However, we should display the task summary of only successful tasks, to follow the behavior of previous versions of spark.

## How was this patch tested?
Added UT. attached screenshot
Before patch:
![screenshot from 2018-11-20 00-36-18](https://user-images.githubusercontent.com/23054875/48729339-62e3a580-ec5d-11e8-81f0-0d191a234ffe.png)

![screenshot from 2018-11-20 01-18-37](https://user-images.githubusercontent.com/23054875/48731112-41d18380-ec62-11e8-8c31-1ffbfa04e746.png)

Closes #23088 from shahidki31/summaryMetrics.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-12-04 11:00:58 -08:00
Gengliang Wang 6e4e70fe7b [SPARK-26235][CORE] Change log level for ClassNotFoundException/NoClassDefFoundError in SparkSubmit to Error
## What changes were proposed in this pull request?

In my local setup, I set log4j root category as ERROR (https://stackoverflow.com/questions/27781187/how-to-stop-info-messages-displaying-on-spark-console , first item show up if we google search "set spark log level".) When I run such command
```
spark-submit --class foo bar.jar
```
Nothing shows up, and the script exits.

After quick investigation, I think the log level for ClassNotFoundException/NoClassDefFoundError in SparkSubmit should be ERROR instead of WARN. Since the whole process exit because of the exception/error.

Before https://github.com/apache/spark/pull/20925, the message is not controlled by `log4j.rootCategory`.

## How was this patch tested?

Manual check.

Closes #23189 from gengliangwang/changeLogLevel.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-12-03 10:02:15 -06:00
pgandhi eebb940edb [SPARK-26253][WEBUI] Task Summary Metrics Table on Stage Page shows empty table when no data is present
Task Summary Metrics Table on Stage Page shows empty table when no data is present instead of showing a message.

## What changes were proposed in this pull request?

Added a custom message to show on the task summary metrics table as well as executor summary table when no data is present.

## How was this patch tested?

**Before:**

![49335550-29277d00-f615-11e8-8e62-a953e76bcebf](https://user-images.githubusercontent.com/22228190/49361520-425a2780-f702-11e8-8df4-08862ab6ceb8.png)

**After:**

<img width="1413" alt="screen shot 2018-12-03 at 1 56 09 pm" src="https://user-images.githubusercontent.com/22228190/49362019-8699f780-f703-11e8-93e1-d02df6572923.png">

Closes #23205 from pgandhi999/SPARK-26253.

Authored-by: pgandhi <pgandhi@oath.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-12-03 07:53:21 -06:00
liuxian 60e4239a1e [MINOR][DOC] Correct some document description errors
## What changes were proposed in this pull request?

Correct some document description errors.

## How was this patch tested?
N/A

Closes #23162 from 10110346/docerror.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-12-01 07:11:31 -06:00
caoxuewen 327ac83f5c [SPARK-26180][CORE][TEST] Reuse withTempDir function to the SparkCore test case
## What changes were proposed in this pull request?

Currently, the common `withTempDir` function is used in Spark SQL test cases. To handle `val dir = Utils. createTempDir()` and `Utils. deleteRecursively (dir)`. Unfortunately, the `withTempDir` function cannot be used in the Spark Core test case. This PR Sharing `withTempDir` function in Spark Sql and SparkCore  to clean up SparkCore test cases. thanks.

## How was this patch tested?

N / A

Closes #23151 from heary-cao/withCreateTempDir.

Authored-by: caoxuewen <cao.xuewen@zte.com.cn>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-01 16:34:11 +08:00
Shahid 8856e9f6a3 [SPARK-26219][CORE] Executor summary should get updated for failure jobs in the history server UI
The root cause of the problem is, whenever the taskEnd event comes after stageCompleted event, execSummary is updating only for live UI. we need to update for history UI too.

To see the previous discussion, refer: PR for https://github.com/apache/spark/pull/23038, https://issues.apache.org/jira/browse/SPARK-26100.

Added UT. Manually verified

Test step to reproduce:

```
bin/spark-shell --master yarn --conf spark.executor.instances=3
sc.parallelize(1 to 10000, 10).map{ x => throw new RuntimeException("Bad executor")}.collect()
```

Open Executors page from the History UI

Before patch:
![screenshot from 2018-11-29 22-13-34](https://user-images.githubusercontent.com/23054875/49246338-a21ead00-f43a-11e8-8214-f1020420be52.png)

After patch:
![screenshot from 2018-11-30 00-54-49](https://user-images.githubusercontent.com/23054875/49246353-aa76e800-f43a-11e8-98ef-7faecaa7a50e.png)

Closes #23181 from shahidki31/executorUpdate.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-11-30 15:22:37 -08:00
schintap 9b23be2e95 [SPARK-26201] Fix python broadcast with encryption
## What changes were proposed in this pull request?
Python with rpc and disk encryption enabled along with a python broadcast variable and just read the value back on the driver side the job failed with:

Traceback (most recent call last): File "broadcast.py", line 37, in <module> words_new.value File "/pyspark.zip/pyspark/broadcast.py", line 137, in value File "pyspark.zip/pyspark/broadcast.py", line 122, in load_from_path File "pyspark.zip/pyspark/broadcast.py", line 128, in load EOFError: Ran out of input

To reproduce use configs: --conf spark.network.crypto.enabled=true --conf spark.io.encryption.enabled=true

Code:

words_new = sc.broadcast(["scala", "java", "hadoop", "spark", "akka"])
words_new.value
print(words_new.value)

## How was this patch tested?
words_new = sc.broadcast([“scala”, “java”, “hadoop”, “spark”, “akka”])
textFile = sc.textFile(“README.md”)
wordCounts = textFile.flatMap(lambda line: line.split()).map(lambda word: (word + words_new.value[1], 1)).reduceByKey(lambda a, b: a+b)
 count = wordCounts.count()
 print(count)
 words_new.value
 print(words_new.value)

Closes #23166 from redsanket/SPARK-26201.

Authored-by: schintap <schintap@oath.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2018-11-30 12:48:56 -06:00
Gabor Somogyi 0166c7373e [SPARK-25501][SS] Add kafka delegation token support.
## What changes were proposed in this pull request?

It adds kafka delegation token support for structured streaming. Please see the relevant [SPIP](https://docs.google.com/document/d/1ouRayzaJf_N5VQtGhVq9FURXVmRpXzEEWYHob0ne3NY/edit?usp=sharing)

What this PR contains:
* Configuration parameters for the feature
* Delegation token fetching from broker
* Usage of token through dynamic JAAS configuration
* Minor refactoring in the existing code

What this PR doesn't contain:
* Documentation changes because design can change

## How was this patch tested?

Existing tests + added small amount of additional unit tests.

Because it's an external service integration mainly tested on cluster.
* 4 node cluster
* Kafka broker version 1.1.0
* Topic with 4 partitions
* security.protocol = SASL_SSL
* sasl.mechanism = SCRAM-SHA-256

An example of obtaining a token:
```
18/10/01 01:07:49 INFO kafka010.TokenUtil: TOKENID         HMAC                           OWNER           RENEWERS                  ISSUEDATE       EXPIRYDATE      MAXDATE
18/10/01 01:07:49 INFO kafka010.TokenUtil: D1-v__Q5T_uHx55rW16Jwg [hidden] User:user    []                        2018-10-01T01:07 2018-10-02T01:07 2018-10-08T01:07
18/10/01 01:07:49 INFO security.KafkaDelegationTokenProvider: Get token from Kafka: Kind: KAFKA_DELEGATION_TOKEN, Service: kafka.server.delegation.token, Ident: 44 31 2d 76 5f 5f 51 35 54 5f 75 48 78 35 35 72 57 31 36 4a 77 67
```

An example token usage:
```
18/10/01 01:08:07 INFO kafka010.KafkaSecurityHelper: Scram JAAS params: org.apache.kafka.common.security.scram.ScramLoginModule required tokenauth=true serviceName="kafka" username="D1-v__Q5T_uHx55rW16Jwg" password="[hidden]";
18/10/01 01:08:07 INFO kafka010.KafkaSourceProvider: Delegation token detected, using it for login.
```

Closes #22598 from gaborgsomogyi/SPARK-25501.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-11-29 18:00:47 -08:00
Wing Yew Poon 59741887e2 [SPARK-25905][CORE] When getting a remote block, avoid forcing a conversion to a ChunkedByteBuffer
## What changes were proposed in this pull request?

In `BlockManager`, `getRemoteValues` gets a `ChunkedByteBuffer` (by calling `getRemoteBytes`) and creates an `InputStream` from it. `getRemoteBytes`, in turn, gets a `ManagedBuffer` and converts it to a `ChunkedByteBuffer`.
Instead, expose a `getRemoteManagedBuffer` method so `getRemoteValues` can just get this `ManagedBuffer` and use its `InputStream`.
When reading a remote cache block from disk, this reduces heap memory usage significantly.
Retain `getRemoteBytes` for other callers.

## How was this patch tested?

Imran Rashid wrote an application (https://github.com/squito/spark_2gb_test/blob/master/src/main/scala/com/cloudera/sparktest/LargeBlocks.scala), that among other things, tests reading remote cache blocks. I ran this application, using 2500MB blocks, to test reading a cache block on disk. Without this change, with `--executor-memory 5g`, the test fails with `java.lang.OutOfMemoryError: Java heap space`. With the change, the test passes with `--executor-memory 2g`.
I also ran the unit tests in core. In particular, `DistributedSuite` has a set of tests that exercise the `getRemoteValues` code path. `BlockManagerSuite` has several tests that call `getRemoteBytes`; I left these unchanged, so `getRemoteBytes` still gets exercised.

Closes #23058 from wypoon/SPARK-25905.

Authored-by: Wing Yew Poon <wypoon@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2018-11-29 14:56:34 -06:00
Shahid 24e78b7f16 [SPARK-26186][SPARK-26184][CORE] Last updated time is not getting updated for the Inprogress application
## What changes were proposed in this pull request?

When the 'spark.history.fs.inProgressOptimization.enabled' is true, inProgress application's last updated time is not getting updated in the History UI. Also, during the cleaning time, InProgress application is getting removed from the listing, even if the last updated time is within the cleaning threshold time.

In this PR, if the fastInprogressOptimization enabled, we update the `lastUpdateTime` of the application as last scan time. This will update the `lastUpdateTime` in the historyUI and also while cleaning, it won't remove if the updateTime is within the cleaning interval

## How was this patch tested?
Added UT, attached screen shot.
Before patch:
![screenshot from 2018-11-27 23-22-38](https://user-images.githubusercontent.com/23054875/49101600-9b5a3380-f29c-11e8-8efc-3fb594e4279a.png)
![screenshot from 2018-11-27 23-20-11](https://user-images.githubusercontent.com/23054875/49101601-9c8b6080-f29c-11e8-928e-643a8c8f4477.png)

After Patch:
![screenshot from 2018-11-27 23-37-10](https://user-images.githubusercontent.com/23054875/49101911-669aac00-f29d-11e8-8181-663e4a08ab0e.png)
![screenshot from 2018-11-27 23-39-04](https://user-images.githubusercontent.com/23054875/49102010-a5306680-f29d-11e8-947a-e8a2a09a785a.png)

Closes #23158 from shahidki31/HistoryLastUpdateTime.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-11-29 09:48:18 -08:00
Brandon Krieger 87bd9c75df [SPARK-25998][CORE] Change TorrentBroadcast to hold weak reference of broadcast object
## What changes were proposed in this pull request?

This PR changes the broadcast object in TorrentBroadcast from a strong reference to a weak reference. This allows it to be garbage collected even if the Dataset is held in memory. This is ok, because the broadcast object can always be re-read.

## How was this patch tested?

Tested in Spark shell by taking a heap dump, full repro steps listed in https://issues.apache.org/jira/browse/SPARK-25998.

Closes #22995 from bkrieger/bk/torrent-broadcast-weak.

Authored-by: Brandon Krieger <bkrieger@palantir.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-28 07:22:48 -08:00
Mark Pavey ce61bac1d8 [SPARK-26137][CORE] Use Java system property "file.separator" inste…
… of hard coded "/" in DependencyUtils

## What changes were proposed in this pull request?

Use Java system property "file.separator" instead of hard coded "/" in DependencyUtils.

## How was this patch tested?

Manual test:
Submit Spark application via REST API that reads data from Elasticsearch using spark-elasticsearch library.

Without fix application fails with error:
18/11/22 10:36:20 ERROR Version: Multiple ES-Hadoop versions detected in the classpath; please use only one
jar:file:/C:/<...>/spark-2.4.0-bin-hadoop2.6/work/driver-20181122103610-0001/myApp-assembly-1.0.jar
jar:file:/C:/<...>/myApp-assembly-1.0.jar

18/11/22 10:36:20 ERROR Main: Application [MyApp] failed:
java.lang.Error: Multiple ES-Hadoop versions detected in the classpath; please use only one
jar:file:/C:/<...>/spark-2.4.0-bin-hadoop2.6/work/driver-20181122103610-0001/myApp-assembly-1.0.jar
jar:file:/C:/<...>/myApp-assembly-1.0.jar

	at org.elasticsearch.hadoop.util.Version.<clinit>(Version.java:73)
	at org.elasticsearch.hadoop.rest.RestService.findPartitions(RestService.java:214)
	at org.elasticsearch.spark.rdd.AbstractEsRDD.esPartitions$lzycompute(AbstractEsRDD.scala:73)
	at org.elasticsearch.spark.rdd.AbstractEsRDD.esPartitions(AbstractEsRDD.scala:72)
	at org.elasticsearch.spark.rdd.AbstractEsRDD.getPartitions(AbstractEsRDD.scala:44)
	at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:253)
	at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:251)
	at scala.Option.getOrElse(Option.scala:121)
	at org.apache.spark.rdd.RDD.partitions(RDD.scala:251)
	at org.apache.spark.rdd.MapPartitionsRDD.getPartitions(MapPartitionsRDD.scala:49)
	at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:253)
	at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:251)
	at scala.Option.getOrElse(Option.scala:121)
	at org.apache.spark.rdd.RDD.partitions(RDD.scala:251)
	at org.apache.spark.SparkContext.runJob(SparkContext.scala:2126)
	at org.apache.spark.rdd.RDD$$anonfun$collect$1.apply(RDD.scala:945)
	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
	at org.apache.spark.rdd.RDD.withScope(RDD.scala:363)
	at org.apache.spark.rdd.RDD.collect(RDD.scala:944)
	...
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at org.apache.spark.deploy.worker.DriverWrapper$.main(DriverWrapper.scala:65)
	at org.apache.spark.deploy.worker.DriverWrapper.main(DriverWrapper.scala)

With fix application runs successfully.

Closes #23102 from markpavey/JIRA_SPARK-26137_DependencyUtilsFileSeparatorFix.

Authored-by: Mark Pavey <markpavey@exabre.co.uk>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-28 07:19:47 -08:00
Sergey Zhemzhitsky 438f8fd675 [SPARK-26114][CORE] ExternalSorter's readingIterator field leak
## What changes were proposed in this pull request?

This pull request fixes [SPARK-26114](https://issues.apache.org/jira/browse/SPARK-26114) issue that occurs when trying to reduce the number of partitions by means of coalesce without shuffling after shuffle-based transformations.

The leak occurs because of not cleaning up `ExternalSorter`'s `readingIterator` field as it's done for its `map` and `buffer` fields.
Additionally there are changes to the `CompletionIterator` to prevent capturing its `sub`-iterator and holding it even after the completion iterator completes. It is necessary because in some cases, e.g. in case of standard scala's `flatMap` iterator (which is used is `CoalescedRDD`'s `compute` method) the next value of the main iterator is assigned to `flatMap`'s `cur` field only after it is available.
For DAGs where ShuffledRDD is a parent of CoalescedRDD it means that the data should be fetched from the map-side of the shuffle, but the process of fetching this data consumes quite a lot of memory in addition to the memory already consumed by the iterator held by `flatMap`'s `cur` field (until it is reassigned).

For the following data
```scala
import org.apache.hadoop.io._
import org.apache.hadoop.io.compress._
import org.apache.commons.lang._
import org.apache.spark._

// generate 100M records of sample data
sc.makeRDD(1 to 1000, 1000)
  .flatMap(item => (1 to 100000)
    .map(i => new Text(RandomStringUtils.randomAlphanumeric(3).toLowerCase) -> new Text(RandomStringUtils.randomAlphanumeric(1024))))
  .saveAsSequenceFile("/tmp/random-strings", Some(classOf[GzipCodec]))
```

and the following job
```scala
import org.apache.hadoop.io._
import org.apache.spark._
import org.apache.spark.storage._

val rdd = sc.sequenceFile("/tmp/random-strings", classOf[Text], classOf[Text])
rdd
  .map(item => item._1.toString -> item._2.toString)
  .repartitionAndSortWithinPartitions(new HashPartitioner(1000))
  .coalesce(10,false)
  .count
```

... executed like the following
```bash
spark-shell \
  --num-executors=5 \
  --executor-cores=2 \
  --master=yarn \
  --deploy-mode=client \
  --conf spark.executor.memoryOverhead=512 \
  --conf spark.executor.memory=1g \
  --conf spark.dynamicAllocation.enabled=false \
  --conf spark.executor.extraJavaOptions='-XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=/tmp -Dio.netty.noUnsafe=true'
```

... executors are always failing with OutOfMemoryErrors.

The main issue is multiple leaks of ExternalSorter references.
For example, in case of 2 tasks per executor it is expected to be 2 simultaneous instances of ExternalSorter per executor but heap dump generated on OutOfMemoryError shows that there are more ones.

![run1-noparams-dominator-tree-externalsorter](https://user-images.githubusercontent.com/1523889/48703665-782ce580-ec05-11e8-95a9-d6c94e8285ab.png)

P.S. This PR does not cover cases with CoGroupedRDDs which use ExternalAppendOnlyMap internally, which itself can lead to OutOfMemoryErrors in many places.

## How was this patch tested?

- Existing unit tests
- New unit tests
- Job executions on the live environment

Here is the screenshot before applying this patch
![run3-noparams-failure-ui-5x2-repartition-and-sort](https://user-images.githubusercontent.com/1523889/48700395-f769eb80-ebfc-11e8-831b-e94c757d416c.png)

Here is the screenshot after applying this patch
![run3-noparams-success-ui-5x2-repartition-and-sort](https://user-images.githubusercontent.com/1523889/48700610-7a8b4180-ebfd-11e8-9761-baaf38a58e66.png)
And in case of reducing the number of executors even more the job is still stable
![run3-noparams-success-ui-2x2-repartition-and-sort](https://user-images.githubusercontent.com/1523889/48700619-82e37c80-ebfd-11e8-98ed-a38e1f1f1fd9.png)

Closes #23083 from szhem/SPARK-26114-externalsorter-leak.

Authored-by: Sergey Zhemzhitsky <szhemzhitski@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-28 20:22:24 +08:00
Reynold Xin 6a064ba8f2 [SPARK-26141] Enable custom metrics implementation in shuffle write
## What changes were proposed in this pull request?
This is the write side counterpart to https://github.com/apache/spark/pull/23105

## How was this patch tested?
No behavior change expected, as it is a straightforward refactoring. Updated all existing test cases.

Closes #23106 from rxin/SPARK-26141.

Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: Reynold Xin <rxin@databricks.com>
2018-11-26 22:35:52 -08:00
Reynold Xin c995e0737d [SPARK-26140] followup: rename ShuffleMetricsReporter
## What changes were proposed in this pull request?
In https://github.com/apache/spark/pull/23105, due to working on two parallel PRs at once, I made the mistake of committing the copy of the PR that used the name ShuffleMetricsReporter for the interface, rather than the appropriate one ShuffleReadMetricsReporter. This patch fixes that.

## How was this patch tested?
This should be fine as long as compilation passes.

Closes #23147 from rxin/ShuffleReadMetricsReporter.

Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-11-26 17:01:56 -08:00
Marcelo Vanzin 6f1a1c1248 [SPARK-25451][HOTFIX] Call stage.attemptNumber instead of attemptId.
Closes #23149 from vanzin/SPARK-25451.hotfix.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-11-26 14:37:41 -08:00
Shahid fbf62b7100 [SPARK-25451][SPARK-26100][CORE] Aggregated metrics table doesn't show the right number of the total tasks
Total tasks in the aggregated table and the tasks table are not matching some times in the WEBUI.
We need to force update the executor summary of the particular executorId, when ever last task of that executor has reached. Currently it force update based on last task on the stage end. So, for some particular executorId task might miss at the stage end.

Tests to reproduce:
```
bin/spark-shell --master yarn --conf spark.executor.instances=3
sc.parallelize(1 to 10000, 10).map{ x => throw new RuntimeException("Bad executor")}.collect()
```
Before patch:
![screenshot from 2018-11-15 02-24-05](https://user-images.githubusercontent.com/23054875/48511776-b0d36480-e87d-11e8-89a8-ab97216e2c21.png)

After patch:
![screenshot from 2018-11-15 02-32-38](https://user-images.githubusercontent.com/23054875/48512141-c39a6900-e87e-11e8-8535-903e1d11d13e.png)

Closes #23038 from shahidki31/SPARK-25451.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-11-26 13:13:45 -08:00
pgandhi 76ef02e499 [SPARK-21809] Change Stage Page to use datatables to support sorting columns and searching
Support column sort, pagination and search for Stage Page using jQuery DataTable and REST API. Before this commit, the Stage page generated a hard-coded HTML table that could not support search. Supporting search and sort (over all applications rather than the 20 entries in the current page) in any case will greatly improve the user experience.
Created the stagespage-template.html for displaying application information in datables. Added REST api endpoint and javascript code to fetch data from the endpoint and display it on the data table.
Because of the above change, certain functionalities in the page had to be modified to support the addition of datatables. For example, the toggle checkbox 'Select All' previously would add the checked fields as columns in the Task table and as rows in the Summary Metrics table, but after the change, only columns are added in the Task Table as it got tricky to add rows dynamically in the datatables.

## How was this patch tested?
I have attached the screenshots of the Stage Page UI before and after the fix.
**Before:**

<img width="1419" alt="30564304-35991e1c-9c8a-11e7-850f-2ac7a347f600" src="https://user-images.githubusercontent.com/22228190/42137915-52054558-7d3a-11e8-8c85-433b2c94161d.png">

<img width="1435" alt="31360592-cbaa2bae-ad14-11e7-941d-95b4c7d14970" src="https://user-images.githubusercontent.com/22228190/42137928-79df500a-7d3a-11e8-9068-5630afe46ff3.png">

**After:**

<img width="1432" alt="31360591-c5650ee4-ad14-11e7-9665-5a08d8f21830" src="https://user-images.githubusercontent.com/22228190/42137936-a3fb9f42-7d3a-11e8-8502-22b3897cbf64.png">

<img width="1388" alt="31360604-d266b6b0-ad14-11e7-94b5-dcc4bb5443f4" src="https://user-images.githubusercontent.com/22228190/42137970-0fabc58c-7d3b-11e8-95ad-383b1bd1f106.png">

Closes #21688 from pgandhi999/SPARK-21809-2.3.

Authored-by: pgandhi <pgandhi@oath.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2018-11-26 14:08:32 -06:00
Katrin Leinweber c5daccb1da [MINOR] Update all DOI links to preferred resolver
## What changes were proposed in this pull request?

The DOI foundation recommends [this new resolver](https://www.doi.org/doi_handbook/3_Resolution.html#3.8). Accordingly, this PR re`sed`s all static DOI links ;-)

## How was this patch tested?

It wasn't, since it seems as safe as a "[typo fix](https://spark.apache.org/contributing.html)".

In case any of the files is included from other projects, and should be updated there, please let me know.

Closes #23129 from katrinleinweber/resolve-DOIs-securely.

Authored-by: Katrin Leinweber <9948149+katrinleinweber@users.noreply.github.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-25 17:43:55 -06:00
Gengliang Wang 0f56977f8c [SPARK-26156][WEBUI] Revise summary section of stage page
## What changes were proposed in this pull request?
In the summary section of stage page:
![image](https://user-images.githubusercontent.com/1097932/48935518-ebef2b00-ef42-11e8-8672-eaa4cac92c5e.png)

1.  the following metrics names can be revised:
Output => Output Size / Records
Shuffle Read: => Shuffle Read Size / Records
Shuffle Write => Shuffle Write Size / Records

After changes, the names are more clear, and consistent with the other names in the same page.

2. The associated job id URL should not contain the 3 tails spaces. Reduce the number of spaces to one, and exclude the space from link. This is consistent with SQL execution page.

## How was this patch tested?

Manual check:
![image](https://user-images.githubusercontent.com/1097932/48935538-f7425680-ef42-11e8-8b2a-a4f388d3ea52.png)

Closes #23125 from gengliangwang/reviseStagePage.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-24 09:12:05 -06:00
liuxian 7f5f7a967d [SPARK-25786][CORE] If the ByteBuffer.hasArray is false , it will throw UnsupportedOperationException for Kryo
## What changes were proposed in this pull request?
`deserialize` for kryo,  the type of input parameter is ByteBuffer, if it is not backed by an accessible byte array. it will throw `UnsupportedOperationException`

Exception Info:
```
java.lang.UnsupportedOperationException was thrown.
java.lang.UnsupportedOperationException
    at java.nio.ByteBuffer.array(ByteBuffer.java:994)
    at org.apache.spark.serializer.KryoSerializerInstance.deserialize(KryoSerializer.scala:362)
```

## How was this patch tested?

Added a unit test

Closes #22779 from 10110346/InputStreamKryo.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-24 09:10:15 -06:00
Reynold Xin de84899204 [SPARK-26140] Enable custom metrics implementation in shuffle reader
## What changes were proposed in this pull request?
This patch defines an internal Spark interface for reporting shuffle metrics and uses that in shuffle reader. Before this patch, shuffle metrics is tied to a specific implementation (using a thread local temporary data structure and accumulators). After this patch, callers that define their own shuffle RDDs can create a custom metrics implementation.

With this patch, we would be able to create a better metrics for the SQL layer, e.g. reporting shuffle metrics in the SQL UI, for each exchange operator.

Note that I'm separating read side and write side implementations, as they are very different, to simplify code review. Write side change is at https://github.com/apache/spark/pull/23106

## How was this patch tested?
No behavior change expected, as it is a straightforward refactoring. Updated all existing test cases.

Closes #23105 from rxin/SPARK-26140.

Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-11-23 14:14:21 -08:00
caoxuewen 466d011d35 [SPARK-26117][CORE][SQL] use SparkOutOfMemoryError instead of OutOfMemoryError when catch exception
## What changes were proposed in this pull request?

the pr #20014 which introduced `SparkOutOfMemoryError` to avoid killing the entire executor when an `OutOfMemoryError `is thrown.
so apply for memory using `MemoryConsumer. allocatePage `when  catch exception, use `SparkOutOfMemoryError `instead of `OutOfMemoryError`

## How was this patch tested?
N / A

Closes #23084 from heary-cao/SparkOutOfMemoryError.

Authored-by: caoxuewen <cao.xuewen@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-23 21:12:25 +08:00
jerryshao 76aae7f1fd
[SPARK-24553][UI][FOLLOWUP] Fix unnecessary UI redirect
## What changes were proposed in this pull request?

This PR is a follow-up PR of #21600 to fix the unnecessary UI redirect.

## How was this patch tested?

Local verification

Closes #23116 from jerryshao/SPARK-24553.

Authored-by: jerryshao <jerryshao@tencent.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-22 14:54:00 -08:00
Maxim Gekk 81550b38e4
[SPARK-26066][SQL] Move truncatedString to sql/catalyst and add spark.sql.debug.maxToStringFields conf
## What changes were proposed in this pull request?

In the PR, I propose:
- new SQL config `spark.sql.debug.maxToStringFields` to control maximum number fields up to which `truncatedString` cuts its input sequences.
- Moving `truncatedString` out of `core` to `sql/catalyst` because it is used only in the `sql/catalyst` packages for restricting number of fields converted to strings from `TreeNode` and expressions of`StructType`.

## How was this patch tested?

Added a test to `QueryExecutionSuite` to check that `spark.sql.debug.maxToStringFields` impacts to behavior of `truncatedString`.

Closes #23039 from MaxGekk/truncated-string-catalyst.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-21 11:16:54 -08:00
Shahid 540afc2b18 [SPARK-26109][WEBUI] Duration in the task summary metrics table and the task table are different
## What changes were proposed in this pull request?
Task summary table displays the summary of the task table in the stage page. However, the 'Duration' metrics of 'task summary' table and 'task table' are not matching. The reason is because, in the 'task summary' we display 'executorRunTime' as the duration, and in the 'task table' the actual duration of the task. Except duration metrics, all other metrics are properly displaying in the task summary.

In Spark2.2, used to show 'executorRunTime' as duration in the 'taskTable'. That is why, in summary metrics also the 'exeuctorRunTime' shows as the duration. So, we need to show 'executorRunTime' as the duration in the tasks table to follow the same behaviour as the previous versions of spark.

## How was this patch tested?
Before patch:

![screenshot from 2018-11-19 04-32-06](https://user-images.githubusercontent.com/23054875/48679263-1e4fff80-ebb4-11e8-9ed5-16d892039e01.png)

After patch:
![screenshot from 2018-11-19 04-37-39](https://user-images.githubusercontent.com/23054875/48679343-e39a9700-ebb4-11e8-8df9-9dc3a28d4bce.png)

Closes #23081 from shahidki31/duratinSummary.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-21 09:31:35 -06:00
Sean Owen 23bcd6ce45 [SPARK-26043][HOTFIX] Hotfix a change to SparkHadoopUtil that doesn't work in 2.11
## What changes were proposed in this pull request?

Hotfix a change to SparkHadoopUtil that doesn't work in 2.11

## How was this patch tested?

Existing tests.

Closes #23097 from srowen/SPARK-26043.2.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-20 18:03:54 -06:00
“attilapiros” ab61ddb34d [SPARK-26118][WEB UI] Introducing spark.ui.requestHeaderSize for setting HTTP requestHeaderSize
## What changes were proposed in this pull request?

Introducing spark.ui.requestHeaderSize for configuring Jetty's HTTP requestHeaderSize.
This way long authorization field does not lead to HTTP 413.

## How was this patch tested?

Manually with curl (which version must be at least 7.55).

With the original default value (8k limit):

```bash
# Starting history server with default requestHeaderSize
$ ./sbin/start-history-server.sh
starting org.apache.spark.deploy.history.HistoryServer, logging to /Users/attilapiros/github/spark/logs/spark-attilapiros-org.apache.spark.deploy.history.HistoryServer-1-apiros-MBP.lan.out

# Creating huge header
$ echo -n "X-Custom-Header: " > cookie
$ printf 'A%.0s' {1..9500} >> cookie

# HTTP GET with huge header fails with 431
$ curl  -H cookie http://458apiros-MBP.lan:18080/
<h1>Bad Message 431</h1><pre>reason: Request Header Fields Too Large</pre>

# The log contains the error
$ tail -1 /Users/attilapiros/github/spark/logs/spark-attilapiros-org.apache.spark.deploy.history.HistoryServer-1-apiros-MBP.lan.out
18/11/19 21:24:28 WARN HttpParser: Header is too large 8193>8192
```

After:

```bash
# Creating the history properties file with the increased requestHeaderSize
$ echo spark.ui.requestHeaderSize=10000 > history.properties

# Starting Spark History Server with the settings
$ ./sbin/start-history-server.sh --properties-file history.properties
starting org.apache.spark.deploy.history.HistoryServer, logging to /Users/attilapiros/github/spark/logs/spark-attilapiros-org.apache.spark.deploy.history.HistoryServer-1-apiros-MBP.lan.out

# HTTP GET with huge header gives back HTML5 (I have added here only just a part of the response)
$ curl  -H cookie http://458apiros-MBP.lan:18080/
<!DOCTYPE html><html>
      <head>...
         <link rel="shortcut icon" href="/static/spark-logo-77x50px-hd.png"></link>
        <title>History Server</title>
      </head>
      <body>
...
```

Closes #23090 from attilapiros/JettyHeaderSize.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2018-11-20 08:56:22 -06:00
Sean Owen 32365f8177 [SPARK-26090][CORE][SQL][ML] Resolve most miscellaneous deprecation and build warnings for Spark 3
## What changes were proposed in this pull request?

The build has a lot of deprecation warnings. Some are new in Scala 2.12 and Java 11. We've fixed some, but I wanted to take a pass at fixing lots of easy miscellaneous ones here.

They're too numerous and small to list here; see the pull request. Some highlights:

- `BeanInfo` is deprecated in 2.12, and BeanInfo classes are pretty ancient in Java. Instead, case classes can explicitly declare getters
- Eta expansion of zero-arg methods; foo() becomes () => foo() in many cases
- Floating-point Range is inexact and deprecated, like 0.0 to 100.0 by 1.0
- finalize() is finally deprecated (just needs to be suppressed)
- StageInfo.attempId was deprecated and easiest to remove here

I'm not now going to touch some chunks of deprecation warnings:

- Parquet deprecations
- Hive deprecations (particularly serde2 classes)
- Deprecations in generated code (mostly Thriftserver CLI)
- ProcessingTime deprecations (we may need to revive this class as internal)
- many MLlib deprecations because they concern methods that may be removed anyway
- a few Kinesis deprecations I couldn't figure out
- Mesos get/setRole, which I don't know well
- Kafka/ZK deprecations (e.g. poll())
- Kinesis
- a few other ones that will probably resolve by deleting a deprecated method

## How was this patch tested?

Existing tests, including manual testing with the 2.11 build and Java 11.

Closes #23065 from srowen/SPARK-26090.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-19 09:16:42 -06:00
Liu,Linhong b58b1fdf90 [SPARK-26068][CORE] ChunkedByteBufferInputStream should handle empty chunks correctly
## What changes were proposed in this pull request?

Empty chunk in ChunkedByteBuffer will truncate the ChunkedByteBufferInputStream.
The detail reason is described in: https://issues.apache.org/jira/browse/SPARK-26068

## How was this patch tested?
Modified current UT to cover this case.

Closes #23040 from LinhongLiu/fix-empty-chunked-byte-buffer.

Lead-authored-by: Liu,Linhong <liulinhong@baidu.com>
Co-authored-by: Xianjin YE <yexianjin@baidu.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-19 22:09:44 +08:00
Sean Owen ce2cdc36e2 [SPARK-26043][CORE] Make SparkHadoopUtil private to Spark
## What changes were proposed in this pull request?

Make SparkHadoopUtil private to Spark

## How was this patch tested?

Existing tests.

Closes #23066 from srowen/SPARK-26043.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-19 08:07:20 -06:00
Matt Molek 696b75a810 [SPARK-25934][MESOS] Don't propagate SPARK_CONF_DIR from spark submit
## What changes were proposed in this pull request?

Don't propagate SPARK_CONF_DIR to the driver in mesos cluster mode.

## How was this patch tested?

I built the 2.3.2 tag with this patch added and deployed a test job to a mesos cluster to confirm that the incorrect SPARK_CONF_DIR was no longer passed from the submit command.

Closes #22937 from mpmolek/fix-conf-dir.

Authored-by: Matt Molek <mpmolek@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-16 10:00:21 -06:00
Shanyu Zhao 9a5fda60e5 [SPARK-26011][SPARK-SUBMIT] Yarn mode pyspark app without python main resource does not honor "spark.jars.packages"
SparkSubmit determines pyspark app by the suffix of primary resource but Livy
uses "spark-internal" as the primary resource when calling spark-submit,
therefore args.isPython is set to false in SparkSubmit.scala.

In Yarn mode, SparkSubmit module is responsible for resolving maven coordinates
and adding them to "spark.submit.pyFiles" so that python's system path can be set correctly.

The fix is to resolve maven coordinates not only when args.isPython is true,
but also when primary resource is spark-internal.

Tested the patch with Livy submitting pyspark app, spark-submit, pyspark with or without packages config.

Signed-off-by: Shanyu Zhao <shzhaomicrosoft.com>

Closes #23009 from shanyu/shanyu-26011.

Authored-by: Shanyu Zhao <shzhao@microsoft.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-15 10:30:16 -06:00
Shahid cae5879dbe [SPARK-26044][WEBUI] Aggregated Metrics table sort based on executor ID
## What changes were proposed in this pull request?
Aggregated Metrics table in the stage page is not sorted based on the executorID properly. Because executorID is string and also the logs of the executors are in the same column.
In this PR, I created a new column for executor logs.

## How was this patch tested?
Before patch:
![screenshot from 2018-11-14 02-05-12](https://user-images.githubusercontent.com/23054875/48441529-caa77580-e7b1-11e8-90ea-b16f63438102.png)

After patch:
![screenshot from 2018-11-14 02-05-29](https://user-images.githubusercontent.com/23054875/48441540-d2671a00-e7b1-11e8-9059-890bfe80c961.png)

Closes #23024 from shahidki31/AggSort.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-15 10:27:57 -06:00
zhengruifeng 91405b3b6e [SPARK-22450][WIP][CORE][MLLIB][FOLLOWUP] Safely register MultivariateGaussian
## What changes were proposed in this pull request?
register following classes in Kryo:
"org.apache.spark.ml.stat.distribution.MultivariateGaussian",
"org.apache.spark.mllib.stat.distribution.MultivariateGaussian"

## How was this patch tested?
added tests

Due to existing module dependency, I can not import spark-core in mllib-local's testsuits, so I do not add testsuite in `org.apache.spark.ml.stat.distribution.MultivariateGaussianSuite`.
And I notice that class `ClusterStats` in `ClusteringEvaluator` is registered in a different way, should it be modified to keep in line with others in ML? srowen

Closes #22974 from zhengruifeng/kryo_MultivariateGaussian.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-15 09:22:31 -06:00
DB Tsai ad853c5678
[SPARK-25956] Make Scala 2.12 as default Scala version in Spark 3.0
## What changes were proposed in this pull request?

This PR makes Spark's default Scala version as 2.12, and Scala 2.11 will be the alternative version. This implies that Scala 2.12 will be used by our CI builds including pull request builds.

We'll update the Jenkins to include a new compile-only jobs for Scala 2.11 to ensure the code can be still compiled with Scala 2.11.

## How was this patch tested?

existing tests

Closes #22967 from dbtsai/scala2.12.

Authored-by: DB Tsai <d_tsai@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-14 16:22:23 -08:00
Yuanjian Li 2977e2312d [SPARK-25986][BUILD] Add rules to ban throw Errors in application code
## What changes were proposed in this pull request?

Add scala and java lint check rules to ban the usage of `throw new xxxErrors` and fix up all exists instance followed by https://github.com/apache/spark/pull/22989#issuecomment-437939830. See more details in https://github.com/apache/spark/pull/22969.

## How was this patch tested?

Local test with lint-scala and lint-java.

Closes #22989 from xuanyuanking/SPARK-25986.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-14 13:05:18 -08:00
Sean Owen 722369ee55 [SPARK-24421][BUILD][CORE] Accessing sun.misc.Cleaner in JDK11
…. Other related changes to get JDK 11 working, to test

## What changes were proposed in this pull request?

- Access `sun.misc.Cleaner` (Java 8) and `jdk.internal.ref.Cleaner` (JDK 9+) by reflection (note: the latter only works if illegal reflective access is allowed)
- Access `sun.misc.Unsafe.invokeCleaner` in Java 9+ instead of `sun.misc.Cleaner` (Java 8)

In order to test anything on JDK 11, I also fixed a few small things, which I include here:

- Fix minor JDK 11 compile issues
- Update scala plugin, Jetty for JDK 11, to facilitate tests too

This doesn't mean JDK 11 tests all pass now, but lots do. Note also that the JDK 9+ solution for the Cleaner has a big caveat.

## How was this patch tested?

Existing tests. Manually tested JDK 11 build and tests, and tests covering this change appear to pass. All Java 8 tests should still pass, but this change alone does not achieve full JDK 11 compatibility.

Closes #22993 from srowen/SPARK-24421.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-14 12:52:54 -08:00
ankurgupta 5f11e8c4cb [SPARK-25118][CORE] Persist Driver Logs in Client mode to Hdfs
Currently, we do not have a mechanism to collect driver logs if a user chooses
to run their application in client mode. This is a big issue as admin teams need
to create their own mechanisms to capture driver logs.

This commit adds a logger which, if enabled, adds a local log appender to the
root logger and asynchronously syncs it an application specific log file on hdfs
(Spark Driver Log Dir).

Additionally, this collects spark-shell driver logs at INFO level by default.
The change is that instead of setting root logger level to WARN, we will set the
consoleAppender threshold to WARN, in case of spark-shell. This ensures that
only WARN logs are printed on CONSOLE but other log appenders still capture INFO
(or the default log level logs).

1. Verified that logs are written to local and remote dir
2. Added a unit test case
3. Verified this for spark-shell, client mode and pyspark.
4. Verified in both non-kerberos and kerberos environment
5. Verified with following unexpected termination conditions: Ctrl + C, Driver
OOM, Large Log Files
6. Ran an application in spark-shell and ensured that driver logs were
captured at INFO level
7. Started the application at WARN level, programmatically changed the level to
INFO and ensured that logs on console were printed at INFO level

Closes #22504 from ankuriitg/ankurgupta/SPARK-25118.

Authored-by: ankurgupta <ankur.gupta@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-11-14 08:23:34 -08:00
Patrick Brown 6cd23482d1 [SPARK-25839][CORE] Implement use of KryoPool in KryoSerializer
## What changes were proposed in this pull request?

* Implement (optional) use of KryoPool in KryoSerializer, an alternative to the existing implementation of caching a Kryo instance inside KryoSerializerInstance
* Add config key & documentation of spark.kryo.pool in order to turn this on
* Add benchmark KryoSerializerBenchmark to compare new and old implementation
* Add results of benchmark

## How was this patch tested?

Added new tests inside KryoSerializerSuite to test the pool implementation as well as added the pool option to the existing regression testing for SPARK-7766

This is my original work and I license the work to the project under the project’s open source license.

Closes #22855 from patrickbrownsync/kryo-pool.

Authored-by: Patrick Brown <patrick.brown@blyncsy.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-10 12:51:24 -06:00
Sean Owen 2d085c13b7 [SPARK-25984][CORE][SQL][STREAMING] Remove deprecated .newInstance(), primitive box class constructor calls
## What changes were proposed in this pull request?

Deprecated in Java 11, replace Class.newInstance with Class.getConstructor.getInstance, and primtive wrapper class constructors with valueOf or equivalent

## How was this patch tested?

Existing tests.

Closes #22988 from srowen/SPARK-25984.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-10 09:52:14 -06:00
Dongjoon Hyun d66a4e82ec [SPARK-25102][SQL] Write Spark version to ORC/Parquet file metadata
## What changes were proposed in this pull request?

Currently, Spark writes Spark version number into Hive Table properties with `spark.sql.create.version`.
```
parameters:{
  spark.sql.sources.schema.part.0={
    "type":"struct",
    "fields":[{"name":"a","type":"integer","nullable":true,"metadata":{}}]
  },
  transient_lastDdlTime=1541142761,
  spark.sql.sources.schema.numParts=1,
  spark.sql.create.version=2.4.0
}
```

This PR aims to write Spark versions to ORC/Parquet file metadata with `org.apache.spark.sql.create.version` because we used `org.apache.` prefix in Parquet metadata already. It's different from Hive Table property key `spark.sql.create.version`, but it seems that we cannot change Hive Table property for backward compatibility.

After this PR, ORC and Parquet file generated by Spark will have the following metadata.

**ORC (`native` and `hive` implmentation)**
```
$ orc-tools meta /tmp/o
File Version: 0.12 with ...
...
User Metadata:
  org.apache.spark.sql.create.version=3.0.0
```

**PARQUET**
```
$ parquet-tools meta /tmp/p
...
creator:     parquet-mr version 1.10.0 (build 031a6654009e3b82020012a18434c582bd74c73a)
extra:       org.apache.spark.sql.create.version = 3.0.0
extra:       org.apache.spark.sql.parquet.row.metadata = {"type":"struct","fields":[{"name":"id","type":"long","nullable":false,"metadata":{}}]}
```

## How was this patch tested?

Pass the Jenkins with newly added test cases.

This closes #22255.

Closes #22932 from dongjoon-hyun/SPARK-25102.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-11-09 22:42:48 -08:00
William Montaz 25f506e2ad [SPARK-25973][CORE] Spark History Main page performance improvement
HistoryPage.scala counts applications (with a predicate depending on if it is displaying incomplete or complete applications) to check if it must display the dataTable.

Since it only checks if allAppsSize > 0, we could use exists method on the iterator. This way we stop iterating at the first occurence found.

Such a change has been relevant (roughly 12s improvement on page loading) on our cluster that runs tens of thousands of jobs per day.

Closes #22982 from Willymontaz/SPARK-25973.

Authored-by: William Montaz <w.montaz@criteo.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-09 08:02:53 -06:00
Sean Owen 0025a8397f [SPARK-25908][CORE][SQL] Remove old deprecated items in Spark 3
## What changes were proposed in this pull request?

- Remove some AccumulableInfo .apply() methods
- Remove non-label-specific multiclass precision/recall/fScore in favor of accuracy
- Remove toDegrees/toRadians in favor of degrees/radians (SparkR: only deprecated)
- Remove approxCountDistinct in favor of approx_count_distinct (SparkR: only deprecated)
- Remove unused Python StorageLevel constants
- Remove Dataset unionAll in favor of union
- Remove unused multiclass option in libsvm parsing
- Remove references to deprecated spark configs like spark.yarn.am.port
- Remove TaskContext.isRunningLocally
- Remove ShuffleMetrics.shuffle* methods
- Remove BaseReadWrite.context in favor of session
- Remove Column.!== in favor of =!=
- Remove Dataset.explode
- Remove Dataset.registerTempTable
- Remove SQLContext.getOrCreate, setActive, clearActive, constructors

Not touched yet

- everything else in MLLib
- HiveContext
- Anything deprecated more recently than 2.0.0, generally

## How was this patch tested?

Existing tests

Closes #22921 from srowen/SPARK-25908.

Lead-authored-by: Sean Owen <sean.owen@databricks.com>
Co-authored-by: hyukjinkwon <gurwls223@apache.org>
Co-authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-07 22:48:50 -06:00
koraseg 0a32238d03 [SPARK-25885][CORE][MINOR] HighlyCompressedMapStatus deserialization/construction optimization
## What changes were proposed in this pull request?

Removal of intermediate structures in HighlyCompressedMapStatus will speed up its creation and deserialization time.

https://issues.apache.org/jira/browse/SPARK-25885

## How was this patch tested?

Additional tests are not necessary for the patch.

Closes #22894 from Koraseg/mapStatusesOptimization.

Authored-by: koraseg <artem.kupchinsky@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-07 09:12:13 -06:00
Imran Rashid 8fbc1830f9 [SPARK-25904][CORE] Allocate arrays smaller than Int.MaxValue
JVMs can't allocate arrays of length exactly Int.MaxValue, so ensure we never try to allocate an array that big.  This commit changes some defaults & configs to gracefully fallover to something that doesn't require one large array in some cases; in other cases it simply improves an error message for cases which will still fail.

Closes #22818 from squito/SPARK-25827.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2018-11-07 13:18:52 +01:00
Dhruve Ashar fdd3bace1d [SPARK-22148][SPARK-15815][SCHEDULER] Acquire new executors to avoid hang because of blacklisting
## What changes were proposed in this pull request?
Every time a task is unschedulable because of the condition where no. of task failures < no. of executors available, we currently abort the taskSet - failing the job. This change tries to acquire new executors so that we can complete the job successfully. We try to acquire a new executor only when we can kill an existing idle executor. We fallback to the older implementation where we abort the job if we cannot find an idle executor.

## How was this patch tested?

I performed some manual tests to check and validate the behavior.

```scala
val rdd = sc.parallelize(Seq(1 to 10), 3)

import org.apache.spark.TaskContext

val mapped = rdd.mapPartitionsWithIndex ( (index, iterator) => { if (index == 2) { Thread.sleep(30 * 1000); val attemptNum = TaskContext.get.attemptNumber; if (attemptNum < 3) throw new Exception("Fail for blacklisting")};  iterator.toList.map (x => x + " -> " + index).iterator } )

mapped.collect
```

Closes #22288 from dhruve/bug/SPARK-22148.

Lead-authored-by: Dhruve Ashar <dhruveashar@gmail.com>
Co-authored-by: Dhruve Ashar <dhruve@users.noreply.github.com>
Co-authored-by: Tom Graves <tgraves@apache.org>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2018-11-06 08:25:32 -06:00
DB Tsai 3ed91c9b89
[SPARK-25946][BUILD] Upgrade ASM to 7.x to support JDK11
## What changes were proposed in this pull request?

Upgrade ASM to 7.x to support JDK11

## How was this patch tested?

Existing tests.

Closes #22953 from dbtsai/asm7.

Authored-by: DB Tsai <d_tsai@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2018-11-06 05:38:59 +00:00
Takuya UESHIN 78fa1be29b [SPARK-25926][CORE] Move config entries in core module to internal.config.
## What changes were proposed in this pull request?

Currently definitions of config entries in `core` module are in several files separately. We should move them into `internal/config` to be easy to manage.

## How was this patch tested?

Existing tests.

Closes #22928 from ueshin/issues/SPARK-25926/single_config_file.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-06 09:18:17 +08:00
Sean Owen c0d1bf0322 [MINOR] Fix typos and misspellings
## What changes were proposed in this pull request?

Fix typos and misspellings, per https://github.com/apache/spark-website/pull/158#issuecomment-435790366

## How was this patch tested?

Existing tests.

Closes #22950 from srowen/Typos.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-05 17:34:23 -06:00
Shahid fc65b4af00 [SPARK-25900][WEBUI] When the page number is more than the total page size, then fall back to the first page
## What changes were proposed in this pull request?

When we give the page number more than the maximum page number, webui is throwing an exception. It would be better if fall back to the default page, instead of throwing the exception in the web ui.

## How was this patch tested?
Before PR:
![screenshot from 2018-10-31 23-41-37](https://user-images.githubusercontent.com/23054875/47816448-354fbe80-dd79-11e8-83d8-6aab196642f7.png)

After PR:
![screenshot from 2018-10-31 23-54-23](https://user-images.githubusercontent.com/23054875/47816461-3ed92680-dd79-11e8-959d-0c531b3a6b2d.png)

Closes #22914 from shahidki31/pageFallBack.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-05 09:13:53 -06:00
liuxian 6c9e5ac9de [SPARK-25776][CORE]The disk write buffer size must be greater than 12
## What changes were proposed in this pull request?

 In `UnsafeSorterSpillWriter.java`, when we write a record to a spill file wtih ` void write(Object baseObject, long baseOffset,  int recordLength, long keyPrefix)`, `recordLength` and `keyPrefix`  will be  written  the disk write buffer  first, and these will take 12 bytes, so the disk write buffer size must be greater than 12.

 If `diskWriteBufferSize` is  10, it will print this exception info:

_java.lang.ArrayIndexOutOfBoundsException: 10
   at org.apache.spark.util.collection.unsafe.sort.UnsafeSorterSpillWriter.writeLongToBuffer (UnsafeSorterSpillWriter.java:91)
	at org.apache.spark.util.collection.unsafe.sort.UnsafeSorterSpillWriter.write(UnsafeSorterSpillWriter.java:123)
	at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.spillIterator(UnsafeExternalSorter.java:498)
	at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.spill(UnsafeExternalSorter.java:222)
	at org.apache.spark.memory.MemoryConsumer.spill(MemoryConsumer.java:65)_

## How was this patch tested?
Existing UT in `UnsafeExternalSorterSuite`

Closes #22754 from 10110346/diskWriteBufferSize.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
2018-11-05 01:55:13 +09:00
Yogesh Garg 0e318acd0c [SPARK-25901][CORE] Use only one thread in BarrierTaskContext companion object
## What changes were proposed in this pull request?

Now we use only one `timer` (and thus a backing thread) in `BarrierTaskContext` companion object, and the objects can add `timerTasks` to that `timer`.

## How was this patch tested?

This was tested manually by generating logs and seeing that they look the same as ones before, namely, that is, a partition waiting on another partition for 5seconds generates 4-5 log messages when the frequency of logging is set to 1second.

Closes #22912 from yogeshg/thread.

Authored-by: Yogesh Garg <1059168+yogeshg@users.noreply.github.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2018-11-03 14:03:50 +08:00
Shahid ed0c57e10d [SPARK-25861][MINOR][WEBUI] Remove unused refreshInterval parameter from the headerSparkPage method.
## What changes were proposed in this pull request?
'refreshInterval' is not used any where in the headerSparkPage method. So, we don't need to pass the parameter while calling the  'headerSparkPage' method.

## How was this patch tested?
Existing tests

Closes #22864 from shahidki31/unusedCode.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-02 17:17:48 -05:00
Imran Rashid 7ea594e787 [SPARK-25827][CORE] Avoid converting incoming encrypted blocks to byte buffers
## What changes were proposed in this pull request?

Avoid converting encrypted bocks to regular ByteBuffers, to ensure they can be sent over the network for replication & remote reads even when > 2GB.

Also updates some TODOs with links to a SPARK-25905 for improving the
handling here.

## How was this patch tested?

Tested on a cluster with encrypted data > 2GB (after SPARK-25904 was
applied as well).

Closes #22917 from squito/real_SPARK-25827.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-11-02 13:24:55 -07:00
Patrick Brown e9d3ca0b79 [SPARK-25837][CORE] Fix potential slowdown in AppStatusListener when cleaning up stages
## What changes were proposed in this pull request?

* Update `AppStatusListener` `cleanupStages` method to remove tasks for those stages in a single pass instead of 1 for each stage.
* This fixes an issue where the cleanupStages method would get backed up, causing a backup in the executor in ElementTrackingStore, resulting in stages and jobs not getting cleaned up properly.

Tasks seem most susceptible to this as there are a lot of them, however a similar issue could arise in other locations the `KVStore` `view` method is used. A broader fix might involve updates to `KVStoreView` and `InMemoryView` as it appears this interface and implementation can lead to multiple and inefficient traversals of the stored data.

## How was this patch tested?

Using existing tests in AppStatusListenerSuite

This is my original work and I license the work to the project under the project’s open source license.

Closes #22883 from patrickbrownsync/cleanup-stages-fix.

Authored-by: Patrick Brown <patrick.brown@blyncsy.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-11-01 09:34:29 -07:00
Shixiong Zhu 6be3cce751 [SPARK-25899][TESTS] Fix flaky CoarseGrainedSchedulerBackendSuite
## What changes were proposed in this pull request?

I saw CoarseGrainedSchedulerBackendSuite failed in my PR and finally reproduced the following error on a very busy machine:
```
sbt.ForkMain$ForkError: org.scalatest.exceptions.TestFailedDueToTimeoutException: The code passed to eventually never returned normally. Attempted 400 times over 10.009828643999999 seconds. Last failure message: ArrayBuffer("2", "0", "3") had length 3 instead of expected length 4.
```

The logs in this test shows executor 1 was not up when the test failed.
```
18/10/30 11:34:03.563 dispatcher-event-loop-12 INFO CoarseGrainedSchedulerBackend$DriverEndpoint: Registered executor NettyRpcEndpointRef(spark-client://Executor) (172.17.0.2:43656) with ID 2
18/10/30 11:34:03.593 dispatcher-event-loop-3 INFO CoarseGrainedSchedulerBackend$DriverEndpoint: Registered executor NettyRpcEndpointRef(spark-client://Executor) (172.17.0.2:43658) with ID 3
18/10/30 11:34:03.629 dispatcher-event-loop-6 INFO CoarseGrainedSchedulerBackend$DriverEndpoint: Registered executor NettyRpcEndpointRef(spark-client://Executor) (172.17.0.2:43654) with ID 0
18/10/30 11:34:03.885 pool-1-thread-1-ScalaTest-running-CoarseGrainedSchedulerBackendSuite INFO CoarseGrainedSchedulerBackendSuite:

===== FINISHED o.a.s.scheduler.CoarseGrainedSchedulerBackendSuite: 'compute max number of concurrent tasks can be launched' =====
```
And the following logs in executor 1 shows it was still doing the initialization when the timeout happened (at 18/10/30 11:34:03.885).
```
18/10/30 11:34:03.463 netty-rpc-connection-0 INFO TransportClientFactory: Successfully created connection to 54b6b6217301/172.17.0.2:33741 after 37 ms (0 ms spent in bootstraps)
18/10/30 11:34:03.959 main INFO DiskBlockManager: Created local directory at /home/jenkins/workspace/core/target/tmp/spark-383518bc-53bd-4d9c-885b-d881f03875bf/executor-61c406e4-178f-40a6-ac2c-7314ee6fb142/blockmgr-03fb84a1-eedc-4055-8743-682eb3ac5c67
18/10/30 11:34:03.993 main INFO MemoryStore: MemoryStore started with capacity 546.3 MB
```

Hence, I think our current 10 seconds is not enough on a slow Jenkins machine. This PR just increases the timeout from 10 seconds to 60 seconds to make the test more stable.

## How was this patch tested?

Jenkins

Closes #22910 from zsxwing/fix-flaky-test.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-10-31 15:14:10 -07:00
Marcelo Vanzin 68dde3481e [SPARK-23781][CORE] Merge token renewer functionality into HadoopDelegationTokenManager.
This avoids having two classes to deal with tokens; now the above
class is a one-stop shop for dealing with delegation tokens. The
YARN backend extends that class instead of doing composition like
before, resulting in a bit less code there too.

The renewer functionality is basically the same code that used to
be in YARN's AMCredentialRenewer. That is also the reason why the
public API of HadoopDelegationTokenManager is a little bit odd;
the YARN AM has some odd requirements for how this all should be
initialized, and the weirdness is needed currently to support that.

Tested:
- YARN with stress app for DT renewal
- Mesos and K8S with basic kerberos tests (both tgt and keytab)

Closes #22624 from vanzin/SPARK-23781.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2018-10-31 13:00:10 -05:00
Imran Rashid af3b816070 [SPARK-25855][CORE] Don't use erasure coding for event logs by default
## What changes were proposed in this pull request?

This turns off hdfs erasure coding by default for event logs, regardless of filesystem defaults.  Because this requires apis only available in hadoop 3, this uses reflection.  EC isn't a very good choice for event logs, as hflush() is a no-op, and so updates to the file are not visible for a long time.  This can still be configured by setting "spark.eventLog.allowErasureCoding=true", which will use filesystem defaults.

## How was this patch tested?

deployed a cluster with the changes with HDFS EC on.  By default, event logs didn't use EC, but configuration still would allow EC.  Also tried writing to the local fs (which doesn't support EC at all) and things worked fine.

Closes #22881 from squito/SPARK-25855.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-10-31 10:52:51 -07:00
Dongjoon Hyun e4cb42ad89
[SPARK-25891][PYTHON] Upgrade to Py4J 0.10.8.1
## What changes were proposed in this pull request?

Py4J 0.10.8.1 is released on October 21st and is the first release of Py4J to support Python 3.7 officially. We had better have this to get the official support. Also, there are some patches related to garbage collections.

https://www.py4j.org/changelog.html#py4j-0-10-8-and-py4j-0-10-8-1

## How was this patch tested?

Pass the Jenkins.

Closes #22901 from dongjoon-hyun/SPARK-25891.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-31 09:55:03 -07:00
Shixiong Zhu c36537fcfd
[SPARK-25773][CORE] Cancel zombie tasks in a result stage when the job finishes
## What changes were proposed in this pull request?

When a job finishes, there may be some zombie tasks still running due to stage retry. Since a result stage will never be used by other jobs, running these tasks are just wasting the cluster resource. This PR just asks TaskScheduler to cancel the running tasks of a result stage when it's already finished. Credits go to srinathshankar who suggested this idea to me.

This PR also fixes two minor issues while I'm touching DAGScheduler:
- Invalid spark.job.interruptOnCancel should not crash DAGScheduler.
- Non fatal errors should not crash DAGScheduler.

## How was this patch tested?

The new unit tests.

Closes #22771 from zsxwing/SPARK-25773.

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-10-30 10:48:04 -07:00
LantaoJin a129f07955 [SPARK-23429][CORE][FOLLOWUP] MetricGetter should rename to ExecutorMetricType in comments
## What changes were proposed in this pull request?

MetricGetter should rename to ExecutorMetricType in comments.

## How was this patch tested?

Just comments, no need to test.

Closes #22884 from LantaoJin/SPARK-23429_FOLLOWUP.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2018-10-30 11:01:55 -05:00
yucai 409d688fb6 [SPARK-25864][SQL][TEST] Make main args accessible for BenchmarkBase's subclass
## What changes were proposed in this pull request?

Set main args correctly in BenchmarkBase, to make it accessible for its subclass.
It will benefit:
- BuiltInDataSourceWriteBenchmark
- AvroWriteBenchmark

## How was this patch tested?

manual tests

Closes #22872 from yucai/main_args.

Authored-by: yucai <yyu1@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-29 20:00:31 +08:00
Shixiong Zhu 86d469aeaa [SPARK-25822][PYSPARK] Fix a race condition when releasing a Python worker
## What changes were proposed in this pull request?

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

0e07b483d2 is a patch to reproduce this issue.

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

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

## How was this patch tested?

Jenkins.

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

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
2018-10-26 13:53:51 +09:00
Devaraj K 46d2d2c74d [SPARK-24787][CORE] Revert hsync in EventLoggingListener and make FsHistoryProvider to read lastBlockBeingWritten data for logs
## What changes were proposed in this pull request?

`hsync` has been added as part of SPARK-19531 to get the latest data in the history sever ui, but that is causing the performance overhead and also leading to drop many history log events. `hsync` uses the force `FileChannel.force` to sync the data to the disk and happens for the data pipeline, it is costly operation and making the application to face overhead and drop the events.

I think getting the latest data in history server can be done in different way (no impact to application while writing events), there is an api `DFSInputStream.getFileLength()` which gives the file length including the `lastBlockBeingWrittenLength`(different from `FileStatus.getLen()`), this api can be used when the file status length and previously cached length are equal to verify whether any new data has been written or not, if there is any update in data length then the history server can update the in progress history log. And also I made this change as configurable with the default value false, and can be enabled for history server if users want to see the updated data in ui.

## How was this patch tested?

Added new test and verified manually, with the added conf `spark.history.fs.inProgressAbsoluteLengthCheck.enabled=true`, history server is reading the logs including the last block data which is being written and updating the Web UI with the latest data.

Closes #22752 from devaraj-kavali/SPARK-24787.

Authored-by: Devaraj K <devaraj@apache.org>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-10-25 13:16:08 -07:00
Behroz Sikander 002f9c169e [SPARK-24794][CORE] Driver launched through rest should use all masters
## What changes were proposed in this pull request?

In standalone cluster mode, one could launch driver with supervise mode
enabled. StandaloneRestServer class uses the host and port of current
master as the spark.master property while launching the driver
(even if you are running in HA mode). This class also ignores the
spark.master property passed as part of the request.

Due to the above problem, if the Spark masters switch due to some reason
and your driver is killed unexpectedly and relaunched, it will try to
connect to the master which is in the driver command specified as
-Dspark.master. But this master will be in STANDBY mode and after trying
multiple times, the SparkContext will kill itself (even though secondary
master was alive and healthy).

This change picks the spark.master property from request and uses it to
launch the driver process. Due to this, the driver process has both
masters in -Dspark.master property. Even if the masters switch, SparkContext
can still connect to the ALIVE master and work correctly.

## How was this patch tested?
This patch was manually tested on a standalone cluster running 2.2.1. It was rebased on current master and all tests were executed. I have added a unit test for this change (but since I am new I hope I have covered all).

Closes #21816 from bsikander/rest_driver_fix.

Authored-by: Behroz Sikander <behroz.sikander@sap.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-25 08:36:44 -05:00
Gengliang Wang b2e3256256 [SPARK-25490][SQL][TEST] Fix OOM of KryoBenchmark due to large 2D array and refactor it to use main method
## What changes were proposed in this pull request?

Before the code changes, I tried to run it with 8G memory:
```
build/sbt -mem 8000  "core/testOnly org.apache.spark.serializer.KryoBenchmark"
```
Still I got got OOM.

This is because the lengths of the arrays are random
669ade3a8e/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala (L90-L91)

And the 2D array is usually large: `10000 * Random.nextInt(0, 10000)`

This PR is to fix it and refactor it to use main method.

The benchmark result is also reason compared to the original one.

## How was this patch tested?

Run with
```
bin/spark-submit --class org.apache.spark.serializer.KryoBenchmark core/target/scala-2.11/spark-core_2.11-3.0.0-SNAPSHOT-tests.jar
```
and
```
SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt  "core/test:runMain org.apache.spark.serializer.KryoBenchmark"

Closes #22663 from gengliangwang/kyroBenchmark.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-24 16:56:17 -05:00
Sean Owen f83fedc9f2 [SPARK-25737][CORE] Remove JavaSparkContextVarargsWorkaround
## What changes were proposed in this pull request?

Remove JavaSparkContextVarargsWorkaround

## How was this patch tested?

Existing tests.

Closes #22729 from srowen/SPARK-25737.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-24 14:43:51 -05:00
Sean Owen b19a28dea0 [SPARK-16775][CORE] Remove deprecated accumulator v1 APIs
## What changes were proposed in this pull request?

Remove deprecated accumulator v1

## How was this patch tested?

Existing tests.

Closes #22730 from srowen/SPARK-16775.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-24 09:08:26 -05:00
Gengliang Wang 65a8d1b87f
[SPARK-25812][UI][TEST] Fix test failure in PagedTableSuite
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/22668, the PR was merged without PR builder test.  And there is a test failure:
https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.7/5070/testReport/org.apache.spark.ui/PagedTableSuite/pageNavigation/

This PR is to fix it.
## How was this patch tested?

Update the test case.

Closes #22808 from gengliangwang/fixPagedTableSuite.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-23 12:37:45 -07:00
Shixiong Zhu bd66c73025
[SPARK-25771][PYSPARK] Fix improper synchronization in PythonWorkerFactory
## What changes were proposed in this pull request?

Fix the following issues in PythonWorkerFactory
1. MonitorThread.run uses a wrong lock.
2. `createSimpleWorker` misses `synchronized` when updating `simpleWorkers`.

Other changes are just to improve the code style to make the thread-safe contract clear.

## How was this patch tested?

Jenkins

Closes #22770 from zsxwing/pwf.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2018-10-22 10:07:11 -07:00
liuxian 81a305dd04 [SPARK-25753][CORE] fix reading small files via BinaryFileRDD
## What changes were proposed in this pull request?

This is a follow up of #21601, `StreamFileInputFormat` and `WholeTextFileInputFormat` have the same problem.

`Minimum split size pernode 5123456 cannot be larger than maximum split size 4194304
java.io.IOException: Minimum split size pernode 5123456 cannot be larger than maximum split size 4194304
        at org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat.getSplits(CombineFileInputFormat.java: 201)
	at org.apache.spark.rdd.BinaryFileRDD.getPartitions(BinaryFileRDD.scala:52)
	at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:254)
	at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)
	at scala.Option.getOrElse(Option.scala:121)
	at org.apache.spark.rdd.RDD.partitions(RDD.scala:252)
	at org.apache.spark.SparkContext.runJob(SparkContext.scala:2138)`

## How was this patch tested?
Added a unit test

Closes #22725 from 10110346/maxSplitSize_node_rack.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2018-10-22 08:53:18 -05:00
shivusondur 4c6c6711d5 [SPARK-25675][SPARK JOB HISTORY] Job UI page does not show pagination with one page
## What changes were proposed in this pull request?
Currently in PagedTable.scala pageNavigation() method, if it is having only one page, they were not using the pagination.
Now it is made to use the pagination, even if it is having one page.

## How was this patch tested?
This tested with Spark webUI and History page in spark local setup.
![pagination](https://user-images.githubusercontent.com/7912929/46592799-93bfaf00-cae3-11e8-881a-ca2e93f17818.png)

Author: shivusondur <shivusondur@gmail.com>

Closes #22668 from shivusondur/pagination.
2018-10-21 11:44:48 -07:00
Imran Rashid 43717dee57 [SPARK-25704][CORE] Allocate a bit less than Int.MaxValue
JVMs don't you allocate arrays of length exactly Int.MaxValue, so leave
a little extra room.  This is necessary when reading blocks >2GB off
the network (for remote reads or for cache replication).

Unit tests via jenkins, ran a test with blocks over 2gb on a cluster

Closes #22705 from squito/SPARK-25704.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2018-10-19 12:52:41 -05:00
shivusondur f704ebe902 [SPARK-25683][CORE] Updated the log for the firstTime event Drop occurs
## What changes were proposed in this pull request?

When the first dropEvent occurs, LastReportTimestamp was printing in the log as
Wed Dec 31 16:00:00 PST 1969
(Dropped 1 events from eventLog since Wed Dec 31 16:00:00 PST 1969.)

The reason is that lastReportTimestamp initialized with 0.

Now log is updated to print "... since the application starts" if 'lastReportTimestamp' == 0.
this will happens first dropEvent occurs.

## How was this patch tested?
Manually verified.

Closes #22677 from shivusondur/AsyncEvent1.

Authored-by: shivusondur <shivusondur@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-10-18 15:05:56 -07:00
Gengliang Wang 1901f06211 [SPARK-25741][WEBUI] Long URLs are not rendered properly in web UI
## What changes were proposed in this pull request?

When the URL for description column in the table of job/stage page is long, WebUI doesn't render it properly.

![beforefix](https://user-images.githubusercontent.com/1097932/47009242-9323ba00-d16e-11e8-8262-0848d814442a.jpeg)

Both job and stage page are using the class `name-link` for the description URL,  so change the style of `a.name-link` to fix it.

## How was this patch tested?

Manual test on my local:
![afterfix](https://user-images.githubusercontent.com/1097932/47009269-a46cc680-d16e-11e8-9ff5-0318a20db634.jpeg)

Closes #22744 from gengliangwang/fixUILink.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-17 09:51:41 -05:00
Wenchen Fan 9690eba16e [SPARK-25680][SQL] SQL execution listener shouldn't happen on execution thread
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

existing tests.

Closes #22674 from cloud-fan/listener.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-17 16:06:07 +08:00
Stavros Kontopoulos bd2c447131 [SPARK-25394][CORE] Add an application status metrics source
- Exposes several metrics regarding application status as a source, useful to scrape them via jmx instead of mining the metrics rest api.  Example use case: prometheus + jmx exporter.
- Metrics are gathered when a job ends at the AppStatusListener side, could be more fine-grained but most metrics like tasks completed are also counted by executors. More metrics could be exposed in the future to avoid scraping executors in some scenarios.
- a config option `spark.app.status.metrics.enabled` is added to disable/enable these metrics, by default they are disabled.

This was manually tested with jmx source enabled and prometheus server on k8s:
![metrics](https://user-images.githubusercontent.com/7945591/45300945-63064d00-b518-11e8-812a-d9b4155ba0c0.png)
In the next pic the job delay is shown for repeated pi calculation (Spark action).
![pi](https://user-images.githubusercontent.com/7945591/45329927-89a1a380-b56b-11e8-9cc1-5e76cb83969f.png)

Closes #22381 from skonto/add_app_status_metrics.

Authored-by: Stavros Kontopoulos <stavros.kontopoulos@lightbend.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-10-16 14:58:26 -07:00
Sean Owen 703e6da1ec [SPARK-25705][BUILD][STREAMING][TEST-MAVEN] Remove Kafka 0.8 integration
## What changes were proposed in this pull request?

Remove Kafka 0.8 integration

## How was this patch tested?

Existing tests, build scripts

Closes #22703 from srowen/SPARK-25705.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-16 09:10:24 -05:00
Ilan Filonenko 6c9c84ffb9 [SPARK-23257][K8S] Kerberos Support for Spark on K8S
## What changes were proposed in this pull request?
This is the work on setting up Secure HDFS interaction with Spark-on-K8S.
The architecture is discussed in this community-wide google [doc](https://docs.google.com/document/d/1RBnXD9jMDjGonOdKJ2bA1lN4AAV_1RwpU_ewFuCNWKg)
This initiative can be broken down into 4 Stages

**STAGE 1**
- [x] Detecting `HADOOP_CONF_DIR` environmental variable and using Config Maps to store all Hadoop config files locally, while also setting `HADOOP_CONF_DIR` locally in the driver / executors

**STAGE 2**
- [x] Grabbing `TGT` from `LTC` or using keytabs+principle and creating a `DT` that will be mounted as a secret or using a pre-populated secret

**STAGE 3**
- [x] Driver

**STAGE 4**
- [x] Executor

## How was this patch tested?
Locally tested on a single-noded, pseudo-distributed Kerberized Hadoop Cluster
- [x] E2E Integration tests https://github.com/apache/spark/pull/22608
- [ ] Unit tests

## Docs and Error Handling?
- [x] Docs
- [x] Error Handling

## Contribution Credit
kimoonkim skonto

Closes #21669 from ifilonenko/secure-hdfs.

Lead-authored-by: Ilan Filonenko <if56@cornell.edu>
Co-authored-by: Ilan Filonenko <ifilondz@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-10-15 15:48:51 -07:00
gatorsmile 9426fd0c24 [SPARK-25372][YARN][K8S][FOLLOW-UP] Deprecate and generalize keytab / principal config
## What changes were proposed in this pull request?
Update the next version of Spark from 2.5 to 3.0

## How was this patch tested?
N/A

Closes #22717 from gatorsmile/followupSPARK-25372.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-14 15:20:01 +08:00
Gengliang Wang 26c1b959cf
[SPARK-25711][CORE] Improve start-history-server.sh: show usage User-Friendly and remove deprecated options
## What changes were proposed in this pull request?

Currently, if we try run
```
./start-history-server.sh -h
```
We will get such error
```
java.io.FileNotFoundException: File -h does not exist
```

1. This is not User-Friendly.  For option `-h` or `--help`, it should be parsed correctly and show the usage of the class/script.
2. We can remove deprecated options for setting event log directory through command line options.

After fix, we can get following output:
```
Usage: ./sbin/start-history-server.sh [options]

Options:
  --properties-file FILE      Path to a custom Spark properties file.
                              Default is conf/spark-defaults.conf.

Configuration options can be set by setting the corresponding JVM system property.
History Server options are always available; additional options depend on the provider.

History Server options:

  spark.history.ui.port              Port where server will listen for connections
                                     (default 18080)
  spark.history.acls.enable          Whether to enable view acls for all applications
                                     (default false)
  spark.history.provider             Name of history provider class (defaults to
                                     file system-based provider)
  spark.history.retainedApplications Max number of application UIs to keep loaded in memory
                                     (default 50)
FsHistoryProvider options:

  spark.history.fs.logDirectory      Directory where app logs are stored
                                     (default: file:/tmp/spark-events)
  spark.history.fs.updateInterval    How often to reload log data from storage
                                     (in seconds, default: 10)

```

## How was this patch tested?

Manual test

Closes #22699 from gengliangwang/refactorSHSUsage.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-13 13:34:31 -07:00
Sean Owen 1ddfab8c4f [SPARK-19287][CORE][STREAMING] JavaPairRDD flatMapValues requires function returning Iterable, not Iterator
## What changes were proposed in this pull request?

Fix old oversight in API: Java `flatMapValues` needs a `FlatMapFunction`

## How was this patch tested?

Existing tests.

Closes #22690 from srowen/SPARK-19287.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-12 18:10:59 -05:00
Yuming Wang e965fb55ac
[SPARK-25664][SQL][TEST] Refactor JoinBenchmark to use main method
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

manual tests

Closes #22661 from wangyum/SPARK-25664.

Lead-authored-by: Yuming Wang <yumwang@ebay.com>
Co-authored-by: Yuming Wang <wgyumg@gmail.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-12 16:08:12 -07:00
Shahid 8e039a7554 [SPARK-25697][CORE] When zstd compression enabled, InProgress application is throwing Error in the history webui
## What changes were proposed in this pull request?
When we enable event log compression and compression codec as 'zstd', we are unable to open the webui of the running application from the history server page.
The reason is that, Replay listener was unable to read from the zstd compressed eventlog due to the zstd frame was not finished yet. This causes truncated error while reading the eventLog.

So, when we try to open the WebUI from the History server page, it throws "truncated error ", and we never able to open running application in the webui, when we enable zstd compression.

In this PR, when the IO excpetion happens, and if it is a running application, we  log the error,
"Failed to read Spark event log: evetLogDirAppName.inprogress", instead of throwing exception.

## How was this patch tested?
Test steps:
1)spark.eventLog.compress =  true
2)spark.io.compression.codec = zstd
3)restart history server
4) launch bin/spark-shell
5) run some queries
6) Open history server page
7) click on the application

**Before fix:**
![screenshot from 2018-10-10 23-52-12](https://user-images.githubusercontent.com/23054875/46757387-9b4fa580-cce7-11e8-96ad-8938400483ed.png)

![screenshot from 2018-10-10 23-52-28](https://user-images.githubusercontent.com/23054875/46757393-a0145980-cce7-11e8-8cb0-44b583dde648.png)

**After fix:**

![screenshot from 2018-10-10 23-43-49](https://user-images.githubusercontent.com/23054875/46756971-6858e200-cce6-11e8-946c-0bffebb2cfba.png)

![screenshot from 2018-10-10 23-44-05](https://user-images.githubusercontent.com/23054875/46756981-6d1d9600-cce6-11e8-95ea-ff8339a2fdfd.png)

(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 #22689 from shahidki31/SPARK-25697.

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

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

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

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

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

**After this PR**

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

Closes #22645 from shahidki31/SPARK-25566.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-12 12:36:35 -05:00
Sean Owen 1bb63ae512 [SPARK-24109][CORE] Remove class SnappyOutputStreamWrapper
## What changes were proposed in this pull request?

Remove SnappyOutputStreamWrapper and other workaround now that new Snappy fixes these.
See also https://github.com/apache/spark/pull/21176 and comments it links to.

## How was this patch tested?

Existing tests

Closes #22691 from srowen/SPARK-24109.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-11 14:04:44 -07:00
Devaraj K 8a7872dc25 [SPARK-25636][CORE] spark-submit cuts off the failure reason when there is an error connecting to master
## What changes were proposed in this pull request?
Cause of the error is wrapped with SparkException, now finding the cause from the wrapped exception and throwing the cause instead of the wrapped exception.

## How was this patch tested?
Verified it manually by checking the cause of the error, it gives the error as shown below.

### Without the PR change

```
[apache-spark]$ ./bin/spark-submit --verbose --master spark://******
....
Error: Exception thrown in awaitResult:
Run with --help for usage help or --verbose for debug output

```

### With the PR change

```
[apache-spark]$ ./bin/spark-submit --verbose --master spark://******
....
Exception in thread "main" org.apache.spark.SparkException: Exception thrown in awaitResult:
        at org.apache.spark.util.ThreadUtils$.awaitResult(ThreadUtils.scala:226)
        at org.apache.spark.rpc.RpcTimeout.awaitResult(RpcTimeout.scala:75)
        ....
        at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
Caused by: java.io.IOException: Failed to connect to devaraj-pc1/10.3.66.65:7077
        at org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:245)
       ....
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)
Caused by: io.netty.channel.AbstractChannel$AnnotatedConnectException: Connection refused: devaraj-pc1/10.3.66.65:7077
        at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
        ....
        at io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138)
        ... 1 more
Caused by: java.net.ConnectException: Connection refused
        ... 11 more

```

Closes #22623 from devaraj-kavali/SPARK-25636.

Authored-by: Devaraj K <devaraj@apache.org>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-10-10 09:24:36 -07:00
Marcelo Vanzin 3eee9e0246 [SPARK-25535][CORE] Work around bad error handling in commons-crypto.
The commons-crypto library does some questionable error handling internally,
which can lead to JVM crashes if some call into native code fails and cleans
up state it should not.

While the library is not fixed, this change adds some workarounds in Spark code
so that when an error is detected in the commons-crypto side, Spark avoids
calling into the library further.

Tested with existing and added unit tests.

Closes #22557 from vanzin/SPARK-25535.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2018-10-09 09:27:08 -05:00
pgandhi deb9588b2a [SPARK-24851][UI] Map a Stage ID to it's Associated Job ID
It would be nice to have a field in Stage Page UI which would show mapping of the current stage id to the job id's to which that stage belongs to.

## What changes were proposed in this pull request?

Added a field in Stage UI to display the corresponding job id for that particular stage.

## How was this patch tested?

<img width="448" alt="screen shot 2018-07-25 at 1 33 07 pm" src="https://user-images.githubusercontent.com/22228190/43220447-a8e94f80-900f-11e8-8a20-a235bbd5a369.png">

Closes #21809 from pgandhi999/SPARK-24851.

Authored-by: pgandhi <pgandhi@oath.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2018-10-09 08:59:21 -05:00
Fokko Driesprong 1a28625355 [SPARK-25408] Move to more ideomatic Java8
While working on another PR, I noticed that there is quite some legacy Java in there that can be beautified. For example the use of features from Java8, such as:
- Collection libraries
- Try-with-resource blocks

No logic has been changed. I think it is important to have a solid codebase with examples that will inspire next PR's to follow up on the best practices.

What are your thoughts on this?

This makes code easier to read, and using try-with-resource makes is less likely to forget to close something.

## What changes were proposed in this pull request?

No changes in the logic of Spark, but more in the aesthetics of the code.

## How was this patch tested?

Using the existing unit tests. Since no logic is changed, the existing unit tests should pass.

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

Closes #22637 from Fokko/SPARK-25408.

Authored-by: Fokko Driesprong <fokkodriesprong@godatadriven.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-08 09:58:52 -05:00
Shahid 756a3ab18c [SPARK-25575][WEBUI][FOLLOWUP] SQL tab in the spark UI support hide tables
## What changes were proposed in this pull request?
After the PR, https://github.com/apache/spark/pull/22592, SQL tab supports collapsing table.
However, after refreshing the page, it doesn't store it previous state. This was due to a typo in the argument list in the collapseTablePageLoadCommand().

## How was this patch tested?
bin/spark-shell
```
sql("create table a (id int)")
for(i <- 1 to 100) sql(s"insert into a values ($i)")
```
![screenshot from 2018-10-06 10-19-30](https://user-images.githubusercontent.com/23054875/46567490-59bea380-c951-11e8-9484-9aa2ee84b816.png)

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

Closes #22650 from shahidki31/SPARK-25575-followUp.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-06 17:12:41 -05:00
Fokko Driesprong ab1650d293 [SPARK-24601] Update Jackson to 2.9.6
Hi all,

Jackson is incompatible with upstream versions, therefore bump the Jackson version to a more recent one. I bumped into some issues with Azure CosmosDB that is using a more recent version of Jackson. This can be fixed by adding exclusions and then it works without any issues. So no breaking changes in the API's.

I would also consider bumping the version of Jackson in Spark. I would suggest to keep up to date with the dependencies, since in the future this issue will pop up more frequently.

## What changes were proposed in this pull request?

Bump Jackson to 2.9.6

## How was this patch tested?

Compiled and tested it locally to see if anything broke.

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

Closes #21596 from Fokko/fd-bump-jackson.

Authored-by: Fokko Driesprong <fokkodriesprong@godatadriven.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-05 16:40:08 +08:00
Wenchen Fan 5ae20cf1a9 Revert "[SPARK-25408] Move to mode ideomatic Java8"
This reverts commit 44c1e1ab1c.
2018-10-05 11:03:41 +08:00
Fokko Driesprong 44c1e1ab1c [SPARK-25408] Move to mode ideomatic Java8
While working on another PR, I noticed that there is quite some legacy Java in there that can be beautified. For example the use og features from Java8, such as:
- Collection libraries
- Try-with-resource blocks

No code has been changed

What are your thoughts on this?

This makes code easier to read, and using try-with-resource makes is less likely to forget to close something.

## What changes were proposed in this pull request?

(Please fill in changes proposed in this fix)

## How was this patch tested?

(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 #22399 from Fokko/SPARK-25408.

Authored-by: Fokko Driesprong <fokkodriesprong@godatadriven.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-05 02:58:25 +01:00
ankurgupta 075dd620e3 [SPARK-25586][CORE] Remove outer objects from logdebug statements in ClosureCleaner
## What changes were proposed in this pull request?

Cause: Recently test_glr_summary failed for PR of SPARK-25118, which enables
spark-shell to run with default log level. It failed because this logdebug was
called for GeneralizedLinearRegressionTrainingSummary which invoked its toString
method, which started a Spark Job and ended up running into an infinite loop.

Fix: Remove logDebug statement for outer objects as closures aren't implemented
with outerclasses in Scala 2.12 and this debug statement looses its purpose

## How was this patch tested?

Ran python pyspark-ml tests on top of PR for SPARK-25118 and ClosureCleaner unit
tests

Closes #22616 from ankuriitg/ankur/SPARK-25586.

Authored-by: ankurgupta <ankur.gupta@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-10-03 16:18:36 -07:00
Gengliang Wang 7b4e94f160
[SPARK-25581][SQL] Rename method benchmark as runBenchmarkSuite in BenchmarkBase
## What changes were proposed in this pull request?

Rename method `benchmark` in `BenchmarkBase` as `runBenchmarkSuite `. Also add comments.
Currently the method name `benchmark` is a bit confusing. Also the name is the same as instances of `Benchmark`:

f246813afb/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala (L330-L339)

## How was this patch tested?

Unit test.

Closes #22599 from gengliangwang/renameBenchmarkSuite.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-02 10:04:47 -07:00
gatorsmile 9bf397c0e4 [SPARK-25592] Setting version to 3.0.0-SNAPSHOT
## What changes were proposed in this pull request?

This patch is to bump the master branch version to 3.0.0-SNAPSHOT.

## How was this patch tested?
N/A

Closes #22606 from gatorsmile/bump3.0.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-10-02 08:48:24 -07:00
Shahid 3422fc0b6c [SPARK-25575][WEBUI][SQL] SQL tab in the spark UI support hide tables, to make it consistent with other tabs.
## What changes were proposed in this pull request?
Currently, SQL tab in the WEBUI doesn't support hiding table. Other tabs in the web ui like, Jobs, stages etc supports hiding table (refer SPARK-23024 https://github.com/apache/spark/pull/20216).
In this PR, added the support for hide table in the sql tab also.

## How was this patch tested?
bin/spark-shell
```
sql("create table a (id int)")
for(i <- 1 to 100) sql(s"insert into a values ($i)")
```
Open SQL tab in the web UI

**Before fix:**

![image](https://user-images.githubusercontent.com/23054875/46249137-f5c44880-c441-11e8-953a-a811e33ac24d.png)

**After fix:** Consistent with the other tabs.

![screenshot from 2018-09-30 00-11-28](https://user-images.githubusercontent.com/23054875/46249354-75074b80-c445-11e8-9417-28751fd8628a.png)

(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 #22592 from shahidki31/SPARK-25575.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-01 17:45:12 -05:00
Darcy Shen 40e6ed8940 [CORE][MINOR] Fix obvious error and compiling for Scala 2.12.7
## What changes were proposed in this pull request?

Fix an obvious error.

## How was this patch tested?

Existing tests.

Closes #22577 from sadhen/minor_fix.

Authored-by: Darcy Shen <sadhen@zoho.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-09-30 09:00:23 -05:00
hyukjinkwon a2f502cf53 [SPARK-25565][BUILD] Add scalastyle rule to check add Locale.ROOT to .toLowerCase and .toUpperCase for internal calls
## What changes were proposed in this pull request?

This PR adds a rule to force `.toLowerCase(Locale.ROOT)` or `toUpperCase(Locale.ROOT)`.

It produces an error as below:

```
[error]       Are you sure that you want to use toUpperCase or toLowerCase without the root locale? In most cases, you
[error]       should use toUpperCase(Locale.ROOT) or toLowerCase(Locale.ROOT) instead.
[error]       If you must use toUpperCase or toLowerCase without the root locale, wrap the code block with
[error]       // scalastyle:off caselocale
[error]       .toUpperCase
[error]       .toLowerCase
[error]       // scalastyle:on caselocale
```

This PR excludes the cases above for SQL code path for external calls like table name, column name and etc.

For test suites, or when it's clear there's no locale problem like Turkish locale problem, it uses `Locale.ROOT`.

One minor problem is, `UTF8String` has both methods, `toLowerCase` and `toUpperCase`, and the new rule detects them as well. They are ignored.

## How was this patch tested?

Manually tested, and Jenkins tests.

Closes #22581 from HyukjinKwon/SPARK-25565.

Authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-30 14:31:04 +08:00
Shixiong Zhu b6b8a6632e [SPARK-25568][CORE] Continue to update the remaining accumulators when failing to update one accumulator
## What changes were proposed in this pull request?

Since we don't fail a job when `AccumulatorV2.merge` fails, we should try to update the remaining accumulators so that they can still report correct values.

## How was this patch tested?

The new unit test.

Closes #22586 from zsxwing/SPARK-25568.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-09-29 18:10:04 -07:00
Mukul Murthy 9362c5cc27
[SPARK-25449][CORE] Heartbeat shouldn't include accumulators for zero metrics
## What changes were proposed in this pull request?

Heartbeat shouldn't include accumulators for zero metrics.

Heartbeats sent from executors to the driver every 10 seconds contain metrics and are generally on the order of a few KBs. However, for large jobs with lots of tasks, heartbeats can be on the order of tens of MBs, causing tasks to die with heartbeat failures. We can mitigate this by not sending zero metrics to the driver.

## How was this patch tested?

(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 #22473 from mukulmurthy/25449-heartbeat.

Authored-by: Mukul Murthy <mukul.murthy@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2018-09-28 16:34:17 -07:00
Liang-Chi Hsieh b7d80349b0
[SPARK-25542][CORE][TEST] Move flaky test in OpenHashMapSuite to OpenHashSetSuite and make it against OpenHashSet
## What changes were proposed in this pull request?

The specified test in OpenHashMapSuite to test large items is somehow flaky to throw OOM.
By considering the original work #6763 that added this test, the test can be against OpenHashSetSuite. And by doing this should be to save memory because OpenHashMap allocates two more arrays when growing the map/set.

## How was this patch tested?

Existing tests.

Closes #22569 from viirya/SPARK-25542.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-09-28 14:29:56 -07:00
Marcelo Vanzin 5fd22d0536
[SPARK-25546][CORE] Don't cache value of EVENT_LOG_CALLSITE_LONG_FORM.
Caching the value of that config means different instances of SparkEnv
will always use whatever was the first value to be read. It also breaks
tests that use RDDInfo outside of the scope of a SparkContext.

Since this is not a performance sensitive area, there's no advantage
in caching the config value.

Closes #22558 from vanzin/SPARK-25546.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-09-27 09:26:50 -07:00
Holden Karau f856fe4839 [SPARK-21436][CORE] Take advantage of known partitioner for distinct on RDDs to avoid a shuffle
## What changes were proposed in this pull request?

Special case the situation where we know the partioner and the number of requested partions output is the same as the current partioner to avoid a shuffle and instead compute distinct inside of each partion.

## How was this patch tested?

New unit test that verifies partitioner does not change if the partitioner is known and distinct is called with the same target # of partition.

Closes #22010 from holdenk/SPARK-21436-take-advantage-of-known-partioner-for-distinct-on-rdds.

Authored-by: Holden Karau <holden@pigscanfly.ca>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-27 20:57:56 +08:00
王小刚 8b727994ed [SPARK-25468][WEBUI] Highlight current page index in the spark UI
## What changes were proposed in this pull request?

This PR is highlight current page index in the spark UI and history server UI,
https://issues.apache.org/jira/browse/SPARK-25468
I have add the following code in webui.css
```
.paginate_button.active>a {
    color: #999999;
    text-decoration: underline;
}
```
## How was this patch tested?
Manual tests for Chrome, Firefox and Safari
Before modifying:
![image](https://user-images.githubusercontent.com/10048468/45914897-01ca6c00-be7e-11e8-8e31-47d45db0c3bf.png)

After modifying:
![image](https://user-images.githubusercontent.com/10048468/45913987-7e564e00-be70-11e8-9c16-de17e2c63308.png)

Closes #22516 from Adamyuanyuan/spark-adam-25468.

Lead-authored-by: 王小刚 <wangxiaogang@chinatelecom.cn>
Co-authored-by: Adam Wang <Adamyuanyuan@users.noreply.github.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-09-27 00:02:05 -05:00
Shahid 5def10e61e
[SPARK-25536][CORE] metric value for METRIC_OUTPUT_RECORDS_WRITTEN is incorrect
## What changes were proposed in this pull request?
changed metric value of METRIC_OUTPUT_RECORDS_WRITTEN from 'task.metrics.inputMetrics.recordsRead' to 'task.metrics.outputMetrics.recordsWritten'.
This bug was introduced in SPARK-22190. https://github.com/apache/spark/pull/19426

## How was this patch tested?
Existing tests

Closes #22555 from shahidki31/SPARK-25536.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-09-26 21:10:39 -07:00
Ilan Filonenko 51540c2fa6 [SPARK-25372][YARN][K8S] Deprecate and generalize keytab / principal config
## What changes were proposed in this pull request?

SparkSubmit already logs in the user if a keytab is provided, the only issue is that it uses the existing configs which have "yarn" in their name. As such, the configs were changed to:

`spark.kerberos.keytab` and `spark.kerberos.principal`.

## How was this patch tested?

Will be tested with K8S tests, but needs to be tested with Yarn

- [x] K8S Secure HDFS tests
- [x] Yarn Secure HDFS tests vanzin

Closes #22362 from ifilonenko/SPARK-25372.

Authored-by: Ilan Filonenko <if56@cornell.edu>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-09-26 17:24:52 -07:00
Shahid 5ee2166183 [SPARK-25533][CORE][WEBUI] AppSummary should hold the information about succeeded Jobs and completed stages only
## What changes were proposed in this pull request?
Currently, In the spark UI, when there are failed jobs or failed stages, display message for the completed jobs and completed stages are not consistent with the previous versions of spark.
Reason is because, AppSummary holds the information about all the jobs and stages. But, In the below code, it checks against the completedJobs and completedStages. So, AppSummary should hold only successful jobs and stages.

66d29870c0/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala (L306)
 66d29870c0/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala (L119)
So, we should  keep only completed jobs and stage information in the AppSummary, to make it consistent with Spark2.2

## How was this patch tested?
Test steps:
 bin/spark-shell
```
sc.parallelize(1 to 5, 5).collect()
sc.parallelize(1 to 5, 2).map{ x => throw new RuntimeException("Fail")}.collect()
```
**Before fix:**

![screenshot from 2018-09-26 03-24-53](https://user-images.githubusercontent.com/23054875/46045669-f60bcd80-c13b-11e8-9aa6-a2e5a2038dba.png)

![screenshot from 2018-09-26 03-25-08](https://user-images.githubusercontent.com/23054875/46045699-0ae86100-c13c-11e8-94e5-ad35944c7615.png)

**After fix:**
![screenshot from 2018-09-26 03-16-14](https://user-images.githubusercontent.com/23054875/46045636-d83e6880-c13b-11e8-98df-f49d15c18958.png)
![screenshot from 2018-09-26 03-16-28](https://user-images.githubusercontent.com/23054875/46045645-e1c7d080-c13b-11e8-8c9c-d32e1f663356.png)

Closes #22549 from shahidki31/SPARK-25533.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-09-26 10:47:49 -07:00
Reynold Xin e702fb1d52 [SPARK-24519][CORE] Compute SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS only once
## What changes were proposed in this pull request?
Previously SPARK-24519 created a modifiable config SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS. However, the config is being parsed for every creation of MapStatus, which could be very expensive. Another problem with the previous approach is that it created the illusion that this can be changed dynamically at runtime, which was not true. This PR changes it so the config is computed only once.

## How was this patch tested?
Removed a test case that's no longer valid.

Closes #22521 from rxin/SPARK-24519.

Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-09-26 10:15:16 -07:00
Reza Safi bd2ae857d1 [SPARK-25318] Add exception handling when wrapping the input stream during the the fetch or stage retry in response to a corrupted block
SPARK-4105 provided a solution to block corruption issue by retrying the fetch or the stage. In that solution there is a step that wraps the input stream with compression and/or encryption. This step is prone to exceptions, but in the current code there is no exception handling for this step and this has caused confusion for the user. The  confusion was that after SPARK-4105 the user expects to see either a fetchFailed exception or a warning about a corrupted block. However an exception during wrapping can fail the job without any of those.  This change adds exception handling for the wrapping step and also adds a fetch retry if we experience a corruption during the wrapping step. The reason for adding the retry is that usually user won't experience the same failure after rerunning the job and so it seems reasonable try to fetch and wrap one more time instead of failing.

Closes #22325 from rezasafi/localcorruption.

Authored-by: Reza Safi <rezasafi@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-09-26 09:29:58 -07:00
Rong Tang a2ac5a72cc [SPARK-25509][CORE] Windows doesn't support POSIX permissions
## What changes were proposed in this pull request?

SHS V2 cannot enabled in Windows, because windows doesn't support POSIX permission.

## How was this patch tested?

test case fails in windows without this fix.
org.apache.spark.deploy.history.HistoryServerDiskManagerSuite  test("leasing space")

SHS V2  cannot run successfully in Windows without this fix.

java.lang.UnsupportedOperationException: 'posix:permissions' not supported as initial attribute
  at sun.nio.fs.WindowsSecurityDescriptor.fromAttribute(WindowsSecurityDescriptor.java:358)

Closes #22520 from jianjianjiao/FixWindowsPermssionsIssue.

Authored-by: Rong Tang <rotang@microsoft.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-09-26 10:37:17 -05: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
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
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
hyukjinkwon c3b4a94a91 [SPARKR] Match pyspark features in SparkR communication protocol 2018-09-24 19:25:02 +08: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
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
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
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
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
Imran Rashid a97001d217 [CORE] Updates to remote cache reads
Covered by tests in DistributedSuite
2018-09-17 14:06:09 -05:00
Imran Rashid 58419b9267 [PYSPARK] Updates to pyspark broadcast 2018-09-17 14:06:09 -05:00
Dongjoon Hyun 538e047878 [SPARK-22713][CORE][TEST][FOLLOWUP] Fix flaky ExternalAppendOnlyMapSuite due to timeout
## What changes were proposed in this pull request?

SPARK-22713 uses [`eventually` with the default timeout `150ms`](https://github.com/apache/spark/pull/21369/files#diff-5bbb6a931b7e4d6a31e4938f51935682R462). It causes flakiness because it's executed once when GC is slow.

```scala
eventually {
  System.gc()
  ...
}
```

**Failures**
```scala
org.scalatest.exceptions.TestFailedDueToTimeoutException:
The code passed to eventually never returned normally.
Attempted 1 times over 501.22261 milliseconds.
Last failure message: tmpIsNull was false.
```
- master-test-sbt-hadoop-2.7
  [4916](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.7/4916)
  [4907](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.7/4907)
  [4906](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.7/4906)

- spark-master-test-sbt-hadoop-2.6
  [4979](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.6/4979)
  [4974](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.6/4974)
  [4967](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.6/4967)
  [4966](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.6/4966)

## How was this patch tested?

Pass the Jenkins.

Closes #22432 from dongjoon-hyun/SPARK-22713.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-17 11:07:51 +08:00
gatorsmile bb2f069cf2 [SPARK-25436] Bump master branch version to 2.5.0-SNAPSHOT
## What changes were proposed in this pull request?
In the dev list, we can still discuss whether the next version is 2.5.0 or 3.0.0. Let us first bump the master branch version to `2.5.0-SNAPSHOT`.

## How was this patch tested?
N/A

Closes #22426 from gatorsmile/bumpVersionMaster.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-09-15 16:24:02 -07:00
Imran Rashid 9deddbb13e [SPARK-25400][CORE][TEST] Increase test timeouts
We've seen some flakiness in jenkins in SchedulerIntegrationSuite which looks like it just needs a
longer timeout.

Closes #22385 from squito/SPARK-25400.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-09-13 14:11:55 -05:00
Kazuaki Ishizaki f60cd7cc3c
[SPARK-25338][TEST] Ensure to call super.beforeAll() and super.afterAll() in test cases
## What changes were proposed in this pull request?

This PR ensures to call `super.afterAll()` in `override afterAll()` method for test suites.

* Some suites did not call `super.afterAll()`
* Some suites may call `super.afterAll()` only under certain condition
* Others never call `super.afterAll()`.

This PR also ensures to call `super.beforeAll()` in `override beforeAll()` for test suites.

## How was this patch tested?

Existing UTs

Closes #22337 from kiszk/SPARK-25338.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-09-13 11:34:22 -07:00
Michael Mior ab25c96790 [SPARK-23820][CORE] Enable use of long form of callsite in logs
This is a rework of #21433 to address some concerns there.

Closes #22398 from michaelmior/long-callsite2.

Authored-by: Michael Mior <mmior@uwaterloo.ca>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-13 09:45:25 +08:00
Sean Owen 97d4afaa13 Revert "[SPARK-23820][CORE] Enable use of long form of callsite in logs"
This reverts commit e58dadb77e.
2018-09-11 14:52:58 -05:00
Sean Owen cfbdd6a1f5 [SPARK-25398] Minor bugs from comparing unrelated types
## What changes were proposed in this pull request?

Correct some comparisons between unrelated types to what they seem to… have been trying to do

## How was this patch tested?

Existing tests.

Closes #22384 from srowen/SPARK-25398.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-09-11 14:46:03 -05:00
Liang-Chi Hsieh 14f3ad2093 [SPARK-24889][CORE] Update block info when unpersist rdds
## What changes were proposed in this pull request?

We will update block info coming from executors, at the timing like caching a RDD. However, when removing RDDs with unpersisting, we don't ask to update block info. So the block info is not updated.

We can fix this with few options:

1. Ask to update block info when unpersisting

This is simplest but changes driver-executor communication a bit.

2. Update block info when processing the event of unpersisting RDD

We send a `SparkListenerUnpersistRDD` event when unpersisting RDD. When processing this event, we can update block info of the RDD. This only changes event processing code so the risk seems to be lower.

Currently this patch takes option 2 for lower risk. If we agree first option has no risk, we can change to it.

## How was this patch tested?

Unit tests.

Closes #22341 from viirya/SPARK-24889.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-09-11 10:31:06 -07:00
Gera Shegalov bcb9a8c83f [SPARK-25221][DEPLOY] Consistent trailing whitespace treatment of conf values
## What changes were proposed in this pull request?

Stop trimming values of properties loaded from a file

## How was this patch tested?

Added unit test demonstrating the issue hit in production.

Closes #22213 from gerashegalov/gera/SPARK-25221.

Authored-by: Gera Shegalov <gera@apache.org>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-09-11 09:28:32 -07:00
gatorsmile 0b9ccd55c2 Revert [SPARK-10399] [SPARK-23879] [SPARK-23762] [SPARK-25317]
## What changes were proposed in this pull request?

When running TPC-DS benchmarks on 2.4 release, npoggi and winglungngai  saw more than 10% performance regression on the following queries: q67, q24a and q24b. After we applying the PR https://github.com/apache/spark/pull/22338, the performance regression still exists. If we revert the changes in https://github.com/apache/spark/pull/19222, npoggi and winglungngai  found the performance regression was resolved. Thus, this PR is to revert the related changes for unblocking the 2.4 release.

In the future release, we still can continue the investigation and find out the root cause of the regression.

## How was this patch tested?

The existing test cases

Closes #22361 from gatorsmile/revertMemoryBlock.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-09 21:25:19 +08:00
Edwina Lu 9241e1e7e6 [SPARK-23429][CORE] Add executor memory metrics to heartbeat and expose in executors REST API
Add new executor level memory metrics (JVM used memory, on/off heap execution memory, on/off heap storage memory, on/off heap unified memory, direct memory, and mapped memory), and expose via the executors REST API. This information will help provide insight into how executor and driver JVM memory is used, and for the different memory regions. It can be used to help determine good values for spark.executor.memory, spark.driver.memory, spark.memory.fraction, and spark.memory.storageFraction.

## What changes were proposed in this pull request?

An ExecutorMetrics class is added, with jvmUsedHeapMemory, jvmUsedNonHeapMemory, onHeapExecutionMemory, offHeapExecutionMemory, onHeapStorageMemory, and offHeapStorageMemory, onHeapUnifiedMemory, offHeapUnifiedMemory, directMemory and mappedMemory. The new ExecutorMetrics is sent by executors to the driver as part of the Heartbeat. A heartbeat is added for the driver as well, to collect these metrics for the driver.

The EventLoggingListener store information about the peak values for each metric, per active stage and executor. When a StageCompleted event is seen, a StageExecutorsMetrics event will be logged for each executor, with peak values for the stage.

The AppStatusListener records the peak values for each memory metric.

The new memory metrics are added to the executors REST API.

## How was this patch tested?

New unit tests have been added. This was also tested on our cluster.

Author: Edwina Lu <edlu@linkedin.com>
Author: Imran Rashid <irashid@cloudera.com>
Author: edwinalu <edwina.lu@gmail.com>

Closes #21221 from edwinalu/SPARK-23429.2.
2018-09-07 10:42:46 -07:00
Sean Owen 4e3365b577 [SPARK-22357][CORE][FOLLOWUP] SparkContext.binaryFiles ignore minPartitions parameter
## What changes were proposed in this pull request?

This adds a test following https://github.com/apache/spark/pull/21638

## How was this patch tested?

Existing tests and new test.

Closes #22356 from srowen/SPARK-22357.2.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-09-06 21:43:14 -07:00
xuejianbest f5817d8bb3 [SPARK-25108][SQL] Fix the show method to display the wide character alignment problem
This is not a perfect solution. It is designed to minimize complexity on the basis of solving problems.

It is effective for English, Chinese characters, Japanese, Korean and so on.

```scala
before:
+---+---------------------------+-------------+
|id |中国                         |s2           |
+---+---------------------------+-------------+
|1  |ab                         |[a]          |
|2  |null                       |[中国, abc]    |
|3  |ab1                        |[hello world]|
|4  |か行 きゃ(kya) きゅ(kyu) きょ(kyo) |[“中国]        |
|5  |中国(你好)a                    |[“中(国), 312] |
|6  |中国山(东)服务区                  |[“中(国)]      |
|7  |中国山东服务区                    |[中(国)]       |
|8  |                           |[中国]         |
+---+---------------------------+-------------+

after:
+---+-----------------------------------+----------------+
|id |中国                               |s2              |
+---+-----------------------------------+----------------+
|1  |ab                                 |[a]             |
|2  |null                               |[中国, abc]     |
|3  |ab1                                |[hello world]   |
|4  |か行 きゃ(kya) きゅ(kyu) きょ(kyo) |[“中国]         |
|5  |中国(你好)a                      |[“中(国), 312]|
|6  |中国山(东)服务区                   |[“中(国)]      |
|7  |中国山东服务区                     |[中(国)]        |
|8  |                                   |[中国]          |
+---+-----------------------------------+----------------+
```

## What changes were proposed in this pull request?

When there are wide characters such as Chinese characters or Japanese characters in the data, the show method has a alignment problem.
Try to fix this problem.

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)

![image](https://user-images.githubusercontent.com/13044869/44250564-69f6b400-a227-11e8-88b2-6cf6960377ff.png)

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

Closes #22048 from xuejianbest/master.

Authored-by: xuejianbest <384329882@qq.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-09-06 07:17:37 -07:00
Yuming Wang 3e033035a3 [SPARK-25258][SPARK-23131][SPARK-25176][BUILD] Upgrade Kryo to 4.0.2
## What changes were proposed in this pull request?

Upgrade chill to 0.9.3, Kryo to 4.0.2, to get bug fixes and improvements.

The resolved tickets includes:
- SPARK-25258 Upgrade kryo package to version 4.0.2
- SPARK-23131 Kryo raises StackOverflow during serializing GLR model
- SPARK-25176 Kryo fails to serialize a parametrised type hierarchy

More details:
https://github.com/twitter/chill/releases/tag/v0.9.3
cc3910d501

## How was this patch tested?

Existing tests.

Closes #22179 from wangyum/SPARK-23131.

Lead-authored-by: Yuming Wang <yumwang@ebay.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-09-05 15:48:41 -07:00
Wenchen Fan 71bd796517 [SPARK-23243][CORE] Fix RDD.repartition() data correctness issue
## What changes were proposed in this pull request?

An alternative fix for https://github.com/apache/spark/pull/21698

When Spark rerun tasks for an RDD, there are 3 different behaviors:
1. determinate. Always return the same result with same order when rerun.
2. unordered. Returns same data set in random order when rerun.
3. indeterminate. Returns different result when rerun.

Normally Spark doesn't need to care about it. Spark runs stages one by one, when a task is failed, just rerun it. Although the rerun task may return a different result, users will not be surprised.

However, Spark may rerun a finished stage when seeing fetch failures. When this happens, Spark needs to rerun all the tasks of all the succeeding stages if the RDD output is indeterminate, because the input of the succeeding stages has been changed.

If the RDD output is determinate, we only need to rerun the failed tasks of the succeeding stages, because the input doesn't change.

If the RDD output is unordered, it's same as determinate, because shuffle partitioner is always deterministic(round-robin partitioner is not a shuffle partitioner that extends `org.apache.spark.Partitioner`), so the reducers will still get the same input data set.

This PR fixed the failure handling for `repartition`, to avoid correctness issues.

For `repartition`, it applies a stateful map function to generate a round-robin id, which is order sensitive and makes the RDD's output indeterminate. When the stage contains `repartition` reruns, we must also rerun all the tasks of all the succeeding stages.

**future improvement:**
1. Currently we can't rollback and rerun a shuffle map stage, and just fail. We should fix it later. https://issues.apache.org/jira/browse/SPARK-25341
2. Currently we can't rollback and rerun a result stage, and just fail. We should fix it later. https://issues.apache.org/jira/browse/SPARK-25342
3. We should provide public API to allow users to tag the random level of the RDD's computing function.

## How is this pull request tested?
a new test case

Closes #22112 from cloud-fan/repartition.

Lead-authored-by: Wenchen Fan <wenchen@databricks.com>
Co-authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2018-09-05 15:36:34 -07:00
pgandhi 559b899ace [SPARK-25231] Fix synchronization of executor heartbeat receiver in TaskSchedulerImpl
Running a large Spark job with speculation turned on was causing executor heartbeats to time out on the driver end after sometime and eventually, after hitting the max number of executor failures, the job would fail.

## What changes were proposed in this pull request?

The main reason for the heartbeat timeouts was that the heartbeat-receiver-event-loop-thread was blocked waiting on the TaskSchedulerImpl object which was being held by one of the dispatcher-event-loop threads executing the method dequeueSpeculativeTasks() in TaskSetManager.scala. On further analysis of the heartbeat receiver method executorHeartbeatReceived() in TaskSchedulerImpl class, we found out that instead of waiting to acquire the lock on the TaskSchedulerImpl object, we can remove that lock and make the operations to the global variables inside the code block to be atomic. The block of code in that method only uses  one global HashMap taskIdToTaskSetManager. Making that map a ConcurrentHashMap, we are ensuring atomicity of operations and speeding up the heartbeat receiver thread operation.

## How was this patch tested?

Screenshots of the thread dump have been attached below:
**heartbeat-receiver-event-loop-thread:**

<img width="1409" alt="screen shot 2018-08-24 at 9 19 57 am" src="https://user-images.githubusercontent.com/22228190/44593413-e25df780-a788-11e8-9520-176a18401a59.png">

**dispatcher-event-loop-thread:**

<img width="1409" alt="screen shot 2018-08-24 at 9 21 56 am" src="https://user-images.githubusercontent.com/22228190/44593484-13d6c300-a789-11e8-8d88-34b1d51d4541.png">

Closes #22221 from pgandhi999/SPARK-25231.

Authored-by: pgandhi <pgandhi@oath.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2018-09-05 16:10:49 -05:00
ankurgupta 39a02d8f75 [SPARK-24415][CORE] Fixed the aggregated stage metrics by retaining stage objects in liveStages until all tasks are complete
The problem occurs because stage object is removed from liveStages in
AppStatusListener onStageCompletion. Because of this any onTaskEnd event
received after onStageCompletion event do not update stage metrics.

The fix is to retain stage objects in liveStages until all tasks are complete.

1. Fixed the reproducible example posted in the JIRA
2. Added unit test

Closes #22209 from ankuriitg/ankurgupta/SPARK-24415.

Authored-by: ankurgupta <ankur.gupta@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-09-05 09:52:04 -07:00
LucaCanali 8440e30728 [SPARK-25228][CORE] Add executor CPU time metric.
## What changes were proposed in this pull request?

Add a new metric to measure the executor's process (JVM) CPU time.

## How was this patch tested?

Manually tested on a Spark cluster (see SPARK-25228 for an example screenshot).

Closes #22218 from LucaCanali/AddExecutrCPUTimeMetric.

Authored-by: LucaCanali <luca.canali@cern.ch>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-09-05 06:58:15 -07:00
liuxian ca861fea21 [SPARK-25300][CORE] Unified the configuration parameter spark.shuffle.service.enabled
## What changes were proposed in this pull request?

The configuration parameter "spark.shuffle.service.enabled"  has defined in `package.scala`,  and it  is also used in many place,  so we can replace it with `SHUFFLE_SERVICE_ENABLED`.
and unified  this configuration parameter "spark.shuffle.service.port"  together.

## How was this patch tested?
N/A

Closes #22306 from 10110346/unifiedserviceenable.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-05 10:43:46 +08:00
Xiangrui Meng 061bb01d9b [SPARK-25248][CORE] Audit barrier Scala APIs for 2.4
## What changes were proposed in this pull request?

I made one pass over barrier APIs added to Spark 2.4 and updates some scopes and docs. I will update Python docs once Scala doc was reviewed.

One major issue is that `BarrierTaskContext` implements `TaskContextImpl` that exposes some public methods. And internally there were several direct references to `TaskContextImpl` methods instead of `TaskContext`. This PR moved some methods from `TaskContextImpl` to `TaskContext`, remaining package private, and used delegate methods to avoid inheriting `TaskContextImp` and exposing unnecessary APIs.

TODOs:
- [x] scala doc
- [x] python doc (#22261 ).

Closes #22240 from mengxr/SPARK-25248.

Authored-by: Xiangrui Meng <meng@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2018-09-04 09:55:53 -07:00
Liang-Chi Hsieh 7c36ee46d9 [SPARK-25290][CORE][TEST] Reduce the size of acquired arrays to avoid OOM error
## What changes were proposed in this pull request?

`BytesToBytesMapOnHeapSuite`.`randomizedStressTest` caused `OutOfMemoryError` on several test runs. Seems better to reduce memory usage in this test.

## How was this patch tested?

Unit tests.

Closes #22297 from viirya/SPARK-25290.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-01 16:25:29 +08:00
Maxim Gekk 32da87dfa4 [SPARK-25286][CORE] Removing the dangerous parmap
## What changes were proposed in this pull request?

I propose to remove one of `parmap` methods which accepts an execution context as a parameter. The method should be removed to eliminate any deadlocks that can occur if `parmap` is called recursively on thread pools restricted by size.

Closes #22292 from MaxGekk/remove-overloaded-parmap.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2018-08-31 10:43:30 -07:00
Thomas Graves ec3e998638 [SPARK-24909][CORE] Always unregister pending partition on task completion.
Spark scheduler can hang when fetch failures, executor lost, task running on lost executor, and multiple stage attempts. To fix this we change to always unregister the pending partition on task completion.

## What changes were proposed in this pull request?
this PR is actually reverting the change in SPARK-19263, so that it always does shuffleStage.pendingPartitions -= task.partitionId.   The change in SPARK-23433, should fix the issue originally from SPARK-19263.

## How was this patch tested?

Unit tests.  The condition happens on a race which I haven't reproduced on a real customer, just see it sometimes on customers jobs in a real cluster.
I am also working on adding spark scheduler integration tests.

Closes #21976 from tgravescs/SPARK-24909.

Authored-by: Thomas Graves <tgraves@unharmedunarmed.corp.ne1.yahoo.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-08-29 16:32:02 -07:00
sarutak 3864480e14 [SPARK-25266][CORE] Fix memory leak in Barrier Execution Mode
## What changes were proposed in this pull request?

BarrierCoordinator uses Timer and TimerTask. `TimerTask#cancel()` is invoked in ContextBarrierState#cancelTimerTask but `Timer#purge()` is never invoked.

Once a TimerTask is scheduled, the reference to it is not released until `Timer#purge()` is invoked even though `TimerTask#cancel()` is invoked.

## How was this patch tested?

I checked the number of instances related to the TimerTask using jmap.

Closes #22258 from sarutak/fix-barrierexec-oom.

Authored-by: sarutak <sarutak@oss.nttdata.co.jp>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2018-08-29 07:13:13 -07:00
Bryan Cutler 82c18c240a [SPARK-23030][SQL][PYTHON] Use Arrow stream format for creating from and collecting Pandas DataFrames
## What changes were proposed in this pull request?

This changes the calls of `toPandas()` and `createDataFrame()` to use the Arrow stream format, when Arrow is enabled.  Previously, Arrow data was written to byte arrays where each chunk is an output of the Arrow file format.  This was mainly due to constraints at the time, and caused some overhead by writing the schema/footer on each chunk of data and then having to read multiple Arrow file inputs and concat them together.

Using the Arrow stream format has improved these by increasing performance, lower memory overhead for the average case, and simplified the code.  Here are the details of this change:

**toPandas()**

_Before:_
Spark internal rows are converted to Arrow file format, each group of records is a complete Arrow file which contains the schema and other metadata.  Next a collect is done and an Array of Arrow files is the result.  After that each Arrow file is sent to Python driver which then loads each file and concats them to a single Arrow DataFrame.

_After:_
Spark internal rows are converted to ArrowRecordBatches directly, which is the simplest Arrow component for IPC data transfers.  The driver JVM then immediately starts serving data to Python as an Arrow stream, sending the schema first. It then starts a Spark job with a custom handler that sends Arrow RecordBatches to Python. Partitions arriving in order are sent immediately, and out-of-order partitions are buffered until the ones that precede it come in. This improves performance, simplifies memory usage on executors, and improves the average memory usage on the JVM driver.  Since the order of partitions must be preserved, the worst case is that the first partition will be the last to arrive all data must be buffered in memory until then. This case is no worse that before when doing a full collect.

**createDataFrame()**

_Before:_
A Pandas DataFrame is split into parts and each part is made into an Arrow file.  Then each file is prefixed by the buffer size and written to a temp file.  The temp file is read and each Arrow file is parallelized as a byte array.

_After:_
A Pandas DataFrame is split into parts, then an Arrow stream is written to a temp file where each part is an ArrowRecordBatch.  The temp file is read as a stream and the Arrow messages are examined.  If the message is an ArrowRecordBatch, the data is saved as a byte array.  After reading the file, each ArrowRecordBatch is parallelized as a byte array.  This has slightly more processing than before because we must look each Arrow message to extract the record batches, but performance ends up a litle better.  It is cleaner in the sense that IPC from Python to JVM is done over a single Arrow stream.

## How was this patch tested?

Added new unit tests for the additions to ArrowConverters in Scala, existing tests for Python.

## Performance Tests - toPandas

Tests run on a 4 node standalone cluster with 32 cores total, 14.04.1-Ubuntu and OpenJDK 8
measured wall clock time to execute `toPandas()` and took the average best time of 5 runs/5 loops each.

Test code
```python
df = spark.range(1 << 25, numPartitions=32).toDF("id").withColumn("x1", rand()).withColumn("x2", rand()).withColumn("x3", rand()).withColumn("x4", rand())
for i in range(5):
	start = time.time()
	_ = df.toPandas()
	elapsed = time.time() - start
```

Current Master | This PR
---------------------|------------
5.803557 | 5.16207
5.409119 | 5.133671
5.493509 | 5.147513
5.433107 | 5.105243
5.488757 | 5.018685

Avg Master | Avg This PR
------------------|--------------
5.5256098 | 5.1134364

Speedup of **1.08060595**

## Performance Tests - createDataFrame

Tests run on a 4 node standalone cluster with 32 cores total, 14.04.1-Ubuntu and OpenJDK 8
measured wall clock time to execute `createDataFrame()` and get the first record. Took the average best time of 5 runs/5 loops each.

Test code
```python
def run():
	pdf = pd.DataFrame(np.random.rand(10000000, 10))
	spark.createDataFrame(pdf).first()

for i in range(6):
	start = time.time()
	run()
	elapsed = time.time() - start
	gc.collect()
	print("Run %d: %f" % (i, elapsed))
```

Current Master | This PR
--------------------|----------
6.234608 | 5.665641
6.32144 | 5.3475
6.527859 | 5.370803
6.95089 | 5.479151
6.235046 | 5.529167

Avg Master | Avg This PR
---------------|----------------
6.4539686 | 5.4784524

Speedup of **1.178064192**

## Memory Improvements

**toPandas()**

The most significant improvement is reduction of the upper bound space complexity in the JVM driver.  Before, the entire dataset was collected in the JVM first before sending it to Python.  With this change, as soon as a partition is collected, the result handler immediately sends it to Python, so the upper bound is the size of the largest partition.  Also, using the Arrow stream format is more efficient because the schema is written once per stream, followed by record batches.  The schema is now only send from driver JVM to Python.  Before, multiple Arrow file formats were used that each contained the schema.  This duplicated schema was created in the executors, sent to the driver JVM, and then Python where all but the first one received are discarded.

I verified the upper bound limit by running a test that would collect data that would exceed the amount of driver JVM memory available.  Using these settings on a standalone cluster:
```
spark.driver.memory 1g
spark.executor.memory 5g
spark.sql.execution.arrow.enabled true
spark.sql.execution.arrow.fallback.enabled false
spark.sql.execution.arrow.maxRecordsPerBatch 0
spark.driver.maxResultSize 2g
```

Test code:
```python
from pyspark.sql.functions import rand
df = spark.range(1 << 25, numPartitions=32).toDF("id").withColumn("x1", rand()).withColumn("x2", rand()).withColumn("x3", rand())
df.toPandas()
```

This makes total data size of 33554432×8×4 = 1073741824

With the current master, it fails with OOM but passes using this PR.

**createDataFrame()**

No significant change in memory except that using the stream format instead of separate file formats avoids duplicated the schema, similar to toPandas above.  The process of reading the stream and parallelizing the batches does cause the record batch message metadata to be copied, but it's size is insignificant.

Closes #21546 from BryanCutler/arrow-toPandas-stream-SPARK-23030.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-29 15:01:12 +08:00
Imran Rashid 38391c9aa8 [SPARK-25253][PYSPARK] Refactor local connection & auth code
This eliminates some duplication in the code to connect to a server on localhost to talk directly to the jvm.  Also it gives consistent ipv6 and error handling.  Two other incidental changes, that shouldn't matter:
1) python barrier tasks perform authentication immediately (rather than waiting for the BARRIER_FUNCTION indicator)
2) for `rdd._load_from_socket`, the timeout is only increased after authentication.

Closes #22247 from squito/py_connection_refactor.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-29 09:47:38 +08:00
Bo Meng bbbf814691 [SPARK-22357][CORE] SparkContext.binaryFiles ignore minPartitions parameter
## What changes were proposed in this pull request?
Fix the issue that minPartitions was not used in the method. This is a simple fix and I am not trying to make it complicated. The purpose is to still allow user to control the defaultParallelism through the value of minPartitions, while also via sc.defaultParallelism parameters.

## How was this patch tested?
I have not provided the additional test since the fix is very straightforward.

Closes #21638 from bomeng/22357.

Lead-authored-by: Bo Meng <mengbo@hotmail.com>
Co-authored-by: Bo Meng <bo.meng@jd.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-08-28 19:39:13 -05:00
Ryan Blue 7ad18ee9f2 [SPARK-25004][CORE] Add spark.executor.pyspark.memory limit.
## What changes were proposed in this pull request?

This adds `spark.executor.pyspark.memory` to configure Python's address space limit, [`resource.RLIMIT_AS`](https://docs.python.org/3/library/resource.html#resource.RLIMIT_AS). Limiting Python's address space allows Python to participate in memory management. In practice, we see fewer cases of Python taking too much memory because it doesn't know to run garbage collection. This results in YARN killing fewer containers. This also improves error messages so users know that Python is consuming too much memory:

```
  File "build/bdist.linux-x86_64/egg/package/library.py", line 265, in fe_engineer
    fe_eval_rec.update(f(src_rec_prep, mat_rec_prep))
  File "build/bdist.linux-x86_64/egg/package/library.py", line 163, in fe_comp
    comparisons = EvaluationUtils.leven_list_compare(src_rec_prep.get(item, []), mat_rec_prep.get(item, []))
  File "build/bdist.linux-x86_64/egg/package/evaluationutils.py", line 25, in leven_list_compare
    permutations = sorted(permutations, reverse=True)
  MemoryError
```

The new pyspark memory setting is used to increase requested YARN container memory, instead of sharing overhead memory between python and off-heap JVM activity.

## How was this patch tested?

Tested memory limits in our YARN cluster and verified that MemoryError is thrown.

Author: Ryan Blue <blue@apache.org>

Closes #21977 from rdblue/SPARK-25004-add-python-memory-limit.
2018-08-28 12:31:33 -07:00
Shixiong Zhu 592e3a42c2
[SPARK-25218][CORE] Fix potential resource leaks in TransportServer and SocketAuthHelper
## What changes were proposed in this pull request?

Make sure TransportServer and SocketAuthHelper close the resources for all types of errors.

## How was this patch tested?

Jenkins

Closes #22210 from zsxwing/SPARK-25218.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2018-08-28 08:36:06 -07:00
liuxian 381a967a76 [SPARK-25249][CORE][TEST] add a unit test for OpenHashMap
## What changes were proposed in this pull request?

This PR adds a unit test for OpenHashMap , this can help developers  to distinguish between the 0/0.0/0L and null

## How was this patch tested?

Closes #22241 from 10110346/openhashmap.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-08-27 12:05:33 -05:00
DB Tsai 3e4f1666a1 [MINOR] Fix Scala 2.12 build
## What changes were proposed in this pull request?

[SPARK-25095](ad45299d04) introduced `ambiguous reference to overloaded definition`

```
[error] /Users/d_tsai/dev/apache-spark/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala:242: ambiguous reference to overloaded definition,
[error] both method addTaskCompletionListener in class TaskContext of type [U](f: org.apache.spark.TaskContext => U)org.apache.spark.TaskContext
[error] and  method addTaskCompletionListener in class TaskContext of type (listener: org.apache.spark.util.TaskCompletionListener)org.apache.spark.TaskContext
[error] match argument types (org.apache.spark.TaskContext => Unit)
[error]           context.addTaskCompletionListener(_ => server.close())
[error]                   ^
[error] one error found
[error] Compile failed at Aug 24, 2018 1:56:06 PM [31.582s]
```
which fails the Scala 2.12 branch build.

## How was this patch tested?

Existing tests

Closes #22229 from dbtsai/fix-2.12-build.

Authored-by: DB Tsai <d_tsai@apple.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-25 13:48:46 +08:00
Liang-Chi Hsieh 49a1993b16
[SPARK-25163][SQL] Fix flaky test: o.a.s.util.collection.ExternalAppendOnlyMapSuiteCheck
## What changes were proposed in this pull request?

`ExternalAppendOnlyMapSuiteCheck` test is flaky.

We use a `SparkListener` to collect spill metrics of completed stages. `withListener` runs the code that does spill. Spill status was checked after the code finishes but it was still in `withListener`. At that time it was possibly not all events to the listener bus are processed.

We should check spill status after all events are processed.

## How was this patch tested?

Locally ran unit tests.

Closes #22181 from viirya/SPARK-25163.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2018-08-22 14:17:05 -07:00
Mukul Murthy 68ec4d641b
[SPARK-25181][CORE] Limit Thread Pool size in BlockManager Master and Slave endpoints
## What changes were proposed in this pull request?

Limit Thread Pool size in BlockManager Master and Slave endpoints.

Currently, BlockManagerMasterEndpoint and BlockManagerSlaveEndpoint both have thread pools with nearly unbounded (Integer.MAX_VALUE) numbers of threads. In certain cases, this can lead to driver OOM errors. This change limits the thread pools to 100 threads; this should not break any existing behavior because any tasks beyond that number will get queued.

## How was this patch tested?

Manual testing

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

Closes #22176 from mukulmurthy/25181-threads.

Authored-by: Mukul Murthy <mukul.murthy@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2018-08-22 10:36:20 -07:00
Marco Gaido 55f36641ff [SPARK-25093][SQL] Avoid recompiling regexp for comments multiple times
## What changes were proposed in this pull request?

The PR moves the compilation of the regexp for code formatting outside the method which is called for each code block when splitting expressions, in order to avoid recompiling the regexp every time.

Credit should be given to Izek Greenfield.

## How was this patch tested?

existing UTs

Closes #22135 from mgaido91/SPARK-25093.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-22 14:31:51 +08:00
Xingbo Jiang ad45299d04 [SPARK-25095][PYSPARK] Python support for BarrierTaskContext
## What changes were proposed in this pull request?

Add method `barrier()` and `getTaskInfos()` in python TaskContext, these two methods are only allowed for barrier tasks.

## How was this patch tested?

Add new tests in `tests.py`

Closes #22085 from jiangxb1987/python.barrier.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2018-08-21 15:54:30 -07:00
Imran Rashid 99d2e4e007 [SPARK-24296][CORE] Replicate large blocks as a stream.
When replicating large cached RDD blocks, it can be helpful to replicate
them as a stream, to avoid using large amounts of memory during the
transfer.  This also allows blocks larger than 2GB to be replicated.

Added unit tests in DistributedSuite.  Also ran tests on a cluster for
blocks > 2gb.

Closes #21451 from squito/clean_replication.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-08-21 11:26:41 -07:00
Xingbo Jiang 5059255d91 [SPARK-25161][CORE] Fix several bugs in failure handling of barrier execution mode
## What changes were proposed in this pull request?

Fix several bugs in failure handling of barrier execution mode:
* Mark TaskSet for a barrier stage as zombie when a task attempt fails;
* Multiple barrier task failures from a single barrier stage should not trigger multiple stage retries;
* Barrier task failure from a previous failed stage attempt should not trigger stage retry;
* Fail the job when a task from a barrier ResultStage failed;
* RDD.isBarrier() should not rely on `ShuffleDependency`s.

## How was this patch tested?

Added corresponding test cases in `DAGSchedulerSuite` and `TaskSchedulerImplSuite`.

Closes #22158 from jiangxb1987/failure.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2018-08-21 08:25:02 -07:00
Ilan Filonenko ba84bcb2c4 [SPARK-24433][K8S] Initial R Bindings for SparkR on K8s
## What changes were proposed in this pull request?

Introducing R Bindings for Spark R on K8s

- [x] Running SparkR Job

## How was this patch tested?

This patch was tested with

- [x] Unit Tests
- [x] Integration Tests

## Example:

Commands to run example spark job:
1. `dev/make-distribution.sh --pip --r --tgz -Psparkr -Phadoop-2.7 -Pkubernetes`
2. `bin/docker-image-tool.sh -m -t testing build`
3.
```
bin/spark-submit \
    --master k8s://https://192.168.64.33:8443 \
    --deploy-mode cluster \
    --name spark-r \
    --conf spark.executor.instances=1 \
    --conf spark.kubernetes.container.image=spark-r:testing \
    local:///opt/spark/examples/src/main/r/dataframe.R
```

This above spark-submit command works given the distribution. (Will include this integration test in PR once PRB is ready).

Author: Ilan Filonenko <if56@cornell.edu>

Closes #21584 from ifilonenko/spark-r.
2018-08-17 16:04:02 -07:00
Sean Owen b3e6fe7c46 [SPARK-23654][BUILD] remove jets3t as a dependency of spark
## What changes were proposed in this pull request?

Remove jets3t dependency, and bouncy castle which it brings in; update licenses and deps
Note this just takes over https://github.com/apache/spark/pull/21146

## How was this patch tested?

Existing tests.

Closes #22081 from srowen/SPARK-23654.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-08-16 12:34:23 -07:00
Xingbo Jiang bfb74394a5 [SPARK-24819][CORE] Fail fast when no enough slots to launch the barrier stage on job submitted
## What changes were proposed in this pull request?

We shall check whether the barrier stage requires more slots (to be able to launch all tasks in the barrier stage together) than the total number of active slots currently, and fail fast if trying to submit a barrier stage that requires more slots than current total number.

This PR proposes to add a new method `getNumSlots()` to try to get the total number of currently active slots in `SchedulerBackend`, support of this new method has been added to all the first-class scheduler backends except `MesosFineGrainedSchedulerBackend`.

## How was this patch tested?

Added new test cases in `BarrierStageOnSubmittedSuite`.

Closes #22001 from jiangxb1987/SPARK-24819.

Lead-authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Co-authored-by: Xiangrui Meng <meng@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2018-08-15 13:31:28 -07:00
Imran Rashid 1024875843 [SPARK-25088][CORE][MESOS][DOCS] Update Rest Server docs & defaults.
## What changes were proposed in this pull request?

(a) disabled rest submission server by default in standalone mode
(b) fails the standalone master if rest server enabled & authentication secret set
(c) fails the mesos cluster dispatcher if authentication secret set
(d) doc updates
(e) when submitting a standalone app, only try the rest submission first if spark.master.rest.enabled=true

otherwise you'd see a 10 second pause like
18/08/09 08:13:22 INFO RestSubmissionClient: Submitting a request to launch an application in spark://...
18/08/09 08:13:33 WARN RestSubmissionClient: Unable to connect to server spark://...

I also made sure the mesos cluster dispatcher failed with the secret enabled, though I had to do that on slightly different code as I don't have mesos native libs around.

## How was this patch tested?

I ran the tests in the mesos module & in core for org.apache.spark.deploy.*

I ran a test on a cluster with standalone master to make sure I could still start with the right configs, and would fail the right way too.

Closes #22071 from squito/rest_doc_updates.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-08-14 13:02:33 -05:00
Eyal Farago 2e3abdff23 [SPARK-22713][CORE] ExternalAppendOnlyMap leaks when spilled during iteration
## What changes were proposed in this pull request?
This PR solves [SPARK-22713](https://issues.apache.org/jira/browse/SPARK-22713) which describes a memory leak that occurs when and ExternalAppendOnlyMap is spilled during iteration (opposed to  insertion).

(Please fill in changes proposed in this fix)
ExternalAppendOnlyMap's iterator supports spilling but it kept a reference to the internal map (via an internal iterator) after spilling, it seems that the original code was actually supposed to 'get rid' of this reference on the next iteration but according to the elaborate investigation described in the JIRA this didn't happen.
the fix was simply replacing the internal iterator immediately after spilling.

## How was this patch tested?
I've introduced a new test to test suite ExternalAppendOnlyMapSuite, this test asserts that neither the external map itself nor its iterator hold any reference to the internal map after a spill.
These approach required some access relaxation of some members variables and nested classes of ExternalAppendOnlyMap, this members are now package provate and annotated with VisibleForTesting.

Closes #21369 from eyalfa/SPARK-22713__ExternalAppendOnlyMap_effective_spill.

Authored-by: Eyal Farago <eyal@nrgene.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-13 20:55:46 +08:00
Xiao Li a9928277da
[SPARK-24420][BUILD][FOLLOW-UP] Upgrade ASM6 APIs
## What changes were proposed in this pull request?
Use ASM 6 APIs after we upgrading it to ASM6.

## How was this patch tested?
N/A

Closes #22082 from gatorsmile/asm6.

Authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2018-08-13 05:59:08 +00:00
10129659 02d0a1ffd9 [SPARK-25069][CORE] Using UnsafeAlignedOffset to make the entire record of 8 byte Items aligned like which is used in UnsafeExternalSorter
## What changes were proposed in this pull request?

The class of UnsafeExternalSorter used UnsafeAlignedOffset to make the entire record of 8 byte Items aligned, but ShuffleExternalSorter not.
The SPARC platform requires this because using a 4 byte Int for record lengths causes the entire record of 8 byte Items to become misaligned by 4 bytes. Using a 8 byte long for record length keeps things 8 byte aligned.

## How was this patch tested?
Existing Test.

Closes #22053 from eatoncys/UnsafeAlignedOffset.

Authored-by: 10129659 <chen.yanshan@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-13 09:09:25 +08:00
Kazuhiro Sera 8ec25cd67e Fix typos detected by github.com/client9/misspell
## What changes were proposed in this pull request?

Fixing typos is sometimes very hard. It's not so easy to visually review them. Recently, I discovered a very useful tool for it, [misspell](https://github.com/client9/misspell).

This pull request fixes minor typos detected by [misspell](https://github.com/client9/misspell) except for the false positives. If you would like me to work on other files as well, let me know.

## How was this patch tested?

### before

```
$ misspell . | grep -v '.js'
R/pkg/R/SQLContext.R:354:43: "definiton" is a misspelling of "definition"
R/pkg/R/SQLContext.R:424:43: "definiton" is a misspelling of "definition"
R/pkg/R/SQLContext.R:445:43: "definiton" is a misspelling of "definition"
R/pkg/R/SQLContext.R:495:43: "definiton" is a misspelling of "definition"
NOTICE-binary:454:16: "containd" is a misspelling of "contained"
R/pkg/R/context.R:46:43: "definiton" is a misspelling of "definition"
R/pkg/R/context.R:74:43: "definiton" is a misspelling of "definition"
R/pkg/R/DataFrame.R:591:48: "persistance" is a misspelling of "persistence"
R/pkg/R/streaming.R:166:44: "occured" is a misspelling of "occurred"
R/pkg/inst/worker/worker.R:65:22: "ouput" is a misspelling of "output"
R/pkg/tests/fulltests/test_utils.R:106:25: "environemnt" is a misspelling of "environment"
common/kvstore/src/test/java/org/apache/spark/util/kvstore/InMemoryStoreSuite.java:38:39: "existant" is a misspelling of "existent"
common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java:83:39: "existant" is a misspelling of "existent"
common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java:243:46: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:234:19: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:238:63: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:244:46: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:276:39: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/util/AbstractFileRegion.java:27:20: "transfered" is a misspelling of "transferred"
common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala:195:15: "orgin" is a misspelling of "origin"
core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala:621:39: "gauranteed" is a misspelling of "guaranteed"
core/src/main/scala/org/apache/spark/status/storeTypes.scala:113:29: "ect" is a misspelling of "etc"
core/src/main/scala/org/apache/spark/storage/DiskStore.scala:282:18: "transfered" is a misspelling of "transferred"
core/src/main/scala/org/apache/spark/util/ListenerBus.scala:64:17: "overriden" is a misspelling of "overridden"
core/src/test/scala/org/apache/spark/ShuffleSuite.scala:211:7: "substracted" is a misspelling of "subtracted"
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:1922:49: "agriculteur" is a misspelling of "agriculture"
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:2468:84: "truely" is a misspelling of "truly"
core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala:25:18: "persistance" is a misspelling of "persistence"
core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala:26:69: "persistance" is a misspelling of "persistence"
data/streaming/AFINN-111.txt:1219:0: "humerous" is a misspelling of "humorous"
dev/run-pip-tests:55:28: "enviroments" is a misspelling of "environments"
dev/run-pip-tests:91:37: "virutal" is a misspelling of "virtual"
dev/merge_spark_pr.py:377:72: "accross" is a misspelling of "across"
dev/merge_spark_pr.py:378:66: "accross" is a misspelling of "across"
dev/run-pip-tests:126:25: "enviroments" is a misspelling of "environments"
docs/configuration.md:1830:82: "overriden" is a misspelling of "overridden"
docs/structured-streaming-programming-guide.md:525:45: "processs" is a misspelling of "processes"
docs/structured-streaming-programming-guide.md:1165:61: "BETWEN" is a misspelling of "BETWEEN"
docs/sql-programming-guide.md:1891:810: "behaivor" is a misspelling of "behavior"
examples/src/main/python/sql/arrow.py:98:8: "substract" is a misspelling of "subtract"
examples/src/main/python/sql/arrow.py:103:27: "substract" is a misspelling of "subtract"
licenses/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING"
licenses/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING"
licenses/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses-binary/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING"
licenses-binary/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses-binary/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING"
licenses-binary/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/hungarian.txt:170:0: "teh" is a misspelling of "the"
mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/portuguese.txt:53:0: "eles" is a misspelling of "eels"
mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:99:20: "Euclidian" is a misspelling of "Euclidean"
mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:539:11: "Euclidian" is a misspelling of "Euclidean"
mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala:77:36: "Teh" is a misspelling of "The"
mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala:230:24: "inital" is a misspelling of "initial"
mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala:276:9: "Euclidian" is a misspelling of "Euclidean"
mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala:237:26: "descripiton" is a misspelling of "descriptions"
python/pyspark/find_spark_home.py:30:13: "enviroment" is a misspelling of "environment"
python/pyspark/context.py:937:12: "supress" is a misspelling of "suppress"
python/pyspark/context.py:938:12: "supress" is a misspelling of "suppress"
python/pyspark/context.py:939:12: "supress" is a misspelling of "suppress"
python/pyspark/context.py:940:12: "supress" is a misspelling of "suppress"
python/pyspark/heapq3.py:6:63: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:7:2: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:263:29: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:263:39: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:270:49: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:270:59: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:275:2: "STICHTING" is a misspelling of "STITCHING"
python/pyspark/heapq3.py:275:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
python/pyspark/heapq3.py:277:29: "STICHTING" is a misspelling of "STITCHING"
python/pyspark/heapq3.py:277:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
python/pyspark/heapq3.py:713:8: "probabilty" is a misspelling of "probability"
python/pyspark/ml/clustering.py:1038:8: "Currenlty" is a misspelling of "Currently"
python/pyspark/ml/stat.py:339:23: "Euclidian" is a misspelling of "Euclidean"
python/pyspark/ml/regression.py:1378:20: "paramter" is a misspelling of "parameter"
python/pyspark/mllib/stat/_statistics.py:262:8: "probabilty" is a misspelling of "probability"
python/pyspark/rdd.py:1363:32: "paramter" is a misspelling of "parameter"
python/pyspark/streaming/tests.py:825:42: "retuns" is a misspelling of "returns"
python/pyspark/sql/tests.py:768:29: "initalization" is a misspelling of "initialization"
python/pyspark/sql/tests.py:3616:31: "initalize" is a misspelling of "initialize"
resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala:120:39: "arbitary" is a misspelling of "arbitrary"
resource-managers/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArgumentsSuite.scala:26:45: "sucessfully" is a misspelling of "successfully"
resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala:358:27: "constaints" is a misspelling of "constraints"
resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala:111:24: "senstive" is a misspelling of "sensitive"
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala:1063:5: "overwirte" is a misspelling of "overwrite"
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala:1348:17: "compatability" is a misspelling of "compatibility"
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:77:36: "paramter" is a misspelling of "parameter"
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:1374:22: "precendence" is a misspelling of "precedence"
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala:238:27: "unnecassary" is a misspelling of "unnecessary"
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala:212:17: "whn" is a misspelling of "when"
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinHelper.scala:147:60: "timestmap" is a misspelling of "timestamp"
sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala:150:45: "precentage" is a misspelling of "percentage"
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala:135:29: "infered" is a misspelling of "inferred"
sql/hive/src/test/resources/golden/udf_instr-1-2e76f819563dbaba4beb51e3a130b922:1:52: "occurance" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_instr-2-32da357fc754badd6e3898dcc8989182:1:52: "occurance" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_locate-1-6e41693c9c6dceea4d7fab4c02884e4e:1:63: "occurance" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_locate-2-d9b5934457931447874d6bb7c13de478:1:63: "occurance" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_translate-2-f7aa38a33ca0df73b7a1e6b6da4b7fe8:9:79: "occurence" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_translate-2-f7aa38a33ca0df73b7a1e6b6da4b7fe8:13:110: "occurence" is a misspelling of "occurrence"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_join.q:46:105: "distint" is a misspelling of "distinct"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q:29:3: "Currenly" is a misspelling of "Currently"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_partitioned.q:72:15: "existant" is a misspelling of "existent"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_udf.q:25:3: "substraction" is a misspelling of "subtraction"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q:16:51: "funtion" is a misspelling of "function"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_8.q:15:30: "issueing" is a misspelling of "issuing"
sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala:669:52: "wiht" is a misspelling of "with"
sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java:474:9: "Refering" is a misspelling of "Referring"
```

### after

```
$ misspell . | grep -v '.js'
common/network-common/src/main/java/org/apache/spark/network/util/AbstractFileRegion.java:27:20: "transfered" is a misspelling of "transferred"
core/src/main/scala/org/apache/spark/status/storeTypes.scala:113:29: "ect" is a misspelling of "etc"
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:1922:49: "agriculteur" is a misspelling of "agriculture"
data/streaming/AFINN-111.txt:1219:0: "humerous" is a misspelling of "humorous"
licenses/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING"
licenses/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING"
licenses/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses-binary/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING"
licenses-binary/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses-binary/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING"
licenses-binary/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/hungarian.txt:170:0: "teh" is a misspelling of "the"
mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/portuguese.txt:53:0: "eles" is a misspelling of "eels"
mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:99:20: "Euclidian" is a misspelling of "Euclidean"
mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:539:11: "Euclidian" is a misspelling of "Euclidean"
mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala:77:36: "Teh" is a misspelling of "The"
mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala:276:9: "Euclidian" is a misspelling of "Euclidean"
python/pyspark/heapq3.py:6:63: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:7:2: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:263:29: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:263:39: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:270:49: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:270:59: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:275:2: "STICHTING" is a misspelling of "STITCHING"
python/pyspark/heapq3.py:275:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
python/pyspark/heapq3.py:277:29: "STICHTING" is a misspelling of "STITCHING"
python/pyspark/heapq3.py:277:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
python/pyspark/ml/stat.py:339:23: "Euclidian" is a misspelling of "Euclidean"
```

Closes #22070 from seratch/fix-typo.

Authored-by: Kazuhiro Sera <seratch@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2018-08-11 21:23:36 -05:00
Xingbo Jiang 4855d5c4b9 [SPARK-24822][PYSPARK] Python support for barrier execution mode
## What changes were proposed in this pull request?

This PR add python support for barrier execution mode, thus enable launch a job containing barrier stage(s) from PySpark.

We just forked the existing `RDDBarrier` and `RDD.barrier()` in Python api.

## How was this patch tested?

Manually tested:
```
>>> rdd = sc.parallelize([1, 2, 3, 4])
>>> def f(iterator): yield sum(iterator)
...
>>> rdd.barrier().mapPartitions(f).isBarrier() == True
True
```

Unit tests will be added in a follow-up PR that implements BarrierTaskContext on python side.

Closes #22011 from jiangxb1987/python.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-11 21:44:45 +08:00
Shixiong Zhu f5aba65739
[SPARK-25081][CORE] Nested spill in ShuffleExternalSorter should not access released memory page
## What changes were proposed in this pull request?

This issue is pretty similar to [SPARK-21907](https://issues.apache.org/jira/browse/SPARK-21907).

"allocateArray" in [ShuffleInMemorySorter.reset](9b8521e53e/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java (L99)) may trigger a spill and cause ShuffleInMemorySorter access the released `array`. Another task may get the same memory page from the pool. This will cause two tasks access the same memory page. When a task reads memory written by another task, many types of failures may happen. Here are some examples I  have seen:

- JVM crash. (This is easy to reproduce in a unit test as we fill newly allocated and deallocated memory with 0xa5 and 0x5a bytes which usually points to an invalid memory address)
- java.lang.IllegalArgumentException: Comparison method violates its general contract!
- java.lang.NullPointerException at org.apache.spark.memory.TaskMemoryManager.getPage(TaskMemoryManager.java:384)
- java.lang.UnsupportedOperationException: Cannot grow BufferHolder by size -536870912 because the size after growing exceeds size limitation 2147483632

This PR resets states in `ShuffleInMemorySorter.reset` before calling `allocateArray` to fix the issue.

## How was this patch tested?

The new unit test will make JVM crash without the fix.

Closes #22062 from zsxwing/SPARK-25081.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2018-08-10 10:53:44 -07:00
Liang-Chi Hsieh 4f17585098 [SPARK-19355][SQL] Use map output statistics to improve global limit's parallelism
## What changes were proposed in this pull request?

A logical `Limit` is performed physically by two operations `LocalLimit` and `GlobalLimit`.

Most of time, we gather all data into a single partition in order to run `GlobalLimit`. If we use a very big limit number, shuffling data causes performance issue also reduces parallelism.

We can avoid shuffling into single partition if we don't care data ordering. This patch implements this idea by doing a map stage during global limit. It collects the info of row numbers at each partition. For each partition, we locally retrieves limited data without any shuffling to finish this global limit.

For example, we have three partitions with rows (100, 100, 50) respectively. In global limit of 100 rows, we may take (34, 33, 33) rows for each partition locally. After global limit we still have three partitions.

If the data partition has certain ordering, we can't distribute required rows evenly to each partitions because it could change data ordering. But we still can avoid shuffling.

## How was this patch tested?

Jenkins tests.

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

Closes #16677 from viirya/improve-global-limit-parallelism.
2018-08-10 11:32:15 +02:00
Kazuaki Ishizaki 56e9e97073 [MINOR][DOC] Fix typo
## What changes were proposed in this pull request?

This PR fixes typo regarding `auxiliary verb + verb[s]`. This is a follow-on of #21956.

## How was this patch tested?

N/A

Closes #22040 from kiszk/spellcheck1.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-09 20:10:17 +08:00
Xingbo Jiang d90f1336d8 [SPARK-25045][CORE] Make RDDBarrier.mapParititions similar to RDD.mapPartitions
## What changes were proposed in this pull request?

Signature of the function passed to `RDDBarrier.mapPartitions()` is different from that of `RDD.mapPartitions`. The later doesn’t take a `TaskContext`. We shall make the function signature the same to avoid confusion and misusage.

This PR proposes the following API changes:
- In `RDDBarrier`, migrate `mapPartitions` from
   ```
        def mapPartitions[S: ClassTag](
            f: (Iterator[T], BarrierTaskContext) => Iterator[S],
            preservesPartitioning: Boolean = false): RDD[S]
        }
   ```
    to
   ```
        def mapPartitions[S: ClassTag](
            f: Iterator[T] => Iterator[S],
            preservesPartitioning: Boolean = false): RDD[S]
        }
   ```
- Add new static method to get a `BarrierTaskContext`:
   ```
        object BarrierTaskContext {
           def get(): BarrierTaskContext
        }
   ```

## How was this patch tested?

Existing test cases.

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #22026 from jiangxb1987/mapPartitions.
2018-08-07 17:32:41 -07:00
Sean Owen 66699c5c30 [SPARK-25029][TESTS] Scala 2.12 issues: TaskNotSerializable and Janino "Two non-abstract methods ..." errors
## What changes were proposed in this pull request?

Fixes for test issues that arose after Scala 2.12 support was added -- ones that only affect the 2.12 build.

## How was this patch tested?

Existing tests.

Closes #22004 from srowen/SPARK-25029.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2018-08-07 17:30:37 -05:00
Maxim Gekk 131ca146ed [SPARK-24005][CORE] Remove usage of Scala’s parallel collection
## What changes were proposed in this pull request?

In the PR, I propose to replace Scala parallel collections by new methods `parmap()`. The methods use futures to transform a sequential collection by applying a lambda function to each element in parallel. The result of `parmap` is another regular (sequential) collection.

The proposed `parmap` method aims to solve the problem of impossibility to interrupt parallel Scala collection. This possibility is needed for reliable task preemption.

## How was this patch tested?

A test was added to `ThreadUtilsSuite`

Closes #21913 from MaxGekk/par-map.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-07 17:14:30 +08:00
Xingbo Jiang 388f5a0635 [SPARK-24817][CORE] Implement BarrierTaskContext.barrier()
## What changes were proposed in this pull request?

Implement BarrierTaskContext.barrier(), to support global sync between all the tasks in a barrier stage.
The function set a global barrier and waits until all tasks in this stage hit this barrier. Similar to MPI_Barrier function in MPI, the barrier() function call blocks until all tasks in the same stage have reached this routine. The global sync shall finish immediately once all tasks in the same barrier stage reaches the same barrier.

This PR implements BarrierTaskContext.barrier() based on netty-based RPC client, introduces new `BarrierCoordinator` and new `BarrierCoordinatorMessage`, and new config to handle timeout issue.

## How was this patch tested?
Add `BarrierTaskContextSuite` to test `BarrierTaskContext.barrier()`

Closes #21898 from jiangxb1987/taskcontext.barrier.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-07 15:06:32 +08:00
deshanxiao 1076e4f002 [MINOR][DOCS] Fix grammatical error in SortShuffleManager
## What changes were proposed in this pull request?

Fix a grammatical error in the comment of SortShuffleManager.

## How was this patch tested?

N/A

Closes #21956 from deshanxiao/master.

Authored-by: deshanxiao <42019462+deshanxiao@users.noreply.github.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-07 09:36:37 +08:00
Marco Gaido 3c96937c7b [SPARK-24948][SHS] Delegate check access permissions to the file system
## What changes were proposed in this pull request?

In `SparkHadoopUtil. checkAccessPermission`,  we consider only basic permissions in order to check wether a user can access a file or not. This is not a complete check, as it ignores ACLs and other policies a file system may apply in its internal. So this can result in returning wrongly that a user cannot access a file (despite he actually can).

The PR proposes to delegate to the filesystem the check whether a file is accessible or not, in order to return the right result. A caching layer is added for performance reasons.

## How was this patch tested?

modified UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21895 from mgaido91/SPARK-24948.
2018-08-06 14:29:05 -07:00
Hieu Huynh 51e2b38d93 [SPARK-24992][CORE] spark should randomize yarn local dir selection
**Description: [SPARK-24992](https://issues.apache.org/jira/browse/SPARK-24992)**
Utils.getLocalDir is used to get path of a temporary directory. However, it always returns the the same directory, which is the first element in the array localRootDirs. When running on YARN, this might causes the case that we always write to one disk, which makes it busy while other disks are free. We should randomize the selection to spread out the loads.

**What changes were proposed in this pull request?**
This PR randomized the selection of local directory inside the method Utils.getLocalDir. This change affects the Utils.fetchFile method since it based on the fact that Utils.getLocalDir always return the same directory to cache file. Therefore, a new variable cachedLocalDir is used to cache the first localDirectory that it gets from Utils.getLocalDir. Also, when getting the configured local directories (inside Utils. getConfiguredLocalDirs), in case we are in yarn mode, the array of directories are also randomized before return.

Author: Hieu Huynh <“Hieu.huynh@oath.com”>

Closes #21953 from hthuynh2/SPARK_24992.
2018-08-06 13:58:28 -05:00