Commit graph

8098 commits

Author SHA1 Message Date
attilapiros 806edf8f44 [SPARK-35610][CORE] Fix the memory leak introduced by the Executor's stop shutdown hook
### What changes were proposed in this pull request?

Fixing the memory leak by deregistering the shutdown hook when the executor is stopped. This way the Garbage Collector can release the executor object early. Which is a huge win for our tests as user's classloader could be also released which keeps references to objects which are created for the jars on the classpath.

### Why are the changes needed?

I have identified this leak by running the Livy tests (I know it is close to the attic but this leak causes a constant OOM there) and it is in our Spark unit tests as well.

This leak can be identified by checking the number of `LeakyEntry` in case of Scala 2.12.14 (and `ZipEntry` for Scala 2.12.10) instances which with its related data can take up a considerable amount of memory (as those are created from the jars which are on the classpath).

I have my own tool for instrumenting JVM code [trace-agent](https://github.com/attilapiros/trace-agent) and with that I am able to call JVM diagnostic commands at specific methods. Let me show how it in action.

It has a single text file embedded into the tool's jar called action.txt.
In this case actions.txt content is:

{noformat}
$ unzip -q -c trace-agent-0.0.7.jar actions.txt
diagnostic_command org.apache.spark.repl.ReplSuite runInterpreter  cmd:gcClassHistogram,limit_output_lines:8,where:beforeAndAfter,with_gc:true
diagnostic_command org.apache.spark.repl.ReplSuite afterAll  cmd:gcClassHistogram,limit_output_lines:8,where:after,with_gc:true
{noformat}

Which creates a class histogram at the beginning and at the end of `org.apache.spark.repl.ReplSuite#runInterpreter()` (after triggering a GC which might not finish as GC is done in a separate thread..) and one histogram in the end of the `org.apache.spark.repl.ReplSuite#afterAll()` method.

And the histograms are the followings on master branch:

```
$ ./build/sbt ";project repl;set Test/javaOptions += \"-javaagent:/Users/attilazsoltpiros/git/attilapiros/memoryLeak/trace-agent-0.0.7.jar\"; testOnly" |grep "ZipEntry\|LeakyEntry"
   3:        197089        9460272  scala.reflect.io.FileZipArchive$LeakyEntry
   3:        197089        9460272  scala.reflect.io.FileZipArchive$LeakyEntry
   3:        197089        9460272  scala.reflect.io.FileZipArchive$LeakyEntry
   3:        197089        9460272  scala.reflect.io.FileZipArchive$LeakyEntry
   3:        197089        9460272  scala.reflect.io.FileZipArchive$LeakyEntry
   3:        197089        9460272  scala.reflect.io.FileZipArchive$LeakyEntry
   3:        394178       18920544  scala.reflect.io.FileZipArchive$LeakyEntry
   3:        394178       18920544  scala.reflect.io.FileZipArchive$LeakyEntry
   3:        591267       28380816  scala.reflect.io.FileZipArchive$LeakyEntry
   3:        591267       28380816  scala.reflect.io.FileZipArchive$LeakyEntry
   3:        788356       37841088  scala.reflect.io.FileZipArchive$LeakyEntry
   3:        788356       37841088  scala.reflect.io.FileZipArchive$LeakyEntry
   3:        985445       47301360  scala.reflect.io.FileZipArchive$LeakyEntry
   3:        985445       47301360  scala.reflect.io.FileZipArchive$LeakyEntry
   3:       1182534       56761632  scala.reflect.io.FileZipArchive$LeakyEntry
   3:       1182534       56761632  scala.reflect.io.FileZipArchive$LeakyEntry
   3:       1379623       66221904  scala.reflect.io.FileZipArchive$LeakyEntry
   3:       1379623       66221904  scala.reflect.io.FileZipArchive$LeakyEntry
   3:       1576712       75682176  scala.reflect.io.FileZipArchive$LeakyEntry
```

Where the header of the table is:

```
num     #instances         #bytes  class name
```

So the `LeakyEntry` in the end is about 75MB (173MB in case of Scala 2.12.10 and before for another class called `ZipEntry`) but the first item (a char/byte arrays) and the second item (strings) in the histogram also relates to this leak:

```
$ ./build/sbt ";project repl;set Test/javaOptions += \"-javaagent:/Users/attilazsoltpiros/git/attilapiros/memoryLeak/trace-agent-0.0.7.jar\"; testOnly" |grep "1:\|2:\|3:"
   1:          2701        3496112  [B
   2:         21855        2607192  [C
   3:          4885         537264  java.lang.Class
   1:        480323       55970208  [C
   2:        480499       11531976  java.lang.String
   3:        197089        9460272  scala.reflect.io.FileZipArchive$LeakyEntry
   1:        481825       56148024  [C
   2:        481998       11567952  java.lang.String
   3:        197089        9460272  scala.reflect.io.FileZipArchive$LeakyEntry
   1:        487056       57550344  [C
   2:        487179       11692296  java.lang.String
   3:        197089        9460272  scala.reflect.io.FileZipArchive$LeakyEntry
   1:        487054       57551008  [C
   2:        487176       11692224  java.lang.String
   3:        197089        9460272  scala.reflect.io.FileZipArchive$LeakyEntry
   1:        927823      107139160  [C
   2:        928072       22273728  java.lang.String
   3:        394178       18920544  scala.reflect.io.FileZipArchive$LeakyEntry
   1:        927793      107129328  [C
   2:        928041       22272984  java.lang.String
   3:        394178       18920544  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       1361851      155555608  [C
   2:       1362261       32694264  java.lang.String
   3:        591267       28380816  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       1361683      155493464  [C
   2:       1362092       32690208  java.lang.String
   3:        591267       28380816  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       1803074      205157728  [C
   2:       1803268       43278432  java.lang.String
   3:        788356       37841088  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       1802385      204938224  [C
   2:       1802579       43261896  java.lang.String
   3:        788356       37841088  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       2236631      253636592  [C
   2:       2237029       53688696  java.lang.String
   3:        985445       47301360  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       2236536      253603008  [C
   2:       2236933       53686392  java.lang.String
   3:        985445       47301360  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       2668892      301893920  [C
   2:       2669510       64068240  java.lang.String
   3:       1182534       56761632  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       2668759      301846376  [C
   2:       2669376       64065024  java.lang.String
   3:       1182534       56761632  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       3101238      350101048  [C
   2:       3102073       74449752  java.lang.String
   3:       1379623       66221904  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       3101240      350101104  [C
   2:       3102075       74449800  java.lang.String
   3:       1379623       66221904  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       3533785      398371760  [C
   2:       3534835       84836040  java.lang.String
   3:       1576712       75682176  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       3533759      398367088  [C
   2:       3534807       84835368  java.lang.String
   3:       1576712       75682176  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       3967049      446893400  [C
   2:       3968314       95239536  java.lang.String
   3:       1773801       85142448  scala.reflect.io.FileZipArchive$LeakyEntry
[info] - SPARK-26633: ExecutorClassLoader.getResourceAsStream find REPL classes (8 seconds, 248 milliseconds)
Setting default log level to "ERROR".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
   1:       3966423      446709584  [C
   2:       3967682       95224368  java.lang.String
   3:       1773801       85142448  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       4399583      495097208  [C
   2:       4401050      105625200  java.lang.String
   3:       1970890       94602720  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       4399578      495070064  [C
   2:       4401040      105624960  java.lang.String
   3:       1970890       94602720  scala.reflect.io.FileZipArchive$LeakyEntry
```

The last three is about 700MB altogether.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?

I used the trace-agent tool with the same settings for the modified code:

```
$ ./build/sbt ";project repl;set Test/javaOptions += \"-javaagent:/Users/attilazsoltpiros/git/attilapiros/memoryLeak/trace-agent-0.0.7.jar\"; testOnly" |grep "1:\|2:\|3:"
   1:          2701        3496112  [B
   2:         21855        2607192  [C
   3:          4885         537264  java.lang.Class
   1:        480323       55970208  [C
   2:        480499       11531976  java.lang.String
   3:        197089        9460272  scala.reflect.io.FileZipArchive$LeakyEntry
   1:        481825       56148024  [C
   2:        481998       11567952  java.lang.String
   3:        197089        9460272  scala.reflect.io.FileZipArchive$LeakyEntry
   1:        487056       57550344  [C
   2:        487179       11692296  java.lang.String
   3:        197089        9460272  scala.reflect.io.FileZipArchive$LeakyEntry
   1:        487054       57551008  [C
   2:        487176       11692224  java.lang.String
   3:        197089        9460272  scala.reflect.io.FileZipArchive$LeakyEntry
   1:        927823      107139160  [C
   2:        928072       22273728  java.lang.String
   3:        394178       18920544  scala.reflect.io.FileZipArchive$LeakyEntry
   1:        927793      107129328  [C
   2:        928041       22272984  java.lang.String
   3:        394178       18920544  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       1361851      155555608  [C
   2:       1362261       32694264  java.lang.String
   3:        591267       28380816  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       1361683      155493464  [C
   2:       1362092       32690208  java.lang.String
   3:        591267       28380816  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       1803074      205157728  [C
   2:       1803268       43278432  java.lang.String
   3:        788356       37841088  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       1802385      204938224  [C
   2:       1802579       43261896  java.lang.String
   3:        788356       37841088  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       2236631      253636592  [C
   2:       2237029       53688696  java.lang.String
   3:        985445       47301360  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       2236536      253603008  [C
   2:       2236933       53686392  java.lang.String
   3:        985445       47301360  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       2668892      301893920  [C
   2:       2669510       64068240  java.lang.String
   3:       1182534       56761632  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       2668759      301846376  [C
   2:       2669376       64065024  java.lang.String
   3:       1182534       56761632  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       3101238      350101048  [C
   2:       3102073       74449752  java.lang.String
   3:       1379623       66221904  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       3101240      350101104  [C
   2:       3102075       74449800  java.lang.String
   3:       1379623       66221904  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       3533785      398371760  [C
   2:       3534835       84836040  java.lang.String
   3:       1576712       75682176  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       3533759      398367088  [C
   2:       3534807       84835368  java.lang.String
   3:       1576712       75682176  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       3967049      446893400  [C
   2:       3968314       95239536  java.lang.String
   3:       1773801       85142448  scala.reflect.io.FileZipArchive$LeakyEntry
[info] - SPARK-26633: ExecutorClassLoader.getResourceAsStream find REPL classes (8 seconds, 248 milliseconds)
Setting default log level to "ERROR".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
   1:       3966423      446709584  [C
   2:       3967682       95224368  java.lang.String
   3:       1773801       85142448  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       4399583      495097208  [C
   2:       4401050      105625200  java.lang.String
   3:       1970890       94602720  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       4399578      495070064  [C
   2:       4401040      105624960  java.lang.String
   3:       1970890       94602720  scala.reflect.io.FileZipArchive$LeakyEntry
[success] Total time: 174 s (02:54), completed Jun 2, 2021 2:00:43 PM
╭─attilazsoltpirosapiros-MBP16 ~/git/attilapiros/memoryLeak ‹SPARK-35610*›
╰─$ vim
╭─attilazsoltpirosapiros-MBP16 ~/git/attilapiros/memoryLeak ‹SPARK-35610*›
╰─$ ./build/sbt ";project repl;set Test/javaOptions += \"-javaagent:/Users/attilazsoltpiros/git/attilapiros/memoryLeak/trace-agent-0.0.7.jar\"; testOnly" |grep "1:\|2:\|3:"
   1:          2685        3457368  [B
   2:         21833        2606712  [C
   3:          4885         537264  java.lang.Class
   1:        480245       55978400  [C
   2:        480421       11530104  java.lang.String
   3:        197089        9460272  scala.reflect.io.FileZipArchive$LeakyEntry
   1:        480460       56005784  [C
   2:        480633       11535192  java.lang.String
   3:        197089        9460272  scala.reflect.io.FileZipArchive$LeakyEntry
   1:        486643       57537784  [C
   2:        486766       11682384  java.lang.String
   3:        197089        9460272  scala.reflect.io.FileZipArchive$LeakyEntry
   1:        486636       57538192  [C
   2:        486758       11682192  java.lang.String
   3:        197089        9460272  scala.reflect.io.FileZipArchive$LeakyEntry
   1:        501208       60411856  [C
   2:        501180       12028320  java.lang.String
   3:        197089        9460272  scala.reflect.io.FileZipArchive$LeakyEntry
   1:        501206       60412960  [C
   2:        501177       12028248  java.lang.String
   3:        197089        9460272  scala.reflect.io.FileZipArchive$LeakyEntry
   1:        934925      108773320  [C
   2:        935058       22441392  java.lang.String
   3:        394178       18920544  scala.reflect.io.FileZipArchive$LeakyEntry
   1:        934912      108769528  [C
   2:        935044       22441056  java.lang.String
   3:        394178       18920544  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       1370351      156901296  [C
   2:       1370318       32887632  java.lang.String
   3:        591267       28380816  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       1369660      156681680  [C
   2:       1369627       32871048  java.lang.String
   3:        591267       28380816  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       1803746      205383136  [C
   2:       1803917       43294008  java.lang.String
   3:        788356       37841088  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       1803658      205353096  [C
   2:       1803828       43291872  java.lang.String
   3:        788356       37841088  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       2235677      253608240  [C
   2:       2236068       53665632  java.lang.String
   3:        985445       47301360  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       2235539      253560088  [C
   2:       2235929       53662296  java.lang.String
   3:        985445       47301360  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       2667775      301799240  [C
   2:       2668383       64041192  java.lang.String
   3:       1182534       56761632  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       2667765      301798568  [C
   2:       2668373       64040952  java.lang.String
   3:       1182534       56761632  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       2666665      301491096  [C
   2:       2667285       64014840  java.lang.String
   3:       1182534       56761632  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       2666648      301490792  [C
   2:       2667266       64014384  java.lang.String
   3:       1182534       56761632  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       2668169      301833032  [C
   2:       2668782       64050768  java.lang.String
   3:       1182534       56761632  scala.reflect.io.FileZipArchive$LeakyEntry
[info] - SPARK-26633: ExecutorClassLoader.getResourceAsStream find REPL classes (6 seconds, 396 milliseconds)
Setting default log level to "ERROR".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
   1:       2235495      253419952  [C
   2:       2235887       53661288  java.lang.String
   3:        985445       47301360  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       2668379      301800768  [C
   2:       2668979       64055496  java.lang.String
   3:       1182534       56761632  scala.reflect.io.FileZipArchive$LeakyEntry
   1:       2236123      253522640  [C
   2:       2236514       53676336  java.lang.String
   3:        985445       47301360  scala.reflect.io.FileZipArchive$LeakyEntry
```

The sum of the last three numbers is about 354MB.

Closes #32748 from attilapiros/SPARK-35610.

Authored-by: attilapiros <piros.attila.zsolt@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-06-02 09:34:28 -07:00
Gengliang Wang 9d0d4edb43 [SPARK-35595][TESTS] Support multiple loggers in testing method withLogAppender
### What changes were proposed in this pull request?

A test case of AdaptiveQueryExecSuite becomes flaky since there are too many debug logs in RootLogger:
https://github.com/Yikun/spark/runs/2715222392?check_suite_focus=true
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/139125/testReport/

To fix it,  I suggest supporting multiple loggers in the testing method withLogAppender. So that the LogAppender gets clean target log outputs.

### Why are the changes needed?

Fix a flaky test case.
Also, reduce unnecessary memory cost in tests.

### Does this PR introduce _any_ user-facing change?

No
### How was this patch tested?

Unit test

Closes #32725 from gengliangwang/fixFlakyLogAppender.

Authored-by: Gengliang Wang <gengliang@apache.org>
Signed-off-by: Gengliang Wang <gengliang@apache.org>
2021-06-02 10:05:29 +08:00
Dongjoon Hyun 35cfabcf5c [SPARK-35589][CORE] BlockManagerMasterEndpoint should not ignore index-only shuffle file during updating
### What changes were proposed in this pull request?

This PR aims to make `BlockManagerMasterEndpoint.updateBlockInfo` not to ignore index-only shuffle files.
In addition, this PR fixes `IndexShuffleBlockResolver.getMigrationBlocks` to return data files first.

### Why are the changes needed?

When [SPARK-20629](a4ca355af8) introduced a worker decommission, index-only shuffle files are not considered properly.
- SPARK-33198 fixed `getMigrationBlocks` to handle index only shuffle files
- SPARK-35589 (this) aims to fix `updateBlockInfo` to handle index only shuffle files.

### Does this PR introduce _any_ user-facing change?

No. This is a bug fix.

### How was this patch tested?

Pass the CIs with the newly added test case.

Closes #32727 from dongjoon-hyun/SPARK-UPDATE-OUTPUT.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-06-01 14:23:24 -07:00
Kent Yao a127d91292 [SPARK-35402][WEBUI] Increase the max thread pool size of jetty server in HistoryServer UI
### What changes were proposed in this pull request?

For different UIs, e.g. History Server or Spark Live UI, maybe need different capabilities to handle HTTP requests. Usually, a History Server is for multi-users and needs more threads to increase concurrency, while  Live UI is per application, which needn't that large pool size.

In this PR, we increase the max pool size of the History Server's jetty backend

### Why are the changes needed?

increase the client concurrency of HistoryServer

### Does this PR introduce _any_ user-facing change?

no

### How was this patch tested?

new tests

Closes #32539 from yaooqinn/SPARK-35402.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Kent Yao <yao@apache.org>
2021-06-02 01:02:41 +08:00
lidiyag b7dd4b37e5 [SPARK-35516][WEBUI] Storage UI tab Storage Level tool tip correction
### What changes were proposed in this pull request?
Fixed tooltip for "Storage" tab in UI

### Why are the changes needed?
Tooltip correction was needed

### Does this PR introduce _any_ user-facing change?
Yes

### How was this patch tested?
Manually tested

Closes #32664 from lidiyag/storagewebui.

Authored-by: lidiyag <lidiya.nixon@huawei.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-06-01 19:00:13 +09:00
Yikun Jiang d773373074 [SPARK-35584][CORE][TESTS] Increase the timeout in FallbackStorageSuite
### What changes were proposed in this pull request?
```
- Upload multi stages *** FAILED ***
{{ The code passed to eventually never returned normally. Attempted 20 times over 10.011176743 seconds. Last failure message: fallbackStorage.exists(0, file) was false. (FallbackStorageSuite.scala:243)}}
```
The error like above was raised in aarch64 randomly and also in github action test[1][2].

[1] https://github.com/apache/spark/actions/runs/489319612
[2]https://github.com/apache/spark/actions/runs/479317320

### Why are the changes needed?
timeout is too short, need to increase to let test case complete.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
build/mvn test -Dtest=none -DwildcardSuites=org.apache.spark.storage.FallbackStorageSuite -pl :spark-core_2.12

Closes #32719 from Yikun/SPARK-35584.

Authored-by: Yikun Jiang <yikunkero@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-06-01 00:45:58 -07:00
yangjie01 09d039da56 [SPARK-35526][CORE][SQL][ML][MLLIB] Re-Cleanup procedure syntax is deprecated compilation warning in Scala 2.13
### What changes were proposed in this pull request?
After SPARK-29291 and SPARK-33352, there are still some compilation warnings about `procedure syntax is deprecated` as follows:

```
[WARNING] [Warn] /spark/core/src/main/scala/org/apache/spark/MapOutputTracker.scala:723: [deprecation   | origin= | version=2.13.0] procedure syntax is deprecated: instead, add `: Unit =` to explicitly declare `registerMergeResult`'s return type
[WARNING] [Warn] /spark/core/src/main/scala/org/apache/spark/MapOutputTracker.scala:748: [deprecation   | origin= | version=2.13.0] procedure syntax is deprecated: instead, add `: Unit =` to explicitly declare `unregisterMergeResult`'s return type
[WARNING] [Warn] /spark/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala:223: [deprecation   | origin= | version=2.13.0] procedure syntax is deprecated: instead, add `: Unit =` to explicitly declare `testSimpleSpillingForAllCodecs`'s return type
[WARNING] [Warn] /spark/mllib-local/src/test/scala/org/apache/spark/ml/linalg/BLASBenchmark.scala:53: [deprecation   | origin= | version=2.13.0] procedure syntax is deprecated: instead, add `: Unit =` to explicitly declare `runBLASBenchmark`'s return type
[WARNING] [Warn] /spark/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DataWritingCommand.scala:110: [deprecation   | origin= | version=2.13.0] procedure syntax is deprecated: instead, add `: Unit =` to explicitly declare `assertEmptyRootPath`'s return type
[WARNING] [Warn] /spark/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala:602: [deprecation   | origin= | version=2.13.0] procedure syntax is deprecated: instead, add `: Unit =` to explicitly declare `executeCTASWithNonEmptyLocation`'s return type
```

So the main change of this pr is cleanup these compilation warnings.

### Why are the changes needed?
Eliminate compilation warnings in Scala 2.13 and this change should be compatible with Scala 2.12

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #32669 from LuciferYang/re-clean-procedure-syntax.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-05-30 16:49:47 -07:00
Vinod KC e3c6907c99 [SPARK-35490][BUILD] Update json4s to 3.7.0-M11
### What changes were proposed in this pull request?
This PR aims to upgrade json4s from   3.7.0-M5  to 3.7.0-M11

Note: json4s version greater than 3.7.0-M11 is not binary compatible with Spark third party jars

### Why are the changes needed?
Multiple defect fixes and improvements  like

https://github.com/json4s/json4s/issues/750
https://github.com/json4s/json4s/issues/554
https://github.com/json4s/json4s/issues/715

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Ran with the existing UTs

Closes #32636 from vinodkc/br_build_upgrade_json4s.

Authored-by: Vinod KC <vinod.kc.in@gmail.com>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-05-26 11:10:14 +03:00
Chendi Xue 7258f69188 [SPARK-35396] Add AutoCloseable close to BlockManager and InMemoryRelation
This PR is proposing a add-on to support to manual close entries in MemoryStore and InMemoryRelation

### What changes were proposed in this pull request?
Currently:
    MemoryStore uses a LinkedHashMap[BlockId, MemoryEntry[_]] to store all OnHeap or OffHeap entries.
And when memoryStore.remove(blockId) is called, codes will simply remove one entry from LinkedHashMap and leverage Java GC to do release work.

This PR:
    We are proposing a add-on to manually close any object stored in MemoryStore and InMemoryRelation if this object is extended from AutoCloseable.

Veifiication:
    In our own use case, we implemented a user-defined off-heap-hashRelation for BHJ, and we verified that by adding this manual close, we can make sure our defined off-heap-hashRelation can be released when evict is called.
    Also, we implemented user-defined cachedBatch and will be release when InMemoryRelation.clearCache() is called by this PR

### Why are the changes needed?
This changes can help to clean some off-heap user-defined object may be cached in InMemoryRelation or MemoryStore

### Does this PR introduce _any_ user-facing change?
NO

### How was this patch tested?
WIP

Signed-off-by: Chendi Xue <chendi.xueintel.com>

Closes #32534 from xuechendi/support_manual_close_in_memorystore.

Authored-by: Chendi Xue <chendi.xue@intel.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-05-25 08:55:25 -05:00
Ankur Dave 58d4da1bdb [SPARK-35486][CORE] TaskMemoryManager: retry if other task takes memory freed by partial self-spill
### What changes were proposed in this pull request?

When a memory reservation triggers a self-spill, `ExecutionMemoryPool#releaseMemory()` will immediately notify waiting tasks that memory has been freed. If there are any waiting tasks with less than 1/2N of the memory pool, they may acquire the newly-freed memory before the current task has a chance to do so. This will cause the original memory reservation to fail. If the initial spill did not release all available memory, the reservation could have been satisfied by asking it to spill again.

This PR adds logic to TaskMemoryManager to detect this case and retry.

### Why are the changes needed?

This bug affects queries with a MemoryConsumer that can spill part of its memory, such as BytesToBytesMap. If the MemoryConsumer is using all available memory and there is a waiting task, then attempting to acquire more memory on the MemoryConsumer will trigger a partial self-spill. However, because the waiting task gets priority, the attempt to acquire memory will fail even if it could have been satisfied by another spill.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Added a test to MemoryManagerSuite that previously failed and now passes.

Closes #32625 from ankurdave/SPARK-35486.

Authored-by: Ankur Dave <ankurdave@gmail.com>
Signed-off-by: yi.wu <yi.wu@databricks.com>
2021-05-25 18:12:59 +08:00
Kent Yao 2e9936db93 [SPARK-35456][CORE] Print the invalid value in config validation error message
### What changes were proposed in this pull request?

Print the invalid value in config validation error message for `checkValue` just like `checkValues`

### Why are the changes needed?

Invalid configuration values may come in many ways, this PR can help different kinds of users or developers to identify what the config the error is related to

### Does this PR introduce _any_ user-facing change?

yes, but only error msg
### How was this patch tested?

yes, modified tests

Closes #32600 from yaooqinn/SPARK-35456.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-21 14:22:29 +09:00
yi.wu 00b63c8dc2 [SPARK-27991][CORE] Defer the fetch request on Netty OOM
### What changes were proposed in this pull request?

This PR proposes a workaround to address the Netty OOM issue (SPARK-24989, SPARK-27991):

Basically, `ShuffleBlockFetcherIterator` would catch the `OutOfDirectMemoryError` from Netty and then set a global flag for the shuffle module. Any pending fetch requests would be deferred if there're in-flight requests until the flag is unset. And the flag will be unset when there's a fetch request succeed.

Note that catching the Netty OOM rather than abort the application is feasible because Netty manage its own memory region (offheap by default) separately. So Netty OOM doesn't mean the memory shortage of Spark.

### Why are the changes needed?

The Netty OOM issue is a very corner case. It usually happens in the large-scale cluster, where a reduce task could fetch shuffle blocks from hundreds of nodes concurrently in a short time. Internally, we found a cluster that has created 260+ clients within 6s before throwing Netty OOM.

Although Spark has configurations, e.g., `spark.reducer.maxReqsInFlight` to tune the number of concurrent requests, it's usually not a easy decision for the user to set a reasonable value regarding the workloads, machine resources, etc. But with this fix, Spark would heal the Netty memory issue itself without any specific configurations.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Added unit tests.

Closes #32287 from Ngone51/SPARK-27991.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-20 04:26:56 +00:00
Yuzhou Sun a72d05c7e6 [SPARK-35106][CORE][SQL] Avoid failing rename caused by destination directory not exist
### What changes were proposed in this pull request?

1. In HadoopMapReduceCommitProtocol, create parent directory before renaming custom partition path staging files
2. In InMemoryCatalog and HiveExternalCatalog, create new partition directory before renaming old partition path
3. Check return value of FileSystem#rename, if false, throw exception to avoid silent data loss cause by rename failure
4. Change DebugFilesystem#rename behavior to make it match HDFS's behavior (return false without rename when dst parent directory not exist)

### Why are the changes needed?

Depends on FileSystem#rename implementation, when destination directory does not exist, file system may
1. return false without renaming file nor throwing exception (e.g. HDFS), or
2. create destination directory, rename files, and return true (e.g. LocalFileSystem)

In the first case above, renames in HadoopMapReduceCommitProtocol for custom partition path will fail silently if the destination partition path does not exist. Failed renames can happen when
1. dynamicPartitionOverwrite == true, the custom partition path directories are deleted by the job before the rename; or
2. the custom partition path directories do not exist before the job; or
3. something else is wrong when file system handle `rename`

The renames in MemoryCatalog and HiveExternalCatalog for partition renaming also have similar issue.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Modified DebugFilesystem#rename, and added new unit tests.

Without the fix in src code, five InsertSuite tests and one AlterTableRenamePartitionSuite test failed:
InsertSuite.SPARK-20236: dynamic partition overwrite with custom partition path (existing test with modified FS)
```
== Results ==
!== Correct Answer - 1 ==   == Spark Answer - 0 ==
struct<>                   struct<>
![2,1,1]
```

InsertSuite.SPARK-35106: insert overwrite with custom partition path
```
== Results ==
!== Correct Answer - 1 ==   == Spark Answer - 0 ==
struct<>                   struct<>
![2,1,1]
```

InsertSuite.SPARK-35106: dynamic partition overwrite with custom partition path
```
== Results ==
!== Correct Answer - 2 ==   == Spark Answer - 1 ==
!struct<>                   struct<i:int,part1:int,part2:int>
 [1,1,1]                    [1,1,1]
![1,1,2]
```

InsertSuite.SPARK-35106: Throw exception when rename custom partition paths returns false
```
Expected exception org.apache.spark.SparkException to be thrown, but no exception was thrown
```

InsertSuite.SPARK-35106: Throw exception when rename dynamic partition paths returns false
```
Expected exception org.apache.spark.SparkException to be thrown, but no exception was thrown
```

AlterTableRenamePartitionSuite.ALTER TABLE .. RENAME PARTITION V1: multi part partition (existing test with modified FS)
```
== Results ==
!== Correct Answer - 1 ==   == Spark Answer - 0 ==
 struct<>                   struct<>
![3,123,3]
```

Closes #32530 from YuzhouSun/SPARK-35106.

Authored-by: Yuzhou Sun <yuzhosun@amazon.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-19 15:46:27 +08:00
Erik Krogen 186477c60e [SPARK-35263][TEST] Refactor ShuffleBlockFetcherIteratorSuite to reduce duplicated code
### What changes were proposed in this pull request?
Introduce new shared methods to `ShuffleBlockFetcherIteratorSuite` to replace copy-pasted code. Use modern, Scala-like Mockito `Answer` syntax.

### Why are the changes needed?
`ShuffleFetcherBlockIteratorSuite` has tons of duplicate code, like 0494dc90af/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala (L172-L185) . It's challenging to tell what the interesting parts are vs. what is just being set to some default/unused value.

Similarly but not as bad, there are many calls like the following
```
verify(transfer, times(1)).fetchBlocks(any(), any(), any(), any(), any(), any())
when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any())).thenAnswer ...
```

These changes result in about 10% reduction in both lines and characters in the file:
```bash
# Before
> wc core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala
    1063    3950   43201 core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala

# After
> wc core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala
     928    3609   39053 core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala
```

It also helps readability, e.g.:
```
    val iterator = createShuffleBlockIteratorWithDefaults(
      transfer,
      blocksByAddress,
      maxBytesInFlight = 1000L
    )
```
Now I can clearly tell that `maxBytesInFlight` is the main parameter we're interested in here.

### Does this PR introduce _any_ user-facing change?
No, test only. There aren't even any behavior changes, just refactoring.

### How was this patch tested?
Unit tests pass.

Closes #32389 from xkrogen/xkrogen-spark-35263-refactor-shuffleblockfetcheriteratorsuite.

Authored-by: Erik Krogen <xkrogen@apache.org>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2021-05-18 22:37:47 -05:00
yi.wu 94bd480761 [SPARK-35206][TESTS][SQL] Extract common used get project path into a function in SparkFunctionSuite
### What changes were proposed in this pull request?

Add a common functions `getWorkspaceFilePath` (which prefixed with spark home) to `SparkFunctionSuite`, and applies these the function to where they're extracted from.

### Why are the changes needed?

Spark sql has test suites to read resources when running tests. The way of getting the path of resources is commonly used in different suites. We can extract them into a function to ease the code maintenance.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Pass existing tests.

Closes #32315 from Ngone51/extract-common-file-path.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-05-14 22:17:50 +08:00
Kent Yao 68239d1b55 [SPARK-35404][CORE] Name the timers in TaskSchedulerImpl
### What changes were proposed in this pull request?

make these threads easier to identify in thread dumps

### Why are the changes needed?

make these threads easier to identify in thread dumps

### Does this PR introduce _any_ user-facing change?

yes. Driver thread dumps will show the timers with pretty names

### How was this patch tested?

verified locally

Closes #32549 from yaooqinn/SPARK-35404.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2021-05-14 19:17:45 +09:00
Hyukjin Kwon f7af9ab8dc [SPARK-34764][UI][FOLLOW-UP] Fix indentation and missing arguments for JavaScript linter
### What changes were proposed in this pull request?

This PR is a followup of https://github.com/apache/spark/pull/32436 which broke JavaScript linter. There was a logical conflict - the linter was added after the last successful test run in that PR.

```
added 118 packages in 1.482s

/__w/spark/spark/core/src/main/resources/org/apache/spark/ui/static/executorspage.js
   34:41  error  'type' is defined but never used. Allowed unused args must match /^_ignored_.*/u  no-unused-vars
   34:47  error  'row' is defined but never used. Allowed unused args must match /^_ignored_.*/u   no-unused-vars
   35:1   error  Expected indentation of 2 spaces but found 4                                      indent
   36:1   error  Expected indentation of 4 spaces but found 7                                      indent
   37:1   error  Expected indentation of 2 spaces but found 4                                      indent
   38:1   error  Expected indentation of 4 spaces but found 7                                      indent
   39:1   error  Expected indentation of 2 spaces but found 4                                      indent
  556:1   error  Expected indentation of 14 spaces but found 16                                    indent
  557:1   error  Expected indentation of 14 spaces but found 16                                    indent
```

### Why are the changes needed?

To recover the build

### Does this PR introduce _any_ user-facing change?

No, dev-only.

### How was this patch tested?

Manually tested:

```bash
 ./dev/lint-js
lint-js checks passed.
```

Closes #32541 from HyukjinKwon/SPARK-34764-followup.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-05-14 12:45:13 +09:00
Holden Karau 160b3bee71 [SPARK-34764][CORE][K8S][UI] Propagate reason for exec loss to Web UI
### What changes were proposed in this pull request?

Adds the exec loss reason to the Spark web UI & in doing so also fix the Kube integration to pass exec loss reason into core.

UI change:

![image](https://user-images.githubusercontent.com/59893/117045762-b975ba80-acc4-11eb-9679-8edab3cfadc2.png)

### Why are the changes needed?

Debugging Spark jobs is *hard*, making it clearer why executors have exited could help.

### Does this PR introduce _any_ user-facing change?

Yes a new column on the executor page.

### How was this patch tested?

K8s unit test updated to validate exec loss reasons are passed through regardless of exec alive state, manual testing to validate the UI.

Closes #32436 from holdenk/SPARK-34764-propegate-reason-for-exec-loss.

Lead-authored-by: Holden Karau <hkarau@apple.com>
Co-authored-by: Holden Karau <holden@pigscanfly.ca>
Signed-off-by: Holden Karau <hkarau@apple.com>
2021-05-13 16:02:31 -07:00
shahid b3c916e5a5 [SPARK-35013][CORE] Don't allow to set spark.driver.cores=0
### What changes were proposed in this pull request?
Currently spark is not allowing to set spark.driver.memory, spark.executor.cores, spark.executor.memory to 0, but allowing driver cores to 0. This PR checks for driver core size as well. Thanks Oleg Lypkan for finding this.

### Why are the changes needed?
To make the configuration check consistent.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Manual testing

Closes #32504 from shahidki31/shahid/drivercore.

Lead-authored-by: shahid <shahidki31@gmail.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Co-authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-05-12 12:45:55 -07:00
Dongjoon Hyun 77b7fe19e1 [SPARK-35383][CORE] Improve s3a magic committer support by inferring missing configs
### What changes were proposed in this pull request?

This PR aims to improve S3A magic committer support by inferring all missing configs from a single minimum configuration, `spark.hadoop.fs.s3a.bucket.<bucket>.committer.magic.enabled=true`.

Given that AWS S3 provides a [strong read-after-write consistency](https://aws.amazon.com/blogs/aws/amazon-s3-update-strong-read-after-write-consistency/) since December 2020, we can ignore DynamoDB-related configurations. As a result, the minimum set of configuration are the following:

```
spark.hadoop.fs.s3a.committer.magic.enabled=true
spark.hadoop.fs.s3a.bucket.<bucket>.committer.magic.enabled=true
spark.hadoop.fs.s3a.committer.name=magic
spark.hadoop.mapreduce.outputcommitter.factory.scheme.s3a=org.apache.hadoop.fs.s3a.commit.S3ACommitterFactory
spark.sql.parquet.output.committer.class=org.apache.spark.internal.io.cloud.BindingParquetOutputCommitter
spark.sql.sources.commitProtocolClass=org.apache.spark.internal.io.cloud.PathOutputCommitProtocol
```

### Why are the changes needed?

To use S3A magic committer in Apache Spark, the users need to setup a set of configurations. And, if something is missed, it will end up with the error messages like the following.
```
Exception in thread "main" org.apache.hadoop.fs.s3a.commit.PathCommitException:
`s3a://my-spark-bucket`: Filesystem does not have support for 'magic' committer enabled in configuration option fs.s3a.committer.magic.enabled
	at org.apache.hadoop.fs.s3a.commit.CommitUtils.verifyIsMagicCommitFS(CommitUtils.java:74)
	at org.apache.hadoop.fs.s3a.commit.CommitUtils.getS3AFileSystem(CommitUtils.java:109)
```

### Does this PR introduce _any_ user-facing change?

Yes, after this improvement PR, all Spark users can use S3A committer by using a single configuration.
```
spark.hadoop.fs.s3a.bucket.<bucket>.committer.magic.enabled=true
```

This PR is going to inferring the missing configurations. So, there is no side-effect if the existing users who have all configurations already.

### How was this patch tested?

Pass the CIs with the newly added test cases.

Closes #32518 from dongjoon-hyun/SPARK-35383.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2021-05-12 11:53:28 -07:00
Kousuke Saruta 2b6640a169 [SPARK-35229][WEBUI] Limit the maximum number of items on the timeline view
### What changes were proposed in this pull request?

This PR proposes to introduces three new configurations to limit the maximum number of jobs/stages/executors on the timeline view.

### Why are the changes needed?

If the number of items on the timeline view grows +1000, rendering can be significantly slow.
https://issues.apache.org/jira/browse/SPARK-35229

The maximum number of tasks on the timeline is already limited by `spark.ui.timeline.tasks.maximum` so l proposed to mitigate this issue with the same manner.

### Does this PR introduce _any_ user-facing change?

Yes. the maximum number of items shown on the timeline view is limited.
I proposed the default value 500 for jobs and stages, and 250 for executors.
A executor has at most 2 items (added and removed) 250 is chosen.

### How was this patch tested?

I manually confirm this change works with the following procedures.
```
# launch a cluster
$ bin/spark-shell --conf spark.ui.retainedDeadExecutors=300 --master "local-cluster[4, 1, 1024]"

// Confirm the maximum number of jobs
(1 to 1000).foreach { _ => sc.parallelize(List(1)).collect }

// Confirm the maximum number of stages
var df = sc.parallelize(1 to 2)
(1 to 1000).foreach { i =>  df = df.repartition(i % 5 + 1) }
df.collect

// Confirm the maximum number of executors
(1 to 300).foreach { _ => try sc.parallelize(List(1)).foreach { _ => System.exit(0) } catch { case e => }}
```

Screenshots here.
![jobs_limited](https://user-images.githubusercontent.com/4736016/116386937-3e8c4a00-a855-11eb-8f4c-151cf7ddd3b8.png)
![stages_limited](https://user-images.githubusercontent.com/4736016/116386990-49df7580-a855-11eb-9f71-8e129e3336ab.png)
![executors_limited](https://user-images.githubusercontent.com/4736016/116387009-4f3cc000-a855-11eb-8697-a2eb4c9c99e6.png)

Closes #32381 from sarutak/mitigate-timeline-issue.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-05-11 20:53:11 +08:00
RoryQi 6f0ef93f9a [SPARK-35297][CORE][DOC][MINOR] Modify the comment about the executor
### What changes were proposed in this pull request?
Now Spark Executor already can be used in Kubernetes scheduler. So we should modify the annotation in the Executor.scala.

### Why are the changes needed?
only comment

### Does this PR introduce _any_ user-facing change?
no

### How was this patch tested?
no

Closes #32426 from jerqi/master.

Authored-by: RoryQi <1242949407@qq.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-05-08 00:03:02 +09:00
Kousuke Saruta 2634dbac35 [SPARK-35175][BUILD] Add linter for JavaScript source files
### What changes were proposed in this pull request?

This PR proposes to add linter for JavaScript source files.
[ESLint](https://eslint.org/) seems to be a popular linter for JavaScript so I choose it.

### Why are the changes needed?

Linter enables us to check style and keeps code clean.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Manually run `dev/lint-js` (Node.js and npm are required).

In this PR, mainly indentation style is also fixed an linter passes.

Closes #32274 from sarutak/introduce-eslint.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-05-07 21:55:08 +09:00
Chao Sun 4fe4b65d9e [SPARK-35315][TESTS] Keep benchmark result consistent between spark-submit and SBT
### What changes were proposed in this pull request?

Set `IS_TESTING` to true in `BenchmarkBase`, before running benchmarks.

### Why are the changes needed?

Currently benchmark can be done via 2 ways: `spark-submit`, or SBT command. However in the former Spark will miss some properties such as `IS_TESTING`, which is necessary to turn on/off certain behavior like codegen (`spark.sql.codegen.factoryMode`). Therefore, the result could differ between the two. In addition, the benchmark GitHub workflow is using the spark-submit approach.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

N/A

Closes #32440 from sunchao/SPARK-35315.

Authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-05-05 18:30:51 +08:00
byungsoo 9b387a1718 [SPARK-35308][TESTS] Fix bug in SPARK-35266 that creates benchmark files in invalid path with wrong name
### What changes were proposed in this pull request?
This PR fixes a bug in [SPARK-35266](https://issues.apache.org/jira/browse/SPARK-35266) that creates benchmark files in the invalid path with the wrong name.
e.g. For `BLASBenchmark`,
- AS-IS: Creates `benchmarksBLASBenchmark-results.txt` in `{SPARK_HOME}/mllib-local/`
- TO-BE: Creates `BLASBenchmark-results.txt` in `{SPARK_HOME}/mllib-local/benchmarks/`

### Why are the changes needed?
As you can see in the above example, new benchmark files cannot be created as intended due to this bug.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
After building Spark, manually tested with the following command:
```
SPARK_GENERATE_BENCHMARK_FILES=1 bin/spark-submit --class \
    org.apache.spark.benchmark.Benchmarks --jars \
    "`find . -name '*-SNAPSHOT-tests.jar' -o -name '*avro*-SNAPSHOT.jar' | paste -sd ',' -`" \
    "`find . -name 'spark-core*-SNAPSHOT-tests.jar'`" \
    "org.apache.spark.ml.linalg.BLASBenchmark"
```
It successfully generated the benchmark files as intended (`BLASBenchmark-results.txt` in `{SPARK_HOME}/mllib-local/benchmarks/`).

Closes #32432 from byungsoo-oh/SPARK-35308.

Lead-authored-by: byungsoo <byungsoo@byungsoo-pc.tn.corp.samsungelectronics.net>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-05-04 19:40:57 +09:00
byungsoo be6ecb6d19 [SPARK-35266][TESTS] Fix error in BenchmarkBase.scala that occurs when creating benchmark files in non-existent directory
### What changes were proposed in this pull request?
This PR fixes an error in `BenchmarkBase.scala` that occurs when creating a benchmark file in a non-existent directory.

### Why are the changes needed?
When submitting a benchmark job using `org.apache.spark.benchmark.Benchmarks` class with `SPARK_GENERATE_BENCHMARK_FILES=1` option, an exception is raised if the directory where the benchmark file will be generated does not exist.
For more information, please refer to [SPARK-35266](https://issues.apache.org/jira/browse/SPARK-35266).

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
After building Spark, manually tested with the following command:
```
SPARK_GENERATE_BENCHMARK_FILES=1 bin/spark-submit --class \
    org.apache.spark.benchmark.Benchmarks --jars \
    "`find . -name '*-SNAPSHOT-tests.jar' -o -name '*avro*-SNAPSHOT.jar' | paste -sd ',' -`" \
    "`find . -name 'spark-core*-SNAPSHOT-tests.jar'`" \
    "org.apache.spark.ml.linalg.BLASBenchmark"
```
It successfully generated the benchmark result files.

**Why it is sufficient:**
As illustrated in the comments in `Benchmarks.scala`, the command below runs all benchmarks and generates the results:
```
SPARK_GENERATE_BENCHMARK_FILES=1 bin/spark-submit --class \
    org.apache.spark.benchmark.Benchmarks --jars \
    "`find . -name '*-SNAPSHOT-tests.jar' -o -name '*avro*-SNAPSHOT.jar' | paste -sd ',' -`" \
    "`find . -name 'spark-core*-SNAPSHOT-tests.jar'`" \
    "*"
```
Of all the benchmarks (55 benchmarks in total), only `BLASBenchmark` fails due to the proposed issue for the current code in the master branch. Thus, it is currently sufficient to test `BLASBenchmark` to validate this change.

Closes #32394 from byungsoo-oh/SPARK-35266.

Authored-by: byungsoo <byungsoo@byungsoo-pc.tn.corp.samsungelectronics.net>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-05-03 18:06:06 +09:00
attilapiros 738cf7f8ff [SPARK-35009][CORE] Avoid creating multiple python worker monitor threads for the same worker and same task context
### What changes were proposed in this pull request?

With this PR Spark avoids creating multiple monitor threads for the same worker and same task context.

### Why are the changes needed?

Without this change unnecessary threads will be created. It even can cause job failure for example when a coalesce (without shuffle) from high partition number goes to very low one. This exception is exactly comes for such a run:

```
py4j.protocol.Py4JJavaError: An error occurred while calling z:org.apache.spark.api.python.PythonRDD.collectAndServe.
: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in stage 0.0 (TID 0) (192.168.1.210 executor driver): java.lang.OutOfMemoryError: unable to create new native thread
	at java.lang.Thread.start0(Native Method)
	at java.lang.Thread.start(Thread.java:717)
	at org.apache.spark.api.python.BasePythonRunner.compute(PythonRunner.scala:166)
	at org.apache.spark.api.python.PythonRDD.compute(PythonRDD.scala:65)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
	at org.apache.spark.rdd.CoalescedRDD.$anonfun$compute$1(CoalescedRDD.scala:99)
	at scala.collection.Iterator$$anon$11.nextCur(Iterator.scala:484)
	at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:490)
	at scala.collection.Iterator.foreach(Iterator.scala:941)
	at scala.collection.Iterator.foreach$(Iterator.scala:941)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
	at scala.collection.generic.Growable.$plus$plus$eq(Growable.scala:62)
	at scala.collection.generic.Growable.$plus$plus$eq$(Growable.scala:53)
	at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:105)
	at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:49)
	at scala.collection.TraversableOnce.to(TraversableOnce.scala:315)
	at scala.collection.TraversableOnce.to$(TraversableOnce.scala:313)
	at scala.collection.AbstractIterator.to(Iterator.scala:1429)
	at scala.collection.TraversableOnce.toBuffer(TraversableOnce.scala:307)
	at scala.collection.TraversableOnce.toBuffer$(TraversableOnce.scala:307)
	at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1429)
	at scala.collection.TraversableOnce.toArray(TraversableOnce.scala:294)
	at scala.collection.TraversableOnce.toArray$(TraversableOnce.scala:288)
	at scala.collection.AbstractIterator.toArray(Iterator.scala:1429)
	at org.apache.spark.rdd.RDD.$anonfun$collect$2(RDD.scala:1030)
	at org.apache.spark.SparkContext.$anonfun$runJob$5(SparkContext.scala:2260)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
	at org.apache.spark.scheduler.Task.run(Task.scala:131)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:498)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1437)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:501)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)

Driver stacktrace:
	at org.apache.spark.scheduler.DAGScheduler.failJobAndIndependentStages(DAGScheduler.scala:2262)
	at org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2(DAGScheduler.scala:2211)
	at org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2$adapted(DAGScheduler.scala:2210)
	at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
	at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
	at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
	at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:2210)
	at org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1(DAGScheduler.scala:1083)
	at org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1$adapted(DAGScheduler.scala:1083)
	at scala.Option.foreach(Option.scala:407)
	at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:1083)
	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:2449)
	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2391)
	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2380)
	at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:49)
	at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:872)
	at org.apache.spark.SparkContext.runJob(SparkContext.scala:2220)
	at org.apache.spark.SparkContext.runJob(SparkContext.scala:2241)
	at org.apache.spark.SparkContext.runJob(SparkContext.scala:2260)
	at org.apache.spark.SparkContext.runJob(SparkContext.scala:2285)
	at org.apache.spark.rdd.RDD.$anonfun$collect$1(RDD.scala:1030)
	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:414)
	at org.apache.spark.rdd.RDD.collect(RDD.scala:1029)
	at org.apache.spark.api.python.PythonRDD$.collectAndServe(PythonRDD.scala:180)
	at org.apache.spark.api.python.PythonRDD.collectAndServe(PythonRDD.scala)
	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 py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)
	at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)
	at py4j.Gateway.invoke(Gateway.java:282)
	at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
	at py4j.commands.CallCommand.execute(CallCommand.java:79)
	at py4j.GatewayConnection.run(GatewayConnection.java:238)
	at java.lang.Thread.run(Thread.java:748)
Caused by: java.lang.OutOfMemoryError: unable to create new native thread
	at java.lang.Thread.start0(Native Method)
	at java.lang.Thread.start(Thread.java:717)
	at org.apache.spark.api.python.BasePythonRunner.compute(PythonRunner.scala:166)
	at org.apache.spark.api.python.PythonRDD.compute(PythonRDD.scala:65)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
	at org.apache.spark.rdd.CoalescedRDD.$anonfun$compute$1(CoalescedRDD.scala:99)
	at scala.collection.Iterator$$anon$11.nextCur(Iterator.scala:484)
	at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:490)
	at scala.collection.Iterator.foreach(Iterator.scala:941)
	at scala.collection.Iterator.foreach$(Iterator.scala:941)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
	at scala.collection.generic.Growable.$plus$plus$eq(Growable.scala:62)
	at scala.collection.generic.Growable.$plus$plus$eq$(Growable.scala:53)
	at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:105)
	at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:49)
	at scala.collection.TraversableOnce.to(TraversableOnce.scala:315)
	at scala.collection.TraversableOnce.to$(TraversableOnce.scala:313)
	at scala.collection.AbstractIterator.to(Iterator.scala:1429)
	at scala.collection.TraversableOnce.toBuffer(TraversableOnce.scala:307)
	at scala.collection.TraversableOnce.toBuffer$(TraversableOnce.scala:307)
	at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1429)
	at scala.collection.TraversableOnce.toArray(TraversableOnce.scala:294)
	at scala.collection.TraversableOnce.toArray$(TraversableOnce.scala:288)
	at scala.collection.AbstractIterator.toArray(Iterator.scala:1429)
	at org.apache.spark.rdd.RDD.$anonfun$collect$2(RDD.scala:1030)
	at org.apache.spark.SparkContext.$anonfun$runJob$5(SparkContext.scala:2260)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
	at org.apache.spark.scheduler.Task.run(Task.scala:131)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:498)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1437)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:501)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	... 1 more
```

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Manually I used a the following Python script used (`reproduce-SPARK-35009.py`):

```
import pyspark

conf = pyspark.SparkConf().setMaster("local[*]").setAppName("Test1")
sc = pyspark.SparkContext.getOrCreate(conf)

rows = 70000
data = list(range(rows))
rdd = sc.parallelize(data, rows)
assert rdd.getNumPartitions() == rows
rdd0 = rdd.filter(lambda x: False)
data = rdd0.coalesce(1).collect()
assert data == []
```

Spark submit:
```
$ ./bin/spark-submit reproduce-SPARK-35009.py
```

#### With this change

Checking the number of monitor threads with jcmd:
```
$ jcmd
85273 sun.tools.jcmd.JCmd
85227 org.apache.spark.deploy.SparkSubmit reproduce-SPARK-35009.py
41020 scala.tools.nsc.MainGenericRunner
$ jcmd 85227 Thread.print | grep -c "Monitor for python"
2
$ jcmd 85227 Thread.print | grep -c "Monitor for python"
2
...
$ jcmd 85227 Thread.print | grep -c "Monitor for python"
2
$ jcmd 85227 Thread.print | grep -c "Monitor for python"
2
$ jcmd 85227 Thread.print | grep -c "Monitor for python"
2
$ jcmd 85227 Thread.print | grep -c "Monitor for python"
2
```
<img width="859" alt="Screenshot 2021-04-14 at 16 06 51" src="https://user-images.githubusercontent.com/2017933/114731755-4969b980-9d42-11eb-8ec5-f60b217bdd96.png">

#### Without this change

```
...
$ jcmd 90052 Thread.print | grep -c "Monitor for python"                                                                                                      [INSERT]
5645
..
```

<img width="856" alt="Screenshot 2021-04-14 at 16 30 18" src="https://user-images.githubusercontent.com/2017933/114731724-4373d880-9d42-11eb-9f9b-d976bf2530e2.png">

Closes #32169 from attilapiros/SPARK-35009.

Authored-by: attilapiros <piros.attila.zsolt@gmail.com>
Signed-off-by: attilapiros <piros.attila.zsolt@gmail.com>
2021-04-29 18:38:31 +02:00
yi.wu 068b6c8be6 [SPARK-35234][CORE] Reserve the format of stage failureMessage
### What changes were proposed in this pull request?

`failureMessage` is already formatted, but `replaceAll("\n", " ")` destroyed the format. This PR fixed it.

### Why are the changes needed?

The formatted error message is easier to read and debug.

### Does this PR introduce _any_ user-facing change?

Yes, users see the clear error message in the application log.

(Note I changed a little bit to let the test throw exception intentionally. The test itself is good.)

Before:
![2141619490903_ pic_hd](https://user-images.githubusercontent.com/16397174/116177970-5a092f00-a747-11eb-9a0f-017391e80c8b.jpg)

After:

![2151619490955_ pic_hd](https://user-images.githubusercontent.com/16397174/116177981-5ecde300-a747-11eb-90ef-fd16e906beeb.jpg)

### How was this patch tested?

Manually tested.

Closes #32356 from Ngone51/format-stage-error-message.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: attilapiros <piros.attila.zsolt@gmail.com>
2021-04-29 16:33:36 +02:00
yangjie01 74b93261af [SPARK-35135][CORE] Turn the WritablePartitionedIterator from a trait into a default implementation class
### What changes were proposed in this pull request?
`WritablePartitionedIterator` define in `WritablePartitionedPairCollection.scala` and there are two implementation of these trait,  but the code for these two implementations is duplicate.

The main change of this pr is turn the `WritablePartitionedIterator` from a trait into a default implementation class because there is only one implementation now.

### Why are the changes needed?
Cleanup duplicate code.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #32232 from LuciferYang/writable-partitioned-iterator.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: yi.wu <yi.wu@databricks.com>
2021-04-29 11:46:24 +08:00
Bo Zhang f738fe07b6 [SPARK-35227][BUILD] Update the resolver for spark-packages in SparkSubmit
### What changes were proposed in this pull request?
This change is to use repos.spark-packages.org instead of Bintray as the repository service for spark-packages.

### Why are the changes needed?
The change is needed because Bintray will no longer be available from May 1st.

### Does this PR introduce _any_ user-facing change?
This should be transparent for users who use SparkSubmit.

### How was this patch tested?
Tested running spark-shell with --packages manually.

Closes #32346 from bozhang2820/replace-bintray.

Authored-by: Bo Zhang <bo.zhang@databricks.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2021-04-27 10:59:30 +09:00
Venkata krishnan Sowrirajan 38ef4771d4 [SPARK-32921][SHUFFLE] MapOutputTracker extensions to support push-based shuffle
### What changes were proposed in this pull request?
This is one of the patches for SPIP SPARK-30602 for push-based shuffle.
Summary of changes:

- Introduce `MergeStatus` which tracks the partition level metadata for a merged shuffle partition in the Spark driver
- Unify `MergeStatus` and `MapStatus` under a single trait to allow code reusing inside `MapOutputTracker`
- Extend `MapOutputTracker` to support registering / unregistering `MergeStatus`, calculate preferred locations for a shuffle taking into consideration of merged shuffle partitions, and serving reducer requests for block fetching locations with merged shuffle partitions.

The added APIs in `MapOutputTracker` will be used by `DAGScheduler` in SPARK-32920 and by `ShuffleBlockFetcherIterator` in SPARK-32922

### Why are the changes needed?
Refer to SPARK-30602

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Added unit tests.

Lead-authored-by: Min Shen mshenlinkedin.com
Co-authored-by: Chandni Singh chsinghlinkedin.com
Co-authored-by: Venkata Sowrirajan vsowrirajanlinkedin.com

Closes #30480 from Victsm/SPARK-32921.

Lead-authored-by: Venkata krishnan Sowrirajan <vsowrirajan@linkedin.com>
Co-authored-by: Min Shen <mshen@linkedin.com>
Co-authored-by: Chandni Singh <singh.chandni@gmail.com>
Co-authored-by: Chandni Singh <chsingh@linkedin.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2021-04-26 00:17:26 -05:00
kyoty 2d6467d6d1 [SPARK-35087][UI] Some columns in table Aggregated Metrics by Executor of stage-detail page shows incorrectly.
### What changes were proposed in this pull request?

 columns like 'Shuffle Read Size / Records', 'Output Size/ Records' etc  in table ` Aggregated Metrics by Executor` of stage-detail page should be sorted as numerical-order instead of lexicographical-order.

### Why are the changes needed?
buf fix,the sorting style should be consistent between different columns.

The correspondence between the table and the index is shown below(it is defined in stagespage-template.html):
| index | column name                            |
| ----- | -------------------------------------- |
| 0     | Executor ID                            |
| 1     | Logs                                   |
| 2     | Address                                |
| 3     | Task Time                              |
| 4     | Total Tasks                            |
| 5     | Failed Tasks                           |
| 6     | Killed Tasks                           |
| 7     | Succeeded Tasks                        |
| 8     | Excluded                               |
| 9     | Input Size / Records                   |
| 10    | Output Size / Records                  |
| 11    | Shuffle Read Size / Records            |
| 12    | Shuffle Write Size / Records           |
| 13    | Spill (Memory)                         |
| 14    | Spill (Disk)                           |
| 15    | Peak JVM Memory OnHeap / OffHeap       |
| 16    | Peak Execution Memory OnHeap / OffHeap |
| 17    | Peak Storage Memory OnHeap / OffHeap   |
| 18    | Peak Pool Memory Direct / Mapped       |

I constructed some data to simulate the sorting results of the index columns from 9 to 18.
As shown below,it can be seen that the sorting results of columns 9-12 are wrong:

![simulate-result](https://user-images.githubusercontent.com/52202080/115120775-c9fa1580-9fe1-11eb-8514-71f29db3a5eb.png)

The reason is that the real data corresponding to columns 9-12 (note that it is not the data displayed on the page) are **all strings similar to`94685/131`(bytes/records),while the real data corresponding to columns 13-18 are all numbers,**
so the sorting corresponding to columns 13-18 loos well, but the results of columns 9-12 are incorrect because the strings are sorted according to lexicographical order.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Only JS was modified, and the manual test result works well.

**before modified:**
![looks-illegal](https://user-images.githubusercontent.com/52202080/115120812-06c60c80-9fe2-11eb-9ada-fa520fe43c4e.png)

**after modified:**
![sort-result-corrent](https://user-images.githubusercontent.com/52202080/114865187-7c847980-9e24-11eb-9fbc-39ee224726d6.png)

Closes #32190 from kyoty/aggregated-metrics-by-executor-sorted-incorrectly.

Authored-by: kyoty <echohlne@gmail.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-04-26 12:13:22 +09:00
weixiuli bcac733bf1 [SPARK-35200][CORE] Avoid to recompute the pending speculative tasks in the ExecutorAllocationManager and remove some unnecessary code
### What changes were proposed in this pull request?
Avoid to recompute the pending speculative tasks in the ExecutorAllocationManager, and remove some unnecessary code.

### Why are the changes needed?

The number of the pending speculative tasks is recomputed in the ExecutorAllocationManager to calculate the maximum number of executors required.  While , it only needs to be computed once to improve performance.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Existing tests.

Closes #32306 from weixiuli/SPARK-35200.

Authored-by: weixiuli <weixiuli@jd.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-04-24 14:32:51 -07:00
kyoty 7242d7f774 [SPARK-35127][UI] When we switch between different stage-detail pages, the entry item in the newly-opened page may be blank
### What changes were proposed in this pull request?

To make sure that pageSize shoud not be shared between different stage pages.
The screenshots of the problem are placed in the attachment of [JIRA](https://issues.apache.org/jira/browse/SPARK-35127)

### Why are the changes needed?
fix the bug.

according to reference:`https://datatables.net/reference/option/lengthMenu`
`-1` represents display all rows, but now we use `totalTasksToShow`, it will cause the select item show as empty when we swich between different stage-detail pages.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
manual test, it is a small io problem, and the modification does not affect the function, but just an adjustment of js configuration

the gif below shows how the problem can be reproduced:
![reproduce](https://user-images.githubusercontent.com/52202080/115204351-f7060f80-a12a-11eb-8900-a009ad0c8870.gif)

![微信截图_20210419162849](https://user-images.githubusercontent.com/52202080/115205675-629cac80-a12c-11eb-9cb8-1939c7450e99.png)

the gif below shows the result after modified:

![after_modified](https://user-images.githubusercontent.com/52202080/115204886-91fee980-a12b-11eb-9ccb-d5900a99095d.gif)

Closes #32223 from kyoty/stages-task-empty-pagesize.

Authored-by: kyoty <echohlne@gmail.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-04-22 21:00:04 +09:00
skotlov b17a0e6931 [SPARK-34674][CORE][K8S] Close SparkContext after the Main method has finished
### What changes were proposed in this pull request?
Close SparkContext after the Main method has finished, to allow SparkApplication on K8S to complete.
This is fixed version of [merged and reverted PR](https://github.com/apache/spark/pull/32081).

### Why are the changes needed?
if I don't call the method sparkContext.stop() explicitly, then a Spark driver process doesn't terminate even after its Main method has been completed. This behaviour is different from spark on yarn, where the manual sparkContext stopping is not required. It looks like, the problem is in using non-daemon threads, which prevent the driver jvm process from terminating.
So I have inserted code that closes sparkContext automatically.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Manually on the production AWS EKS environment in my company.

Closes #32283 from kotlovs/close-spark-context-on-exit-2.

Authored-by: skotlov <skotlov@joom.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-04-21 22:54:16 -07:00
Shardul Mahadik 83f753e4e1 [SPARK-34472][YARN] Ship ivySettings file to driver in cluster mode
### What changes were proposed in this pull request?

In YARN, ship the `spark.jars.ivySettings` file to the driver when using `cluster` deploy mode so that `addJar` is able to find it in order to resolve ivy paths.

### Why are the changes needed?

SPARK-33084 introduced support for Ivy paths in `sc.addJar` or Spark SQL `ADD JAR`. If we use a custom ivySettings file using `spark.jars.ivySettings`, it is loaded at b26e7b510b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala (L1280). However, this file is only accessible on the client machine. In YARN cluster mode, this file is not available on the driver and so `addJar` fails to find it.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Added unit tests to verify that the `ivySettings` file is localized by the YARN client and that a YARN cluster mode application is able to find to load the `ivySettings` file.

Closes #31591 from shardulm94/SPARK-34472.

Authored-by: Shardul Mahadik <smahadik@linkedin.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2021-04-20 13:35:57 -05:00
SaurabhChawla 1e64b4fa27 [SPARK-34877][CORE][YARN] Add the code change for adding the Spark AM log link in spark UI
### What changes were proposed in this pull request?
On Running Spark job with yarn and deployment mode as client, Spark Driver and Spark Application master launch in two separate containers. In various scenarios there is need to see Spark Application master logs to see the resource allocation, Decommissioning status and other information shared between yarn RM and Spark Application master.

In Cluster mode Spark driver and Spark AM is on same container, So Log link of the driver already there to see the logs in Spark UI

This PR is for adding the spark AM log link for spark job running in the client mode for yarn. Instead of searching the container id and then find the logs. We can directly check in the Spark UI

This change is only for showing the AM log links in the Client mode when resource manager is yarn.

### Why are the changes needed?
Till now the only way to check this by finding the container id of the AM and check the logs either using Yarn utility or Yarn RM Application History server.

This PR is for adding the spark AM log link for spark job running in the client mode for yarn. Instead of searching the container id and then find the logs. We can directly check in the Spark UI

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Added the unit test also checked the Spark UI
**In Yarn Client mode**
Before Change

![image](https://user-images.githubusercontent.com/34540906/112644861-e1733200-8e6b-11eb-939b-c76ca9902a4e.png)

After the Change - The AM info is there

![image](https://user-images.githubusercontent.com/34540906/115264198-b7075280-a153-11eb-98f3-2aed66ffad2a.png)

AM Log

![image](https://user-images.githubusercontent.com/34540906/112645680-c0f7a780-8e6c-11eb-8b82-4ccc0aee927b.png)

**In Yarn Cluster Mode**  - The AM log link will not be there

![image](https://user-images.githubusercontent.com/34540906/112649512-86900980-8e70-11eb-9b37-69d5c4b53ffa.png)

Closes #31974 from SaurabhChawla100/SPARK-34877.

Authored-by: SaurabhChawla <s.saurabhtim@gmail.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2021-04-20 08:56:07 -05:00
Yingyi Bu f4926d1c8b [SPARK-35052][SQL] Use static bits for AttributeReference and Literal
### What changes were proposed in this pull request?

- Share a static ImmutableBitSet for `treePatternBits` in all object instances of AttributeReference.
- Share three static ImmutableBitSets for  `treePatternBits` in three kinds of Literals.
- Add an ImmutableBitSet as a subclass of BitSet.

### Why are the changes needed?

Reduce the additional memory usage caused by `treePatternBits`.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Existing tests.

Closes #32157 from sigmod/leaf.

Authored-by: Yingyi Bu <yingyi.bu@databricks.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-04-20 13:13:16 +08:00
Adam Binford e55ff83d77 [SPARK-35117][UI] Change progress bar back to highlight ratio of tasks in progress
### What changes were proposed in this pull request?
Small UI update to add highlighting the number of tasks in progress in a stage/job instead of highlighting the whole in progress stage/job. This was the behavior pre Spark 3.1 and the bootstrap 4 upgrade.

### Why are the changes needed?

To add back in functionality lost between 3.0 and 3.1. This provides a great visual queue of how much of a stage/job is currently being run.

### Does this PR introduce _any_ user-facing change?

Small UI change.

Before:
![image](https://user-images.githubusercontent.com/3536454/115216189-3fddaa00-a0d2-11eb-88e0-e3be925c92f0.png)

After (and pre Spark 3.1):
![image](https://user-images.githubusercontent.com/3536454/115216216-48ce7b80-a0d2-11eb-9953-2adb3b377133.png)

### How was this patch tested?

Updated existing UT.

Closes #32214 from Kimahriman/progress-bar-started.

Authored-by: Adam Binford <adamq43@gmail.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-04-20 11:59:56 +09:00
Sander Goos d37d18dd7f [SPARK-35136] Remove initial null value of LiveStage.info
### What changes were proposed in this pull request?
To prevent potential NullPointerExceptions, this PR changes the `LiveStage` constructor to take `info` as a constructor parameter and adds a nullcheck in  `AppStatusListener.activeStages`.

### Why are the changes needed?
The `AppStatusListener.getOrCreateStage` would create a LiveStage object with the `info` field set to null and right after that set it to a specific StageInfo object. This can lead to a race condition when the `livestages` are read in between those calls. This could then lead to a null pointer exception in, for instance: `AppStatusListener.activeStages`.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Regular CI/CD tests

Closes #32233 from sander-goos/SPARK-35136-livestage.

Authored-by: Sander Goos <sander.goos@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-04-19 15:09:29 +00:00
kyoty 978cd0bf49 [SPARK-35092][UI] the auto-generated rdd's name in the storage tab should be truncated if it is too long
### What changes were proposed in this pull request?
the auto-generated rdd's name in the storage tab should be truncated  as a single line if it is too long.

### Why are the changes needed?
to make the ui shows more friendly.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
just a simple modifition in css, manual test works well like below:

before modified:
![the rdd title in storage page shows too long](https://user-images.githubusercontent.com/52202080/115009655-17da2500-9edf-11eb-86a7-088bed7ef8f7.png)

after modified:
Tht titile  needs just one line:

![storage标题过长修改后](https://user-images.githubusercontent.com/52202080/114872091-8c07c080-9e2c-11eb-81a8-0c097b1a77bf.png)

Closes #32191 from kyoty/storage-rdd-titile-display-improve.

Authored-by: kyoty <echohlne@gmail.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-04-19 10:46:51 +09:00
kyoty 94849afc54 [SPARK-34787][CORE] Option variable in Spark historyServer log should be displayed as actual value instead of Some(XX)
### What changes were proposed in this pull request?
Make the attemptId in the log of historyServer to be more easily to read.

### Why are the changes needed?
Option variable in Spark historyServer log should be displayed as actual value instead of Some(XX)

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
manual test

Closes #32189 from kyoty/history-server-print-option-variable.

Authored-by: kyoty <echohlne@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-04-17 15:24:11 -07:00
ulysses-you 345c380778 [SPARK-35083][CORE] Support remote scheduler pool files
### What changes were proposed in this pull request?

Use hadoop FileSystem instead of FileInputStream.

### Why are the changes needed?

Make `spark.scheduler.allocation.file` suport remote file. When using Spark as a server (e.g. SparkThriftServer), it's hard for user to specify a local path as the scheduler pool.

### Does this PR introduce _any_ user-facing change?

Yes, a minor feature.

### How was this patch tested?

Pass `core/src/test/scala/org/apache/spark/scheduler/PoolSuite.scala` and manul test
After add config `spark.scheduler.allocation.file=hdfs:///tmp/fairscheduler.xml`. We intrudoce the configed pool.
![pool1](https://user-images.githubusercontent.com/12025282/114810037-df065700-9ddd-11eb-8d7a-54b59a07ee7b.jpg)

Closes #32184 from ulysses-you/SPARK-35083.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-04-16 00:18:35 -07:00
yi.wu 2cb962b132 [MINOR][CORE] Correct the number of started fetch requests in log
### What changes were proposed in this pull request?

When counting the number of started fetch requests, we should exclude the deferred requests.

### Why are the changes needed?

Fix the wrong number in the log.

### Does this PR introduce _any_ user-facing change?

Yes, users see the correct number of started requests in logs.

### How was this patch tested?

Manually tested.

Closes #32180 from Ngone51/count-deferred-request.

Lead-authored-by: yi.wu <yi.wu@databricks.com>
Co-authored-by: wuyi <yi.wu@databricks.com>
Signed-off-by: attilapiros <piros.attila.zsolt@gmail.com>
2021-04-15 10:39:52 +02:00
Kousuke Saruta 767ea86ecf [SPARK-34225][CORE][FOLLOWUP] Replace Hadoop's Path with Utils.resolveURI to make the way to get URI simple
### What changes were proposed in this pull request?

This PR proposes to replace Hadoop's `Path` with `Utils.resolveURI` to make the way to get URI simple in `SparkContext`.

### Why are the changes needed?

Keep the code simple.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Existing tests.

Closes #32164 from sarutak/followup-SPARK-34225.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-04-14 22:24:29 -07:00
Angerszhuuuu 9b2e0d6191 [SPARK-35086][SQL][CORE] --verbose should be passed to Spark SQL CLI too
### What changes were proposed in this pull request?
In current code, if we run spark sql with
```
./bin/spark-sql --verbose
```
It won't be passed to end SparkSQLCliDriver, then the SessionState won't call `setIsVerbose`

In the CLI option, it shows
```
CLI options:
 -v,--verbose                     Verbose mode (echo executed SQL to the
                                  console)
```

It's not consistent. This pr fix this issue
### Why are the changes needed?
Fix bug

### Does this PR introduce _any_ user-facing change?
when user call `-v` when run spark sql, sql will be echoed to console.

### How was this patch tested?
Added UT

Closes #32163 from AngersZhuuuu/SPARK-35086.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Yuming Wang <yumwang@ebay.com>
2021-04-15 12:59:20 +08:00
“attilapiros” 8a3815f722 [SPARK-34789][TEST] Introduce Jetty based construct for integration tests where HTTP server is used
### What changes were proposed in this pull request?

Introducing a new test construct:
```
  withHttpServer() { baseURL =>
    ...
  }
```
Which starts and stops a Jetty server to serve files via HTTP.

Moreover this PR uses this new construct in the test `Run SparkRemoteFileTest using a remote data file`.

### Why are the changes needed?

Before this PR github URLs was used like "https://raw.githubusercontent.com/apache/spark/master/data/mllib/pagerank_data.txt".
This connects two Spark version in an unhealthy way like connecting the "master" branch which is moving part with the committed test code which is a non-moving (as it might be even released).
So this way a test running for an earlier version of Spark expects something (filename, content, path) from a the latter release and what is worse when the moving version is changed the earlier test will break.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Existing unit test.

Closes #31935 from attilapiros/SPARK-34789.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-04-14 21:22:52 -07:00
Angerszhuuuu ee7d838aaf [SPARK-35049][CORE] Remove unused MapOutputTracker in BlockStoreShuffleReader
### What changes were proposed in this pull request?
Remove unused MapOutputTracker in BlockStoreShuffleReader

### Why are the changes needed?
Remove unused MapOutputTracker in BlockStoreShuffleReader

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Not need

Closes #32148 from AngersZhuuuu/SPARK-35049.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2021-04-13 13:30:27 -05:00
yangjie01 aae4ab278b [SPARK-35029][CORE] Extract tryOrFetchFailedException method to eliminate duplicate code in BufferReleasingInputStream
### What changes were proposed in this pull request?
The main change of this pr is extract a `private` method named `tryOrFetchFailedException` to eliminate duplicate code in `BufferReleasingInputStream`.

The patterns of duplicate code as follows:

```
try {
   block
 } catch {
    case e: IOException if detectCorruption =>
        IOUtils.closeQuietly(this)
        iterator.throwFetchFailedException(blockId, mapIndex, address, e)
 }
```

### Why are the changes needed?
Eliminate duplicate code.

### Does this PR introduce _any_ user-facing change?
 No

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #32130 from LuciferYang/SPARK-35029.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-04-13 12:59:45 +09:00
Yingyi Bu 3db8ec258c [SPARK-34916][SQL] Add condition lambda and rule id to the transform family for early stopping
### What changes were proposed in this pull request?

This PR contains:
- TreeNode, QueryPlan, AnalysisHelper changes to allow the transform function family to stop earlier without traversing the entire tree;
- Example changes in a few rules to support such pruning, e.g., ReorderJoin and OptimizeIn.

Here is a [design doc](https://docs.google.com/document/d/1SEUhkbo8X-0cYAJFYFDQhxUnKJBz4lLn3u4xR2qfWqk) that elaborates the ideas and benchmark numbers.

### Why are the changes needed?

It's a framework-level change for reducing the query compilation time.
In particular, if we update existing rules and transform call sites as per the examples in this PR, the analysis time and query optimization time can be reduced as described in this [doc](https://docs.google.com/document/d/1SEUhkbo8X-0cYAJFYFDQhxUnKJBz4lLn3u4xR2qfWqk) .

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

It is tested by existing tests.

Closes #32060 from sigmod/bits.

Authored-by: Yingyi Bu <yingyi.bu@databricks.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-04-12 11:21:16 +08:00
yangjie01 c06758834e [SPARK-35004][TEST] Fix Incorrect assertion of master/worker web ui available behind front-end reverseProxy in MasterSuite
### What changes were proposed in this pull request?
Line 425 in `MasterSuite` is considered as unused expression by Intellij IDE,

bfba7fadd2/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala (L421-L426)

If we merge lines 424 and 425 into one as:

```
System.getProperty("spark.ui.proxyBase") should startWith (s"$reverseProxyUrl/proxy/worker-")
```

this assertion will fail:

```
- master/worker web ui available behind front-end reverseProxy *** FAILED ***
  The code passed to eventually never returned normally. Attempted 45 times over 5.091914027 seconds. Last failure message: "http://proxyhost:8080/path/to/spark" did not start with substring "http://proxyhost:8080/path/to/spark/proxy/worker-". (MasterSuite.scala:405)
```

`System.getProperty("spark.ui.proxyBase")` should be `reverseProxyUrl` because `Master#onStart` and `Worker#handleRegisterResponse` have not changed it.

So the main purpose of this pr is to fix the condition of this assertion.

### Why are the changes needed?
Bug fix.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?

- Pass the Jenkins or GitHub Action

- Manual test:

1. merge lines 424 and 425 in `MasterSuite` into one to eliminate the unused expression:

```
System.getProperty("spark.ui.proxyBase") should startWith (s"$reverseProxyUrl/proxy/worker-")
```

2. execute `mvn clean test -pl core -Dtest=none -DwildcardSuites=org.apache.spark.deploy.master.MasterSuite`

**Before**

```
- master/worker web ui available behind front-end reverseProxy *** FAILED ***
  The code passed to eventually never returned normally. Attempted 45 times over 5.091914027 seconds. Last failure message: "http://proxyhost:8080/path/to/spark" did not start with substring "http://proxyhost:8080/path/to/spark/proxy/worker-". (MasterSuite.scala:405)

Run completed in 1 minute, 14 seconds.
Total number of tests run: 32
Suites: completed 2, aborted 0
Tests: succeeded 31, failed 1, canceled 0, ignored 0, pending 0
*** 1 TEST FAILED ***

```

**After**

```
Run completed in 1 minute, 11 seconds.
Total number of tests run: 32
Suites: completed 2, aborted 0
Tests: succeeded 32, failed 0, canceled 0, ignored 0, pending 0
All tests passed.
```

Closes #32105 from LuciferYang/SPARK-35004.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-04-09 21:18:49 +08:00
Dongjoon Hyun ed3f103ee8 Revert "[SPARK-34674][CORE][K8S] Close SparkContext after the Main method has finished"
This reverts commit ab97db75b2.
2021-04-08 21:50:14 -07:00
skotlov ab97db75b2 [SPARK-34674][CORE][K8S] Close SparkContext after the Main method has finished
### What changes were proposed in this pull request?
Close SparkContext after the Main method has finished, to allow SparkApplication on K8S to complete

### Why are the changes needed?
if I don't call the method sparkContext.stop() explicitly, then a Spark driver process doesn't terminate even after its Main method has been completed. This behaviour is different from spark on yarn, where the manual sparkContext stopping is not required. It looks like, the problem is in using non-daemon threads, which prevent the driver jvm process from terminating.
So I have inserted code that closes sparkContext automatically.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Manually on the production AWS EKS environment in my company.

Closes #32081 from kotlovs/close-spark-context-on-exit.

Authored-by: skotlov <skotlov@joom.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-04-08 16:51:38 -07:00
Sumeet Gajjar a9ca1978ae [SPARK-34949][CORE] Prevent BlockManager reregister when Executor is shutting down
### What changes were proposed in this pull request?

This PR prevents reregistering BlockManager when a Executor is shutting down. It is achieved by checking  `executorShutdown` before calling `env.blockManager.reregister()`.

### Why are the changes needed?

This change is required since Spark reports executors as active, even they are removed.
I was testing Dynamic Allocation on K8s with about 300 executors. While doing so, when the executors were torn down due to `spark.dynamicAllocation.executorIdleTimeout`, I noticed all the executor pods being removed from K8s, however, under the "Executors" tab in SparkUI, I could see some executors listed as alive.  [spark.sparkContext.statusTracker.getExecutorInfos.length](65da9287bc/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala (L105)) also returned a value greater than 1.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Added a new test.

## Logs
Following are the logs of the executor(Id:303) which re-registers `BlockManager`
```
21/04/02 21:33:28 INFO CoarseGrainedExecutorBackend: Got assigned task 1076
21/04/02 21:33:28 INFO Executor: Running task 4.0 in stage 3.0 (TID 1076)
21/04/02 21:33:28 INFO MapOutputTrackerWorker: Updating epoch to 302 and clearing cache
21/04/02 21:33:28 INFO TorrentBroadcast: Started reading broadcast variable 3
21/04/02 21:33:28 INFO TransportClientFactory: Successfully created connection to /100.100.195.227:33703 after 76 ms (62 ms spent in bootstraps)
21/04/02 21:33:28 INFO MemoryStore: Block broadcast_3_piece0 stored as bytes in memory (estimated size 2.4 KB, free 168.0 MB)
21/04/02 21:33:28 INFO TorrentBroadcast: Reading broadcast variable 3 took 168 ms
21/04/02 21:33:28 INFO MemoryStore: Block broadcast_3 stored as values in memory (estimated size 3.9 KB, free 168.0 MB)
21/04/02 21:33:29 INFO MapOutputTrackerWorker: Don't have map outputs for shuffle 1, fetching them
21/04/02 21:33:29 INFO MapOutputTrackerWorker: Doing the fetch; tracker endpoint = NettyRpcEndpointRef(spark://MapOutputTrackerda-lite-test-4-7a57e478947d206d-driver-svc.dex-app-n5ttnbmg.svc:7078)
21/04/02 21:33:29 INFO MapOutputTrackerWorker: Got the output locations
21/04/02 21:33:29 INFO ShuffleBlockFetcherIterator: Getting 2 non-empty blocks including 1 local blocks and 1 remote blocks
21/04/02 21:33:30 INFO TransportClientFactory: Successfully created connection to /100.100.80.103:40971 after 660 ms (528 ms spent in bootstraps)
21/04/02 21:33:30 INFO ShuffleBlockFetcherIterator: Started 1 remote fetches in 1042 ms
21/04/02 21:33:31 INFO Executor: Finished task 4.0 in stage 3.0 (TID 1076). 1276 bytes result sent to driver
.
.
.
21/04/02 21:34:16 INFO CoarseGrainedExecutorBackend: Driver commanded a shutdown
21/04/02 21:34:16 INFO Executor: Told to re-register on heartbeat
21/04/02 21:34:16 INFO BlockManager: BlockManager BlockManagerId(303, 100.100.122.34, 41265, None) re-registering with master
21/04/02 21:34:16 INFO BlockManagerMaster: Registering BlockManager BlockManagerId(303, 100.100.122.34, 41265, None)
21/04/02 21:34:16 INFO BlockManagerMaster: Registered BlockManager BlockManagerId(303, 100.100.122.34, 41265, None)
21/04/02 21:34:16 INFO BlockManager: Reporting 0 blocks to the master.
21/04/02 21:34:16 INFO MemoryStore: MemoryStore cleared
21/04/02 21:34:16 INFO BlockManager: BlockManager stopped
21/04/02 21:34:16 INFO FileDataSink: Closing sink with output file = /tmp/safari-events/.des_analysis/safari-events/hdp_spark_monitoring_random-container-037caf27-6c77-433f-820f-03cd9c7d9b6e-spark-8a492407d60b401bbf4309a14ea02ca2_events.tsv
21/04/02 21:34:16 INFO HonestProfilerBasedThreadSnapshotProvider: Stopping agent
21/04/02 21:34:16 INFO HonestProfilerHandler: Stopping honest profiler agent
21/04/02 21:34:17 INFO ShutdownHookManager: Shutdown hook called
21/04/02 21:34:17 INFO ShutdownHookManager: Deleting directory /var/data/spark-d886588c-2a7e-491d-bbcb-4f58b3e31001/spark-4aa337a0-60c0-45da-9562-8c50eaff3cea

```

Closes #32043 from sumeetgajjar/SPARK-34949.

Authored-by: Sumeet Gajjar <sumeetgajjar93@gmail.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2021-04-05 17:32:43 -05:00
Harsh Panchal aff6c0febb [SPARK-34934] Fix race condition while adding/removing sources in MetricsSystem
### What changes were proposed in this pull request?

Synchronise access to `registerSource` and `removeSource` method since underlying `ArrayBuffer` is not thread safe.

### Why are the changes needed?

Unexpected behaviours are possible due to lack of thread safety, Like we got `ArrayIndexOutOfBoundsException` while adding new source.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Closes #32024 from BOOTMGR/SPARK-34934.

Lead-authored-by: Harsh Panchal <BOOTMGR@users.noreply.github.com>
Co-authored-by: BOOTMGR <panchal.harsh18@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-04-05 08:42:02 -05:00
Liang-Chi Hsieh 571acc87fe [SPARK-34939][CORE] Throw fetch failure exception when unable to deserialize broadcasted map statuses
### What changes were proposed in this pull request?

This patch catches `IOException`, which is possibly thrown due to unable to deserialize map statuses (e.g., broadcasted value is destroyed), when deserilizing map statuses. Once `IOException` is caught, `MetadataFetchFailedException` is thrown to let Spark handle it.

### Why are the changes needed?

One customer encountered application error. From the log, it is caused by accessing non-existing broadcasted value. The broadcasted value is map statuses. E.g.,

```
[info]   Cause: java.io.IOException: org.apache.spark.SparkException: Failed to get broadcast_0_piece0 of broadcast_0
[info]   at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1410)
[info]   at org.apache.spark.broadcast.TorrentBroadcast.readBroadcastBlock(TorrentBroadcast.scala:226)
[info]   at org.apache.spark.broadcast.TorrentBroadcast.getValue(TorrentBroadcast.scala:103)
[info]   at org.apache.spark.broadcast.Broadcast.value(Broadcast.scala:70)
[info]   at org.apache.spark.MapOutputTracker$.$anonfun$deserializeMapStatuses$3(MapOutputTracker.scala:967)
[info]   at org.apache.spark.internal.Logging.logInfo(Logging.scala:57)
[info]   at org.apache.spark.internal.Logging.logInfo$(Logging.scala:56)
[info]   at org.apache.spark.MapOutputTracker$.logInfo(MapOutputTracker.scala:887)
[info]   at org.apache.spark.MapOutputTracker$.deserializeMapStatuses(MapOutputTracker.scala:967)
```

There is a race-condition. After map statuses are broadcasted and the executors obtain serialized broadcasted map statuses. If any fetch failure happens after, Spark scheduler invalidates cached map statuses and destroy broadcasted value of the map statuses. Then any executor trying to deserialize serialized broadcasted map statuses and access broadcasted value, `IOException` will be thrown. Currently we don't catch it in `MapOutputTrackerWorker` and above exception will fail the application.

Normally we should throw a fetch failure exception for such case. Spark scheduler will handle this.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Unit test.

Closes #32033 from viirya/fix-broadcast-master.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-04-03 18:37:50 -07:00
HyukjinKwon ebf01ec3c1 [SPARK-34950][TESTS] Update benchmark results to the ones created by GitHub Actions machines
### What changes were proposed in this pull request?

https://github.com/apache/spark/pull/32015 added a way to run benchmarks much more easily in the same GitHub Actions build. This PR updates the benchmark results by using the way.

**NOTE** that looks like GitHub Actions use four types of CPU given my observations:

- Intel(R) Xeon(R) Platinum 8171M CPU  2.60GHz
- Intel(R) Xeon(R) CPU E5-2673 v4  2.30GHz
- Intel(R) Xeon(R) CPU E5-2673 v3  2.40GHz
- Intel(R) Xeon(R) Platinum 8272CL CPU  2.60GHz

Given my quick research, seems like they perform roughly similarly:

![Screen Shot 2021-04-03 at 9 31 23 PM](https://user-images.githubusercontent.com/6477701/113478478-f4b57b80-94c3-11eb-9047-f81ca8c59672.png)

I couldn't find enough information about Intel(R) Xeon(R) Platinum 8272CL CPU  2.60GHz but the performance seems roughly similar given the numbers.

So shouldn't be a big deal especially given that this way is much easier, encourages contributors to run more and guarantee the same number of cores and same memory with the same softwares.

### Why are the changes needed?

To have a base line of the benchmarks accordingly.

### Does this PR introduce _any_ user-facing change?

No, dev-only.

### How was this patch tested?

It was generated from:

- [Run benchmarks: * (JDK 11)](https://github.com/HyukjinKwon/spark/actions/runs/713575465)
- [Run benchmarks: * (JDK 8)](https://github.com/HyukjinKwon/spark/actions/runs/713154337)

Closes #32044 from HyukjinKwon/SPARK-34950.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Max Gekk <max.gekk@gmail.com>
2021-04-03 23:02:56 +03:00
HyukjinKwon 71effba5f2 [SPARK-34821][INFRA] Set up a workflow for developers to run benchmark in their fork
### What changes were proposed in this pull request?

This PR proposes to add a workflow that allows developers to run benchmarks and download the results files.  After this PR, developers can run benchmarks in GitHub Actions in their fork.

### Why are the changes needed?

1. Very easy to use.
2. We can use the (almost) same environment to run the benchmarks. Given my few experiments and observation, the CPU, cores, and memory are same.
3. Does not burden ASF's resource at GitHub Actions.

### Does this PR introduce _any_ user-facing change?

No, dev-only.

### How was this patch tested?

Manually tested in https://github.com/HyukjinKwon/spark/pull/31.

Entire benchmarks are being run as below:
- [Run benchmarks: * (JDK 11)](https://github.com/HyukjinKwon/spark/actions/runs/713575465)
- [Run benchmarks: * (JDK 8)](https://github.com/HyukjinKwon/spark/actions/runs/713154337)

### How do developers use it in their fork?

1. **Go to Actions in your fork, and click "Run benchmarks"**

    ![Screen Shot 2021-03-31 at 10 15 13 PM](https://user-images.githubusercontent.com/6477701/113150018-99d71680-926e-11eb-8647-4ecf062c55f2.png)

2. **Run the benchmarks with JDK 8 or 11 with benchmark classes to run. Glob pattern is supported just like `testOnly` in SBT**

    ![Screen Shot 2021-04-02 at 8 35 02 PM](https://user-images.githubusercontent.com/6477701/113412599-ab95f680-93f3-11eb-9a15-c6ed54587b9d.png)

3. **After finishing the jobs, the benchmark results are available on the top in the underlying workflow:**

    ![Screen Shot 2021-03-31 at 10 17 21 PM](https://user-images.githubusercontent.com/6477701/113150332-ede1fb00-926e-11eb-9c0e-97d195070508.png)

4. **After downloading it, unzip and untar at Spark git root directory:**

    ```bash
    cd .../spark
    mv ~/Downloads/benchmark-results-8.zip .
    unzip benchmark-results-8.zip
    tar -xvf benchmark-results-8.tar
    ```

5. **Check the results:**

    ```bash
    git status
    ```

    ```
    ...
        modified:   core/benchmarks/MapStatusesSerDeserBenchmark-results.txt
    ```

Closes #32015 from HyukjinKwon/SPARK-34821-pr.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-04-03 20:55:54 +09:00
Baohe Zhang f03c7c0e9d [SPARK-34779][CORE] ExecutorMetricsPoller should keep stage entry in stageTCMP until a heartbeat occurs
### What changes were proposed in this pull request?
Allow ExecutorMetricsPoller to keep stage entries in stageTCMP until a heartbeat occurs even if the entries have task count = 0.

### Why are the changes needed?
This is an improvement.

The current implementation of ExecutorMetricsPoller keeps a map, stageTCMP of (stageId, stageAttemptId) to (count of running tasks, executor metric peaks). The entry for the stage is removed on task completion if the task count decreases to 0. In the case of an executor with a single core, this leads to unnecessary removal and insertion of entries for a given stage.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
A new unit test is added.

Closes #31871 from baohe-zhang/SPARK-34779.

Authored-by: Baohe Zhang <baohe.zhang@verizonmedia.com>
Signed-off-by: “attilapiros” <piros.attila.zsolt@gmail.com>
2021-04-02 07:14:18 +02:00
Angerszhuuuu 2796812cea [SPARK-26399][WEBUI][CORE] Add new stage-level REST APIs and parameters
### What changes were proposed in this pull request?
Add more flexable parameters for stage end point
endpoint /application/{app-id}/stages.  It can be:

/application/{app-id}/stages?details=[true|false]&status=[ACTIVE|COMPLETE|FAILED|PENDING|SKIPPED]&withSummaries=[true|false]$quantiles=[comma separated quantiles string]&taskStatus=[RUNNING|SUCCESS|FAILED|PENDING]

where
```
query parameter details=true is to show the detailed task information within each stage.  The default value is details=false;
query parameter status can select those stages with the specified status.  When status parameter is not specified, a list of all stages are generated.  
query parameter withSummaries=true is to show both task summary information in percentile distribution and executor summary information in percentile distribution.  The default value is withSummaries=false.
query parameter quantiles support user defined quantiles, default quantiles is `0.0,0.25,0.5,0.75,1.0`
query parameter taskStatus is to show only those tasks with the specified status within their corresponding stages.  This parameter will be set when details=true (i.e. this parameter will be ignored when details=false).
```

### Why are the changes needed?
More flexable restful API

### Does this PR introduce _any_ user-facing change?

### How was this patch tested?
UT

Closes #31204 from AngersZhuuuu/SPARK-26399-NEW.

Lead-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-04-01 12:48:26 -05:00
HyukjinKwon 065cbf1c8b [SPARK-34907][TESTS] Add main class that detects and runs all benchmarks
### What changes were proposed in this pull request?

This PR proposes to add a script that detects and runs all benchmarks.

### Why are the changes needed?

To run the benchmarks easily. This is actually for SPARK-34821.

### Does this PR introduce _any_ user-facing change?

No, dev-only.

### How was this patch tested?

Manually tested with the command below after building Spark:

```bash
SPARK_GENERATE_BENCHMARK_FILES=1 bin/spark-submit --class \
     org.apache.spark.benchmark.Benchmarks --jars \
     "`find . -name "*3.2.0-SNAPSHOT-tests.jar" | paste -sd ',' -`" \
     ./core/target/scala-2.12/spark-core_2.12-3.2.0-SNAPSHOT-tests.jar
 ```

 This is ongoing work. I will double check with working on SPARK-34821 and updating the results.

Closes #32005 from HyukjinKwon/SPARK-34907.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-31 13:39:32 +09:00
Erik Krogen 9f065ff375 [SPARK-34828][YARN] Make shuffle service name configurable on client side and allow for classpath-based config override on server side
### What changes were proposed in this pull request?
Add a new config, `spark.shuffle.service.name`, which allows for Spark applications to look for a YARN shuffle service which is defined at a name other than the default `spark_shuffle`.

Add a new config, `spark.yarn.shuffle.service.metrics.namespace`, which allows for configuring the namespace used when emitting metrics from the shuffle service into the NodeManager's `metrics2` system.

Add a new mechanism by which to override shuffle service configurations independently of the configurations in the NodeManager. When a resource `spark-shuffle-site.xml` is present on the classpath of the shuffle service, the configs present within it will be used to override the configs coming from `yarn-site.xml` (via the NodeManager).

### Why are the changes needed?
There are two use cases which can benefit from these changes.

One use case is to run multiple instances of the shuffle service side-by-side in the same NodeManager. This can be helpful, for example, when running a YARN cluster with a mixed workload of applications running multiple Spark versions, since a given version of the shuffle service is not always compatible with other versions of Spark (e.g. see SPARK-27780). With this PR, it is possible to run two shuffle services like `spark_shuffle` and `spark_shuffle_3.2.0`, one of which is "legacy" and one of which is for new applications. This is possible because YARN versions since 2.9.0 support the ability to run shuffle services within an isolated classloader (see YARN-4577), meaning multiple Spark versions can coexist.

Besides this, the separation of shuffle service configs into `spark-shuffle-site.xml` can be useful for administrators who want to change and/or deploy Spark shuffle service configurations independently of the configurations for the NodeManager (e.g., perhaps they are owned by two different teams).

### Does this PR introduce _any_ user-facing change?
Yes. There are two new configurations related to the external shuffle service, and a new mechanism which can optionally be used to configure the shuffle service. `docs/running-on-yarn.md` has been updated to provide user instructions; please see this guide for more details.

### How was this patch tested?
In addition to the new unit tests added, I have deployed this to a live YARN cluster and successfully deployed two Spark shuffle services simultaneously, one running a modified version of Spark 2.3.0 (which supports some of the newer shuffle protocols) and one running Spark 3.1.1. Spark applications of both versions are able to communicate with their respective shuffle services without issue.

Closes #31936 from xkrogen/xkrogen-SPARK-34828-shufflecompat-config-from-classpath.

Authored-by: Erik Krogen <xkrogen@apache.org>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2021-03-30 10:09:00 -05:00
yangjie01 7158e7f986 [SPARK-34900][TEST] Make sure benchmarks can run using spark-submit cmd described in the guide
### What changes were proposed in this pull request?
Some `spark-submit`  commands used to run benchmarks in the user's guide is wrong, we can't use these commands to run benchmarks successful.

So the major changes of this pr is correct these wrong commands, for example, run a benchmark which inherits from `SqlBasedBenchmark`, we must specify `--jars <spark core test jar>,<spark catalyst test jar>` because `SqlBasedBenchmark` based benchmark extends `BenchmarkBase(defined in spark core test jar)` and `SQLHelper(defined in spark catalyst test jar)`.

Another change of this pr is removed the `scalatest Assertions` dependency of Benchmarks because `scalatest-*.jar` are not in the distribution package, it will be troublesome to use.

### Why are the changes needed?
Make sure benchmarks can run using spark-submit cmd described in the guide

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
Use the corrected `spark-submit` commands to run benchmarks successfully.

Closes #31995 from LuciferYang/fix-benchmark-guide.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-30 11:58:01 +09:00
Baohe Zhang b2bfe985e8 [SPARK-34845][CORE] ProcfsMetricsGetter shouldn't return partial procfs metrics
### What changes were proposed in this pull request?
In ProcfsMetricsGetter.scala, propogating IOException from addProcfsMetricsFromOneProcess to computeAllMetrics when the child pid's proc stat file is unavailable. As a result, the for-loop in computeAllMetrics() can terminate earlier and return an all-0 procfs metric.

### Why are the changes needed?
In the case of a child pid's stat file missing and the subsequent child pids' stat files exist, ProcfsMetricsGetter.computeAllMetrics() will return partial metrics (the sum of a subset of child pids), which can be misleading and is undesired per the existing code comments in https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala#L214.

Also, a side effect of this bug is that it can lead to a verbose warning log if many pids' stat files are missing. An early terminating can make the warning logs more concise.

The unit test can also explain the bug well.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
A unit test is added.

Closes #31945 from baohe-zhang/SPARK-34845.

Authored-by: Baohe Zhang <baohe.zhang@verizonmedia.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-29 07:46:58 -07:00
Angerszhuuuu 066c055b52 [SPARK-34092][SQL] Support Stage level restful api filter task details by task status
### What changes were proposed in this pull request?
When we want to get stage's detail info with task information, it will return all tasks, the content is huge and always we just want to know some failed tasks/running tasks  with whole stage info to judge is a task has some problem. This pr support
user to use
```
/application/[appid]/stages/[stage-id]?details=true&taskStatus=xxx
/application/[appid]/stages/[stage-id]/[stage-attempted-id]?details=true&taskStatus=xxx
```
to filter task details by task status

### Why are the changes needed?
More flexiable Restful API

### Does this PR introduce _any_ user-facing change?
User can use
```
/application/[appid]/stages/[stage-id]?details=true&taskStatus=xxx
/application/[appid]/stages/[stage-id]/[stage-attempted-id]?details=true&taskStatus=xxx
```
to filter task details by task status

### How was this patch tested?
Added

Closes #31165 from AngersZhuuuu/SPARK-34092.

Lead-authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Co-authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-03-27 16:26:07 -05:00
Angerszhuuuu 15bf01ef85 [SPARK-34848][SQL][FLLOW-UP] Fix merge conflict issue cause UT failed
### What changes were proposed in this pull request?
Fix issue of  https://github.com/apache/spark/pull/31948#issuecomment-808647121

### Why are the changes needed?
fix ut failed

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Existed UT

Closes #31977 from AngersZhuuuu/SPARK-34848-followup.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-27 00:44:53 -07:00
Angerszhuuuu 36a51044a4 [SPARK-34848][CORE] Add duration to TaskMetricDistributions
### What changes were proposed in this pull request?
Task duration distribution is also very important for us to judge whether a stage's task is skew enough.

### Why are the changes needed?
Add important information in TaskMetricsDistribution

### Does this PR introduce _any_ user-facing change?
People can see duration distribution from TaskMetricsDistribution

### How was this patch tested?
Existed UT

Closes #31948 from AngersZhuuuu/SPARK-34848.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-03-26 12:32:20 -05:00
Angerszhuuuu 8ed5808f64 [SPARK-34488][CORE] Support task Metrics Distributions and executor Metrics Distributions in the REST API call for a specified stage
### What changes were proposed in this pull request?
For a specific stage, it is useful to show the task metrics in percentile distribution.  This information can help users know whether or not there is a skew/bottleneck among tasks in a given stage.  We list an example in taskMetricsDistributions.json

Similarly, it is useful to show the executor metrics in percentile distribution for a specific stage. This information can show whether or not there is a skewed load on some executors.  We list an example in executorMetricsDistributions.json

We define `withSummaries` and `quantiles` query parameter in the REST API for a specific stage as:

applications/<application_id>/<application_attempt/stages/<stage_id>/<stage_attempt>?withSummaries=[true|false]& quantiles=0.05,0.25,0.5,0.75,0.95

1. withSummaries: default is false, define whether to show current stage's taskMetricsDistribution and executorMetricsDistribution
2. quantiles: default is `0.0,0.25,0.5,0.75,1.0` only effect when `withSummaries=true`, it define the quantiles we use when calculating metrics distributions.

When withSummaries=true, both task metrics in percentile distribution and executor metrics in percentile distribution are included in the REST API output.  The default value of withSummaries is false, i.e. no metrics percentile distribution will be included in the REST API output.

 

### Why are the changes needed?
For a specific stage, it is useful to show the task metrics in percentile distribution.  This information can help users know whether or not there is a skew/bottleneck among tasks in a given stage.  We list an example in taskMetricsDistributions.json

### Does this PR introduce _any_ user-facing change?
User can  use  below restful API to get task metrics distribution and executor metrics distribution for indivial stage
```
applications/<application_id>/<application_attempt/stages/<stage_id>/<stage_attempt>?withSummaries=[true|false]
```

### How was this patch tested?
Added UT

Closes #31611 from AngersZhuuuu/SPARK-34488.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-03-24 08:50:45 -05:00
Liang-Chi Hsieh 95c61df0fa [SPARK-34295][CORE] Exclude filesystems from token renewal at YARN
### What changes were proposed in this pull request?

This patch adds a config `spark.yarn.kerberos.renewal.excludeHadoopFileSystems` which lists the filesystems to be excluded from delegation token renewal at YARN.

### Why are the changes needed?

MapReduce jobs can instruct YARN to skip renewal of tokens obtained from certain hosts by specifying the hosts with configuration mapreduce.job.hdfs-servers.token-renewal.exclude=<host1>,<host2>,..,<hostN>.

But seems Spark lacks of similar option. So the job submission fails if YARN fails to renew DelegationToken for any of the remote HDFS cluster. The failure in DT renewal can happen due to many reason like Remote HDFS does not trust Kerberos identity of YARN etc. We have a customer facing such issue.

### Does this PR introduce _any_ user-facing change?

No, if the config is not set. Yes, as users can use this config to instruct YARN not to renew delegation token from certain filesystems.

### How was this patch tested?

It is hard to do unit test for this. We did verify it work from the customer using this fix in the production environment.

Closes #31761 from viirya/SPARK-34295.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-03-24 01:11:53 -07:00
Shardul Mahadik 2298cebcf8 [SPARK-34477][CORE] Register KryoSerializers for Avro GenericData classes
### What changes were proposed in this pull request?
1) Modify `GenericAvroSerializer` to support serialization of any `GenericContainer`
2) Register `KryoSerializer`s for `GenericData.{Array, EnumSymbol, Fixed}` using the modified `GenericAvroSerializer`

### Why are the changes needed?
Without this change, Kryo throws NPEs when trying to serialize `GenericData.{Array, EnumSymbol, Fixed}`. More details in SPARK-34477 Jira

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Added unit tests for testing roundtrip serialization and deserialization of `GenericData.{Array, EnumSymbol, Fixed}` using `GenericAvroSerializer` directly and also indirectly through `KryoSerializer`

Closes #31597 from shardulm94/avro-array-serializer.

Authored-by: Shardul Mahadik <smahadik@linkedin.com>
Signed-off-by: Gengliang Wang <ltnwgl@gmail.com>
2021-03-24 15:31:34 +08:00
yi.wu e00afd31a7 [SPARK-34087][FOLLOW-UP][SQL] Manage ExecutionListenerBus register inside itself
### What changes were proposed in this pull request?

Move `ExecutionListenerBus` register (both `ListenerBus` and `ContextCleaner` register) into  itself.

Also with a minor change that put `registerSparkListenerForCleanup` to a better place.

### Why are the changes needed?

improve code

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Pass existing tests.

Closes #31919 from Ngone51/SPARK-34087-followup.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-23 07:38:43 +00:00
hezuojiao 39542bb81f [SPARK-34790][CORE] Disable fetching shuffle blocks in batch when io encryption is enabled
### What changes were proposed in this pull request?

This patch proposes to disable fetching shuffle blocks in batch when io encryption is enabled. Adaptive Query Execution fetch contiguous shuffle blocks for the same map task in batch to reduce IO and improve performance. However, we found that batch fetching is incompatible with io encryption.

### Why are the changes needed?
Before this patch, we set `spark.io.encryption.enabled` to true, then run some queries which coalesced partitions by AEQ, may got following error message:
```14:05:52.638 WARN org.apache.spark.scheduler.TaskSetManager: Lost task 1.0 in stage 2.0 (TID 3) (11.240.37.88 executor driver): FetchFailed(BlockManagerId(driver, 11.240.37.88, 63574, None), shuffleId=0, mapIndex=0, mapId=0, reduceId=2, message=
org.apache.spark.shuffle.FetchFailedException: Stream is corrupted
	at org.apache.spark.storage.ShuffleBlockFetcherIterator.throwFetchFailedException(ShuffleBlockFetcherIterator.scala:772)
	at org.apache.spark.storage.BufferReleasingInputStream.read(ShuffleBlockFetcherIterator.scala:845)
	at java.io.BufferedInputStream.fill(BufferedInputStream.java:246)
	at java.io.BufferedInputStream.read(BufferedInputStream.java:265)
	at java.io.DataInputStream.readInt(DataInputStream.java:387)
	at org.apache.spark.sql.execution.UnsafeRowSerializerInstance$$anon$2$$anon$3.readSize(UnsafeRowSerializer.scala:113)
	at org.apache.spark.sql.execution.UnsafeRowSerializerInstance$$anon$2$$anon$3.next(UnsafeRowSerializer.scala:129)
	at org.apache.spark.sql.execution.UnsafeRowSerializerInstance$$anon$2$$anon$3.next(UnsafeRowSerializer.scala:110)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:494)
	at scala.collection.Iterator$$anon$10.next(Iterator.scala:459)
	at org.apache.spark.util.CompletionIterator.next(CompletionIterator.scala:29)
	at org.apache.spark.InterruptibleIterator.next(InterruptibleIterator.scala:40)
	at scala.collection.Iterator$$anon$10.next(Iterator.scala:459)
	at org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:345)
	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:898)
	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:898)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:337)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
	at org.apache.spark.scheduler.Task.run(Task.scala:131)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:498)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1437)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:501)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
Caused by: java.io.IOException: Stream is corrupted
	at net.jpountz.lz4.LZ4BlockInputStream.refill(LZ4BlockInputStream.java:200)
	at net.jpountz.lz4.LZ4BlockInputStream.refill(LZ4BlockInputStream.java:226)
	at net.jpountz.lz4.LZ4BlockInputStream.read(LZ4BlockInputStream.java:157)
	at org.apache.spark.storage.BufferReleasingInputStream.read(ShuffleBlockFetcherIterator.scala:841)
	... 25 more

)
```

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

New tests.

Closes #31898 from hezuojiao/fetch_shuffle_in_batch.

Authored-by: hezuojiao <hezuojiao@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-22 13:06:12 -07:00
PengLei 85581f6dac [SPARK-33925][CORE][FOLLOW-UP] Remove the unused variables 'secMgr'
### What changes were proposed in this pull request?
Remove the unused variable 'secMgr' in SparkSubmit.scala and DriverWrapper.scala
In jira https://issues.apache.org/jira/browse/SPARK-33925, The last usage of SecurityManager in Utils.fetchFile was removed. We don't need the variable anymore

### Why are the changes needed?
For better readablity of codes

### Does this PR introduce _any_ user-facing change?
No,dev-only

### How was this patch tested?
Manually complied. Github Actions and Jenkins build should test it out as well.

Closes #31928 from Peng-Lei/rm_secMgr.

Authored-by: PengLei <18066542445@189.cn>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-03-22 12:02:25 -05:00
yi.wu e4bb97526c [SPARK-34089][CORE] HybridRowQueue should respect the configured memory mode
### What changes were proposed in this pull request?

This PR fixes the `HybridRowQueue ` to respect the configured memory mode.

Besides, this PR also refactored the constructor of `MemoryConsumer` to accept the memory mode explicitly.

### Why are the changes needed?

`HybridRowQueue` supports both onHeap and offHeap manipulation. But it inherited the wrong `MemoryConsumer` constructor, which hard-coded the memory mode to `onHeap`.

### Does this PR introduce _any_ user-facing change?

No. (Maybe yes in some cases where users can't complete the job before could complete successfully after the fix because of `HybridRowQueue` is able to spill under offHeap mode now. )

### How was this patch tested?

Updated the existing test to make it test both offHeap and onHeap modes.

Closes #31152 from Ngone51/fix-MemoryConsumer-memorymode.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-03-22 08:12:08 +00:00
Bo Zhang 3bef2dc01a Revert "[SPARK-34757][CORE][DEPLOY] Ignore cache for SNAPSHOT dependencies in spark-submit"
### What changes were proposed in this pull request?

This reverts commit 86ea520320.

### Why are the changes needed?

The test added in the change was flaky.

Closes #31918 from bozhang2820/revert-spark-34757.

Authored-by: Bo Zhang <bo.zhang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-22 15:07:15 +09:00
Kousuke Saruta 0734101bb7 [SPARK-34225][CORE] Don't encode further when a URI form string is passed to addFile or addJar
### What changes were proposed in this pull request?

This PR fixes an issue that `addFile` and `addJar` further encode even though a URI form string is passed.
For example, the following operation will throw exception even though the file exists.
```
sc.addFile("file:/foo/test%20file.txt")
```

Another case is `--files` and `--jars` option when we submit an application.
```
bin/spark-shell --files "/foo/test file.txt"
```
The path above is transformed to URI form [here](ecf4811764/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala (L400)) and passed to `addFile` so the same issue happens.

### Why are the changes needed?

This is a bug.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

New test.

Closes #31718 from sarutak/fix-uri-encode-double.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Kousuke Saruta <sarutak@oss.nttdata.com>
2021-03-22 14:06:41 +09:00
Dongjoon Hyun 3bc6fe4e77 [SPARK-34809][CORE] Enable spark.hadoopRDD.ignoreEmptySplits by default
### What changes were proposed in this pull request?

This PR aims to enable `spark.hadoopRDD.ignoreEmptySplits` by default for Apache Spark 3.2.0.

### Why are the changes needed?

Although this is a safe improvement, this hasn't been enabled by default to avoid the explicit behavior change. This PR aims to switch the default explicitly in Apache Spark 3.2.0.

### Does this PR introduce _any_ user-facing change?

Yes, the behavior change is documented.

### How was this patch tested?

Pass the existing CIs.

Closes #31909 from dongjoon-hyun/SPARK-34809.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-21 14:34:02 -07:00
Dongjoon Hyun 3c32b54a0f [SPARK-34811][CORE] Redact fs.s3a.access.key like secret and token
### What changes were proposed in this pull request?

Like we redact secrets and tokens, this PR aims to redact access key.

### Why are the changes needed?

Access key is also worth to hide.

### Does this PR introduce _any_ user-facing change?

This will hide this information from SparkUI (`Spark Properties` and `Hadoop Properties` and logs).

### How was this patch tested?

Pass the newly updated UT.

Closes #31912 from dongjoon-hyun/SPARK-34811.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-21 14:08:34 -07:00
Ruifeng Zheng f11950f08f [SPARK-32384][CORE] repartitionAndSortWithinPartitions avoid shuffle with same partitioner
### What changes were proposed in this pull request?
avoid unnecessary shuffle if possible

### Why are the changes needed?
avoid unnecessary shuffle.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
added testsuites and existing testsuites

Closes #31480 from zhengruifeng/repartitionAndSortWithinPartitions_opt_II.

Authored-by: Ruifeng Zheng <ruifengz@foxmail.com>
Signed-off-by: yi.wu <yi.wu@databricks.com>
2021-03-20 10:29:48 +08:00
Bo Zhang 86ea520320 [SPARK-34757][CORE][DEPLOY] Ignore cache for SNAPSHOT dependencies in spark-submit
### What changes were proposed in this pull request?
This change is to ignore cache for SNAPSHOT dependencies in spark-submit.

### Why are the changes needed?
When spark-submit is executed with --packages, it will not download the dependency jars when they are available in cache (e.g. ivy cache), even when the dependencies are SNAPSHOT.

This might block developers who work on external modules in Spark (e.g. spark-avro), since they need to remove the cache manually every time when they update the code during developments (which generates SNAPSHOT jars). Without knowing this, they could be blocked wondering why their code changes are not reflected in spark-submit executions.

### Does this PR introduce _any_ user-facing change?
Yes. With this change, developers/users who run spark-submit with SNAPSHOT dependencies do not need to remove the cache every time when the SNAPSHOT dependencies are updated.

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

Closes #31849 from bozhang2820/spark-submit-cache-ignore.

Authored-by: Bo Zhang <bo.zhang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-18 15:32:29 +09:00
Bruce Robbins f8a8b340b3 [SPARK-34731][CORE] Avoid ConcurrentModificationException when redacting properties in EventLoggingListener
### What changes were proposed in this pull request?

Change DAGScheduler to pass a clone of the Properties object, rather than the original object, to the SparkListenerJobStart event.

### Why are the changes needed?

 DAGScheduler might modify the Properties object (e.g., in addPySparkConfigsToProperties) after firing off the SparkListenerJobStart event. Since the handler for that event (onJobStart in EventLoggingListener) will iterate over the elements of the Property object, this sometimes results in a ConcurrentModificationException.

This can be demonstrated using these steps:
```
$ bin/spark-shell --conf spark.ui.showConsoleProgress=false \
--conf spark.executor.cores=1 --driver-memory 4g --conf \
"spark.ui.showConsoleProgress=false" \
--conf spark.eventLog.enabled=true \
--conf spark.eventLog.dir=/tmp/spark-events
...
scala> (0 to 500).foreach { i =>
     |   val df = spark.range(0, 20000).toDF("a")
     |   df.filter("a > 12").count
     | }
21/03/12 18:16:44 ERROR AsyncEventQueue: Listener EventLoggingListener threw an exception
java.util.ConcurrentModificationException
	at java.util.Hashtable$Enumerator.next(Hashtable.java:1387)
```

I've not actually seen a ConcurrentModificationException in onStageSubmitted, only in onJobStart. However, they both iterate over the Properties object, so for safety's sake I pass a clone to SparkListenerStageSubmitted as well.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
By repeatedly running the reproduction steps from above.

Closes #31826 from bersprockets/elconcurrent.

Authored-by: Bruce Robbins <bersprockets@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-18 14:59:57 +09:00
yi.wu 4d90c5dc0e [SPARK-34087][SQL] Fix memory leak of ExecutionListenerBus
### What changes were proposed in this pull request?

This PR proposes an alternative way to fix the memory leak of `ExecutionListenerBus`, which would automatically clean them up.

Basically, the idea is to add `registerSparkListenerForCleanup` to `ContextCleaner`, so we can remove the `ExecutionListenerBus` from `LiveListenerBus` when the `SparkSession` is GC'ed.

On the other hand, to make the `SparkSession` GC-able, we need to get rid of the reference of `SparkSession` in `ExecutionListenerBus`. Therefore, we introduced the `sessionUUID`, which is a unique identifier for SparkSession, to replace the  `SparkSession` object.

Note that, the proposal wouldn't take effect when `spark.cleaner.referenceTracking=false` since it depends on `ContextCleaner`.

### Why are the changes needed?

Fix the memory leak caused by `ExecutionListenerBus` mentioned in SPARK-34087.

### Does this PR introduce _any_ user-facing change?

Yes, save memory for users.

### How was this patch tested?

Added unit test.

Closes #31839 from Ngone51/fix-mem-leak-of-ExecutionListenerBus.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-03-18 13:27:03 +09:00
yangjie01 e757091820 [SPARK-34722][CORE][SQL][TEST] Clean up deprecated API usage related to JUnit4
### What changes were proposed in this pull request?
The main change of this pr as follows:

- Use `org.junit.Assert.assertThrows(String, Class, ThrowingRunnable)` method instead of  `ExpectedException.none()`
- Use `org.hamcrest.MatcherAssert.assertThat()` method instead of   `org.junit.Assert.assertThat(T, org.hamcrest.Matcher<? super T>)`

### Why are the changes needed?
Clean up deprecated API usage

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #31815 from LuciferYang/SPARK-34722.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-14 23:33:03 -07:00
wankunde 60e324aa9f [SPARK-34688][PYTHON] Upgrade to Py4J 0.10.9.2
### What changes were proposed in this pull request?
This PR upgrade Py4J from 0.10.9.1 to 0.10.9.2 that contains some bug fixes and improvements.

* expose shell parameter in Popen inside launch_gateway. ([bartdag/py4j220efc3](220efc3716))
* fixed Flake8 errors ([bartdag/py4j6c6ee9a](6c6ee9aedc))

### Why are the changes needed?
To leverage fixes from the upstream in Py4J.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
Jenkins build and GitHub Actions will test it out.

Closes #31796 from wankunde/py4j.

Authored-by: wankunde <wankunde@163.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-03-11 09:51:41 -06:00
Dongjoon Hyun ba7e525a11 [SPARK-34670][BUILD] Upgrade ZSTD-JNI to 1.4.9-1
### What changes were proposed in this pull request?

This PR aims to upgrade ZSTD-JNI to 1.4.9-1.

### Why are the changes needed?

ZStandard 1.4.9 and its corresponding JNI brings the following bug fixes and improvements.
- https://github.com/facebook/zstd/releases/tag/v1.4.9

One of notable improvement of ZStandard 1.4.9 is `2x faster Long Distance Mode`, but we are not using it yet.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Pass the CIs with the existing tests and there is no regression in ZStandardBenchmark.

Closes #31784 from dongjoon-hyun/ZSTD-149.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-08 22:40:49 -08:00
yikf f340857757 [SPARK-34541][CORE] Fixed an issue where data could not be cleaned up when unregisterShuffle
### What changes were proposed in this pull request?
Fixed an issue where data could not be cleaned up when unregisterShuffle.

### Why are the changes needed?
While we use the old shuffle fetch protocol, we use partitionId as mapId in the ShuffleBlockId construction,but we use `context.taskAttemptId()` as mapId that it is cached in `taskIdMapsForShuffle` when we `getWriter[K, V]`.

where data could not be cleaned up when unregisterShuffle ,because we remove a shuffle's metadata from the `taskIdMapsForShuffle`'s mapIds, the mapId is `context.taskAttemptId()` instead of partitionId.

### Does this PR introduce _any_ user-facing change?
yes

### How was this patch tested?
add new test.

Closes #31664 from yikf/master.

Authored-by: yikf <13468507104@163.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-03-08 07:08:29 -06:00
Peter Toth ab8a9a0ceb [SPARK-34545][SQL] Fix issues with valueCompare feature of pyrolite
### What changes were proposed in this pull request?

pyrolite 4.21 introduced and enabled value comparison by default (`valueCompare=true`) during object memoization and serialization: https://github.com/irmen/Pyrolite/blob/pyrolite-4.21/java/src/main/java/net/razorvine/pickle/Pickler.java#L112-L122
This change has undesired effect when we serialize a row (actually `GenericRowWithSchema`) to be passed to python: https://github.com/apache/spark/blob/branch-3.0/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala#L60. A simple example is that
```
new GenericRowWithSchema(Array(1.0, 1.0), StructType(Seq(StructField("_1", DoubleType), StructField("_2", DoubleType))))
```
and
```
new GenericRowWithSchema(Array(1, 1), StructType(Seq(StructField("_1", IntegerType), StructField("_2", IntegerType))))
```
are currently equal and the second instance is replaced to the short code of the first one during serialization.

### Why are the changes needed?
The above can cause nasty issues like the one in https://issues.apache.org/jira/browse/SPARK-34545 description:

```
>>> from pyspark.sql.functions import udf
>>> from pyspark.sql.types import *
>>>
>>> def udf1(data_type):
        def u1(e):
            return e[0]
        return udf(u1, data_type)
>>>
>>> df = spark.createDataFrame([((1.0, 1.0), (1, 1))], ['c1', 'c2'])
>>>
>>> df = df.withColumn("c3", udf1(DoubleType())("c1"))
>>> df = df.withColumn("c4", udf1(IntegerType())("c2"))
>>>
>>> df.select("c3").show()
+---+
| c3|
+---+
|1.0|
+---+

>>> df.select("c4").show()
+---+
| c4|
+---+
|  1|
+---+

>>> df.select("c3", "c4").show()
+---+----+
| c3|  c4|
+---+----+
|1.0|null|
+---+----+
```
This is because during serialization from JVM to Python `GenericRowWithSchema(1.0, 1.0)` (`c1`) is memoized first and when `GenericRowWithSchema(1, 1)` (`c2`) comes next, it is replaced to some short code of the `c1` (instead of serializing `c2` out) as they are `equal()`. The python functions then runs but the return type of `c4` is expected to be `IntegerType` and if a different type (`DoubleType`) comes back from python then it is discarded: https://github.com/apache/spark/blob/branch-3.0/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala#L108-L113

After this PR:
```
>>> df.select("c3", "c4").show()
+---+---+
| c3| c4|
+---+---+
|1.0|  1|
+---+---+
```

### Does this PR introduce _any_ user-facing change?
Yes, fixes a correctness issue.

### How was this patch tested?
Added new UT + manual tests.

Closes #31682 from peter-toth/SPARK-34545-fix-row-comparison.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-03-07 19:12:42 -06:00
Dongjoon Hyun 1f6089b165 [SPARK-34647][CORE] Use ZSTD JNI NoFinalizer classes and bump to 1.4.8-7
### What changes were proposed in this pull request?

This PR aims to use `ZstdInputStreamNoFinalizer` and `ZstdOutputStreamNoFinalizer` classes and upgrade ZSTD JNI to 1.4.8-7.

### Why are the changes needed?

`1.4.8-7` makes `NoFinalizer` classes public again. This improves the performance.
- 57d53a09d2

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Pass the CIs.

Closes #31762 from dongjoon-hyun/SPARK-ZSTD-NOFINALIZER.

Lead-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-06 10:32:27 -08:00
Shardul Mahadik 1fd73686ba [SPARK-34624][CORE] Exclude non-jar dependencies of ivy/maven packages
### What changes were proposed in this pull request?
Exclude non-jar dependencies of the ivy/maven packages we want to resolve as our current dependency resolution code assumes artifacts to be jars. 17601e014c/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala (L1215) and 17601e014c/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala (L318)

### Why are the changes needed?
Some maven artifacts define non-jar dependencies. One such example is `hive-exec`'s dependency on the `pom` of `apache-curator` https://repo1.maven.org/maven2/org/apache/hive/hive-exec/2.3.8/hive-exec-2.3.8.pom

Today trying to depend on such an artifact using `--packages` will print an error but continue without including the non-jar dependency. Doing the same using `spark.sql("ADD JAR ivy://org.apache.hive:hive-exec:2.3.8?exclude=org.pentaho:pentaho-aggdesigner-algorithm")` will cause a failure. Detailed stacktraces can be found in SPARK-34624.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Added unit test. Retried the same example in `spark-shell` which produced the stacktrace in the JIRA.

Closes #31741 from shardulm94/add-jar-filter-poms.

Authored-by: Shardul Mahadik <smahadik@linkedin.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-05 10:05:51 -08:00
Liang-Chi Hsieh c91a756204 [SPARK-34592][WEBUI] Mark indeterminate RDD in Web UI
### What changes were proposed in this pull request?

This patch proposes to mark indeterminate RDD in Web UI.

### Why are the changes needed?

It is somehow hard to track which part is indeterminate in a graph of RDDs. In some cases we may need to track indeterminate RDDs. For example, indeterminate map stage fails and Spark is unable to fallback some parent stages. The developers are usually unable to easily identify indeterminate part from the complicated RDD computation. If Web UI can show up indeterminate RDD like cached RDD, it could be useful to track it.

### Does this PR introduce _any_ user-facing change?

Yes, there is a UI change for users.

### How was this patch tested?

Manual check with Web UI locally. Updated existing unit tests.

<img width="544" alt="Screen Shot 2021-03-02 at 12 38 02 AM" src="https://user-images.githubusercontent.com/68855/109621580-020bce80-7af0-11eb-834f-46b0f89d47c0.png">
<img width="390" alt="Screen Shot 2021-03-05 at 9 27 14 AM" src="https://user-images.githubusercontent.com/68855/110151181-04db1d80-7d95-11eb-8b3a-7235f7fe9eac.png">

Closes #31707 from viirya/SPARK-34592.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Liang-Chi Hsieh <viirya@gmail.com>
2021-03-05 09:31:31 -08:00
Gengliang Wang 358697b386 [SPARK-34635][UI] Add trailing slashes in URLs to reduce unnecessary redirects
### What changes were proposed in this pull request?

Add trailing slashes in URLs of Spark UI pages.

### Why are the changes needed?

When a user accesses a URL without a trailing slash, Spark UI always responds with a 302 redirect to a URL with a trailing slash.
![image](https://user-images.githubusercontent.com/1097932/110072744-1be92380-7d33-11eb-98d4-50df12f59ae3.png)

Adding trailing slash to URLs in UI pages can reduce such unnecessary redirects

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Manual test. It's a very simple change.

Closes #31753 from gengliangwang/reduceRedirect.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-04 23:23:53 -08:00
Takeshi Yamamuro dbce74d39d [SPARK-34607][SQL] Add Utils.isMemberClass to fix a malformed class name error on jdk8u
### What changes were proposed in this pull request?

This PR intends to fix a bug of `objects.NewInstance` if a user runs Spark on jdk8u and a given `cls` in `NewInstance` is a deeply-nested inner class, e.g.,.
```
  object OuterLevelWithVeryVeryVeryLongClassName1 {
    object OuterLevelWithVeryVeryVeryLongClassName2 {
      object OuterLevelWithVeryVeryVeryLongClassName3 {
        object OuterLevelWithVeryVeryVeryLongClassName4 {
          object OuterLevelWithVeryVeryVeryLongClassName5 {
            object OuterLevelWithVeryVeryVeryLongClassName6 {
              object OuterLevelWithVeryVeryVeryLongClassName7 {
                object OuterLevelWithVeryVeryVeryLongClassName8 {
                  object OuterLevelWithVeryVeryVeryLongClassName9 {
                    object OuterLevelWithVeryVeryVeryLongClassName10 {
                      object OuterLevelWithVeryVeryVeryLongClassName11 {
                        object OuterLevelWithVeryVeryVeryLongClassName12 {
                          object OuterLevelWithVeryVeryVeryLongClassName13 {
                            object OuterLevelWithVeryVeryVeryLongClassName14 {
                              object OuterLevelWithVeryVeryVeryLongClassName15 {
                                object OuterLevelWithVeryVeryVeryLongClassName16 {
                                  object OuterLevelWithVeryVeryVeryLongClassName17 {
                                    object OuterLevelWithVeryVeryVeryLongClassName18 {
                                      object OuterLevelWithVeryVeryVeryLongClassName19 {
                                        object OuterLevelWithVeryVeryVeryLongClassName20 {
                                          case class MalformedNameExample2(x: Int)
                                        }}}}}}}}}}}}}}}}}}}}
```

The root cause that Kris (rednaxelafx) investigated is as follows (Kudos to Kris);

The reason why the test case above is so convoluted is in the way Scala generates the class name for nested classes. In general, Scala generates a class name for a nested class by inserting the dollar-sign ( `$` ) in between each level of class nesting. The problem is that this format can concatenate into a very long string that goes beyond certain limits, so Scala will change the class name format beyond certain length threshold.

For the example above, we can see that the first two levels of class nesting have class names that look like this:
```
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassName1$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassName1$OuterLevelWithVeryVeryVeryLongClassName2$
```
If we leave out the fact that Scala uses a dollar-sign ( `$` ) suffix for the class name of the companion object, `OuterLevelWithVeryVeryVeryLongClassName1`'s full name is a prefix (substring) of `OuterLevelWithVeryVeryVeryLongClassName2`.

But if we keep going deeper into the levels of nesting, you'll find names that look like:
```
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$2a1321b953c615695d7442b2adb1$$$$ryVeryLongClassName8$OuterLevelWithVeryVeryVeryLongClassName9$OuterLevelWithVeryVeryVeryLongClassName10$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$2a1321b953c615695d7442b2adb1$$$$ryVeryLongClassName8$OuterLevelWithVeryVeryVeryLongClassName9$OuterLevelWithVeryVeryVeryLongClassName10$OuterLevelWithVeryVeryVeryLongClassName11$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$85f068777e7ecf112afcbe997d461b$$$$VeryLongClassName11$OuterLevelWithVeryVeryVeryLongClassName12$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$85f068777e7ecf112afcbe997d461b$$$$VeryLongClassName11$OuterLevelWithVeryVeryVeryLongClassName12$OuterLevelWithVeryVeryVeryLongClassName13$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$85f068777e7ecf112afcbe997d461b$$$$VeryLongClassName11$OuterLevelWithVeryVeryVeryLongClassName12$OuterLevelWithVeryVeryVeryLongClassName13$OuterLevelWithVeryVeryVeryLongClassName14$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$5f7ad51804cb1be53938ea804699fa$$$$VeryLongClassName14$OuterLevelWithVeryVeryVeryLongClassName15$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$5f7ad51804cb1be53938ea804699fa$$$$VeryLongClassName14$OuterLevelWithVeryVeryVeryLongClassName15$OuterLevelWithVeryVeryVeryLongClassName16$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$5f7ad51804cb1be53938ea804699fa$$$$VeryLongClassName14$OuterLevelWithVeryVeryVeryLongClassName15$OuterLevelWithVeryVeryVeryLongClassName16$OuterLevelWithVeryVeryVeryLongClassName17$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$69b54f16b1965a31e88968df1a58d8$$$$VeryLongClassName17$OuterLevelWithVeryVeryVeryLongClassName18$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$69b54f16b1965a31e88968df1a58d8$$$$VeryLongClassName17$OuterLevelWithVeryVeryVeryLongClassName18$OuterLevelWithVeryVeryVeryLongClassName19$
org.apache.spark.sql.catalyst.encoders.ExpressionEncoderSuite$OuterLevelWithVeryVeryVeryLongClassNam$$$$69b54f16b1965a31e88968df1a58d8$$$$VeryLongClassName17$OuterLevelWithVeryVeryVeryLongClassName18$OuterLevelWithVeryVeryVeryLongClassName19$OuterLevelWithVeryVeryVeryLongClassName20$
```
with a hash code in the middle and various levels of nesting omitted.

The `java.lang.Class.isMemberClass` method is implemented in JDK8u as:
http://hg.openjdk.java.net/jdk8u/jdk8u/jdk/file/tip/src/share/classes/java/lang/Class.java#l1425
```
    /**
     * Returns {code true} if and only if the underlying class
     * is a member class.
     *
     * return {code true} if and only if this class is a member class.
     * since 1.5
     */
    public boolean isMemberClass() {
        return getSimpleBinaryName() != null && !isLocalOrAnonymousClass();
    }

    /**
     * Returns the "simple binary name" of the underlying class, i.e.,
     * the binary name without the leading enclosing class name.
     * Returns {code null} if the underlying class is a top level
     * class.
     */
    private String getSimpleBinaryName() {
        Class<?> enclosingClass = getEnclosingClass();
        if (enclosingClass == null) // top level class
            return null;
        // Otherwise, strip the enclosing class' name
        try {
            return getName().substring(enclosingClass.getName().length());
        } catch (IndexOutOfBoundsException ex) {
            throw new InternalError("Malformed class name", ex);
        }
    }
```
and the problematic code is `getName().substring(enclosingClass.getName().length())` -- if a class's enclosing class's full name is *longer* than the nested class's full name, this logic would end up going out of bounds.

The bug has been fixed in JDK9 by https://bugs.java.com/bugdatabase/view_bug.do?bug_id=8057919 , but still exists in the latest JDK8u release. So from the Spark side we'd need to do something to avoid hitting this problem.

### Why are the changes needed?

Bugfix on jdk8u.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Added tests.

Closes #31733 from maropu/SPARK-34607.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-05 08:59:30 +09:00
Baohe Zhang 9ac5ee2e17 [SPARK-32924][WEBUI] Make duration column in master UI sorted in the correct order
### What changes were proposed in this pull request?

Make the "duration" column in standalone mode master UI sorted by numeric duration, hence the column can be sorted by the correct order.

Before changes:
![image](https://user-images.githubusercontent.com/26694233/110025426-f5a49300-7cf4-11eb-86f0-2febade86be9.png)

After changes:
![image](https://user-images.githubusercontent.com/26694233/110025604-33092080-7cf5-11eb-8b34-215688faf56d.png)

### Why are the changes needed?

Fix a UI bug to make the sorting consistent across different pages.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
Ran several apps with different durations and verified the duration column on the master page can be sorted correctly.

Closes #31743 from baohe-zhang/SPARK-32924.

Authored-by: Baohe Zhang <baohe.zhang@verizonmedia.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-03-04 15:37:33 -08:00
Shardul Mahadik 0216051aca [SPARK-34506][CORE] ADD JAR with ivy coordinates should be compatible with Hive transitive behavior
### What changes were proposed in this pull request?
SPARK-33084 added the ability to use ivy coordinates with `SparkContext.addJar`. PR #29966 claims to mimic Hive behavior although I found a few cases where it doesn't

1) The default value of the transitive parameter is false, both in case of parameter not being specified in coordinate or parameter value being invalid. The Hive behavior is that transitive is [true if not specified](cb2ac3dcc6/ql/src/java/org/apache/hadoop/hive/ql/util/DependencyResolver.java (L169)) in the coordinate and [false for invalid values](cb2ac3dcc6/ql/src/java/org/apache/hadoop/hive/ql/util/DependencyResolver.java (L124)). Also, regardless of Hive, I think a default of true for the transitive parameter also matches [ivy's own defaults](https://ant.apache.org/ivy/history/2.5.0/ivyfile/dependency.html#_attributes).

2) The parameter value for transitive parameter is regarded as case-sensitive [based on the understanding](https://github.com/apache/spark/pull/29966#discussion_r547752259) that Hive behavior is case-sensitive. However, this is not correct, Hive [treats the parameter value case-insensitively](cb2ac3dcc6/ql/src/java/org/apache/hadoop/hive/ql/util/DependencyResolver.java (L122)).

I propose that we be compatible with Hive for these behaviors

### Why are the changes needed?
To make `ADD JAR` with ivy coordinates compatible with Hive's transitive behavior

### Does this PR introduce _any_ user-facing change?

The user-facing changes here are within master as the feature introduced in SPARK-33084 has not been released yet
1. Previously an ivy coordinate without `transitive` parameter specified did not resolve transitive dependency, now it does.
2. Previously an `transitive` parameter value was treated case-sensitively. e.g. `transitive=TRUE` would be treated as false as it did not match exactly `true`. Now it will be treated case-insensitively.

### How was this patch tested?

Modified existing unit tests to test new behavior
Add new unit test to cover usage of `exclude` with unspecified `transitive`

Closes #31623 from shardulm94/spark-34506.

Authored-by: Shardul Mahadik <smahadik@linkedin.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2021-03-01 09:10:20 +09:00
HyukjinKwon 8a1e172b51 [SPARK-34520][CORE] Remove unused SecurityManager references
### What changes were proposed in this pull request?

This is kind of a followup of https://github.com/apache/spark/pull/24033 and https://github.com/apache/spark/pull/30945.
Many of references in `SecurityManager` were introduced from SPARK-1189, and related usages were removed later from https://github.com/apache/spark/pull/24033 and https://github.com/apache/spark/pull/30945. This PR proposes to remove them out.

### Why are the changes needed?

For better readability of codes.

### Does this PR introduce _any_ user-facing change?

No, dev-only.

### How was this patch tested?

Manually complied. GitHub Actions and Jenkins build should test it out as well.

Closes #31636 from HyukjinKwon/SPARK-34520.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-24 20:38:03 -08:00
HyukjinKwon 22383e312d [SPARK-34531][CORE] Remove Experimental API tag in PrometheusServlet
### What changes were proposed in this pull request?

The endpoints of Prometheus metrics are properly marked and documented as an experimental (SPARK-31674). The class `PrometheusServlet` itself is not the part of an API so this PR proposes to remove it.

### Why are the changes needed?

To avoid marking a non-API as an API.

### Does this PR introduce _any_ user-facing change?

No, the class is already `private[spark]`.

### How was this patch tested?

Existing tests should cover.

Closes #31640 from HyukjinKwon/SPARK-34531.

Lead-authored-by: HyukjinKwon <gurwls223@apache.org>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-24 18:11:25 -08:00
yi.wu f542ecdb0d [SPARK-34245][CORE] Ensure Master removes executors that failed to send finished state
### What changes were proposed in this pull request?

Use `ask` instead of `send` to sync the `ExecutorStateChanged` between Worker and Master and retry(up to 5 times) on the failure until the message is successfully handled by the Master. And the Worker would exit itself if the message can not be sent after 5 times retry.

### Why are the changes needed?

If the Worker fails to send ExecutorStateChanged to the Master due to some unexpected errors, e.g., temporary network error, then the Master can't remove the finished executor normally and think the executor is still alive. In the worst case, if the executor is the only executor for the application, the application can get hang.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Pass existing tests.

Closes #31348 from Ngone51/periodically-trigger-master-schedule.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-24 05:05:04 +00:00
Dongjoon Hyun 2e31e2c5f3 [SPARK-34503][CORE] Use zstd for spark.eventLog.compression.codec by default
### What changes were proposed in this pull request?

Apache Spark 3.0 introduced `spark.eventLog.compression.codec` configuration.
For Apache Spark 3.2, this PR aims to set `zstd` as the default value for `spark.eventLog.compression.codec` configuration.
This only affects creating a new log file.

### Why are the changes needed?

The main purpose of event logs is archiving. Many logs are generated and occupy the storage, but most of them are never accessed by users.

**1. Save storage resources (and money)**

In general, ZSTD is much smaller than LZ4.
For example, in case of TPCDS (Scale 200) log, ZSTD generates about 3 times smaller log files than LZ4.

| CODEC | SIZE (bytes) |
|---------|-------------|
| LZ4         | 184001434|
| ZSTD      |  64522396|

And, the plain file is 17.6 times bigger.
```
-rw-r--r--    1 dongjoon  staff  1135464691 Feb 21 22:31 spark-a1843ead29834f46b1125a03eca32679
-rw-r--r--    1 dongjoon  staff    64522396 Feb 21 22:31 spark-a1843ead29834f46b1125a03eca32679.zstd
```

**2. Better Usability**

We cannot decompress Spark-generated LZ4 event log files via CLI while we can for ZSTD event log files. Spark's LZ4 event log files are inconvenient to some users who want to uncompress and access them.
```
$ lz4 -d spark-d3deba027bd34435ba849e14fc2c42ef.lz4
Decoding file spark-d3deba027bd34435ba849e14fc2c42ef
Error 44 : Unrecognized header : file cannot be decoded
```
```
$ zstd -d spark-a1843ead29834f46b1125a03eca32679.zstd
spark-a1843ead29834f46b1125a03eca32679.zstd: 1135464691 bytes
```

**3. Speed**
The following results are collected by running [lzbench](https://github.com/inikep/lzbench) on the above Spark event log. Note that
- This is not a direct comparison of Spark compression/decompression codec.
- `lzbench` is an in-memory benchmark. So, it doesn't show the benefit of the reduced network traffic due to the small size of ZSTD.

Here,
- To get ZSTD 1.4.8-1 result, `lzbench` `master` branch is used because Spark is using ZSTD 1.4.8.
- To get LZ4 1.7.5 result, `lzbench` `v1.7` branch is used because Spark is using LZ4 1.7.1.
```
Compressor name      Compress. Decompress. Compr. size  Ratio Filename
memcpy               7393 MB/s  7166 MB/s  1135464691 100.00 spark-a1843ead29834f46b1125a03eca32679
zstd 1.4.8 -1        1344 MB/s  3351 MB/s    56665767   4.99 spark-a1843ead29834f46b1125a03eca32679
lz4 1.7.5            1385 MB/s  4782 MB/s   127662168  11.24 spark-a1843ead29834f46b1125a03eca32679
```

### Does this PR introduce _any_ user-facing change?

- No for the apps which doesn't use `spark.eventLog.compress` because `spark.eventLog.compress` is disabled by default.
- No for the apps using `spark.eventLog.compression.codec` explicitly because this is a change of the default value.
- Yes for the apps using `spark.eventLog.compress` without setting `spark.eventLog.compression.codec`. In this case, previously `spark.io.compression.codec` value was used whose default is `lz4`.

So this JIRA issue, SPARK-34503, is labeled with `releasenotes`.

### How was this patch tested?

Pass the updated UT.

Closes #31618 from dongjoon-hyun/SPARK-34503.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-23 16:37:29 -08:00
yi.wu 546d2eb5d4 [SPARK-34384][CORE] Add missing docs for ResourceProfile APIs
### What changes were proposed in this pull request?

This PR adds missing docs for ResourceProfile related APIs. Besides, it includes a few minor changes on API:

* ResourceProfileBuilder.build -> ResourceProfileBuilder.builder()
* Provides java specific API `allSupportedExecutorResourcesJList`
* private `ResourceAllocator` since it was mistakenly exposed previously

### Why are the changes needed?

Add missing API docs

### Does this PR introduce _any_ user-facing change?

No, as Apache Spark 3.1 hasn't officially released.

### How was this patch tested?

Updated unit tests due to the signature change of `build()`.

Closes #31496 from Ngone51/resource-profile-api-cleanup.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-21 18:29:44 +09:00
Gera Shegalov fadd0f5d9b [SPARK-20977][CORE] Use a non-final field for the state of CollectionAccumulator
This PR is a fix for the JLS 17.5.3 violation identified in
zsxwing's [19/Feb/19 11:47 comment](https://issues.apache.org/jira/browse/SPARK-20977?focusedCommentId=16772277&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-16772277) on the JIRA.

### What changes were proposed in this pull request?
- Use a var field to hold the state of the collection accumulator

### Why are the changes needed?
AccumulatorV2 auto-registration of accumulator during readObject doesn't work with final fields that are post-processed outside readObject. As it stands incompletely initialized objects are published to heartbeat thread. This leads to sporadic exceptions knocking out executors which increases the cost of the jobs. We observe such failures on a regular basis https://github.com/NVIDIA/spark-rapids/issues/1522.

### Does this PR introduce _any_ user-facing change?
None

### How was this patch tested?
- this is a concurrency bug that is almost impossible to reproduce as a quick unit test.
- By trial and error I crafted a command https://github.com/NVIDIA/spark-rapids/pull/1688 that reproduces the issue on my dev box several times per hour, with the first occurrence often within a few minutes. After the patch, these Exceptions have not shown up after running overnight for 10+ hours
- existing unit tests in *`AccumulatorV2Suite` and *`LiveEntitySuite`

Closes #31540 from gerashegalov/SPARK-20977.

Authored-by: Gera Shegalov <gera@apache.org>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-02-20 20:57:14 -06:00
yi.wu 4dc16f2d59 [SPARK-24818][CORE] Support delay scheduling for barrier execution
### What changes were proposed in this pull request?

This PR tries to support the (non-legacy) delay scheduling for the barrier execution.

The idea is, adding a pending launch tasks list(`barrierPendingLaunchTasks`) in the barrier `TaskSetManager`. And we don't really add those pending launch tasks to the running list and post task start event to the listeners and so on until all tasks in the barrier `TaskSetManager` has been added to `barrierPendingLaunchTasks` after a single round `resourceOffers()`. If there're only partial tasks that are able to launch after a single `rousourceOffers()` round, we'll revert all the assigned resources to those tasks which were added in `barrierPendingLaunchTasks` and clear `barrierPendingLaunchTasks` and wait for the next `resourceOffers()` round. The barrier `TaskSetManager` should be launched finally since we've ensured enough slots before the scheduling.

### Why are the changes needed?

Currently, with delay scheduling enabled for the barrier execution, the application can abort immediately when there're only partial tasks can be launched. This is really bad, especially when the application already completed many stages before the barrier stage. For example, the application may do some ETL jobs before the barrier job(for ML).

After this PR, this scenario  should no longer happen.

### Does this PR introduce _any_ user-facing change?

Yes, users will no longer face the `Fail resource offers for barrier stage...` error.

### How was this patch tested?

Added/updated unit tests.

Closes #30650 from Ngone51/barrier-delay-scheduling.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Mridul Muralidharan <mridul<at>gmail.com>
2021-02-19 16:04:44 -06:00
Dongjoon Hyun 331c6fd4ef [SPARK-34467][BUILD] Upgrade Zstd-jni to 1.4.8-4
### What changes were proposed in this pull request?

This PR aims to upgrade Zstd-JNI library to 1.4.8-4 to bring JNI side optimization.
`ZStandardBenchmark` shows that there is no regression in terms of performance and show some improvements.

### Why are the changes needed?

https://github.com/luben/zstd-jni/commits/v1.4.8-4
- be9be47fae
- be51ebade1
- 44ff8b6f95

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Pass the CIs.

Closes #31585 from dongjoon-hyun/SPARK-ZSTD-1.4.8-4.

Lead-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-18 13:35:49 -08:00
Kousuke Saruta 5167228172 [SPARK-34449][BUILD] Upgrade Jetty to fix CVE-2020-27218
### What changes were proposed in this pull request?

This PR upgrades Jetty from `9.4.34` to `9.4.36`.

### Why are the changes needed?

CVE-2020-27218 affects currently used Jetty 9.4.34.
https://cve.mitre.org/cgi-bin/cvename.cgi?name=CVE-2020-27218

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Modified existing test and new test which comply with the new version of Jetty.

Closes #31574 from sarutak/upgrade-jetty-9.4.36.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-18 18:02:34 +09:00
“attilapiros” 76e5d75e36 [SPARK-33763] Add metrics for better tracking of dynamic allocation
### What changes were proposed in this pull request?

This PR adds the following metrics to track executor remove reasons during dynamic allocation:
-  `numberExecutorsGracefullyDecommissioned`: number of executors which reached the finished decommissioning state and shut itself down cleanly
- `numberExecutorsDecommissionUnfinished`: executors which requested to decommission but they stopped without reaching the finished decommissioning state
- `numberExecutorsKilledByDriver`: executors killed by the driver (requested to stop)
-  `numberExecutorsExitedUnexpectedly`: executors exited without driver request

### Why are the changes needed?

For supporting monitoring of dynamic allocation better with these metrics.

### Does this PR introduce _any_ user-facing change?

Yes. The new metrics will be available for monitoring.

### How was this patch tested?

With unit and integration tests.

Finally manually checked the new metrics in jconsole:
<img width="1054" alt="jmx" src="https://user-images.githubusercontent.com/2017933/107458686-de8adf00-6b54-11eb-86f7-41faf2fb638f.png">

Closes #31450 from attilapiros/SPARK-33763-final.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
2021-02-17 13:44:36 -08:00
“attilapiros” 5f91245cc2 [SPARK-34426][K8S][TESTS] Add driver and executors POD logs to integration tests log when the test fails
### What changes were proposed in this pull request?

This PR introduces a new protected method in `SparkFunSuite` which is only called when the test failed and can be used to collect logs for failed test. By this PR it is implemented in the Kubernetes tests by `KubernetesSuite` class where it collects all the POD logs and logs them out.

This unfortunately cannot be realized with a simple "after" method as in the "after" method the test outcome is not available.

Moreover this PR removes the `appLocator` as a method argument as `appLocator` is available as a member variable.

### Why are the changes needed?

Currently both the driver and executors logs are lost.

In [developer-tools](https://spark.apache.org/developer-tools.html) there is a hint:
"Getting logs from the pods and containers directly is an exercise left to the reader."

But when the test is executed by Jenkins and a failure happened we really need the POD logs to analyze problem.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

By integration testing. I have checked what would happen if one test fails, the output would be:

```
21/02/14 11:05:34.261 ScalaTest-main-running-KubernetesSuite INFO KubernetesSuite:

===== EXTRA LOGS FOR THE FAILED TEST

21/02/14 11:05:34.278 ScalaTest-main-running-KubernetesSuite INFO KubernetesSuite: BEGIN driver POD log
++ id -u
+ myuid=185
++ id -g
+ mygid=0
+ set +e
++ getent passwd 185
+ uidentry=
+ set -e
+ '[' -z '' ']'
+ '[' -w /etc/passwd ']'
+ echo '185185:0:anonymous uid:/opt/spark:/bin/false'
+ SPARK_CLASSPATH=':/opt/spark/jars/*'
+ env
+ grep SPARK_JAVA_OPT_
+ sort -t_ -k4 -n
+ sed 's/[^=]*=\(.*\)/\1/g'
+ readarray -t SPARK_EXECUTOR_JAVA_OPTS
+ '[' -n '' ']'
+ '[' -z ']'
+ '[' -z ']'
+ '[' -n '' ']'
+ '[' -z ']'
+ '[' -z x ']'
+ SPARK_CLASSPATH='/opt/spark/conf::/opt/spark/jars/*'
+ case "$1" in
+ shift 1
+ CMD=("$SPARK_HOME/bin/spark-submit" --conf "spark.driver.bindAddress=$SPARK_DRIVER_BIND_ADDRESS" --deploy-mode client "$")
+ exec /usr/bin/tini -s -- /opt/spark/bin/spark-submit --conf spark.driver.bindAddress=172.17.0.3 --deploy-mode client --properties-file /opt/spark/conf/spark.properties --class org.apache.spark.deploy.PythonRunner local:///opt/spark/tests/decommissioning.py
21/02/14 10:02:28 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Starting decom test
Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties
21/02/14 10:02:29 INFO SparkContext: Running Spark version 3.2.0-SNAPSHOT
21/02/14 10:02:29 INFO ResourceUtils: ==============================================================
21/02/14 10:02:29 INFO ResourceUtils: No custom resources configured for spark.driver.
21/02/14 10:02:29 INFO ResourceUtils: ==============================================================
...
21/02/14 10:03:17 INFO ShutdownHookManager: Deleting directory /var/data/spark-fa6961ed-a2c1-444c-bfeb-20e63ba0b5cf/spark-ab4b0287-6e24-4b39-837e-9b0b62c1f26f
21/02/14 10:03:17 INFO ShutdownHookManager: Deleting directory /tmp/spark-d6b11e7d-6a03-4a1d-8559-37cb853319bf

21/02/14 11:05:34.279 ScalaTest-main-running-KubernetesSuite INFO KubernetesSuite: END driver POD log
```

Closes #31561 from attilapiros/SPARK-34426.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-02-17 05:49:16 +09:00
herman 4fd3247bca [SPARK-34431][CORE] Only load hive-site.xml once
### What changes were proposed in this pull request?
Lazily load Hive's configuration properties from `hive-site.xml` only once.

### Why are the changes needed?
It is expensive to parse the same file over and over.

### Does this PR introduce _any_ user-facing change?
Should not. The changes can improve performance slightly.

### How was this patch tested?
By existing test suites such as `SparkContextSuite`.

Closes #31556 from MaxGekk/load-hive-site-once.

Authored-by: herman <herman@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-15 09:31:51 -08:00
Holden Karau 5248ecb5ab [SPARK-34104][SPARK-34105][CORE][K8S] Maximum decommissioning time & allow decommissioning for excludes
### What changes were proposed in this pull request?

Allow users to have Spark attempt to decommission excluded executors.
Since excluded executors may be flaky, this also adds the ability for users to specify a time limit after which a decommissioning executor will be killed by Spark.

### Why are the changes needed?

This may help prevent fetch failures from excluded executors, and also handle the situation in which executors

### Does this PR introduce _any_ user-facing change?

Yes, two new configuration flags for the behaviour.

### How was this patch tested?

Extended unit and integration tests.

Closes #31539 from holdenk/re=enable-SPARK-34104-SPARK-34105.

Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: Holden Karau <hkarau@apple.com>
2021-02-09 18:16:09 -08:00
HyukjinKwon c8628c943c Revert "[SPARK-34104][SPARK-34105][CORE][K8S] Maximum decommissioning time & allow decommissioning for excludes"
This reverts commit 50641d2e3d.
2021-02-10 08:00:03 +09:00
Holden Karau 50641d2e3d [SPARK-34104][SPARK-34105][CORE][K8S] Maximum decommissioning time & allow decommissioning for excludes
### What changes were proposed in this pull request?

Allow users to have Spark attempt to decommission excluded executors.
Since excluded executors may be flaky, this also adds the ability for users to specify a time limit after which a decommissioning executor will be killed by Spark.

### Why are the changes needed?

This may help prevent fetch failures from excluded executors, and also handle the situation in which executors

### Does this PR introduce _any_ user-facing change?

Yes, two new configuration flags for the behaviour.

### How was this patch tested?

Extended unit and integration tests.

Closes #31249 from holdenk/configure-inaccessibleList-kill-to-use-decommissioning.

Lead-authored-by: Holden Karau <hkarau@apple.com>
Co-authored-by: Holden Karau <holden@pigscanfly.ca>
Signed-off-by: Holden Karau <hkarau@apple.com>
2021-02-09 14:21:24 -08:00
Holden Karau 2b51843ca4 [SPARK-34363][CORE] Add an option for limiting storage for migrated shuffle blocks
### What changes were proposed in this pull request?

Allow users to configure a maximum amount of shuffle blocks to be stored and reject remote shuffle blocks when this threshold is exceeded.

### Why are the changes needed?

In disk constrained environments with large amount of shuffle data, migrations may result in excessive disk pressure on the nodes. On Kube nodes this can result in cascading failures when combined with `emptyDir`.

### Does this PR introduce _any_ user-facing change?

Yes, new configuration parameter.

### How was this patch tested?

New unit tests.

Closes #31493 from holdenk/SPARK-34337-reject-disk-blocks-when-under-disk-pressure.

Lead-authored-by: Holden Karau <hkarau@apple.com>
Co-authored-by: Holden Karau <holden@pigscanfly.ca>
Signed-off-by: Holden Karau <hkarau@apple.com>
2021-02-09 10:21:56 -08:00
Angerszhuuuu 7ea3a336b9 [SPARK-34355][CORE][SQL][FOLLOWUP] Log commit time in all File Writer
### What changes were proposed in this pull request?
When doing https://issues.apache.org/jira/browse/SPARK-34399 based  on https://github.com/apache/spark/pull/31471
Found FileBatchWrite will use `FileFormatWrite.processStates()` too. We need log commit duration  in other writer too.
In this pr:

1. Extract a commit job method in SparkHadoopWriter
2. address other commit writer

### Why are the changes needed?

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
No

Closes #31520 from AngersZhuuuu/SPARK-34355-followup.

Authored-by: Angerszhuuuu <angers.zhu@gmail.com>
Signed-off-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
2021-02-09 16:05:39 +09:00
wyp a1e75edc39 [SPARK-34405][CORE] Fix mean value of timersLabels in the PrometheusServlet class
### What changes were proposed in this pull request?
The getMetricsSnapshot method of the PrometheusServlet class has a wrong value, It should be taking the mean value but it's taking the max value.

### Why are the changes needed?

The mean value of timersLabels in the PrometheusServlet class is wrong, You can look at line 105 of this class: L105.

```
sb.append(s"${prefix}Mean$timersLabels ${snapshot.getMax}\n")
```
it should be
```
sb.append(s"${prefix}Mean$timersLabels ${snapshot.getMean}\n")
```

### Does this PR introduce _any_ user-facing change?

No
### How was this patch tested?

![image](https://user-images.githubusercontent.com/5170878/107313576-cc199280-6acd-11eb-9384-b6abf71c0f90.png)

Closes #31532 from 397090770/SPARK-34405.

Authored-by: wyp <wyphao.2007@163.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-08 21:18:29 -08:00
yangjie01 b344e91368 [SPARK-34375][CORE][K8S][TEST] Replaces 'Mockito.initMocks' with 'Mockito.openMocks'
### What changes were proposed in this pull request?
`Mockito.initMocks(Object)` is a deprecated api, should use `Mockito.openMocks(Object).close()` instead.

### Why are the changes needed?
Cleanup deprecation api usage.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #31487 from LuciferYang/mockito-api.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-08 15:13:00 +09:00
Dongjoon Hyun dcaf62afea [SPARK-34346][CORE][TESTS][FOLLOWUP] Fix UT by removing core-site.xml
### What changes were proposed in this pull request?

This is a follow-up for SPARK-34346 which causes a flakiness due to `core-site.xml` test resource file addition. This PR aims to remove the test resource `core/src/test/resources/core-site.xml` from `core` module.

### Why are the changes needed?

Due to the test resource `core-site.xml`, YARN UT becomes flaky in GitHub Action and Jenkins.
```
$ build/sbt "yarn/testOnly *.YarnClusterSuite -- -z SPARK-16414" -Pyarn
...
[info] YarnClusterSuite:
[info] - yarn-cluster should respect conf overrides in SparkHadoopUtil (SPARK-16414, SPARK-23630) *** FAILED *** (20 seconds, 209 milliseconds)
[info]   FAILED did not equal FINISHED (stdout/stderr was not captured) (BaseYarnClusterSuite.scala:210)
```

To isolate more, we may use `SPARK_TEST_HADOOP_CONF_DIR` like `yarn` module's `yarn/Client`, but it seems an overkill in `core` module.
```
// SPARK-23630: during testing, Spark scripts filter out hadoop conf dirs so that user's
// environments do not interfere with tests. This allows a special env variable during
// tests so that custom conf dirs can be used by unit tests.
val confDirs = Seq("HADOOP_CONF_DIR", "YARN_CONF_DIR") ++
  (if (Utils.isTesting) Seq("SPARK_TEST_HADOOP_CONF_DIR") else Nil)
```

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Pass the CIs.

Closes #31515 from dongjoon-hyun/SPARK-34346-2.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-08 11:32:23 +09:00
Dongjoon Hyun eb5558ed35 [SPARK-34390][CORE] Enable Zstandard buffer pool by default
### What changes were proposed in this pull request?

This PR aims to enable ZStandard JNI BufferPool by default in Apache Spark 3.2.0.

### Why are the changes needed?

**1. SPEED UP**
SPARK-34387 shows the speed-up on both Java8/Java11 by adding [ZStandardBenchmark](https://github.com/apache/spark/blob/master/core/src/test/scala/org/apache/spark/io/ZStandardBenchmark.scala).

**2. MEMORY USAGE**
The followings are the memory usage graphs while running [ZStandardBenchmark](https://github.com/apache/spark/blob/master/core/src/test/scala/org/apache/spark/io/ZStandardBenchmark.scala) on Java11 with the increased N (100000) value in order to visualize easily. In the charts, the first half is the memory consumption without buffer pool while the last last half is one with buffer pool. The difference is noticeable.
```scala
-  val N = 10000
+  val N = 100000
```

- Compression
![Screenshot from 2021-02-06 18-41-17](https://user-images.githubusercontent.com/9700541/107134909-0c4cfb00-68ab-11eb-9273-82cbecdebfba.png)

- Decompression
![Screenshot from 2021-02-06 18-43-05](https://user-images.githubusercontent.com/9700541/107134927-2edf1400-68ab-11eb-97c4-5cd101e91bb0.png)

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Pass the existing UTs.

Closes #31502 from dongjoon-hyun/SPARK-34390.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-06 23:55:17 -08:00
Dongjoon Hyun 466c045bfa [SPARK-34387][CORE][TESTS] Add ZStandardBenchmark
### What changes were proposed in this pull request?

This PR aims to add ZStandardBenchmark as a base-line.

### Why are the changes needed?

This will prevent any regression when we upgrade Zstandard library in the future.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Manually.

Closes #31498 from dongjoon-hyun/SPARK-ZSTD-BENCH.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-06 15:34:40 -08:00
Kent Yao 961c85166a [SPARK-34346][CORE][SQL] io.file.buffer.size set by spark.buffer.size will override by loading hive-site.xml accidentally may cause perf regression
### What changes were proposed in this pull request?

In many real-world cases, when interacting with hive catalog through Spark SQL, users may just share the `hive-site.xml` for their hive jobs and make a copy to `SPARK_HOME`/conf w/o modification. In Spark, when we generate Hadoop configurations, we will use `spark.buffer.size(65536)` to reset `io.file.buffer.size(4096)`. But when we load the hive-site.xml, we may ignore this behavior and reset `io.file.buffer.size` again according to `hive-site.xml`.

1. The configuration priority for setting Hadoop and Hive config here is not right, while literally, the order should be `spark > spark.hive > spark.hadoop > hive > hadoop`

2. This breaks `spark.buffer.size` congfig's behavior for tuning the IO performance w/ HDFS if there is an existing `io.file.buffer.size` in hive-site.xml

### Why are the changes needed?

bugfix for configuration behavior and fix performance regression by that behavior change

### Does this PR introduce _any_ user-facing change?

this pr restores silent user face change

### How was this patch tested?

new tests

Closes #31460 from yaooqinn/SPARK-34346.

Authored-by: Kent Yao <yao@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-05 10:13:19 +09:00
Jungtaek Lim (HeartSaVioR) fbe726f5b1 [SPARK-34339][CORE][SQL] Expose the number of total paths in Utils.buildLocationMetadata()
### What changes were proposed in this pull request?

This PR proposes to expose the number of total paths in Utils.buildLocationMetadata(), with relaxing space usage a bit (around 10+ chars).

Suppose the first 2 of 5 paths are only fit to the threshold, the outputs between the twos are below:

* before the change: `[path1, path2]`
* after the change: `(5 paths)[path1, path2, ...]`

### Why are the changes needed?

SPARK-31793 silently truncates the paths hence end users can't indicate how many paths are truncated, and even more, whether paths are truncated or not.

### Does this PR introduce _any_ user-facing change?

Yes, the location metadata will also show how many paths are truncated (not shown), instead of silently truncated.

### How was this patch tested?

Modified UTs

Closes #31464 from HeartSaVioR/SPARK-34339.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-05 09:37:38 +09:00
Jungtaek Lim (HeartSaVioR) 44dcf0062c [SPARK-34326][CORE][SQL] Fix UTs added in SPARK-31793 depending on the length of temp path
### What changes were proposed in this pull request?

This PR proposes to fix the UTs being added in SPARK-31793, so that all things contributing the length limit are properly accounted.

### Why are the changes needed?

The test `DataSourceScanExecRedactionSuite.SPARK-31793: FileSourceScanExec metadata should contain limited file paths` is failing conditionally, depending on the length of the temp directory.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Modified UTs explain the missing points, which also do the test.

Closes #31449 from HeartSaVioR/SPARK-34326-v2.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-02-04 08:46:11 +09:00
Dongjoon Hyun 8e28218106 [SPARK-34340][CORE] Support ZSTD JNI BufferPool
### What changes were proposed in this pull request?

This PR aims two goals.
1. Support ZSTD JNI BufferPool feature by adding a new configuration, `spark.io.compression.zstd.bufferPool.enabled`, for Apache Spark 3.2.0.
2. Make Spark independent from ZSTD JNI library's default buffer pool policy change.

### Why are the changes needed?

ZSTD JNI library has different behaviors across its versions.

| Version | Description | Commit |
| ---------- | --------------- | ----------- |
| v1.4.5-7 | `BufferPool` was added and used it by default | 4f55c89172 |
| v1.4.5-8 | `RecyclingBufferPool` was added and `BufferPool` became an interface to allow custom BufferPool implementation | dd2588edd3 |
| v1.4.7+ | `NoPool` is used by default and user should specify buffer pool explicitly | f7c8279bc1 |

### Does this PR introduce _any_ user-facing change?

No, the default value (`false`) is consistent with the AS-IS ZSTD-JNI library's default buffer pool.

### How was this patch tested?

Pass the CIs with the updated UT.

Closes #31453 from dongjoon-hyun/SPARK-34340.

Lead-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-02-03 09:11:20 -08:00
Erik Krogen 791de00ddf [SPARK-34182][AVRO] Improve error messages when matching Catalyst-to-Avro schemas
### What changes were proposed in this pull request?
Improve the error messages for incompatibilities between Avro and Catalyst schemas. First, make `AvroSerializer` more similar to `AvroDeserializer` in printing out contextual information such as hierarchical field names. Standardize exception messages in both serializer and deserializer to always include such contextual information, and include a top-level exception which shows the full schemas which were being parsed when the incompatibility was found. Both now print out the hierarchical name for both the Avro and Catalyst fields, since they may be different due to case sensitivity and Avro union handling.

### Why are the changes needed?
The error messages in this type of failure scenario are very lacking in information on the write path (`AvroSerializer`). Below are two examples of messages that provide insufficient information to determine what went wrong (lacking in field names, context about the overall schema structure, etc.).
```
org.apache.spark.sql.avro.IncompatibleSchemaException: Cannot convert Catalyst type IntegerType to Avro type "float".

org.apache.spark.sql.avro.IncompatibleSchemaException: Cannot convert Catalyst type StructType(StructField(bar,IntegerType,true)) to Avro type {"type":"record","name":"test","fields":[{"name":"NOTbar","type":["null","int"],"default":null}]}.
```
The error messages currently existing in `AvroDeserializer` are much better, but still not very internally consistent, and it would be better if they were consistent with the newly added exception messages in `AvroSerializer`.

### Does this PR introduce _any_ user-facing change?
Error messages when there are incompatibilities between Avro and Catalyst schemas will be greatly improved on when writing Avro data using the `avroSchema` option, a little bit improved when reading Avro data, and much more consistent between the two.

Below is an example of a new message. See `AvroSerdeSuite` for more examples.
```
org.apache.spark.sql.avro.IncompatibleSchemaException: Cannot convert Catalyst type STRUCT<`foo`: STRUCT<`bar`: INT>> to Avro type {"type":"record","name":"top","fields":[{"name":"foo","type":"int"}]}
	at org.apache.spark.sql.avro.AvroSerializer.liftedTree1$1(AvroSerializer.scala:83)
...
Caused by: org.apache.spark.sql.avro.IncompatibleSchemaException: Cannot convert Catalyst field 'foo' to Avro field 'foo' because schema is incompatible (sqlType = STRUCT<`bar`: INT>, avroType = "int")
	at org.apache.spark.sql.avro.AvroSerializer.newConverter(AvroSerializer.scala:230)
...
```

### How was this patch tested?
New unit test suite, `AvroSerdeSuite`, was added to test corner cases on `AvroSerializer` and `AvroDeserializer` and verify that the exception messages are as expected. Existing tests in `AvroSuite` also continue to pass, with modifications in places where assertions were made about the exceptions that would be thrown.

Closes #31333 from xkrogen/xkrogen-SPARK-34182-avro-serde-errormessages.

Authored-by: Erik Krogen <xkrogen@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-03 08:49:36 +00:00
HyukjinKwon e927bf90e0 Revert "[SPARK-34326][CORE][SQL] Fix UTs added in SPARK-31793 depending on the length of temp path"
This reverts commit 63866025d2.
2021-02-03 12:32:39 +09:00
offthewall123 60c71c6d2d [SPARK-34325][CORE] Remove unused shuffleBlockResolver variable inSortShuffleWriter
### What changes were proposed in this pull request?
Remove unused shuffleBlockResolver variable in SortShuffleWriter.

### Why are the changes needed?
For better code understanding.

### Does this PR introduce _any_ user-facing change?
NO.

### How was this patch tested?
End to End.

Closes #31433 from offthewall123/remove_shuffleBlockResolver_in_SortShuffleWriter.

Authored-by: offthewall123 <dingyu.xu@intel.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-02-03 09:40:08 +09:00
Jungtaek Lim (HeartSaVioR) 63866025d2 [SPARK-34326][CORE][SQL] Fix UTs added in SPARK-31793 depending on the length of temp path
### What changes were proposed in this pull request?

This PR proposes to fix the UTs being added in SPARK-31793, so that all things contributing the length limit are properly accounted.

### Why are the changes needed?

The test `DataSourceScanExecRedactionSuite.SPARK-31793: FileSourceScanExec metadata should contain limited file paths` is failing conditionally, depending on the length of the temp directory.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Modified UTs explain the missing points, which also do the test.

Closes #31435 from HeartSaVioR/SPARK-34326.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
2021-02-03 07:35:22 +09:00
yangjie01 9db566a882 [SPARK-34310][CORE][SQL] Replaces map and flatten with flatMap
### What changes were proposed in this pull request?
Replaces `collection.map(f1).flatten(f2)` with `collection.flatMap` if possible. it's semantically consistent, but looks simpler.

### Why are the changes needed?
Code Simpilefications.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #31416 from LuciferYang/SPARK-34310.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-02-01 08:21:35 -06:00
neko bd9eeebce0 [SPARK-34288][WEBUI] Add a tip info for the resources column in the executors page
### What changes were proposed in this pull request?
This is  an extension of PR #25613, I try to add a  tip info for `Resource` column to make it easier for users to know what the column actually means and avoid confusion.

### Why are the changes needed?
After upgrading from 2.3.2 to 3.0.1, the new `Resources` column in the executors page is always blank because it does not use GPU/FPGA,
and there is no tip info, so users are often confused when they do not know the exact meaning of this column.

### Does this PR introduce _any_ user-facing change?
add a tip info in the executors page.

### How was this patch tested?
 manual test  works well as below:
![fixed](https://user-images.githubusercontent.com/52202080/106248350-d032ee00-624b-11eb-9ae4-92319ed11110.png)

Closes #31392 from akiyamaneko/executors-resources-tips.

Authored-by: neko <echohlne@gmail.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2021-01-30 10:23:52 +08:00
yangjie01 2e192b6f45 [SPARK-34284][CORE][TESTS] Fix deprecated API usage of Apache commons-io
### What changes were proposed in this pull request?
There are some deprecated API usage compilation warning related to Apache commons-io as follows:

 ```
[WARNING] [Warn] /spark/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala:1109: [deprecation  org.apache.spark.deploy.SparkSubmitSuite.checkDownloadedFile.$org_scalatest_assert_macro_expr.$org_scalatest_assert_macro_left | origin=org.apache.commons.io.FileUtils.readFileToString | version=] method readFileToString in class FileUtils is deprecated
[WARNING] [Warn] /spark/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala:1110: [deprecation  org.apache.spark.deploy.SparkSubmitSuite.checkDownloadedFile.$org_scalatest_assert_macro_expr.$org_scalatest_assert_macro_right | origin=org.apache.commons.io.FileUtils.readFileToString | version=] method readFileToString in class FileUtils is deprecated
[WARNING] [Warn] /spark/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala:1152: [deprecation  org.apache.spark.deploy.SparkSubmitSuite | origin=org.apache.commons.io.FileUtils.write | version=] method write in class FileUtils is deprecated
[WARNING] [Warn] /spark/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala:1167: [deprecation  org.apache.spark.deploy.SparkSubmitSuite | origin=org.apache.commons.io.FileUtils.write | version=] method write in class FileUtils is deprecated
[WARNING] [Warn] /spark/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala:201: [deprecation  org.apache.spark.deploy.history.HistoryServerSuite.<local HistoryServerSuite>.$anonfun.exp | origin=org.apache.commons.io.IOUtils.toString | version=] method toString in class IOUtils is deprecated
[WARNING] [Warn] /spark/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala:716: [deprecation  org.apache.spark.deploy.history.HistoryServerSuite.getContentAndCode.inString.$anonfun | origin=org.apache.commons.io.IOUtils.toString | version=] method toString in class IOUtils is deprecated
[WARNING] [Warn] /spark/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala:732: [deprecation  org.apache.spark.deploy.history.HistoryServerSuite.connectAndGetInputStream.errString.$anonfun | origin=org.apache.commons.io.IOUtils.toString | version=] method toString in class IOUtils is deprecated
[WARNING] [Warn] /spark/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala:267: [deprecation  org.apache.spark.streaming.InputStreamsSuite.<local InputStreamsSuite>.$anonfun.$anonfun.write | origin=org.apache.commons.io.IOUtils.write | version=] method write in class IOUtils is deprecated
[WARNING] [Warn] /spark/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala:912: [deprecation  org.apache.spark.streaming.StreamingContextSuite.createCorruptedCheckpoint | origin=org.apache.commons.io.FileUtils.write | version=] method write in class FileUtils is deprecated
```

The main API change is to need to add a `java.nio.charset.Charset` parameter when the corresponding method is called, so the main change of is pr is add a `StandardCharsets.UTF_8` parameter to the these method.

### Why are the changes needed?
Fix deprecated API usage of Apache commons-io.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #31389 from LuciferYang/SPARK-34284.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-29 17:50:14 +09:00
Dongjoon Hyun bc41c5a0e5 [SPARK-34273][CORE] Do not reregister BlockManager when SparkContext is stopped
### What changes were proposed in this pull request?

This PR aims to prevent `HeartbeatReceiver` asks `Executor` to re-register blocker manager when the SparkContext is already stopped.

### Why are the changes needed?

Currently, `HeartbeatReceiver` blindly asks re-registration for the new heartbeat message.
However, when SparkContext is stopped, we don't need to re-register new block manager.
Re-registration causes unnecessary executors' logs and and a delay on job termination.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Pass the CIs with the newly added test case.

Closes #31373 from dongjoon-hyun/SPARK-34273.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-28 13:06:42 -08:00
Dongjoon Hyun 23d4f6b393 [SPARK-34278][CORE] Make BlockManagerMaster driver heartbeat timeout configurable
### What changes were proposed in this pull request?

This PR adds a new configuration, `spark.storage.blockManagerMasterDriverHeartbeatTimeoutMs`.

### Why are the changes needed?

Currently, it's a hard-coded `10 minutes`.

### Does this PR introduce _any_ user-facing change?

No. The default value is the same.

### How was this patch tested?

Pass the CIs.

Closes #31383 from dongjoon-hyun/SPARK-34278.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-28 09:23:40 -08:00
yangjie01 15445a8d9e [SPARK-34275][CORE][SQL][MLLIB] Replaces filter and size with count
### What changes were proposed in this pull request?
Use `count` to simplify `find + size(or length)` operation, it's semantically consistent, but looks simpler.

**Before**
```
seq.filter(p).size
```

**After**
```
seq.count(p)
```

### Why are the changes needed?
Code Simpilefications.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #31374 from LuciferYang/SPARK-34275.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-28 15:27:07 +09:00
Holden Karau 9d83d62f14 [SPARK-34193][CORE] TorrentBroadcast block manager decommissioning race fix
### What changes were proposed in this pull request?

Allow broadcast blocks to be put during decommissioning since migrations don't apply to them and they may be stored as part of job exec.

### Why are the changes needed?

Potential race condition.

### Does this PR introduce _any_ user-facing change?

Removal of race condition.

### How was this patch tested?

New unit test.

Closes #31298 from holdenk/SPARK-34193-torrentbroadcast-blockmanager-decommissioning-potential-race-condition.

Authored-by: Holden Karau <hkarau@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-28 06:15:35 +09:00
neko f1bc37e624 [SPARK-34221][WEBUI] Ensure if a stage fails in the UI page, the corresponding error message can be displayed correctly
### What changes were proposed in this pull request?
Ensure that if a stage fails in the UI page, the corresponding error message can be displayed correctly.

### Why are the changes needed?
errormessage is not handled properly in JavaScript. If the 'at' is not exist, the error message on the page will be blank.
I made wochanges,
1. `msg.indexOf("at")` => `msg.indexOf("\n")`

![image](https://user-images.githubusercontent.com/52202080/105663531-7362cb00-5f0d-11eb-87fd-008ed65c33ca.png)

  As shows ablove, truncated at the 'at' position will result in a strange abstract of the error message. If there is a `\n` worit is more reasonable to truncate at the '\n' position.

2. If the `\n` does not exist check whether the msg  is more than 100. If true, then truncate the display to avoid too long error message

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Manual test shows as belows, just a js change:

before modified:
![problem](https://user-images.githubusercontent.com/52202080/105712153-661cff00-5f54-11eb-80bf-e33c323c4e55.png)

after modified
![after mdified](https://user-images.githubusercontent.com/52202080/105712180-6c12e000-5f54-11eb-8998-ff8bc8a0a503.png)

Closes #31314 from akiyamaneko/error_message_display_empty.

Authored-by: neko <echohlne@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-27 10:01:57 -08:00
Chao Sun abf7e81712 [SPARK-33212][FOLLOW-UP][BUILD] Bring back duplicate dependency check and add more strict Hadoop version check
### What changes were proposed in this pull request?

1. Add back Maven enforcer for duplicate dependencies check
2. More strict check on Hadoop versions which support shaded client in `IsolatedClientLoader`. To do proper version check, this adds a util function `majorMinorPatchVersion` to extract major/minor/patch version from a string.
3. Cleanup unnecessary code

### Why are the changes needed?

The Maven enforcer was removed as part of #30556. This proposes to add it back.

Also, Hadoop shaded client doesn't work in certain cases (see [these comments](https://github.com/apache/spark/pull/30701#discussion_r558522227) for details). This strictly checks that the current Hadoop version (i.e., 3.2.2 at the moment) has good support of shaded client or otherwise fallback to old unshaded ones.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Existing tests.

Closes #31203 from sunchao/SPARK-33212-followup.

Lead-authored-by: Chao Sun <sunchao@apple.com>
Co-authored-by: Chao Sun <sunchao@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-26 15:34:55 -08:00
yangjie01 8999e8805d [SPARK-34224][CORE][SQL][SS][DSTREAM][YARN][TEST][EXAMPLES] Ensure all resource opened by Source.fromXXX are closed
### What changes were proposed in this pull request?
Using a function like `.mkString` or `.getLines` directly on a `scala.io.Source` opened by `fromFile`, `fromURL`, `fromURI ` will leak the underlying file handle,  this pr use the `Utils.tryWithResource` method wrap the `BufferedSource` to ensure these `BufferedSource` closed.

### Why are the changes needed?
Avoid file handle leak.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #31323 from LuciferYang/source-not-closed.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-26 19:06:37 +09:00
Warren Zhu 68b765e6b8 [SPARK-34232][CORE] Redact SparkListenerEnvironmentUpdate event in log
### What changes were proposed in this pull request?
Redact event SparkListenerEnvironmentUpdate in log when its processing time exceeded logSlowEventThreshold

### Why are the changes needed?
Credentials could be exposed when its processing time exceeded logSlowEventThreshold

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Manually tested

Closes #31335 from warrenzhu25/34232.

Authored-by: Warren Zhu <warren.zhu25@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-26 17:10:53 +09:00
Yuanjian Li 59cbacaddf [SPARK-34185][DOCS] Review and fix issues in API docs
### What changes were proposed in this pull request?
Compare the 3.1.1 API doc with the latest release version 3.0.1. Fix the following issues:
- Add missing `Since` annotation for new APIs
- Remove the leaking class/object in API doc

### Why are the changes needed?
Fix the issues in the Spark 3.1.1 release API docs.

### Does this PR introduce _any_ user-facing change?
Yes, API doc changes.

### How was this patch tested?
Manually test.

Closes #31271 from xuanyuanking/SPARK-34185.

Lead-authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-25 11:38:20 +09:00
Ismaël Mejía e9e81f798f [SPARK-27733][CORE] Upgrade Avro to version 1.10.1
### What changes were proposed in this pull request?

Update Avro dependency to version 1.10.1

### Why are the changes needed?

To catch up multiple improvements of Avro as well as fix security issues on transitive dependencies.

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Since there were no API changes required we just run the tests

Closes #31232 from iemejia/SPARK-27733-avro-upgrade.

Authored-by: Ismaël Mejía <iemejia@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-20 15:42:27 -08:00
ulysses-you 2145f07e22 [SPARK-34166][CORE][TESTS] Fix flaky test in DecommissionWorkerSuite
### What changes were proposed in this pull request?

Add executor number check.

### Why are the changes needed?

The test `decommission workers ensure that shuffle output is regenerated even with shuffle service` assumes it has two executor and both of two tasks can execute concurrently.

The two tasks will execute serially if there only one executor. The result is test is unexpceted. E.g.

```
[info] 5 did not equal 4 Expected 4 tasks but got List(0:0:0:0-SUCCESS, 0:0:1:0-FAILED, 0:0:1:1-SUCCESS, 0:1:0:0-SUCCESS, 1:0:0:0-SUCCESS) (DecommissionWorkerSuite.scala:190)
```
The failed task due to the first task finished and decommission the worker.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Pass exists test.

Closes #31255 from ulysses-you/SPARK-34166.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-19 21:26:24 -08:00
Norbert Schultz c3d8352ca1 [SPARK-34115][CORE] Check SPARK_TESTING as lazy val to avoid slowdown
### What changes were proposed in this pull request?
Check SPARK_TESTING as lazy val to avoid slow down when there are many environment variables

### Why are the changes needed?
If there are many environment variables, sys.env slows is very slow. As Utils.isTesting is called very often during Dataframe-Optimization, this can slow down evaluation very much.

An example for triggering the problem can be found in the bug ticket https://issues.apache.org/jira/browse/SPARK-34115

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
With the example provided in the ticket.

Closes #31244 from nob13/bug/34115.

Lead-authored-by: Norbert Schultz <norbert.schultz@reactivecore.de>
Co-authored-by: Norbert Schultz <noschultz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-20 09:39:13 +09:00
Dongjoon Hyun 415506cc04 [SPARK-34142][CORE] Support Fallback Storage Cleanup during stopping SparkContext
### What changes were proposed in this pull request?

This PR aims to support fallback storage clean-up during stopping `SparkContext`.

### Why are the changes needed?

SPARK-33545 added `Support Fallback Storage during worker decommission` for the managed cloud-storages with TTL support.  Usually, it's one day. This PR will add an additional clean-up feature during stopping `SparkContext` in order to save some money before TTL or the other HDFS-compatible storage which doesn't have TTL support.

### Does this PR introduce _any_ user-facing change?

Yes, but this is a new feature.

### How was this patch tested?

Pass the newly added UT.

Closes #31215 from dongjoon-hyun/SPARK-34142.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-17 16:54:01 -08:00
mohan3d ebd8bc934d [SPARK-34123][WEB UI] optimize spark history summary page loading
### What changes were proposed in this pull request?
Display history server entries using datatables instead of Mustache + Datatables which proved to be faster and non-blocking for the webpage while searching (using search bar in the page)

### Why are the changes needed?
Small changes in the attempts (entries) and removed part of HTML (Mustache template).

### Does this PR introduce _any_ user-facing change?
Not very sure, but it's not supposed to change the way the page looks rather it changes how entries are displayed.

### How was this patch tested?
Running test, since it's not adding new functionality.

Closes #31191 from mohan3d/feat/history-server-ui-optimization.

Lead-authored-by: mohan3d <mohan3d94@gmail.com>
Co-authored-by: Author: mohan3d <mohan3d94@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-01-17 14:37:28 -06:00
Chao Sun b6f46ca297 [SPARK-33212][BUILD] Upgrade to Hadoop 3.2.2 and move to shaded clients for Hadoop 3.x profile
### What changes were proposed in this pull request?

This:
1. switches Spark to use shaded Hadoop clients, namely hadoop-client-api and hadoop-client-runtime, for Hadoop 3.x.
2. upgrade built-in version for Hadoop 3.x to Hadoop 3.2.2

Note that for Hadoop 2.7, we'll still use the same modules such as hadoop-client.

In order to still keep default Hadoop profile to be hadoop-3.2, this defines the following Maven properties:

```
hadoop-client-api.artifact
hadoop-client-runtime.artifact
hadoop-client-minicluster.artifact
```

which default to:
```
hadoop-client-api
hadoop-client-runtime
hadoop-client-minicluster
```
but all switch to `hadoop-client` when the Hadoop profile is hadoop-2.7. A side affect from this is we'll import the same dependency multiple times. For this I have to disable Maven enforcer `banDuplicatePomDependencyVersions`.

Besides above, there are the following changes:
- explicitly add a few dependencies which are imported via transitive dependencies from Hadoop jars, but are removed from the shaded client jars.
- removed the use of `ProxyUriUtils.getPath` from `ApplicationMaster` which is a server-side/private API.
- modified `IsolatedClientLoader` to exclude `hadoop-auth` jars when Hadoop version is 3.x. This change should only matter when we're not sharing Hadoop classes with Spark (which is _mostly_ used in tests).

### Why are the changes needed?

Hadoop 3.2.2 is released with new features and bug fixes, so it's good for the Spark community to adopt it. However, latest Hadoop versions starting from Hadoop 3.2.1 have upgraded to use Guava 27+. In order to resolve Guava conflicts, this takes the approach by switching to shaded client jars provided by Hadoop. This also has the benefits of avoid pulling other 3rd party dependencies from Hadoop side so as to avoid more potential future conflicts.

### Does this PR introduce _any_ user-facing change?

When people use Spark with `hadoop-provided` option, they should make sure class path contains `hadoop-client-api` and `hadoop-client-runtime` jars. In addition, they may need to make sure these jars appear before other Hadoop jars in the order. Otherwise, classes may be loaded from the other non-shaded Hadoop jars and cause potential conflicts.

### How was this patch tested?

Relying on existing tests.

Closes #30701 from sunchao/test-hadoop-3.2.2.

Authored-by: Chao Sun <sunchao@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2021-01-15 14:06:50 -08:00
KevinSmile c75c29dcaa [SPARK-32598][SCHEDULER] Fix missing driver logs under UI App-Executors tab in standalone cluster mode
### What changes were proposed in this pull request?
Fix  [SPARK-32598] (missing driver logs under UI-ApplicationDetails-Executors tab in standalone cluster mode) .

The direct bug is: the original author forgot to implement `getDriverLogUrls` in `StandaloneSchedulerBackend`

1de272f98d/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala (L70-L75)

So we set DriverLogUrls as env in `DriverRunner`, and retrieve it at `StandaloneSchedulerBackend`.

### Why are the changes needed?
Fix bug  [SPARK-32598].

### Does this PR introduce _any_ user-facing change?
Yes. User will see driver logs (standalone cluster mode) under UI-ApplicationDetails-Executors tab now.

Before:
![image](https://user-images.githubusercontent.com/17903517/93901055-b5de8600-fd28-11ea-879a-d97e6f70cc6e.png)

After:
![image](https://user-images.githubusercontent.com/17903517/93901080-baa33a00-fd28-11ea-8895-3787c5efbf88.png)

### How was this patch tested?
Re-check the real case in [SPARK-32598] and found this user-facing bug fixed.

Closes #29644 from KevinSmile/kw-dev-master.

Authored-by: KevinSmile <kevinwang013@hotmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-01-15 09:01:26 -06:00
yangjie01 9e33d49b5b [SPARK-33346][CORE][SQL][MLLIB][DSTREAM][K8S] Change the never changed 'var' to 'val'
### What changes were proposed in this pull request?
Some local variables are declared as `var`, but they are never reassigned and should be declared as `val`, so this pr turn these  from `var` to  `val` except for `mockito` related cases.

### Why are the changes needed?
Use `val` instead of `var` when possible.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #31142 from LuciferYang/SPARK-33346.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-01-15 08:47:02 -06:00
yangjie01 8ed23ed499 [SPARK-34118][CORE][SQL] Replaces filter and check for emptiness with exists or forall
### What changes were proposed in this pull request?
This pr use `exists` or `forall` to simplify `filter + emptiness check`, it's semantically consistent, but looks simpler. The rule as follow:

- `seq.filter(p).size == 0)` -> `!seq.exists(p)`
- `seq.filter(p).length > 0` -> `seq.exists(p)`
- `seq.filterNot(p).isEmpty` -> `seq.forall(p)`
- `seq.filterNot(p).nonEmpty` -> `!seq.forall(p)`

### Why are the changes needed?
Code Simpilefications.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #31184 from LuciferYang/SPARK-34118.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-15 12:12:33 +09:00
Gengliang Wang 467d758973 [SPARK-34075][SQL][CORE] Hidden directories are being listed for partition inference
### What changes were proposed in this pull request?

Fix a regression from https://github.com/apache/spark/pull/29959.

In Spark, the following file paths are considered as hidden paths and they are ignored on file reads:
1. starts with "_" and doesn't contain "="
2. starts with "."

However, after the refactoring PR https://github.com/apache/spark/pull/29959, the hidden paths are not filtered out on partition inference: https://github.com/apache/spark/pull/29959/files#r556432426

This PR is to fix the bug. To archive the goal, the method `InMemoryFileIndex.shouldFilterOut` is refactored as `HadoopFSUtils.shouldFilterOutPathName`

### Why are the changes needed?

Bugfix

### Does this PR introduce _any_ user-facing change?

Yes, it fixes a bug for reading file paths with partitions.

### How was this patch tested?

Unit test

Closes #31169 from gengliangwang/fileListingBug.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-14 09:39:38 +09:00
yangjie01 8b1ba233f1 [SPARK-34068][CORE][SQL][MLLIB][GRAPHX] Remove redundant collection conversion
### What changes were proposed in this pull request?
There are some redundant collection conversion can be removed, for version compatibility, clean up these with Scala-2.13 profile.

### Why are the changes needed?
Remove redundant collection conversion

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
- Pass the Jenkins or GitHub  Action
- Manual test `core`, `graphx`, `mllib`, `mllib-local`, `sql`, `yarn`,`kafka-0-10` in Scala 2.13 passed

Closes #31125 from LuciferYang/SPARK-34068.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-01-13 18:07:02 -06:00
yangjie01 8c5fecda73 [SPARK-34070][CORE][SQL] Replaces find and emptiness check with exists
### What changes were proposed in this pull request?
This pr use `exists` to simplify `find + emptiness check`, it's semantically consistent, but looks simpler.

**Before**

```
seq.find(p).isDefined

or

seq.find(p).isEmpty
```

**After**

```
seq.exists(p)

or

!seq.exists(p)
```
### Why are the changes needed?
Code Simpilefications.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Pass the Jenkins or GitHub Action

Closes #31130 from LuciferYang/SPARK-34070.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2021-01-13 10:42:24 -06:00
schintap bd5039fc35 [SPARK-33741][CORE] Add min threshold time speculation config
### What changes were proposed in this pull request?
Add min threshold time speculation config

### Why are the changes needed?
When we turn on speculation with default configs we have the last 10% of the tasks subject to speculation. There are a lot of stages where the stage runs for few seconds to minutes. Also in general we don't want to speculate tasks that run within a minimum threshold. By setting a minimum threshold for speculation config gives us better control for speculative tasks

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Unit test

Closes #30710 from redsanket/SPARK-33741.

Lead-authored-by: schintap <schintap@verizonmedia.com>
Co-authored-by: Sanket Chintapalli <chintapalli.sanketreddy@gmail.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2021-01-13 08:57:56 -06:00
ulysses-you f64297d290 [SPARK-32850][TEST][FOLLOWUP] Fix flaky test due to timeout
### What changes were proposed in this pull request?

Increase test timeout.

### Why are the changes needed?

It's more reasonable to use 60s instead of 6s since many code place use the 60s in `TestUtils.waitUntilExecutorsUp`.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Pass exists test.

Closes #31166 from ulysses-you/SPARK-32850-FOLLOWUP.

Authored-by: ulysses-you <ulyssesyou18@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2021-01-13 19:21:44 +09:00