Commit graph

268 commits

Author SHA1 Message Date
Devaraj K 51066b437b [SPARK-14228][CORE][YARN] Lost executor of RPC disassociated, and occurs exception: Could not find CoarseGrainedScheduler or it has been stopped
## What changes were proposed in this pull request?
I see the two instances where the exception is occurring.

**Instance 1:**

```
17/11/10 15:49:32 ERROR util.Utils: Uncaught exception in thread driver-revive-thread
org.apache.spark.SparkException: Could not find CoarseGrainedScheduler.
        at org.apache.spark.rpc.netty.Dispatcher.postMessage(Dispatcher.scala:160)
        at org.apache.spark.rpc.netty.Dispatcher.postOneWayMessage(Dispatcher.scala:140)
        at org.apache.spark.rpc.netty.NettyRpcEnv.send(NettyRpcEnv.scala:187)
        at org.apache.spark.rpc.netty.NettyRpcEndpointRef.send(NettyRpcEnv.scala:521)
        at org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend$DriverEndpoint$$anon$1$$anonfun$run$1$$anonfun$apply$mcV$sp$1.apply(CoarseGrainedSchedulerBackend.scala:125)
        at org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend$DriverEndpoint$$anon$1$$anonfun$run$1$$anonfun$apply$mcV$sp$1.apply(CoarseGrainedSchedulerBackend.scala:125)
        at scala.Option.foreach(Option.scala:257)
        at org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend$DriverEndpoint$$anon$1$$anonfun$run$1.apply$mcV$sp(CoarseGrainedSchedulerBackend.scala:125)
        at org.apache.spark.util.Utils$.tryLogNonFatalError(Utils.scala:1344)
        at org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend$DriverEndpoint$$anon$1.run(CoarseGrainedSchedulerBackend.scala:124)
        at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
        at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
        at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
        at java.lang.Thread.run(Thread.java:745)
```

In CoarseGrainedSchedulerBackend.scala, driver-revive-thread starts with DriverEndpoint.onStart() and keeps sending the ReviveOffers messages periodically till it gets shutdown as part DriverEndpoint.onStop(). There is no proper coordination between the driver-revive-thread(shutdown) and the RpcEndpoint unregister, RpcEndpoint unregister happens first and then driver-revive-thread shuts down as part of DriverEndpoint.onStop(), In-between driver-revive-thread may try to send the ReviveOffers message which is leading to the above exception.

To fix this issue, this PR moves the shutting down of driver-revive-thread to CoarseGrainedSchedulerBackend.stop() which executes before the DriverEndpoint unregister.

**Instance 2:**

```
17/11/10 16:31:38 ERROR cluster.YarnSchedulerBackend$YarnSchedulerEndpoint: Error requesting driver to remove executor 1 for reason Executor for container container_1508535467865_0226_01_000002 exited because of a YARN event (e.g., pre-emption) and not because of an error in the running job.
org.apache.spark.SparkException: Could not find CoarseGrainedScheduler.
        at org.apache.spark.rpc.netty.Dispatcher.postMessage(Dispatcher.scala:160)
        at org.apache.spark.rpc.netty.Dispatcher.postLocalMessage(Dispatcher.scala:135)
        at org.apache.spark.rpc.netty.NettyRpcEnv.ask(NettyRpcEnv.scala:229)
        at org.apache.spark.rpc.netty.NettyRpcEndpointRef.ask(NettyRpcEnv.scala:516)
        at org.apache.spark.rpc.RpcEndpointRef.ask(RpcEndpointRef.scala:63)
        at org.apache.spark.scheduler.cluster.YarnSchedulerBackend$YarnSchedulerEndpoint$$anonfun$receive$1.applyOrElse(YarnSchedulerBackend.scala:269)
        at org.apache.spark.rpc.netty.Inbox$$anonfun$process$1.apply$mcV$sp(Inbox.scala:117)
        at org.apache.spark.rpc.netty.Inbox.safelyCall(Inbox.scala:205)
        at org.apache.spark.rpc.netty.Inbox.process(Inbox.scala:101)
        at org.apache.spark.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:221)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
        at java.lang.Thread.run(Thread.java:745)
```

Here YarnDriverEndpoint tries to send remove executor messages after the Yarn scheduler backend service stop, which is leading to the above exception. To avoid the above exception,
1) We may add a condition(which checks whether service has stopped or not) before sending executor remove message
2) Add a warn log message in onFailure case when the service is already stopped

In this PR, chosen the 2) option which adds a log message in the case of onFailure without the exception stack trace since the option 1) would need to to go through for every remove executor message.

## How was this patch tested?
I verified it manually, I don't see these exceptions with the PR changes.

Author: Devaraj K <devaraj@apache.org>

Closes #19741 from devaraj-kavali/SPARK-14228.
2017-12-06 10:39:15 -08:00
Marcelo Vanzin e1dd03e42c [SPARK-22372][CORE, YARN] Make cluster submission use SparkApplication.
The main goal of this change is to allow multiple cluster-mode
submissions from the same JVM, without having them end up with
mixed configuration. That is done by extending the SparkApplication
trait, and doing so was reasonably trivial for standalone and
mesos modes.

For YARN mode, there was a complication. YARN used a "SPARK_YARN_MODE"
system property to control behavior indirectly in a whole bunch of
places, mainly in the SparkHadoopUtil / YarnSparkHadoopUtil classes.
Most of the changes here are removing that.

Since we removed support for Hadoop 1.x, some methods that lived in
YarnSparkHadoopUtil can now live in SparkHadoopUtil. The remaining
methods don't need to be part of the class, and can be called directly
from the YarnSparkHadoopUtil object, so now there's a single
implementation of SparkHadoopUtil.

There were two places in the code that relied on  SPARK_YARN_MODE to
make decisions about YARN-specific functionality, and now explicitly check
the master from the configuration for that instead:

* fetching the external shuffle service port, which can come from the YARN
  configuration.

* propagation of the authentication secret using Hadoop credentials. This also
  was cleaned up a little to not need so many methods in `SparkHadoopUtil`.

With those out of the way, actually changing the YARN client
to extend SparkApplication was easy.

Tested with existing unit tests, and also by running YARN apps
with auth and kerberos both on and off in a real cluster.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19631 from vanzin/SPARK-22372.
2017-12-04 11:05:03 -08:00
Yinan Li e9b2070ab2 [SPARK-18278][SCHEDULER] Spark on Kubernetes - Basic Scheduler Backend
## What changes were proposed in this pull request?

This is a stripped down version of the `KubernetesClusterSchedulerBackend` for Spark with the following components:
- Static Allocation of Executors
- Executor Pod Factory
- Executor Recovery Semantics

It's step 1 from the step-wise plan documented [here](https://github.com/apache-spark-on-k8s/spark/issues/441#issuecomment-330802935).
This addition is covered by the [SPIP vote](http://apache-spark-developers-list.1001551.n3.nabble.com/SPIP-Spark-on-Kubernetes-td22147.html) which passed on Aug 31 .

## How was this patch tested?

- The patch contains unit tests which are passing.
- Manual testing: `./build/mvn -Pkubernetes clean package` succeeded.
- It is a **subset** of the entire changelist hosted in http://github.com/apache-spark-on-k8s/spark which is in active use in several organizations.
- There is integration testing enabled in the fork currently [hosted by PepperData](spark-k8s-jenkins.pepperdata.org:8080) which is being moved over to RiseLAB CI.
- Detailed documentation on trying out the patch in its entirety is in: https://apache-spark-on-k8s.github.io/userdocs/running-on-kubernetes.html

cc rxin felixcheung mateiz (shepherd)
k8s-big-data SIG members & contributors: mccheah ash211 ssuchter varunkatta kimoonkim erikerlandson liyinan926 tnachen ifilonenko

Author: Yinan Li <liyinan926@gmail.com>
Author: foxish <ramanathana@google.com>
Author: mcheah <mcheah@palantir.com>

Closes #19468 from foxish/spark-kubernetes-3.
2017-11-28 23:02:09 -08:00
ArtRand 1e82335413 [SPARK-21842][MESOS] Support Kerberos ticket renewal and creation in Mesos
## What changes were proposed in this pull request?
tl;dr: Add a class, `MesosHadoopDelegationTokenManager` that updates delegation tokens on a schedule on the behalf of Spark Drivers. Broadcast renewed credentials to the executors.

## The problem
We recently added Kerberos support to Mesos-based Spark jobs as well as Secrets support to the Mesos Dispatcher (SPARK-16742, SPARK-20812, respectively). However the delegation tokens have a defined expiration. This poses a problem for long running Spark jobs (e.g. Spark Streaming applications). YARN has a solution for this where a thread is scheduled to renew the tokens they reach 75% of their way to expiration. It then writes the tokens to HDFS for the executors to find (uses a monotonically increasing suffix).

## This solution
We replace the current method in `CoarseGrainedSchedulerBackend` which used to discard the token renewal time with a protected method `fetchHadoopDelegationTokens`. Now the individual cluster backends are responsible for overriding this method to fetch and manage token renewal. The delegation tokens themselves, are still part of the `CoarseGrainedSchedulerBackend` as before.
In the case of Mesos renewed Credentials are broadcasted to the executors. This maintains all transfer of Credentials within Spark (as opposed to Spark-to-HDFS). It also does not require any writing of Credentials to disk. It also does not require any GC of old files.

## How was this patch tested?
Manually against a Kerberized HDFS cluster.

Thank you for the reviews.

Author: ArtRand <arand@soe.ucsc.edu>

Closes #19272 from ArtRand/spark-21842-450-kerberos-ticket-renewal.
2017-11-15 15:53:05 -08:00
Devaraj K eaff295a23 [SPARK-22519][YARN] Remove unnecessary stagingDirPath null check in ApplicationMaster.cleanupStagingDir()
## What changes were proposed in this pull request?
Removed the unnecessary stagingDirPath null check in ApplicationMaster.cleanupStagingDir().

## How was this patch tested?
I verified with the existing test cases.

Author: Devaraj K <devaraj@apache.org>

Closes #19749 from devaraj-kavali/SPARK-22519.
2017-11-14 15:20:03 -08:00
Kent Yao c755b0d910 [SPARK-22463][YARN][SQL][HIVE] add hadoop/hive/hbase/etc configuration files in SPARK_CONF_DIR to distribute archive
## What changes were proposed in this pull request?
When I ran self contained sql apps, such as
```scala
import org.apache.spark.sql.SparkSession

object ShowHiveTables {
  def main(args: Array[String]): Unit = {
    val spark = SparkSession
      .builder()
      .appName("Show Hive Tables")
      .enableHiveSupport()
      .getOrCreate()
    spark.sql("show tables").show()
    spark.stop()
  }
}
```
with **yarn cluster** mode and `hive-site.xml` correctly within `$SPARK_HOME/conf`,they failed to connect the right hive metestore for not seeing hive-site.xml in AM/Driver's classpath.

Although submitting them with `--files/--jars local/path/to/hive-site.xml` or puting it to `$HADOOP_CONF_DIR/YARN_CONF_DIR` can make these apps works well in cluster mode as client mode, according to the official doc, see  http://spark.apache.org/docs/latest/sql-programming-guide.html#hive-tables
> Configuration of Hive is done by placing your hive-site.xml, core-site.xml (for security configuration), and hdfs-site.xml (for HDFS configuration) file in conf/.

We may respect these configuration files too or modify the doc for hive-tables in cluster mode.
## How was this patch tested?

cc cloud-fan gatorsmile

Author: Kent Yao <yaooqinn@hotmail.com>

Closes #19663 from yaooqinn/SPARK-21888.
2017-11-09 09:22:33 +01:00
Marcelo Vanzin d212ef14be [SPARK-22341][YARN] Impersonate correct user when preparing resources.
The bug was introduced in SPARK-22290, which changed how the app's user
is impersonated in the AM. The changed missed an initialization function
that needs to be run as the app owner (who has the right credentials to
read from HDFS).

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19566 from vanzin/SPARK-22341.
2017-10-25 13:53:01 -07:00
Marcelo Vanzin dc2714da50 [SPARK-22290][CORE] Avoid creating Hive delegation tokens when not necessary.
Hive delegation tokens are only needed when the Spark driver has no access
to the kerberos TGT. That happens only in two situations:

- when using a proxy user
- when using cluster mode without a keytab

This change modifies the Hive provider so that it only generates delegation
tokens in those situations, and tweaks the YARN AM so that it makes the proper
user visible to the Hive code when running with keytabs, so that the TGT
can be used instead of a delegation token.

The effect of this change is that now it's possible to initialize multiple,
non-concurrent SparkContext instances in the same JVM. Before, the second
invocation would fail to fetch a new Hive delegation token, which then could
make the second (or third or...) application fail once the token expired.
With this change, the TGT will be used to authenticate to the HMS instead.

This change also avoids polluting the current logged in user's credentials
when launching applications. The credentials are copied only when running
applications as a proxy user. This makes it possible to implement SPARK-11035
later, where multiple threads might be launching applications, and each app
should have its own set of credentials.

Tested by verifying HDFS and Hive access in following scenarios:
- client and cluster mode
- client and cluster mode with proxy user
- client and cluster mode with principal / keytab
- long-running cluster app with principal / keytab
- pyspark app that creates (and stops) multiple SparkContext instances
  through its lifetime

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19509 from vanzin/SPARK-22290.
2017-10-19 14:56:48 +08:00
Devaraj K 55d5fa79db [SPARK-21384][YARN] Spark + YARN fails with LocalFileSystem as default FS
## What changes were proposed in this pull request?

When the libraries temp directory(i.e. __spark_libs__*.zip dir) file system and staging dir(destination) file systems are the same then the __spark_libs__*.zip is not copying to the staging directory. But after making this decision the libraries zip file is getting deleted immediately and becoming unavailable for the Node Manager's localization.

With this change, client copies the files to remote always when the source scheme is "file".

## How was this patch tested?

I have verified it manually in yarn/cluster and yarn/client modes with hdfs and local file systems.

Author: Devaraj K <devaraj@apache.org>

Closes #19141 from devaraj-kavali/SPARK-21384.
2017-09-20 16:22:36 -07:00
Marcelo Vanzin bb9c0697d5 [SPARK-18838][HOTFIX][YARN] Check internal context state before stopping it.
The live listener bus now cleans up after itself and releases listeners
after stopping, so code cannot get references to listeners after the
Spark context is stopped.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19297 from vanzin/SPARK-18838.hotfix.
2017-09-20 15:53:28 -07:00
Sean Owen 12ab7f7e89 [SPARK-14280][BUILD][WIP] Update change-version.sh and pom.xml to add Scala 2.12 profiles and enable 2.12 compilation
…build; fix some things that will be warnings or errors in 2.12; restore Scala 2.12 profile infrastructure

## What changes were proposed in this pull request?

This change adds back the infrastructure for a Scala 2.12 build, but does not enable it in the release or Python test scripts.

In order to make that meaningful, it also resolves compile errors that the code hits in 2.12 only, in a way that still works with 2.11.

It also updates dependencies to the earliest minor release of dependencies whose current version does not yet support Scala 2.12. This is in a sense covered by other JIRAs under the main umbrella, but implemented here. The versions below still work with 2.11, and are the _latest_ maintenance release in the _earliest_ viable minor release.

- Scalatest 2.x -> 3.0.3
- Chill 0.8.0 -> 0.8.4
- Clapper 1.0.x -> 1.1.2
- json4s 3.2.x -> 3.4.2
- Jackson 2.6.x -> 2.7.9 (required by json4s)

This change does _not_ fully enable a Scala 2.12 build:

- It will also require dropping support for Kafka before 0.10. Easy enough, just didn't do it yet here
- It will require recreating `SparkILoop` and `Main` for REPL 2.12, which is SPARK-14650. Possible to do here too.

What it does do is make changes that resolve much of the remaining gap without affecting the current 2.11 build.

## How was this patch tested?

Existing tests and build. Manually tested with `./dev/change-scala-version.sh 2.12` to verify it compiles, modulo the exceptions above.

Author: Sean Owen <sowen@cloudera.com>

Closes #18645 from srowen/SPARK-14280.
2017-09-01 19:21:21 +01:00
jerryshao 4482ff23ad [SPARK-17321][YARN] Avoid writing shuffle metadata to disk if NM recovery is disabled
In the current code, if NM recovery is not enabled then `YarnShuffleService` will write shuffle metadata to NM local dir-1, if this local dir-1 is on bad disk, then `YarnShuffleService` will be failed to start. So to solve this issue, in Spark side if NM recovery is not enabled, then Spark will not persist data into leveldb, in that case yarn shuffle service can still be served but lose the ability for recovery, (it is fine because the failure of NM will kill the containers as well as applications).

Tested in the local cluster with NM recovery off and on to see if folder is created or not. MiniCluster UT isn't added because in MiniCluster NM will always set port to 0, but NM recovery requires non-ephemeral port.

Author: jerryshao <sshao@hortonworks.com>

Closes #19032 from jerryshao/SPARK-17321.

Change-Id: I8f2fe73d175e2ad2c4e380caede3873e0192d027
2017-08-31 09:26:20 +08:00
Andrew Ash a2db5c5761 [MINOR][TYPO] Fix typos: runnning and Excecutors
## What changes were proposed in this pull request?

Fix typos

## How was this patch tested?

Existing tests

Author: Andrew Ash <andrew@andrewash.com>

Closes #18996 from ash211/patch-2.
2017-08-18 13:43:42 -07:00
ArtRand bfdc361ede [SPARK-16742] Mesos Kerberos Support
## What changes were proposed in this pull request?

Add Kerberos Support to Mesos.   This includes kinit and --keytab support, but does not include delegation token renewal.

## How was this patch tested?

Manually against a Secure DC/OS Apache HDFS cluster.

Author: ArtRand <arand@soe.ucsc.edu>
Author: Michael Gummelt <mgummelt@mesosphere.io>

Closes #18519 from mgummelt/SPARK-16742-kerberos.
2017-08-17 15:47:07 -07:00
Marcelo Vanzin 3f958a9992 [SPARK-21731][BUILD] Upgrade scalastyle to 0.9.
This version fixes a few issues in the import order checker; it provides
better error messages, and detects more improper ordering (thus the need
to change a lot of files in this patch). The main fix is that it correctly
complains about the order of packages vs. classes.

As part of the above, I moved some "SparkSession" import in ML examples
inside the "$example on$" blocks; that didn't seem consistent across
different source files to start with, and avoids having to add more on/off blocks
around specific imports.

The new scalastyle also seems to have a better header detector, so a few
license headers had to be updated to match the expected indentation.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #18943 from vanzin/SPARK-21731.
2017-08-15 13:59:00 -07:00
Marcelo Vanzin 6735433cde [SPARK-20079][YARN] Fix client AM not allocating executors after restart.
The main goal of this change is to avoid the situation described
in the bug, where an AM restart in the middle of a job may cause
no new executors to be allocated because of faulty logic in the
reset path.

The change does two things:

- fixes the executor alloc manager's reset() so that it does not
  stop allocation after a reset() in the middle of a job
- re-orders the initialization of the YarnAllocator class so that
  it fetches the current executor ID before triggering the reset()
  above.

This ensures both that the new allocator gets new requests for executors,
and that it starts from the correct executor id.

Tested with unit tests and by manually causing AM restarts while
running jobs using spark-shell in YARN mode.

Closes #17882

Author: Marcelo Vanzin <vanzin@cloudera.com>
Author: Guoqiang Li <witgo@qq.com>

Closes #18663 from vanzin/SPARK-20079.
2017-08-01 10:12:30 -07:00
pgandhi 97ccc63f7d [SPARK-21585] Application Master marking application status as Failed for Client Mode
The fix deployed for SPARK-21541 resulted in the Application Master to set the final status of a spark application as Failed for the client mode as the flag 'registered' was not being set to true for client mode. So, in order to fix the issue, I have set the flag 'registered' as true in client mode on successfully registering Application Master.

## How was this patch tested?
Tested the patch manually.

Before:
<img width="1275" alt="screen shot-before2" src="https://user-images.githubusercontent.com/22228190/28799641-02b5ed78-760f-11e7-9eb0-bf8407dad0ad.png">

After:
<img width="1221" alt="screen shot-after2" src="https://user-images.githubusercontent.com/22228190/28799646-0ac9ef14-760f-11e7-8bf5-9dfd743d0f2f.png">

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

Author: pgandhi <pgandhi@yahoo-inc.com>
Author: pgandhi999 <parthkgandhi9@gmail.com>

Closes #18788 from pgandhi999/SPARK-21585.
2017-08-01 08:42:24 -05:00
pgandhi 69ab0e4bdd [SPARK-21541][YARN] Spark Logs show incorrect job status for a job that does not create SparkContext
If you run a spark job without creating the SparkSession or SparkContext, the spark job logs says it succeeded but yarn says it fails and retries 3 times. Also, since, Application Master unregisters with Resource Manager and exits successfully, it deletes the spark staging directory, so when yarn makes subsequent retries, it fails to find the staging directory and thus, the retries fail.

Added a flag to check whether user has initialized SparkContext. If it is true, we let Application Master unregister with Resource Manager else, we do not let AM unregister with RM.

## How was this patch tested?
Manually tested the fix.
Before:
<img width="1253" alt="screen shot-before" src="https://user-images.githubusercontent.com/22228190/28647214-69bf81e2-722b-11e7-9ed0-d416d2bf23be.png">

After:
<img width="1319" alt="screen shot-after" src="https://user-images.githubusercontent.com/22228190/28647220-70f9eea2-722b-11e7-85c6-e56276b15614.png">

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

Author: pgandhi <pgandhi@yahoo-inc.com>
Author: pgandhi999 <parthkgandhi9@gmail.com>

Closes #18741 from pgandhi999/SPARK-21541.
2017-07-28 09:23:08 -05:00
Marcelo Vanzin 300807c6e3 [SPARK-21494][NETWORK] Use correct app id when authenticating to external service.
There was some code based on the old SASL handler in the new auth client that
was incorrectly using the SASL user as the user to authenticate against the
external shuffle service. This caused the external service to not be able to
find the correct secret to authenticate the connection, failing the connection.

In the course of debugging, I found that some log messages from the YARN shuffle
service were a little noisy, so I silenced some of them, and also added a couple
of new ones that helped find this issue. On top of that, I found that a check
in the code that records app secrets was wrong, causing more log spam and also
using an O(n) operation instead of an O(1) call.

Also added a new integration suite for the YARN shuffle service with auth on,
and verified it failed before, and passes now.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #18706 from vanzin/SPARK-21494.
2017-07-25 17:57:26 -07:00
DjvuLee 8de080d9f9 [SPARK-21383][YARN] Fix the YarnAllocator allocates more Resource
When NodeManagers launching Executors,
the `missing` value will exceed the
real value when the launch is slow, this can lead to YARN allocates more resource.

We add the `numExecutorsRunning` when calculate the `missing` to avoid this.

Test by experiment.

Author: DjvuLee <lihu@bytedance.com>

Closes #18651 from djvulee/YarnAllocate.
2017-07-25 10:21:25 -07:00
Marcelo Vanzin 264b0f36ce [SPARK-21408][CORE] Better default number of RPC dispatch threads.
Instead of using the host's cpu count, use the number of cores allocated
for the Spark process when sizing the RPC dispatch thread pool. This avoids
creating large thread pools on large machines when the number of allocated
cores is small.

Tested by verifying number of threads with spark.executor.cores set
to 1 and 4; same thing for YARN AM.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #18639 from vanzin/SPARK-21408.
2017-07-18 13:36:10 -07:00
jerryshao cde64add18 [SPARK-21411][YARN] Lazily create FS within kerberized UGI to avoid token acquiring failure
## What changes were proposed in this pull request?

In the current `YARNHadoopDelegationTokenManager`, `FileSystem` to which to get tokens are created out of KDC logged UGI, using these `FileSystem` to get new tokens will lead to exception. The main thing is that Spark code trying to get new tokens from the FS created with token auth-ed UGI, but Hadoop can only grant new tokens in kerberized UGI. To fix this issue, we should lazily create these FileSystem within KDC logged UGI.

## How was this patch tested?

Manual verification in secure cluster.

CC vanzin mgummelt please help to review, thanks!

Author: jerryshao <sshao@hortonworks.com>

Closes #18633 from jerryshao/SPARK-21411.
2017-07-18 11:44:01 -07:00
jerryshao 53465075c5 [SPARK-21377][YARN] Make jars specify with --jars/--packages load-able in AM's credential renwer
## What changes were proposed in this pull request?

In this issue we have a long running Spark application with secure HBase, which requires `HBaseCredentialProvider` to get tokens periodically, we specify HBase related jars with `--packages`, but these dependencies are not added into AM classpath, so when `HBaseCredentialProvider` tries to initialize HBase connections to get tokens, it will be failed.

Currently because jars specified with `--jars` or `--packages` are not added into AM classpath, the only way to extend AM classpath is to use "spark.driver.extraClassPath" which supposed to be used in yarn cluster mode.

So in this fix, we proposed to use/reuse a classloader for `AMCredentialRenewer` to acquire new tokens.

Also in this patch, we fixed AM cannot get tokens from HDFS issue, it is because FileSystem is gotten before kerberos logged, so using this FS to get tokens will throw exception.

## How was this patch tested?

Manual verification.

Author: jerryshao <sshao@hortonworks.com>

Closes #18616 from jerryshao/SPARK-21377.
2017-07-17 13:11:30 -07:00
John Lee 0e07a29cf4 [SPARK-21321][SPARK CORE] Spark very verbose on shutdown
## What changes were proposed in this pull request?

The current code is very verbose on shutdown.

The changes I propose is to change the log level when the driver is shutting down and the RPC connections are closed (RpcEnvStoppedException).

## How was this patch tested?

Tested with word count(deploy-mode = cluster, master = yarn, num-executors = 4) with 300GB of data.

Author: John Lee <jlee2@yahoo-inc.com>

Closes #18547 from yoonlee95/SPARK-21321.
2017-07-17 13:13:35 -05:00
Marcelo Vanzin 601a237b30 [SPARK-9825][YARN] Do not overwrite final Hadoop config entries.
When localizing the gateway config files in a YARN application, avoid
overwriting final configs by distributing the gateway files to a separate
directory, and explicitly loading them into the Hadoop config, instead
of placing those files before the cluster's files in the classpath.

This is done by saving the gateway's config to a separate XML file
distributed with the rest of the Spark app's config, and loading that
file when creating a new config through `YarnSparkHadoopUtil`.

Tested with existing unit tests, and by verifying the behavior in a YARN
cluster (final values are not overridden, non-final values are).

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #18370 from vanzin/SPARK-9825.
2017-07-14 14:32:19 -07:00
jerryshao cb8d5cc90f [SPARK-21376][YARN] Fix yarn client token expire issue when cleaning the staging files in long running scenario
## What changes were proposed in this pull request?

This issue happens in long running application with yarn cluster mode, because yarn#client doesn't sync token with AM, so it will always keep the initial token, this token may be expired in the long running scenario, so when yarn#client tries to clean up staging directory after application finished, it will use this expired token and meet token expire issue.

## How was this patch tested?

Manual verification is secure cluster.

Author: jerryshao <sshao@hortonworks.com>

Closes #18617 from jerryshao/SPARK-21376.
2017-07-13 15:25:38 -07:00
Marcelo Vanzin 1cad31f006 [SPARK-16019][YARN] Use separate RM poll interval when starting client AM.
Currently the code monitoring the launch of the client AM uses the value of
spark.yarn.report.interval as the interval for polling the RM; if someone
has that value to a really large interval, it would take that long to detect
that the client AM has started, which is not expected.

Instead, have a separate config for the interval to use when the client AM is
starting. The other config is still used in cluster mode, and to detect the
status of the client AM after it is already running.

Tested by running client and cluster mode apps with a modified value of
spark.yarn.report.interval, verifying client AM launch is detected before
that interval elapses.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #18380 from vanzin/SPARK-16019.
2017-07-11 11:25:40 -07:00
Dongjoon Hyun c8d0aba198 [SPARK-21278][PYSPARK] Upgrade to Py4J 0.10.6
## What changes were proposed in this pull request?

This PR aims to bump Py4J in order to fix the following float/double bug.
Py4J 0.10.5 fixes this (https://github.com/bartdag/py4j/issues/272) and the latest Py4J is 0.10.6.

**BEFORE**
```
>>> df = spark.range(1)
>>> df.select(df['id'] + 17.133574204226083).show()
+--------------------+
|(id + 17.1335742042)|
+--------------------+
|       17.1335742042|
+--------------------+
```

**AFTER**
```
>>> df = spark.range(1)
>>> df.select(df['id'] + 17.133574204226083).show()
+-------------------------+
|(id + 17.133574204226083)|
+-------------------------+
|       17.133574204226083|
+-------------------------+
```

## How was this patch tested?

Manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #18546 from dongjoon-hyun/SPARK-21278.
2017-07-05 16:33:23 -07:00
sharkdtu 3d4d11a80f [SPARK-21138][YARN] Cannot delete staging dir when the clusters of "spark.yarn.stagingDir" and "spark.hadoop.fs.defaultFS" are different
## What changes were proposed in this pull request?

When I set different clusters for "spark.hadoop.fs.defaultFS" and "spark.yarn.stagingDir" as follows:
```
spark.hadoop.fs.defaultFS  hdfs://tl-nn-tdw.tencent-distribute.com:54310
spark.yarn.stagingDir hdfs://ss-teg-2-v2/tmp/spark
```
The staging dir can not be deleted, it will prompt following message:
```
java.lang.IllegalArgumentException: Wrong FS: hdfs://ss-teg-2-v2/tmp/spark/.sparkStaging/application_1496819138021_77618, expected: hdfs://tl-nn-tdw.tencent-distribute.com:54310
```

## How was this patch tested?

Existing tests

Author: sharkdtu <sharkdtu@tencent.com>

Closes #18352 from sharkdtu/master.
2017-06-19 14:54:54 -07:00
Michael Gummelt a18d637112 [SPARK-20434][YARN][CORE] Move Hadoop delegation token code from yarn to core
## What changes were proposed in this pull request?

Move Hadoop delegation token code from `spark-yarn` to `spark-core`, so that other schedulers (such as Mesos), may use it.  In order to avoid exposing Hadoop interfaces in spark-core, the new Hadoop delegation token classes are kept private.  In order to provider backward compatiblity, and to allow YARN users to continue to load their own delegation token providers via Java service loading, the old YARN interfaces, as well as the client code that uses them, have been retained.

Summary:
- Move registered `yarn.security.ServiceCredentialProvider` classes from `spark-yarn` to `spark-core`.  Moved them into a new, private hierarchy under `HadoopDelegationTokenProvider`.  Client code in `HadoopDelegationTokenManager` now loads credentials from a whitelist of three providers (`HadoopFSDelegationTokenProvider`, `HiveDelegationTokenProvider`, `HBaseDelegationTokenProvider`), instead of service loading, which means that users are not able to implement their own delegation token providers, as they are in the `spark-yarn` module.

- The `yarn.security.ServiceCredentialProvider` interface has been kept for backwards compatibility, and to continue to allow YARN users to implement their own delegation token provider implementations.  Client code in YARN now fetches tokens via the new `YARNHadoopDelegationTokenManager` class, which fetches tokens from the core providers through `HadoopDelegationTokenManager`, as well as service loads them from `yarn.security.ServiceCredentialProvider`.

Old Hierarchy:

```
yarn.security.ServiceCredentialProvider (service loaded)
  HadoopFSCredentialProvider
  HiveCredentialProvider
  HBaseCredentialProvider
yarn.security.ConfigurableCredentialManager
```

New Hierarchy:

```
HadoopDelegationTokenManager
HadoopDelegationTokenProvider (not service loaded)
  HadoopFSDelegationTokenProvider
  HiveDelegationTokenProvider
  HBaseDelegationTokenProvider

yarn.security.ServiceCredentialProvider (service loaded)
yarn.security.YARNHadoopDelegationTokenManager
```
## How was this patch tested?

unit tests

Author: Michael Gummelt <mgummelt@mesosphere.io>
Author: Dr. Stefan Schimanski <sttts@mesosphere.io>

Closes #17723 from mgummelt/SPARK-20434-refactor-kerberos.
2017-06-15 11:46:00 -07:00
Li Yichao 640afa49aa [SPARK-20365][YARN] Remove local scheme when add path to ClassPath.
In Spark on YARN, when configuring "spark.yarn.jars" with local jars (jars started with "local" scheme), we will get inaccurate classpath for AM and containers. This is because we don't remove "local" scheme when concatenating classpath. It is OK to run because classpath is separated with ":" and java treat "local" as a separate jar. But we could improve it to remove the scheme.

Updated `ClientSuite` to check "local" is not in the classpath.

cc jerryshao

Author: Li Yichao <lyc@zhihu.com>
Author: Li Yichao <liyichao.good@gmail.com>

Closes #18129 from liyichao/SPARK-20365.
2017-06-01 14:40:05 -07:00
Lior Regev 7306d55690 [SPARK-20741][SPARK SUBMIT] Added cleanup of JARs archive generated by SparkSubmit
## What changes were proposed in this pull request?

Deleted generated JARs archive after distribution to HDFS

## How was this patch tested?

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

Author: Lior Regev <lioregev@gmail.com>

Closes #17986 from liorregev/master.
2017-05-25 17:08:19 +01:00
jerryshao 829cd7b8b7 [SPARK-20605][CORE][YARN][MESOS] Deprecate not used AM and executor port configuration
## What changes were proposed in this pull request?

After SPARK-10997, client mode Netty RpcEnv doesn't require to start server, so port configurations are not used any more, here propose to remove these two configurations: "spark.executor.port" and "spark.am.port".

## How was this patch tested?

Existing UTs.

Author: jerryshao <sshao@hortonworks.com>

Closes #17866 from jerryshao/SPARK-20605.
2017-05-08 14:27:56 -07:00
Xianyang Liu aeb2ecc0cd [SPARK-20621][DEPLOY] Delete deprecated config parameter in 'spark-env.sh'
## What changes were proposed in this pull request?

Currently, `spark.executor.instances` is deprecated in `spark-env.sh`, because we suggest config it in `spark-defaults.conf` or other config file. And also this parameter is useless even if you set it in `spark-env.sh`, so remove it in this patch.

## How was this patch tested?

Existing tests.

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

Author: Xianyang Liu <xianyang.liu@intel.com>

Closes #17881 from ConeyLiu/deprecatedParam.
2017-05-08 10:25:24 -07:00
Sean Owen 16fab6b0ef [SPARK-20523][BUILD] Clean up build warnings for 2.2.0 release
## What changes were proposed in this pull request?

Fix build warnings primarily related to Breeze 0.13 operator changes, Java style problems

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #17803 from srowen/SPARK-20523.
2017-05-03 10:18:35 +01:00
Mark Grover 66636ef0b0 [SPARK-20435][CORE] More thorough redaction of sensitive information
This change does a more thorough redaction of sensitive information from logs and UI
Add unit tests that ensure that no regressions happen that leak sensitive information to the logs.

The motivation for this change was appearance of password like so in `SparkListenerEnvironmentUpdate` in event logs under some JVM configurations:
`"sun.java.command":"org.apache.spark.deploy.SparkSubmit ... --conf spark.executorEnv.HADOOP_CREDSTORE_PASSWORD=secret_password ..."
`
Previously redaction logic was only checking if the key matched the secret regex pattern, it'd redact it's value. That worked for most cases. However, in the above case, the key (sun.java.command) doesn't tell much, so the value needs to be searched. This PR expands the check to check for values as well.

## How was this patch tested?

New unit tests added that ensure that no sensitive information is present in the event logs or the yarn logs. Old unit test in UtilsSuite was modified because the test was asserting that a non-sensitive property's value won't be redacted. However, the non-sensitive value had the literal "secret" in it which was causing it to redact. Simply updating the non-sensitive property's value to another arbitrary value (that didn't have "secret" in it) fixed it.

Author: Mark Grover <mark@apache.org>

Closes #17725 from markgrover/spark-20435.
2017-04-26 17:06:21 -07:00
Josh Rosen f44c8a843c [SPARK-20453] Bump master branch version to 2.3.0-SNAPSHOT
This patch bumps the master branch version to `2.3.0-SNAPSHOT`.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #17753 from JoshRosen/SPARK-20453.
2017-04-24 21:48:04 -07:00
Andrew Ash 0075562dd2 Typo fix: distitrbuted -> distributed
## What changes were proposed in this pull request?

Typo fix: distitrbuted -> distributed

## How was this patch tested?

Existing tests

Author: Andrew Ash <andrew@andrewash.com>

Closes #17664 from ash211/patch-1.
2017-04-17 17:56:33 -07:00
Sean Owen a26e3ed5e4 [SPARK-20156][CORE][SQL][STREAMING][MLLIB] Java String toLowerCase "Turkish locale bug" causes Spark problems
## What changes were proposed in this pull request?

Add Locale.ROOT to internal calls to String `toLowerCase`, `toUpperCase`, to avoid inadvertent locale-sensitive variation in behavior (aka the "Turkish locale problem").

The change looks large but it is just adding `Locale.ROOT` (the locale with no country or language specified) to every call to these methods.

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #17527 from srowen/SPARK-20156.
2017-04-10 20:11:56 +01:00
Marcelo Vanzin 0736980f39 [SPARK-20191][YARN] Crate wrapper for RackResolver so tests can override it.
Current test code tries to override the RackResolver used by setting
configuration params, but because YARN libs statically initialize the
resolver the first time it's used, that means that those configs don't
really take effect during Spark tests.

This change adds a wrapper class that easily allows tests to override the
behavior of the resolver for the Spark code that uses it.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #17508 from vanzin/SPARK-20191.
2017-04-04 11:38:05 -07:00
jerryshao c622a87c44 [SPARK-20059][YARN] Use the correct classloader for HBaseCredentialProvider
## What changes were proposed in this pull request?

Currently we use system classloader to find HBase jars, if it is specified by `--jars`, then it will be failed with ClassNotFound issue. So here changing to use child classloader.

Also putting added jars and main jar into classpath of submitted application in yarn cluster mode, otherwise HBase jars specified with `--jars` will never be honored in cluster mode, and fetching tokens in client side will always be failed.

## How was this patch tested?

Unit test and local verification.

Author: jerryshao <sshao@hortonworks.com>

Closes #17388 from jerryshao/SPARK-20059.
2017-03-29 10:09:58 -07:00
jerryshao 17eddb35a2 [SPARK-19995][YARN] Register tokens to current UGI to avoid re-issuing of tokens in yarn client mode
## What changes were proposed in this pull request?

In the current Spark on YARN code, we will obtain tokens from provided services, but we're not going to add these tokens to the current user's credentials. This will make all the following operations to these services still require TGT rather than delegation tokens. This is unnecessary since we already got the tokens, also this will lead to failure in user impersonation scenario, because the TGT is granted by real user, not proxy user.

So here changing to put all the tokens to the current UGI, so that following operations to these services will honor tokens rather than TGT, and this will further handle the proxy user issue mentioned above.

## How was this patch tested?

Local verified in secure cluster.

vanzin tgravescs mridulm  dongjoon-hyun please help to review, thanks a lot.

Author: jerryshao <sshao@hortonworks.com>

Closes #17335 from jerryshao/SPARK-19995.
2017-03-28 10:41:11 -07:00
Juan Rodriguez Hortala 362ee93296 logging improvements
## What changes were proposed in this pull request?
Adding additional information to existing logging messages:
  - YarnAllocator: log the executor ID together with the container id when a container for an executor is launched.
  - NettyRpcEnv: log the receiver address when there is a timeout waiting for an answer to a remote call.
  - ExecutorAllocationManager: fix a typo in the logging message for the list of executors to be removed.

## How was this patch tested?
Build spark and submit the word count example to a YARN cluster using cluster mode

Author: Juan Rodriguez Hortala <hortala@amazon.com>

Closes #17411 from juanrh/logging-improvements.
2017-03-26 10:39:05 +01:00
Yong Tang 8f0490e22b [SPARK-17979][SPARK-14453] Remove deprecated SPARK_YARN_USER_ENV and SPARK_JAVA_OPTS
This fix removes deprecated support for config `SPARK_YARN_USER_ENV`, as is mentioned in SPARK-17979.
This fix also removes deprecated support for the following:
```
SPARK_YARN_USER_ENV
SPARK_JAVA_OPTS
SPARK_CLASSPATH
SPARK_WORKER_INSTANCES
```

Related JIRA:
[SPARK-14453]: https://issues.apache.org/jira/browse/SPARK-14453
[SPARK-12344]: https://issues.apache.org/jira/browse/SPARK-12344
[SPARK-15781]: https://issues.apache.org/jira/browse/SPARK-15781

Existing tests should pass.

Author: Yong Tang <yong.tang.github@outlook.com>

Closes #17212 from yongtang/SPARK-17979.
2017-03-10 13:34:01 -08:00
Marcelo Vanzin 8e41c2eed8 [SPARK-19857][YARN] Correctly calculate next credential update time.
Add parentheses so that both lines form a single statement; also add
a log message so that the issue becomes more explicit if it shows up
again.

Tested manually with integration test that exercises the feature.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #17198 from vanzin/SPARK-19857.
2017-03-07 16:21:18 -08:00
Jeff Zhang 330c3e33bd [SPARK-13330][PYSPARK] PYTHONHASHSEED is not propgated to python worker
## What changes were proposed in this pull request?
self.environment will be propagated to executor. Should set PYTHONHASHSEED as long as the python version is greater than 3.3

## How was this patch tested?
Manually tested it.

Author: Jeff Zhang <zjffdu@apache.org>

Closes #11211 from zjffdu/SPARK-13330.
2017-02-24 15:04:42 -08:00
jerryshao a920a43694 [SPARK-19038][YARN] Avoid overwriting keytab configuration in yarn-client
## What changes were proposed in this pull request?

Because yarn#client will reset the `spark.yarn.keytab` configuration to point to the location in distributed file, so if user still uses the old `SparkConf` to create `SparkSession` with Hive enabled, it will read keytab from the path in distributed cached. This is OK for yarn cluster mode, but in yarn client mode where driver is running out of container, it will be failed to fetch the keytab.

So here we should avoid reseting this configuration in the `yarn#client` and only overwriting it for AM, so using `spark.yarn.keytab` could get correct keytab path no matter running in client (keytab in local fs) or cluster (keytab in distributed cache) mode.

## How was this patch tested?

Verified in security cluster.

Author: jerryshao <sshao@hortonworks.com>

Closes #16923 from jerryshao/SPARK-19038.
2017-02-24 09:31:52 -08:00
Marcelo Vanzin 4661d30b98 [SPARK-19554][UI,YARN] Allow SHS URL to be used for tracking in YARN RM.
Allow an application to use the History Server URL as the tracking
URL in the YARN RM, so there's still a link to the web UI somewhere
in YARN even if the driver's UI is disabled. This is useful, for
example, if an admin wants to disable the driver UI by default for
applications, since it's harder to secure it (since it involves non
trivial ssl certificate and auth management that admins may not want
to expose to user apps).

This needs to be opt-in, because of the way the YARN proxy works, so
a new configuration was added to enable the option.

The YARN RM will proxy requests to live AMs instead of redirecting
the client, so pages in the SHS UI will not render correctly since
they'll reference invalid paths in the RM UI. The proxy base support
in the SHS cannot be used since that would prevent direct access to
the SHS.

So, to solve this problem, for the feature to work end-to-end, a new
YARN-specific filter was added that detects whether the requests come
from the proxy and redirects the client appropriatly. The SHS admin has
to add this filter manually if they want the feature to work.

Tested with new unit test, and by running with the documented configuration
set in a test cluster. Also verified the driver UI is used when it's
enabled.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #16946 from vanzin/SPARK-19554.
2017-02-22 14:37:53 -08:00
Kent Yao 7363dde634 [SPARK-19626][YARN] Using the correct config to set credentials update time
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/14065, we introduced a configurable credential manager for Spark running on YARN. Also two configs `spark.yarn.credentials.renewalTime` and `spark.yarn.credentials.updateTime` were added, one is for the credential renewer and the other updater. But now we just query `spark.yarn.credentials.renewalTime` by mistake during CREDENTIALS UPDATING, where should be actually `spark.yarn.credentials.updateTime` .

This PR fixes this mistake.

## How was this patch tested?

existing test

cc jerryshao vanzin

Author: Kent Yao <yaooqinn@hotmail.com>

Closes #16955 from yaooqinn/cred_update.
2017-02-21 09:57:40 -08:00
jinxing ba8912e5f3
[SPARK-19450] Replace askWithRetry with askSync.
## What changes were proposed in this pull request?

`askSync` is already added in `RpcEndpointRef` (see SPARK-19347 and https://github.com/apache/spark/pull/16690#issuecomment-276850068) and `askWithRetry` is marked as deprecated.
As mentioned SPARK-18113(https://github.com/apache/spark/pull/16503#event-927953218):

>askWithRetry is basically an unneeded API, and a leftover from the akka days that doesn't make sense anymore. It's prone to cause deadlocks (exactly because it's blocking), it imposes restrictions on the caller (e.g. idempotency) and other things that people generally don't pay that much attention to when using it.

Since `askWithRetry` is just used inside spark and not in user logic. It might make sense to replace all of them with `askSync`.

## How was this patch tested?
This PR doesn't change code logic, existing unit test can cover.

Author: jinxing <jinxing@meituan.com>

Closes #16790 from jinxing64/SPARK-19450.
2017-02-19 04:34:07 -08:00
Sean Owen 0e2405490f
[SPARK-19550][BUILD][CORE][WIP] Remove Java 7 support
- Move external/java8-tests tests into core, streaming, sql and remove
- Remove MaxPermGen and related options
- Fix some reflection / TODOs around Java 8+ methods
- Update doc references to 1.7/1.8 differences
- Remove Java 7/8 related build profiles
- Update some plugins for better Java 8 compatibility
- Fix a few Java-related warnings

For the future:

- Update Java 8 examples to fully use Java 8
- Update Java tests to use lambdas for simplicity
- Update Java internal implementations to use lambdas

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #16871 from srowen/SPARK-19493.
2017-02-16 12:32:45 +00:00
Jong Wook Kim ab9872db1f [SPARK-19501][YARN] Reduce the number of HDFS RPCs during YARN deployment
## What changes were proposed in this pull request?

As discussed in [JIRA](https://issues.apache.org/jira/browse/SPARK-19501), this patch addresses the problem where too many HDFS RPCs are made when there are many URIs specified in `spark.yarn.jars`, potentially adding hundreds of RTTs to YARN before the application launches. This becomes significant when submitting the application to a non-local YARN cluster (where the RTT may be in order of 100ms, for example). For each URI specified, the current implementation makes at least two HDFS RPCs, for:

- [Calling `getFileStatus()` before uploading each file to the distributed cache in `ClientDistributedCacheManager.addResource()`](https://github.com/apache/spark/blob/v2.1.0/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala#L71).
- [Resolving any symbolic links in each of the file URI](https://github.com/apache/spark/blob/v2.1.0/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala#L377-L379), which repeatedly makes HDFS RPCs until the all symlinks are resolved. (see [`FileContext.resolve(Path)`](https://github.com/apache/hadoop/blob/release-2.7.1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java#L2189-L2195), [`FSLinkResolver.resolve(FileContext, Path)`](https://github.com/apache/hadoop/blob/release-2.7.1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSLinkResolver.java#L79-L112), and [`AbstractFileSystem.resolvePath()`](https://github.com/apache/hadoop/blob/release-2.7.1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java#L464-L468).)

The first `getFileStatus` RPC can be removed, using `statCache` populated with the file statuses retrieved with [the previous `globStatus` call](https://github.com/apache/spark/blob/v2.1.0/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala#L531).

The second one can be largely reduced by caching the symlink resolution results in a mutable.HashMap. This patch adds a local variable in `yarn.Client.prepareLocalResources()` and passes it as an additional parameter to `yarn.Client.copyFileToRemote`.  [The symlink resolution code was added in 2013](a35472e1dd (diff-b050df3f55b82065803d6e83453b9706R187)) and has not changed since. I am assuming that this is still required, but otherwise we can remove using `symlinkCache` and symlink resolution altogether.

## How was this patch tested?

This patch is based off 8e8afb3, currently the latest YARN patch on master. All tests except a few in spark-hive passed with `./dev/run-tests` on my machine, using JDK 1.8.0_112 on macOS 10.12.3; also tested myself with this modified version of SPARK 2.2.0-SNAPSHOT which performed a normal deployment and execution on a YARN cluster without errors.

Author: Jong Wook Kim <jongwook@nyu.edu>

Closes #16916 from jongwook/SPARK-19501.
2017-02-14 11:33:31 -08:00
jerryshao 8e8afb3a34
[SPARK-19545][YARN] Fix compile issue for Spark on Yarn when building against Hadoop 2.6.0~2.6.3
## What changes were proposed in this pull request?

Due to the newly added API in Hadoop 2.6.4+, Spark builds against Hadoop 2.6.0~2.6.3 will meet compile error. So here still reverting back to use reflection to handle this issue.

## How was this patch tested?

Manual verification.

Author: jerryshao <sshao@hortonworks.com>

Closes #16884 from jerryshao/SPARK-19545.
2017-02-10 13:44:26 +00:00
Sean Owen 15627ac743
[SPARK-19464][BUILD][HOTFIX][TEST-HADOOP2.6] Add back mockito test dep in YARN module, as it ends up being required in a Maven build
Add back mockito test dep in YARN module, as it ends up being required in a Maven build

## How was this patch tested?

PR builder again, but also a local `mvn` run using the command that the broken Jenkins job uses

Author: Sean Owen <sowen@cloudera.com>

Closes #16853 from srowen/SPARK-19464.2.
2017-02-08 17:27:20 +00:00
Sean Owen e8d3fca450
[SPARK-19464][CORE][YARN][TEST-HADOOP2.6] Remove support for Hadoop 2.5 and earlier
## What changes were proposed in this pull request?

- Remove support for Hadoop 2.5 and earlier
- Remove reflection and code constructs only needed to support multiple versions at once
- Update docs to reflect newer versions
- Remove older versions' builds and profiles.

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #16810 from srowen/SPARK-19464.
2017-02-08 12:20:07 +00:00
Marcelo Vanzin 76db394f2b [SPARK-18750][YARN] Avoid using "mapValues" when allocating containers.
That method is prone to stack overflows when the input map is really
large; instead, use plain "map". Also includes a unit test that was
tested and caused stack overflows without the fix.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #16667 from vanzin/SPARK-18750.
2017-01-25 08:18:41 -06:00
uncleGen eefdf9f9dd
[SPARK-19227][SPARK-19251] remove unused imports and outdated comments
## What changes were proposed in this pull request?
remove ununsed imports and outdated comments, and fix some minor code style issue.

## How was this patch tested?
existing ut

Author: uncleGen <hustyugm@gmail.com>

Closes #16591 from uncleGen/SPARK-19227.
2017-01-18 09:44:32 +00:00
jerryshao b79cc7ceb4 [SPARK-19179][YARN] Change spark.yarn.access.namenodes config and update docs
## What changes were proposed in this pull request?

`spark.yarn.access.namenodes` configuration cannot actually reflects the usage of it, inside the code it is the Hadoop filesystems we get tokens, not NNs. So here propose to update the name of this configuration, also change the related code and doc.

## How was this patch tested?

Local verification.

Author: jerryshao <sshao@hortonworks.com>

Closes #16560 from jerryshao/SPARK-19179.
2017-01-17 09:30:56 -06:00
Yanbo Liang 84f0b645b4 [MINOR][YARN] Move YarnSchedulerBackendSuite to resource-managers/yarn directory.
## What changes were proposed in this pull request?
#16092 moves YARN resource manager related code to resource-managers/yarn directory. The test case ```YarnSchedulerBackendSuite``` was added after that but with the wrong place. I move it to correct directory in this PR.

## How was this patch tested?
Existing test.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #16595 from yanboliang/yarn.
2017-01-17 00:28:12 -08:00
jerryshao 4239a1081a [SPARK-19021][YARN] Generailize HDFSCredentialProvider to support non HDFS security filesystems
Currently Spark can only get token renewal interval from security HDFS (hdfs://), if Spark runs with other security file systems like webHDFS (webhdfs://), wasb (wasb://), ADLS, it will ignore these tokens and not get token renewal intervals from these tokens. These will make Spark unable to work with these security clusters. So instead of only checking HDFS token, we should generalize to support different DelegationTokenIdentifier.

## How was this patch tested?

Manually verified in security cluster.

Author: jerryshao <sshao@hortonworks.com>

Closes #16432 from jerryshao/SPARK-19021.
2017-01-11 09:24:02 -06:00
Niranjan Padmanabhan a1e40b1f5d
[MINOR][DOCS] Remove consecutive duplicated words/typo in Spark Repo
## What changes were proposed in this pull request?
There are many locations in the Spark repo where the same word occurs consecutively. Sometimes they are appropriately placed, but many times they are not. This PR removes the inappropriately duplicated words.

## How was this patch tested?
N/A since only docs or comments were updated.

Author: Niranjan Padmanabhan <niranjan.padmanabhan@gmail.com>

Closes #16455 from neurons/np.structure_streaming_doc.
2017-01-04 15:07:29 +00:00
mingfei fe1c895e16
[SPARK-19073] LauncherState should be only set to SUBMITTED after the application is submitted
## What changes were proposed in this pull request?
LauncherState should be only set to SUBMITTED after the application is submitted.
Currently the state is set before the application is actually submitted.

## How was this patch tested?
no test is added in this patch

Author: mingfei <mingfei.smf@alipay.com>

Closes #16459 from shimingfei/fixLauncher.
2017-01-04 10:27:51 +00:00
Liang-Chi Hsieh 0ac2f1e71f
[MINOR][DOC] Minor doc change for YARN credential providers
## What changes were proposed in this pull request?

The configuration `spark.yarn.security.tokens.{service}.enabled` is deprecated. Now we should use `spark.yarn.security.credentials.{service}.enabled`. Some places in the doc is not updated yet.

## How was this patch tested?

N/A. Just doc change.

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

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

Closes #16444 from viirya/minor-credential-provider-doc.
2017-01-02 14:41:57 +00:00
Ryan Williams afd9bc1d8a [SPARK-17807][CORE] split test-tags into test-JAR
Remove spark-tag's compile-scope dependency (and, indirectly, spark-core's compile-scope transitive-dependency) on scalatest by splitting test-oriented tags into spark-tags' test JAR.

Alternative to #16303.

Author: Ryan Williams <ryan.blake.williams@gmail.com>

Closes #16311 from ryan-williams/tt.
2016-12-21 16:37:20 -08:00
Imran Rashid 32ff964526 [SPARK-8425][SCHEDULER][HOTFIX] fix scala 2.10 compile error
## What changes were proposed in this pull request?

93cdb8a7d0 Introduced a compile error under scala 2.10, this fixes that error.

## How was this patch tested?

locally ran
```
dev/change-version-to-2.10.sh
build/sbt -Pyarn -Phadoop-2.4 -Dhadoop.version=2.6.0 -Dscala-2.10 "project yarn" "test-only *YarnAllocatorSuite"
```
(which failed at test compilation before this change)

Author: Imran Rashid <irashid@cloudera.com>

Closes #16298 from squito/blacklist-2.10.
2016-12-15 15:36:48 -08:00
Imran Rashid 93cdb8a7d0 [SPARK-8425][CORE] Application Level Blacklisting
## What changes were proposed in this pull request?

This builds upon the blacklisting introduced in SPARK-17675 to add blacklisting of executors and nodes for an entire Spark application.  Resources are blacklisted based on tasks that fail, in tasksets that eventually complete successfully; they are automatically returned to the pool of active resources based on a timeout.  Full details are available in a design doc attached to the jira.
## How was this patch tested?

Added unit tests, ran them via Jenkins, also ran a handful of them in a loop to check for flakiness.

The added tests include:
- verifying BlacklistTracker works correctly
- verifying TaskSchedulerImpl interacts with BlacklistTracker correctly (via a mock BlacklistTracker)
- an integration test for the entire scheduler with blacklisting in a few different scenarios

Author: Imran Rashid <irashid@cloudera.com>
Author: mwws <wei.mao@intel.com>

Closes #14079 from squito/blacklist-SPARK-8425.
2016-12-15 08:29:56 -06:00
jerryshao 43298d157d [SPARK-18840][YARN] Avoid throw exception when getting token renewal interval in non HDFS security environment
## What changes were proposed in this pull request?

Fix `java.util.NoSuchElementException` when running Spark in non-hdfs security environment.

In the current code, we assume `HDFS_DELEGATION_KIND` token will be found in Credentials. But in some cloud environments, HDFS is not required, so we should avoid this exception.

## How was this patch tested?

Manually verified in local environment.

Author: jerryshao <sshao@hortonworks.com>

Closes #16265 from jerryshao/SPARK-18840.
2016-12-13 10:37:45 -08:00
Anirudh 81e5619ca1 [SPARK-18662] Move resource managers to separate directory
## What changes were proposed in this pull request?

* Moves yarn and mesos scheduler backends to resource-managers/ sub-directory (in preparation for https://issues.apache.org/jira/browse/SPARK-18278)
* Corresponding change in top-level pom.xml.

Ref: https://github.com/apache/spark/pull/16061#issuecomment-263649340

## How was this patch tested?

* Manual tests

/cc rxin

Author: Anirudh <ramanathana@google.com>

Closes #16092 from foxish/fix-scheduler-structure-2.
2016-12-06 16:23:27 -08:00