Commit graph

4855 commits

Author SHA1 Message Date
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
jerryshao 12dc385a49 Add Spark multi-user support for standalone mode and Mesos 2013-11-07 11:18:09 +08:00
Reynold Xin aadeda5e76 Merge pull request #144 from liancheng/runjob-clean
Removed unused return value in SparkContext.runJob

Return type of this `runJob` version is `Unit`:

    def runJob[T, U: ClassManifest](
        rdd: RDD[T],
        func: (TaskContext, Iterator[T]) => U,
        partitions: Seq[Int],
        allowLocal: Boolean,
        resultHandler: (Int, U) => Unit) {
        ...
    }

It's obviously unnecessary to "return" `result`.
2013-11-06 13:27:47 -08:00
Reynold Xin 951024feea Merge pull request #145 from aarondav/sls-fix
Attempt to fix SparkListenerSuite breakage

Could not reproduce locally, but this test could've been flaky if the build machine was too fast, due to typo. (index 0 is intentionally slowed down to ensure total time is >= 1 ms)

This should be merged into branch-0.8 as well.
2013-11-06 09:36:14 -08:00
Aaron Davidson 80e98d2bd7 Attempt to fix SparkListenerSuite breakage
Could not reproduce locally, but this test could've been flaky if the
build machine was too fast.
2013-11-06 08:03:35 -08:00
Lian, Cheng a0c4565183 Removed unused return value in SparkContext.runJob 2013-11-06 23:18:59 +08:00
Reynold Xin bf4e6131cc Merge pull request #143 from rxin/scheduler-hang
Ignore a task update status if the executor doesn't exist anymore.

Otherwise if the scheduler receives a task update message when the executor's been removed, the scheduler would hang.

It is pretty hard to add unit tests for these right now because it is hard to mock the cluster scheduler. We should do that once @kayousterhout finishes merging the local scheduler and the cluster scheduler.
2013-11-05 23:14:09 -08:00
Reynold Xin a02eed6811 Ignore a task update status if the executor doesn't exist anymore. 2013-11-05 18:46:38 -08:00
Reynold Xin 9f7b9bb1cd Merge pull request #142 from liancheng/dagscheduler-pattern-matching
Using case class deep match to simplify code in DAGScheduler.processEvent

Since all `XxxEvent` pushed in `DAGScheduler.eventQueue` are case classes, deep pattern matching is more convenient to extract event object components.
2013-11-05 10:42:19 -08:00
Lian, Cheng 8b4c994e8c Using compact case class pattern matching syntax to simplify code in DAGScheduler.processEvent 2013-11-05 17:18:42 +08:00
Reynold Xin 81065321c0 Merge pull request #139 from aarondav/shuffle-next
Never store shuffle blocks in BlockManager

After the BlockId refactor (PR #114), it became very clear that ShuffleBlocks are of no use
within BlockManager (they had a no-arg constructor!). This patch completely eliminates
them, saving us around 100-150 bytes per shuffle block.
The total, system-wide overhead per shuffle block is now a flat 8 bytes, excluding
state saved by the MapOutputTracker.

Note: This should *not* be merged directly into 0.8.0 -- see #138
2013-11-04 20:47:14 -08:00
Aaron Davidson 93c90844cb Never store shuffle blocks in BlockManager
After the BlockId refactor (PR #114), it became very clear that ShuffleBlocks are of no use
within BlockManager (they had a no-arg constructor!). This patch completely eliminates
them, saving us around 100-150 bytes per shuffle block.
The total, system-wide overhead per shuffle block is now a flat 8 bytes, excluding
state saved by the MapOutputTracker.
2013-11-04 18:43:42 -08:00
Reynold Xin 0b26a392df Merge pull request #128 from shimingfei/joblogger-doc
add javadoc to JobLogger, and some small fix

against Spark-941

add javadoc to JobLogger, output more info for RDD, modify recordStageDepGraph to avoid output duplicate stage dependency information

(cherry picked from commit 518cf22eb2)
Signed-off-by: Reynold Xin <rxin@apache.org>
2013-11-04 18:22:06 -08:00
Reynold Xin 7a26104ab7 Merge pull request #130 from aarondav/shuffle
Memory-optimized shuffle file consolidation

Reduces overhead of each shuffle block for consolidation from >300 bytes to 8 bytes (1 primitive Long). Verified via profiler testing with 1 mil shuffle blocks, net overhead was ~8,400,000 bytes.

Despite the memory-optimized implementation incurring extra CPU overhead, the runtime of the shuffle phase in this test was only around 2% slower, while the reduce phase was 40% faster, when compared to not using any shuffle file consolidation.

This is accomplished by replacing the map from ShuffleBlockId to FileSegment (i.e., block id to where it's located), which had high overhead due to being a gigantic, timestamped, concurrent map with a more space-efficient structure. Namely, the following are introduced (I have omitted the word "Shuffle" from some names for clarity):
**ShuffleFile** - there is one ShuffleFile per consolidated shuffle file on disk. We store an array of offsets into the physical shuffle file for each ShuffleMapTask that wrote into the file. This is sufficient to reconstruct FileSegments for mappers that are in the file.
**FileGroup** - contains a set of ShuffleFiles, one per reducer, that a MapTask can use to write its output. There is one FileGroup created per _concurrent_ MapTask. The FileGroup contains an array of the mapIds that have been written to all files in the group. The positions of elements in this array map directly onto the positions in each ShuffleFile's offsets array.

In order to locate the FileSegment associated with a BlockId, we have another structure which maps each reducer to the set of ShuffleFiles that were created for it. (There will be as many ShuffleFiles per reducer as there are FileGroups.) To lookup a given ShuffleBlockId (shuffleId, reducerId, mapId), we thus search through all ShuffleFiles associated with that reducer.

As a time optimization, we ensure that FileGroups are only reused for MapTasks with monotonically increasing mapIds. This allows us to perform a binary search to locate a mapId inside a group, and also enables potential future optimization (based on the usual monotonic access order).
2013-11-04 17:54:06 -08:00
Aaron Davidson 1ba11b1c6a Minor cleanup in ShuffleBlockManager 2013-11-04 17:16:41 -08:00
tgravescs a35472e1dd Allow spark on yarn to be run from HDFS. Allows the spark.jar, app.jar, and log4j.properties to be put into hdfs. 2013-11-04 16:16:28 -06:00
Aaron Davidson 6201e5e249 Refactor ShuffleBlockManager to reduce public interface
- ShuffleBlocks has been removed and replaced by ShuffleWriterGroup.
- ShuffleWriterGroup no longer contains a reference to a ShuffleFileGroup.
- ShuffleFile has been removed and its contents are now within ShuffleFileGroup.
- ShuffleBlockManager.forShuffle has been replaced by a more stateful forMapTask.
2013-11-04 09:41:04 -08:00
Aaron Davidson b0cf19fe3c Add javadoc and remove unused code 2013-11-03 22:16:58 -08:00
Aaron Davidson 39d93ed4b9 Clean up test files properly
For some reason, even calling
java.nio.Files.createTempDirectory().getFile.deleteOnExit()
does not delete the directory on exit. Guava's analagous function
seems to work, however.
2013-11-03 21:52:59 -08:00