aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorAndy Konwinski <andyk@berkeley.edu>2012-09-12 23:05:47 -0700
committerAndy Konwinski <andyk@berkeley.edu>2012-09-12 23:25:07 -0700
commitca2c999e0fd97a29b20bd3990b6e57d9e0db5d0a (patch)
treef90eb6c5cd2bc2a342490d305677f90f7e936c0f
parentc4db09ea76802df22f52826e228f9d15c0cf13d9 (diff)
downloadspark-ca2c999e0fd97a29b20bd3990b6e57d9e0db5d0a.tar.gz
spark-ca2c999e0fd97a29b20bd3990b6e57d9e0db5d0a.tar.bz2
spark-ca2c999e0fd97a29b20bd3990b6e57d9e0db5d0a.zip
Making the link to api scaladocs work and migrating other code snippets
to use pygments syntax highlighting.
-rwxr-xr-xdocs/_layouts/global.html2
-rw-r--r--docs/api.md11
-rw-r--r--docs/bagel-programming-guide.md163
-rw-r--r--docs/running-on-mesos.md23
-rw-r--r--docs/spark-debugger.md99
5 files changed, 175 insertions, 123 deletions
diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html
index 8bfd0e7284..7286d589a8 100755
--- a/docs/_layouts/global.html
+++ b/docs/_layouts/global.html
@@ -44,7 +44,7 @@
<!--TODO(andyk): Add class="active" attribute to li some how.-->
<li><a href="{{HOME_PATH}}index.html">Home</a></li>
<li><a href="{{HOME_PATH}}programming-guide.html">Programming Guide</a></li>
- <li><a href="{{HOME_PATH}}api">API (Scaladoc)</a></li>
+ <li><a href="{{HOME_PATH}}api.html">API (Scaladoc)</a></li>
<!--
<li class="dropdown">
<a href="#" class="dropdown-toggle" data-toggle="dropdown">Versions ({{ page.spark-version }})<b class="caret"></b></a>
diff --git a/docs/api.md b/docs/api.md
new file mode 100644
index 0000000000..b0acbe34c5
--- /dev/null
+++ b/docs/api.md
@@ -0,0 +1,11 @@
+---
+layout: global
+title: Spark API documentation (Scaladoc)
+---
+
+Here you can find links to the Scaladoc generated for the Spark sbt subprojects. If the following links don't work, try running `sbt/sbt doc` from the Spark project home directory.
+
+- [Core]({{HOME_PATH}}../../core/target/scala-2.9.1/api/index.html)
+- [Examples]({{HOME_PATH}}../../examples/target/scala-2.9.1/api/index.html)
+- [Repl]({{HOME_PATH}}../../repl/target/scala-2.9.1/api/index.html)
+- [Bagel]({{HOME_PATH}}../../bagel/target/scala-2.9.1/api/index.html)
diff --git a/docs/bagel-programming-guide.md b/docs/bagel-programming-guide.md
index 3f6ab7df96..b133376a97 100644
--- a/docs/bagel-programming-guide.md
+++ b/docs/bagel-programming-guide.md
@@ -27,58 +27,66 @@ We first extend the default `Vertex` class to store a `Double`
representing the current PageRank of the vertex, and similarly extend
the `Message` and `Edge` classes. Note that these need to be marked `@serializable` to allow Spark to transfer them across machines. We also import the Bagel types and implicit conversions.
- import spark.bagel._
- import spark.bagel.Bagel._
-
- @serializable class PREdge(val targetId: String) extends Edge
-
- @serializable class PRVertex(
- val id: String, val rank: Double, val outEdges: Seq[Edge],
- val active: Boolean) extends Vertex
-
- @serializable class PRMessage(
- val targetId: String, val rankShare: Double) extends Message
+{% highlight scala %}
+import spark.bagel._
+import spark.bagel.Bagel._
+
+@serializable class PREdge(val targetId: String) extends Edge
+
+@serializable class PRVertex(
+ val id: String, val rank: Double, val outEdges: Seq[Edge],
+ val active: Boolean) extends Vertex
+
+@serializable class PRMessage(
+ val targetId: String, val rankShare: Double) extends Message
+{% endhighlight %}
Next, we load a sample graph from a text file as a distributed dataset and package it into `PRVertex` objects. We also cache the distributed dataset because Bagel will use it multiple times and we'd like to avoid recomputing it.
- val input = sc.textFile("pagerank_data.txt")
+{% highlight scala %}
+val input = sc.textFile("pagerank_data.txt")
- val numVerts = input.count()
+val numVerts = input.count()
- val verts = input.map(line => {
- val fields = line.split('\t')
- val (id, linksStr) = (fields(0), fields(1))
- val links = linksStr.split(',').map(new PREdge(_))
- (id, new PRVertex(id, 1.0 / numVerts, links, true))
- }).cache
+val verts = input.map(line => {
+ val fields = line.split('\t')
+ val (id, linksStr) = (fields(0), fields(1))
+ val links = linksStr.split(',').map(new PREdge(_))
+ (id, new PRVertex(id, 1.0 / numVerts, links, true))
+}).cache
+{% endhighlight %}
We run the Bagel job, passing in `verts`, an empty distributed dataset of messages, and a custom compute function that runs PageRank for 10 iterations.
- val emptyMsgs = sc.parallelize(List[(String, PRMessage)]())
-
- def compute(self: PRVertex, msgs: Option[Seq[PRMessage]], superstep: Int)
- : (PRVertex, Iterable[PRMessage]) = {
- val msgSum = msgs.getOrElse(List()).map(_.rankShare).sum
- val newRank =
- if (msgSum != 0)
- 0.15 / numVerts + 0.85 * msgSum
- else
- self.rank
- val halt = superstep >= 10
- val msgsOut =
- if (!halt)
- self.outEdges.map(edge =>
- new PRMessage(edge.targetId, newRank / self.outEdges.size))
- else
- List()
- (new PRVertex(self.id, newRank, self.outEdges, !halt), msgsOut)
- }
-
- val result = Bagel.run(sc, verts, emptyMsgs)()(compute)
+{% highlight scala %}
+val emptyMsgs = sc.parallelize(List[(String, PRMessage)]())
+
+def compute(self: PRVertex, msgs: Option[Seq[PRMessage]], superstep: Int)
+: (PRVertex, Iterable[PRMessage]) = {
+ val msgSum = msgs.getOrElse(List()).map(_.rankShare).sum
+ val newRank =
+ if (msgSum != 0)
+ 0.15 / numVerts + 0.85 * msgSum
+ else
+ self.rank
+ val halt = superstep >= 10
+ val msgsOut =
+ if (!halt)
+ self.outEdges.map(edge =>
+ new PRMessage(edge.targetId, newRank / self.outEdges.size))
+ else
+ List()
+ (new PRVertex(self.id, newRank, self.outEdges, !halt), msgsOut)
+}
+{% endhighlight %}
+
+val result = Bagel.run(sc, verts, emptyMsgs)()(compute)
Finally, we print the results.
- println(result.map(v => "%s\t%s\n".format(v.id, v.rank)).collect.mkString)
+{% highlight scala %}
+println(result.map(v => "%s\t%s\n".format(v.id, v.rank)).collect.mkString)
+{% endhighlight %}
### Combiners
@@ -102,41 +110,50 @@ Here are the actions and types in the Bagel API. See [Bagel.scala](https://githu
#### Actions
- # Full form
- Bagel.run(sc, vertices, messages, combiner, aggregator, partitioner, numSplits)(compute)
- where compute takes (vertex: V, combinedMessages: Option[C], aggregated: Option[A], superstep: Int)
- and returns (newVertex: V, outMessages: Array[M])
- # Abbreviated forms
- Bagel.run(sc, vertices, messages, combiner, partitioner, numSplits)(compute)
- where compute takes (vertex: V, combinedMessages: Option[C], superstep: Int)
- and returns (newVertex: V, outMessages: Array[M])
- Bagel.run(sc, vertices, messages, combiner, numSplits)(compute)
- where compute takes (vertex: V, combinedMessages: Option[C], superstep: Int)
- and returns (newVertex: V, outMessages: Array[M])
- Bagel.run(sc, vertices, messages, numSplits)(compute)
- where compute takes (vertex: V, messages: Option[Array[M]], superstep: Int)
- and returns (newVertex: V, outMessages: Array[M])
+{% highlight scala %}
+/*** Full form ***/
+
+Bagel.run(sc, vertices, messages, combiner, aggregator, partitioner, numSplits)(compute)
+// where compute takes (vertex: V, combinedMessages: Option[C], aggregated: Option[A], superstep: Int)
+// and returns (newVertex: V, outMessages: Array[M])
+
+/*** Abbreviated forms ***/
+
+Bagel.run(sc, vertices, messages, combiner, partitioner, numSplits)(compute)
+// where compute takes (vertex: V, combinedMessages: Option[C], superstep: Int)
+// and returns (newVertex: V, outMessages: Array[M])
+
+Bagel.run(sc, vertices, messages, combiner, numSplits)(compute)
+// where compute takes (vertex: V, combinedMessages: Option[C], superstep: Int)
+// and returns (newVertex: V, outMessages: Array[M])
+
+Bagel.run(sc, vertices, messages, numSplits)(compute)
+// where compute takes (vertex: V, messages: Option[Array[M]], superstep: Int)
+// and returns (newVertex: V, outMessages: Array[M])
+{% endhighlight %}
#### Types
- trait Combiner[M, C] {
- def createCombiner(msg: M): C
- def mergeMsg(combiner: C, msg: M): C
- def mergeCombiners(a: C, b: C): C
- }
-
- trait Aggregator[V, A] {
- def createAggregator(vert: V): A
- def mergeAggregators(a: A, b: A): A
- }
-
- trait Vertex {
- def active: Boolean
- }
-
- trait Message[K] {
- def targetId: K
- }
+{% highlight scala %}
+trait Combiner[M, C] {
+ def createCombiner(msg: M): C
+ def mergeMsg(combiner: C, msg: M): C
+ def mergeCombiners(a: C, b: C): C
+}
+
+trait Aggregator[V, A] {
+ def createAggregator(vert: V): A
+ def mergeAggregators(a: A, b: A): A
+}
+
+trait Vertex {
+ def active: Boolean
+}
+
+trait Message[K] {
+ def targetId: K
+}
+{% endhighlight %}
## Where to Go from Here
diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md
index 947de13855..063ffe1e1d 100644
--- a/docs/running-on-mesos.md
+++ b/docs/running-on-mesos.md
@@ -21,19 +21,27 @@ To run on a cluster, Spark uses the [Apache Mesos](http://incubator.apache.org/m
* Run `<prefix>/sbin/mesos-start-cluster.sh` on your master to start Mesos. If all goes well, you should see Mesos's web UI on port 8080 of the master machine.
* See Mesos's README file for more information on deploying it.
8. To run a Spark job against the cluster, when you create your `SparkContext`, pass the string `HOST:5050` as the first parameter, where `HOST` is the machine running your Mesos master. In addition, pass the location of Spark on your nodes as the third parameter, and a list of JAR files containing your JAR's code as the fourth (these will automatically get copied to the workers). For example:
-<pre>new SparkContext("HOST:5050", "My Job Name", "/home/user/spark", List("my-job.jar"))</pre>
+
+{% highlight scala %}
+new SparkContext("HOST:5050", "My Job Name", "/home/user/spark", List("my-job.jar"))
+{% endhighlight %}
### For Spark versions before 0.5:
1. Download and build Spark using the instructions [here]({{ HOME_DIR }}Home).
2. Download either revision 1205738 of Mesos if you're using the master branch of Spark, or the pre-protobuf branch of Mesos if you're using Spark 0.3 or earlier (note that for new users, _we recommend the master branch instead of 0.3_). For revision 1205738 of Mesos, use:
-<pre>
+
+{% highlight bash %}
svn checkout -r 1205738 http://svn.apache.org/repos/asf/incubator/mesos/trunk mesos
-</pre>
+{% endhighlight %}
+
For the pre-protobuf branch (for Spark 0.3 and earlier), use:
-<pre>git clone git://github.com/mesos/mesos
+{% highlight bash %}
+git clone git://github.com/mesos/mesos
cd mesos
-git checkout --track origin/pre-protobuf</pre>
+git checkout --track origin/pre-protobuf
+{% endhighlight %}
+
3. Configure Mesos using the `configure` script, passing the location of your `JAVA_HOME` using `--with-java-home`. Mesos comes with "template" configure scripts for different platforms, such as `configure.template.macosx`, so you can just run the one on your platform if it exists. See the [Mesos wiki](https://github.com/mesos/mesos/wiki) for other configuration options.
4. Build Mesos using `make`.
5. In Spark's `conf/spark-env.sh` file, add `export MESOS_HOME=<path to Mesos directory>`. If you don't have a `spark-env.sh`, copy `conf/spark-env.sh.template`. You should also set `SCALA_HOME` there if it's not on your system's default path.
@@ -43,7 +51,10 @@ git checkout --track origin/pre-protobuf</pre>
* Run `MESOS_HOME/deploy/start-mesos` to start it up. If all goes well, you should see Mesos's web UI on port 8080 of the master machine.
* See Mesos's [deploy instructions](https://github.com/mesos/mesos/wiki/Deploy-Scripts) for more information on deploying it.
8. To run a Spark job against the cluster, when you create your `SparkContext`, pass the string `master@HOST:5050` as the first parameter, where `HOST` is the machine running your Mesos master. In addition, pass the location of Spark on your nodes as the third parameter, and a list of JAR files containing your JAR's code as the fourth (these will automatically get copied to the workers). For example:
-<pre>new SparkContext("master@HOST:5050", "My Job Name", "/home/user/spark", List("my-job.jar"))</pre>
+
+{% highlight scala %}
+new SparkContext("master@HOST:5050", "My Job Name", "/home/user/spark", List("my-job.jar"))
+{% endhighlight %}
## Running on Amazon EC2
diff --git a/docs/spark-debugger.md b/docs/spark-debugger.md
index 71f9d001d4..f6f0988858 100644
--- a/docs/spark-debugger.md
+++ b/docs/spark-debugger.md
@@ -27,82 +27,95 @@ _A note on nondeterminism:_ For fault recovery, Spark requires RDD transformatio
### Enabling the event log
-* To turn on event logging for your program, set `$SPARK_JAVA_OPTS` in `conf/spark-env.sh` as follows:
-
- export SPARK_JAVA_OPTS='-Dspark.arthur.logPath=path/to/event-log'
-
- where `path/to/event-log` is where you want the event log to go relative to `$SPARK_HOME`.
+To turn on event logging for your program, set `$SPARK_JAVA_OPTS` in `conf/spark-env.sh` as follows:
- **Warning:** If `path/to/event-log` already exists, event logging will be automatically disabled.
+{% highlight bash %}
+export SPARK_JAVA_OPTS='-Dspark.arthur.logPath=path/to/event-log'
+{% endhighlight %}
+
+where `path/to/event-log` is where you want the event log to go relative to `$SPARK_HOME`.
+
+**Warning:** If `path/to/event-log` already exists, event logging will be automatically disabled.
### Loading the event log into the debugger
1. Run a Spark shell with `MASTER=<i>host</i> ./spark-shell`.
2. Use `EventLogReader` to load the event log as follows:
-
- spark> val r = new spark.EventLogReader(sc, Some("path/to/event-log"))
- r: spark.EventLogReader = spark.EventLogReader@726b37ad
+ {% highlight scala %}
+spark> val r = new spark.EventLogReader(sc, Some("path/to/event-log"))
+r: spark.EventLogReader = spark.EventLogReader@726b37ad
+{% endhighlight %}
**Warning:** If the event log doesn't exist or is unreadable, this will silently fail and `r.events` will be empty.
### Exploring intermediate RDDs
-* Use `r.rdds` to get a list of intermediate RDDs generated during your program's execution. An RDD with id _x_ is located at <code>r.rdds(<i>x</i>)</code>. For example:
+Use `r.rdds` to get a list of intermediate RDDs generated during your program's execution. An RDD with id _x_ is located at <code>r.rdds(<i>x</i>)</code>. For example:
- scala> r.rdds
- res8: scala.collection.mutable.ArrayBuffer[spark.RDD[_]] = ArrayBuffer(spark.HadoopRDD@fe85adf, spark.MappedRDD@5fa5eea1, spark.MappedRDD@6d5bd16, spark.ShuffledRDD@3a70f2db, spark.FlatMappedValuesRDD@4d5825d6, spark.MappedValuesRDD@561c2c45, spark.CoGroupedRDD@539e922d, spark.MappedValuesRDD@4f8ef33e, spark.FlatMappedRDD@32039440, spark.ShuffledRDD@8fa0f67, spark.MappedValuesRDD@590937cb, spark.CoGroupedRDD@6c2e1e17, spark.MappedValuesRDD@47b9af7d, spark.FlatMappedRDD@6fb05c54, spark.ShuffledRDD@237dc815, spark.MappedValuesRDD@16daece7, spark.CoGroupedRDD@7ef73d69, spark.MappedValuesRDD@19e0f99e, spark.FlatMappedRDD@1240158, spark.ShuffledRDD@62d438fd, spark.MappedValuesRDD@5ae99cbb, spark.FilteredRDD@1f30e79e, spark.MappedRDD@43b64611)
+{% highlight scala %}
+scala> r.rdds
+res8: scala.collection.mutable.ArrayBuffer[spark.RDD[_]] = ArrayBuffer(spark.HadoopRDD@fe85adf, spark.MappedRDD@5fa5eea1, spark.MappedRDD@6d5bd16, spark.ShuffledRDD@3a70f2db, spark.FlatMappedValuesRDD@4d5825d6, spark.MappedValuesRDD@561c2c45, spark.CoGroupedRDD@539e922d, spark.MappedValuesRDD@4f8ef33e, spark.FlatMappedRDD@32039440, spark.ShuffledRDD@8fa0f67, spark.MappedValuesRDD@590937cb, spark.CoGroupedRDD@6c2e1e17, spark.MappedValuesRDD@47b9af7d, spark.FlatMappedRDD@6fb05c54, spark.ShuffledRDD@237dc815, spark.MappedValuesRDD@16daece7, spark.CoGroupedRDD@7ef73d69, spark.MappedValuesRDD@19e0f99e, spark.FlatMappedRDD@1240158, spark.ShuffledRDD@62d438fd, spark.MappedValuesRDD@5ae99cbb, spark.FilteredRDD@1f30e79e, spark.MappedRDD@43b64611)
+{% endhighlight %}
-* Use `r.printRDDs()` to get a formatted list of intermediate RDDs, along with the source location where they were created. For example:
+Use `r.printRDDs()` to get a formatted list of intermediate RDDs, along with the source location where they were created. For example:
- scala> r.printRDDs
- #00: HadoopRDD spark.bagel.examples.WikipediaPageRankStandalone$.main(WikipediaPageRankStandalone.scala:31)
- #01: MappedRDD spark.bagel.examples.WikipediaPageRankStandalone$.main(WikipediaPageRankStandalone.scala:31)
- #02: MappedRDD spark.bagel.examples.WikipediaPageRankStandalone$.main(WikipediaPageRankStandalone.scala:35)
- #03: ShuffledRDD spark.bagel.examples.WikipediaPageRankStandalone$.main(WikipediaPageRankStandalone.scala:35)
- #04: FlatMappedValuesRDD spark.bagel.examples.WikipediaPageRankStandalone$.main(WikipediaPageRankStandalone.scala:35)
- #05: MappedValuesRDD spark.bagel.examples.WikipediaPageRankStandalone$.pageRank(WikipediaPageRankStandalone.scala:91)
- #06: CoGroupedRDD spark.bagel.examples.WikipediaPageRankStandalone$.pageRank(WikipediaPageRankStandalone.scala:92)
- [...]
+{% highlight scala %}
+scala> r.printRDDs
+#00: HadoopRDD spark.bagel.examples.WikipediaPageRankStandalone$.main(WikipediaPageRankStandalone.scala:31)
+#01: MappedRDD spark.bagel.examples.WikipediaPageRankStandalone$.main(WikipediaPageRankStandalone.scala:31)
+#02: MappedRDD spark.bagel.examples.WikipediaPageRankStandalone$.main(WikipediaPageRankStandalone.scala:35)
+#03: ShuffledRDD spark.bagel.examples.WikipediaPageRankStandalone$.main(WikipediaPageRankStandalone.scala:35)
+#04: FlatMappedValuesRDD spark.bagel.examples.WikipediaPageRankStandalone$.main(WikipediaPageRankStandalone.scala:35)
+#05: MappedValuesRDD spark.bagel.examples.WikipediaPageRankStandalone$.pageRank(WikipediaPageRankStandalone.scala:91)
+#06: CoGroupedRDD spark.bagel.examples.WikipediaPageRankStandalone$.pageRank(WikipediaPageRankStandalone.scala:92)
+[...]
+{% endhighlight %}
-* Use `r.visualizeRDDs()` to visualize the RDDs as a dependency graph. For example:
+Use `r.visualizeRDDs()` to visualize the RDDs as a dependency graph. For example:
- scala> r.visualizeRDDs
- /tmp/spark-rdds-3758182885839775712.pdf
+{% highlight scala %}
+scala> r.visualizeRDDs
+/tmp/spark-rdds-3758182885839775712.pdf
+{% endhighlight %}
- ![Example RDD dependency graph](http://www.ankurdave.com/images/rdd-dep-graph.png)
+![Example RDD dependency graph](http://www.ankurdave.com/images/rdd-dep-graph.png)
-* Iterate over the `RDDCreation` entries in `r.events` (e.g. `for (RDDCreation(rdd, location) <- events)`) to access the RDD creation locations as well as the RDDs themselves.
+Iterate over the `RDDCreation` entries in `r.events` (e.g. `for (RDDCreation(rdd, location) <- events)`) to access the RDD creation locations as well as the RDDs themselves.
### Debugging a particular task
1. Find the task you want to debug. If the task threw an exception, the `ExceptionEvent` that was created will have a reference to the task. For example:
-
- spark> val task = r.events.collect { case e: ExceptionEvent => e }.head.task
-
+ {% highlight scala %}
+spark> val task = r.events.collect { case e: ExceptionEvent => e }.head.task
+{% endhighlight %}
Otherwise, look through the list of all tasks in `r.tasks`, or browse tasks by RDD using <code>r.tasksForRDD(<i>rdd</i>)</code>, which returns a list of tasks whose input is the given RDD.
2. Run the task by calling <code>r.debugTask(<i>taskStageId</i>, <i>taskPartition</i>)</code>. The task should contain these two values; you can extract them as follows:
-
- val (taskStageId, taskPartition) = task match {
- case rt: ResultTask[_, _] => (rt.stageId, rt.partition)
- case smt: ShuffleMapTask => (smt.stageId, smt.partition)
- case _ => throw new UnsupportedOperationException
- })
-
- The Spark debugger will launch the task in a separate JVM, but you will see the task's stdout and stderr inline with the Spark shell. If you want to pass custom debugging arguments to the task's JVM (for example, to change the debugging port), set the optional `debugOpts` argument to `r.debugTask`. When `debugOpts` is left unset, it defaults to
-
- -Xdebug -agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=8000
+ {% highlight scala %}
+val (taskStageId, taskPartition) = task match {
+ case rt: ResultTask[_, _] => (rt.stageId, rt.partition)
+ case smt: ShuffleMapTask => (smt.stageId, smt.partition)
+ case _ => throw new UnsupportedOperationException
+})
+{% endhighlight %}
+ The Spark debugger will launch the task in a separate JVM, but you will see the task's stdout and stderr inline with the Spark shell. If you want to pass custom debugging arguments to the task's JVM (for example, to change the debugging port), set the optional `debugOpts` argument to `r.debugTask`. When `debugOpts` is left unset, it defaults to:
+ {% highlight scala %}
+-Xdebug -agentlib:jdwp=transport=dt_socket,server=y,suspend=y,address=8000
+{% endhighlight %}
3. In another terminal, attach your favorite conventional debugger to the Spark shell. For example, if you want to use jdb, run `jdb -attach 8000`.
4. Debug the task as you would debug a normal program. For example, to break when an exception is thrown:
-
- > catch org.xml.sax.SAXParseException
+ {% highlight scala %}
+> catch org.xml.sax.SAXParseException
+{% endhighlight %}
5. When the task ends, its JVM will quit and control will return to the main Spark shell. To stop it prematurely, you can kill it from the debugger, or interrupt it from the terminal with Ctrl-C.
### Detecting nondeterminism in your transformations
When a task gets run more than once, Arthur is able to compare the checksums of the task's output. If they are different, Arthur will insert a `ChecksumEvent` into `r.checksumMismatches` and print a warning like the following:
+ {% highlight scala %}
+12/04/07 11:42:44 WARN spark.EventLogWriter: Nondeterminism detected in shuffle output on RDD 2, partition 3, output split 0
+{% endhighlight %}
- 12/04/07 11:42:44 WARN spark.EventLogWriter: Nondeterminism detected in shuffle output on RDD 2, partition 3, output split 0