Commit graph

7089 commits

Author SHA1 Message Date
Gabor Somogyi 2f55809425 [SPARK-27294][SS] Add multi-cluster Kafka delegation token
## What changes were proposed in this pull request?

The actual implementation doesn't support multi-cluster Kafka connection with delegation token. In this PR I've added this functionality.

What this PR contains:
* New way of configuration
* Multiple delegation token obtain/store/use functionality
* Documentation
* The change works on DStreams also

## How was this patch tested?

Existing + additional unit tests.
Additionally tested on cluster.

Test scenario:

* 2 * 4 node clusters
* The 4-4 nodes are in different kerberos realms
* Cross-Realm trust between the 2 realms
* Yarn
* Kafka broker version 2.1.0
* security.protocol = SASL_SSL
* sasl.mechanism = SCRAM-SHA-512
* Artificial exceptions during processing
* Source reads from realm1 sink writes to realm2

Kafka broker settings:

* delegation.token.expiry.time.ms=600000 (10 min)
* delegation.token.max.lifetime.ms=1200000 (20 min)
* delegation.token.expiry.check.interval.ms=300000 (5 min)

Closes #24305 from gaborgsomogyi/SPARK-27294.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-05-07 11:40:43 -07:00
Adi Muraru 8ef4da753d [SPARK-27610][YARN] Shade netty native libraries
## What changes were proposed in this pull request?

Fixed the `spark-<version>-yarn-shuffle.jar` artifact packaging to shade the native netty libraries:
- shade the `META-INF/native/libnetty_*` native libraries when packagin
the yarn shuffle service jar. This is required as netty library loader
derives that based on shaded package name.
- updated the `org/spark_project` shade package prefix to `org/sparkproject`
(i.e. removed underscore) as the former breaks the netty native lib loading.

This was causing the yarn external shuffle service to fail
when spark.shuffle.io.mode=EPOLL

## How was this patch tested?
Manual tests

Closes #24502 from amuraru/SPARK-27610_master.

Authored-by: Adi Muraru <amuraru@adobe.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-05-07 10:47:36 -07:00
Wenchen Fan d124ce9c7e [SPARK-27590][CORE] do not consider skipped tasks when scheduling speculative tasks
## What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/24375

When `TaskSetManager` skips a task because its corresponding partition is already completed by other `TaskSetManager`s, we should not consider the duration of the task that is finished by other `TaskSetManager`s to schedule the speculative tasks of this `TaskSetManager`.

## How was this patch tested?

updated test case

Closes #24485 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2019-05-07 12:02:08 -05:00
Liang-Chi Hsieh d9bcacf94b [SPARK-27629][PYSPARK] Prevent Unpickler from intervening each unpickling
## What changes were proposed in this pull request?

In SPARK-27612, one correctness issue was reported. When protocol 4 is used to pickle Python objects, we found that unpickled objects were wrong. A temporary fix was proposed by not using highest protocol.

It was found that Opcodes.MEMOIZE was appeared in the opcodes in protocol 4. It is suspect to this issue.

A deeper dive found that Opcodes.MEMOIZE stores objects into internal map of Unpickler object. We use single Unpickler object to unpickle serialized Python bytes. Stored objects intervenes next round of unpickling, if the map is not cleared.

We has two options:

1. Continues to reuse Unpickler, but calls its close after each unpickling.
2. Not to reuse Unpickler and create new Unpickler object in each unpickling.

This patch takes option 1.

## How was this patch tested?

Passing the test added in SPARK-27612 (#24519).

Closes #24521 from viirya/SPARK-27629.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-04 13:21:08 +09:00
wuyi 51de86baed [SPARK-27510][CORE] Avoid Master falls into dead loop while launching executor failed in Worker
## What changes were proposed in this pull request?

This is a long standing issue which I met before and I've seen other people got trouble with it:
[test cases stuck on "local-cluster mode" of ReplSuite?](http://apache-spark-developers-list.1001551.n3.nabble.com/test-cases-stuck-on-quot-local-cluster-mode-quot-of-ReplSuite-td3086.html)
[Spark tests hang on local machine due to "testGuavaOptional" in JavaAPISuite](http://apache-spark-developers-list.1001551.n3.nabble.com/Spark-tests-hang-on-local-machine-due-to-quot-testGuavaOptional-quot-in-JavaAPISuite-tc10999.html)

When running test under local-cluster mode with wrong SPARK_HOME(spark.test.home), test just get stuck and no response forever. After looking into SPARK_WORKER_DIR, I found there's endless executor directories under it. So, this explains what happens during test getting stuck.

The whole process looks like:

1. Driver submits an app to Master and asks for N executors
2. Master inits executor state with LAUNCHING and sends `LaunchExecutor` to Worker
3. Worker receives `LaunchExecutor`, launches ExecutorRunner asynchronously and sends `ExecutorStateChanged(state=RUNNING)` to Mater immediately
4. Master receives `ExecutorStateChanged(state=RUNNING)` and reset `_retyCount` to 0.
5. ExecutorRunner throws exception during executor launching, sends `ExecutorStateChanged(state=FAILED)` to Worker, Worker forwards the msg to Master
6. Master receives `ExecutorStateChanged(state=FAILED)`. Since Master always reset `_retyCount` when it receives RUNNING msg, so, event if a Worker fails to launch executor for continuous many times, ` _retryCount` would never exceed `maxExecutorRetries`. So, Master continue to launch executor and fall into the dead loop.

The problem exists in step 3. Worker sends `ExecutorStateChanged(state=RUNNING)` to Master immediately while executor is still launching. And, when Master receive that msg, it believes the executor has launched successfully, and reset `_retryCount` subsequently. However, that's not true.

This pr suggests to remove step 3 and requires Worker only send `ExecutorStateChanged(state=RUNNING)` after executor has really launched successfully.

## How was this patch tested?

Tested Manually.

Closes #24408 from Ngone51/fix-dead-loop.

Authored-by: wuyi <ngone_5451@163.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2019-05-03 15:48:37 -07:00
Xingbo Jiang f950e53930 [MINOR][CORE] Update taskName in PythonRunner
## What changes were proposed in this pull request?

Update taskName in PythonRunner so it keeps align with that in Executor.

## How was this patch tested?

N/A

Closes #24510 from jiangxb1987/pylog.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2019-05-02 15:37:25 -07:00
Jungtaek Lim (HeartSaVioR) 75b40a53d3 [SPARK-27575][CORE][YARN] Yarn file-related confs should merge new value with existing value
## What changes were proposed in this pull request?

This patch fixes a bug which YARN file-related configurations are being overwritten when there're some values to assign - e.g. if `--file` is specified as an argument, `spark.yarn.dist.files` is overwritten with the value of argument. After this patch the existing value and new value will be merged before assigning to the value of configuration.

## How was this patch tested?

Added UT, and manually tested with below command:

> ./bin/spark-submit --verbose --files /etc/spark2/conf/spark-defaults.conf.template --master yarn-cluster --class org.apache.spark.examples.SparkPi examples/jars/spark-examples_2.11-2.4.0.jar 10

where the spark conf file has

`spark.yarn.dist.files=file:/etc/spark2/conf/atlas-application.properties.yarn#atlas-application.properties`

```
Spark config:
...
(spark.yarn.dist.files,file:/etc/spark2/conf/atlas-application.properties.yarn#atlas-application.properties,file:///etc/spark2/conf/spark-defaults.conf.template)
...
```

Closes #24465 from HeartSaVioR/SPARK-27575.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-04-29 10:14:59 -07:00
Gabor Somogyi fb6b19ab7c [SPARK-23014][SS] Fully remove V1 memory sink.
## What changes were proposed in this pull request?

There is a MemorySink v2 already so v1 can be removed. In this PR I've removed it completely.
What this PR contains:
* V1 memory sink removal
* V2 memory sink renamed to become the only implementation
* Since DSv2 sends exceptions in a chained format (linking them with cause field) I've made python side compliant
* Adapted all the tests

## How was this patch tested?

Existing unit tests.

Closes #24403 from gaborgsomogyi/SPARK-23014.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-04-29 09:44:23 -07:00
Sean Owen a6716d3f03 [SPARK-27571][CORE][YARN][EXAMPLES] Avoid scala.language.reflectiveCalls
## What changes were proposed in this pull request?

This PR avoids usage of reflective calls in Scala. It removes the import that suppresses the warnings and rewrites code in small ways to avoid accessing methods that aren't technically accessible.

## How was this patch tested?

Existing tests.

Closes #24463 from srowen/SPARK-27571.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-29 11:16:45 -05:00
Sean Owen 8a17d26784 [SPARK-27536][CORE][ML][SQL][STREAMING] Remove most use of scala.language.existentials
## What changes were proposed in this pull request?

I want to get rid of as much use of `scala.language.existentials` as possible for 3.0. It's a complicated language feature that generates warnings unless this value is imported. It might even be on the way out of Scala: https://contributors.scala-lang.org/t/proposal-to-remove-existential-types-from-the-language/2785

For Spark, it comes up mostly where the code plays fast and loose with generic types, not the advanced situations you'll often see referenced where this feature is explained. For example, it comes up in cases where a function returns something like `(String, Class[_])`. Scala doesn't like matching this to any other instance of `(String, Class[_])` because doing so requires inferring the existence of some type that satisfies both. Seems obvious if the generic type is a wildcard, but, not technically something Scala likes to let you get away with.

This is a large PR, and it only gets rid of _most_ instances of `scala.language.existentials`. The change should be all compile-time and shouldn't affect APIs or logic.

Many of the changes simply touch up sloppiness about generic types, making the known correct value explicit in the code.

Some fixes involve being more explicit about the existence of generic types in methods. For instance, `def foo(arg: Class[_])` seems innocent enough but should really be declared `def foo[T](arg: Class[T])` to let Scala select and fix a single type when evaluating calls to `foo`.

For kind of surprising reasons, this comes up in places where code evaluates a tuple of things that involve a generic type, but is OK if the two parts of the tuple are evaluated separately.

One key change was altering `Utils.classForName(...): Class[_]` to the more correct `Utils.classForName[T](...): Class[T]`. This caused a number of small but positive changes to callers that otherwise had to cast the result.

In several tests, `Dataset[_]` was used where `DataFrame` seems to be the clear intent.

Finally, in a few cases in MLlib, the return type `this.type` was used where there are no subclasses of the class that uses it. This really isn't needed and causes issues for Scala reasoning about the return type. These are just changed to be concrete classes as return types.

After this change, we have only a few classes that still import `scala.language.existentials` (because modifying them would require extensive rewrites to fix) and no build warnings.

## How was this patch tested?

Existing tests.

Closes #24431 from srowen/SPARK-27536.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-29 11:02:01 -05:00
Wenchen Fan 05b85eb8cb [SPARK-27474][CORE] avoid retrying a task failed with CommitDeniedException many times
## What changes were proposed in this pull request?

https://issues.apache.org/jira/browse/SPARK-25250 reports a bug that, a task which is failed with `CommitDeniedException` gets retried many times.

This can happen when a stage has 2 task set managers, one is zombie, one is active. A task from the zombie TSM completes, and commits to a central coordinator(assuming it's a file writing task). Then the corresponding task from the active TSM will fail with `CommitDeniedException`. `CommitDeniedException.countTowardsTaskFailures` is false, so the active TSM will keep retrying this task, until the job finishes. This wastes resource a lot.

#21131 firstly implements that a previous successful completed task from zombie `TaskSetManager` could mark the task of the same partition completed in the active `TaskSetManager`. Later #23871 improves the implementation to cover a corner case that, an active `TaskSetManager` hasn't been created when a previous task succeed.

However, #23871 has a bug and was reverted in #24359. With hindsight, #23781 is fragile because we need to sync the states between `DAGScheduler` and `TaskScheduler`, about which partitions are completed.

This PR proposes a new fix:
1. When `DAGScheduler` gets a task success event from an earlier attempt, notify the `TaskSchedulerImpl` about it
2. When `TaskSchedulerImpl` knows a partition is already completed, ask the active `TaskSetManager` to mark the corresponding task as finished, if the task is not finished yet.

This fix covers the corner case, because:
1. If `DAGScheduler` gets the task completion event from zombie TSM before submitting the new stage attempt, then `DAGScheduler` knows that this partition is completed, and it will exclude this partition when creating task set for the new stage attempt. See `DAGScheduler.submitMissingTasks`
2. If `DAGScheduler` gets the task completion event from zombie TSM after submitting the new stage attempt, then the active TSM is already created.

Compared to the previous fix, the message loop becomes longer, so it's likely that, the active task set manager has already retried the task multiple times. But this failure window won't be too big, and we want to avoid the worse case that retries the task many times until the job finishes. So this solution is acceptable.

## How was this patch tested?

a new test case.

Closes #24375 from cloud-fan/fix2.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-29 14:20:58 +08:00
gatorsmile cd4a284030 [SPARK-27460][FOLLOW-UP][TESTS] Fix flaky tests
## What changes were proposed in this pull request?

This patch makes several test flakiness fixes.

## How was this patch tested?
N/A

Closes #24434 from gatorsmile/fixFlakyTest.

Lead-authored-by: gatorsmile <gatorsmile@gmail.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-24 17:36:29 +08:00
Sean Owen 596a5ff273 [MINOR][BUILD] Update genjavadoc to 0.13
## What changes were proposed in this pull request?

Kind of related to https://github.com/gatorsmile/spark/pull/5 - let's update genjavadoc to see if it generates fewer spurious javadoc errors to begin with.

## How was this patch tested?

Existing docs build

Closes #24443 from srowen/genjavadoc013.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-24 13:44:48 +09:00
uncleGen ecfdffcb35 [SPARK-27503][DSTREAM] JobGenerator thread exit for some fatal errors but application keeps running
## What changes were proposed in this pull request?

In some corner cases, `JobGenerator` thread (including some other EventLoop threads) may exit for some fatal error, like OOM, but Spark Streaming job keep running with no batch job generating. Currently, we only report any non-fatal error.

```
override def run(): Unit = {
      try {
        while (!stopped.get) {
          val event = eventQueue.take()
          try {
            onReceive(event)
          } catch {
            case NonFatal(e) =>
              try {
                onError(e)
              } catch {
                case NonFatal(e) => logError("Unexpected error in " + name, e)
              }
          }
        }
      } catch {
        case ie: InterruptedException => // exit even if eventQueue is not empty
        case NonFatal(e) => logError("Unexpected error in " + name, e)
      }
    }
```

In this PR, we double check if event thread alive when post Event

## How was this patch tested?

existing unit tests

Closes #24400 from uncleGen/SPARK-27503.

Authored-by: uncleGen <hustyugm@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-23 07:11:58 -07:00
Shixiong Zhu 009059e3c2 [SPARK-27496][CORE] Fatal errors should also be sent back to the sender
## What changes were proposed in this pull request?

When a fatal error (such as StackOverflowError) throws from "receiveAndReply", we should try our best to notify the sender. Otherwise, the sender will hang until timeout.

In addition, when a MessageLoop is dying unexpectedly, it should resubmit a new one so that Dispatcher is still working.

## How was this patch tested?

New unit tests.

Closes #24396 from zsxwing/SPARK-27496.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-21 17:00:07 -07:00
Shahid 16bbe0f798 [SPARK-27486][CORE][TEST] Enable History server storage information test in the HistoryServerSuite
## What changes were proposed in this pull request?

We have disabled a test related to storage in the History server suite after SPARK-13845. But, after SPARK-22050, we can store the information about block updated events to eventLog, if we enable "spark.eventLog.logBlockUpdates.enabled=true".

   So, we can enable the test, by adding an eventlog corresponding to the application, which has enabled the configuration, "spark.eventLog.logBlockUpdates.enabled=true"

## How was this patch tested?
Existing UTs

Closes #24390 from shahidki31/enableRddStorageTest.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-19 08:12:20 -07:00
pengbo 54b0d1e0ef [SPARK-27416][SQL] UnsafeMapData & UnsafeArrayData Kryo serialization …
## What changes were proposed in this pull request?
Finish the rest work of https://github.com/apache/spark/pull/24317, https://github.com/apache/spark/pull/9030
a. Implement Kryo serialization for UnsafeArrayData
b. fix UnsafeMapData Java/Kryo Serialization issue when two machines have different Oops size
c. Move the duplicate code "getBytes()" to Utils.

## How was this patch tested?
According Units has been added & tested

Closes #24357 from pengbo/SPARK-27416_new.

Authored-by: pengbo <bo.peng1019@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-17 13:03:00 +08:00
shivusondur 88d9de26dd [SPARK-27464][CORE] Added Constant instead of referring string literal used from many places
## What changes were proposed in this pull request?

Added Constant instead of referring the same String literal "spark.buffer.pageSize" from many places
## How was this patch tested?
Run the corresponding Unit Test Cases manually.

Closes #24368 from shivusondur/Constant.

Authored-by: shivusondur <shivusondur@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-16 09:30:46 -05:00
Kazuaki Ishizaki 257d01a6b8 [SPARK-27397][CORE] Take care of OpenJ9 JVM in Spark
## What changes were proposed in this pull request?

This PR supports `OpenJ9` in addition to `IBM JDK` and `OpenJDK` in Spark by handling `System.getProperty("java.vendor") = "Eclipse OpenJ9"`.

In `inferDefaultMemory()` and `getKrb5LoginModuleName()`, this PR uses non `IBM` way.

```
$ ~/jdk-11.0.2+9_openj9-0.12.1/bin/jshell
|  Welcome to JShell -- Version 11.0.2
|  For an introduction type: /help intro

jshell> System.out.println(System.getProperty("java.vendor"))
Eclipse OpenJ9

jshell> System.out.println(System.getProperty("java.vm.info"))
JRE 11 Linux amd64-64-Bit Compressed References 20190204_127 (JIT enabled, AOT enabled)
OpenJ9   - 90dd8cb40
OMR      - d2f4534b
JCL      - 289c70b6844 based on jdk-11.0.2+9

jshell> System.out.println(Class.forName("com.ibm.lang.management.OperatingSystemMXBean").getDeclaredMethod("getTotalPhysicalMemory"))
public abstract long com.ibm.lang.management.OperatingSystemMXBean.getTotalPhysicalMemory()

jshell> System.out.println(Class.forName("com.sun.management.OperatingSystemMXBean").getDeclaredMethod("getTotalPhysicalMemorySize"))
public abstract long com.sun.management.OperatingSystemMXBean.getTotalPhysicalMemorySize()

jshell> System.out.println(Class.forName("com.ibm.security.auth.module.Krb5LoginModule"))
|  Exception java.lang.ClassNotFoundException: com.ibm.security.auth.module.Krb5LoginModule
|        at Class.forNameImpl (Native Method)
|        at Class.forName (Class.java:339)
|        at (#1:1)

jshell> System.out.println(Class.forName("com.sun.security.auth.module.Krb5LoginModule"))
class com.sun.security.auth.module.Krb5LoginModule
```

## How was this patch tested?

Existing test suites
Manual testing with OpenJ9.

Closes #24308 from kiszk/SPARK-27397.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-16 09:11:47 -05:00
Sean Owen 8718367e2e [SPARK-27470][PYSPARK] Update pyrolite to 4.23
## What changes were proposed in this pull request?

 Update pyrolite to 4.23 to pick up bug and security fixes.

## How was this patch tested?

Existing tests.

Closes #24381 from srowen/SPARK-27470.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-16 19:41:40 +09:00
Wenchen Fan 0bb716bac3 Revert [SPARK-23433][SPARK-25250][CORE] Later created TaskSet should learn about the finished partitions
## What changes were proposed in this pull request?

Our customer has a very complicated job. Sometimes it successes and sometimes it fails with
```
Caused by: org.apache.spark.SparkException: Job aborted due to stage failure: ShuffleMapStage 4  has failed the maximum allowable number of times: 4.
Most recent failure reason: org.apache.spark.shuffle.FetchFailedException
```

However, with the patch https://github.com/apache/spark/pull/23871 , the job hangs forever.

When I investigated it, I found that `DAGScheduler` and `TaskSchedulerImpl` define stage completion differently. `DAGScheduler` thinks a stage is completed if all its partitions are marked as completed ([result stage](https://github.com/apache/spark/blob/v2.4.1/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L1362-L1368) and [shuffle stage](https://github.com/apache/spark/blob/v2.4.1/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L1400)). `TaskSchedulerImpl` thinks a stage's task set is completed when all tasks finish (see the [code](https://github.com/apache/spark/blob/v2.4.1/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala#L779-L784)).

Ideally this two definition should be consistent, but #23871 breaks it. In our customer's Spark log, I found that, a stage's task set completes, but the stage never completes. More specifically, `DAGScheduler` submits a task set for stage 4.1 with 1000 tasks, but the `TaskSetManager` skips to run the first 100 tasks. Later on, `TaskSetManager` finishes 900 tasks and marks the task set as completed. However, `DAGScheduler` doesn't agree with it and hangs forever, waiting for more task completion events of stage 4.1.

With hindsight, I think `TaskSchedulerIImpl.stageIdToFinishedPartitions` is fragile. We need to pay more effort to make sure this is consistent with `DAGScheduler`'s knowledge. When `DAGScheduler` marks some partitions from finished to unfinished, `TaskSchedulerIImpl.stageIdToFinishedPartitions` should be updated as well.

This PR reverts #23871, let's think of a more robust idea later.

## How was this patch tested?

N/A

Closes #24359 from cloud-fan/revert.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-14 16:57:41 +08:00
Bago Amirbekian eea3f55a31 [SPARK-27446][R] Use existing spark conf if available.
## What changes were proposed in this pull request?

The RBackend and RBackendHandler create new conf objects that don't pick up conf values from the existing SparkSession and therefore always use the default conf values instead of values specified by the user.

In this fix we check to see if the spark env already exists, and get the conf from there. We fall back to creating a new conf. This follows the pattern used in other places including this: 3725b1324f/core/src/main/scala/org/apache/spark/api/r/BaseRRunner.scala (L261)

## How was this patch tested?

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

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

Closes #24353 from MrBago/r-backend-use-existing-conf.

Authored-by: Bago Amirbekian <bago@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-14 17:09:12 +09:00
Sean Owen 4ec7f631aa [SPARK-27404][CORE][SQL][STREAMING][YARN] Fix build warnings for 3.0: postfixOps edition
## What changes were proposed in this pull request?

Fix build warnings -- see some details below.

But mostly, remove use of postfix syntax where it causes warnings without the `scala.language.postfixOps` import. This is mostly in expressions like "120000 milliseconds". Which, I'd like to simplify to things like "2.minutes" anyway.

## How was this patch tested?

Existing tests.

Closes #24314 from srowen/SPARK-27404.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-11 13:43:44 -05:00
Maxim Gekk d33ae2e9ed [SPARK-26953][CORE][TEST] Disable result checking in the test: java.lang.ArrayIndexOutOfBoundsException in TimSort
## What changes were proposed in this pull request?

I propose to disable (comment) result checking in `SorterSuite`.`java.lang.ArrayIndexOutOfBoundsException in TimSort` because:
1. The check is optional, and correctness of TimSort is checked by another tests. Purpose of the test is to check that TimSort doesn't fail with `ArrayIndexOutOfBoundsException`.
2. Significantly drops execution time of the test.

Here are timing of running the test locally:
```
Sort: 1.4 seconds
Result checking: 15.6 seconds
```

## How was this patch tested?

By `SorterSuite`.

Closes #24343 from MaxGekk/timsort-test-speedup.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-11 07:58:57 -07:00
Shixiong Zhu 5ff39cd5ee [SPARK-27394][WEBUI] Flush LiveEntity if necessary when receiving SparkListenerExecutorMetricsUpdate
## What changes were proposed in this pull request?

This PR updates `AppStatusListener` to flush `LiveEntity` if necessary when receiving `SparkListenerExecutorMetricsUpdate`. This will ensure the staleness of Spark UI doesn't last more than the executor heartbeat interval.

## How was this patch tested?

The new unit test.

Closes #24303 from zsxwing/SPARK-27394.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-04-09 08:26:00 -07:00
LantaoJin 52838e74af [SPARK-13704][CORE][YARN] Reduce rack resolution time
## What changes were proposed in this pull request?

When you submit a stage on a large cluster, rack resolving takes a long time when initializing TaskSetManager because a script is invoked to resolve the rack of each host, one by one.
Based on current implementation, it takes 30~40 seconds to resolve the racks in our 5000 nodes' cluster. After applied the patch, it decreased to less than 15 seconds.

YARN-9332 has added an interface to handle multiple hosts in one invocation to save time. But before upgrading to the newest Hadoop, we could construct the same tool in Spark to resolve this issue.

## How was this patch tested?

UT and manually testing on a 5000 node cluster.

Closes #24245 from squito/SPARK-13704_update.

Lead-authored-by: LantaoJin <jinlantao@gmail.com>
Co-authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2019-04-08 10:47:06 -05:00
liulijia 39f75b4588 [SPARK-27192][CORE] spark.task.cpus should be less or equal than spark.executor.cores
## What changes were proposed in this pull request?
check spark.task.cpus before creating TaskScheduler in SparkContext

## How was this patch tested?
UT

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

Closes #24261 from liutang123/SPARK-27192.

Authored-by: liulijia <liutang123@yeah.net>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-05 13:55:57 -05:00
Dongjoon Hyun 982c4c8e3c [SPARK-27390][CORE][SQL][TEST] Fix package name mismatch
## What changes were proposed in this pull request?

This PR aims to clean up package name mismatches.

## How was this patch tested?

Pass the Jenkins.

Closes #24300 from dongjoon-hyun/SPARK-27390.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-05 11:50:37 -07:00
Sean Owen 23bde44797 [SPARK-27358][UI] Update jquery to 1.12.x to pick up security fixes
## What changes were proposed in this pull request?

Update jquery -> 1.12.4, datatables -> 1.10.18, mustache -> 2.3.12.
Add missing mustache license

## How was this patch tested?

I manually tested the UI locally with the javascript console open and didn't observe any problems or JS errors. The only 'risky' change seems to be mustache, but on reading its release notes, don't think the changes from 0.8.1 to 2.x would affect Spark's simple usage.

Closes #24288 from srowen/SPARK-27358.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-05 12:54:01 -05:00
Wenchen Fan b56e433b54 [SPARK-27338][CORE][FOLLOWUP] remove trailing space
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/24265 breaks the lint check, because it has trailing space. (not sure why it passed jenkins). This PR fixes it.

## How was this patch tested?

N/A

Closes #24289 from cloud-fan/fix.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-04 11:43:20 +08:00
Venkata krishnan Sowrirajan 6c4552c650 [SPARK-27338][CORE] Fix deadlock in UnsafeExternalSorter.SpillableIterator when locking both UnsafeExternalSorter.SpillableIterator and TaskMemoryManager
## What changes were proposed in this pull request?

In `UnsafeExternalSorter.SpillableIterator#loadNext()` takes lock on the `UnsafeExternalSorter` and calls `freePage` once the `lastPage` is consumed which needs to take a lock on `TaskMemoryManager`. At the same time, there can be another MemoryConsumer using `UnsafeExternalSorter` as part of sorting can try to `allocatePage` needs to get lock on `TaskMemoryManager` which can cause spill to happen which requires lock on `UnsafeExternalSorter` again causing deadlock. This is a classic deadlock situation happening similar to the SPARK-26265.

To fix this, we can move the `freePage` call in `loadNext` outside of `Synchronized` block similar to the fix in SPARK-26265

## How was this patch tested?

Manual tests were being done and will also try to add a test.

Closes #24265 from venkata91/deadlock-sorter.

Authored-by: Venkata krishnan Sowrirajan <vsowrirajan@qubole.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-04 09:58:05 +08:00
LantaoJin 69dd44af19 [SPARK-27216][CORE] Upgrade RoaringBitmap to 0.7.45 to fix Kryo unsafe ser/dser issue
## What changes were proposed in this pull request?

HighlyCompressedMapStatus uses RoaringBitmap to record the empty blocks. But RoaringBitmap couldn't be ser/deser with unsafe KryoSerializer.

It's a bug of RoaringBitmap-0.5.11 and fixed in latest version.

This is an update of #24157

## How was this patch tested?

Add a UT

Closes #24264 from LantaoJin/SPARK-27216.

Lead-authored-by: LantaoJin <jinlantao@gmail.com>
Co-authored-by: Lantao Jin <jinlantao@gmail.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2019-04-03 20:09:50 -05:00
Gabor Somogyi 57aff93886 [SPARK-26998][CORE] Remove SSL configuration from executors
## What changes were proposed in this pull request?

Different SSL passwords shown up as command line argument on executor side in standalone mode:
* keyStorePassword
* keyPassword
* trustStorePassword

In this PR I've removed SSL configurations from executors.

## How was this patch tested?

Existing + additional unit tests.
Additionally tested with standalone mode and checked the command line arguments:
```
[gaborsomogyi:~/spark] SPARK-26998(+4/-0,3)+ ± jps
94803 CoarseGrainedExecutorBackend
94818 Jps
90149 RemoteMavenServer
91925 Nailgun
94793 SparkSubmit
94680 Worker
94556 Master
398
[gaborsomogyi:~/spark] SPARK-26998(+4/-1,3)+ ± ps -ef | egrep "94556|94680|94793|94803"
  502 94556     1   0  2:02PM ttys007    0:07.39 /Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/bin/java -cp /Users/gaborsomogyi/spark/conf/:/Users/gaborsomogyi/spark/assembly/target/scala-2.12/jars/* -Xmx1g org.apache.spark.deploy.master.Master --host gsomogyi-MBP.local --port 7077 --webui-port 8080 --properties-file conf/spark-defaults.conf
  502 94680     1   0  2:02PM ttys007    0:07.27 /Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/bin/java -cp /Users/gaborsomogyi/spark/conf/:/Users/gaborsomogyi/spark/assembly/target/scala-2.12/jars/* -Xmx1g org.apache.spark.deploy.worker.Worker --webui-port 8081 --properties-file conf/spark-defaults.conf spark://gsomogyi-MBP.local:7077
  502 94793 94782   0  2:02PM ttys007    0:35.52 /Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/bin/java -cp /Users/gaborsomogyi/spark/conf/:/Users/gaborsomogyi/spark/assembly/target/scala-2.12/jars/* -Dscala.usejavacp=true -Xmx1g org.apache.spark.deploy.SparkSubmit --master spark://gsomogyi-MBP.local:7077 --class org.apache.spark.repl.Main --name Spark shell spark-shell
  502 94803 94680   0  2:03PM ttys007    0:05.20 /Library/Java/JavaVirtualMachines/jdk1.8.0_152.jdk/Contents/Home/bin/java -cp /Users/gaborsomogyi/spark/conf/:/Users/gaborsomogyi/spark/assembly/target/scala-2.12/jars/* -Xmx1024M -Dspark.ssl.ui.port=0 -Dspark.driver.port=60902 org.apache.spark.executor.CoarseGrainedExecutorBackend --driver-url spark://CoarseGrainedScheduler172.30.65.186:60902 --executor-id 0 --hostname 172.30.65.186 --cores 8 --app-id app-20190326140311-0000 --worker-url spark://Worker172.30.65.186:60899
  502 94910 57352   0  2:05PM ttys008    0:00.00 egrep 94556|94680|94793|94803
```

Closes #24170 from gaborgsomogyi/SPARK-26998.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-04-02 09:18:43 -07:00
Sean Owen d4420b455a [SPARK-27323][CORE][SQL][STREAMING] Use Single-Abstract-Method support in Scala 2.12 to simplify code
## What changes were proposed in this pull request?

Use Single Abstract Method syntax where possible (and minor related cleanup). Comments below. No logic should change here.

## How was this patch tested?

Existing tests.

Closes #24241 from srowen/SPARK-27323.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-02 07:37:05 -07:00
“attilapiros” 9eb896cc3b [SPARK-27333][TEST] Update thread audit whitelist to skip broadcast-exchange-.*, process reaper and StatisticsDataReferenceCleaner threads
## What changes were proposed in this pull request?

Update thread audit whitelist to skip threads of the global broadcast exchange thread pool, process reaper and Hadoop FS statistics data reference cleaner thread.

## How was this patch tested?

Via existing UT using broadcast exchange via `sbt` i.e:

```
> project sql
> testOnly *.SessionStateSuite -- -z "fork new sessions and run query on inherited table"
```

Before (wrapped long line for manually to save horizontal scrolling for reviewers):

```
===== POSSIBLE THREAD LEAK IN SUITE o.a.s.sql.SessionStateSuite,
thread names: broadcast-exchange-6, broadcast-exchange-0,
broadcast-exchange-2, broadcast-exchange-5, broadcast-exchange-7,
broadcast-exchange-4, broadcast-exchange-1, process reaper, broadcast-exchange-3,
 org.apache.hadoop.fs.FileSystem$Statistics$StatisticsDataReferenceCleaner =====
```

After this change no possible thread leak detected.

Closes #24244 from attilapiros/thread-audit-minor.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-31 17:33:31 -07:00
gatorsmile 92b6f86f6d [SPARK-27244][CORE][TEST][FOLLOWUP] toDebugString redacts sensitive information
## What changes were proposed in this pull request?
This PR is a FollowUp of https://github.com/apache/spark/pull/24196. It improves the test case by using the parameters that are being used in the actual scenarios.

## How was this patch tested?
N/A

Closes #24257 from gatorsmile/followupSPARK-27244.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-30 22:58:28 -07:00
Sean Owen 754f820035 [SPARK-26918][DOCS] All .md should have ASF license header
## What changes were proposed in this pull request?

Add AL2 license to metadata of all .md files.
This seemed to be the tidiest way as it will get ignored by .md renderers and other tools. Attempts to write them as markdown comments revealed that there is no such standard thing.

## How was this patch tested?

Doc build

Closes #24243 from srowen/SPARK-26918.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-30 19:49:45 -05:00
Dongjoon Hyun 88ea319871 Revert "[SPARK-27192][CORE] spark.task.cpus should be less or equal than spark.executor.cores"
This reverts commit f8fa564dec.
2019-03-30 16:35:34 -07:00
liulijia f8fa564dec [SPARK-27192][CORE] spark.task.cpus should be less or equal than spark.executor.cores
## What changes were proposed in this pull request?
spark.task.cpus should be less or equal than spark.executor.cores when use static executor allocation
## How was this patch tested?
manual

Closes #24131 from liutang123/SPARK-27192.

Authored-by: liulijia <liutang123@yeah.net>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-30 12:38:05 -05:00
Ninad Ingole dbc7ce18b9 [SPARK-27244][CORE] Redact Passwords While Using Option logConf=true
## What changes were proposed in this pull request?

When logConf is set to true, config keys that contain password were printed in cleartext in driver log. This change uses the already present redact method in Utils, to redact all the passwords based on redact pattern in SparkConf and then print the conf to driver log thus ensuring that sensitive information like passwords is not printed in clear text.

## How was this patch tested?
This patch was tested through `SparkConfSuite` & then entire unit test through sbt

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

Closes #24196 from ninadingole/SPARK-27244.

Authored-by: Ninad Ingole <robert.wallis@example.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-29 14:16:53 -05:00
Wenchen Fan e4a968d829 [MINOR][CORE] Remove import scala.collection.Set in TaskSchedulerImpl
## What changes were proposed in this pull request?

I was playing with the scheduler and found this weird thing. In `TaskSchedulerImpl` we import `scala.collection.Set` without any reason. This is bad in practice, as it silently changes the actual class when we simply type `Set`, which by default should point to the immutable set.

This change only affects one method: `getExecutorsAliveOnHost`. I checked all the caller side and none of them need a general `Set` type.

## How was this patch tested?

N/A

Closes #24231 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-28 21:12:18 +09:00
Sean Owen 3a8398df5c [SPARK-26660][FOLLOWUP] Raise task serialized size warning threshold to 1000 KiB
## What changes were proposed in this pull request?

Raise the threshold size for serialized task size at which a warning is generated from 100KiB to 1000KiB.

As several people have noted, the original change for this JIRA highlighted that this threshold is low. Test output regularly shows:

```
- sorting on StringType with nullable=false, sortOrder=List('a DESC NULLS LAST)
22:47:53.320 WARN org.apache.spark.scheduler.TaskSetManager: Stage 80 contains a task of very large size (755 KiB). The maximum recommended task size is 100 KiB.
22:47:53.348 WARN org.apache.spark.scheduler.TaskSetManager: Stage 81 contains a task of very large size (755 KiB). The maximum recommended task size is 100 KiB.
22:47:53.417 WARN org.apache.spark.scheduler.TaskSetManager: Stage 83 contains a task of very large size (755 KiB). The maximum recommended task size is 100 KiB.
22:47:53.444 WARN org.apache.spark.scheduler.TaskSetManager: Stage 84 contains a task of very large size (755 KiB). The maximum recommended task size is 100 KiB.

...

- SPARK-20688: correctly check analysis for scalar sub-queries
22:49:10.314 WARN org.apache.spark.scheduler.DAGScheduler: Broadcasting large task binary with size 150.8 KiB
- SPARK-21835: Join in correlated subquery should be duplicateResolved: case 1
22:49:10.595 WARN org.apache.spark.scheduler.DAGScheduler: Broadcasting large task binary with size 150.7 KiB
22:49:10.744 WARN org.apache.spark.scheduler.DAGScheduler: Broadcasting large task binary with size 150.7 KiB
22:49:10.894 WARN org.apache.spark.scheduler.DAGScheduler: Broadcasting large task binary with size 150.7 KiB
- SPARK-21835: Join in correlated subquery should be duplicateResolved: case 2
- SPARK-21835: Join in correlated subquery should be duplicateResolved: case 3
- SPARK-23316: AnalysisException after max iteration reached for IN query
22:49:11.559 WARN org.apache.spark.scheduler.DAGScheduler: Broadcasting large task binary with size 154.2 KiB
```

It seems that a larger threshold of about 1MB is more suitable.

## How was this patch tested?

Existing tests.

Closes #24226 from srowen/SPARK-26660.2.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-03-27 10:42:26 +09:00
Stavros Kontopoulos 05168e725d [SPARK-24793][K8S] Enhance spark-submit for app management
- supports `--kill` & `--status` flags.
- supports globs which is useful in general check this long standing [issue](https://github.com/kubernetes/kubernetes/issues/17144#issuecomment-272052461) for kubectl.

Manually against running apps. Example output:

Submission Id reported at launch time:

```
2019-01-20 23:47:56 INFO  Client:58 - Waiting for application spark-pi with submissionId spark:spark-pi-1548020873671-driver to finish...
```

Killing the app:

```
./bin/spark-submit --kill spark:spark-pi-1548020873671-driver --master  k8s://https://192.168.2.8:8443
2019-01-20 23:48:07 WARN  Utils:70 - Your hostname, universe resolves to a loopback address: 127.0.0.1; using 192.168.2.8 instead (on interface wlp2s0)
2019-01-20 23:48:07 WARN  Utils:70 - Set SPARK_LOCAL_IP if you need to bind to another address

```

App terminates with 143 (SIGTERM, since we have tiny this should lead to [graceful shutdown](https://cloud.google.com/solutions/best-practices-for-building-containers)):

```
2019-01-20 23:48:08 INFO  LoggingPodStatusWatcherImpl:58 - State changed, new state:
	 pod name: spark-pi-1548020873671-driver
	 namespace: spark
	 labels: spark-app-selector -> spark-e4730c80e1014b72aa77915a2203ae05, spark-role -> driver
	 pod uid: 0ba9a794-1cfd-11e9-8215-a434d9270a65
	 creation time: 2019-01-20T21:47:55Z
	 service account name: spark-sa
	 volumes: spark-local-dir-1, spark-conf-volume, spark-sa-token-b7wcm
	 node name: minikube
	 start time: 2019-01-20T21:47:55Z
	 phase: Running
	 container status:
		 container name: spark-kubernetes-driver
		 container image: skonto/spark:k8s-3.0.0
		 container state: running
		 container started at: 2019-01-20T21:48:00Z
2019-01-20 23:48:09 INFO  LoggingPodStatusWatcherImpl:58 - State changed, new state:
	 pod name: spark-pi-1548020873671-driver
	 namespace: spark
	 labels: spark-app-selector -> spark-e4730c80e1014b72aa77915a2203ae05, spark-role -> driver
	 pod uid: 0ba9a794-1cfd-11e9-8215-a434d9270a65
	 creation time: 2019-01-20T21:47:55Z
	 service account name: spark-sa
	 volumes: spark-local-dir-1, spark-conf-volume, spark-sa-token-b7wcm
	 node name: minikube
	 start time: 2019-01-20T21:47:55Z
	 phase: Failed
	 container status:
		 container name: spark-kubernetes-driver
		 container image: skonto/spark:k8s-3.0.0
		 container state: terminated
		 container started at: 2019-01-20T21:48:00Z
		 container finished at: 2019-01-20T21:48:08Z
		 exit code: 143
		 termination reason: Error
2019-01-20 23:48:09 INFO  LoggingPodStatusWatcherImpl:58 - Container final statuses:
	 container name: spark-kubernetes-driver
	 container image: skonto/spark:k8s-3.0.0
	 container state: terminated
	 container started at: 2019-01-20T21:48:00Z
	 container finished at: 2019-01-20T21:48:08Z
	 exit code: 143
	 termination reason: Error
2019-01-20 23:48:09 INFO  Client:58 - Application spark-pi with submissionId spark:spark-pi-1548020873671-driver finished.
2019-01-20 23:48:09 INFO  ShutdownHookManager:58 - Shutdown hook called
2019-01-20 23:48:09 INFO  ShutdownHookManager:58 - Deleting directory /tmp/spark-f114b2e0-5605-4083-9203-a4b1c1f6059e

```

Glob scenario:

```
./bin/spark-submit --status spark:spark-pi* --master  k8s://https://192.168.2.8:8443
2019-01-20 22:27:44 WARN  Utils:70 - Your hostname, universe resolves to a loopback address: 127.0.0.1; using 192.168.2.8 instead (on interface wlp2s0)
2019-01-20 22:27:44 WARN  Utils:70 - Set SPARK_LOCAL_IP if you need to bind to another address
Application status (driver):
	 pod name: spark-pi-1547948600328-driver
	 namespace: spark
	 labels: spark-app-selector -> spark-f13f01702f0b4503975ce98252d59b94, spark-role -> driver
	 pod uid: c576e1c6-1c54-11e9-8215-a434d9270a65
	 creation time: 2019-01-20T01:43:22Z
	 service account name: spark-sa
	 volumes: spark-local-dir-1, spark-conf-volume, spark-sa-token-b7wcm
	 node name: minikube
	 start time: 2019-01-20T01:43:22Z
	 phase: Running
	 container status:
		 container name: spark-kubernetes-driver
		 container image: skonto/spark:k8s-3.0.0
		 container state: running
		 container started at: 2019-01-20T01:43:27Z
Application status (driver):
	 pod name: spark-pi-1547948792539-driver
	 namespace: spark
	 labels: spark-app-selector -> spark-006d252db9b24f25b5069df357c30264, spark-role -> driver
	 pod uid: 38375b4b-1c55-11e9-8215-a434d9270a65
	 creation time: 2019-01-20T01:46:35Z
	 service account name: spark-sa
	 volumes: spark-local-dir-1, spark-conf-volume, spark-sa-token-b7wcm
	 node name: minikube
	 start time: 2019-01-20T01:46:35Z
	 phase: Succeeded
	 container status:
		 container name: spark-kubernetes-driver
		 container image: skonto/spark:k8s-3.0.0
		 container state: terminated
		 container started at: 2019-01-20T01:46:39Z
		 container finished at: 2019-01-20T01:46:56Z
		 exit code: 0
		 termination reason: Completed

```

Closes #23599 from skonto/submit_ops_extension.

Authored-by: Stavros Kontopoulos <stavros.kontopoulos@lightbend.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-26 11:55:03 -07:00
Ajith b61dce23d2 [SPARK-26961][CORE] Enable parallel classloading capability
## What changes were proposed in this pull request?

As per https://docs.oracle.com/javase/8/docs/api/java/lang/ClassLoader.html
``Class loaders that support concurrent loading of classes are known as parallel capable class loaders and are required to register themselves at their class initialization time by invoking the ClassLoader.registerAsParallelCapable method. Note that the ClassLoader class is registered as parallel capable by default. However, its subclasses still need to register themselves if they are parallel capable. ``
 i.e we can have finer class loading locks by registering classloaders as parallel capable.  (Refer to deadlock due to macro lock  https://issues.apache.org/jira/browse/SPARK-26961).
All the classloaders we have are wrapper of URLClassLoader which by itself is parallel capable.
But this cannot be achieved by scala code due to static registration Refer https://github.com/scala/bug/issues/11429

## How was this patch tested?

All Existing UT must pass

Closes #24126 from ajithme/driverlock.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-25 19:07:30 -05:00
liuxian e4b36df2c0 [SPARK-27256][CORE][SQL] If the configuration is used to set the number of bytes, we'd better use bytesConf'.
## What changes were proposed in this pull request?
Currently, if we want to configure `spark.sql.files.maxPartitionBytes` to 256 megabytes, we must set  `spark.sql.files.maxPartitionBytes=268435456`, which is very unfriendly to users.

And if we set it like this:`spark.sql.files.maxPartitionBytes=256M`, we will  encounter this exception:
```
Exception in thread "main" java.lang.IllegalArgumentException:
 spark.sql.files.maxPartitionBytes should be long, but was 256M
        at org.apache.spark.internal.config.ConfigHelpers$.toNumber(ConfigBuilder.scala)
```
This PR use `bytesConf` to replace `longConf` or `intConf`,  if the configuration is used to set the number of bytes.
Configuration change list:
`spark.files.maxPartitionBytes`
`spark.files.openCostInBytes`
`spark.shuffle.sort.initialBufferSize`
`spark.shuffle.spill.initialMemoryThreshold`
`spark.sql.autoBroadcastJoinThreshold`
`spark.sql.files.maxPartitionBytes`
`spark.sql.files.openCostInBytes`
`spark.sql.defaultSizeInBytes`
## How was this patch tested?
1.Existing unit tests
2.Manual testing

Closes #24187 from 10110346/bytesConf.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-25 14:47:40 -07:00
Luca Canali 4b2b3da766 [SPARK-26928][CORE][FOLLOWUP] Fix JVMCPUSource file name and minor updates to doc
## What changes were proposed in this pull request?

This applies some minor updates/cleaning following up SPARK-26928, notably renaming JVMCPU.scala to JVMCPUSource.scala.

## How was this patch tested?

Manually tested

Closes #24201 from LucaCanali/fixupSPARK-26928.

Authored-by: Luca Canali <luca.canali@cern.ch>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-25 15:35:24 -05:00
Sean Owen 8bc304f97e [SPARK-26132][BUILD][CORE] Remove support for Scala 2.11 in Spark 3.0.0
## What changes were proposed in this pull request?

Remove Scala 2.11 support in build files and docs, and in various parts of code that accommodated 2.11. See some targeted comments below.

## How was this patch tested?

Existing tests.

Closes #23098 from srowen/SPARK-26132.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-25 10:46:42 -05:00
Maxim Gekk 027ed2d11b [SPARK-23643][CORE][SQL][ML] Shrinking the buffer in hashSeed up to size of the seed parameter
## What changes were proposed in this pull request?

The hashSeed method allocates 64 bytes instead of 8. Other bytes are always zeros (thanks to default behavior of ByteBuffer). And they could be excluded from hash calculation because they don't differentiate inputs.

## How was this patch tested?

By running the existing tests - XORShiftRandomSuite

Closes #20793 from MaxGekk/hash-buff-size.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-23 11:26:09 -05:00
hehuiyuan 68abf77b1a [SPARK-27184][CORE] Avoid hardcoded 'spark.jars', 'spark.files', 'spark.submit.pyFiles' and 'spark.submit.deployMode'
## What changes were proposed in this pull request?

For [SPARK-27184](https://issues.apache.org/jira/browse/SPARK-27184)

In the `org.apache.spark.internal.config`, we define the variables of `FILES` and `JARS`, we can use them instead of "spark.jars" and "spark.files".

```scala
private[spark] val JARS = ConfigBuilder("spark.jars")
  .stringConf
  .toSequence
  .createWithDefault(Nil)
```

```scala
private[spark] val FILES = ConfigBuilder("spark.files")
  .stringConf
  .toSequence
  .createWithDefault(Nil)
```

Other :
In the `org.apache.spark.internal.config`, we define the variables of `SUBMIT_PYTHON_FILES ` and `SUBMIT_DEPLOY_MODE `, we can use them instead of "spark.submit.pyFiles" and "spark.submit.deployMode".
```scala
private[spark] val SUBMIT_PYTHON_FILES = ConfigBuilder("spark.submit.pyFiles")
    .stringConf
    .toSequence
    .createWithDefault(Nil)

```
```scala
private[spark] val SUBMIT_DEPLOY_MODE = ConfigBuilder("spark.submit.deployMode")
    .stringConf
    .createWithDefault("client")
```

Closes #24123 from hehuiyuan/hehuiyuan-patch-6.

Authored-by: hehuiyuan <hehuiyuan@ZBMAC-C02WD3K5H.local>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-23 09:43:00 +09:00
Jungtaek Lim (HeartSaVioR) 8a9eb05137 [SPARK-26606][CORE] Handle driver options properly when submitting to standalone cluster mode via legacy Client
## What changes were proposed in this pull request?

This patch fixes the issue that ClientEndpoint in standalone cluster doesn't recognize about driver options which are passed to SparkConf instead of system properties. When `Client` is executed via cli they should be provided as system properties, but with `spark-submit` they can be provided as SparkConf. (SpartSubmit will call `ClientApp.start` with SparkConf which would contain these options.)

## How was this patch tested?

Manually tested via following steps:

1) setup standalone cluster (launch master and worker via `./sbin/start-all.sh`)

2) submit one of example app with standalone cluster mode

```
./bin/spark-submit --class org.apache.spark.examples.SparkPi --master "spark://localhost:7077" --conf "spark.driver.extraJavaOptions=-Dfoo=BAR" --deploy-mode "cluster" --num-executors 1 --driver-memory 512m --executor-memory 512m --executor-cores 1 examples/jars/spark-examples*.jar 10
```

3) check whether `foo=BAR` is provided in system properties in Spark UI

<img width="877" alt="Screen Shot 2019-03-21 at 8 18 04 AM" src="https://user-images.githubusercontent.com/1317309/54728501-97db1700-4bc1-11e9-89da-078445c71e9b.png">

Closes #24163 from HeartSaVioR/SPARK-26606.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-22 15:07:49 -07:00