Commit graph

4692 commits

Author SHA1 Message Date
Kay Ousterhout 5125cd3466 Don't ignore spark.cores.max when using Mesos Coarse mode 2013-11-13 23:06:17 -08:00
Matei Zaharia 2054c61a18 Merge pull request #159 from liancheng/dagscheduler-actor-refine
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.
2013-11-13 16:49:55 -08:00
Matei Zaharia 9290e5bcd2 Merge pull request #165 from NathanHowell/kerberos-master
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.
2013-11-13 16:48:44 -08:00
Ahir Reddy 0ea1f8b225 Write Spark UI url to driver file on HDFS 2013-11-13 15:23:36 -08:00
Matei Zaharia 39af914b27 Merge pull request #166 from ahirreddy/simr-spark-ui
SIMR Backend Scheduler will now write Spark UI URL to HDFS, which is to ...

...be retrieved by SIMR clients
2013-11-13 08:39:05 -08:00
Matei Zaharia f49ea28d25 Merge pull request #137 from tgravescs/sparkYarnJarsHdfsRebase
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.
2013-11-12 19:13:39 -08:00
Matei Zaharia 87f2f4e5c2 Merge pull request #153 from ankurdave/stop-spot-cluster
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.
2013-11-12 16:26:09 -08:00
Matei Zaharia b8bf04a085 Merge pull request #160 from xiajunluan/JIRA-923
Fix bug JIRA-923

Fix column sort issue in UI for JIRA-923.
https://spark-project.atlassian.net/browse/SPARK-923

Conflicts:
	core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
	core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala
2013-11-12 16:19:50 -08:00
Ahir Reddy ccb099e804 SIMR Backend Scheduler will now write Spark UI URL to HDFS, which is to be retrieved by SIMR clients 2013-11-12 15:58:41 -08:00
Nathan Howell 48eac0bcbf Upgrade to sbt-assembly 0.9.2 2013-11-12 13:29:25 -08:00
Nathan Howell 23146a6705 spark-assembly.jar fails to authenticate with YARN ResourceManager
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
2013-11-12 13:27:50 -08:00
Matei Zaharia dfd1ebc2d1 Merge pull request #164 from tdas/kafka-fix
Made block generator thread safe to fix Kafka bug.

This is a very important bug fix. Data can and was being lost in the kafka due to this.
2013-11-12 09:10:05 -08:00
Tathagata Das 7ccbbdacb9 Made block generator thread safe to fix Kafka bug. 2013-11-12 00:10:45 -08:00
Ankur Dave bc9f7eacb9 Enable stopping and starting a spot cluster 2013-11-11 17:50:31 -08:00
Matei Zaharia 23b53efccd Merge pull request #156 from haoyuan/master
add tachyon module
2013-11-11 12:30:02 -08:00
tgravescs 17bb9a27b2 Add mockito to the sbt build 2013-11-11 10:01:23 -06:00
Andrew xia e13da05424 fix format error 2013-11-11 19:15:45 +08:00
Andrew xia 37d2f3749e cut lines to less than 100 2013-11-11 15:49:32 +08:00
Andrew xia b3208063af Fix bug JIRA-923 2013-11-11 15:39:10 +08:00
Lian, Cheng e2a43b3dcc Made some changes according to suggestions from @aarondav 2013-11-11 12:21:54 +08:00
Josh Rosen 13122ceb8c FramedSerializer: _dumps => dumps, _loads => loads. 2013-11-10 17:53:25 -08:00
Josh Rosen ffa5bedf46 Send PySpark commands as bytes insetad of strings. 2013-11-10 16:46:00 -08:00
Josh Rosen cbb7f04aef Add custom serializer support to PySpark.
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().
2013-11-10 16:45:38 -08:00
Haoyuan Li 6f455553c9 expose UI port only 2013-11-10 16:00:09 -08:00
Lian, Cheng ba55285177 Put the periodical resubmitFailedStages() call into a scheduled task 2013-11-11 01:25:35 +08:00
Matei Zaharia 58d4f6c8a5 Merge pull request #157 from rxin/kryo
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.
2013-11-10 09:23:56 -08:00
Reynold Xin c845611fc3 Moved the Spark internal class registration for Kryo into an object, and added more classes (e.g. MapStatus, BlockManagerId) to the registration. 2013-11-09 23:00:08 -08:00
Haoyuan Li 77cedf81c7 add tachyon module 2013-11-09 22:52:23 -08:00
Reynold Xin 7c5f70d873 Call Kryo setReferences before calling user specified Kryo registrator. 2013-11-09 22:43:36 -08:00
Matei Zaharia 3efc019562 Merge pull request #147 from JoshRosen/fix-java-api-completeness-checker
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.
2013-11-09 17:53:49 -08:00
Matei Zaharia 87954d4c85 Merge pull request #154 from soulmachine/ClusterScheduler
Replace the thread inside ClusterScheduler.start() with an Akka scheduler

Threads are precious resources so that we shouldn't abuse them
2013-11-09 17:53:25 -08:00
Reynold Xin 83bf1920c8 Merge pull request #155 from rxin/jobgroup
Don't reset job group when a new job description is set.
2013-11-09 15:40:29 -08:00
Reynold Xin 28f27097cf Don't reset job group when a new job description is set. 2013-11-09 13:59:31 -08:00
Matei Zaharia 8af99f2356 Merge pull request #149 from tgravescs/fixSecureHdfsAccess
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.
2013-11-09 13:48:00 -08:00
Josh Rosen a37ff0f1db Add spark-tools assembly to spark-class classpath.
This allows the JavaAPICompletenessChecker to be
run with Spark 0.8+.
2013-11-09 13:42:45 -08:00
Matei Zaharia 72a601ec31 Merge pull request #152 from rxin/repl
Propagate SparkContext local properties from spark-repl caller thread to the repl execution thread.
2013-11-09 11:55:16 -08:00
soulmachine 28115fa8cb replace the thread with a Akka scheduler 2013-11-09 22:38:27 +08:00
Lian, Cheng 765ebca04f Remove unnecessary null checking 2013-11-09 21:13:03 +08:00
Lian, Cheng 2539c06745 Replaced the daemon thread started by DAGScheduler with an actor 2013-11-09 19:05:18 +08:00
Reynold Xin 319299941d Propagate the SparkContext local property from the thread that calls the spark-repl to the actual execution thread. 2013-11-09 00:32:14 -08:00
Russell Cardullo ef85a51f85 Add graphite sink for metrics
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.
2013-11-08 16:36:03 -08:00
Aaron Davidson dd63c548c2 Use SPARK_HOME instead of user.dir in ExecutorRunnerTest 2013-11-08 12:51:05 -08:00
tgravescs 13a19505e4 Don't call the doAs if user is unknown or the same user that is already running 2013-11-08 12:04:09 -06:00
tgravescs f95cb04e40 Remove the runAsUser as it breaks secure hdfs access 2013-11-08 10:07:15 -06:00
tgravescs 5f9ed51719 Fix access to Secure HDFS 2013-11-08 08:41:57 -06:00
Reynold Xin 3d4ad84b63 Merge pull request #148 from squito/include_appId
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`.
2013-11-07 11:08:27 -08:00
Imran Rashid ca66f5d5a2 fix formatting 2013-11-07 07:23:59 -06:00
Imran Rashid 8d3cdda9a2 very basic regression test to make sure appId doesnt get dropped in future 2013-11-07 01:35:48 -06:00
Reynold Xin be7e8da98a Merge pull request #23 from jerryshao/multi-user
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).
2013-11-06 23:22:47 -08:00
Imran Rashid 36e832bff0 include the appid in the cmd line arguments to Executors 2013-11-07 01:11:49 -06:00