Don't ignore spark.cores.max when using Mesos Coarse mode
totalCoresAcquired is decremented but never incremented, causing Spark to effectively ignore spark.cores.max in coarse grained Mesos mode.
Migrate the daemon thread started by DAGScheduler to Akka actor
`DAGScheduler` adopts an event queue and a daemon thread polling the it to process events sent to a `DAGScheduler`. This is a classical actor use case. By migrating this thread to Akka actor, we may benefit from both cleaner code and better performance (context switching cost of Akka actor is much less than that of a native thread).
But things become a little complicated when taking existing test code into consideration.
Code in `DAGSchedulerSuite` is somewhat tightly coupled with `DAGScheduler`, and directly calls `DAGScheduler.processEvent` instead of posting event messages to `DAGScheduler`. To minimize code change, I chose to let the actor to delegate messages to `processEvent`. Maybe this doesn't follow conventional actor usage, but I tried to make it apparently correct.
Another tricky part is that, since `DAGScheduler` depends on the `ActorSystem` provided by its field `env`, `env` cannot be null. But the `dagScheduler` field created in `DAGSchedulerSuite.before` was given a null `env`. What's more, `BlockManager.blockIdsToBlockManagers` checks whether `env` is null to determine whether to run the production code or the test code (bad smell here, huh?). I went through all callers of `BlockManager.blockIdsToBlockManagers`, and made sure that if `env != null` holds, then `blockManagerMaster == null` must also hold. That's the logic behind `BlockManager.scala` [line 896](https://github.com/liancheng/incubator-spark/compare/dagscheduler-actor-refine?expand=1#diff-2b643ea78c1add0381754b1f47eec132L896).
At last, since `DAGScheduler` instances are always `start()`ed after creation, I removed the `start()` method, and starts the `eventProcessActor` within the constructor.
spark-assembly.jar fails to authenticate with YARN ResourceManager
The META-INF/services/ sbt MergeStrategy was discarding support for Kerberos, among others. This pull request changes to a merge strategy similar to sbt-assembly's default. I've also included an update to sbt-assembly 0.9.2, a minor fix to it's zip file handling.
Allow spark on yarn to be run from HDFS.
Allows the spark.jar, app.jar, and log4j.properties to be put into hdfs. Allows you to specify the files on a different hdfs cluster and it will copy them over. It makes sure permissions are correct and makes sure to put things into public distributed cache so they can be reused amongst users if their permissions are appropriate. Also add a bit of error handling for missing arguments.
Enable stopping and starting a spot cluster
Clusters launched using `--spot-price` contain an on-demand master and spot slaves. Because EC2 does not support stopping spot instances, the spark-ec2 script previously could only destroy such clusters.
This pull request makes it possible to stop and restart a spot cluster.
* The `stop` command works as expected for a spot cluster: the master is stopped and the slaves are terminated.
* To start a stopped spot cluster, the user must invoke `launch --use-existing-master`. This launches fresh spot slaves but resumes the existing master.
sbt-assembly is setup to pick the first META-INF/services/org.apache.hadoop.security.SecurityInfo file instead of merging them. This causes Kerberos authentication to fail, this manifests itself in the "info:null" debug log statement:
DEBUG SaslRpcClient: Get token info proto:interface org.apache.hadoop.yarn.api.ApplicationClientProtocolPB info:null
DEBUG SaslRpcClient: Get kerberos info proto:interface org.apache.hadoop.yarn.api.ApplicationClientProtocolPB info:null
ERROR UserGroupInformation: PriviledgedActionException as:foo@BAR (auth:KERBEROS) cause:org.apache.hadoop.security.AccessControlException: Client cannot authenticate via:[TOKEN, KERBEROS]
DEBUG UserGroupInformation: PrivilegedAction as:foo@BAR (auth:KERBEROS) from:org.apache.hadoop.ipc.Client$Connection.handleSaslConnectionFailure(Client.java:583)
WARN Client: Exception encountered while connecting to the server : org.apache.hadoop.security.AccessControlException: Client cannot authenticate via:[TOKEN, KERBEROS]
ERROR UserGroupInformation: PriviledgedActionException as:foo@BAR (auth:KERBEROS) cause:java.io.IOException: org.apache.hadoop.security.AccessControlException: Client cannot authenticate via:[TOKEN, KERBEROS]
This previously would just contain a single class:
$ unzip -c assembly/target/scala-2.10/spark-assembly-0.9.0-incubating-SNAPSHOT-hadoop2.2.0.jar META-INF/services/org.apache.hadoop.security.SecurityInfo
Archive: assembly/target/scala-2.10/spark-assembly-0.9.0-incubating-SNAPSHOT-hadoop2.2.0.jar
inflating: META-INF/services/org.apache.hadoop.security.SecurityInfo
org.apache.hadoop.security.AnnotatedSecurityInfo
And now has the full list of classes:
$ unzip -c assembly/target/scala-2.10/spark-assembly-0.9.0-incubating-SNAPSHOT-hadoop2.2.0.jar META-INF/services/org.apache.hadoop.security.SecurityInfoArchive: assembly/target/scala-2.10/spark-assembly-0.9.0-incubating-SNAPSHOT-hadoop2.2.0.jar
inflating: META-INF/services/org.apache.hadoop.security.SecurityInfo
org.apache.hadoop.security.AnnotatedSecurityInfo
org.apache.hadoop.mapreduce.v2.app.MRClientSecurityInfo
org.apache.hadoop.mapreduce.v2.security.client.ClientHSSecurityInfo
org.apache.hadoop.yarn.security.client.ClientRMSecurityInfo
org.apache.hadoop.yarn.security.ContainerManagerSecurityInfo
org.apache.hadoop.yarn.security.SchedulerSecurityInfo
org.apache.hadoop.yarn.security.admin.AdminSecurityInfo
org.apache.hadoop.yarn.server.RMNMSecurityInfoClass
ByteCodeUtils.invokedMethod(), which we use in mapReduceTriplets, throws
a ClassNotFoundException when called with a closure defined in the
console. This commit catches the exception and conservatively assumes
the closure references all edge attributes.