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
For now, this only adds MarshalSerializer, but it lays the groundwork
for other supporting custom serializers. Many of these mechanisms
can also be used to support deserialization of different data formats
sent by Java, such as data encoded by MsgPack.
This also fixes a bug in SparkContext.union().
3 Kryo related changes.
1. Call Kryo setReferences before calling user specified Kryo registrator. This is done so the user specified registrator can override the default setting.
2. Register more internal classes (MapStatus, BlockManagerId).
3. Slightly refactored the internal class registration to allocate less memory.
Add spark-tools assembly to spark-class'ss classpath
This commit adds an assembly for `spark-tools` and adds it to `spark-class`'s classpath, allowing the JavaAPICompletenessChecker to be run against Spark 0.8+ with
./spark-class org.apache.spark.tools.JavaAPICompletenessChecker
Previously, this tool was run through the `run` script. I chose to add this to `run-example` because I didn't want to duplicate code in a `run-tool` script.
Fix secure hdfs access for spark on yarn
https://github.com/apache/incubator-spark/pull/23 broke secure hdfs access. Not sure if it works with secure hdfs on standalone. Fixing it at least for spark on yarn.
The broadcasting of jobconf change also broke secure hdfs access as it didn't take into account things calling the getPartitions before sparkContext is initialized. The DAGScheduler does this as it tries to getShuffleMapStage.
This adds a metrics sink for graphite. The sink must
be configured with the host and port of a graphite node
and optionally may be configured with a prefix that will
be prepended to all metrics that are sent to graphite.
Include appId in executor cmd line args
add the appId back into the executor cmd line args.
I also made a pretty lame regression test, just to make sure it doesn't get dropped in the future. not sure it will run on the build server, though, b/c `ExecutorRunner.buildCommandSeq()` expects to be abel to run the scripts in `bin`.
Add Spark multi-user support for standalone mode and Mesos
This PR add multi-user support for Spark both standalone mode and Mesos (coarse and fine grained ) mode, user can specify the user name who submit app through environment variable `SPARK_USER` or use default one. Executor will communicate with Hadoop using specified user name.
Also I fixed one bug in JobLogger when different user wrote job log to specified folder which has no right file permission.
I separate previous [PR750](https://github.com/mesos/spark/pull/750) into two PRs, in this PR I only solve multi-user support problem. I will try to solve security auth problem in subsequent PR because security auth is a complicated problem especially for Shark Server like long-run app (both Kerberos TGT and HDFS delegation token should be renewed or re-created through app's run time).