aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--assembly/pom.xml116
-rw-r--r--assembly/src/deb/RELEASE2
-rw-r--r--assembly/src/deb/control/control (renamed from repl-bin/src/deb/control/control)0
-rwxr-xr-x[-rw-r--r--]bin/spark-class2.cmd2
-rwxr-xr-x[-rw-r--r--]bin/spark-shell.cmd4
-rw-r--r--core/src/main/scala/org/apache/spark/Accumulators.scala40
-rw-r--r--core/src/main/scala/org/apache/spark/Aggregator.scala25
-rw-r--r--core/src/main/scala/org/apache/spark/SparkConf.scala20
-rw-r--r--core/src/main/scala/org/apache/spark/SparkContext.scala4
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala127
-rw-r--r--core/src/main/scala/org/apache/spark/api/java/package.scala23
-rw-r--r--core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala33
-rw-r--r--core/src/main/scala/org/apache/spark/broadcast/package.scala25
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala17
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala17
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala5
-rw-r--r--core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala17
-rw-r--r--core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala3
-rw-r--r--core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala2
-rw-r--r--core/src/main/scala/org/apache/spark/storage/BlockManager.scala2
-rw-r--r--core/src/test/scala/org/apache/spark/DistributedSuite.scala17
-rw-r--r--core/src/test/scala/org/apache/spark/SparkConfSuite.scala17
-rw-r--r--core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala17
-rw-r--r--core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala17
-rw-r--r--core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala17
-rw-r--r--docs/building-with-maven.md6
-rw-r--r--docs/configuration.md2
-rw-r--r--docs/graphx-programming-guide.md315
-rw-r--r--docs/mllib-guide.md3
-rw-r--r--docs/python-programming-guide.md4
-rw-r--r--docs/streaming-programming-guide.md32
-rw-r--r--docs/tuning.md3
-rw-r--r--examples/pom.xml6
-rw-r--r--examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala3
-rw-r--r--examples/src/main/scala/org/apache/spark/streaming/examples/StreamingExamples.scala17
-rw-r--r--external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala1
-rw-r--r--external/flume/src/test/resources/log4j.properties2
-rw-r--r--external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala2
-rw-r--r--external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala4
-rw-r--r--external/kafka/src/test/resources/log4j.properties2
-rw-r--r--external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala1
-rw-r--r--external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala4
-rw-r--r--external/mqtt/src/test/resources/log4j.properties2
-rw-r--r--external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala1
-rw-r--r--external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala4
-rw-r--r--external/twitter/src/test/resources/log4j.properties2
-rw-r--r--external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala1
-rw-r--r--external/zeromq/src/test/resources/log4j.properties2
-rw-r--r--external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala1
-rw-r--r--graphx/pom.xml1
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/Edge.scala25
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala17
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala21
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala21
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/Graph.scala35
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala19
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala17
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala49
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala31
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala25
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala42
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala33
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala27
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala19
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala49
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala29
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala47
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala33
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala27
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala61
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/impl/package.scala19
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala17
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala21
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala21
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala29
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala23
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala19
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/package.scala21
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala17
-rw-r--r--graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala29
-rw-r--r--graphx/src/test/resources/als-test.data16
-rw-r--r--graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala27
-rw-r--r--graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala45
-rw-r--r--graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala17
-rw-r--r--graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala25
-rw-r--r--graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala35
-rw-r--r--graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala17
-rw-r--r--graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala19
-rw-r--r--graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala17
-rw-r--r--graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala19
-rw-r--r--graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala17
-rw-r--r--graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala27
-rw-r--r--graphx/src/test/scala/org/apache/spark/graphx/lib/StronglyConnectedComponentsSuite.scala17
-rw-r--r--graphx/src/test/scala/org/apache/spark/graphx/lib/TriangleCountSuite.scala17
-rw-r--r--graphx/src/test/scala/org/apache/spark/graphx/util/BytecodeUtilsSuite.scala17
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala4
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala2
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala4
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala2
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala6
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala4
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala17
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala2
-rw-r--r--mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala2
-rw-r--r--mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java17
-rw-r--r--mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala6
-rw-r--r--mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala9
-rw-r--r--mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala3
-rw-r--r--mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala1
-rw-r--r--mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala6
-rw-r--r--mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala5
-rw-r--r--mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala3
-rw-r--r--pom.xml10
-rw-r--r--repl-bin/pom.xml184
-rwxr-xr-xrepl-bin/src/deb/bin/run57
-rwxr-xr-xrepl-bin/src/deb/bin/spark-executor22
-rwxr-xr-xrepl-bin/src/deb/bin/spark-shell21
-rw-r--r--repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala17
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala10
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/ContextWaiter.scala17
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala2
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala40
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala13
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala20
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala26
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala24
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala9
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala5
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/package.scala38
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala2
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/util/RateLimitedOutputStream.scala1
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala1
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala1
-rw-r--r--streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java6
-rw-r--r--streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala3
-rw-r--r--streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala1
-rw-r--r--streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala30
-rw-r--r--streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala8
-rw-r--r--streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala1
-rw-r--r--streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala7
-rw-r--r--yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala371
-rw-r--r--yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala126
-rw-r--r--yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala410
-rw-r--r--yarn/common/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnableUtil.scala176
-rw-r--r--yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala365
-rw-r--r--yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala130
146 files changed, 2613 insertions, 1799 deletions
diff --git a/assembly/pom.xml b/assembly/pom.xml
index 54a25910ce..dcd9601fe4 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -30,6 +30,13 @@
<name>Spark Project Assembly</name>
<url>http://spark.incubator.apache.org/</url>
+ <properties>
+ <spark.jar>${project.build.directory}/scala-${scala.binary.version}/${project.artifactId}-${project.version}-hadoop${hadoop.version}.jar</spark.jar>
+ <deb.pkg.name>spark</deb.pkg.name>
+ <deb.install.path>/usr/share/spark</deb.install.path>
+ <deb.user>root</deb.user>
+ </properties>
+
<repositories>
<!-- A repository in the local filesystem for the Py4J JAR, which is not in Maven central -->
<repository>
@@ -79,7 +86,7 @@
<artifactId>maven-shade-plugin</artifactId>
<configuration>
<shadedArtifactAttached>false</shadedArtifactAttached>
- <outputFile>${project.build.directory}/scala-${scala.binary.version}/${project.artifactId}-${project.version}-hadoop${hadoop.version}.jar</outputFile>
+ <outputFile>${spark.jar}</outputFile>
<artifactSet>
<includes>
<include>*:*</include>
@@ -171,5 +178,112 @@
</plugins>
</build>
</profile>
+ <profile>
+ <id>deb</id>
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>buildnumber-maven-plugin</artifactId>
+ <version>1.1</version>
+ <executions>
+ <execution>
+ <phase>validate</phase>
+ <goals>
+ <goal>create</goal>
+ </goals>
+ <configuration>
+ <shortRevisionLength>8</shortRevisionLength>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ <plugin>
+ <groupId>org.vafer</groupId>
+ <artifactId>jdeb</artifactId>
+ <version>0.11</version>
+ <executions>
+ <execution>
+ <phase>package</phase>
+ <goals>
+ <goal>jdeb</goal>
+ </goals>
+ <configuration>
+ <deb>${project.build.directory}/${deb.pkg.name}_${project.version}-${buildNumber}_all.deb</deb>
+ <attach>false</attach>
+ <compression>gzip</compression>
+ <dataSet>
+ <data>
+ <src>${spark.jar}</src>
+ <type>file</type>
+ <mapper>
+ <type>perm</type>
+ <user>${deb.user}</user>
+ <group>${deb.user}</group>
+ <prefix>${deb.install.path}/jars</prefix>
+ </mapper>
+ </data>
+ <data>
+ <src>${basedir}/src/deb/RELEASE</src>
+ <type>file</type>
+ <mapper>
+ <type>perm</type>
+ <user>${deb.user}</user>
+ <group>${deb.user}</group>
+ <prefix>${deb.install.path}</prefix>
+ </mapper>
+ </data>
+ <data>
+ <src>${basedir}/../conf</src>
+ <type>directory</type>
+ <mapper>
+ <type>perm</type>
+ <user>${deb.user}</user>
+ <group>${deb.user}</group>
+ <prefix>${deb.install.path}/conf</prefix>
+ <filemode>744</filemode>
+ </mapper>
+ </data>
+ <data>
+ <src>${basedir}/../bin</src>
+ <type>directory</type>
+ <mapper>
+ <type>perm</type>
+ <user>${deb.user}</user>
+ <group>${deb.user}</group>
+ <prefix>${deb.install.path}/bin</prefix>
+ <filemode>744</filemode>
+ </mapper>
+ </data>
+ <data>
+ <src>${basedir}/../sbin</src>
+ <type>directory</type>
+ <mapper>
+ <type>perm</type>
+ <user>${deb.user}</user>
+ <group>${deb.user}</group>
+ <prefix>${deb.install.path}/sbin</prefix>
+ <filemode>744</filemode>
+ </mapper>
+ </data>
+ <data>
+ <src>${basedir}/../python</src>
+ <type>directory</type>
+ <mapper>
+ <type>perm</type>
+ <user>${deb.user}</user>
+ <group>${deb.user}</group>
+ <prefix>${deb.install.path}/python</prefix>
+ <filemode>744</filemode>
+ </mapper>
+ </data>
+ </dataSet>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+ </plugins>
+ </build>
+ </profile>
</profiles>
</project>
diff --git a/assembly/src/deb/RELEASE b/assembly/src/deb/RELEASE
new file mode 100644
index 0000000000..aad50ee73a
--- /dev/null
+++ b/assembly/src/deb/RELEASE
@@ -0,0 +1,2 @@
+compute-classpath.sh uses the existence of this file to decide whether to put the assembly jar on the
+classpath or instead to use classfiles in the source tree. \ No newline at end of file
diff --git a/repl-bin/src/deb/control/control b/assembly/src/deb/control/control
index a6b4471d48..a6b4471d48 100644
--- a/repl-bin/src/deb/control/control
+++ b/assembly/src/deb/control/control
diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd
index 460e661476..80818c78ec 100644..100755
--- a/bin/spark-class2.cmd
+++ b/bin/spark-class2.cmd
@@ -73,7 +73,7 @@ for %%d in ("%TOOLS_DIR%\target\scala-%SCALA_VERSION%\spark-tools*assembly*.jar"
rem Compute classpath using external script
set DONT_PRINT_CLASSPATH=1
-call "%FWDIR%sbin\compute-classpath.cmd"
+call "%FWDIR%bin\compute-classpath.cmd"
set DONT_PRINT_CLASSPATH=0
set CLASSPATH=%CLASSPATH%;%SPARK_TOOLS_JAR%
diff --git a/bin/spark-shell.cmd b/bin/spark-shell.cmd
index 23973e3e3d..99799128eb 100644..100755
--- a/bin/spark-shell.cmd
+++ b/bin/spark-shell.cmd
@@ -18,6 +18,6 @@ rem limitations under the License.
rem
rem Find the path of sbin
-set SBIN=%~dp0..\sbin\
+set BIN=%~dp0..\bin\
-cmd /V /E /C %SBIN%spark-class2.cmd org.apache.spark.repl.Main %*
+cmd /V /E /C %BIN%spark-class2.cmd org.apache.spark.repl.Main %*
diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala
index 2ba871a600..df01b2e942 100644
--- a/core/src/main/scala/org/apache/spark/Accumulators.scala
+++ b/core/src/main/scala/org/apache/spark/Accumulators.scala
@@ -17,17 +17,17 @@
package org.apache.spark
-import java.io._
+import java.io.{ObjectInputStream, Serializable}
import scala.collection.mutable.Map
import scala.collection.generic.Growable
import org.apache.spark.serializer.JavaSerializer
/**
- * A datatype that can be accumulated, ie has an commutative and associative "add" operation,
+ * A data type that can be accumulated, ie has an commutative and associative "add" operation,
* but where the result type, `R`, may be different from the element type being added, `T`.
*
- * You must define how to add data, and how to merge two of these together. For some datatypes,
+ * You must define how to add data, and how to merge two of these together. For some data types,
* such as a counter, these might be the same operation. In that case, you can use the simpler
* [[org.apache.spark.Accumulator]]. They won't always be the same, though -- e.g., imagine you are
* accumulating a set. You will add items to the set, and you will union two sets together.
@@ -45,7 +45,7 @@ class Accumulable[R, T] (
val id = Accumulators.newId
@transient private var value_ = initialValue // Current value on master
val zero = param.zero(initialValue) // Zero value to be passed to workers
- var deserialized = false
+ private var deserialized = false
Accumulators.register(this, true)
@@ -127,7 +127,7 @@ class Accumulable[R, T] (
/**
* Helper object defining how to accumulate values of a particular type. An implicit
- * AccumulableParam needs to be available when you create Accumulables of a specific type.
+ * AccumulableParam needs to be available when you create [[Accumulable]]s of a specific type.
*
* @tparam R the full accumulated data (result type)
* @tparam T partial data that can be added in
@@ -186,7 +186,29 @@ class GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Ser
/**
* A simpler value of [[Accumulable]] where the result type being accumulated is the same
- * as the types of elements being merged.
+ * as the types of elements being merged, i.e. variables that are only "added" to through an
+ * associative operation and can therefore be efficiently supported in parallel. They can be used
+ * to implement counters (as in MapReduce) or sums. Spark natively supports accumulators of type
+ * `Int` and `Double`, and programmers can add support for new types.
+ *
+ * An accumulator is created from an initial value `v` by calling [[SparkContext#accumulator]].
+ * Tasks running on the cluster can then add to it using the [[Accumulable#+=]] operator.
+ * However, they cannot read its value. Only the driver program can read the accumulator's value,
+ * using its value method.
+ *
+ * The interpreter session below shows an accumulator being used to add up the elements of an array:
+ *
+ * {{{
+ * scala> val accum = sc.accumulator(0)
+ * accum: spark.Accumulator[Int] = 0
+ *
+ * scala> sc.parallelize(Array(1, 2, 3, 4)).foreach(x => accum += x)
+ * ...
+ * 10/09/29 18:41:08 INFO SparkContext: Tasks finished in 0.317106 s
+ *
+ * scala> accum.value
+ * res2: Int = 10
+ * }}}
*
* @param initialValue initial value of accumulator
* @param param helper object defining how to add elements of type `T`
@@ -196,9 +218,9 @@ class Accumulator[T](@transient initialValue: T, param: AccumulatorParam[T])
extends Accumulable[T,T](initialValue, param)
/**
- * A simpler version of [[org.apache.spark.AccumulableParam]] where the only datatype you can add in is the same type
- * as the accumulated value. An implicit AccumulatorParam object needs to be available when you create
- * Accumulators of a specific type.
+ * A simpler version of [[org.apache.spark.AccumulableParam]] where the only data type you can add
+ * in is the same type as the accumulated value. An implicit AccumulatorParam object needs to be
+ * available when you create Accumulators of a specific type.
*
* @tparam T type of value to accumulate
*/
diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala
index 6d439fdc68..c4579cf6ad 100644
--- a/core/src/main/scala/org/apache/spark/Aggregator.scala
+++ b/core/src/main/scala/org/apache/spark/Aggregator.scala
@@ -17,6 +17,8 @@
package org.apache.spark
+import scala.{Option, deprecated}
+
import org.apache.spark.util.collection.{AppendOnlyMap, ExternalAppendOnlyMap}
/**
@@ -31,11 +33,14 @@ case class Aggregator[K, V, C] (
mergeValue: (C, V) => C,
mergeCombiners: (C, C) => C) {
- private val sparkConf = SparkEnv.get.conf
- private val externalSorting = sparkConf.getBoolean("spark.shuffle.spill", true)
+ private val externalSorting = SparkEnv.get.conf.getBoolean("spark.shuffle.spill", true)
+
+ @deprecated("use combineValuesByKey with TaskContext argument", "0.9.0")
+ def combineValuesByKey(iter: Iterator[_ <: Product2[K, V]]): Iterator[(K, C)] =
+ combineValuesByKey(iter, null)
def combineValuesByKey(iter: Iterator[_ <: Product2[K, V]],
- context: TaskContext) : Iterator[(K, C)] = {
+ context: TaskContext): Iterator[(K, C)] = {
if (!externalSorting) {
val combiners = new AppendOnlyMap[K,C]
var kv: Product2[K, V] = null
@@ -53,12 +58,17 @@ case class Aggregator[K, V, C] (
val (k, v) = iter.next()
combiners.insert(k, v)
}
- context.taskMetrics.memoryBytesSpilled = combiners.memoryBytesSpilled
- context.taskMetrics.diskBytesSpilled = combiners.diskBytesSpilled
+ // TODO: Make this non optional in a future release
+ Option(context).foreach(c => c.taskMetrics.memoryBytesSpilled = combiners.memoryBytesSpilled)
+ Option(context).foreach(c => c.taskMetrics.diskBytesSpilled = combiners.diskBytesSpilled)
combiners.iterator
}
}
+ @deprecated("use combineCombinersByKey with TaskContext argument", "0.9.0")
+ def combineCombinersByKey(iter: Iterator[(K, C)]) : Iterator[(K, C)] =
+ combineCombinersByKey(iter, null)
+
def combineCombinersByKey(iter: Iterator[(K, C)], context: TaskContext) : Iterator[(K, C)] = {
if (!externalSorting) {
val combiners = new AppendOnlyMap[K,C]
@@ -77,8 +87,9 @@ case class Aggregator[K, V, C] (
val (k, c) = iter.next()
combiners.insert(k, c)
}
- context.taskMetrics.memoryBytesSpilled = combiners.memoryBytesSpilled
- context.taskMetrics.diskBytesSpilled = combiners.diskBytesSpilled
+ // TODO: Make this non optional in a future release
+ Option(context).foreach(c => c.taskMetrics.memoryBytesSpilled = combiners.memoryBytesSpilled)
+ Option(context).foreach(c => c.taskMetrics.diskBytesSpilled = combiners.diskBytesSpilled)
combiners.iterator
}
}
diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala
index 2de32231e8..369c6ce78f 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -1,9 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark
import scala.collection.JavaConverters._
import scala.collection.mutable.HashMap
import com.typesafe.config.ConfigFactory
+import java.io.{ObjectInputStream, ObjectOutputStream, IOException}
/**
* Configuration for a Spark application. Used to set various Spark parameters as key-value pairs.
@@ -24,7 +42,7 @@ import com.typesafe.config.ConfigFactory
*
* @param loadDefaults whether to load values from the system properties and classpath
*/
-class SparkConf(loadDefaults: Boolean) extends Serializable with Cloneable with Logging {
+class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging {
/** Create a SparkConf that loads defaults from system properties and the classpath */
def this() = this(true)
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 55ac76bf63..ba3e91effb 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -340,8 +340,8 @@ class SparkContext(
* Hadoop-supported file system URI, and return it as an RDD of Strings.
*/
def textFile(path: String, minSplits: Int = defaultMinSplits): RDD[String] = {
- hadoopFile(path, classOf[TextInputFormat], classOf[LongWritable], classOf[Text], minSplits)
- .map(pair => pair._2.toString)
+ hadoopFile(path, classOf[TextInputFormat], classOf[LongWritable], classOf[Text],
+ minSplits, cloneRecords = false).map(pair => pair._2.toString)
}
/**
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
index 7a6f044965..4b73a0352f 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
@@ -34,11 +34,11 @@ import org.apache.spark.SparkContext.IntAccumulatorParam
import org.apache.spark.SparkContext.DoubleAccumulatorParam
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.rdd.RDD
-import scala.Tuple2
+
/**
- * A Java-friendly version of [[org.apache.spark.SparkContext]] that returns [[org.apache.spark.api.java.JavaRDD]]s and
- * works with Java collections instead of Scala ones.
+ * A Java-friendly version of [[org.apache.spark.SparkContext]] that returns
+ * [[org.apache.spark.api.java.JavaRDD]]s and works with Java collections instead of Scala ones.
*/
class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWorkaround {
/**
@@ -137,7 +137,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
*/
def textFile(path: String, minSplits: Int): JavaRDD[String] = sc.textFile(path, minSplits)
- /**Get an RDD for a Hadoop SequenceFile with given key and value types. */
+ /** Get an RDD for a Hadoop SequenceFile with given key and value types. */
def sequenceFile[K, V](path: String,
keyClass: Class[K],
valueClass: Class[V],
@@ -148,7 +148,19 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
new JavaPairRDD(sc.sequenceFile(path, keyClass, valueClass, minSplits))
}
- /**Get an RDD for a Hadoop SequenceFile. */
+ /** Get an RDD for a Hadoop SequenceFile with given key and value types. */
+ def sequenceFile[K, V](path: String,
+ keyClass: Class[K],
+ valueClass: Class[V],
+ minSplits: Int,
+ cloneRecords: Boolean
+ ): JavaPairRDD[K, V] = {
+ implicit val kcm: ClassTag[K] = ClassTag(keyClass)
+ implicit val vcm: ClassTag[V] = ClassTag(valueClass)
+ new JavaPairRDD(sc.sequenceFile(path, keyClass, valueClass, minSplits, cloneRecords))
+ }
+
+ /** Get an RDD for a Hadoop SequenceFile. */
def sequenceFile[K, V](path: String, keyClass: Class[K], valueClass: Class[V]):
JavaPairRDD[K, V] = {
implicit val kcm: ClassTag[K] = ClassTag(keyClass)
@@ -156,6 +168,15 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
new JavaPairRDD(sc.sequenceFile(path, keyClass, valueClass))
}
+ /** Get an RDD for a Hadoop SequenceFile. */
+ def sequenceFile[K, V](path: String, keyClass: Class[K], valueClass: Class[V],
+ cloneRecords: Boolean):
+ JavaPairRDD[K, V] = {
+ implicit val kcm: ClassTag[K] = ClassTag(keyClass)
+ implicit val vcm: ClassTag[V] = ClassTag(valueClass)
+ new JavaPairRDD(sc.sequenceFile(path, keyClass, valueClass, cloneRecords))
+ }
+
/**
* Load an RDD saved as a SequenceFile containing serialized objects, with NullWritable keys and
* BytesWritable values that contain a serialized partition. This is still an experimental storage
@@ -197,6 +218,37 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
new JavaPairRDD(sc.hadoopRDD(conf, inputFormatClass, keyClass, valueClass, minSplits))
}
+
+ /**
+ * Get an RDD for a Hadoop-readable dataset from a Hadoop JobConf given its InputFormat and other
+ * necessary info (e.g. file name for a filesystem-based dataset, table name for HyperTable),
+ * using the older MapReduce API (`org.apache.hadoop.mapred`).
+ *
+ * @param conf JobConf for setting up the dataset
+ * @param inputFormatClass Class of the [[InputFormat]]
+ * @param keyClass Class of the keys
+ * @param valueClass Class of the values
+ * @param minSplits Minimum number of Hadoop Splits to generate.
+ * @param cloneRecords If true, Spark will clone the records produced by Hadoop RecordReader.
+ * Most RecordReader implementations reuse wrapper objects across multiple
+ * records, and can cause problems in RDD collect or aggregation operations.
+ * By default the records are cloned in Spark. However, application
+ * programmers can explicitly disable the cloning for better performance.
+ */
+ def hadoopRDD[K, V, F <: InputFormat[K, V]](
+ conf: JobConf,
+ inputFormatClass: Class[F],
+ keyClass: Class[K],
+ valueClass: Class[V],
+ minSplits: Int,
+ cloneRecords: Boolean
+ ): JavaPairRDD[K, V] = {
+ implicit val kcm: ClassTag[K] = ClassTag(keyClass)
+ implicit val vcm: ClassTag[V] = ClassTag(valueClass)
+ new JavaPairRDD(sc.hadoopRDD(conf, inputFormatClass, keyClass, valueClass, minSplits,
+ cloneRecords))
+ }
+
/**
* Get an RDD for a Hadoop-readable dataset from a Hadooop JobConf giving its InputFormat and any
* other necessary info (e.g. file name for a filesystem-based dataset, table name for HyperTable,
@@ -231,6 +283,21 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
path: String,
inputFormatClass: Class[F],
keyClass: Class[K],
+ valueClass: Class[V],
+ minSplits: Int,
+ cloneRecords: Boolean
+ ): JavaPairRDD[K, V] = {
+ implicit val kcm: ClassTag[K] = ClassTag(keyClass)
+ implicit val vcm: ClassTag[V] = ClassTag(valueClass)
+ new JavaPairRDD(sc.hadoopFile(path, inputFormatClass, keyClass, valueClass,
+ minSplits, cloneRecords))
+ }
+
+ /** Get an RDD for a Hadoop file with an arbitrary InputFormat */
+ def hadoopFile[K, V, F <: InputFormat[K, V]](
+ path: String,
+ inputFormatClass: Class[F],
+ keyClass: Class[K],
valueClass: Class[V]
): JavaPairRDD[K, V] = {
implicit val kcm: ClassTag[K] = ClassTag(keyClass)
@@ -239,6 +306,20 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
inputFormatClass, keyClass, valueClass))
}
+ /** Get an RDD for a Hadoop file with an arbitrary InputFormat */
+ def hadoopFile[K, V, F <: InputFormat[K, V]](
+ path: String,
+ inputFormatClass: Class[F],
+ keyClass: Class[K],
+ valueClass: Class[V],
+ cloneRecords: Boolean
+ ): JavaPairRDD[K, V] = {
+ implicit val kcm: ClassTag[K] = ClassTag(keyClass)
+ implicit val vcm: ClassTag[V] = ClassTag(valueClass)
+ new JavaPairRDD(sc.hadoopFile(path,
+ inputFormatClass, keyClass, valueClass, cloneRecords = cloneRecords))
+ }
+
/**
* Get an RDD for a given Hadoop file with an arbitrary new API InputFormat
* and extra configuration options to pass to the input format.
@@ -258,6 +339,22 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
* Get an RDD for a given Hadoop file with an arbitrary new API InputFormat
* and extra configuration options to pass to the input format.
*/
+ def newAPIHadoopFile[K, V, F <: NewInputFormat[K, V]](
+ path: String,
+ fClass: Class[F],
+ kClass: Class[K],
+ vClass: Class[V],
+ conf: Configuration,
+ cloneRecords: Boolean): JavaPairRDD[K, V] = {
+ implicit val kcm: ClassTag[K] = ClassTag(kClass)
+ implicit val vcm: ClassTag[V] = ClassTag(vClass)
+ new JavaPairRDD(sc.newAPIHadoopFile(path, fClass, kClass, vClass, conf, cloneRecords))
+ }
+
+ /**
+ * Get an RDD for a given Hadoop file with an arbitrary new API InputFormat
+ * and extra configuration options to pass to the input format.
+ */
def newAPIHadoopRDD[K, V, F <: NewInputFormat[K, V]](
conf: Configuration,
fClass: Class[F],
@@ -268,6 +365,21 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
new JavaPairRDD(sc.newAPIHadoopRDD(conf, fClass, kClass, vClass))
}
+ /**
+ * Get an RDD for a given Hadoop file with an arbitrary new API InputFormat
+ * and extra configuration options to pass to the input format.
+ */
+ def newAPIHadoopRDD[K, V, F <: NewInputFormat[K, V]](
+ conf: Configuration,
+ fClass: Class[F],
+ kClass: Class[K],
+ vClass: Class[V],
+ cloneRecords: Boolean): JavaPairRDD[K, V] = {
+ implicit val kcm: ClassTag[K] = ClassTag(kClass)
+ implicit val vcm: ClassTag[V] = ClassTag(vClass)
+ new JavaPairRDD(sc.newAPIHadoopRDD(conf, fClass, kClass, vClass, cloneRecords))
+ }
+
/** Build the union of two or more RDDs. */
override def union[T](first: JavaRDD[T], rest: java.util.List[JavaRDD[T]]): JavaRDD[T] = {
val rdds: Seq[RDD[T]] = (Seq(first) ++ asScalaBuffer(rest)).map(_.rdd)
@@ -333,8 +445,9 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
sc.accumulable(initialValue)(param)
/**
- * Broadcast a read-only variable to the cluster, returning a [[org.apache.spark.Broadcast]] object for
- * reading it in distributed functions. The variable will be sent to each cluster only once.
+ * Broadcast a read-only variable to the cluster, returning a
+ * [[org.apache.spark.broadcast.Broadcast]] object for reading it in distributed functions.
+ * The variable will be sent to each cluster only once.
*/
def broadcast[T](value: T): Broadcast[T] = sc.broadcast(value)
diff --git a/core/src/main/scala/org/apache/spark/api/java/package.scala b/core/src/main/scala/org/apache/spark/api/java/package.scala
new file mode 100644
index 0000000000..8ec770046a
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/api/java/package.scala
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.api
+
+/** Spark Java programming APIs. */
+package object java {
+ // For package docs only
+}
diff --git a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala
index 6bfe2cb4a2..d113d40405 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala
@@ -17,13 +17,40 @@
package org.apache.spark.broadcast
-import java.io._
+import java.io.Serializable
import java.util.concurrent.atomic.AtomicLong
import org.apache.spark._
-private[spark]
-abstract class Broadcast[T](private[spark] val id: Long) extends Serializable {
+/**
+ * A broadcast variable. Broadcast variables allow the programmer to keep a read-only variable
+ * cached on each machine rather than shipping a copy of it with tasks. They can be used, for
+ * example, to give every node a copy of a large input dataset in an efficient manner. Spark also
+ * attempts to distribute broadcast variables using efficient broadcast algorithms to reduce
+ * communication cost.
+ *
+ * Broadcast variables are created from a variable `v` by calling [[SparkContext#broadcast]].
+ * The broadcast variable is a wrapper around `v`, and its value can be accessed by calling the
+ * `value` method. The interpreter session below shows this:
+ *
+ * {{{
+ * scala> val broadcastVar = sc.broadcast(Array(1, 2, 3))
+ * broadcastVar: spark.Broadcast[Array[Int]] = spark.Broadcast(b5c40191-a864-4c7d-b9bf-d87e1a4e787c)
+ *
+ * scala> broadcastVar.value
+ * res0: Array[Int] = Array(1, 2, 3)
+ * }}}
+ *
+ * After the broadcast variable is created, it should be used instead of the value `v` in any
+ * functions run on the cluster so that `v` is not shipped to the nodes more than once.
+ * In addition, the object `v` should not be modified after it is broadcast in order to ensure
+ * that all nodes get the same value of the broadcast variable (e.g. if the variable is shipped
+ * to a new node later).
+ *
+ * @param id A unique identifier for the broadcast variable.
+ * @tparam T Type of the data contained in the broadcast variable.
+ */
+abstract class Broadcast[T](val id: Long) extends Serializable {
def value: T
// We cannot have an abstract readObject here due to some weird issues with
diff --git a/core/src/main/scala/org/apache/spark/broadcast/package.scala b/core/src/main/scala/org/apache/spark/broadcast/package.scala
new file mode 100644
index 0000000000..01bf88629a
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/broadcast/package.scala
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark
+
+/**
+ * Package for broadcast variables. See [[broadcast.Broadcast]] for details.
+ */
+package object broadcast {
+ // For package docs only
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala
index cf6a23339d..460883ec7a 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.deploy.worker
import java.io.{File, FileOutputStream, IOException, InputStream}
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
index 1640d5fee0..6f6c101547 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.deploy.worker
import akka.actor._
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index 5182dcbb2a..312560d706 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -209,8 +209,11 @@ private[spark] class Worker(
logWarning("Invalid Master (" + masterUrl + ") attempted to launch executor.")
} else {
logInfo("Asked to launch executor %s/%d for %s".format(appId, execId, appDesc.name))
+ // TODO (pwendell): We shuld make sparkHome an Option[String] in
+ // ApplicationDescription to be more explicit about this.
+ val effectiveSparkHome = Option(execSparkHome_).getOrElse(sparkHome.getAbsolutePath)
val manager = new ExecutorRunner(appId, execId, appDesc, cores_, memory_,
- self, workerId, host, new File(execSparkHome_), workDir, akkaUrl, ExecutorState.RUNNING)
+ self, workerId, host, new File(effectiveSparkHome), workDir, akkaUrl, ExecutorState.RUNNING)
executors(appId + "/" + execId) = manager
manager.start()
coresUsed += cores_
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala
index 0e0d0cd626..1dc39c450e 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.deploy.worker
import akka.actor.{Actor, Address, AddressFromURIString}
diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
index 9c6b308804..f2feb406f7 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
@@ -66,7 +66,6 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part:
private type CoGroupValue = (Any, Int) // Int is dependency number
private type CoGroupCombiner = Seq[CoGroup]
- private val sparkConf = SparkEnv.get.conf
private var serializerClass: String = null
def setSerializer(cls: String): CoGroupedRDD[K] = {
@@ -106,7 +105,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part:
override val partitioner = Some(part)
override def compute(s: Partition, context: TaskContext): Iterator[(K, CoGroupCombiner)] = {
-
+ val sparkConf = SparkEnv.get.conf
val externalSorting = sparkConf.getBoolean("spark.shuffle.externalSorting", true)
val split = s.asInstanceOf[CoGroupPartition]
val numRdds = split.deps.size
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
index fc0ee07089..5ad00a1ed1 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
@@ -629,7 +629,7 @@ private[spark] class TaskSetManager(
}
// Also re-enqueue any tasks that were running on the node
for ((tid, info) <- taskInfos if info.running && info.executorId == execId) {
- handleFailedTask(tid, TaskState.KILLED, None)
+ handleFailedTask(tid, TaskState.FAILED, None)
}
}
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
index 6461deee32..ed53558566 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
@@ -81,7 +81,7 @@ private[spark] class BlockManager(
// Whether to compress RDD partitions that are stored serialized
val compressRdds = conf.getBoolean("spark.rdd.compress", false)
// Whether to compress shuffle output temporarily spilled to disk
- val compressShuffleSpill = conf.getBoolean("spark.shuffle.spill.compress", false)
+ val compressShuffleSpill = conf.getBoolean("spark.shuffle.spill.compress", true)
val heartBeatFrequency = BlockManager.getHeartBeatFrequency(conf)
diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala
index d9cb7fead5..8de7a328d1 100644
--- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala
+++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala
@@ -125,6 +125,23 @@ class DistributedSuite extends FunSuite with ShouldMatchers with BeforeAndAfter
assert(thrown.getMessage.contains("failed 4 times"))
}
+ test("repeatedly failing task that crashes JVM") {
+ // Ensures that if a task fails in a way that crashes the JVM, the job eventually fails rather
+ // than hanging due to retrying the failed task infinitely many times (eventually the
+ // standalone scheduler will remove the application, causing the job to hang waiting to
+ // reconnect to the master).
+ sc = new SparkContext(clusterUrl, "test")
+ failAfter(Span(100000, Millis)) {
+ val thrown = intercept[SparkException] {
+ // One of the tasks always fails.
+ sc.parallelize(1 to 10, 2).foreach { x => if (x == 1) System.exit(42) }
+ }
+ assert(thrown.getClass === classOf[SparkException])
+ System.out.println(thrown.getMessage)
+ assert(thrown.getMessage.contains("failed 4 times"))
+ }
+ }
+
test("caching") {
sc = new SparkContext(clusterUrl, "test")
val data = sc.parallelize(1 to 1000, 10).cache()
diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
index ef5936dd2f..fa49974db4 100644
--- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark
import org.scalatest.FunSuite
diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala
index 45dbcaffae..0c50261264 100644
--- a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.deploy.worker
import java.io.File
diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala
index 94d88d307a..1f1d8d1380 100644
--- a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.deploy.worker
diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala
index c3391f3e53..bb4dc0fcd3 100644
--- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.util.collection
import scala.collection.mutable.ArrayBuffer
diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md
index b9ff0af76f..6a9a8d6817 100644
--- a/docs/building-with-maven.md
+++ b/docs/building-with-maven.md
@@ -71,8 +71,8 @@ This setup works fine in IntelliJ IDEA 11.1.4. After opening the project via the
## Building Spark Debian Packages ##
-It includes support for building a Debian package containing a 'fat-jar' which includes the repl, the examples and bagel. This can be created by specifying the following profiles:
+The maven build includes support for building a Debian package containing the assembly 'fat-jar', PySpark, and the necessary scripts and configuration files. This can be created by specifying the following:
- $ mvn -Prepl-bin -Pdeb clean package
+ $ mvn -Pdeb -DskipTests clean package
-The debian package can then be found under repl/target. We added the short commit hash to the file name so that we can distinguish individual packages build for SNAPSHOT versions.
+The debian package can then be found under assembly/target. We added the short commit hash to the file name so that we can distinguish individual packages built for SNAPSHOT versions.
diff --git a/docs/configuration.md b/docs/configuration.md
index be06bd19be..da70cabba2 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -156,7 +156,7 @@ Apart from these, the following properties are also available, and may be useful
</tr>
<tr>
<td>spark.shuffle.spill.compress</td>
- <td>false</td>
+ <td>true</td>
<td>
Whether to compress data spilled during shuffles.
</td>
diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md
index 9fbde4eb09..3dfed7bea9 100644
--- a/docs/graphx-programming-guide.md
+++ b/docs/graphx-programming-guide.md
@@ -18,7 +18,7 @@ title: GraphX Programming Guide
GraphX is the new (alpha) Spark API for graphs and graph-parallel computation. At a high-level,
GraphX extends the Spark [RDD](api/core/index.html#org.apache.spark.rdd.RDD) by introducing the
-[Resilient Distributed property Graph (RDG)](#property_graph): a directed multigraph with properties
+[Resilient Distributed Property Graph](#property_graph): a directed multigraph with properties
attached to each vertex and edge. To support graph computation, GraphX exposes a set of fundamental
operators (e.g., [subgraph](#structural_operators), [joinVertices](#join_operators), and
[mapReduceTriplets](#mrTriplets)) as well as an optimized variant of the [Pregel](#pregel) API. In
@@ -29,7 +29,7 @@ addition, GraphX includes a growing collection of graph [algorithms](#graph_algo
From social networks to language modeling, the growing scale and importance of
graph data has driven the development of numerous new *graph-parallel* systems
-(e.g., [Giraph](http://http://giraph.apache.org) and
+(e.g., [Giraph](http://giraph.apache.org) and
[GraphLab](http://graphlab.org)). By restricting the types of computation that can be
expressed and introducing new techniques to partition and distribute graphs,
these systems can efficiently execute sophisticated graph algorithms orders of
@@ -43,12 +43,25 @@ magnitude faster than more general *data-parallel* systems.
<!-- Images are downsized intentionally to improve quality on retina displays -->
</p>
-However, the same restrictions that enable these substantial performance gains
-also make it difficult to express many of the important stages in a typical graph-analytics pipeline:
-constructing the graph, modifying its structure, or expressing computation that
-spans multiple graphs. As a consequence, existing graph analytics pipelines
-compose graph-parallel and data-parallel systems, leading to extensive data
-movement and duplication and a complicated programming model.
+However, the same restrictions that enable these substantial performance gains also make it
+difficult to express many of the important stages in a typical graph-analytics pipeline:
+constructing the graph, modifying its structure, or expressing computation that spans multiple
+graphs. Furthermore, how we look at data depends on our objectives and the same raw data may have
+many different table and graph views.
+
+<p style="text-align: center;">
+ <img src="img/tables_and_graphs.png"
+ title="Tables and Graphs"
+ alt="Tables and Graphs"
+ width="50%" />
+ <!-- Images are downsized intentionally to improve quality on retina displays -->
+</p>
+
+As a consequence, it is often necessary to be able to move between table and graph views of the same
+physical data and to leverage the properties of each view to easily and efficiently express
+computation. However, existing graph analytics pipelines must compose graph-parallel and data-
+parallel systems, leading to extensive data movement and duplication and a complicated programming
+model.
<p style="text-align: center;">
<img src="img/graph_analytics_pipeline.png"
@@ -95,17 +108,20 @@ with user defined objects attached to each vertex and edge. A directed multigra
graph with potentially multiple parallel edges sharing the same source and destination vertex. The
ability to support parallel edges simplifies modeling scenarios where there can be multiple
relationships (e.g., co-worker and friend) between the same vertices. Each vertex is keyed by a
-*unique* 64-bit long identifier (`VertexId`). Similarly, edges have corresponding source and
-destination vertex identifiers. GraphX does not impose any ordering or constraints on the vertex
-identifiers. The property graph is parameterized over the vertex `VD` and edge `ED` types. These
+*unique* 64-bit long identifier (`VertexID`). GraphX does not impose any ordering constraints on
+the vertex identifiers. Similarly, edges have corresponding source and destination vertex
+identifiers.
+
+The property graph is parameterized over the vertex (`VD`) and edge (`ED`) types. These
are the types of the objects associated with each vertex and edge respectively.
-> GraphX optimizes the representation of `VD` and `ED` when they are plain old data-types (e.g.,
-> int, double, etc...) reducing the in memory footprint.
+> GraphX optimizes the representation of vertex and edge types when they are plain old data-types
+> (e.g., int, double, etc...) reducing the in memory footprint by storing them in specialized
+> arrays.
-In some cases we may wish to have vertices with different property types in the same graph. This can
-be accomplished through inheritance. For example to model users and products as a bipartite graph
-we might do the following:
+In some cases it may be desirable to have vertices with different property types in the same graph.
+This can be accomplished through inheritance. For example to model users and products as a
+bipartite graph we might do the following:
{% highlight scala %}
class VertexProperty()
@@ -116,9 +132,11 @@ var graph: Graph[VertexProperty, String] = null
{% endhighlight %}
Like RDDs, property graphs are immutable, distributed, and fault-tolerant. Changes to the values or
-structure of the graph are accomplished by producing a new graph with the desired changes. The graph
-is partitioned across the workers using a range of vertex-partitioning heuristics. As with RDDs,
-each partition of the graph can be recreated on a different machine in the event of a failure.
+structure of the graph are accomplished by producing a new graph with the desired changes. Note
+that substantial parts of the original graph (i.e., unaffected structure, attributes, and indicies)
+are reused in the new graph reducing the cost of this inherently functional data-structure. The
+graph is partitioned across the workers using a range of vertex-partitioning heuristics. As with
+RDDs, each partition of the graph can be recreated on a different machine in the event of a failure.
Logically the property graph corresponds to a pair of typed collections (RDDs) encoding the
properties for each vertex and edge. As a consequence, the graph class contains members to access
@@ -131,12 +149,12 @@ class Graph[VD, ED] {
}
{% endhighlight %}
-The classes `VertexRDD[VD]` and `EdgeRDD[ED]` extend and are optimized versions of `RDD[(VertexId,
+The classes `VertexRDD[VD]` and `EdgeRDD[ED]` extend and are optimized versions of `RDD[(VertexID,
VD)]` and `RDD[Edge[ED]]` respectively. Both `VertexRDD[VD]` and `EdgeRDD[ED]` provide additional
functionality built around graph computation and leverage internal optimizations. We discuss the
`VertexRDD` and `EdgeRDD` API in greater detail in the section on [vertex and edge
RDDs](#vertex_and_edge_rdds) but for now they can be thought of as simply RDDs of the form:
-`RDD[(VertexId, VD)]` and `RDD[Edge[ED]]`.
+`RDD[(VertexID, VD)]` and `RDD[Edge[ED]]`.
### Example Property Graph
@@ -168,7 +186,7 @@ code constructs a graph from a collection of RDDs:
// Assume the SparkContext has already been constructed
val sc: SparkContext
// Create an RDD for the vertices
-val users: RDD[(VertexID, (String, String))] =
+val users: RDD[(VertexId, (String, String))] =
sc.parallelize(Array((3L, ("rxin", "student")), (7L, ("jgonzal", "postdoc")),
(5L, ("franklin", "prof")), (2L, ("istoica", "prof"))))
// Create an RDD for edges
@@ -183,7 +201,7 @@ val graph = Graph(users, relationships, defaultUser)
In the above example we make use of the [`Edge`][Edge] case class. Edges have a `srcId` and a
`dstId` corresponding to the source and destination vertex identifiers. In addition, the `Edge`
-class contains the `attr` member which contains the edge property.
+class has an `attr` member which stores the edge property.
[Edge]: api/graphx/index.html#org.apache.spark.graphx.Edge
@@ -199,7 +217,7 @@ graph.edges.filter(e => e.srcId > e.dstId).count
{% endhighlight %}
> Note that `graph.vertices` returns an `VertexRDD[(String, String)]` which extends
-> `RDD[(VertexId, (String, String))]` and so we use the scala `case` expression to deconstruct the
+> `RDD[(VertexID, (String, String))]` and so we use the scala `case` expression to deconstruct the
> tuple. On the other hand, `graph.edges` returns an `EdgeRDD` containing `Edge[String]` objects.
> We could have also used the case class type constructor as in the following:
> {% highlight scala %}
@@ -266,6 +284,75 @@ able to support different graph representations in the future. Each graph repre
provide implementations of the core operations and reuse many of the useful operations defined in
[`GraphOps`][GraphOps].
+### Summary List of Operators
+The following is a quick summary of the functionality defined in both [`Graph`][Graph] and
+[`GraphOps`][GraphOps] but presented as members of Graph for simplicity. Note that some function
+signatures have been simplified (e.g., default arguments and type constraints removed) and some more
+advanced functionality has been removed so please consult the API docs for the official list of
+operations.
+
+{% highlight scala %}
+/** Summary of the functionality in the property graph */
+class Graph[VD, ED] {
+ // Information about the Graph ===================================================================
+ val numEdges: Long
+ val numVertices: Long
+ val inDegrees: VertexRDD[Int]
+ val outDegrees: VertexRDD[Int]
+ val degrees: VertexRDD[Int]
+ // Views of the graph as collections =============================================================
+ val vertices: VertexRDD[VD]
+ val edges: EdgeRDD[ED]
+ val triplets: RDD[EdgeTriplet[VD, ED]]
+ // Functions for caching graphs ==================================================================
+ def persist(newLevel: StorageLevel = StorageLevel.MEMORY_ONLY): Graph[VD, ED]
+ def cache(): Graph[VD, ED]
+ def unpersistVertices(blocking: Boolean = true): Graph[VD, ED]
+ // Change the partitioning heuristic ============================================================
+ def partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED]
+ // Transform vertex and edge attributes ==========================================================
+ def mapVertices[VD2](map: (VertexID, VD) => VD2): Graph[VD2, ED]
+ def mapEdges[ED2](map: Edge[ED] => ED2): Graph[VD, ED2]
+ def mapEdges[ED2](map: (PartitionID, Iterator[Edge[ED]]) => Iterator[ED2]): Graph[VD, ED2]
+ def mapTriplets[ED2](map: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2]
+ def mapTriplets[ED2](map: (PartitionID, Iterator[EdgeTriplet[VD, ED]]) => Iterator[ED2])
+ : Graph[VD, ED2]
+ // Modify the graph structure ====================================================================
+ def reverse: Graph[VD, ED]
+ def subgraph(
+ epred: EdgeTriplet[VD,ED] => Boolean = (x => true),
+ vpred: (VertexID, VD) => Boolean = ((v, d) => true))
+ : Graph[VD, ED]
+ def mask[VD2, ED2](other: Graph[VD2, ED2]): Graph[VD, ED]
+ def groupEdges(merge: (ED, ED) => ED): Graph[VD, ED]
+ // Join RDDs with the graph ======================================================================
+ def joinVertices[U](table: RDD[(VertexID, U)])(mapFunc: (VertexID, VD, U) => VD): Graph[VD, ED]
+ def outerJoinVertices[U, VD2](other: RDD[(VertexID, U)])
+ (mapFunc: (VertexID, VD, Option[U]) => VD2)
+ : Graph[VD2, ED]
+ // Aggregate information about adjacent triplets =================================================
+ def collectNeighborIds(edgeDirection: EdgeDirection): VertexRDD[Array[VertexID]]
+ def collectNeighbors(edgeDirection: EdgeDirection): VertexRDD[Array[(VertexID, VD)]]
+ def mapReduceTriplets[A: ClassTag](
+ mapFunc: EdgeTriplet[VD, ED] => Iterator[(VertexID, A)],
+ reduceFunc: (A, A) => A,
+ activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None)
+ : VertexRDD[A]
+ // Iterative graph-parallel computation ==========================================================
+ def pregel[A](initialMsg: A, maxIterations: Int, activeDirection: EdgeDirection)(
+ vprog: (VertexID, VD, A) => VD,
+ sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexID,A)],
+ mergeMsg: (A, A) => A)
+ : Graph[VD, ED]
+ // Basic graph algorithms ========================================================================
+ def pageRank(tol: Double, resetProb: Double = 0.15): Graph[Double, Double]
+ def connectedComponents(): Graph[VertexID, ED]
+ def triangleCount(): Graph[Int, ED]
+ def stronglyConnectedComponents(numIter: Int): Graph[VertexID, ED]
+}
+{% endhighlight %}
+
+
## Property Operators
In direct analogy to the RDD `map` operator, the property
@@ -273,7 +360,7 @@ graph contains the following:
{% highlight scala %}
class Graph[VD, ED] {
- def mapVertices[VD2](map: (VertexID, VD) => VD2): Graph[VD2, ED]
+ def mapVertices[VD2](map: (VertexId, VD) => VD2): Graph[VD2, ED]
def mapEdges[ED2](map: Edge[ED] => ED2): Graph[VD, ED2]
def mapTriplets[ED2](map: EdgeTriplet[VD, ED] => ED2): Graph[VD, ED2]
}
@@ -295,7 +382,7 @@ val newGraph = Graph(newVertices, graph.edges)
val newGraph = graph.mapVertices((id, attr) => mapUdf(id, attr))
{% endhighlight %}
-[Graph.mapVertices]: api/graphx/index.html#org.apache.spark.graphx.Graph@mapVertices[VD2]((VertexID,VD)⇒VD2)(ClassTag[VD2]):Graph[VD2,ED]
+[Graph.mapVertices]: api/graphx/index.html#org.apache.spark.graphx.Graph@mapVertices[VD2]((VertexId,VD)⇒VD2)(ClassTag[VD2]):Graph[VD2,ED]
These operators are often used to initialize the graph for a particular computation or project away
unnecessary properties. For example, given a graph with the out-degrees as the vertex properties
@@ -321,7 +408,7 @@ add more in the future. The following is a list of the basic structural operato
class Graph[VD, ED] {
def reverse: Graph[VD, ED]
def subgraph(epred: EdgeTriplet[VD,ED] => Boolean,
- vpred: (VertexID, VD) => Boolean): Graph[VD, ED]
+ vpred: (VertexId, VD) => Boolean): Graph[VD, ED]
def mask[VD2, ED2](other: Graph[VD2, ED2]): Graph[VD, ED]
def groupEdges(merge: (ED, ED) => ED): Graph[VD,ED]
}
@@ -340,11 +427,11 @@ satisfy the edge predicate *and connect vertices that satisfy the vertex predica
operator can be used in number of situations to restrict the graph to the vertices and edges of
interest or eliminate broken links. For example in the following code we remove broken links:
-[Graph.subgraph]: api/graphx/index.html#org.apache.spark.graphx.Graph@subgraph((EdgeTriplet[VD,ED])⇒Boolean,(VertexID,VD)⇒Boolean):Graph[VD,ED]
+[Graph.subgraph]: api/graphx/index.html#org.apache.spark.graphx.Graph@subgraph((EdgeTriplet[VD,ED])⇒Boolean,(VertexId,VD)⇒Boolean):Graph[VD,ED]
{% highlight scala %}
// Create an RDD for the vertices
-val users: RDD[(VertexID, (String, String))] =
+val users: RDD[(VertexId, (String, String))] =
sc.parallelize(Array((3L, ("rxin", "student")), (7L, ("jgonzal", "postdoc")),
(5L, ("franklin", "prof")), (2L, ("istoica", "prof")),
(4L, ("peter", "student"))))
@@ -407,9 +494,9 @@ using the *join* operators. Below we list the key join operators:
{% highlight scala %}
class Graph[VD, ED] {
- def joinVertices[U](table: RDD[(VertexID, U)])(map: (VertexID, VD, U) => VD)
+ def joinVertices[U](table: RDD[(VertexId, U)])(map: (VertexId, VD, U) => VD)
: Graph[VD, ED]
- def outerJoinVertices[U, VD2](table: RDD[(VertexID, U)])(map: (VertexID, VD, Option[U]) => VD2)
+ def outerJoinVertices[U, VD2](table: RDD[(VertexId, U)])(map: (VertexId, VD, Option[U]) => VD2)
: Graph[VD2, ED]
}
{% endhighlight %}
@@ -419,13 +506,13 @@ returns a new graph with the vertex properties obtained by applying the user def
to the result of the joined vertices. Vertices without a matching value in the RDD retain their
original value.
-[GraphOps.joinVertices]: api/graphx/index.html#org.apache.spark.graphx.GraphOps@joinVertices[U](RDD[(VertexID,U)])((VertexID,VD,U)⇒VD)(ClassTag[U]):Graph[VD,ED]
+[GraphOps.joinVertices]: api/graphx/index.html#org.apache.spark.graphx.GraphOps@joinVertices[U](RDD[(VertexId,U)])((VertexId,VD,U)⇒VD)(ClassTag[U]):Graph[VD,ED]
> Note that if the RDD contains more than one value for a given vertex only one will be used. It
> is therefore recommended that the input RDD be first made unique using the following which will
> also *pre-index* the resulting values to substantially accelerate the subsequent join.
> {% highlight scala %}
-val nonUniqueCosts: RDD[(VertexId, Double)]
+val nonUniqueCosts: RDD[(VertexID, Double)]
val uniqueCosts: VertexRDD[Double] =
graph.vertices.aggregateUsingIndex(nonUnique, (a,b) => a + b)
val joinedGraph = graph.joinVertices(uniqueCosts)(
@@ -438,7 +525,7 @@ property type. Because not all vertices may have a matching value in the input
function takes an `Option` type. For example, we can setup a graph for PageRank by initializing
vertex properties with their `outDegree`.
-[Graph.outerJoinVertices]: api/graphx/index.html#org.apache.spark.graphx.Graph@outerJoinVertices[U,VD2](RDD[(VertexID,U)])((VertexID,VD,Option[U])⇒VD2)(ClassTag[U],ClassTag[VD2]):Graph[VD2,ED]
+[Graph.outerJoinVertices]: api/graphx/index.html#org.apache.spark.graphx.Graph@outerJoinVertices[U,VD2](RDD[(VertexId,U)])((VertexId,VD,Option[U])⇒VD2)(ClassTag[U],ClassTag[VD2]):Graph[VD2,ED]
{% highlight scala %}
@@ -457,7 +544,7 @@ val degreeGraph = graph.outerJoinVertices(outDegrees) { (id, oldAttr, outDegOpt)
> provide type annotation for the user defined function:
> {% highlight scala %}
val joinedGraph = graph.joinVertices(uniqueCosts,
- (id: VertexId, oldCost: Double, extraCost: Double) => oldCost + extraCost)
+ (id: VertexID, oldCost: Double, extraCost: Double) => oldCost + extraCost)
{% endhighlight %}
@@ -472,7 +559,7 @@ PageRank Value, shortest path to the source, and smallest reachable vertex id).
### Map Reduce Triplets (mapReduceTriplets)
<a name="mrTriplets"></a>
-[Graph.mapReduceTriplets]: api/graphx/index.html#org.apache.spark.graphx.Graph@mapReduceTriplets[A](mapFunc:org.apache.spark.graphx.EdgeTriplet[VD,ED]=&gt;Iterator[(org.apache.spark.graphx.VertexID,A)],reduceFunc:(A,A)=&gt;A,activeSetOpt:Option[(org.apache.spark.graphx.VertexRDD[_],org.apache.spark.graphx.EdgeDirection)])(implicitevidence$10:scala.reflect.ClassTag[A]):org.apache.spark.graphx.VertexRDD[A]
+[Graph.mapReduceTriplets]: api/graphx/index.html#org.apache.spark.graphx.Graph@mapReduceTriplets[A](mapFunc:org.apache.spark.graphx.EdgeTriplet[VD,ED]=&gt;Iterator[(org.apache.spark.graphx.VertexId,A)],reduceFunc:(A,A)=&gt;A,activeSetOpt:Option[(org.apache.spark.graphx.VertexRDD[_],org.apache.spark.graphx.EdgeDirection)])(implicitevidence$10:scala.reflect.ClassTag[A]):org.apache.spark.graphx.VertexRDD[A]
The core (heavily optimized) aggregation primitive in GraphX is the
[`mapReduceTriplets`][Graph.mapReduceTriplets] operator:
@@ -480,7 +567,7 @@ The core (heavily optimized) aggregation primitive in GraphX is the
{% highlight scala %}
class Graph[VD, ED] {
def mapReduceTriplets[A](
- map: EdgeTriplet[VD, ED] => Iterator[(VertexID, A)],
+ map: EdgeTriplet[VD, ED] => Iterator[(VertexId, A)],
reduce: (A, A) => A)
: VertexRDD[A]
}
@@ -495,26 +582,26 @@ containing the aggregate message (of type `A`) destined to each vertex. Vertice
receive a message are not included in the returned `VertexRDD`.
<blockquote>
-<p>
-Note that <code>mapReduceTriplets</code> takes an additional optional <code>activeSet</code>
-(see API docs) which restricts the map phase to edges adjacent to the vertices in the provided
-<code>VertexRDD</code>:
-</p>
+
+<p>Note that <code>mapReduceTriplets</code> takes an additional optional <code>activeSet</code>
+(not shown above see API docs for details) which restricts the map phase to edges adjacent to the
+vertices in the provided <code>VertexRDD</code>: </p>
+
{% highlight scala %}
activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None
{% endhighlight %}
-<p>
-The EdgeDirection specifies which edges adjacent to the vertex set are included in the map phase. If
-the direction is <code>In</code>, <code>mapFunc</code> will only be run only on edges with
-destination in the active set. If the direction is <code>Out</code>, <code>mapFunc</code> will only
-be run only on edges originating from vertices in the active set. If the direction is
-<code>Either</code>, <code>mapFunc</code> will be run only on edges with <i>either</i> vertex in the
-active set. If the direction is <code>Both</code>, <code>mapFunc</code> will be run only on edges
-with both vertices in the active set. The active set must be derived from the set of vertices in
-the graph. Restricting computation to triplets adjacent to a subset of the vertices is often
-necessary in incremental iterative computation and is a key part of the GraphX implementation of
-Pregel.
-</p>
+
+<p>The EdgeDirection specifies which edges adjacent to the vertex set are included in the map
+phase. If the direction is <code>In</code>, then the user defined <code>map</code> function will
+only be run only on edges with the destination vertex in the active set. If the direction is
+<code>Out</code>, then the <code>map</code> function will only be run only on edges originating from
+vertices in the active set. If the direction is <code>Either</code>, then the <code>map</code>
+function will be run only on edges with <i>either</i> vertex in the active set. If the direction is
+<code>Both</code>, then the <code>map</code> function will be run only on edges with both vertices
+in the active set. The active set must be derived from the set of vertices in the graph.
+Restricting computation to triplets adjacent to a subset of the vertices is often necessary in
+incremental iterative computation and is a key part of the GraphX implementation of Pregel. </p>
+
</blockquote>
In the following example we use the `mapReduceTriplets` operator to compute the average age of the
@@ -547,8 +634,8 @@ val avgAgeOfOlderFollowers: VertexRDD[Double] =
avgAgeOfOlderFollowers.collect.foreach(println(_))
{% endhighlight %}
-> Note that the `mapReduceTriplets` operation performs optimally when the messages (and their sums)
-> are constant sized (e.g., floats and addition instead of lists and concatenation). More
+> Note that the `mapReduceTriplets` operation performs optimally when the messages (and the sums of
+> messages) are constant sized (e.g., floats and addition instead of lists and concatenation). More
> precisely, the result of `mapReduceTriplets` should ideally be sub-linear in the degree of each
> vertex.
@@ -562,13 +649,13 @@ compute the max in, out, and total degrees:
{% highlight scala %}
// Define a reduce operation to compute the highest degree vertex
-def max(a: (VertexID, Int), b: (VertexID, Int)): (VertexID, Int) = {
+def max(a: (VertexId, Int), b: (VertexId, Int)): (VertexId, Int) = {
if (a._2 > b._2) a else b
}
// Compute the max degrees
-val maxInDegree: (VertexID, Int) = graph.inDegrees.reduce(max)
-val maxOutDegree: (VertexID, Int) = graph.outDegrees.reduce(max)
-val maxDegrees: (VertexID, Int) = graph.degrees.reduce(max)
+val maxInDegree: (VertexId, Int) = graph.inDegrees.reduce(max)
+val maxOutDegree: (VertexId, Int) = graph.outDegrees.reduce(max)
+val maxDegrees: (VertexId, Int) = graph.degrees.reduce(max)
{% endhighlight %}
### Collecting Neighbors
@@ -578,14 +665,14 @@ attributes at each vertex. This can be easily accomplished using the
[`collectNeighborIds`][GraphOps.collectNeighborIds] and the
[`collectNeighbors`][GraphOps.collectNeighbors] operators.
-[GraphOps.collectNeighborIds]: api/graphx/index.html#org.apache.spark.graphx.GraphOps@collectNeighborIds(EdgeDirection):VertexRDD[Array[VertexID]]
-[GraphOps.collectNeighbors]: api/graphx/index.html#org.apache.spark.graphx.GraphOps@collectNeighbors(EdgeDirection):VertexRDD[Array[(VertexID,VD)]]
+[GraphOps.collectNeighborIds]: api/graphx/index.html#org.apache.spark.graphx.GraphOps@collectNeighborIds(EdgeDirection):VertexRDD[Array[VertexId]]
+[GraphOps.collectNeighbors]: api/graphx/index.html#org.apache.spark.graphx.GraphOps@collectNeighbors(EdgeDirection):VertexRDD[Array[(VertexId,VD)]]
{% highlight scala %}
class GraphOps[VD, ED] {
- def collectNeighborIds(edgeDirection: EdgeDirection): VertexRDD[Array[VertexID]]
- def collectNeighbors(edgeDirection: EdgeDirection): VertexRDD[ Array[(VertexID, VD)] ]
+ def collectNeighborIds(edgeDirection: EdgeDirection): VertexRDD[Array[VertexId]]
+ def collectNeighbors(edgeDirection: EdgeDirection): VertexRDD[ Array[(VertexId, VD)] ]
}
{% endhighlight %}
@@ -593,11 +680,20 @@ class GraphOps[VD, ED] {
> substantial communication. If possible try expressing the same computation using the
> `mapReduceTriplets` operator directly.
+## Caching and Uncaching
+
+In Spark, RDDs are not persisted in memory by default. To avoid recomputation, they must be explicitly cached when using them multiple times (see the [Spark Programming Guide][RDD Persistence]). Graphs in GraphX behave the same way. **When using a graph multiple times, make sure to call [`Graph.cache()`][Graph.cache] on it first.**
+
+[RDD Persistence]: scala-programming-guide.html#rdd-persistence
+[Graph.cache]: api/graphx/index.html#org.apache.spark.graphx.Graph@cache():Graph[VD,ED]
+
+In iterative computations, *uncaching* may also be necessary for best performance. By default, cached RDDs and graphs will remain in memory until memory pressure forces them to be evicted in LRU order. For iterative computation, intermediate results from previous iterations will fill up the cache. Though they will eventually be evicted, the unnecessary data stored in memory will slow down garbage collection. It would be more efficient to uncache intermediate results as soon as they are no longer necessary. This involves materializing (caching and forcing) a graph or RDD every iteration, uncaching all other datasets, and only using the materialized dataset in future iterations. However, because graphs are composed of multiple RDDs, it can be difficult to unpersist them correctly. **For iterative computation we recommend using the Pregel API, which correctly unpersists intermediate results.**
+
# Pregel API
<a name="pregel"></a>
Graphs are inherently recursive data-structures as properties of vertices depend on properties of
-their neighbors which intern depend on properties of *their* neighbors. As a
+their neighbors which in turn depend on properties of *their* neighbors. As a
consequence many important graph algorithms iteratively recompute the properties of each vertex
until a fixed-point condition is reached. A range of graph-parallel abstractions have been proposed
to express these iterative algorithms. GraphX exposes a Pregel-like operator which is a fusion of
@@ -620,7 +716,7 @@ messages remaining.
The following is the type signature of the [Pregel operator][GraphOps.pregel] as well as a *sketch*
of its implementation (note calls to graph.cache have been removed):
-[GraphOps.pregel]: api/graphx/index.html#org.apache.spark.graphx.GraphOps@pregel[A](A,Int,EdgeDirection)((VertexID,VD,A)⇒VD,(EdgeTriplet[VD,ED])⇒Iterator[(VertexID,A)],(A,A)⇒A)(ClassTag[A]):Graph[VD,ED]
+[GraphOps.pregel]: api/graphx/index.html#org.apache.spark.graphx.GraphOps@pregel[A](A,Int,EdgeDirection)((VertexId,VD,A)⇒VD,(EdgeTriplet[VD,ED])⇒Iterator[(VertexId,A)],(A,A)⇒A)(ClassTag[A]):Graph[VD,ED]
{% highlight scala %}
class GraphOps[VD, ED] {
@@ -628,8 +724,8 @@ class GraphOps[VD, ED] {
(initialMsg: A,
maxIter: Int = Int.MaxValue,
activeDir: EdgeDirection = EdgeDirection.Out)
- (vprog: (VertexID, VD, A) => VD,
- sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexID, A)],
+ (vprog: (VertexId, VD, A) => VD,
+ sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexId, A)],
mergeMsg: (A, A) => A)
: Graph[VD, ED] = {
// Receive the initial message at each vertex
@@ -674,7 +770,7 @@ import org.apache.spark.graphx.util.GraphGenerators
// A graph with edge attributes containing distances
val graph: Graph[Int, Double] =
GraphGenerators.logNormalGraph(sc, numVertices = 100).mapEdges(e => e.attr.toDouble)
-val sourceId: VertexID = 42 // The ultimate source
+val sourceId: VertexId = 42 // The ultimate source
// Initialize the graph such that all vertices except the root have distance infinity.
val initialGraph = graph.mapVertices((id, _) => if (id == sourceId) 0.0 else Double.PositiveInfinity)
val sssp = initialGraph.pregel(Double.PositiveInfinity)(
@@ -721,7 +817,7 @@ It creates a `Graph` from the specified edges, automatically creating any vertic
{% highlight scala %}
object Graph {
def apply[VD, ED](
- vertices: RDD[(VertexID, VD)],
+ vertices: RDD[(VertexId, VD)],
edges: RDD[Edge[ED]],
defaultVertexAttr: VD = null)
: Graph[VD, ED]
@@ -731,7 +827,7 @@ object Graph {
defaultValue: VD): Graph[VD, ED]
def fromEdgeTuples[VD](
- rawEdges: RDD[(VertexID, VertexID)],
+ rawEdges: RDD[(VertexId, VertexId)],
defaultValue: VD,
uniqueEdges: Option[PartitionStrategy] = None): Graph[VD, Int]
@@ -747,8 +843,8 @@ object Graph {
[PartitionStrategy]: api/graphx/index.html#org.apache.spark.graphx.PartitionStrategy$
[GraphLoader.edgeListFile]: api/graphx/index.html#org.apache.spark.graphx.GraphLoader$@edgeListFile(SparkContext,String,Boolean,Int):Graph[Int,Int]
-[Graph.apply]: api/graphx/index.html#org.apache.spark.graphx.Graph$@apply[VD,ED](RDD[(VertexID,VD)],RDD[Edge[ED]],VD)(ClassTag[VD],ClassTag[ED]):Graph[VD,ED]
-[Graph.fromEdgeTuples]: api/graphx/index.html#org.apache.spark.graphx.Graph$@fromEdgeTuples[VD](RDD[(VertexID,VertexID)],VD,Option[PartitionStrategy])(ClassTag[VD]):Graph[VD,Int]
+[Graph.apply]: api/graphx/index.html#org.apache.spark.graphx.Graph$@apply[VD,ED](RDD[(VertexId,VD)],RDD[Edge[ED]],VD)(ClassTag[VD],ClassTag[ED]):Graph[VD,ED]
+[Graph.fromEdgeTuples]: api/graphx/index.html#org.apache.spark.graphx.Graph$@fromEdgeTuples[VD](RDD[(VertexId,VertexId)],VD,Option[PartitionStrategy])(ClassTag[VD]):Graph[VD,Int]
[Graph.fromEdges]: api/graphx/index.html#org.apache.spark.graphx.Graph$@fromEdges[VD,ED](RDD[Edge[ED]],VD)(ClassTag[VD],ClassTag[ED]):Graph[VD,ED]
# Vertex and Edge RDDs
@@ -761,47 +857,46 @@ respectively. In this section we review some of the additional useful functiona
## VertexRDDs
-The `VertexRDD[A]` extends the more traditional `RDD[(VertexId, A)]` but adds the additional
-constraint that each `VertexId` occurs only *once*. Moreover, `VertexRDD[A]` represents a *set* of
-vertices each with an attribute of type `A`. Internally, this is achieved by storing the vertex
-attributes in a reusable hash-map data-structure. As a consequence if two `VertexRDD`s are derived
-from the same base `VertexRDD` (e.g., by `filter` or `mapValues`) they can be joined in constant
-time without hash evaluations. To leverage this indexed data-structure, the `VertexRDD` exposes the
-following additional functionality:
+The `VertexRDD[A]` extends `RDD[(VertexID, A)]` and adds the additional constraint that each
+`VertexID` occurs only *once*. Moreover, `VertexRDD[A]` represents a *set* of vertices each with an
+attribute of type `A`. Internally, this is achieved by storing the vertex attributes in a reusable
+hash-map data-structure. As a consequence if two `VertexRDD`s are derived from the same base
+`VertexRDD` (e.g., by `filter` or `mapValues`) they can be joined in constant time without hash
+evaluations. To leverage this indexed data-structure, the `VertexRDD` exposes the following
+additional functionality:
{% highlight scala %}
-class VertexRDD[VD] {
+class VertexRDD[VD] extends RDD[(VertexID, VD)] {
// Filter the vertex set but preserves the internal index
- def filter(pred: Tuple2[VertexID, VD] => Boolean): VertexRDD[VD]
+ def filter(pred: Tuple2[VertexId, VD] => Boolean): VertexRDD[VD]
// Transform the values without changing the ids (preserves the internal index)
def mapValues[VD2](map: VD => VD2): VertexRDD[VD2]
- def mapValues[VD2](map: (VertexID, VD) => VD2): VertexRDD[VD2]
+ def mapValues[VD2](map: (VertexId, VD) => VD2): VertexRDD[VD2]
// Remove vertices from this set that appear in the other set
def diff(other: VertexRDD[VD]): VertexRDD[VD]
// Join operators that take advantage of the internal indexing to accelerate joins (substantially)
- def leftJoin[VD2, VD3](other: RDD[(VertexID, VD2)])(f: (VertexID, VD, Option[VD2]) => VD3): VertexRDD[VD3]
- def innerJoin[U, VD2](other: RDD[(VertexID, U)])(f: (VertexID, VD, U) => VD2): VertexRDD[VD2]
+ def leftJoin[VD2, VD3](other: RDD[(VertexId, VD2)])(f: (VertexId, VD, Option[VD2]) => VD3): VertexRDD[VD3]
+ def innerJoin[U, VD2](other: RDD[(VertexId, U)])(f: (VertexId, VD, U) => VD2): VertexRDD[VD2]
// Use the index on this RDD to accelerate a `reduceByKey` operation on the input RDD.
- def aggregateUsingIndex[VD2](other: RDD[(VertexID, VD2)], reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2]
+ def aggregateUsingIndex[VD2](other: RDD[(VertexId, VD2)], reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2]
}
{% endhighlight %}
Notice, for example, how the `filter` operator returns an `VertexRDD`. Filter is actually
implemented using a `BitSet` thereby reusing the index and preserving the ability to do fast joins
with other `VertexRDD`s. Likewise, the `mapValues` operators do not allow the `map` function to
-change the `VertexId` thereby enabling the same `HashMap` data-structures to be reused. Both the
+change the `VertexID` thereby enabling the same `HashMap` data-structures to be reused. Both the
`leftJoin` and `innerJoin` are able to identify when joining two `VertexRDD`s derived from the same
`HashMap` and implement the join by linear scan rather than costly point lookups.
-The `aggregateUsingIndex` operator can be slightly confusing but is also useful for efficient
-construction of a new `VertexRDD` from an `RDD[(VertexId, A)]`. Conceptually, if I have constructed
-a `VertexRDD[B]` over a set of vertices, *which is a super-set* of the vertices in some
-`RDD[(VertexId, A)]` then I can reuse the index to both aggregate and then subsequently index the
-RDD. For example:
+The `aggregateUsingIndex` operator is useful for efficient construction of a new `VertexRDD` from an
+`RDD[(VertexID, A)]`. Conceptually, if I have constructed a `VertexRDD[B]` over a set of vertices,
+*which is a super-set* of the vertices in some `RDD[(VertexID, A)]` then I can reuse the index to
+both aggregate and then subsequently index the `RDD[(VertexID, A)]`. For example:
{% highlight scala %}
val setA: VertexRDD[Int] = VertexRDD(sc.parallelize(0L until 100L).map(id => (id, 1)))
-val rddB: RDD[(VertexID, Double)] = sc.parallelize(0L until 100L).flatMap(id => List((id, 1.0), (id, 2.0)))
+val rddB: RDD[(VertexId, Double)] = sc.parallelize(0L until 100L).flatMap(id => List((id, 1.0), (id, 2.0)))
// There should be 200 entries in rddB
rddB.count
val setB: VertexRDD[Double] = setA.aggregateUsingIndex(rddB, _ + _)
@@ -813,10 +908,10 @@ val setC: VertexRDD[Double] = setA.innerJoin(setB)((id, a, b) => a + b)
## EdgeRDDs
-The `EdgeRDD[ED]`, which extends `RDD[Edge[ED]]` is considerably simpler than the `VertexRDD`.
-GraphX organizes the edges in blocks partitioned using one of the various partitioning strategies
-defined in [`PartitionStrategy`][PartitionStrategy]. Within each partition, edge attributes and
-adjacency structure, are stored separately enabling maximum reuse when changing attribute values.
+The `EdgeRDD[ED]`, which extends `RDD[Edge[ED]]` organizes the edges in blocks partitioned using one
+of the various partitioning strategies defined in [`PartitionStrategy`][PartitionStrategy]. Within
+each partition, edge attributes and adjacency structure, are stored separately enabling maximum
+reuse when changing attribute values.
[PartitionStrategy]: api/graphx/index.html#org.apache.spark.graphx.PartitionStrategy
@@ -827,11 +922,11 @@ def mapValues[ED2](f: Edge[ED] => ED2): EdgeRDD[ED2]
// Revere the edges reusing both attributes and structure
def reverse: EdgeRDD[ED]
// Join two `EdgeRDD`s partitioned using the same partitioning strategy.
-def innerJoin[ED2, ED3](other: EdgeRDD[ED2])(f: (VertexID, VertexID, ED, ED2) => ED3): EdgeRDD[ED3]
+def innerJoin[ED2, ED3](other: EdgeRDD[ED2])(f: (VertexId, VertexId, ED, ED2) => ED3): EdgeRDD[ED3]
{% endhighlight %}
In most applications we have found that operations on the `EdgeRDD` are accomplished through the
-graph or rely on operations defined in the base `RDD` class.
+graph operators or rely on operations defined in the base `RDD` class.
# Optimized Representation
@@ -853,7 +948,9 @@ reduce both the communication and storage overhead. Logically, this corresponds
to machines and allowing vertices to span multiple machines. The exact method of assigning edges
depends on the [`PartitionStrategy`][PartitionStrategy] and there are several tradeoffs to the
various heuristics. Users can choose between different strategies by repartitioning the graph with
-the [`Graph.partitionBy`][Graph.partitionBy] operator.
+the [`Graph.partitionBy`][Graph.partitionBy] operator. The default partitioning strategy is to use
+the initial partitioning of the edges as provided on graph construction. However, users can easily
+switch to 2D-partitioning or other heuristics included in GraphX.
[Graph.partitionBy]: api/graphx/index.html#org.apache.spark.graphx.Graph$@partitionBy(partitionStrategy:org.apache.spark.graphx.PartitionStrategy):org.apache.spark.graphx.Graph[VD,ED]
@@ -867,16 +964,15 @@ the [`Graph.partitionBy`][Graph.partitionBy] operator.
Once the edges have be partitioned the key challenge to efficient graph-parallel computation is
efficiently joining vertex attributes with the edges. Because real-world graphs typically have more
-edges than vertices, we move vertex attributes to the edges.
-
-
-
-
+edges than vertices, we move vertex attributes to the edges. Because not all partitions will
+contain edges adjacent to all vertices we internally maintain a routing table which identifies where
+to broadcast vertices when implementing the join required for operations like `triplets` and
+`mapReduceTriplets`.
# Graph Algorithms
<a name="graph_algorithms"></a>
-GraphX includes a set of graph algorithms in to simplify analytics. The algorithms are contained in the `org.apache.spark.graphx.lib` package and can be accessed directly as methods on `Graph` via [`GraphOps`][GraphOps]. This section describes the algorithms and how they are used.
+GraphX includes a set of graph algorithms to simplify analytics tasks. The algorithms are contained in the `org.apache.spark.graphx.lib` package and can be accessed directly as methods on `Graph` via [`GraphOps`][GraphOps]. This section describes the algorithms and how they are used.
## PageRank
<a name="pagerank"></a>
@@ -953,13 +1049,6 @@ val triCountByUsername = users.join(triCounts).map { case (id, (username, tc)) =
println(triCountByUsername.collect().mkString("\n"))
{% endhighlight %}
-<p style="text-align: center;">
- <img src="img/tables_and_graphs.png"
- title="Tables and Graphs"
- alt="Tables and Graphs"
- width="50%" />
- <!-- Images are downsized intentionally to improve quality on retina displays -->
-</p>
# Examples
diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md
index 5be8ce1ebe..0cc5505b50 100644
--- a/docs/mllib-guide.md
+++ b/docs/mllib-guide.md
@@ -21,7 +21,8 @@ depends on native Fortran routines. You may need to install the
if it is not already present on your nodes. MLlib will throw a linking error if it cannot
detect these libraries automatically.
-To use MLlib in Python, you will also need [NumPy](http://www.numpy.org) version 1.7 or newer.
+To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.7 or newer
+and Python 2.7.
# Binary Classification
diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md
index b07899c2e1..7c5283fb0b 100644
--- a/docs/python-programming-guide.md
+++ b/docs/python-programming-guide.md
@@ -52,7 +52,7 @@ In addition, PySpark fully supports interactive use---simply run `./bin/pyspark`
# Installing and Configuring PySpark
-PySpark requires Python 2.7 or higher.
+PySpark requires Python 2.6 or higher.
PySpark applications are executed using a standard CPython interpreter in order to support Python modules that use C extensions.
We have not tested PySpark with Python 3 or with alternative Python interpreters, such as [PyPy](http://pypy.org/) or [Jython](http://www.jython.org/).
@@ -152,7 +152,7 @@ Many of the methods also contain [doctests](http://docs.python.org/2/library/doc
# Libraries
[MLlib](mllib-guide.html) is also available in PySpark. To use it, you'll need
-[NumPy](http://www.numpy.org) version 1.7 or newer. The [MLlib guide](mllib-guide.html) contains
+[NumPy](http://www.numpy.org) version 1.7 or newer, and Python 2.7. The [MLlib guide](mllib-guide.html) contains
some example applications.
# Where to Go from Here
diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md
index 4e8a680a75..07c4c55633 100644
--- a/docs/streaming-programming-guide.md
+++ b/docs/streaming-programming-guide.md
@@ -21,6 +21,8 @@ Add the following SBT or Maven dependency to your project to use Spark Streaming
artifactId = spark-streaming_{{site.SCALA_VERSION}}
version = {{site.SPARK_VERSION}}
+For ingesting data from sources like Kafka and Flume, add the corresponding artifact `spark-streaming-xyz_{{site.SCALA_VERSION}}` to the dependencies. For example, `spark-streaming-kafka_{{site.SCALA_VERSION}}` for Kafka, `spark-streaming-flume_{{site.SCALA_VERSION}}`, etc. Please refer to the [Apache repository](http://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.apache.spark%22%20AND%20v%3A%22{{site.SPARK_VERSION}}%22) for the full list of supported sources / artifacts.
+
# Initializing Spark Streaming
The first thing a Spark Streaming program must do is create a `StreamingContext` object, which tells Spark how to access a cluster. A `StreamingContext` can be created by using
@@ -28,26 +30,28 @@ The first thing a Spark Streaming program must do is create a `StreamingContext`
new StreamingContext(master, appName, batchDuration, [sparkHome], [jars])
{% endhighlight %}
-The `master` parameter is a standard [Spark cluster URL](scala-programming-guide.html#master-urls) and can be "local" for local testing. The `appName` is a name of your program, which will be shown on your cluster's web UI. The `batchDuration` is the size of the batches (as explained earlier). This must be set carefully such that the cluster can keep up with the processing of the data streams. Start with something conservative like 5 seconds. See the [Performance Tuning](#setting-the-right-batch-size) section for a detailed discussion. Finally, `sparkHome` and `jars` are necessary when running on a cluster to specify the location of your code, as described in the [Spark programming guide](scala-programming-guide.html#deploying-code-on-a-cluster).
+The `master` parameter is a standard [Spark cluster URL](scala-programming-guide.html#master-urls) and can be "local" for local testing. The `appName` is a name of your program, which will be shown on your cluster's web UI. The `batchDuration` is the size of the batches (as explained earlier). This must be set carefully such that the cluster can keep up with the processing of the data streams. Start with something conservative like 5 seconds. See the [Performance Tuning](#setting-the-right-batch-size) section for a detailed discussion. Finally, `sparkHome` and `jars` are optional parameters, which need to be set when running on a cluster to specify the location of your code, as described in the [Spark programming guide](scala-programming-guide.html#deploying-code-on-a-cluster).
-This constructor creates a SparkContext for your job as well, which can be accessed with `streamingContext.sparkContext`.
+{% highlight scala %}
+new SparkConf(conf, batchDuration)
+{% endhighlight %}
+where `conf` is a [SparkConf](api/core/index.html#org.apache.spark.SparkConf)
+object used for more advanced configuration. In both cases, a [SparkContext](api/core/index.html#org.apache.spark.SparkContext) is created as well which can be accessed with `streamingContext.sparkContext`.
-# Attaching Input Sources - InputDStreams
-The StreamingContext is used to creating InputDStreams from input sources:
+# Attaching Input Sources
+The StreamingContext is used to creating input streams from data sources:
{% highlight scala %}
// Assuming ssc is the StreamingContext
-ssc.textFileStream(directory) // Creates a stream by monitoring and processing new files in a HDFS directory
-ssc.socketStream(hostname, port) // Creates a stream that uses a TCP socket to read data from hostname:port
+ssc.textFileStream(directory) // Creates a stream that monitors and processes new files in a HDFS directory
+ssc.socketStream(hostname, port) // Creates a stream that uses a TCP socket to read data from hostname:port
{% endhighlight %}
-We also provide a input streams for Kafka, Flume, Akka actor, etc. For a complete list of input streams, take a look at the [StreamingContext API documentation](api/streaming/index.html#org.apache.spark.streaming.StreamingContext).
-
-
+The core Spark Streaming API provides input streams for files, sockets, and Akka actors. Additional functionality for Kafka, Flume, ZeroMQ, Twitter, etc. can be imported by adding the right dependencies as explained in the [linking](#linking-with-spark-streaming) section.
# DStream Operations
-Data received from the input streams can be processed using _DStream operations_. There are two kinds of operations - _transformations_ and _output operations_. Similar to RDD transformations, DStream transformations operate on one or more DStreams to create new DStreams with transformed data. After applying a sequence of transformations to the input streams, you'll need to call the output operations, which writies data out to an external source.
+Data received from the input streams can be processed using _DStream operations_. There are two kinds of operations - _transformations_ and _output operations_. Similar to RDD transformations, DStream transformations operate on one or more DStreams to create new DStreams with transformed data. After applying a sequence of transformations to the input streams, output operations need to called, which write data out to an external data sink like a file system or a database.
## Transformations
@@ -234,7 +238,7 @@ wordCounts.print()
ssc.start()
{% endhighlight %}
-The `socketTextStream` returns a DStream of lines received from a TCP socket-based source. The `lines` DStream is _transformed_ into a DStream using the `flatMap` operation, where each line is split into words. This `words` DStream is then mapped to a DStream of `(word, 1)` pairs, which is finally reduced to get the word counts. `wordCounts.print()` will print 10 of the counts generated every second.
+The `socketTextStream` returns a DStream of text data received from a TCP server socket. The `lines` DStream is _transformed_ into a DStream using the `flatMap` operation, where each line is split into words. This `words` DStream is then mapped to a DStream of `(word, 1)` pairs, which is finally reduced to get the word counts. `wordCounts.print()` will print 10 of the counts generated every second.
To run this example on your local machine, you need to first run a Netcat server by using
@@ -270,14 +274,12 @@ hello world
{% highlight bash %}
# TERMINAL 2: RUNNING NetworkWordCount
...
-2012-12-31 18:47:10,446 INFO SparkContext: Job finished: run at ThreadPoolExecutor.java:886, took 0.038817 s
-------------------------------------------
Time: 1357008430000 ms
-------------------------------------------
(hello,1)
(world,1)
-2012-12-31 18:47:10,447 INFO JobManager: Total delay: 0.44700 s for job 8 (execution: 0.44000 s)
...
{% endhighlight %}
</td>
@@ -384,7 +386,7 @@ A system that is required to operate 24/7 needs to be able tolerate the failure
1. The configuration of each DStream (checkpoint interval, etc.)
1. The RDD checkpoint files of each DStream
-All this is periodically saved in the file `<checkpoint directory>/graph`. To recover, a new Streaming Context can be created with this directory by using
+All this is periodically saved in the checkpoint directory. To recover, a new `StreamingContext` can be created with this directory by using
{% highlight scala %}
val ssc = new StreamingContext(checkpointDirectory)
@@ -395,7 +397,7 @@ On calling `ssc.start()` on this new context, the following steps are taken by t
1. Schedule the transformations and output operations for all the time steps between the time when the driver failed and when it last checkpointed. This is also done for those time steps that were previously scheduled but not processed due to the failure. This will make the system recompute all the intermediate data from the checkpointed RDD files, etc.
1. Restart the network receivers, if any, and continue receiving new data.
-In the current _alpha_ release, there are two different failure behaviors based on which input sources are used.
+There are two different failure behaviors based on which input sources are used.
1. _Using HDFS files as input source_ - Since the data is reliably stored on HDFS, all data can re-computed and therefore no data will be lost due to any failure.
1. _Using any input source that receives data through a network_ - The received input data is replicated in memory to multiple nodes. Since, all the data in the Spark worker's memory is lost when the Spark driver fails, the past input data will not be accessible and driver recovers. Hence, if stateful and window-based operations are used (like `updateStateByKey`, `window`, `countByValueAndWindow`, etc.), then the intermediate state will not be recovered completely.
diff --git a/docs/tuning.md b/docs/tuning.md
index bbb870085c..6b010aed61 100644
--- a/docs/tuning.md
+++ b/docs/tuning.md
@@ -119,8 +119,7 @@ pointer-based data structures and wrapper objects. There are several ways to do
2. Avoid nested structures with a lot of small objects and pointers when possible.
3. Consider using numeric IDs or enumeration objects instead of strings for keys.
4. If you have less than 32 GB of RAM, set the JVM flag `-XX:+UseCompressedOops` to make pointers be
- four bytes instead of eight. Also, on Java 7 or later, try `-XX:+UseCompressedStrings` to store
- ASCII strings as just 8 bits per character. You can add these options in
+ four bytes instead of eight. You can add these options in
[`spark-env.sh`](configuration.html#environment-variables-in-spark-envsh).
## Serialized RDD Storage
diff --git a/examples/pom.xml b/examples/pom.xml
index cb4f7ee33b..7855706389 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -73,6 +73,12 @@
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
+ <artifactId>spark-graphx_${scala.binary.version}</artifactId>
+ <version>${project.version}</version>
+ <scope>provided</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.spark</groupId>
<artifactId>spark-streaming-twitter_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala
index 25f7013307..0226475712 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala
@@ -19,6 +19,7 @@ package org.apache.spark.streaming.examples
import org.apache.spark.streaming.{Seconds, StreamingContext}
import org.apache.spark.streaming.StreamingContext._
+import org.apache.spark.storage.StorageLevel
/**
* Counts words in text encoded with UTF8 received from the network every second.
@@ -48,7 +49,7 @@ object NetworkWordCount {
// Create a NetworkInputDStream on target ip:port and count the
// words in input stream of \n delimited text (eg. generated by 'nc')
- val lines = ssc.socketTextStream(args(1), args(2).toInt)
+ val lines = ssc.socketTextStream(args(1), args(2).toInt, StorageLevel.MEMORY_ONLY_SER)
val words = lines.flatMap(_.split(" "))
val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
wordCounts.print()
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/StreamingExamples.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/StreamingExamples.scala
index d41d84a980..99f1502046 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/StreamingExamples.scala
+++ b/examples/src/main/scala/org/apache/spark/streaming/examples/StreamingExamples.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.streaming.examples
import org.apache.spark.Logging
diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala
index d53b66dd46..654ba451e7 100644
--- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala
+++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala
@@ -37,7 +37,6 @@ object FlumeUtils {
storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
): DStream[SparkFlumeEvent] = {
val inputStream = new FlumeInputDStream[SparkFlumeEvent](ssc, hostname, port, storageLevel)
- ssc.registerInputStream(inputStream)
inputStream
}
diff --git a/external/flume/src/test/resources/log4j.properties b/external/flume/src/test/resources/log4j.properties
index 063529a9cb..d1bd73a843 100644
--- a/external/flume/src/test/resources/log4j.properties
+++ b/external/flume/src/test/resources/log4j.properties
@@ -20,7 +20,7 @@ log4j.rootCategory=INFO, file
# log4j.appender.file=org.apache.log4j.FileAppender
log4j.appender.file=org.apache.log4j.FileAppender
log4j.appender.file.append=false
-log4j.appender.file.file=streaming/target/unit-tests.log
+log4j.appender.file.file=external/flume/target/unit-tests.log
log4j.appender.file.layout=org.apache.log4j.PatternLayout
log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n
diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala
index 2e8e9fac45..8bc43972ab 100644
--- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala
+++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala
@@ -43,7 +43,7 @@ class FlumeStreamSuite extends TestSuiteBase {
val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]]
with SynchronizedBuffer[Seq[SparkFlumeEvent]]
val outputStream = new TestOutputStream(flumeStream, outputBuffer)
- ssc.registerOutputStream(outputStream)
+ outputStream.register()
ssc.start()
val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala
index 37c03be4e7..15a2daa008 100644
--- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala
+++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala
@@ -71,9 +71,7 @@ object KafkaUtils {
topics: Map[String, Int],
storageLevel: StorageLevel
): DStream[(K, V)] = {
- val inputStream = new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, storageLevel)
- ssc.registerInputStream(inputStream)
- inputStream
+ new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, storageLevel)
}
/**
diff --git a/external/kafka/src/test/resources/log4j.properties b/external/kafka/src/test/resources/log4j.properties
index 063529a9cb..38910d1130 100644
--- a/external/kafka/src/test/resources/log4j.properties
+++ b/external/kafka/src/test/resources/log4j.properties
@@ -20,7 +20,7 @@ log4j.rootCategory=INFO, file
# log4j.appender.file=org.apache.log4j.FileAppender
log4j.appender.file=org.apache.log4j.FileAppender
log4j.appender.file.append=false
-log4j.appender.file.file=streaming/target/unit-tests.log
+log4j.appender.file.file=external/kafka/target/unit-tests.log
log4j.appender.file.layout=org.apache.log4j.PatternLayout
log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n
diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala
index 9c81f23c19..d9809f6409 100644
--- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala
+++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala
@@ -35,5 +35,6 @@ class KafkaStreamSuite extends TestSuiteBase {
ssc, kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2)
// TODO: Actually test receiving data
+ ssc.stop()
}
}
diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala
index 3636e46bb8..1b09ee5dc8 100644
--- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala
+++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala
@@ -37,9 +37,7 @@ object MQTTUtils {
topic: String,
storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
): DStream[String] = {
- val inputStream = new MQTTInputDStream[String](ssc, brokerUrl, topic, storageLevel)
- ssc.registerInputStream(inputStream)
- inputStream
+ new MQTTInputDStream[String](ssc, brokerUrl, topic, storageLevel)
}
/**
diff --git a/external/mqtt/src/test/resources/log4j.properties b/external/mqtt/src/test/resources/log4j.properties
index 063529a9cb..d0462c7336 100644
--- a/external/mqtt/src/test/resources/log4j.properties
+++ b/external/mqtt/src/test/resources/log4j.properties
@@ -20,7 +20,7 @@ log4j.rootCategory=INFO, file
# log4j.appender.file=org.apache.log4j.FileAppender
log4j.appender.file=org.apache.log4j.FileAppender
log4j.appender.file.append=false
-log4j.appender.file.file=streaming/target/unit-tests.log
+log4j.appender.file.file=external/mqtt/target/unit-tests.log
log4j.appender.file.layout=org.apache.log4j.PatternLayout
log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n
diff --git a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala
index 73e7ce6e96..89c40ad461 100644
--- a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala
+++ b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala
@@ -32,5 +32,6 @@ class MQTTStreamSuite extends TestSuiteBase {
val test2 = MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_AND_DISK_SER_2)
// TODO: Actually test receiving data
+ ssc.stop()
}
}
diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala
index b8bae7b6d3..e8433b7e9f 100644
--- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala
+++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala
@@ -41,9 +41,7 @@ object TwitterUtils {
filters: Seq[String] = Nil,
storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
): DStream[Status] = {
- val inputStream = new TwitterInputDStream(ssc, twitterAuth, filters, storageLevel)
- ssc.registerInputStream(inputStream)
- inputStream
+ new TwitterInputDStream(ssc, twitterAuth, filters, storageLevel)
}
/**
diff --git a/external/twitter/src/test/resources/log4j.properties b/external/twitter/src/test/resources/log4j.properties
index 063529a9cb..c918335fcd 100644
--- a/external/twitter/src/test/resources/log4j.properties
+++ b/external/twitter/src/test/resources/log4j.properties
@@ -20,7 +20,7 @@ log4j.rootCategory=INFO, file
# log4j.appender.file=org.apache.log4j.FileAppender
log4j.appender.file=org.apache.log4j.FileAppender
log4j.appender.file.append=false
-log4j.appender.file.file=streaming/target/unit-tests.log
+log4j.appender.file.file=external/twitter/target/unit-tests.log
log4j.appender.file.layout=org.apache.log4j.PatternLayout
log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n
diff --git a/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala b/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala
index ccc38784ef..06ab0cdaf3 100644
--- a/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala
+++ b/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala
@@ -39,5 +39,6 @@ class TwitterStreamSuite extends TestSuiteBase {
// Note that actually testing the data receiving is hard as authentication keys are
// necessary for accessing Twitter live stream
+ ssc.stop()
}
}
diff --git a/external/zeromq/src/test/resources/log4j.properties b/external/zeromq/src/test/resources/log4j.properties
index 063529a9cb..304683dd0b 100644
--- a/external/zeromq/src/test/resources/log4j.properties
+++ b/external/zeromq/src/test/resources/log4j.properties
@@ -20,7 +20,7 @@ log4j.rootCategory=INFO, file
# log4j.appender.file=org.apache.log4j.FileAppender
log4j.appender.file=org.apache.log4j.FileAppender
log4j.appender.file.append=false
-log4j.appender.file.file=streaming/target/unit-tests.log
+log4j.appender.file.file=external/zeromq/target/unit-tests.log
log4j.appender.file.layout=org.apache.log4j.PatternLayout
log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n
diff --git a/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala b/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala
index 4193b8a02f..92d55a7a7b 100644
--- a/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala
+++ b/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala
@@ -40,5 +40,6 @@ class ZeroMQStreamSuite extends TestSuiteBase {
StorageLevel.MEMORY_AND_DISK_SER_2, SupervisorStrategy.defaultStrategy)
// TODO: Actually test data receiving
+ ssc.stop()
}
}
diff --git a/graphx/pom.xml b/graphx/pom.xml
index 3e5faf230d..4eca4747ea 100644
--- a/graphx/pom.xml
+++ b/graphx/pom.xml
@@ -36,7 +36,6 @@
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_${scala.binary.version}</artifactId>
<version>${project.version}</version>
- <scope>provided</scope>
</dependency>
<dependency>
<groupId>org.eclipse.jetty</groupId>
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala
index 738a38b27f..580faa0866 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx
/**
@@ -11,8 +28,8 @@ package org.apache.spark.graphx
* @param attr The attribute associated with the edge
*/
case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED] (
- var srcId: VertexID = 0,
- var dstId: VertexID = 0,
+ var srcId: VertexId = 0,
+ var dstId: VertexId = 0,
var attr: ED = null.asInstanceOf[ED])
extends Serializable {
@@ -22,7 +39,7 @@ case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED]
* @param vid the id one of the two vertices on the edge.
* @return the id of the other vertex on the edge.
*/
- def otherVertexId(vid: VertexID): VertexID =
+ def otherVertexId(vid: VertexId): VertexId =
if (srcId == vid) dstId else { assert(dstId == vid); srcId }
/**
@@ -33,7 +50,7 @@ case class Edge[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED]
* @return the relative direction of the edge to the corresponding
* vertex.
*/
- def relativeDirection(vid: VertexID): EdgeDirection =
+ def relativeDirection(vid: VertexId): EdgeDirection =
if (vid == srcId) EdgeDirection.Out else { assert(vid == dstId); EdgeDirection.In }
}
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala
index f265764006..6f03eb1439 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeDirection.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx
/**
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala
index 832b7816fe..fe03ae4a62 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx
import scala.reflect.{classTag, ClassTag}
@@ -85,7 +102,7 @@ class EdgeRDD[@specialized ED: ClassTag](
*/
def innerJoin[ED2: ClassTag, ED3: ClassTag]
(other: EdgeRDD[ED2])
- (f: (VertexID, VertexID, ED, ED2) => ED3): EdgeRDD[ED3] = {
+ (f: (VertexId, VertexId, ED, ED2) => ED3): EdgeRDD[ED3] = {
val ed2Tag = classTag[ED2]
val ed3Tag = classTag[ED3]
new EdgeRDD[ED3](partitionsRDD.zipPartitions(other.partitionsRDD, true) {
@@ -96,7 +113,7 @@ class EdgeRDD[@specialized ED: ClassTag](
})
}
- private[graphx] def collectVertexIDs(): RDD[VertexID] = {
+ private[graphx] def collectVertexIds(): RDD[VertexId] = {
partitionsRDD.flatMap { case (_, p) => Array.concat(p.srcIds, p.dstIds) }
}
}
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala
index 4253b24b5a..fea43c3b2b 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx
/**
@@ -33,7 +50,7 @@ class EdgeTriplet[VD, ED] extends Edge[ED] {
* @param vid the id one of the two vertices on the edge
* @return the attribute for the other vertex on the edge
*/
- def otherVertexAttr(vid: VertexID): VD =
+ def otherVertexAttr(vid: VertexId): VD =
if (srcId == vid) dstAttr else { assert(dstId == vid); srcAttr }
/**
@@ -42,7 +59,7 @@ class EdgeTriplet[VD, ED] extends Edge[ED] {
* @param vid the id of one of the two vertices on the edge
* @return the attr for the vertex with that id
*/
- def vertexAttr(vid: VertexID): VD =
+ def vertexAttr(vid: VertexId): VD =
if (srcId == vid) srcAttr else { assert(dstId == vid); dstAttr }
override def toString = ((srcId, srcAttr), (dstId, dstAttr), attr).toString()
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
index 9dd05ade0a..eea95d38d5 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx
import scala.reflect.ClassTag
@@ -109,7 +126,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab
* }}}
*
*/
- def mapVertices[VD2: ClassTag](map: (VertexID, VD) => VD2): Graph[VD2, ED]
+ def mapVertices[VD2: ClassTag](map: (VertexId, VD) => VD2): Graph[VD2, ED]
/**
* Transforms each edge attribute in the graph using the map function. The map function is not
@@ -225,7 +242,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab
*/
def subgraph(
epred: EdgeTriplet[VD,ED] => Boolean = (x => true),
- vpred: (VertexID, VD) => Boolean = ((v, d) => true))
+ vpred: (VertexId, VD) => Boolean = ((v, d) => true))
: Graph[VD, ED]
/**
@@ -275,7 +292,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab
* vertex
* {{{
* val rawGraph: Graph[(),()] = Graph.textFile("twittergraph")
- * val inDeg: RDD[(VertexID, Int)] =
+ * val inDeg: RDD[(VertexId, Int)] =
* mapReduceTriplets[Int](et => Iterator((et.dst.id, 1)), _ + _)
* }}}
*
@@ -287,7 +304,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab
*
*/
def mapReduceTriplets[A: ClassTag](
- mapFunc: EdgeTriplet[VD, ED] => Iterator[(VertexID, A)],
+ mapFunc: EdgeTriplet[VD, ED] => Iterator[(VertexId, A)],
reduceFunc: (A, A) => A,
activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None)
: VertexRDD[A]
@@ -311,14 +328,14 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab
*
* {{{
* val rawGraph: Graph[_, _] = Graph.textFile("webgraph")
- * val outDeg: RDD[(VertexID, Int)] = rawGraph.outDegrees()
+ * val outDeg: RDD[(VertexId, Int)] = rawGraph.outDegrees()
* val graph = rawGraph.outerJoinVertices(outDeg) {
* (vid, data, optDeg) => optDeg.getOrElse(0)
* }
* }}}
*/
- def outerJoinVertices[U: ClassTag, VD2: ClassTag](other: RDD[(VertexID, U)])
- (mapFunc: (VertexID, VD, Option[U]) => VD2)
+ def outerJoinVertices[U: ClassTag, VD2: ClassTag](other: RDD[(VertexId, U)])
+ (mapFunc: (VertexId, VD, Option[U]) => VD2)
: Graph[VD2, ED]
/**
@@ -347,7 +364,7 @@ object Graph {
* (if `uniqueEdges` is `None`) and vertex attributes containing the total degree of each vertex.
*/
def fromEdgeTuples[VD: ClassTag](
- rawEdges: RDD[(VertexID, VertexID)],
+ rawEdges: RDD[(VertexId, VertexId)],
defaultValue: VD,
uniqueEdges: Option[PartitionStrategy] = None): Graph[VD, Int] =
{
@@ -388,7 +405,7 @@ object Graph {
* mentioned in edges but not in vertices
*/
def apply[VD: ClassTag, ED: ClassTag](
- vertices: RDD[(VertexID, VD)],
+ vertices: RDD[(VertexId, VD)],
edges: RDD[Edge[ED]],
defaultVertexAttr: VD = null.asInstanceOf[VD]): Graph[VD, ED] = {
GraphImpl(vertices, edges, defaultVertexAttr)
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala
index d79bdf9618..dd380d8c18 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx
import com.esotericsoftware.kryo.Kryo
@@ -16,7 +33,7 @@ class GraphKryoRegistrator extends KryoRegistrator {
kryo.register(classOf[Edge[Object]])
kryo.register(classOf[MessageToPartition[Object]])
kryo.register(classOf[VertexBroadcastMsg[Object]])
- kryo.register(classOf[(VertexID, Object)])
+ kryo.register(classOf[(VertexId, Object)])
kryo.register(classOf[EdgePartition[Object]])
kryo.register(classOf[BitSet])
kryo.register(classOf[VertexIdToIndexMap])
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala
index 5904aa3a28..18858466db 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx
import org.apache.spark.{Logging, SparkContext}
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
index f10e63f059..0fc1e4df68 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx
import scala.reflect.ClassTag
@@ -63,19 +80,19 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali
*
* @return the set of neighboring ids for each vertex
*/
- def collectNeighborIds(edgeDirection: EdgeDirection): VertexRDD[Array[VertexID]] = {
+ def collectNeighborIds(edgeDirection: EdgeDirection): VertexRDD[Array[VertexId]] = {
val nbrs =
if (edgeDirection == EdgeDirection.Either) {
- graph.mapReduceTriplets[Array[VertexID]](
+ graph.mapReduceTriplets[Array[VertexId]](
mapFunc = et => Iterator((et.srcId, Array(et.dstId)), (et.dstId, Array(et.srcId))),
reduceFunc = _ ++ _
)
} else if (edgeDirection == EdgeDirection.Out) {
- graph.mapReduceTriplets[Array[VertexID]](
+ graph.mapReduceTriplets[Array[VertexId]](
mapFunc = et => Iterator((et.srcId, Array(et.dstId))),
reduceFunc = _ ++ _)
} else if (edgeDirection == EdgeDirection.In) {
- graph.mapReduceTriplets[Array[VertexID]](
+ graph.mapReduceTriplets[Array[VertexId]](
mapFunc = et => Iterator((et.dstId, Array(et.srcId))),
reduceFunc = _ ++ _)
} else {
@@ -83,7 +100,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali
"direction. (EdgeDirection.Both is not supported; use EdgeDirection.Either instead.)")
}
graph.vertices.leftZipJoin(nbrs) { (vid, vdata, nbrsOpt) =>
- nbrsOpt.getOrElse(Array.empty[VertexID])
+ nbrsOpt.getOrElse(Array.empty[VertexId])
}
} // end of collectNeighborIds
@@ -99,8 +116,8 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali
*
* @return the vertex set of neighboring vertex attributes for each vertex
*/
- def collectNeighbors(edgeDirection: EdgeDirection): VertexRDD[Array[(VertexID, VD)]] = {
- val nbrs = graph.mapReduceTriplets[Array[(VertexID,VD)]](
+ def collectNeighbors(edgeDirection: EdgeDirection): VertexRDD[Array[(VertexId, VD)]] = {
+ val nbrs = graph.mapReduceTriplets[Array[(VertexId,VD)]](
edge => {
val msgToSrc = (edge.srcId, Array((edge.dstId, edge.dstAttr)))
val msgToDst = (edge.dstId, Array((edge.srcId, edge.srcAttr)))
@@ -116,7 +133,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali
(a, b) => a ++ b)
graph.vertices.leftZipJoin(nbrs) { (vid, vdata, nbrsOpt) =>
- nbrsOpt.getOrElse(Array.empty[(VertexID, VD)])
+ nbrsOpt.getOrElse(Array.empty[(VertexId, VD)])
}
} // end of collectNeighbor
@@ -147,9 +164,9 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali
* }}}
*
*/
- def joinVertices[U: ClassTag](table: RDD[(VertexID, U)])(mapFunc: (VertexID, VD, U) => VD)
+ def joinVertices[U: ClassTag](table: RDD[(VertexId, U)])(mapFunc: (VertexId, VD, U) => VD)
: Graph[VD, ED] = {
- val uf = (id: VertexID, data: VD, o: Option[U]) => {
+ val uf = (id: VertexId, data: VD, o: Option[U]) => {
o match {
case Some(u) => mapFunc(id, data, u)
case None => data
@@ -180,7 +197,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali
* val degrees: VertexRDD[Int] = graph.outDegrees
* graph.outerJoinVertices(degrees) {(vid, data, deg) => deg.getOrElse(0)}
* },
- * vpred = (vid: VertexID, deg:Int) => deg > 0
+ * vpred = (vid: VertexId, deg:Int) => deg > 0
* )
* }}}
*
@@ -188,7 +205,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali
def filter[VD2: ClassTag, ED2: ClassTag](
preprocess: Graph[VD, ED] => Graph[VD2, ED2],
epred: (EdgeTriplet[VD2, ED2]) => Boolean = (x: EdgeTriplet[VD2, ED2]) => true,
- vpred: (VertexID, VD2) => Boolean = (v:VertexID, d:VD2) => true): Graph[VD, ED] = {
+ vpred: (VertexId, VD2) => Boolean = (v:VertexId, d:VD2) => true): Graph[VD, ED] = {
graph.mask(preprocess(graph).subgraph(epred, vpred))
}
@@ -243,8 +260,8 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali
initialMsg: A,
maxIterations: Int = Int.MaxValue,
activeDirection: EdgeDirection = EdgeDirection.Either)(
- vprog: (VertexID, VD, A) => VD,
- sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexID,A)],
+ vprog: (VertexId, VD, A) => VD,
+ sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexId,A)],
mergeMsg: (A, A) => A)
: Graph[VD, ED] = {
Pregel(graph, initialMsg, maxIterations, activeDirection)(vprog, sendMsg, mergeMsg)
@@ -276,7 +293,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali
*
* @see [[org.apache.spark.graphx.lib.ConnectedComponents$#run]]
*/
- def connectedComponents(): Graph[VertexID, ED] = {
+ def connectedComponents(): Graph[VertexId, ED] = {
ConnectedComponents.run(graph)
}
@@ -295,7 +312,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali
*
* @see [[org.apache.spark.graphx.lib.StronglyConnectedComponents$#run]]
*/
- def stronglyConnectedComponents(numIter: Int): Graph[VertexID, ED] = {
+ def stronglyConnectedComponents(numIter: Int): Graph[VertexId, ED] = {
StronglyConnectedComponents.run(graph, numIter)
}
} // end of GraphOps
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala
index 6d2990a3f6..929915362c 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx
/**
@@ -6,7 +23,7 @@ package org.apache.spark.graphx
*/
trait PartitionStrategy extends Serializable {
/** Returns the partition number for a given edge. */
- def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID
+ def getPartition(src: VertexId, dst: VertexId, numParts: PartitionID): PartitionID
}
/**
@@ -56,9 +73,9 @@ object PartitionStrategy {
* is used.
*/
case object EdgePartition2D extends PartitionStrategy {
- override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = {
+ override def getPartition(src: VertexId, dst: VertexId, numParts: PartitionID): PartitionID = {
val ceilSqrtNumParts: PartitionID = math.ceil(math.sqrt(numParts)).toInt
- val mixingPrime: VertexID = 1125899906842597L
+ val mixingPrime: VertexId = 1125899906842597L
val col: PartitionID = ((math.abs(src) * mixingPrime) % ceilSqrtNumParts).toInt
val row: PartitionID = ((math.abs(dst) * mixingPrime) % ceilSqrtNumParts).toInt
(col * ceilSqrtNumParts + row) % numParts
@@ -70,8 +87,8 @@ object PartitionStrategy {
* source.
*/
case object EdgePartition1D extends PartitionStrategy {
- override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = {
- val mixingPrime: VertexID = 1125899906842597L
+ override def getPartition(src: VertexId, dst: VertexId, numParts: PartitionID): PartitionID = {
+ val mixingPrime: VertexId = 1125899906842597L
(math.abs(src) * mixingPrime).toInt % numParts
}
}
@@ -82,7 +99,7 @@ object PartitionStrategy {
* random vertex cut that colocates all same-direction edges between two vertices.
*/
case object RandomVertexCut extends PartitionStrategy {
- override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = {
+ override def getPartition(src: VertexId, dst: VertexId, numParts: PartitionID): PartitionID = {
math.abs((src, dst).hashCode()) % numParts
}
}
@@ -94,7 +111,7 @@ object PartitionStrategy {
* regardless of direction.
*/
case object CanonicalRandomVertexCut extends PartitionStrategy {
- override def getPartition(src: VertexID, dst: VertexID, numParts: PartitionID): PartitionID = {
+ override def getPartition(src: VertexId, dst: VertexId, numParts: PartitionID): PartitionID = {
val lower = math.min(src, dst)
val higher = math.max(src, dst)
math.abs((lower, higher).hashCode()) % numParts
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala
index fc18f7e785..ac07a594a1 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx
import scala.reflect.ClassTag
@@ -23,9 +40,9 @@ import scala.reflect.ClassTag
* // Set the vertex attributes to the initial pagerank values
* .mapVertices((id, attr) => 1.0)
*
- * def vertexProgram(id: VertexID, attr: Double, msgSum: Double): Double =
+ * def vertexProgram(id: VertexId, attr: Double, msgSum: Double): Double =
* resetProb + (1.0 - resetProb) * msgSum
- * def sendMessage(id: VertexID, edge: EdgeTriplet[Double, Double]): Iterator[(VertexId, Double)] =
+ * def sendMessage(id: VertexId, edge: EdgeTriplet[Double, Double]): Iterator[(VertexId, Double)] =
* Iterator((edge.dstId, edge.srcAttr * edge.attr))
* def messageCombiner(a: Double, b: Double): Double = a + b
* val initialMessage = 0.0
@@ -96,8 +113,8 @@ object Pregel {
initialMsg: A,
maxIterations: Int = Int.MaxValue,
activeDirection: EdgeDirection = EdgeDirection.Either)
- (vprog: (VertexID, VD, A) => VD,
- sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexID, A)],
+ (vprog: (VertexId, VD, A) => VD,
+ sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexId, A)],
mergeMsg: (A, A) => A)
: Graph[VD, ED] =
{
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala
index 9a95364cb1..edd59bcf32 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala
@@ -28,7 +28,7 @@ import org.apache.spark.graphx.impl.MsgRDDFunctions
import org.apache.spark.graphx.impl.VertexPartition
/**
- * Extends `RDD[(VertexID, VD)]` by ensuring that there is only one entry for each vertex and by
+ * Extends `RDD[(VertexId, VD)]` by ensuring that there is only one entry for each vertex and by
* pre-indexing the entries for fast, efficient joins. Two VertexRDDs with the same index can be
* joined efficiently. All operations except [[reindex]] preserve the index. To construct a
* `VertexRDD`, use the [[org.apache.spark.graphx.VertexRDD$ VertexRDD object]].
@@ -36,12 +36,12 @@ import org.apache.spark.graphx.impl.VertexPartition
* @example Construct a `VertexRDD` from a plain RDD:
* {{{
* // Construct an initial vertex set
- * val someData: RDD[(VertexID, SomeType)] = loadData(someFile)
+ * val someData: RDD[(VertexId, SomeType)] = loadData(someFile)
* val vset = VertexRDD(someData)
* // If there were redundant values in someData we would use a reduceFunc
* val vset2 = VertexRDD(someData, reduceFunc)
* // Finally we can use the VertexRDD to index another dataset
- * val otherData: RDD[(VertexID, OtherType)] = loadData(otherFile)
+ * val otherData: RDD[(VertexId, OtherType)] = loadData(otherFile)
* val vset3 = vset2.innerJoin(otherData) { (vid, a, b) => b }
* // Now we can construct very fast joins between the two sets
* val vset4: VertexRDD[(SomeType, OtherType)] = vset.leftJoin(vset3)
@@ -51,7 +51,7 @@ import org.apache.spark.graphx.impl.VertexPartition
*/
class VertexRDD[@specialized VD: ClassTag](
val partitionsRDD: RDD[VertexPartition[VD]])
- extends RDD[(VertexID, VD)](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) {
+ extends RDD[(VertexId, VD)](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) {
require(partitionsRDD.partitioner.isDefined)
@@ -92,9 +92,9 @@ class VertexRDD[@specialized VD: ClassTag](
}
/**
- * Provides the `RDD[(VertexID, VD)]` equivalent output.
+ * Provides the `RDD[(VertexId, VD)]` equivalent output.
*/
- override def compute(part: Partition, context: TaskContext): Iterator[(VertexID, VD)] = {
+ override def compute(part: Partition, context: TaskContext): Iterator[(VertexId, VD)] = {
firstParent[VertexPartition[VD]].iterator(part, context).next.iterator
}
@@ -114,9 +114,9 @@ class VertexRDD[@specialized VD: ClassTag](
* rather than allocating new memory.
*
* @param pred the user defined predicate, which takes a tuple to conform to the
- * `RDD[(VertexID, VD)]` interface
+ * `RDD[(VertexId, VD)]` interface
*/
- override def filter(pred: Tuple2[VertexID, VD] => Boolean): VertexRDD[VD] =
+ override def filter(pred: Tuple2[VertexId, VD] => Boolean): VertexRDD[VD] =
this.mapVertexPartitions(_.filter(Function.untupled(pred)))
/**
@@ -140,7 +140,7 @@ class VertexRDD[@specialized VD: ClassTag](
* @return a new VertexRDD with values obtained by applying `f` to each of the entries in the
* original VertexRDD. The resulting VertexRDD retains the same index.
*/
- def mapValues[VD2: ClassTag](f: (VertexID, VD) => VD2): VertexRDD[VD2] =
+ def mapValues[VD2: ClassTag](f: (VertexId, VD) => VD2): VertexRDD[VD2] =
this.mapVertexPartitions(_.map(f))
/**
@@ -172,7 +172,7 @@ class VertexRDD[@specialized VD: ClassTag](
* @return a VertexRDD containing the results of `f`
*/
def leftZipJoin[VD2: ClassTag, VD3: ClassTag]
- (other: VertexRDD[VD2])(f: (VertexID, VD, Option[VD2]) => VD3): VertexRDD[VD3] = {
+ (other: VertexRDD[VD2])(f: (VertexId, VD, Option[VD2]) => VD3): VertexRDD[VD3] = {
val newPartitionsRDD = partitionsRDD.zipPartitions(
other.partitionsRDD, preservesPartitioning = true
) { (thisIter, otherIter) =>
@@ -200,8 +200,8 @@ class VertexRDD[@specialized VD: ClassTag](
* by `f`.
*/
def leftJoin[VD2: ClassTag, VD3: ClassTag]
- (other: RDD[(VertexID, VD2)])
- (f: (VertexID, VD, Option[VD2]) => VD3)
+ (other: RDD[(VertexId, VD2)])
+ (f: (VertexId, VD, Option[VD2]) => VD3)
: VertexRDD[VD3] = {
// Test if the other vertex is a VertexRDD to choose the optimal join strategy.
// If the other set is a VertexRDD then we use the much more efficient leftZipJoin
@@ -225,7 +225,7 @@ class VertexRDD[@specialized VD: ClassTag](
* [[innerJoin]] for the behavior of the join.
*/
def innerZipJoin[U: ClassTag, VD2: ClassTag](other: VertexRDD[U])
- (f: (VertexID, VD, U) => VD2): VertexRDD[VD2] = {
+ (f: (VertexId, VD, U) => VD2): VertexRDD[VD2] = {
val newPartitionsRDD = partitionsRDD.zipPartitions(
other.partitionsRDD, preservesPartitioning = true
) { (thisIter, otherIter) =>
@@ -247,8 +247,8 @@ class VertexRDD[@specialized VD: ClassTag](
* @return a VertexRDD co-indexed with `this`, containing only vertices that appear in both `this`
* and `other`, with values supplied by `f`
*/
- def innerJoin[U: ClassTag, VD2: ClassTag](other: RDD[(VertexID, U)])
- (f: (VertexID, VD, U) => VD2): VertexRDD[VD2] = {
+ def innerJoin[U: ClassTag, VD2: ClassTag](other: RDD[(VertexId, U)])
+ (f: (VertexId, VD, U) => VD2): VertexRDD[VD2] = {
// Test if the other vertex is a VertexRDD to choose the optimal join strategy.
// If the other set is a VertexRDD then we use the much more efficient innerZipJoin
other match {
@@ -278,7 +278,7 @@ class VertexRDD[@specialized VD: ClassTag](
* messages.
*/
def aggregateUsingIndex[VD2: ClassTag](
- messages: RDD[(VertexID, VD2)], reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2] = {
+ messages: RDD[(VertexId, VD2)], reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2] = {
val shuffled = MsgRDDFunctions.partitionForAggregation(messages, this.partitioner.get)
val parts = partitionsRDD.zipPartitions(shuffled, true) { (thisIter, msgIter) =>
val vertexPartition: VertexPartition[VD] = thisIter.next()
@@ -303,8 +303,8 @@ object VertexRDD {
*
* @param rdd the collection of vertex-attribute pairs
*/
- def apply[VD: ClassTag](rdd: RDD[(VertexID, VD)]): VertexRDD[VD] = {
- val partitioned: RDD[(VertexID, VD)] = rdd.partitioner match {
+ def apply[VD: ClassTag](rdd: RDD[(VertexId, VD)]): VertexRDD[VD] = {
+ val partitioned: RDD[(VertexId, VD)] = rdd.partitioner match {
case Some(p) => rdd
case None => rdd.partitionBy(new HashPartitioner(rdd.partitions.size))
}
@@ -323,8 +323,8 @@ object VertexRDD {
* @param rdd the collection of vertex-attribute pairs
* @param mergeFunc the associative, commutative merge function.
*/
- def apply[VD: ClassTag](rdd: RDD[(VertexID, VD)], mergeFunc: (VD, VD) => VD): VertexRDD[VD] = {
- val partitioned: RDD[(VertexID, VD)] = rdd.partitioner match {
+ def apply[VD: ClassTag](rdd: RDD[(VertexId, VD)], mergeFunc: (VD, VD) => VD): VertexRDD[VD] = {
+ val partitioned: RDD[(VertexId, VD)] = rdd.partitioner match {
case Some(p) => rdd
case None => rdd.partitionBy(new HashPartitioner(rdd.partitions.size))
}
@@ -338,7 +338,7 @@ object VertexRDD {
* Constructs a VertexRDD from the vertex IDs in `vids`, taking attributes from `rdd` and using
* `defaultVal` otherwise.
*/
- def apply[VD: ClassTag](vids: RDD[VertexID], rdd: RDD[(VertexID, VD)], defaultVal: VD)
+ def apply[VD: ClassTag](vids: RDD[VertexId], rdd: RDD[(VertexId, VD)], defaultVal: VD)
: VertexRDD[VD] = {
VertexRDD(vids.map(vid => (vid, defaultVal))).leftJoin(rdd) { (vid, default, value) =>
value.getOrElse(default)
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala
index ee95ead3ad..57fa5eefd5 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx.impl
import scala.reflect.ClassTag
@@ -17,10 +34,10 @@ import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap
*/
private[graphx]
class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag](
- val srcIds: Array[VertexID],
- val dstIds: Array[VertexID],
+ val srcIds: Array[VertexId],
+ val dstIds: Array[VertexId],
val data: Array[ED],
- val index: PrimitiveKeyOpenHashMap[VertexID, Int]) extends Serializable {
+ val index: PrimitiveKeyOpenHashMap[VertexId, Int]) extends Serializable {
/**
* Reverse all the edges in this partition.
@@ -101,8 +118,8 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double)
*/
def groupEdges(merge: (ED, ED) => ED): EdgePartition[ED] = {
val builder = new EdgePartitionBuilder[ED]
- var currSrcId: VertexID = null.asInstanceOf[VertexID]
- var currDstId: VertexID = null.asInstanceOf[VertexID]
+ var currSrcId: VertexId = null.asInstanceOf[VertexId]
+ var currDstId: VertexId = null.asInstanceOf[VertexId]
var currAttr: ED = null.asInstanceOf[ED]
var i = 0
while (i < size) {
@@ -136,7 +153,7 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double)
*/
def innerJoin[ED2: ClassTag, ED3: ClassTag]
(other: EdgePartition[ED2])
- (f: (VertexID, VertexID, ED, ED2) => ED3): EdgePartition[ED3] = {
+ (f: (VertexId, VertexId, ED, ED2) => ED3): EdgePartition[ED3] = {
val builder = new EdgePartitionBuilder[ED3]
var i = 0
var j = 0
@@ -193,14 +210,14 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double)
* iterator is generated using an index scan, so it is efficient at skipping edges that don't
* match srcIdPred.
*/
- def indexIterator(srcIdPred: VertexID => Boolean): Iterator[Edge[ED]] =
+ def indexIterator(srcIdPred: VertexId => Boolean): Iterator[Edge[ED]] =
index.iterator.filter(kv => srcIdPred(kv._1)).flatMap(Function.tupled(clusterIterator))
/**
* Get an iterator over the cluster of edges in this partition with source vertex id `srcId`. The
* cluster must start at position `index`.
*/
- private def clusterIterator(srcId: VertexID, index: Int) = new Iterator[Edge[ED]] {
+ private def clusterIterator(srcId: VertexId, index: Int) = new Iterator[Edge[ED]] {
private[this] val edge = new Edge[ED]
private[this] var pos = index
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala
index 9d072f9335..63ccccb056 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx.impl
import scala.reflect.ClassTag
@@ -12,22 +29,22 @@ class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag](size: I
var edges = new PrimitiveVector[Edge[ED]](size)
/** Add a new edge to the partition. */
- def add(src: VertexID, dst: VertexID, d: ED) {
+ def add(src: VertexId, dst: VertexId, d: ED) {
edges += Edge(src, dst, d)
}
def toEdgePartition: EdgePartition[ED] = {
val edgeArray = edges.trim().array
Sorting.quickSort(edgeArray)(Edge.lexicographicOrdering)
- val srcIds = new Array[VertexID](edgeArray.size)
- val dstIds = new Array[VertexID](edgeArray.size)
+ val srcIds = new Array[VertexId](edgeArray.size)
+ val dstIds = new Array[VertexId](edgeArray.size)
val data = new Array[ED](edgeArray.size)
- val index = new PrimitiveKeyOpenHashMap[VertexID, Int]
+ val index = new PrimitiveKeyOpenHashMap[VertexId, Int]
// Copy edges into columnar structures, tracking the beginnings of source vertex id clusters and
// adding them to the index
if (edgeArray.length > 0) {
index.update(srcIds(0), 0)
- var currSrcId: VertexID = srcIds(0)
+ var currSrcId: VertexId = srcIds(0)
var i = 0
while (i < edgeArray.size) {
srcIds(i) = edgeArray(i).srcId
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala
index bad840f1cd..886c250d7c 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx.impl
import scala.reflect.ClassTag
@@ -24,7 +41,7 @@ class EdgeTripletIterator[VD: ClassTag, ED: ClassTag](
// allocating too many temporary Java objects.
private val triplet = new EdgeTriplet[VD, ED]
- private val vmap = new PrimitiveKeyOpenHashMap[VertexID, VD](vidToIndex, vertexArray)
+ private val vmap = new PrimitiveKeyOpenHashMap[VertexId, VD](vidToIndex, vertexArray)
override def hasNext: Boolean = pos < edgePartition.size
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
index 56d1d9efea..1d029bf009 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx.impl
import scala.reflect.{classTag, ClassTag}
@@ -88,7 +105,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected (
new GraphImpl(vertices, newETable, routingTable, replicatedVertexView)
}
- override def mapVertices[VD2: ClassTag](f: (VertexID, VD) => VD2): Graph[VD2, ED] = {
+ override def mapVertices[VD2: ClassTag](f: (VertexId, VD) => VD2): Graph[VD2, ED] = {
if (classTag[VD] equals classTag[VD2]) {
// The map preserves type, so we can use incremental replication
val newVerts = vertices.mapVertexPartitions(_.map(f)).cache()
@@ -136,7 +153,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected (
override def subgraph(
epred: EdgeTriplet[VD, ED] => Boolean = x => true,
- vpred: (VertexID, VD) => Boolean = (a, b) => true): Graph[VD, ED] = {
+ vpred: (VertexId, VD) => Boolean = (a, b) => true): Graph[VD, ED] = {
// Filter the vertices, reusing the partitioner and the index from this graph
val newVerts = vertices.mapVertexPartitions(_.filter(vpred))
@@ -178,7 +195,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected (
//////////////////////////////////////////////////////////////////////////////////////////////////
override def mapReduceTriplets[A: ClassTag](
- mapFunc: EdgeTriplet[VD, ED] => Iterator[(VertexID, A)],
+ mapFunc: EdgeTriplet[VD, ED] => Iterator[(VertexId, A)],
reduceFunc: (A, A) => A,
activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None) = {
@@ -208,7 +225,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected (
val edgeIter = activeDirectionOpt match {
case Some(EdgeDirection.Both) =>
if (activeFraction < 0.8) {
- edgePartition.indexIterator(srcVertexID => vPart.isActive(srcVertexID))
+ edgePartition.indexIterator(srcVertexId => vPart.isActive(srcVertexId))
.filter(e => vPart.isActive(e.dstId))
} else {
edgePartition.iterator.filter(e => vPart.isActive(e.srcId) && vPart.isActive(e.dstId))
@@ -219,7 +236,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected (
edgePartition.iterator.filter(e => vPart.isActive(e.srcId) || vPart.isActive(e.dstId))
case Some(EdgeDirection.Out) =>
if (activeFraction < 0.8) {
- edgePartition.indexIterator(srcVertexID => vPart.isActive(srcVertexID))
+ edgePartition.indexIterator(srcVertexId => vPart.isActive(srcVertexId))
} else {
edgePartition.iterator.filter(e => vPart.isActive(e.srcId))
}
@@ -250,8 +267,8 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected (
} // end of mapReduceTriplets
override def outerJoinVertices[U: ClassTag, VD2: ClassTag]
- (other: RDD[(VertexID, U)])
- (updateF: (VertexID, VD, Option[U]) => VD2): Graph[VD2, ED] =
+ (other: RDD[(VertexId, U)])
+ (updateF: (VertexId, VD, Option[U]) => VD2): Graph[VD2, ED] =
{
if (classTag[VD] equals classTag[VD2]) {
// updateF preserves type, so we can use incremental replication
@@ -295,7 +312,7 @@ object GraphImpl {
}
def apply[VD: ClassTag, ED: ClassTag](
- vertices: RDD[(VertexID, VD)],
+ vertices: RDD[(VertexId, VD)],
edges: RDD[Edge[ED]],
defaultVertexAttr: VD): GraphImpl[VD, ED] =
{
@@ -304,7 +321,7 @@ object GraphImpl {
// Get the set of all vids
val partitioner = Partitioner.defaultPartitioner(vertices)
val vPartitioned = vertices.partitionBy(partitioner)
- val vidsFromEdges = collectVertexIDsFromEdges(edgeRDD, partitioner)
+ val vidsFromEdges = collectVertexIdsFromEdges(edgeRDD, partitioner)
val vids = vPartitioned.zipPartitions(vidsFromEdges) { (vertexIter, vidsFromEdgesIter) =>
vertexIter.map(_._1) ++ vidsFromEdgesIter.map(_._1)
}
@@ -338,7 +355,7 @@ object GraphImpl {
/**
* Create the edge RDD, which is much more efficient for Java heap storage than the normal edges
- * data structure (RDD[(VertexID, VertexID, ED)]).
+ * data structure (RDD[(VertexId, VertexId, ED)]).
*
* The edge RDD contains multiple partitions, and each partition contains only one RDD key-value
* pair: the key is the partition id, and the value is an EdgePartition object containing all the
@@ -361,19 +378,19 @@ object GraphImpl {
defaultVertexAttr: VD): GraphImpl[VD, ED] = {
edges.cache()
// Get the set of all vids
- val vids = collectVertexIDsFromEdges(edges, new HashPartitioner(edges.partitions.size))
+ val vids = collectVertexIdsFromEdges(edges, new HashPartitioner(edges.partitions.size))
// Create the VertexRDD.
val vertices = VertexRDD(vids.mapValues(x => defaultVertexAttr))
GraphImpl(vertices, edges)
}
/** Collects all vids mentioned in edges and partitions them by partitioner. */
- private def collectVertexIDsFromEdges(
+ private def collectVertexIdsFromEdges(
edges: EdgeRDD[_],
- partitioner: Partitioner): RDD[(VertexID, Int)] = {
+ partitioner: Partitioner): RDD[(VertexId, Int)] = {
// TODO: Consider doing map side distinct before shuffle.
- new ShuffledRDD[VertexID, Int, (VertexID, Int)](
- edges.collectVertexIDs.map(vid => (vid, 0)), partitioner)
- .setSerializer(classOf[VertexIDMsgSerializer].getName)
+ new ShuffledRDD[VertexId, Int, (VertexId, Int)](
+ edges.collectVertexIds.map(vid => (vid, 0)), partitioner)
+ .setSerializer(classOf[VertexIdMsgSerializer].getName)
}
} // end of object GraphImpl
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala
index 05508ff716..e9ee09c361 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala
@@ -1,18 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx.impl
import scala.reflect.{classTag, ClassTag}
import org.apache.spark.Partitioner
-import org.apache.spark.graphx.{PartitionID, VertexID}
+import org.apache.spark.graphx.{PartitionID, VertexId}
import org.apache.spark.rdd.{ShuffledRDD, RDD}
private[graphx]
class VertexBroadcastMsg[@specialized(Int, Long, Double, Boolean) T](
@transient var partition: PartitionID,
- var vid: VertexID,
+ var vid: VertexId,
var data: T)
- extends Product2[PartitionID, (VertexID, T)] with Serializable {
+ extends Product2[PartitionID, (VertexId, T)] with Serializable {
override def _1 = partition
@@ -44,7 +61,7 @@ class MessageToPartition[@specialized(Int, Long, Double, Char, Boolean/*, AnyRef
private[graphx]
class VertexBroadcastMsgRDDFunctions[T: ClassTag](self: RDD[VertexBroadcastMsg[T]]) {
def partitionBy(partitioner: Partitioner): RDD[VertexBroadcastMsg[T]] = {
- val rdd = new ShuffledRDD[PartitionID, (VertexID, T), VertexBroadcastMsg[T]](self, partitioner)
+ val rdd = new ShuffledRDD[PartitionID, (VertexId, T), VertexBroadcastMsg[T]](self, partitioner)
// Set a custom serializer if the data is of int or double type.
if (classTag[T] == ClassTag.Int) {
@@ -82,8 +99,8 @@ object MsgRDDFunctions {
new VertexBroadcastMsgRDDFunctions(rdd)
}
- def partitionForAggregation[T: ClassTag](msgs: RDD[(VertexID, T)], partitioner: Partitioner) = {
- val rdd = new ShuffledRDD[VertexID, T, (VertexID, T)](msgs, partitioner)
+ def partitionForAggregation[T: ClassTag](msgs: RDD[(VertexId, T)], partitioner: Partitioner) = {
+ val rdd = new ShuffledRDD[VertexId, T, (VertexId, T)](msgs, partitioner)
// Set a custom serializer if the data is of int or double type.
if (classTag[T] == ClassTag.Int) {
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala
index 4ebe0b0267..a8154b63ce 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ReplicatedVertexView.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx.impl
import scala.reflect.{classTag, ClassTag}
@@ -33,9 +50,9 @@ class ReplicatedVertexView[VD: ClassTag](
* vids from both the source and destination of edges. It must always include both source and
* destination vids because some operations, such as GraphImpl.mapReduceTriplets, rely on this.
*/
- private val localVertexIDMap: RDD[(Int, VertexIdToIndexMap)] = prevViewOpt match {
+ private val localVertexIdMap: RDD[(Int, VertexIdToIndexMap)] = prevViewOpt match {
case Some(prevView) =>
- prevView.localVertexIDMap
+ prevView.localVertexIdMap
case None =>
edges.partitionsRDD.mapPartitions(_.map {
case (pid, epart) =>
@@ -45,7 +62,7 @@ class ReplicatedVertexView[VD: ClassTag](
vidToIndex.add(e.dstId)
}
(pid, vidToIndex)
- }, preservesPartitioning = true).cache().setName("ReplicatedVertexView localVertexIDMap")
+ }, preservesPartitioning = true).cache().setName("ReplicatedVertexView localVertexIdMap")
}
private lazy val bothAttrs: RDD[(PartitionID, VertexPartition[VD])] = create(true, true)
@@ -58,7 +75,7 @@ class ReplicatedVertexView[VD: ClassTag](
srcAttrOnly.unpersist(blocking)
dstAttrOnly.unpersist(blocking)
noAttrs.unpersist(blocking)
- // Don't unpersist localVertexIDMap because a future ReplicatedVertexView may be using it
+ // Don't unpersist localVertexIdMap because a future ReplicatedVertexView may be using it
// without modification
this
}
@@ -116,8 +133,8 @@ class ReplicatedVertexView[VD: ClassTag](
case None =>
// Within each edge partition, place the shipped vertex attributes into the correct
- // locations specified in localVertexIDMap
- localVertexIDMap.zipPartitions(shippedVerts) { (mapIter, shippedVertsIter) =>
+ // locations specified in localVertexIdMap
+ localVertexIdMap.zipPartitions(shippedVerts) { (mapIter, shippedVertsIter) =>
val (pid, vidToIndex) = mapIter.next()
assert(!mapIter.hasNext)
// Populate the vertex array using the vidToIndex map
@@ -140,15 +157,15 @@ class ReplicatedVertexView[VD: ClassTag](
private object ReplicatedVertexView {
protected def buildBuffer[VD: ClassTag](
- pid2vidIter: Iterator[Array[Array[VertexID]]],
+ pid2vidIter: Iterator[Array[Array[VertexId]]],
vertexPartIter: Iterator[VertexPartition[VD]]) = {
- val pid2vid: Array[Array[VertexID]] = pid2vidIter.next()
+ val pid2vid: Array[Array[VertexId]] = pid2vidIter.next()
val vertexPart: VertexPartition[VD] = vertexPartIter.next()
Iterator.tabulate(pid2vid.size) { pid =>
val vidsCandidate = pid2vid(pid)
val size = vidsCandidate.length
- val vids = new PrimitiveVector[VertexID](pid2vid(pid).size)
+ val vids = new PrimitiveVector[VertexId](pid2vid(pid).size)
val attrs = new PrimitiveVector[VD](pid2vid(pid).size)
var i = 0
while (i < size) {
@@ -164,16 +181,16 @@ private object ReplicatedVertexView {
}
protected def buildActiveBuffer(
- pid2vidIter: Iterator[Array[Array[VertexID]]],
+ pid2vidIter: Iterator[Array[Array[VertexId]]],
activePartIter: Iterator[VertexPartition[_]])
- : Iterator[(Int, Array[VertexID])] = {
- val pid2vid: Array[Array[VertexID]] = pid2vidIter.next()
+ : Iterator[(Int, Array[VertexId])] = {
+ val pid2vid: Array[Array[VertexId]] = pid2vidIter.next()
val activePart: VertexPartition[_] = activePartIter.next()
Iterator.tabulate(pid2vid.size) { pid =>
val vidsCandidate = pid2vid(pid)
val size = vidsCandidate.length
- val actives = new PrimitiveVector[VertexID](vidsCandidate.size)
+ val actives = new PrimitiveVector[VertexId](vidsCandidate.size)
var i = 0
while (i < size) {
val vid = vidsCandidate(i)
@@ -188,8 +205,8 @@ private object ReplicatedVertexView {
}
private[graphx]
-class VertexAttributeBlock[VD: ClassTag](val vids: Array[VertexID], val attrs: Array[VD])
+class VertexAttributeBlock[VD: ClassTag](val vids: Array[VertexId], val attrs: Array[VD])
extends Serializable {
- def iterator: Iterator[(VertexID, VD)] =
+ def iterator: Iterator[(VertexId, VD)] =
(0 until vids.size).iterator.map { i => (vids(i), attrs(i)) }
}
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala
index f342fd7437..fe44e1ee0c 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTable.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx.impl
import org.apache.spark.SparkContext._
@@ -15,12 +32,12 @@ import org.apache.spark.util.collection.PrimitiveVector
private[impl]
class RoutingTable(edges: EdgeRDD[_], vertices: VertexRDD[_]) {
- val bothAttrs: RDD[Array[Array[VertexID]]] = createPid2Vid(true, true)
- val srcAttrOnly: RDD[Array[Array[VertexID]]] = createPid2Vid(true, false)
- val dstAttrOnly: RDD[Array[Array[VertexID]]] = createPid2Vid(false, true)
- val noAttrs: RDD[Array[Array[VertexID]]] = createPid2Vid(false, false)
+ val bothAttrs: RDD[Array[Array[VertexId]]] = createPid2Vid(true, true)
+ val srcAttrOnly: RDD[Array[Array[VertexId]]] = createPid2Vid(true, false)
+ val dstAttrOnly: RDD[Array[Array[VertexId]]] = createPid2Vid(false, true)
+ val noAttrs: RDD[Array[Array[VertexId]]] = createPid2Vid(false, false)
- def get(includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[VertexID]]] =
+ def get(includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[VertexId]]] =
(includeSrcAttr, includeDstAttr) match {
case (true, true) => bothAttrs
case (true, false) => srcAttrOnly
@@ -29,9 +46,9 @@ class RoutingTable(edges: EdgeRDD[_], vertices: VertexRDD[_]) {
}
private def createPid2Vid(
- includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[VertexID]]] = {
+ includeSrcAttr: Boolean, includeDstAttr: Boolean): RDD[Array[Array[VertexId]]] = {
// Determine which vertices each edge partition needs by creating a mapping from vid to pid.
- val vid2pid: RDD[(VertexID, PartitionID)] = edges.partitionsRDD.mapPartitions { iter =>
+ val vid2pid: RDD[(VertexId, PartitionID)] = edges.partitionsRDD.mapPartitions { iter =>
val (pid: PartitionID, edgePartition: EdgePartition[_]) = iter.next()
val numEdges = edgePartition.size
val vSet = new VertexSet
@@ -54,7 +71,7 @@ class RoutingTable(edges: EdgeRDD[_], vertices: VertexRDD[_]) {
val numPartitions = vertices.partitions.size
vid2pid.partitionBy(vertices.partitioner.get).mapPartitions { iter =>
- val pid2vid = Array.fill(numPartitions)(new PrimitiveVector[VertexID])
+ val pid2vid = Array.fill(numPartitions)(new PrimitiveVector[VertexId])
for ((vid, pid) <- iter) {
pid2vid(pid) += vid
}
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala
index cbd6318f33..c74d487e20 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx.impl
import java.io.{EOFException, InputStream, OutputStream}
@@ -8,12 +25,12 @@ import org.apache.spark.graphx._
import org.apache.spark.serializer._
private[graphx]
-class VertexIDMsgSerializer(conf: SparkConf) extends Serializer {
+class VertexIdMsgSerializer(conf: SparkConf) extends Serializer {
override def newInstance(): SerializerInstance = new ShuffleSerializerInstance {
override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) {
def writeObject[T](t: T) = {
- val msg = t.asInstanceOf[(VertexID, _)]
+ val msg = t.asInstanceOf[(VertexId, _)]
writeVarLong(msg._1, optimizePositive = false)
this
}
@@ -106,7 +123,7 @@ class IntAggMsgSerializer(conf: SparkConf) extends Serializer {
override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) {
def writeObject[T](t: T) = {
- val msg = t.asInstanceOf[(VertexID, Int)]
+ val msg = t.asInstanceOf[(VertexId, Int)]
writeVarLong(msg._1, optimizePositive = false)
writeUnsignedVarInt(msg._2)
this
@@ -130,7 +147,7 @@ class LongAggMsgSerializer(conf: SparkConf) extends Serializer {
override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) {
def writeObject[T](t: T) = {
- val msg = t.asInstanceOf[(VertexID, Long)]
+ val msg = t.asInstanceOf[(VertexId, Long)]
writeVarLong(msg._1, optimizePositive = false)
writeVarLong(msg._2, optimizePositive = true)
this
@@ -154,7 +171,7 @@ class DoubleAggMsgSerializer(conf: SparkConf) extends Serializer {
override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) {
def writeObject[T](t: T) = {
- val msg = t.asInstanceOf[(VertexID, Double)]
+ val msg = t.asInstanceOf[(VertexId, Double)]
writeVarLong(msg._1, optimizePositive = false)
writeDouble(msg._2)
this
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala
index f97ff75fb2..7a54b413dc 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexPartition.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx.impl
import scala.reflect.ClassTag
@@ -9,18 +26,18 @@ import org.apache.spark.util.collection.BitSet
private[graphx] object VertexPartition {
- def apply[VD: ClassTag](iter: Iterator[(VertexID, VD)]): VertexPartition[VD] = {
- val map = new PrimitiveKeyOpenHashMap[VertexID, VD]
+ def apply[VD: ClassTag](iter: Iterator[(VertexId, VD)]): VertexPartition[VD] = {
+ val map = new PrimitiveKeyOpenHashMap[VertexId, VD]
iter.foreach { case (k, v) =>
map(k) = v
}
new VertexPartition(map.keySet, map._values, map.keySet.getBitSet)
}
- def apply[VD: ClassTag](iter: Iterator[(VertexID, VD)], mergeFunc: (VD, VD) => VD)
+ def apply[VD: ClassTag](iter: Iterator[(VertexId, VD)], mergeFunc: (VD, VD) => VD)
: VertexPartition[VD] =
{
- val map = new PrimitiveKeyOpenHashMap[VertexID, VD]
+ val map = new PrimitiveKeyOpenHashMap[VertexId, VD]
iter.foreach { case (k, v) =>
map.setMerge(k, v, mergeFunc)
}
@@ -43,15 +60,15 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag](
def size: Int = mask.cardinality()
/** Return the vertex attribute for the given vertex ID. */
- def apply(vid: VertexID): VD = values(index.getPos(vid))
+ def apply(vid: VertexId): VD = values(index.getPos(vid))
- def isDefined(vid: VertexID): Boolean = {
+ def isDefined(vid: VertexId): Boolean = {
val pos = index.getPos(vid)
pos >= 0 && mask.get(pos)
}
/** Look up vid in activeSet, throwing an exception if it is None. */
- def isActive(vid: VertexID): Boolean = {
+ def isActive(vid: VertexId): Boolean = {
activeSet.get.contains(vid)
}
@@ -71,7 +88,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag](
* each of the entries in the original VertexRDD. The resulting
* VertexPartition retains the same index.
*/
- def map[VD2: ClassTag](f: (VertexID, VD) => VD2): VertexPartition[VD2] = {
+ def map[VD2: ClassTag](f: (VertexId, VD) => VD2): VertexPartition[VD2] = {
// Construct a view of the map transformation
val newValues = new Array[VD2](capacity)
var i = mask.nextSetBit(0)
@@ -91,7 +108,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag](
* RDD can be easily joined with the original vertex-set. Furthermore, the filter only
* modifies the bitmap index and so no new values are allocated.
*/
- def filter(pred: (VertexID, VD) => Boolean): VertexPartition[VD] = {
+ def filter(pred: (VertexId, VD) => Boolean): VertexPartition[VD] = {
// Allocate the array to store the results into
val newMask = new BitSet(capacity)
// Iterate over the active bits in the old mask and evaluate the predicate
@@ -129,7 +146,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag](
/** Left outer join another VertexPartition. */
def leftJoin[VD2: ClassTag, VD3: ClassTag]
(other: VertexPartition[VD2])
- (f: (VertexID, VD, Option[VD2]) => VD3): VertexPartition[VD3] = {
+ (f: (VertexId, VD, Option[VD2]) => VD3): VertexPartition[VD3] = {
if (index != other.index) {
logWarning("Joining two VertexPartitions with different indexes is slow.")
leftJoin(createUsingIndex(other.iterator))(f)
@@ -148,14 +165,14 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag](
/** Left outer join another iterator of messages. */
def leftJoin[VD2: ClassTag, VD3: ClassTag]
- (other: Iterator[(VertexID, VD2)])
- (f: (VertexID, VD, Option[VD2]) => VD3): VertexPartition[VD3] = {
+ (other: Iterator[(VertexId, VD2)])
+ (f: (VertexId, VD, Option[VD2]) => VD3): VertexPartition[VD3] = {
leftJoin(createUsingIndex(other))(f)
}
/** Inner join another VertexPartition. */
def innerJoin[U: ClassTag, VD2: ClassTag](other: VertexPartition[U])
- (f: (VertexID, VD, U) => VD2): VertexPartition[VD2] = {
+ (f: (VertexId, VD, U) => VD2): VertexPartition[VD2] = {
if (index != other.index) {
logWarning("Joining two VertexPartitions with different indexes is slow.")
innerJoin(createUsingIndex(other.iterator))(f)
@@ -175,15 +192,15 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag](
* Inner join an iterator of messages.
*/
def innerJoin[U: ClassTag, VD2: ClassTag]
- (iter: Iterator[Product2[VertexID, U]])
- (f: (VertexID, VD, U) => VD2): VertexPartition[VD2] = {
+ (iter: Iterator[Product2[VertexId, U]])
+ (f: (VertexId, VD, U) => VD2): VertexPartition[VD2] = {
innerJoin(createUsingIndex(iter))(f)
}
/**
* Similar effect as aggregateUsingIndex((a, b) => a)
*/
- def createUsingIndex[VD2: ClassTag](iter: Iterator[Product2[VertexID, VD2]])
+ def createUsingIndex[VD2: ClassTag](iter: Iterator[Product2[VertexId, VD2]])
: VertexPartition[VD2] = {
val newMask = new BitSet(capacity)
val newValues = new Array[VD2](capacity)
@@ -201,7 +218,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag](
* Similar to innerJoin, but vertices from the left side that don't appear in iter will remain in
* the partition, hidden by the bitmask.
*/
- def innerJoinKeepLeft(iter: Iterator[Product2[VertexID, VD]]): VertexPartition[VD] = {
+ def innerJoinKeepLeft(iter: Iterator[Product2[VertexId, VD]]): VertexPartition[VD] = {
val newMask = new BitSet(capacity)
val newValues = new Array[VD](capacity)
System.arraycopy(values, 0, newValues, 0, newValues.length)
@@ -216,7 +233,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag](
}
def aggregateUsingIndex[VD2: ClassTag](
- iter: Iterator[Product2[VertexID, VD2]],
+ iter: Iterator[Product2[VertexId, VD2]],
reduceFunc: (VD2, VD2) => VD2): VertexPartition[VD2] = {
val newMask = new BitSet(capacity)
val newValues = new Array[VD2](capacity)
@@ -236,7 +253,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag](
new VertexPartition[VD2](index, newValues, newMask)
}
- def replaceActives(iter: Iterator[VertexID]): VertexPartition[VD] = {
+ def replaceActives(iter: Iterator[VertexId]): VertexPartition[VD] = {
val newActiveSet = new VertexSet
iter.foreach(newActiveSet.add(_))
new VertexPartition(index, values, mask, Some(newActiveSet))
@@ -246,7 +263,7 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag](
* Construct a new VertexPartition whose index contains only the vertices in the mask.
*/
def reindex(): VertexPartition[VD] = {
- val hashMap = new PrimitiveKeyOpenHashMap[VertexID, VD]
+ val hashMap = new PrimitiveKeyOpenHashMap[VertexId, VD]
val arbitraryMerge = (a: VD, b: VD) => a
for ((k, v) <- this.iterator) {
hashMap.setMerge(k, v, arbitraryMerge)
@@ -254,8 +271,8 @@ class VertexPartition[@specialized(Long, Int, Double) VD: ClassTag](
new VertexPartition(hashMap.keySet, hashMap._values, hashMap.keySet.getBitSet)
}
- def iterator: Iterator[(VertexID, VD)] =
+ def iterator: Iterator[(VertexId, VD)] =
mask.iterator.map(ind => (index.getValue(ind), values(ind)))
- def vidIterator: Iterator[VertexID] = mask.iterator.map(ind => index.getValue(ind))
+ def vidIterator: Iterator[VertexId] = mask.iterator.map(ind => index.getValue(ind))
}
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/package.scala
index cfc3281b64..79549fe060 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/package.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/package.scala
@@ -1,7 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx
import org.apache.spark.util.collection.OpenHashSet
package object impl {
- private[graphx] type VertexIdToIndexMap = OpenHashSet[VertexID]
+ private[graphx] type VertexIdToIndexMap = OpenHashSet[VertexId]
}
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala
index e0aff5644e..f914e0565c 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx.lib
import org.apache.spark._
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala
index 4d1f5e74df..e2f6cc1389 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/ConnectedComponents.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx.lib
import scala.reflect.ClassTag
@@ -18,9 +35,9 @@ object ConnectedComponents {
* @return a graph with vertex attributes containing the smallest vertex in each
* connected component
*/
- def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]): Graph[VertexID, ED] = {
+ def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]): Graph[VertexId, ED] = {
val ccGraph = graph.mapVertices { case (vid, _) => vid }
- def sendMessage(edge: EdgeTriplet[VertexID, ED]) = {
+ def sendMessage(edge: EdgeTriplet[VertexId, ED]) = {
if (edge.srcAttr < edge.dstAttr) {
Iterator((edge.dstId, edge.srcAttr))
} else if (edge.srcAttr > edge.dstAttr) {
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala
index 2f4d6d6864..614555a054 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx.lib
import scala.reflect.ClassTag
@@ -75,7 +92,7 @@ object PageRank extends Logging {
// Define the three functions needed to implement PageRank in the GraphX
// version of Pregel
- def vertexProgram(id: VertexID, attr: Double, msgSum: Double): Double =
+ def vertexProgram(id: VertexId, attr: Double, msgSum: Double): Double =
resetProb + (1.0 - resetProb) * msgSum
def sendMessage(edge: EdgeTriplet[Double, Double]) =
Iterator((edge.dstId, edge.srcAttr * edge.attr))
@@ -120,7 +137,7 @@ object PageRank extends Logging {
// Define the three functions needed to implement PageRank in the GraphX
// version of Pregel
- def vertexProgram(id: VertexID, attr: (Double, Double), msgSum: Double): (Double, Double) = {
+ def vertexProgram(id: VertexId, attr: (Double, Double), msgSum: Double): (Double, Double) = {
val (oldPR, lastDelta) = attr
val newPR = oldPR + (1.0 - resetProb) * msgSum
(newPR, newPR - oldPR)
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala
index ba6517e012..c327ce7935 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx.lib
import scala.util.Random
@@ -62,13 +79,13 @@ object SVDPlusPlus {
(g1: (Long, Double), g2: (Long, Double)) => (g1._1 + g2._1, g1._2 + g2._2))
g = g.outerJoinVertices(t0) {
- (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[(Long, Double)]) =>
+ (vid: VertexId, vd: (RealVector, RealVector, Double, Double), msg: Option[(Long, Double)]) =>
(vd._1, vd._2, msg.get._2 / msg.get._1, 1.0 / scala.math.sqrt(msg.get._1))
}
def mapTrainF(conf: Conf, u: Double)
(et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double])
- : Iterator[(VertexID, (RealVector, RealVector, Double))] = {
+ : Iterator[(VertexId, (RealVector, RealVector, Double))] = {
val (usr, itm) = (et.srcAttr, et.dstAttr)
val (p, q) = (usr._1, itm._1)
var pred = u + usr._3 + itm._3 + q.dotProduct(usr._2)
@@ -95,7 +112,7 @@ object SVDPlusPlus {
et => Iterator((et.srcId, et.dstAttr._2)),
(g1: RealVector, g2: RealVector) => g1.add(g2))
g = g.outerJoinVertices(t1) {
- (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[RealVector]) =>
+ (vid: VertexId, vd: (RealVector, RealVector, Double, Double), msg: Option[RealVector]) =>
if (msg.isDefined) (vd._1, vd._1.add(msg.get.mapMultiply(vd._4)), vd._3, vd._4) else vd
}
@@ -106,7 +123,7 @@ object SVDPlusPlus {
(g1: (RealVector, RealVector, Double), g2: (RealVector, RealVector, Double)) =>
(g1._1.add(g2._1), g1._2.add(g2._2), g1._3 + g2._3))
g = g.outerJoinVertices(t2) {
- (vid: VertexID,
+ (vid: VertexId,
vd: (RealVector, RealVector, Double, Double),
msg: Option[(RealVector, RealVector, Double)]) =>
(vd._1.add(msg.get._1), vd._2.add(msg.get._2), vd._3 + msg.get._3, vd._4)
@@ -116,7 +133,7 @@ object SVDPlusPlus {
// calculate error on training set
def mapTestF(conf: Conf, u: Double)
(et: EdgeTriplet[(RealVector, RealVector, Double, Double), Double])
- : Iterator[(VertexID, Double)] =
+ : Iterator[(VertexId, Double)] =
{
val (usr, itm) = (et.srcAttr, et.dstAttr)
val (p, q) = (usr._1, itm._1)
@@ -129,7 +146,7 @@ object SVDPlusPlus {
g.cache()
val t3 = g.mapReduceTriplets(mapTestF(conf, u), (g1: Double, g2: Double) => g1 + g2)
g = g.outerJoinVertices(t3) {
- (vid: VertexID, vd: (RealVector, RealVector, Double, Double), msg: Option[Double]) =>
+ (vid: VertexId, vd: (RealVector, RealVector, Double, Double), msg: Option[Double]) =>
if (msg.isDefined) (vd._1, vd._2, vd._3, msg.get) else vd
}
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala
index d3d496e335..46da38eeb7 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx.lib
import scala.reflect.ClassTag
@@ -18,7 +35,7 @@ object StronglyConnectedComponents {
*
* @return a graph with vertex attributes containing the smallest vertex id in each SCC
*/
- def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED], numIter: Int): Graph[VertexID, ED] = {
+ def run[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED], numIter: Int): Graph[VertexId, ED] = {
// the graph we update with final SCC ids, and the graph we return at the end
var sccGraph = graph.mapVertices { case (vid, _) => vid }
@@ -54,7 +71,7 @@ object StronglyConnectedComponents {
// collect min of all my neighbor's scc values, update if it's smaller than mine
// then notify any neighbors with scc values larger than mine
- sccWorkGraph = Pregel[(VertexID, Boolean), ED, VertexID](
+ sccWorkGraph = Pregel[(VertexId, Boolean), ED, VertexId](
sccWorkGraph, Long.MaxValue, activeDirection = EdgeDirection.Out)(
(vid, myScc, neighborScc) => (math.min(myScc._1, neighborScc), myScc._2),
e => {
@@ -68,7 +85,7 @@ object StronglyConnectedComponents {
// start at root of SCCs. Traverse values in reverse, notify all my neighbors
// do not propagate if colors do not match!
- sccWorkGraph = Pregel[(VertexID, Boolean), ED, Boolean](
+ sccWorkGraph = Pregel[(VertexId, Boolean), ED, Boolean](
sccWorkGraph, false, activeDirection = EdgeDirection.In)(
// vertex is final if it is the root of a color
// or it has the same color as a neighbor that is final
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala
index 23c9c40594..7c396e6e66 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/TriangleCount.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx.lib
import scala.reflect.ClassTag
@@ -44,7 +61,7 @@ object TriangleCount {
(vid, _, optSet) => optSet.getOrElse(null)
}
// Edge function computes intersection of smaller vertex with larger vertex
- def edgeFunc(et: EdgeTriplet[VertexSet, ED]): Iterator[(VertexID, Int)] = {
+ def edgeFunc(et: EdgeTriplet[VertexSet, ED]): Iterator[(VertexId, Int)] = {
assert(et.srcAttr != null)
assert(et.dstAttr != null)
val (smallSet, largeSet) = if (et.srcAttr.size < et.dstAttr.size) {
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/package.scala
index 60dfc1dc37..425a5164ca 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/package.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/package.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark
import org.apache.spark.util.collection.OpenHashSet
@@ -8,11 +25,11 @@ package object graphx {
* A 64-bit vertex identifier that uniquely identifies a vertex within a graph. It does not need
* to follow any ordering or any constraints other than uniqueness.
*/
- type VertexID = Long
+ type VertexId = Long
/** Integer identifer of a graph partition. */
// TODO: Consider using Char.
type PartitionID = Int
- private[graphx] type VertexSet = OpenHashSet[VertexID]
+ private[graphx] type VertexSet = OpenHashSet[VertexId]
}
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala
index 1c5b234d74..d1528e2f07 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx.util
import java.io.{ByteArrayInputStream, ByteArrayOutputStream}
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala
index 57422ce3f1..7677641bfe 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx.util
import scala.annotation.tailrec
@@ -33,7 +50,7 @@ object GraphGenerators {
val mu = 4
val sigma = 1.3
- val vertices: RDD[(VertexID, Int)] = sc.parallelize(0 until numVertices).map{
+ val vertices: RDD[(VertexId, Int)] = sc.parallelize(0 until numVertices).map{
src => (src, sampleLogNormal(mu, sigma, numVertices))
}
val edges = vertices.flatMap { v =>
@@ -42,9 +59,9 @@ object GraphGenerators {
Graph(vertices, edges, 0)
}
- def generateRandomEdges(src: Int, numEdges: Int, maxVertexID: Int): Array[Edge[Int]] = {
+ def generateRandomEdges(src: Int, numEdges: Int, maxVertexId: Int): Array[Edge[Int]] = {
val rand = new Random()
- Array.fill(maxVertexID) { Edge[Int](src, rand.nextInt(maxVertexID), 1) }
+ Array.fill(maxVertexId) { Edge[Int](src, rand.nextInt(maxVertexId), 1) }
}
/**
@@ -189,9 +206,9 @@ object GraphGenerators {
*/
def gridGraph(sc: SparkContext, rows: Int, cols: Int): Graph[(Int,Int), Double] = {
// Convert row column address into vertex ids (row major order)
- def sub2ind(r: Int, c: Int): VertexID = r * cols + c
+ def sub2ind(r: Int, c: Int): VertexId = r * cols + c
- val vertices: RDD[(VertexID, (Int,Int))] =
+ val vertices: RDD[(VertexId, (Int,Int))] =
sc.parallelize(0 until rows).flatMap( r => (0 until cols).map( c => (sub2ind(r,c), (r,c)) ) )
val edges: RDD[Edge[Double]] =
vertices.flatMap{ case (vid, (r,c)) =>
@@ -211,7 +228,7 @@ object GraphGenerators {
* being the center vertex.
*/
def starGraph(sc: SparkContext, nverts: Int): Graph[Int, Int] = {
- val edges: RDD[(VertexID, VertexID)] = sc.parallelize(1 until nverts).map(vid => (vid, 0))
+ val edges: RDD[(VertexId, VertexId)] = sc.parallelize(1 until nverts).map(vid => (vid, 0))
Graph.fromEdgeTuples(edges, 1)
} // end of starGraph
diff --git a/graphx/src/test/resources/als-test.data b/graphx/src/test/resources/als-test.data
new file mode 100644
index 0000000000..e476cc23e0
--- /dev/null
+++ b/graphx/src/test/resources/als-test.data
@@ -0,0 +1,16 @@
+1,1,5.0
+1,2,1.0
+1,3,5.0
+1,4,1.0
+2,1,5.0
+2,2,1.0
+2,3,5.0
+2,4,1.0
+3,1,1.0
+3,2,5.0
+3,3,1.0
+3,4,5.0
+4,1,1.0
+4,2,5.0
+4,3,1.0
+4,4,5.0
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala
index 280f50e39a..bc2ad5677f 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx
import org.apache.spark.SparkContext
@@ -11,12 +28,12 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext {
test("joinVertices") {
withSpark { sc =>
val vertices =
- sc.parallelize(Seq[(VertexID, String)]((1, "one"), (2, "two"), (3, "three")), 2)
+ sc.parallelize(Seq[(VertexId, String)]((1, "one"), (2, "two"), (3, "three")), 2)
val edges = sc.parallelize((Seq(Edge(1, 2, "onetwo"))))
val g: Graph[String, String] = Graph(vertices, edges)
- val tbl = sc.parallelize(Seq[(VertexID, Int)]((1, 10), (2, 20)))
- val g1 = g.joinVertices(tbl) { (vid: VertexID, attr: String, u: Int) => attr + u }
+ val tbl = sc.parallelize(Seq[(VertexId, Int)]((1, 10), (2, 20)))
+ val g1 = g.joinVertices(tbl) { (vid: VertexId, attr: String, u: Int) => attr + u }
val v = g1.vertices.collect().toSet
assert(v === Set((1, "one10"), (2, "two20"), (3, "three")))
@@ -43,7 +60,7 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext {
test ("filter") {
withSpark { sc =>
val n = 5
- val vertices = sc.parallelize((0 to n).map(x => (x:VertexID, x)))
+ val vertices = sc.parallelize((0 to n).map(x => (x:VertexId, x)))
val edges = sc.parallelize((1 to n).map(x => Edge(0, x, x)))
val graph: Graph[Int, Int] = Graph(vertices, edges).cache()
val filteredGraph = graph.filter(
@@ -51,7 +68,7 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext {
val degrees: VertexRDD[Int] = graph.outDegrees
graph.outerJoinVertices(degrees) {(vid, data, deg) => deg.getOrElse(0)}
},
- vpred = (vid: VertexID, deg:Int) => deg > 0
+ vpred = (vid: VertexId, deg:Int) => deg > 0
).cache()
val v = filteredGraph.vertices.collect().toSet
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala
index 9587f04c3e..28d34dd9a1 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx
import org.scalatest.FunSuite
@@ -10,7 +27,7 @@ import org.apache.spark.rdd._
class GraphSuite extends FunSuite with LocalSparkContext {
def starGraph(sc: SparkContext, n: Int): Graph[String, Int] = {
- Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: VertexID, x: VertexID)), 3), "v")
+ Graph.fromEdgeTuples(sc.parallelize((1 to n).map(x => (0: VertexId, x: VertexId)), 3), "v")
}
test("Graph.fromEdgeTuples") {
@@ -40,7 +57,7 @@ class GraphSuite extends FunSuite with LocalSparkContext {
withSpark { sc =>
val rawEdges = (0L to 98L).zip((1L to 99L) :+ 0L)
val edges: RDD[Edge[Int]] = sc.parallelize(rawEdges).map { case (s, t) => Edge(s, t, 1) }
- val vertices: RDD[(VertexID, Boolean)] = sc.parallelize((0L until 10L).map(id => (id, true)))
+ val vertices: RDD[(VertexId, Boolean)] = sc.parallelize((0L until 10L).map(id => (id, true)))
val graph = Graph(vertices, edges, false)
assert( graph.edges.count() === rawEdges.size )
// Vertices not explicitly provided but referenced by edges should be created automatically
@@ -57,7 +74,7 @@ class GraphSuite extends FunSuite with LocalSparkContext {
val n = 5
val star = starGraph(sc, n)
assert(star.triplets.map(et => (et.srcId, et.dstId, et.srcAttr, et.dstAttr)).collect.toSet ===
- (1 to n).map(x => (0: VertexID, x: VertexID, "v", "v")).toSet)
+ (1 to n).map(x => (0: VertexId, x: VertexId, "v", "v")).toSet)
}
}
@@ -93,7 +110,7 @@ class GraphSuite extends FunSuite with LocalSparkContext {
val p = 100
val verts = 1 to n
val graph = Graph.fromEdgeTuples(sc.parallelize(verts.flatMap(x =>
- verts.filter(y => y % x == 0).map(y => (x: VertexID, y: VertexID))), p), 0)
+ verts.filter(y => y % x == 0).map(y => (x: VertexId, y: VertexId))), p), 0)
assert(graph.edges.partitions.length === p)
val partitionedGraph = graph.partitionBy(EdgePartition2D)
assert(graph.edges.partitions.length === p)
@@ -119,10 +136,10 @@ class GraphSuite extends FunSuite with LocalSparkContext {
val star = starGraph(sc, n)
// mapVertices preserving type
val mappedVAttrs = star.mapVertices((vid, attr) => attr + "2")
- assert(mappedVAttrs.vertices.collect.toSet === (0 to n).map(x => (x: VertexID, "v2")).toSet)
+ assert(mappedVAttrs.vertices.collect.toSet === (0 to n).map(x => (x: VertexId, "v2")).toSet)
// mapVertices changing type
val mappedVAttrs2 = star.mapVertices((vid, attr) => attr.length)
- assert(mappedVAttrs2.vertices.collect.toSet === (0 to n).map(x => (x: VertexID, 1)).toSet)
+ assert(mappedVAttrs2.vertices.collect.toSet === (0 to n).map(x => (x: VertexId, 1)).toSet)
}
}
@@ -151,7 +168,7 @@ class GraphSuite extends FunSuite with LocalSparkContext {
withSpark { sc =>
val n = 5
val star = starGraph(sc, n)
- assert(star.reverse.outDegrees.collect.toSet === (1 to n).map(x => (x: VertexID, 1)).toSet)
+ assert(star.reverse.outDegrees.collect.toSet === (1 to n).map(x => (x: VertexId, 1)).toSet)
}
}
@@ -174,7 +191,7 @@ class GraphSuite extends FunSuite with LocalSparkContext {
test("mask") {
withSpark { sc =>
val n = 5
- val vertices = sc.parallelize((0 to n).map(x => (x:VertexID, x)))
+ val vertices = sc.parallelize((0 to n).map(x => (x:VertexId, x)))
val edges = sc.parallelize((1 to n).map(x => Edge(0, x, x)))
val graph: Graph[Int, Int] = Graph(vertices, edges).cache()
@@ -201,7 +218,7 @@ class GraphSuite extends FunSuite with LocalSparkContext {
val star = starGraph(sc, n)
val doubleStar = Graph.fromEdgeTuples(
sc.parallelize((1 to n).flatMap(x =>
- List((0: VertexID, x: VertexID), (0: VertexID, x: VertexID))), 1), "v")
+ List((0: VertexId, x: VertexId), (0: VertexId, x: VertexId))), 1), "v")
val star2 = doubleStar.groupEdges { (a, b) => a}
assert(star2.edges.collect.toArray.sorted(Edge.lexicographicOrdering[Int]) ===
star.edges.collect.toArray.sorted(Edge.lexicographicOrdering[Int]))
@@ -220,7 +237,7 @@ class GraphSuite extends FunSuite with LocalSparkContext {
assert(neighborDegreeSums.collect().toSet === (0 to n).map(x => (x, n)).toSet)
// activeSetOpt
- val allPairs = for (x <- 1 to n; y <- 1 to n) yield (x: VertexID, y: VertexID)
+ val allPairs = for (x <- 1 to n; y <- 1 to n) yield (x: VertexId, y: VertexId)
val complete = Graph.fromEdgeTuples(sc.parallelize(allPairs, 3), 0)
val vids = complete.mapVertices((vid, attr) => vid).cache()
val active = vids.vertices.filter { case (vid, attr) => attr % 2 == 0 }
@@ -231,10 +248,10 @@ class GraphSuite extends FunSuite with LocalSparkContext {
}
Iterator((et.srcId, 1))
}, (a: Int, b: Int) => a + b, Some((active, EdgeDirection.In))).collect.toSet
- assert(numEvenNeighbors === (1 to n).map(x => (x: VertexID, n / 2)).toSet)
+ assert(numEvenNeighbors === (1 to n).map(x => (x: VertexId, n / 2)).toSet)
// outerJoinVertices followed by mapReduceTriplets(activeSetOpt)
- val ringEdges = sc.parallelize((0 until n).map(x => (x: VertexID, (x+1) % n: VertexID)), 3)
+ val ringEdges = sc.parallelize((0 until n).map(x => (x: VertexId, (x+1) % n: VertexId)), 3)
val ring = Graph.fromEdgeTuples(ringEdges, 0) .mapVertices((vid, attr) => vid).cache()
val changed = ring.vertices.filter { case (vid, attr) => attr % 2 == 1 }.mapValues(-_).cache()
val changedGraph = ring.outerJoinVertices(changed) { (vid, old, newOpt) => newOpt.getOrElse(old) }
@@ -245,7 +262,7 @@ class GraphSuite extends FunSuite with LocalSparkContext {
}
Iterator((et.dstId, 1))
}, (a: Int, b: Int) => a + b, Some(changed, EdgeDirection.Out)).collect.toSet
- assert(numOddNeighbors === (2 to n by 2).map(x => (x: VertexID, 1)).toSet)
+ assert(numOddNeighbors === (2 to n by 2).map(x => (x: VertexId, 1)).toSet)
}
}
@@ -260,7 +277,7 @@ class GraphSuite extends FunSuite with LocalSparkContext {
val neighborDegreeSums = reverseStarDegrees.mapReduceTriplets(
et => Iterator((et.srcId, et.dstAttr), (et.dstId, et.srcAttr)),
(a: Int, b: Int) => a + b).collect.toSet
- assert(neighborDegreeSums === Set((0: VertexID, n)) ++ (1 to n).map(x => (x: VertexID, 0)))
+ assert(neighborDegreeSums === Set((0: VertexId, n)) ++ (1 to n).map(x => (x: VertexId, 0)))
// outerJoinVertices preserving type
val messages = reverseStar.vertices.mapValues { (vid, attr) => vid.toString }
val newReverseStar =
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala b/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala
index aa9ba84084..51f02f94e0 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx
import org.scalatest.Suite
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala
index bceff11b8e..490b94429e 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/PregelSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx
import org.scalatest.FunSuite
@@ -10,7 +27,7 @@ class PregelSuite extends FunSuite with LocalSparkContext {
test("1 iteration") {
withSpark { sc =>
val n = 5
- val starEdges = (1 to n).map(x => (0: VertexID, x: VertexID))
+ val starEdges = (1 to n).map(x => (0: VertexId, x: VertexId))
val star = Graph.fromEdgeTuples(sc.parallelize(starEdges, 3), "v").cache()
val result = Pregel(star, 0)(
(vid, attr, msg) => attr,
@@ -24,12 +41,12 @@ class PregelSuite extends FunSuite with LocalSparkContext {
withSpark { sc =>
val n = 5
val chain = Graph.fromEdgeTuples(
- sc.parallelize((1 until n).map(x => (x: VertexID, x + 1: VertexID)), 3),
+ sc.parallelize((1 until n).map(x => (x: VertexId, x + 1: VertexId)), 3),
0).cache()
- assert(chain.vertices.collect.toSet === (1 to n).map(x => (x: VertexID, 0)).toSet)
+ assert(chain.vertices.collect.toSet === (1 to n).map(x => (x: VertexId, 0)).toSet)
val chainWithSeed = chain.mapVertices { (vid, attr) => if (vid == 1) 1 else 0 }.cache()
assert(chainWithSeed.vertices.collect.toSet ===
- Set((1: VertexID, 1)) ++ (2 to n).map(x => (x: VertexID, 0)).toSet)
+ Set((1: VertexId, 1)) ++ (2 to n).map(x => (x: VertexId, 0)).toSet)
val result = Pregel(chainWithSeed, 0)(
(vid, attr, msg) => math.max(msg, attr),
et => if (et.dstAttr != et.srcAttr) Iterator((et.dstId, et.srcAttr)) else Iterator.empty,
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala
index 3ba412c1f8..e5a582b47b 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx
import java.io.{EOFException, ByteArrayInputStream, ByteArrayOutputStream}
@@ -82,7 +99,7 @@ class SerializerSuite extends FunSuite with LocalSparkContext {
test("IntAggMsgSerializer") {
val conf = new SparkConf(false)
- val outMsg = (4: VertexID, 5)
+ val outMsg = (4: VertexId, 5)
val bout = new ByteArrayOutputStream
val outStrm = new IntAggMsgSerializer(conf).newInstance().serializeStream(bout)
outStrm.writeObject(outMsg)
@@ -90,8 +107,8 @@ class SerializerSuite extends FunSuite with LocalSparkContext {
bout.flush()
val bin = new ByteArrayInputStream(bout.toByteArray)
val inStrm = new IntAggMsgSerializer(conf).newInstance().deserializeStream(bin)
- val inMsg1: (VertexID, Int) = inStrm.readObject()
- val inMsg2: (VertexID, Int) = inStrm.readObject()
+ val inMsg1: (VertexId, Int) = inStrm.readObject()
+ val inMsg2: (VertexId, Int) = inStrm.readObject()
assert(outMsg === inMsg1)
assert(outMsg === inMsg2)
@@ -102,7 +119,7 @@ class SerializerSuite extends FunSuite with LocalSparkContext {
test("LongAggMsgSerializer") {
val conf = new SparkConf(false)
- val outMsg = (4: VertexID, 1L << 32)
+ val outMsg = (4: VertexId, 1L << 32)
val bout = new ByteArrayOutputStream
val outStrm = new LongAggMsgSerializer(conf).newInstance().serializeStream(bout)
outStrm.writeObject(outMsg)
@@ -110,8 +127,8 @@ class SerializerSuite extends FunSuite with LocalSparkContext {
bout.flush()
val bin = new ByteArrayInputStream(bout.toByteArray)
val inStrm = new LongAggMsgSerializer(conf).newInstance().deserializeStream(bin)
- val inMsg1: (VertexID, Long) = inStrm.readObject()
- val inMsg2: (VertexID, Long) = inStrm.readObject()
+ val inMsg1: (VertexId, Long) = inStrm.readObject()
+ val inMsg2: (VertexId, Long) = inStrm.readObject()
assert(outMsg === inMsg1)
assert(outMsg === inMsg2)
@@ -122,7 +139,7 @@ class SerializerSuite extends FunSuite with LocalSparkContext {
test("DoubleAggMsgSerializer") {
val conf = new SparkConf(false)
- val outMsg = (4: VertexID, 5.0)
+ val outMsg = (4: VertexId, 5.0)
val bout = new ByteArrayOutputStream
val outStrm = new DoubleAggMsgSerializer(conf).newInstance().serializeStream(bout)
outStrm.writeObject(outMsg)
@@ -130,8 +147,8 @@ class SerializerSuite extends FunSuite with LocalSparkContext {
bout.flush()
val bin = new ByteArrayInputStream(bout.toByteArray)
val inStrm = new DoubleAggMsgSerializer(conf).newInstance().deserializeStream(bin)
- val inMsg1: (VertexID, Double) = inStrm.readObject()
- val inMsg2: (VertexID, Double) = inStrm.readObject()
+ val inMsg1: (VertexId, Double) = inStrm.readObject()
+ val inMsg2: (VertexId, Double) = inStrm.readObject()
assert(outMsg === inMsg1)
assert(outMsg === inMsg2)
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala
index d94a3aa67c..cc86bafd2d 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx
import org.apache.spark.SparkContext
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala
index eb82436f09..e135d1d7ad 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx.impl
import scala.reflect.ClassTag
@@ -62,7 +79,7 @@ class EdgePartitionSuite extends FunSuite {
test("innerJoin") {
def makeEdgePartition[A: ClassTag](xs: Iterable[(Int, Int, A)]): EdgePartition[A] = {
val builder = new EdgePartitionBuilder[A]
- for ((src, dst, attr) <- xs) { builder.add(src: VertexID, dst: VertexID, attr) }
+ for ((src, dst, attr) <- xs) { builder.add(src: VertexId, dst: VertexId, attr) }
builder.toEdgePartition
}
val aList = List((0, 1, 0), (1, 0, 0), (1, 2, 0), (5, 4, 0), (5, 5, 0))
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala
index d37d64e8c8..a048d13fd1 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx.impl
import org.apache.spark.graphx._
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala
index 27c8705bca..3915be15b3 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx.lib
import org.scalatest.FunSuite
@@ -83,7 +100,7 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext {
test("Connected Components on a Toy Connected Graph") {
withSpark { sc =>
// Create an RDD for the vertices
- val users: RDD[(VertexID, (String, String))] =
+ val users: RDD[(VertexId, (String, String))] =
sc.parallelize(Array((3L, ("rxin", "student")), (7L, ("jgonzal", "postdoc")),
(5L, ("franklin", "prof")), (2L, ("istoica", "prof")),
(4L, ("peter", "student"))))
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala
index fe7e4261f8..fc491ae327 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx.lib
import org.scalatest.FunSuite
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala
index e173c652a5..e01df56e94 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala
@@ -1,12 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx.lib
import org.scalatest.FunSuite
-import org.apache.spark.SparkContext
-import org.apache.spark.SparkContext._
import org.apache.spark.graphx._
-import org.apache.spark.graphx.util.GraphGenerators
-import org.apache.spark.rdd._
class SVDPlusPlusSuite extends FunSuite with LocalSparkContext {
@@ -14,16 +27,16 @@ class SVDPlusPlusSuite extends FunSuite with LocalSparkContext {
test("Test SVD++ with mean square error on training set") {
withSpark { sc =>
val svdppErr = 8.0
- val edges = sc.textFile("mllib/data/als/test.data").map { line =>
+ val edges = sc.textFile(getClass.getResource("/als-test.data").getFile).map { line =>
val fields = line.split(",")
Edge(fields(0).toLong * 2, fields(1).toLong * 2 + 1, fields(2).toDouble)
}
val conf = new SVDPlusPlus.Conf(10, 2, 0.0, 5.0, 0.007, 0.007, 0.005, 0.015) // 2 iterations
var (graph, u) = SVDPlusPlus.run(edges, conf)
graph.cache()
- val err = graph.vertices.collect.map{ case (vid, vd) =>
+ val err = graph.vertices.collect().map{ case (vid, vd) =>
if (vid % 2 == 1) vd._4 else 0.0
- }.reduce(_ + _) / graph.triplets.collect.size
+ }.reduce(_ + _) / graph.triplets.collect().size
assert(err <= svdppErr)
}
}
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/StronglyConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/StronglyConnectedComponentsSuite.scala
index 0458311661..df54aa37ca 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/lib/StronglyConnectedComponentsSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/StronglyConnectedComponentsSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx.lib
import org.scalatest.FunSuite
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/TriangleCountSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/TriangleCountSuite.scala
index 3452ce9764..293c7f3ba4 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/lib/TriangleCountSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/TriangleCountSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx.lib
import org.scalatest.FunSuite
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/util/BytecodeUtilsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/util/BytecodeUtilsSuite.scala
index 11db339750..f3b3738db0 100644
--- a/graphx/src/test/scala/org/apache/spark/graphx/util/BytecodeUtilsSuite.scala
+++ b/graphx/src/test/scala/org/apache/spark/graphx/util/BytecodeUtilsSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.graphx.util
import org.scalatest.FunSuite
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala
index 3fec1a909d..efc0eb9353 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala
@@ -24,7 +24,6 @@ import org.apache.spark.mllib.recommendation._
import org.apache.spark.rdd.RDD
import java.nio.ByteBuffer
import java.nio.ByteOrder
-import java.nio.DoubleBuffer
/**
* The Java stubs necessary for the Python mllib bindings.
@@ -81,7 +80,6 @@ class PythonMLLibAPI extends Serializable {
}
val db = bb.asDoubleBuffer()
val ans = new Array[Array[Double]](rows.toInt)
- var i = 0
for (i <- 0 until rows.toInt) {
ans(i) = new Array[Double](cols.toInt)
db.get(ans(i))
@@ -236,7 +234,7 @@ class PythonMLLibAPI extends Serializable {
* Serialize a Rating object into an array of bytes.
* It can be deserialized using RatingDeserializer().
*
- * @param rate
+ * @param rate the Rating object to serialize
* @return
*/
private[spark] def serializeRating(rate: Rating): Array[Byte] = {
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala
index f2964ea446..6dff29dfb4 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala
@@ -17,8 +17,6 @@
package org.apache.spark.mllib.classification
-import scala.math.signum
-
import org.apache.spark.SparkContext
import org.apache.spark.rdd.RDD
import org.apache.spark.mllib.optimization._
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala
index cfc81c985a..980be93157 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala
@@ -19,8 +19,6 @@ package org.apache.spark.mllib.clustering
import org.apache.spark.rdd.RDD
import org.apache.spark.SparkContext._
-import org.apache.spark.mllib.util.MLUtils
-
/**
* A clustering model for K-means. Each point belongs to the cluster with the closest center.
@@ -39,6 +37,6 @@ class KMeansModel(val clusterCenters: Array[Array[Double]]) extends Serializable
* model on the given data.
*/
def computeCost(data: RDD[Array[Double]]): Double = {
- data.map(p => KMeans.pointCost(clusterCenters, p)).sum
+ data.map(p => KMeans.pointCost(clusterCenters, p)).sum()
}
}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala
index fe5cce064b..df599fde76 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala
@@ -17,7 +17,7 @@
package org.apache.spark.mllib.regression
-import org.apache.spark.{Logging, SparkContext}
+import org.apache.spark.SparkContext
import org.apache.spark.rdd.RDD
import org.apache.spark.mllib.optimization._
import org.apache.spark.mllib.util.MLUtils
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala
index c125c6797a..0c0e67fb7b 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala
@@ -17,7 +17,7 @@
package org.apache.spark.mllib.regression
-import org.apache.spark.{Logging, SparkContext}
+import org.apache.spark.SparkContext
import org.apache.spark.rdd.RDD
import org.apache.spark.mllib.optimization._
import org.apache.spark.mllib.util.MLUtils
@@ -76,7 +76,7 @@ class RidgeRegressionWithSGD private (
def createModel(weights: Array[Double], intercept: Double) = {
val weightsMat = new DoubleMatrix(weights.length + 1, 1, (Array(intercept) ++ weights):_*)
val weightsScaled = weightsMat.div(xColSd)
- val interceptScaled = yMean - (weightsMat.transpose().mmul(xColMean.div(xColSd)).get(0))
+ val interceptScaled = yMean - weightsMat.transpose().mmul(xColMean.div(xColSd)).get(0)
new RidgeRegressionModel(weightsScaled.data, interceptScaled)
}
@@ -86,7 +86,7 @@ class RidgeRegressionWithSGD private (
initialWeights: Array[Double])
: RidgeRegressionModel =
{
- val nfeatures: Int = input.first.features.length
+ val nfeatures: Int = input.first().features.length
val nexamples: Long = input.count()
// To avoid penalizing the intercept, we center and scale the data.
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala
index bc5045fb05..2e03684e62 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala
@@ -25,7 +25,6 @@ import org.jblas.DoubleMatrix
import org.apache.spark.SparkContext
import org.apache.spark.rdd.RDD
import org.apache.spark.mllib.regression.LabeledPoint
-import org.apache.spark.mllib.regression.LabeledPoint
/**
* Generate sample data used for Linear Data. This class generates
@@ -73,7 +72,7 @@ object LinearDataGenerator {
val x = Array.fill[Array[Double]](nPoints)(
Array.fill[Double](weights.length)(2 * rnd.nextDouble - 1.0))
val y = x.map { xi =>
- (new DoubleMatrix(1, xi.length, xi:_*)).dot(weightsMat) + intercept + eps * rnd.nextGaussian()
+ new DoubleMatrix(1, xi.length, xi: _*).dot(weightsMat) + intercept + eps * rnd.nextGaussian()
}
y.zip(x).map(p => LabeledPoint(p._1, p._2))
}
@@ -86,7 +85,6 @@ object LinearDataGenerator {
* @param nexamples Number of examples that will be contained in the RDD.
* @param nfeatures Number of features to generate for each example.
* @param eps Epsilon factor by which examples are scaled.
- * @param weights Weights associated with the first weights.length features.
* @param nparts Number of partitions in the RDD. Default value is 2.
*
* @return RDD of LabeledPoint containing sample data.
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala
index d5f3f6b8db..348aba1dea 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.mllib.recommendation
+package org.apache.spark.mllib.util
import scala.util.Random
@@ -23,7 +23,6 @@ import org.jblas.DoubleMatrix
import org.apache.spark.SparkContext
import org.apache.spark.rdd.RDD
-import org.apache.spark.mllib.util.MLUtils
/**
* Generate RDD(s) containing data for Matrix Factorization.
@@ -31,9 +30,9 @@ import org.apache.spark.mllib.util.MLUtils
* This method samples training entries according to the oversampling factor
* 'trainSampFact', which is a multiplicative factor of the number of
* degrees of freedom of the matrix: rank*(m+n-rank).
-*
-* It optionally samples entries for a testing matrix using
-* 'testSampFact', the percentage of the number of training entries
+*
+* It optionally samples entries for a testing matrix using
+* 'testSampFact', the percentage of the number of training entries
* to use for testing.
*
* This method takes the following inputs:
@@ -73,7 +72,7 @@ object MFDataGenerator{
val A = DoubleMatrix.randn(m, rank)
val B = DoubleMatrix.randn(rank, n)
- val z = 1 / (scala.math.sqrt(scala.math.sqrt(rank)))
+ val z = 1 / scala.math.sqrt(scala.math.sqrt(rank))
A.mmuli(z)
B.mmuli(z)
val fullData = A.mmul(B)
@@ -91,7 +90,7 @@ object MFDataGenerator{
.map(x => (fullData.indexRows(x - 1), fullData.indexColumns(x - 1), fullData.get(x - 1)))
// optionally add gaussian noise
- if (noise) {
+ if (noise) {
trainData.map(x => (x._1, x._2, x._3 + rand.nextGaussian * sigma))
}
@@ -107,8 +106,8 @@ object MFDataGenerator{
.map(x => (fullData.indexRows(x - 1), fullData.indexColumns(x - 1), fullData.get(x - 1)))
testData.map(x => x._1 + "," + x._2 + "," + x._3).saveAsTextFile(outputPath)
}
-
+
sc.stop()
-
+
}
}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala
index d91b74c3ac..64c6136a8b 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala
@@ -97,7 +97,7 @@ object MLUtils {
while (col < nfeatures) {
xColMean.put(col, xColSumsMap(col)._1 / nexamples)
val variance =
- (xColSumsMap(col)._2 - (math.pow(xColSumsMap(col)._1, 2) / nexamples)) / (nexamples)
+ (xColSumsMap(col)._2 - (math.pow(xColSumsMap(col)._1, 2) / nexamples)) / nexamples
xColSd.put(col, math.sqrt(variance))
col += 1
}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala
index 07022093f3..c96c94f70e 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala
@@ -56,7 +56,7 @@ object SVMDataGenerator {
val x = Array.fill[Double](nfeatures) {
rnd.nextDouble() * 2.0 - 1.0
}
- val yD = (new DoubleMatrix(1, x.length, x:_*)).dot(trueWeights) + rnd.nextGaussian() * 0.1
+ val yD = new DoubleMatrix(1, x.length, x: _*).dot(trueWeights) + rnd.nextGaussian() * 0.1
val y = if (yD < 0) 0.0 else 1.0
LabeledPoint(y, x)
}
diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java
index 23ea3548b9..073ded6f36 100644
--- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.mllib.classification;
import org.apache.spark.api.java.JavaRDD;
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala
index 34c67294e9..02ede71137 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala
@@ -80,9 +80,9 @@ class LogisticRegressionSuite extends FunSuite with BeforeAndAfterAll with Shoul
}
def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]) {
- val numOffPredictions = predictions.zip(input).filter { case (prediction, expected) =>
- (prediction != expected.label)
- }.size
+ val numOffPredictions = predictions.zip(input).count { case (prediction, expected) =>
+ prediction != expected.label
+ }
// At least 83% of the predictions should be on.
((input.length - numOffPredictions).toDouble / input.length) should be > 0.83
}
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala
index 6a957e3ddc..3357b86f9b 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala
@@ -18,7 +18,6 @@
package org.apache.spark.mllib.classification
import scala.util.Random
-import scala.math.signum
import scala.collection.JavaConversions._
import org.scalatest.BeforeAndAfterAll
@@ -50,7 +49,7 @@ object SVMSuite {
val x = Array.fill[Array[Double]](nPoints)(
Array.fill[Double](weights.length)(rnd.nextDouble() * 2.0 - 1.0))
val y = x.map { xi =>
- val yD = (new DoubleMatrix(1, xi.length, xi:_*)).dot(weightsMat) +
+ val yD = new DoubleMatrix(1, xi.length, xi: _*).dot(weightsMat) +
intercept + 0.01 * rnd.nextGaussian()
if (yD < 0) 0.0 else 1.0
}
@@ -72,9 +71,9 @@ class SVMSuite extends FunSuite with BeforeAndAfterAll {
}
def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]) {
- val numOffPredictions = predictions.zip(input).filter { case (prediction, expected) =>
- (prediction != expected.label)
- }.size
+ val numOffPredictions = predictions.zip(input).count { case (prediction, expected) =>
+ prediction != expected.label
+ }
// At least 80% of the predictions should be on.
assert(numOffPredictions < input.length / 5)
}
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala
index 94245f6027..73657cac89 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala
@@ -17,15 +17,12 @@
package org.apache.spark.mllib.clustering
-import scala.util.Random
import org.scalatest.BeforeAndAfterAll
import org.scalatest.FunSuite
import org.apache.spark.SparkContext
-import org.apache.spark.SparkContext._
-import org.jblas._
class KMeansSuite extends FunSuite with BeforeAndAfterAll {
@transient private var sc: SparkContext = _
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala
index e683a90f57..4e8dbde658 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala
@@ -24,7 +24,6 @@ import org.scalatest.BeforeAndAfterAll
import org.scalatest.FunSuite
import org.apache.spark.SparkContext
-import org.apache.spark.SparkContext._
import org.jblas._
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala
index db980c7bae..b2c8df97a8 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala
@@ -17,8 +17,6 @@
package org.apache.spark.mllib.regression
-import scala.collection.JavaConversions._
-import scala.util.Random
import org.scalatest.BeforeAndAfterAll
import org.scalatest.FunSuite
@@ -41,10 +39,10 @@ class LassoSuite extends FunSuite with BeforeAndAfterAll {
}
def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]) {
- val numOffPredictions = predictions.zip(input).filter { case (prediction, expected) =>
+ val numOffPredictions = predictions.zip(input).count { case (prediction, expected) =>
// A prediction is off if the prediction is more than 0.5 away from expected value.
math.abs(prediction - expected.label) > 0.5
- }.size
+ }
// At least 80% of the predictions should be on.
assert(numOffPredictions < input.length / 5)
}
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala
index ef500c704c..406afbaa3e 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala
@@ -21,7 +21,6 @@ import org.scalatest.BeforeAndAfterAll
import org.scalatest.FunSuite
import org.apache.spark.SparkContext
-import org.apache.spark.SparkContext._
import org.apache.spark.mllib.util.LinearDataGenerator
class LinearRegressionSuite extends FunSuite with BeforeAndAfterAll {
@@ -37,10 +36,10 @@ class LinearRegressionSuite extends FunSuite with BeforeAndAfterAll {
}
def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]) {
- val numOffPredictions = predictions.zip(input).filter { case (prediction, expected) =>
+ val numOffPredictions = predictions.zip(input).count { case (prediction, expected) =>
// A prediction is off if the prediction is more than 0.5 away from expected value.
math.abs(prediction - expected.label) > 0.5
- }.size
+ }
// At least 80% of the predictions should be on.
assert(numOffPredictions < input.length / 5)
}
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala
index c18092d804..1d6a10b66e 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala
@@ -17,15 +17,12 @@
package org.apache.spark.mllib.regression
-import scala.collection.JavaConversions._
-import scala.util.Random
import org.jblas.DoubleMatrix
import org.scalatest.BeforeAndAfterAll
import org.scalatest.FunSuite
import org.apache.spark.SparkContext
-import org.apache.spark.SparkContext._
import org.apache.spark.mllib.util.LinearDataGenerator
class RidgeRegressionSuite extends FunSuite with BeforeAndAfterAll {
diff --git a/pom.xml b/pom.xml
index b25d9d7ef8..54072b053c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -754,15 +754,5 @@
</modules>
</profile>
-
- <profile>
- <id>repl-bin</id>
- <activation>
- <activeByDefault>false</activeByDefault>
- </activation>
- <modules>
- <module>repl-bin</module>
- </modules>
- </profile>
</profiles>
</project>
diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml
deleted file mode 100644
index 869dbdb9b0..0000000000
--- a/repl-bin/pom.xml
+++ /dev/null
@@ -1,184 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
- ~ Licensed to the Apache Software Foundation (ASF) under one or more
- ~ contributor license agreements. See the NOTICE file distributed with
- ~ this work for additional information regarding copyright ownership.
- ~ The ASF licenses this file to You under the Apache License, Version 2.0
- ~ (the "License"); you may not use this file except in compliance with
- ~ the License. You may obtain a copy of the License at
- ~
- ~ http://www.apache.org/licenses/LICENSE-2.0
- ~
- ~ Unless required by applicable law or agreed to in writing, software
- ~ distributed under the License is distributed on an "AS IS" BASIS,
- ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- ~ See the License for the specific language governing permissions and
- ~ limitations under the License.
- -->
-
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
- <modelVersion>4.0.0</modelVersion>
- <parent>
- <groupId>org.apache.spark</groupId>
- <artifactId>spark-parent</artifactId>
- <version>0.9.0-incubating-SNAPSHOT</version>
- <relativePath>../pom.xml</relativePath>
- </parent>
-
- <groupId>org.apache.spark</groupId>
- <artifactId>spark-repl-bin_2.10</artifactId>
- <packaging>pom</packaging>
- <name>Spark Project REPL binary packaging</name>
- <url>http://spark.incubator.apache.org/</url>
-
- <properties>
- <deb.pkg.name>spark</deb.pkg.name>
- <deb.install.path>/usr/share/spark</deb.install.path>
- <deb.user>root</deb.user>
- </properties>
-
- <dependencies>
- <dependency>
- <groupId>org.apache.spark</groupId>
- <artifactId>spark-core_${scala.binary.version}</artifactId>
- <version>${project.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.spark</groupId>
- <artifactId>spark-bagel_${scala.binary.version}</artifactId>
- <version>${project.version}</version>
- <scope>runtime</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.spark</groupId>
- <artifactId>spark-repl_${scala.binary.version}</artifactId>
- <version>${project.version}</version>
- <scope>runtime</scope>
- </dependency>
- </dependencies>
-
- <build>
- <plugins>
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-shade-plugin</artifactId>
- <configuration>
- <shadedArtifactAttached>false</shadedArtifactAttached>
- <outputFile>${project.build.directory}/${project.artifactId}-${project.version}-shaded.jar</outputFile>
- <artifactSet>
- <includes>
- <include>*:*</include>
- </includes>
- </artifactSet>
- <filters>
- <filter>
- <artifact>*:*</artifact>
- <excludes>
- <exclude>META-INF/*.SF</exclude>
- <exclude>META-INF/*.DSA</exclude>
- <exclude>META-INF/*.RSA</exclude>
- </excludes>
- </filter>
- </filters>
- </configuration>
- <executions>
- <execution>
- <phase>package</phase>
- <goals>
- <goal>shade</goal>
- </goals>
- <configuration>
- <transformers>
- <transformer implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer" />
- <transformer implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
- <resource>reference.conf</resource>
- </transformer>
- <transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
- <mainClass>spark.repl.Main</mainClass>
- </transformer>
- </transformers>
- </configuration>
- </execution>
- </executions>
- </plugin>
- </plugins>
- </build>
-
- <profiles>
- <profile>
- <id>deb</id>
- <build>
- <plugins>
- <plugin>
- <groupId>org.codehaus.mojo</groupId>
- <artifactId>buildnumber-maven-plugin</artifactId>
- <version>1.1</version>
- <executions>
- <execution>
- <phase>validate</phase>
- <goals>
- <goal>create</goal>
- </goals>
- <configuration>
- <shortRevisionLength>8</shortRevisionLength>
- </configuration>
- </execution>
- </executions>
- </plugin>
- <plugin>
- <groupId>org.vafer</groupId>
- <artifactId>jdeb</artifactId>
- <version>0.11</version>
- <executions>
- <execution>
- <phase>package</phase>
- <goals>
- <goal>jdeb</goal>
- </goals>
- <configuration>
- <deb>${project.build.directory}/${deb.pkg.name}_${project.version}-${buildNumber}_all.deb</deb>
- <attach>false</attach>
- <compression>gzip</compression>
- <dataSet>
- <data>
- <src>${project.build.directory}/${project.artifactId}-${project.version}-shaded.jar</src>
- <type>file</type>
- <mapper>
- <type>perm</type>
- <user>${deb.user}</user>
- <group>${deb.user}</group>
- <prefix>${deb.install.path}</prefix>
- </mapper>
- </data>
- <data>
- <src>${basedir}/src/deb/bin</src>
- <type>directory</type>
- <mapper>
- <type>perm</type>
- <user>${deb.user}</user>
- <group>${deb.user}</group>
- <prefix>${deb.install.path}</prefix>
- <filemode>744</filemode>
- </mapper>
- </data>
- <data>
- <src>${basedir}/../conf</src>
- <type>directory</type>
- <mapper>
- <type>perm</type>
- <user>${deb.user}</user>
- <group>${deb.user}</group>
- <prefix>${deb.install.path}/conf</prefix>
- <filemode>744</filemode>
- </mapper>
- </data>
- </dataSet>
- </configuration>
- </execution>
- </executions>
- </plugin>
- </plugins>
- </build>
- </profile>
- </profiles>
-</project>
diff --git a/repl-bin/src/deb/bin/run b/repl-bin/src/deb/bin/run
deleted file mode 100755
index 3a6f22f41f..0000000000
--- a/repl-bin/src/deb/bin/run
+++ /dev/null
@@ -1,57 +0,0 @@
-#!/usr/bin/env bash
-
-#
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements. See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-SCALA_VERSION=2.10
-
-# Figure out where the Scala framework is installed
-FWDIR="$(cd `dirname $0`; pwd)"
-
-# Export this as SPARK_HOME
-export SPARK_HOME="$FWDIR"
-
-# Load environment variables from conf/spark-env.sh, if it exists
-if [ -e $FWDIR/conf/spark-env.sh ] ; then
- . $FWDIR/conf/spark-env.sh
-fi
-
-# Figure out how much memory to use per executor and set it as an environment
-# variable so that our process sees it and can report it to Mesos
-if [ -z "$SPARK_MEM" ] ; then
- SPARK_MEM="512m"
-fi
-export SPARK_MEM
-
-# Set JAVA_OPTS to be able to load native libraries and to set heap size
-JAVA_OPTS="$SPARK_JAVA_OPTS"
-JAVA_OPTS+=" -Djava.library.path=$SPARK_LIBRARY_PATH"
-JAVA_OPTS+=" -Xms$SPARK_MEM -Xmx$SPARK_MEM"
-# Load extra JAVA_OPTS from conf/java-opts, if it exists
-if [ -e $FWDIR/conf/java-opts ] ; then
- JAVA_OPTS+=" `cat $FWDIR/conf/java-opts`"
-fi
-export JAVA_OPTS
-
-# Build up classpath
-CLASSPATH=":$FWDIR/conf"
-for jar in `find $FWDIR -name '*jar'`; do
- CLASSPATH+=":$jar"
-done
-export CLASSPATH
-
-exec java -Dscala.usejavacp=true -Djline.shutdownhook=true -cp "$CLASSPATH" $JAVA_OPTS $EXTRA_ARGS "$@"
diff --git a/repl-bin/src/deb/bin/spark-executor b/repl-bin/src/deb/bin/spark-executor
deleted file mode 100755
index 052d76fb8d..0000000000
--- a/repl-bin/src/deb/bin/spark-executor
+++ /dev/null
@@ -1,22 +0,0 @@
-#!/usr/bin/env bash
-
-#
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements. See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-FWDIR="$(cd `dirname $0`; pwd)"
-echo "Running spark-executor with framework dir = $FWDIR"
-exec $FWDIR/run org.apache.spark.executor.MesosExecutorBackend
diff --git a/repl-bin/src/deb/bin/spark-shell b/repl-bin/src/deb/bin/spark-shell
deleted file mode 100755
index 118349d7c3..0000000000
--- a/repl-bin/src/deb/bin/spark-shell
+++ /dev/null
@@ -1,21 +0,0 @@
-#!/usr/bin/env bash
-
-#
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements. See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-FWDIR="$(cd `dirname $0`; pwd)"
-exec $FWDIR/run org.apache.spark.repl.Main "$@"
diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala
index 8aad273665..8203b8f612 100644
--- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala
+++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.repl
import java.io._
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
index 5046a1d53f..4d778dc4d4 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
@@ -42,11 +42,13 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time)
val checkpointDuration = ssc.checkpointDuration
val pendingTimes = ssc.scheduler.getPendingTimes().toArray
val delaySeconds = MetadataCleaner.getDelaySeconds(ssc.conf)
- val sparkConf = ssc.conf
+ val sparkConfPairs = ssc.conf.getAll
- // These should be unset when a checkpoint is deserialized,
- // otherwise the SparkContext won't initialize correctly.
- sparkConf.remove("spark.driver.host").remove("spark.driver.port")
+ def sparkConf = {
+ new SparkConf(false).setAll(sparkConfPairs)
+ .remove("spark.driver.host")
+ .remove("spark.driver.port")
+ }
def validate() {
assert(master != null, "Checkpoint.master is null")
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ContextWaiter.scala b/streaming/src/main/scala/org/apache/spark/streaming/ContextWaiter.scala
index 1f5dacb543..86753360a0 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/ContextWaiter.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/ContextWaiter.scala
@@ -1,3 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
package org.apache.spark.streaming
private[streaming] class ContextWaiter {
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala
index 8faa79f8c7..0683113bd0 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala
@@ -163,8 +163,10 @@ final private[streaming] class DStreamGraph extends Serializable with Logging {
logDebug("DStreamGraph.writeObject used")
this.synchronized {
checkpointInProgress = true
+ logDebug("Enabled checkpoint mode")
oos.defaultWriteObject()
checkpointInProgress = false
+ logDebug("Disabled checkpoint mode")
}
}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
index 7b27933403..26257e652e 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
@@ -199,10 +199,7 @@ class StreamingContext private[streaming] (
*/
def networkStream[T: ClassTag](
receiver: NetworkReceiver[T]): DStream[T] = {
- val inputStream = new PluggableInputDStream[T](this,
- receiver)
- graph.addInputStream(inputStream)
- inputStream
+ new PluggableInputDStream[T](this, receiver)
}
/**
@@ -259,9 +256,7 @@ class StreamingContext private[streaming] (
converter: (InputStream) => Iterator[T],
storageLevel: StorageLevel
): DStream[T] = {
- val inputStream = new SocketInputDStream[T](this, hostname, port, converter, storageLevel)
- registerInputStream(inputStream)
- inputStream
+ new SocketInputDStream[T](this, hostname, port, converter, storageLevel)
}
/**
@@ -280,9 +275,7 @@ class StreamingContext private[streaming] (
port: Int,
storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
): DStream[T] = {
- val inputStream = new RawInputDStream[T](this, hostname, port, storageLevel)
- registerInputStream(inputStream)
- inputStream
+ new RawInputDStream[T](this, hostname, port, storageLevel)
}
/**
@@ -300,9 +293,7 @@ class StreamingContext private[streaming] (
V: ClassTag,
F <: NewInputFormat[K, V]: ClassTag
] (directory: String): DStream[(K, V)] = {
- val inputStream = new FileInputDStream[K, V, F](this, directory)
- registerInputStream(inputStream)
- inputStream
+ new FileInputDStream[K, V, F](this, directory)
}
/**
@@ -322,9 +313,7 @@ class StreamingContext private[streaming] (
V: ClassTag,
F <: NewInputFormat[K, V]: ClassTag
] (directory: String, filter: Path => Boolean, newFilesOnly: Boolean): DStream[(K, V)] = {
- val inputStream = new FileInputDStream[K, V, F](this, directory, filter, newFilesOnly)
- registerInputStream(inputStream)
- inputStream
+ new FileInputDStream[K, V, F](this, directory, filter, newFilesOnly)
}
/**
@@ -367,9 +356,7 @@ class StreamingContext private[streaming] (
oneAtATime: Boolean,
defaultRDD: RDD[T]
): DStream[T] = {
- val inputStream = new QueueInputDStream(this, queue, oneAtATime, defaultRDD)
- registerInputStream(inputStream)
- inputStream
+ new QueueInputDStream(this, queue, oneAtATime, defaultRDD)
}
/**
@@ -390,21 +377,6 @@ class StreamingContext private[streaming] (
new TransformedDStream[T](dstreams, sparkContext.clean(transformFunc))
}
- /**
- * Register an input stream that will be started (InputDStream.start() called) to get the
- * input data.
- */
- def registerInputStream(inputStream: InputDStream[_]) {
- graph.addInputStream(inputStream)
- }
-
- /**
- * Register an output stream that will be computed every interval
- */
- def registerOutputStream(outputStream: DStream[_]) {
- graph.addOutputStream(outputStream)
- }
-
/** Add a [[org.apache.spark.streaming.scheduler.StreamingListener]] object for
* receiving system events related to streaming.
*/
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
index 108950466a..613683ca40 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
@@ -141,8 +141,12 @@ class JavaStreamingContext(val ssc: StreamingContext) {
*/
def this(path: String, hadoopConf: Configuration) = this(new StreamingContext(path, hadoopConf))
+
+ @deprecated("use sparkContext", "0.9.0")
+ val sc: JavaSparkContext = sparkContext
+
/** The underlying SparkContext */
- val sc: JavaSparkContext = new JavaSparkContext(ssc.sc)
+ val sparkContext = new JavaSparkContext(ssc.sc)
/**
* Create a input stream from network source hostname:port. Data is received using
@@ -320,13 +324,6 @@ class JavaStreamingContext(val ssc: StreamingContext) {
}
/**
- * Registers an output stream that will be computed every interval
- */
- def registerOutputStream(outputStream: JavaDStreamLike[_, _, _]) {
- ssc.registerOutputStream(outputStream.dstream)
- }
-
- /**
* Creates a input stream from an queue of RDDs. In each batch,
* it will process either one or all of the RDDs returned by the queue.
*
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala
index 426f61e24a..71a4c5c93e 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala
@@ -36,11 +36,12 @@ import org.apache.spark.streaming.Duration
/**
* A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous
* sequence of RDDs (of the same type) representing a continuous stream of data (see
- * [[org.apache.spark.rdd.RDD]] for more details on RDDs). DStreams can either be created from
- * live data (such as, data from * HDFS, Kafka or Flume) or it can be generated by transformation
- * existing DStreams using operations such as `map`, `window` and `reduceByKeyAndWindow`.
- * While a Spark Streaming program is running, each DStream periodically generates a RDD,
- * either from live data or by transforming the RDD generated by a parent DStream.
+ * org.apache.spark.rdd.RDD in the Spark core documentation for more details on RDDs).
+ * DStreams can either be created from live data (such as, data from Kafka, Flume, sockets, HDFS)
+ * or it can be generated by transforming existing DStreams using operations such as `map`,
+ * `window` and `reduceByKeyAndWindow`. While a Spark Streaming program is running, each DStream
+ * periodically generates a RDD, either from live data or by transforming the RDD generated by a
+ * parent DStream.
*
* This class contains the basic operations available on all DStreams, such as `map`, `filter` and
* `window`. In addition, [[org.apache.spark.streaming.dstream.PairDStreamFunctions]] contains
@@ -523,7 +524,7 @@ abstract class DStream[T: ClassTag] (
* 'this' DStream will be registered as an output stream and therefore materialized.
*/
def foreachRDD(foreachFunc: (RDD[T], Time) => Unit) {
- ssc.registerOutputStream(new ForEachDStream(this, context.sparkContext.clean(foreachFunc)))
+ new ForEachDStream(this, context.sparkContext.clean(foreachFunc)).register()
}
/**
@@ -590,8 +591,7 @@ abstract class DStream[T: ClassTag] (
if (first11.size > 10) println("...")
println()
}
- val newStream = new ForEachDStream(this, context.sparkContext.clean(foreachFunc))
- ssc.registerOutputStream(newStream)
+ new ForEachDStream(this, context.sparkContext.clean(foreachFunc)).register()
}
/**
@@ -768,8 +768,8 @@ abstract class DStream[T: ClassTag] (
* Register this streaming as an output stream. This would ensure that RDDs of this
* DStream will be generated.
*/
- def register(): DStream[T] = {
- ssc.registerOutputStream(this)
+ private[streaming] def register(): DStream[T] = {
+ ssc.graph.addOutputStream(this)
this
}
}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala
index 38bad5ac80..906a16e508 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala
@@ -19,7 +19,7 @@ package org.apache.spark.streaming.dstream
import scala.collection.mutable.HashMap
import scala.reflect.ClassTag
-import java.io.{ObjectInputStream, IOException}
+import java.io.{ObjectOutputStream, ObjectInputStream, IOException}
import org.apache.hadoop.fs.Path
import org.apache.hadoop.fs.FileSystem
import org.apache.spark.Logging
@@ -118,7 +118,31 @@ class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T])
}
@throws(classOf[IOException])
+ private def writeObject(oos: ObjectOutputStream) {
+ logDebug(this.getClass().getSimpleName + ".writeObject used")
+ if (dstream.context.graph != null) {
+ dstream.context.graph.synchronized {
+ if (dstream.context.graph.checkpointInProgress) {
+ oos.defaultWriteObject()
+ } else {
+ val msg = "Object of " + this.getClass.getName + " is being serialized " +
+ " possibly as a part of closure of an RDD operation. This is because " +
+ " the DStream object is being referred to from within the closure. " +
+ " Please rewrite the RDD operation inside this DStream to avoid this. " +
+ " This has been enforced to avoid bloating of Spark tasks " +
+ " with unnecessary objects."
+ throw new java.io.NotSerializableException(msg)
+ }
+ }
+ } else {
+ throw new java.io.NotSerializableException(
+ "Graph is unexpectedly null when DStream is being serialized.")
+ }
+ }
+
+ @throws(classOf[IOException])
private def readObject(ois: ObjectInputStream) {
+ logDebug(this.getClass().getSimpleName + ".readObject used")
ois.defaultReadObject()
timeToOldestCheckpointFileTime = new HashMap[Time, Time]
timeToCheckpointFile = new HashMap[Time, String]
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala
index a1075ad304..27303390d9 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala
@@ -22,20 +22,24 @@ import org.apache.spark.streaming.{Time, Duration, StreamingContext}
import scala.reflect.ClassTag
/**
- * This is the abstract base class for all input streams. This class provides to methods
- * start() and stop() which called by the scheduler to start and stop receiving data/
- * Input streams that can generated RDDs from new data just by running a service on
- * the driver node (that is, without running a receiver onworker nodes) can be
- * implemented by directly subclassing this InputDStream. For example,
- * FileInputDStream, a subclass of InputDStream, monitors a HDFS directory for
- * new files and generates RDDs on the new files. For implementing input streams
- * that requires running a receiver on the worker nodes, use NetworkInputDStream
- * as the parent class.
+ * This is the abstract base class for all input streams. This class provides methods
+ * start() and stop() which is called by Spark Streaming system to start and stop receiving data.
+ * Input streams that can generate RDDs from new data by running a service/thread only on
+ * the driver node (that is, without running a receiver on worker nodes), can be
+ * implemented by directly inheriting this InputDStream. For example,
+ * FileInputDStream, a subclass of InputDStream, monitors a HDFS directory from the driver for
+ * new files and generates RDDs with the new files. For implementing input streams
+ * that requires running a receiver on the worker nodes, use
+ * [[org.apache.spark.streaming.dstream.NetworkInputDStream]] as the parent class.
+ *
+ * @param ssc_ Streaming context that will execute this input stream
*/
abstract class InputDStream[T: ClassTag] (@transient ssc_ : StreamingContext)
extends DStream[T](ssc_) {
- var lastValidTime: Time = null
+ private[streaming] var lastValidTime: Time = null
+
+ ssc.graph.addInputStream(this)
/**
* Checks whether the 'time' is valid wrt slideDuration for generating RDD.
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala
index 0f1f6fc2ce..ce153f065d 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala
@@ -36,11 +36,12 @@ import org.apache.spark.storage.{BlockId, StorageLevel, StreamBlockId}
import org.apache.spark.streaming.scheduler.{DeregisterReceiver, AddBlocks, RegisterReceiver}
/**
- * Abstract class for defining any InputDStream that has to start a receiver on worker
- * nodes to receive external data. Specific implementations of NetworkInputDStream must
+ * Abstract class for defining any [[org.apache.spark.streaming.dstream.InputDStream]]
+ * that has to start a receiver on worker nodes to receive external data.
+ * Specific implementations of NetworkInputDStream must
* define the getReceiver() function that gets the receiver object of type
- * [[org.apache.spark.streaming.dstream.NetworkReceiver]] that will be sent to the workers to receive
- * data.
+ * [[org.apache.spark.streaming.dstream.NetworkReceiver]] that will be sent
+ * to the workers to receive data.
* @param ssc_ Streaming context that will execute this input stream
* @tparam T Class type of the object of this stream
*/
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala
index 6b3e48382e..f57762321c 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala
@@ -35,6 +35,11 @@ import org.apache.hadoop.security.UserGroupInformation
import org.apache.hadoop.conf.Configuration
import org.apache.spark.streaming.{Time, Duration}
+/**
+ * Extra functions available on DStream of (key, value) pairs through an implicit conversion.
+ * Import `org.apache.spark.streaming.StreamingContext._` at the top of your program to use
+ * these functions.
+ */
class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)])
extends Serializable {
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/package.scala b/streaming/src/main/scala/org/apache/spark/streaming/package.scala
new file mode 100644
index 0000000000..4dd985cf5a
--- /dev/null
+++ b/streaming/src/main/scala/org/apache/spark/streaming/package.scala
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark
+
+/**
+ * Spark Streaming functionality. [[org.apache.spark.streaming.StreamingContext]] serves as the main
+ * entry point to Spark Streaming, while [[org.apache.spark.streaming.dstream.DStream]] is the data
+ * type representing a continuous sequence of RDDs, representing a continuous stream of data.
+ *
+ * In addition, [[org.apache.spark.streaming.dstream.PairDStreamFunctions]] contains operations
+ * available only on DStreams
+ * of key-value pairs, such as `groupByKey` and `reduceByKey`. These operations are automatically
+ * available on any DStream of the right type (e.g. DStream[(Int, Int)] through implicit
+ * conversions when you `import org.apache.spark.streaming.StreamingContext._`.
+ *
+ * For the Java API of Spark Streaming, take a look at the
+ * [[org.apache.spark.streaming.api.java.JavaStreamingContext]] which serves as the entry point, and
+ * the [[org.apache.spark.streaming.api.java.JavaDStream]] and the
+ * [[org.apache.spark.streaming.api.java.JavaPairDStream]] which have the DStream functionality.
+ */
+package object streaming {
+ // For package docs only
+}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala
index be67af3a64..54813934b8 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala
@@ -191,7 +191,7 @@ object MasterFailureTest extends Logging {
val inputStream = ssc.textFileStream(testDir.toString)
val operatedStream = operation(inputStream)
val outputStream = new TestOutputStream(operatedStream)
- ssc.registerOutputStream(outputStream)
+ outputStream.register()
ssc
}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RateLimitedOutputStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RateLimitedOutputStream.scala
index b9c0596378..179fd75939 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/util/RateLimitedOutputStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RateLimitedOutputStream.scala
@@ -22,6 +22,7 @@ import scala.annotation.tailrec
import java.io.OutputStream
import java.util.concurrent.TimeUnit._
+private[streaming]
class RateLimitedOutputStream(out: OutputStream, bytesPerSec: Int) extends OutputStream {
val SYNC_INTERVAL = NANOSECONDS.convert(10, SECONDS)
val CHUNK_SIZE = 8192
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala
index 5b6c048a39..07021ebb58 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala
@@ -22,6 +22,7 @@ import org.apache.spark.SparkContext._
import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap}
import scala.collection.JavaConversions.mapAsScalaMap
+private[streaming]
object RawTextHelper {
/**
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala
index 463617a713..684b38e8b3 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala
@@ -33,6 +33,7 @@ import org.apache.spark.util.IntParam
* A helper program that sends blocks of Kryo-serialized text strings out on a socket at a
* specified rate. Used to feed data into RawInputDStream.
*/
+private[streaming]
object RawTextSender extends Logging {
def main(args: Array[String]) {
if (args.length != 4) {
diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
index 8b7d7709bf..4fbbce9b8b 100644
--- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
+++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
@@ -297,9 +297,9 @@ public class JavaAPISuite extends LocalJavaStreamingContext implements Serializa
Arrays.asList(7,8,9));
JavaSparkContext jsc = new JavaSparkContext(ssc.ssc().sc());
- JavaRDD<Integer> rdd1 = ssc.sc().parallelize(Arrays.asList(1, 2, 3));
- JavaRDD<Integer> rdd2 = ssc.sc().parallelize(Arrays.asList(4, 5, 6));
- JavaRDD<Integer> rdd3 = ssc.sc().parallelize(Arrays.asList(7,8,9));
+ JavaRDD<Integer> rdd1 = ssc.sparkContext().parallelize(Arrays.asList(1, 2, 3));
+ JavaRDD<Integer> rdd2 = ssc.sparkContext().parallelize(Arrays.asList(4, 5, 6));
+ JavaRDD<Integer> rdd3 = ssc.sparkContext().parallelize(Arrays.asList(7,8,9));
LinkedList<JavaRDD<Integer>> rdds = Lists.newLinkedList();
rdds.add(rdd1);
diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala b/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala
index 42ab9590d6..33f6df8f88 100644
--- a/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala
+++ b/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala
@@ -43,7 +43,6 @@ trait JavaTestBase extends TestSuiteBase {
implicit val cm: ClassTag[T] =
implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
val dstream = new TestInputStream[T](ssc.ssc, seqData, numPartitions)
- ssc.ssc.registerInputStream(dstream)
new JavaDStream[T](dstream)
}
@@ -57,7 +56,7 @@ trait JavaTestBase extends TestSuiteBase {
implicit val cm: ClassTag[T] =
implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]]
val ostream = new TestOutputStreamWithPartitions(dstream.dstream)
- dstream.dstream.ssc.registerOutputStream(ostream)
+ ostream.register()
}
/**
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
index b73edf81d4..bcb0c28bf0 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
@@ -381,7 +381,6 @@ class BasicOperationsSuite extends TestSuiteBase {
val ssc = new StreamingContext(conf, Seconds(1))
val input = Seq(Seq(1), Seq(2), Seq(3), Seq(4))
val stream = new TestInputStream[Int](ssc, input, 2)
- ssc.registerInputStream(stream)
stream.foreachRDD(_ => {}) // Dummy output stream
ssc.start()
Thread.sleep(2000)
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
index 0c68c44ddb..831e7c1471 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
@@ -151,17 +151,29 @@ class CheckpointSuite extends TestSuiteBase {
val value = "myvalue"
System.setProperty(key, value)
ssc = new StreamingContext(master, framework, batchDuration)
+ val originalConf = ssc.conf
+
val cp = new Checkpoint(ssc, Time(1000))
- assert(!cp.sparkConf.contains("spark.driver.host"))
- assert(!cp.sparkConf.contains("spark.driver.port"))
- assert(!cp.sparkConf.contains("spark.hostPort"))
- assert(cp.sparkConf.get(key) === value)
+ val cpConf = cp.sparkConf
+ assert(cpConf.get("spark.master") === originalConf.get("spark.master"))
+ assert(cpConf.get("spark.app.name") === originalConf.get("spark.app.name"))
+ assert(cpConf.get(key) === value)
ssc.stop()
+
+ // Serialize/deserialize to simulate write to storage and reading it back
val newCp = Utils.deserialize[Checkpoint](Utils.serialize(cp))
- assert(!newCp.sparkConf.contains("spark.driver.host"))
- assert(!newCp.sparkConf.contains("spark.driver.port"))
- assert(!newCp.sparkConf.contains("spark.hostPort"))
- assert(newCp.sparkConf.get(key) === value)
+
+ val newCpConf = newCp.sparkConf
+ assert(newCpConf.get("spark.master") === originalConf.get("spark.master"))
+ assert(newCpConf.get("spark.app.name") === originalConf.get("spark.app.name"))
+ assert(newCpConf.get(key) === value)
+ assert(!newCpConf.contains("spark.driver.host"))
+ assert(!newCpConf.contains("spark.driver.port"))
+
+ // Check if all the parameters have been restored
+ ssc = new StreamingContext(null, newCp, null)
+ val restoredConf = ssc.conf
+ assert(restoredConf.get(key) === value)
}
@@ -237,7 +249,7 @@ class CheckpointSuite extends TestSuiteBase {
val reducedStream = mappedStream.reduceByWindow(_ + _, Seconds(30), Seconds(1))
val outputBuffer = new ArrayBuffer[Seq[Int]]
var outputStream = new TestOutputStream(reducedStream, outputBuffer)
- ssc.registerOutputStream(outputStream)
+ outputStream.register()
ssc.start()
// Create files and advance manual clock to process them
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
index a8e053278a..95bf40ba75 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
@@ -54,7 +54,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]]
val outputStream = new TestOutputStream(networkStream, outputBuffer)
def output = outputBuffer.flatMap(x => x)
- ssc.registerOutputStream(outputStream)
+ outputStream.register()
ssc.start()
// Feed data to the server to send to the network receiver
@@ -103,7 +103,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]]
def output = outputBuffer.flatMap(x => x)
val outputStream = new TestOutputStream(fileStream, outputBuffer)
- ssc.registerOutputStream(outputStream)
+ outputStream.register()
ssc.start()
// Create files in the temporary directory so that Spark Streaming can read data from it
@@ -156,7 +156,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]]
val outputStream = new TestOutputStream(networkStream, outputBuffer)
def output = outputBuffer.flatMap(x => x)
- ssc.registerOutputStream(outputStream)
+ outputStream.register()
ssc.start()
// Feed data to the server to send to the network receiver
@@ -209,7 +209,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
val outputBuffer = new ArrayBuffer[Seq[Long]] with SynchronizedBuffer[Seq[Long]]
val outputStream = new TestOutputStream(countStream, outputBuffer)
def output = outputBuffer.flatMap(x => x)
- ssc.registerOutputStream(outputStream)
+ outputStream.register()
ssc.start()
// Let the data from the receiver be received
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala
index f7f3346f81..717da8e004 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala
@@ -211,7 +211,6 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts {
def addInputStream(s: StreamingContext): DStream[Int] = {
val input = (1 to 100).map(i => (1 to i))
val inputStream = new TestInputStream(s, input, 1)
- s.registerInputStream(inputStream)
inputStream
}
}
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
index 535e5bd1f1..201630672a 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
@@ -181,8 +181,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
val operatedStream = operation(inputStream)
val outputStream = new TestOutputStreamWithPartitions(operatedStream,
new ArrayBuffer[Seq[Seq[V]]] with SynchronizedBuffer[Seq[Seq[V]]])
- ssc.registerInputStream(inputStream)
- ssc.registerOutputStream(outputStream)
+ outputStream.register()
ssc
}
@@ -207,9 +206,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging {
val operatedStream = operation(inputStream1, inputStream2)
val outputStream = new TestOutputStreamWithPartitions(operatedStream,
new ArrayBuffer[Seq[Seq[W]]] with SynchronizedBuffer[Seq[Seq[W]]])
- ssc.registerInputStream(inputStream1)
- ssc.registerInputStream(inputStream2)
- ssc.registerOutputStream(outputStream)
+ outputStream.register()
ssc
}
diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index e56bc02897..71a64ecf58 100644
--- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -17,19 +17,10 @@
package org.apache.spark.deploy.yarn
-import java.net.{InetAddress, UnknownHostException, URI}
import java.nio.ByteBuffer
-import scala.collection.JavaConversions._
-import scala.collection.mutable.HashMap
-import scala.collection.mutable.Map
-
import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.{FileContext, FileStatus, FileSystem, Path, FileUtil}
-import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.io.DataOutputBuffer
-import org.apache.hadoop.mapred.Master
-import org.apache.hadoop.net.NetUtils
import org.apache.hadoop.security.UserGroupInformation
import org.apache.hadoop.yarn.api._
import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
@@ -41,29 +32,22 @@ import org.apache.hadoop.yarn.ipc.YarnRPC
import org.apache.hadoop.yarn.util.{Apps, Records}
import org.apache.spark.{Logging, SparkConf}
-import org.apache.spark.util.Utils
-import org.apache.spark.deploy.SparkHadoopUtil
-class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf)
- extends YarnClientImpl with Logging {
+class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: SparkConf)
+ extends YarnClientImpl with ClientBase with Logging {
- def this(args: ClientArguments, sparkConf: SparkConf) =
- this(args, new Configuration(), sparkConf)
+ def this(clientArgs: ClientArguments, spConf: SparkConf) =
+ this(clientArgs, new Configuration(), spConf)
- def this(args: ClientArguments) = this(args, new SparkConf())
+ def this(clientArgs: ClientArguments) = this(clientArgs, new SparkConf())
+ val args = clientArgs
+ val conf = hadoopConf
+ val sparkConf = spConf
var rpc: YarnRPC = YarnRPC.create(conf)
val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
- val credentials = UserGroupInformation.getCurrentUser().getCredentials()
- private val SPARK_STAGING: String = ".sparkStaging"
- private val distCacheMgr = new ClientDistributedCacheManager()
-
- // Staging directory is private! -> rwx--------
- val STAGING_DIR_PERMISSION: FsPermission = FsPermission.createImmutable(0700:Short)
- // App files are world-wide readable and owner writable -> rw-r--r--
- val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644:Short)
// for client user who want to monitor app status by itself.
def runApp() = {
@@ -83,6 +67,11 @@ class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf)
val env = setupLaunchEnv(localResources, appStagingDir)
val amContainer = createContainerLaunchContext(newApp, localResources, env)
+ val capability = Records.newRecord(classOf[Resource]).asInstanceOf[Resource]
+ // Memory for the ApplicationMaster.
+ capability.setMemory(args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD)
+ amContainer.setResource(capability)
+
appContext.setQueue(args.amQueue)
appContext.setAMContainerSpec(amContainer)
appContext.setUser(UserGroupInformation.getCurrentUser().getShortUserName())
@@ -97,28 +86,6 @@ class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf)
System.exit(0)
}
- def validateArgs() = {
- Map(
- (System.getenv("SPARK_JAR") == null) -> "Error: You must set SPARK_JAR environment variable!",
- (args.userJar == null) -> "Error: You must specify a user jar!",
- (args.userClass == null) -> "Error: You must specify a user class!",
- (args.numWorkers <= 0) -> "Error: You must specify at least 1 worker!",
- (args.amMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: AM memory size must be " +
- "greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD),
- (args.workerMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: Worker memory size " +
- "must be greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD)
- ).foreach { case(cond, errStr) =>
- if (cond) {
- logError(errStr)
- args.printUsageAndExit(1)
- }
- }
- }
-
- def getAppStagingDir(appId: ApplicationId): String = {
- SPARK_STAGING + Path.SEPARATOR + appId.toString() + Path.SEPARATOR
- }
-
def logClusterResourceDetails() {
val clusterMetrics: YarnClusterMetrics = super.getYarnClusterMetrics
logInfo("Got Cluster metric info from ASM, numNodeManagers = " +
@@ -134,24 +101,6 @@ class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf)
queueInfo.getChildQueues.size))
}
- def verifyClusterResources(app: GetNewApplicationResponse) = {
- val maxMem = app.getMaximumResourceCapability().getMemory()
- logInfo("Max mem capabililty of a single resource in this cluster " + maxMem)
-
- // If we have requested more then the clusters max for a single resource then exit.
- if (args.workerMemory > maxMem) {
- logError("the worker size is to large to run on this cluster " + args.workerMemory)
- System.exit(1)
- }
- val amMem = args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD
- if (amMem > maxMem) {
- logError("AM size is to large to run on this cluster " + amMem)
- System.exit(1)
- }
-
- // We could add checks to make sure the entire cluster has enough resources but that involves
- // getting all the node reports and computing ourselves
- }
def createApplicationSubmissionContext(appId: ApplicationId): ApplicationSubmissionContext = {
logInfo("Setting up application submission context for ASM")
@@ -161,263 +110,19 @@ class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf)
appContext
}
- /** See if two file systems are the same or not. */
- private def compareFs(srcFs: FileSystem, destFs: FileSystem): Boolean = {
- val srcUri = srcFs.getUri()
- val dstUri = destFs.getUri()
- if (srcUri.getScheme() == null) {
- return false
- }
- if (!srcUri.getScheme().equals(dstUri.getScheme())) {
- return false
- }
- var srcHost = srcUri.getHost()
- var dstHost = dstUri.getHost()
- if ((srcHost != null) && (dstHost != null)) {
- try {
- srcHost = InetAddress.getByName(srcHost).getCanonicalHostName()
- dstHost = InetAddress.getByName(dstHost).getCanonicalHostName()
- } catch {
- case e: UnknownHostException =>
- return false
- }
- if (!srcHost.equals(dstHost)) {
- return false
- }
- } else if (srcHost == null && dstHost != null) {
- return false
- } else if (srcHost != null && dstHost == null) {
- return false
- }
- //check for ports
- if (srcUri.getPort() != dstUri.getPort()) {
- return false
- }
-
- true
- }
-
- /** Copy the file into HDFS if needed. */
- private def copyRemoteFile(
- dstDir: Path,
- originalPath: Path,
- replication: Short,
- setPerms: Boolean = false): Path = {
- val fs = FileSystem.get(conf)
- val remoteFs = originalPath.getFileSystem(conf)
- var newPath = originalPath
- if (! compareFs(remoteFs, fs)) {
- newPath = new Path(dstDir, originalPath.getName())
- logInfo("Uploading " + originalPath + " to " + newPath)
- FileUtil.copy(remoteFs, originalPath, fs, newPath, false, conf)
- fs.setReplication(newPath, replication)
- if (setPerms) fs.setPermission(newPath, new FsPermission(APP_FILE_PERMISSION))
- }
- // Resolve any symlinks in the URI path so using a "current" symlink to point to a specific
- // version shows the specific version in the distributed cache configuration
- val qualPath = fs.makeQualified(newPath)
- val fc = FileContext.getFileContext(qualPath.toUri(), conf)
- val destPath = fc.resolvePath(qualPath)
- destPath
+ def calculateAMMemory(newApp: GetNewApplicationResponse): Int = {
+ val minResMemory = newApp.getMinimumResourceCapability().getMemory()
+ val amMemory = ((args.amMemory / minResMemory) * minResMemory) +
+ ((if ((args.amMemory % minResMemory) == 0) 0 else minResMemory) -
+ YarnAllocationHandler.MEMORY_OVERHEAD)
+ amMemory
}
- def prepareLocalResources(appStagingDir: String): HashMap[String, LocalResource] = {
- logInfo("Preparing Local resources")
- // Upload Spark and the application JAR to the remote file system if necessary. Add them as
- // local resources to the AM.
- val fs = FileSystem.get(conf)
-
- val delegTokenRenewer = Master.getMasterPrincipal(conf)
- if (UserGroupInformation.isSecurityEnabled()) {
- if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) {
- logError("Can't get Master Kerberos principal for use as renewer")
- System.exit(1)
- }
- }
- val dst = new Path(fs.getHomeDirectory(), appStagingDir)
- val replication = sparkConf.getInt("spark.yarn.submit.file.replication", 3).toShort
-
- if (UserGroupInformation.isSecurityEnabled()) {
- val dstFs = dst.getFileSystem(conf)
- dstFs.addDelegationTokens(delegTokenRenewer, credentials)
- }
- val localResources = HashMap[String, LocalResource]()
- FileSystem.mkdirs(fs, dst, new FsPermission(STAGING_DIR_PERMISSION))
-
- val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
-
- Map(Client.SPARK_JAR -> System.getenv("SPARK_JAR"), Client.APP_JAR -> args.userJar,
- Client.LOG4J_PROP -> System.getenv("SPARK_LOG4J_CONF"))
- .foreach { case(destName, _localPath) =>
- val localPath: String = if (_localPath != null) _localPath.trim() else ""
- if (! localPath.isEmpty()) {
- var localURI = new URI(localPath)
- // if not specified assume these are in the local filesystem to keep behavior like Hadoop
- if (localURI.getScheme() == null) {
- localURI = new URI(FileSystem.getLocal(conf).makeQualified(new Path(localPath)).toString)
- }
- val setPermissions = if (destName.equals(Client.APP_JAR)) true else false
- val destPath = copyRemoteFile(dst, new Path(localURI), replication, setPermissions)
- distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
- destName, statCache)
- }
- }
-
- // handle any add jars
- if ((args.addJars != null) && (!args.addJars.isEmpty())){
- args.addJars.split(',').foreach { case file: String =>
- val localURI = new URI(file.trim())
- val localPath = new Path(localURI)
- val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
- val destPath = copyRemoteFile(dst, localPath, replication)
- distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
- linkname, statCache, true)
- }
- }
-
- // handle any distributed cache files
- if ((args.files != null) && (!args.files.isEmpty())){
- args.files.split(',').foreach { case file: String =>
- val localURI = new URI(file.trim())
- val localPath = new Path(localURI)
- val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
- val destPath = copyRemoteFile(dst, localPath, replication)
- distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
- linkname, statCache)
- }
- }
-
- // handle any distributed cache archives
- if ((args.archives != null) && (!args.archives.isEmpty())) {
- args.archives.split(',').foreach { case file:String =>
- val localURI = new URI(file.trim())
- val localPath = new Path(localURI)
- val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
- val destPath = copyRemoteFile(dst, localPath, replication)
- distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE,
- linkname, statCache)
- }
- }
-
- UserGroupInformation.getCurrentUser().addCredentials(credentials)
- localResources
- }
-
- def setupLaunchEnv(
- localResources: HashMap[String, LocalResource],
- stagingDir: String): HashMap[String, String] = {
- logInfo("Setting up the launch environment")
- val log4jConfLocalRes = localResources.getOrElse(Client.LOG4J_PROP, null)
-
- val env = new HashMap[String, String]()
-
- Client.populateClasspath(yarnConf, sparkConf, log4jConfLocalRes != null, env)
- env("SPARK_YARN_MODE") = "true"
- env("SPARK_YARN_STAGING_DIR") = stagingDir
-
- // Set the environment variables to be passed on to the Workers.
- distCacheMgr.setDistFilesEnv(env)
- distCacheMgr.setDistArchivesEnv(env)
-
- // Allow users to specify some environment variables.
- Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"))
-
- // Add each SPARK-* key to the environment.
- System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v }
- env
- }
-
- def userArgsToString(clientArgs: ClientArguments): String = {
- val prefix = " --args "
- val args = clientArgs.userArgs
- val retval = new StringBuilder()
- for (arg <- args) {
- retval.append(prefix).append(" '").append(arg).append("' ")
- }
- retval.toString
- }
-
- def createContainerLaunchContext(
- newApp: GetNewApplicationResponse,
- localResources: HashMap[String, LocalResource],
- env: HashMap[String, String]): ContainerLaunchContext = {
- logInfo("Setting up container launch context")
- val amContainer = Records.newRecord(classOf[ContainerLaunchContext])
- amContainer.setLocalResources(localResources)
- amContainer.setEnvironment(env)
-
- val minResMemory: Int = newApp.getMinimumResourceCapability().getMemory()
-
- // TODO(harvey): This can probably be a val.
- var amMemory = ((args.amMemory / minResMemory) * minResMemory) +
- ((if ((args.amMemory % minResMemory) == 0) 0 else minResMemory) -
- YarnAllocationHandler.MEMORY_OVERHEAD)
-
- // Extra options for the JVM
- var JAVA_OPTS = ""
-
- // Add Xmx for am memory
- JAVA_OPTS += "-Xmx" + amMemory + "m "
-
- JAVA_OPTS += " -Djava.io.tmpdir=" +
- new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + " "
-
- // Commenting it out for now - so that people can refer to the properties if required. Remove
- // it once cpuset version is pushed out. The context is, default gc for server class machines
- // end up using all cores to do gc - hence if there are multiple containers in same node,
- // spark gc effects all other containers performance (which can also be other spark containers)
- // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in
- // multi-tenant environments. Not sure how default java gc behaves if it is limited to subset
- // of cores on a node.
- val useConcurrentAndIncrementalGC = env.isDefinedAt("SPARK_USE_CONC_INCR_GC") &&
- java.lang.Boolean.parseBoolean(env("SPARK_USE_CONC_INCR_GC"))
- if (useConcurrentAndIncrementalGC) {
- // In our expts, using (default) throughput collector has severe perf ramnifications in
- // multi-tenant machines
- JAVA_OPTS += " -XX:+UseConcMarkSweepGC "
- JAVA_OPTS += " -XX:+CMSIncrementalMode "
- JAVA_OPTS += " -XX:+CMSIncrementalPacing "
- JAVA_OPTS += " -XX:CMSIncrementalDutyCycleMin=0 "
- JAVA_OPTS += " -XX:CMSIncrementalDutyCycle=10 "
- }
-
- if (env.isDefinedAt("SPARK_JAVA_OPTS")) {
- JAVA_OPTS += env("SPARK_JAVA_OPTS") + " "
- }
-
- // Command for the ApplicationMaster
- var javaCommand = "java"
- val javaHome = System.getenv("JAVA_HOME")
- if ((javaHome != null && !javaHome.isEmpty()) || env.isDefinedAt("JAVA_HOME")) {
- javaCommand = Environment.JAVA_HOME.$() + "/bin/java"
- }
-
- val commands = List[String](javaCommand +
- " -server " +
- JAVA_OPTS +
- " " + args.amClass +
- " --class " + args.userClass +
- " --jar " + args.userJar +
- userArgsToString(args) +
- " --worker-memory " + args.workerMemory +
- " --worker-cores " + args.workerCores +
- " --num-workers " + args.numWorkers +
- " 1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout" +
- " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr")
- logInfo("Command for the ApplicationMaster: " + commands(0))
- amContainer.setCommands(commands)
-
- val capability = Records.newRecord(classOf[Resource]).asInstanceOf[Resource]
- // Memory for the ApplicationMaster.
- capability.setMemory(args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD)
- amContainer.setResource(capability)
-
+ def setupSecurityToken(amContainer: ContainerLaunchContext) = {
// Setup security tokens.
val dob = new DataOutputBuffer()
credentials.writeTokenStorageToStream(dob)
amContainer.setContainerTokens(ByteBuffer.wrap(dob.getData()))
-
- amContainer
}
def submitApp(appContext: ApplicationSubmissionContext) = {
@@ -449,7 +154,6 @@ class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf)
)
val state = report.getYarnApplicationState()
- val dsStatus = report.getFinalApplicationStatus()
if (state == YarnApplicationState.FINISHED ||
state == YarnApplicationState.FAILED ||
state == YarnApplicationState.KILLED) {
@@ -461,9 +165,6 @@ class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf)
}
object Client {
- val SPARK_JAR: String = "spark.jar"
- val APP_JAR: String = "app.jar"
- val LOG4J_PROP: String = "log4j.properties"
def main(argStrings: Array[String]) {
// Set an env variable indicating we are running in YARN mode.
@@ -475,36 +176,4 @@ object Client {
new Client(args, sparkConf).run
}
-
- // Based on code from org.apache.hadoop.mapreduce.v2.util.MRApps
- def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String]) {
- for (c <- conf.getStrings(YarnConfiguration.YARN_APPLICATION_CLASSPATH)) {
- Apps.addToEnvironment(env, Environment.CLASSPATH.name, c.trim)
- }
- }
-
- def populateClasspath(conf: Configuration, sparkConf: SparkConf, addLog4j: Boolean, env: HashMap[String, String]) {
- Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$())
- // If log4j present, ensure ours overrides all others
- if (addLog4j) {
- Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
- Path.SEPARATOR + LOG4J_PROP)
- }
- // Normally the users app.jar is last in case conflicts with spark jars
- val userClasspathFirst = sparkConf.get("spark.yarn.user.classpath.first", "false").toBoolean
- if (userClasspathFirst) {
- Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
- Path.SEPARATOR + APP_JAR)
- }
- Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
- Path.SEPARATOR + SPARK_JAR)
- Client.populateHadoopClasspath(conf, env)
-
- if (!userClasspathFirst) {
- Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
- Path.SEPARATOR + APP_JAR)
- }
- Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
- Path.SEPARATOR + "*")
- }
}
diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
index d32cdcc879..8c686e393f 100644
--- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
+++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
@@ -25,12 +25,10 @@ import scala.collection.JavaConversions._
import scala.collection.mutable.HashMap
import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.Path
import org.apache.hadoop.io.DataOutputBuffer
import org.apache.hadoop.net.NetUtils
import org.apache.hadoop.security.UserGroupInformation
import org.apache.hadoop.yarn.api._
-import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
import org.apache.hadoop.yarn.api.records._
import org.apache.hadoop.yarn.api.protocolrecords._
import org.apache.hadoop.yarn.conf.YarnConfiguration
@@ -43,16 +41,17 @@ import org.apache.spark.{SparkConf, Logging}
class WorkerRunnable(
container: Container,
conf: Configuration,
- sparkConf: SparkConf,
+ spConf: SparkConf,
masterAddress: String,
slaveId: String,
hostname: String,
workerMemory: Int,
workerCores: Int)
- extends Runnable with Logging {
+ extends Runnable with WorkerRunnableUtil with Logging {
var rpc: YarnRPC = YarnRPC.create(conf)
var cm: ContainerManager = _
+ val sparkConf = spConf
val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
def run = {
@@ -75,43 +74,6 @@ class WorkerRunnable(
val env = prepareEnvironment
ctx.setEnvironment(env)
- // Extra options for the JVM
- var JAVA_OPTS = ""
- // Set the JVM memory
- val workerMemoryString = workerMemory + "m"
- JAVA_OPTS += "-Xms" + workerMemoryString + " -Xmx" + workerMemoryString + " "
- if (env.isDefinedAt("SPARK_JAVA_OPTS")) {
- JAVA_OPTS += env("SPARK_JAVA_OPTS") + " "
- }
-
- JAVA_OPTS += " -Djava.io.tmpdir=" +
- new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + " "
-
- // Commenting it out for now - so that people can refer to the properties if required. Remove
- // it once cpuset version is pushed out.
- // The context is, default gc for server class machines end up using all cores to do gc - hence
- // if there are multiple containers in same node, spark gc effects all other containers
- // performance (which can also be other spark containers)
- // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in
- // multi-tenant environments. Not sure how default java gc behaves if it is limited to subset
- // of cores on a node.
-/*
- else {
- // If no java_opts specified, default to using -XX:+CMSIncrementalMode
- // It might be possible that other modes/config is being done in SPARK_JAVA_OPTS, so we dont
- // want to mess with it.
- // In our expts, using (default) throughput collector has severe perf ramnifications in
- // multi-tennent machines
- // The options are based on
- // http://www.oracle.com/technetwork/java/gc-tuning-5-138395.html#0.0.0.%20When%20to%20Use%20the%20Concurrent%20Low%20Pause%20Collector|outline
- JAVA_OPTS += " -XX:+UseConcMarkSweepGC "
- JAVA_OPTS += " -XX:+CMSIncrementalMode "
- JAVA_OPTS += " -XX:+CMSIncrementalPacing "
- JAVA_OPTS += " -XX:CMSIncrementalDutyCycleMin=0 "
- JAVA_OPTS += " -XX:CMSIncrementalDutyCycle=10 "
- }
-*/
-
ctx.setUser(UserGroupInformation.getCurrentUser().getShortUserName())
val credentials = UserGroupInformation.getCurrentUser().getCredentials()
@@ -119,28 +81,7 @@ class WorkerRunnable(
credentials.writeTokenStorageToStream(dob)
ctx.setContainerTokens(ByteBuffer.wrap(dob.getData()))
- var javaCommand = "java"
- val javaHome = System.getenv("JAVA_HOME")
- if ((javaHome != null && !javaHome.isEmpty()) || env.isDefinedAt("JAVA_HOME")) {
- javaCommand = Environment.JAVA_HOME.$() + "/bin/java"
- }
-
- val commands = List[String](javaCommand +
- " -server " +
- // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling.
- // Not killing the task leaves various aspects of the worker and (to some extent) the jvm in
- // an inconsistent state.
- // TODO: If the OOM is not recoverable by rescheduling it on different node, then do
- // 'something' to fail job ... akin to blacklisting trackers in mapred ?
- " -XX:OnOutOfMemoryError='kill %p' " +
- JAVA_OPTS +
- " org.apache.spark.executor.CoarseGrainedExecutorBackend " +
- masterAddress + " " +
- slaveId + " " +
- hostname + " " +
- workerCores +
- " 1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout" +
- " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr")
+ val commands = prepareCommand(masterAddress, slaveId, hostname, workerMemory, workerCores)
logInfo("Setting up worker with commands: " + commands)
ctx.setCommands(commands)
@@ -151,65 +92,6 @@ class WorkerRunnable(
cm.startContainer(startReq)
}
- private def setupDistributedCache(
- file: String,
- rtype: LocalResourceType,
- localResources: HashMap[String, LocalResource],
- timestamp: String,
- size: String,
- vis: String) = {
- val uri = new URI(file)
- val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource]
- amJarRsrc.setType(rtype)
- amJarRsrc.setVisibility(LocalResourceVisibility.valueOf(vis))
- amJarRsrc.setResource(ConverterUtils.getYarnUrlFromURI(uri))
- amJarRsrc.setTimestamp(timestamp.toLong)
- amJarRsrc.setSize(size.toLong)
- localResources(uri.getFragment()) = amJarRsrc
- }
-
- def prepareLocalResources: HashMap[String, LocalResource] = {
- logInfo("Preparing Local resources")
- val localResources = HashMap[String, LocalResource]()
-
- if (System.getenv("SPARK_YARN_CACHE_FILES") != null) {
- val timeStamps = System.getenv("SPARK_YARN_CACHE_FILES_TIME_STAMPS").split(',')
- val fileSizes = System.getenv("SPARK_YARN_CACHE_FILES_FILE_SIZES").split(',')
- val distFiles = System.getenv("SPARK_YARN_CACHE_FILES").split(',')
- val visibilities = System.getenv("SPARK_YARN_CACHE_FILES_VISIBILITIES").split(',')
- for( i <- 0 to distFiles.length - 1) {
- setupDistributedCache(distFiles(i), LocalResourceType.FILE, localResources, timeStamps(i),
- fileSizes(i), visibilities(i))
- }
- }
-
- if (System.getenv("SPARK_YARN_CACHE_ARCHIVES") != null) {
- val timeStamps = System.getenv("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS").split(',')
- val fileSizes = System.getenv("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES").split(',')
- val distArchives = System.getenv("SPARK_YARN_CACHE_ARCHIVES").split(',')
- val visibilities = System.getenv("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES").split(',')
- for( i <- 0 to distArchives.length - 1) {
- setupDistributedCache(distArchives(i), LocalResourceType.ARCHIVE, localResources,
- timeStamps(i), fileSizes(i), visibilities(i))
- }
- }
-
- logInfo("Prepared Local resources " + localResources)
- localResources
- }
-
- def prepareEnvironment: HashMap[String, String] = {
- val env = new HashMap[String, String]()
-
- Client.populateClasspath(yarnConf, sparkConf, System.getenv("SPARK_YARN_LOG4J_PATH") != null, env)
-
- // Allow users to specify some environment variables
- Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"))
-
- System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v }
- env
- }
-
def connectToCM: ContainerManager = {
val cmHostPortStr = container.getNodeId().getHost() + ":" + container.getNodeId().getPort()
val cmAddress = NetUtils.createSocketAddr(cmHostPortStr)
diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala
new file mode 100644
index 0000000000..2db5744be1
--- /dev/null
+++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala
@@ -0,0 +1,410 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.deploy.yarn
+
+import java.net.{InetAddress, UnknownHostException, URI}
+import java.nio.ByteBuffer
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.HashMap
+import scala.collection.mutable.Map
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs._
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.io.DataOutputBuffer
+import org.apache.hadoop.mapred.Master
+import org.apache.hadoop.net.NetUtils
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.yarn.api._
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
+import org.apache.hadoop.yarn.api.protocolrecords._
+import org.apache.hadoop.yarn.api.records._
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+import org.apache.hadoop.yarn.ipc.YarnRPC
+import org.apache.hadoop.yarn.util.{Records, Apps}
+
+import org.apache.spark.{Logging, SparkConf}
+import org.apache.spark.util.Utils
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
+
+
+/**
+ * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. The
+ * Client submits an application to the global ResourceManager to launch Spark's ApplicationMaster,
+ * which will launch a Spark master process and negotiate resources throughout its duration.
+ */
+trait ClientBase extends Logging {
+ val args: ClientArguments
+ val conf: Configuration
+ val sparkConf: SparkConf
+ val yarnConf: YarnConfiguration
+ val credentials = UserGroupInformation.getCurrentUser().getCredentials()
+ private val SPARK_STAGING: String = ".sparkStaging"
+ private val distCacheMgr = new ClientDistributedCacheManager()
+
+ // Staging directory is private! -> rwx--------
+ val STAGING_DIR_PERMISSION: FsPermission = FsPermission.createImmutable(0700: Short)
+ // App files are world-wide readable and owner writable -> rw-r--r--
+ val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644: Short)
+
+ // TODO(harvey): This could just go in ClientArguments.
+ def validateArgs() = {
+ Map(
+ (System.getenv("SPARK_JAR") == null) -> "Error: You must set SPARK_JAR environment variable!",
+ (args.userJar == null) -> "Error: You must specify a user jar!",
+ (args.userClass == null) -> "Error: You must specify a user class!",
+ (args.numWorkers <= 0) -> "Error: You must specify at least 1 worker!",
+ (args.amMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: AM memory size must be" +
+ "greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD),
+ (args.workerMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: Worker memory size" +
+ "must be greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD.toString)
+ ).foreach { case(cond, errStr) =>
+ if (cond) {
+ logError(errStr)
+ args.printUsageAndExit(1)
+ }
+ }
+ }
+
+ def getAppStagingDir(appId: ApplicationId): String = {
+ SPARK_STAGING + Path.SEPARATOR + appId.toString() + Path.SEPARATOR
+ }
+
+ def verifyClusterResources(app: GetNewApplicationResponse) = {
+ val maxMem = app.getMaximumResourceCapability().getMemory()
+ logInfo("Max mem capabililty of a single resource in this cluster " + maxMem)
+
+ // If we have requested more then the clusters max for a single resource then exit.
+ if (args.workerMemory > maxMem) {
+ logError("Required worker memory (%d MB), is above the max threshold (%d MB) of this cluster.".
+ format(args.workerMemory, maxMem))
+ System.exit(1)
+ }
+ val amMem = args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD
+ if (amMem > maxMem) {
+ logError("Required AM memory (%d) is above the max threshold (%d) of this cluster".
+ format(args.amMemory, maxMem))
+ System.exit(1)
+ }
+
+ // We could add checks to make sure the entire cluster has enough resources but that involves
+ // getting all the node reports and computing ourselves.
+ }
+
+ /** See if two file systems are the same or not. */
+ private def compareFs(srcFs: FileSystem, destFs: FileSystem): Boolean = {
+ val srcUri = srcFs.getUri()
+ val dstUri = destFs.getUri()
+ if (srcUri.getScheme() == null) {
+ return false
+ }
+ if (!srcUri.getScheme().equals(dstUri.getScheme())) {
+ return false
+ }
+ var srcHost = srcUri.getHost()
+ var dstHost = dstUri.getHost()
+ if ((srcHost != null) && (dstHost != null)) {
+ try {
+ srcHost = InetAddress.getByName(srcHost).getCanonicalHostName()
+ dstHost = InetAddress.getByName(dstHost).getCanonicalHostName()
+ } catch {
+ case e: UnknownHostException =>
+ return false
+ }
+ if (!srcHost.equals(dstHost)) {
+ return false
+ }
+ } else if (srcHost == null && dstHost != null) {
+ return false
+ } else if (srcHost != null && dstHost == null) {
+ return false
+ }
+ //check for ports
+ if (srcUri.getPort() != dstUri.getPort()) {
+ false
+ } else {
+ true
+ }
+ }
+
+ /** Copy the file into HDFS if needed. */
+ private def copyRemoteFile(
+ dstDir: Path,
+ originalPath: Path,
+ replication: Short,
+ setPerms: Boolean = false): Path = {
+ val fs = FileSystem.get(conf)
+ val remoteFs = originalPath.getFileSystem(conf)
+ var newPath = originalPath
+ if (! compareFs(remoteFs, fs)) {
+ newPath = new Path(dstDir, originalPath.getName())
+ logInfo("Uploading " + originalPath + " to " + newPath)
+ FileUtil.copy(remoteFs, originalPath, fs, newPath, false, conf)
+ fs.setReplication(newPath, replication)
+ if (setPerms) fs.setPermission(newPath, new FsPermission(APP_FILE_PERMISSION))
+ }
+ // Resolve any symlinks in the URI path so using a "current" symlink to point to a specific
+ // version shows the specific version in the distributed cache configuration
+ val qualPath = fs.makeQualified(newPath)
+ val fc = FileContext.getFileContext(qualPath.toUri(), conf)
+ val destPath = fc.resolvePath(qualPath)
+ destPath
+ }
+
+ def qualifyForLocal(localURI: URI): Path = {
+ var qualifiedURI = localURI
+ // If not specified assume these are in the local filesystem to keep behavior like Hadoop
+ if (qualifiedURI.getScheme() == null) {
+ qualifiedURI = new URI(FileSystem.getLocal(conf).makeQualified(new Path(qualifiedURI)).toString)
+ }
+ val qualPath = new Path(qualifiedURI)
+ qualPath
+ }
+
+ def prepareLocalResources(appStagingDir: String): HashMap[String, LocalResource] = {
+ logInfo("Preparing Local resources")
+ // Upload Spark and the application JAR to the remote file system if necessary. Add them as
+ // local resources to the application master.
+ val fs = FileSystem.get(conf)
+
+ val delegTokenRenewer = Master.getMasterPrincipal(conf)
+ if (UserGroupInformation.isSecurityEnabled()) {
+ if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) {
+ logError("Can't get Master Kerberos principal for use as renewer")
+ System.exit(1)
+ }
+ }
+ val dst = new Path(fs.getHomeDirectory(), appStagingDir)
+ val replication = sparkConf.getInt("spark.yarn.submit.file.replication", 3).toShort
+
+ if (UserGroupInformation.isSecurityEnabled()) {
+ val dstFs = dst.getFileSystem(conf)
+ dstFs.addDelegationTokens(delegTokenRenewer, credentials)
+ }
+
+ val localResources = HashMap[String, LocalResource]()
+ FileSystem.mkdirs(fs, dst, new FsPermission(STAGING_DIR_PERMISSION))
+
+ val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
+
+ Map(
+ ClientBase.SPARK_JAR -> System.getenv("SPARK_JAR"), ClientBase.APP_JAR -> args.userJar,
+ ClientBase.LOG4J_PROP -> System.getenv("SPARK_LOG4J_CONF")
+ ).foreach { case(destName, _localPath) =>
+ val localPath: String = if (_localPath != null) _localPath.trim() else ""
+ if (! localPath.isEmpty()) {
+ val localURI = new URI(localPath)
+ val setPermissions = if (destName.equals(ClientBase.APP_JAR)) true else false
+ val destPath = copyRemoteFile(dst, qualifyForLocal(localURI), replication, setPermissions)
+ distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
+ destName, statCache)
+ }
+ }
+
+ // Handle jars local to the ApplicationMaster.
+ if ((args.addJars != null) && (!args.addJars.isEmpty())){
+ args.addJars.split(',').foreach { case file: String =>
+ val localURI = new URI(file.trim())
+ val localPath = new Path(localURI)
+ val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
+ val destPath = copyRemoteFile(dst, localPath, replication)
+ // Only add the resource to the Spark ApplicationMaster.
+ val appMasterOnly = true
+ distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
+ linkname, statCache, appMasterOnly)
+ }
+ }
+
+ // Handle any distributed cache files
+ if ((args.files != null) && (!args.files.isEmpty())){
+ args.files.split(',').foreach { case file: String =>
+ val localURI = new URI(file.trim())
+ val localPath = new Path(localURI)
+ val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
+ val destPath = copyRemoteFile(dst, localPath, replication)
+ distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
+ linkname, statCache)
+ }
+ }
+
+ // Handle any distributed cache archives
+ if ((args.archives != null) && (!args.archives.isEmpty())) {
+ args.archives.split(',').foreach { case file:String =>
+ val localURI = new URI(file.trim())
+ val localPath = new Path(localURI)
+ val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
+ val destPath = copyRemoteFile(dst, localPath, replication)
+ distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE,
+ linkname, statCache)
+ }
+ }
+
+ UserGroupInformation.getCurrentUser().addCredentials(credentials)
+ localResources
+ }
+
+ def setupLaunchEnv(
+ localResources: HashMap[String, LocalResource],
+ stagingDir: String): HashMap[String, String] = {
+ logInfo("Setting up the launch environment")
+ val log4jConfLocalRes = localResources.getOrElse(ClientBase.LOG4J_PROP, null)
+
+ val env = new HashMap[String, String]()
+
+ ClientBase.populateClasspath(yarnConf, sparkConf, log4jConfLocalRes != null, env)
+ env("SPARK_YARN_MODE") = "true"
+ env("SPARK_YARN_STAGING_DIR") = stagingDir
+
+ // Set the environment variables to be passed on to the Workers.
+ distCacheMgr.setDistFilesEnv(env)
+ distCacheMgr.setDistArchivesEnv(env)
+
+ // Allow users to specify some environment variables.
+ Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"))
+
+ // Add each SPARK_* key to the environment.
+ System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v }
+
+ env
+ }
+
+ def userArgsToString(clientArgs: ClientArguments): String = {
+ val prefix = " --args "
+ val args = clientArgs.userArgs
+ val retval = new StringBuilder()
+ for (arg <- args) {
+ retval.append(prefix).append(" '").append(arg).append("' ")
+ }
+ retval.toString
+ }
+
+ def calculateAMMemory(newApp: GetNewApplicationResponse): Int
+
+ def setupSecurityToken(amContainer: ContainerLaunchContext)
+
+ def createContainerLaunchContext(
+ newApp: GetNewApplicationResponse,
+ localResources: HashMap[String, LocalResource],
+ env: HashMap[String, String]): ContainerLaunchContext = {
+ logInfo("Setting up container launch context")
+ val amContainer = Records.newRecord(classOf[ContainerLaunchContext])
+ amContainer.setLocalResources(localResources)
+ amContainer.setEnvironment(env)
+
+ val amMemory = calculateAMMemory(newApp)
+
+ var JAVA_OPTS = ""
+
+ // Add Xmx for AM memory
+ JAVA_OPTS += "-Xmx" + amMemory + "m"
+
+ val tmpDir = new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR)
+ JAVA_OPTS += " -Djava.io.tmpdir=" + tmpDir
+
+ // TODO: Remove once cpuset version is pushed out.
+ // The context is, default gc for server class machines ends up using all cores to do gc -
+ // hence if there are multiple containers in same node, Spark GC affects all other containers'
+ // performance (which can be that of other Spark containers)
+ // Instead of using this, rely on cpusets by YARN to enforce "proper" Spark behavior in
+ // multi-tenant environments. Not sure how default Java GC behaves if it is limited to subset
+ // of cores on a node.
+ val useConcurrentAndIncrementalGC = env.isDefinedAt("SPARK_USE_CONC_INCR_GC") &&
+ java.lang.Boolean.parseBoolean(env("SPARK_USE_CONC_INCR_GC"))
+ if (useConcurrentAndIncrementalGC) {
+ // In our expts, using (default) throughput collector has severe perf ramifications in
+ // multi-tenant machines
+ JAVA_OPTS += " -XX:+UseConcMarkSweepGC "
+ JAVA_OPTS += " -XX:+CMSIncrementalMode "
+ JAVA_OPTS += " -XX:+CMSIncrementalPacing "
+ JAVA_OPTS += " -XX:CMSIncrementalDutyCycleMin=0 "
+ JAVA_OPTS += " -XX:CMSIncrementalDutyCycle=10 "
+ }
+
+ if (env.isDefinedAt("SPARK_JAVA_OPTS")) {
+ JAVA_OPTS += " " + env("SPARK_JAVA_OPTS")
+ }
+
+ // Command for the ApplicationMaster
+ var javaCommand = "java"
+ val javaHome = System.getenv("JAVA_HOME")
+ if ((javaHome != null && !javaHome.isEmpty()) || env.isDefinedAt("JAVA_HOME")) {
+ javaCommand = Environment.JAVA_HOME.$() + "/bin/java"
+ }
+
+ val commands = List[String](
+ javaCommand +
+ " -server " +
+ JAVA_OPTS +
+ " " + args.amClass +
+ " --class " + args.userClass +
+ " --jar " + args.userJar +
+ userArgsToString(args) +
+ " --worker-memory " + args.workerMemory +
+ " --worker-cores " + args.workerCores +
+ " --num-workers " + args.numWorkers +
+ " 1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout" +
+ " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr")
+
+ logInfo("Command for starting the Spark ApplicationMaster: " + commands(0))
+ amContainer.setCommands(commands)
+
+ setupSecurityToken(amContainer)
+ amContainer
+ }
+}
+
+object ClientBase {
+ val SPARK_JAR: String = "spark.jar"
+ val APP_JAR: String = "app.jar"
+ val LOG4J_PROP: String = "log4j.properties"
+
+ // Based on code from org.apache.hadoop.mapreduce.v2.util.MRApps
+ def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String]) {
+ for (c <- conf.getStrings(YarnConfiguration.YARN_APPLICATION_CLASSPATH)) {
+ Apps.addToEnvironment(env, Environment.CLASSPATH.name, c.trim)
+ }
+ }
+
+ def populateClasspath(conf: Configuration, sparkConf: SparkConf, addLog4j: Boolean, env: HashMap[String, String]) {
+ Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$())
+ // If log4j present, ensure ours overrides all others
+ if (addLog4j) {
+ Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
+ Path.SEPARATOR + LOG4J_PROP)
+ }
+ // Normally the users app.jar is last in case conflicts with spark jars
+ val userClasspathFirst = sparkConf.get("spark.yarn.user.classpath.first", "false")
+ .toBoolean
+ if (userClasspathFirst) {
+ Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
+ Path.SEPARATOR + APP_JAR)
+ }
+ Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
+ Path.SEPARATOR + SPARK_JAR)
+ ClientBase.populateHadoopClasspath(conf, env)
+
+ if (!userClasspathFirst) {
+ Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
+ Path.SEPARATOR + APP_JAR)
+ }
+ Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
+ Path.SEPARATOR + "*")
+ }
+}
diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnableUtil.scala
new file mode 100644
index 0000000000..bfa8f84bf7
--- /dev/null
+++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnableUtil.scala
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.deploy.yarn
+
+import java.net.URI
+import java.nio.ByteBuffer
+import java.security.PrivilegedExceptionAction
+
+import scala.collection.JavaConversions._
+import scala.collection.mutable.HashMap
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.io.DataOutputBuffer
+import org.apache.hadoop.net.NetUtils
+import org.apache.hadoop.security.UserGroupInformation
+import org.apache.hadoop.yarn.api._
+import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
+import org.apache.hadoop.yarn.api.records._
+import org.apache.hadoop.yarn.api.protocolrecords._
+import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records}
+
+import org.apache.spark.{SparkConf, Logging}
+import org.apache.hadoop.yarn.conf.YarnConfiguration
+
+
+trait WorkerRunnableUtil extends Logging {
+
+ val yarnConf: YarnConfiguration
+ val sparkConf: SparkConf
+ lazy val env = prepareEnvironment
+
+ def prepareCommand(
+ masterAddress: String,
+ slaveId: String,
+ hostname: String,
+ workerMemory: Int,
+ workerCores: Int) = {
+ // Extra options for the JVM
+ var JAVA_OPTS = ""
+ // Set the JVM memory
+ val workerMemoryString = workerMemory + "m"
+ JAVA_OPTS += "-Xms" + workerMemoryString + " -Xmx" + workerMemoryString + " "
+ if (env.isDefinedAt("SPARK_JAVA_OPTS")) {
+ JAVA_OPTS += env("SPARK_JAVA_OPTS") + " "
+ }
+
+ JAVA_OPTS += " -Djava.io.tmpdir=" +
+ new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + " "
+
+ // Commenting it out for now - so that people can refer to the properties if required. Remove
+ // it once cpuset version is pushed out.
+ // The context is, default gc for server class machines end up using all cores to do gc - hence
+ // if there are multiple containers in same node, spark gc effects all other containers
+ // performance (which can also be other spark containers)
+ // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in
+ // multi-tenant environments. Not sure how default java gc behaves if it is limited to subset
+ // of cores on a node.
+ /*
+ else {
+ // If no java_opts specified, default to using -XX:+CMSIncrementalMode
+ // It might be possible that other modes/config is being done in SPARK_JAVA_OPTS, so we dont
+ // want to mess with it.
+ // In our expts, using (default) throughput collector has severe perf ramnifications in
+ // multi-tennent machines
+ // The options are based on
+ // http://www.oracle.com/technetwork/java/gc-tuning-5-138395.html#0.0.0.%20When%20to%20Use%20the%20Concurrent%20Low%20Pause%20Collector|outline
+ JAVA_OPTS += " -XX:+UseConcMarkSweepGC "
+ JAVA_OPTS += " -XX:+CMSIncrementalMode "
+ JAVA_OPTS += " -XX:+CMSIncrementalPacing "
+ JAVA_OPTS += " -XX:CMSIncrementalDutyCycleMin=0 "
+ JAVA_OPTS += " -XX:CMSIncrementalDutyCycle=10 "
+ }
+ */
+
+ var javaCommand = "java"
+ val javaHome = System.getenv("JAVA_HOME")
+ if ((javaHome != null && !javaHome.isEmpty()) || env.isDefinedAt("JAVA_HOME")) {
+ javaCommand = Environment.JAVA_HOME.$() + "/bin/java"
+ }
+
+ val commands = List[String](javaCommand +
+ " -server " +
+ // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling.
+ // Not killing the task leaves various aspects of the worker and (to some extent) the jvm in
+ // an inconsistent state.
+ // TODO: If the OOM is not recoverable by rescheduling it on different node, then do
+ // 'something' to fail job ... akin to blacklisting trackers in mapred ?
+ " -XX:OnOutOfMemoryError='kill %p' " +
+ JAVA_OPTS +
+ " org.apache.spark.executor.CoarseGrainedExecutorBackend " +
+ masterAddress + " " +
+ slaveId + " " +
+ hostname + " " +
+ workerCores +
+ " 1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout" +
+ " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr")
+
+ commands
+ }
+
+ private def setupDistributedCache(
+ file: String,
+ rtype: LocalResourceType,
+ localResources: HashMap[String, LocalResource],
+ timestamp: String,
+ size: String,
+ vis: String) = {
+ val uri = new URI(file)
+ val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource]
+ amJarRsrc.setType(rtype)
+ amJarRsrc.setVisibility(LocalResourceVisibility.valueOf(vis))
+ amJarRsrc.setResource(ConverterUtils.getYarnUrlFromURI(uri))
+ amJarRsrc.setTimestamp(timestamp.toLong)
+ amJarRsrc.setSize(size.toLong)
+ localResources(uri.getFragment()) = amJarRsrc
+ }
+
+ def prepareLocalResources: HashMap[String, LocalResource] = {
+ logInfo("Preparing Local resources")
+ val localResources = HashMap[String, LocalResource]()
+
+ if (System.getenv("SPARK_YARN_CACHE_FILES") != null) {
+ val timeStamps = System.getenv("SPARK_YARN_CACHE_FILES_TIME_STAMPS").split(',')
+ val fileSizes = System.getenv("SPARK_YARN_CACHE_FILES_FILE_SIZES").split(',')
+ val distFiles = System.getenv("SPARK_YARN_CACHE_FILES").split(',')
+ val visibilities = System.getenv("SPARK_YARN_CACHE_FILES_VISIBILITIES").split(',')
+ for( i <- 0 to distFiles.length - 1) {
+ setupDistributedCache(distFiles(i), LocalResourceType.FILE, localResources, timeStamps(i),
+ fileSizes(i), visibilities(i))
+ }
+ }
+
+ if (System.getenv("SPARK_YARN_CACHE_ARCHIVES") != null) {
+ val timeStamps = System.getenv("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS").split(',')
+ val fileSizes = System.getenv("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES").split(',')
+ val distArchives = System.getenv("SPARK_YARN_CACHE_ARCHIVES").split(',')
+ val visibilities = System.getenv("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES").split(',')
+ for( i <- 0 to distArchives.length - 1) {
+ setupDistributedCache(distArchives(i), LocalResourceType.ARCHIVE, localResources,
+ timeStamps(i), fileSizes(i), visibilities(i))
+ }
+ }
+
+ logInfo("Prepared Local resources " + localResources)
+ localResources
+ }
+
+ def prepareEnvironment: HashMap[String, String] = {
+ val env = new HashMap[String, String]()
+
+ ClientBase.populateClasspath(yarnConf, sparkConf, System.getenv("SPARK_YARN_LOG4J_PATH") != null, env)
+
+ // Allow users to specify some environment variables
+ Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"))
+
+ System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v }
+ env
+ }
+
+}
diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index 51d9adb9d4..837b7e12cb 100644
--- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -17,20 +17,10 @@
package org.apache.spark.deploy.yarn
-import java.net.{InetAddress, UnknownHostException, URI}
import java.nio.ByteBuffer
-import scala.collection.JavaConversions._
-import scala.collection.mutable.HashMap
-import scala.collection.mutable.Map
-
import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.{FileContext, FileStatus, FileSystem, Path, FileUtil}
-import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.io.DataOutputBuffer
-import org.apache.hadoop.mapred.Master
-import org.apache.hadoop.net.NetUtils
-import org.apache.hadoop.security.UserGroupInformation
import org.apache.hadoop.yarn.api._
import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
import org.apache.hadoop.yarn.api.protocolrecords._
@@ -41,8 +31,6 @@ import org.apache.hadoop.yarn.ipc.YarnRPC
import org.apache.hadoop.yarn.util.{Apps, Records}
import org.apache.spark.{Logging, SparkConf}
-import org.apache.spark.util.Utils
-import org.apache.spark.deploy.SparkHadoopUtil
/**
@@ -50,24 +38,19 @@ import org.apache.spark.deploy.SparkHadoopUtil
* Client submits an application to the global ResourceManager to launch Spark's ApplicationMaster,
* which will launch a Spark master process and negotiate resources throughout its duration.
*/
-class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf)
- extends YarnClientImpl with Logging {
+class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: SparkConf)
+ extends YarnClientImpl with ClientBase with Logging {
- def this(args: ClientArguments, sparkConf: SparkConf) =
- this(args, new Configuration(), sparkConf)
+ def this(clientArgs: ClientArguments, spConf: SparkConf) =
+ this(clientArgs, new Configuration(), spConf)
- def this(args: ClientArguments) = this(args, new SparkConf())
+ def this(clientArgs: ClientArguments) = this(clientArgs, new SparkConf())
+ val args = clientArgs
+ val conf = hadoopConf
+ val sparkConf = spConf
var rpc: YarnRPC = YarnRPC.create(conf)
val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
- val credentials = UserGroupInformation.getCurrentUser().getCredentials()
- private val SPARK_STAGING: String = ".sparkStaging"
- private val distCacheMgr = new ClientDistributedCacheManager()
-
- // Staging directory is private! -> rwx--------
- val STAGING_DIR_PERMISSION: FsPermission = FsPermission.createImmutable(0700: Short)
- // App files are world-wide readable and owner writable -> rw-r--r--
- val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644: Short)
def runApp(): ApplicationId = {
validateArgs()
@@ -117,29 +100,6 @@ class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf)
System.exit(0)
}
- // TODO(harvey): This could just go in ClientArguments.
- def validateArgs() = {
- Map(
- (System.getenv("SPARK_JAR") == null) -> "Error: You must set SPARK_JAR environment variable!",
- (args.userJar == null) -> "Error: You must specify a user jar!",
- (args.userClass == null) -> "Error: You must specify a user class!",
- (args.numWorkers <= 0) -> "Error: You must specify at least 1 worker!",
- (args.amMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: AM memory size must be" +
- "greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD),
- (args.workerMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: Worker memory size" +
- "must be greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD.toString)
- ).foreach { case(cond, errStr) =>
- if (cond) {
- logError(errStr)
- args.printUsageAndExit(1)
- }
- }
- }
-
- def getAppStagingDir(appId: ApplicationId): String = {
- SPARK_STAGING + Path.SEPARATOR + appId.toString() + Path.SEPARATOR
- }
-
def logClusterResourceDetails() {
val clusterMetrics: YarnClusterMetrics = super.getYarnClusterMetrics
logInfo("Got Cluster metric info from ApplicationsManager (ASM), number of NodeManagers: " +
@@ -155,285 +115,20 @@ class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf)
queueInfo.getChildQueues.size))
}
- def verifyClusterResources(app: GetNewApplicationResponse) = {
- val maxMem = app.getMaximumResourceCapability().getMemory()
- logInfo("Max mem capabililty of a single resource in this cluster " + maxMem)
-
- // If we have requested more then the clusters max for a single resource then exit.
- if (args.workerMemory > maxMem) {
- logError("Required worker memory (%d MB), is above the max threshold (%d MB) of this cluster.".
- format(args.workerMemory, maxMem))
- System.exit(1)
- }
- val amMem = args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD
- if (amMem > maxMem) {
- logError("Required AM memory (%d) is above the max threshold (%d) of this cluster".
- format(args.amMemory, maxMem))
- System.exit(1)
- }
-
- // We could add checks to make sure the entire cluster has enough resources but that involves
- // getting all the node reports and computing ourselves.
- }
-
- /** See if two file systems are the same or not. */
- private def compareFs(srcFs: FileSystem, destFs: FileSystem): Boolean = {
- val srcUri = srcFs.getUri()
- val dstUri = destFs.getUri()
- if (srcUri.getScheme() == null) {
- return false
- }
- if (!srcUri.getScheme().equals(dstUri.getScheme())) {
- return false
- }
- var srcHost = srcUri.getHost()
- var dstHost = dstUri.getHost()
- if ((srcHost != null) && (dstHost != null)) {
- try {
- srcHost = InetAddress.getByName(srcHost).getCanonicalHostName()
- dstHost = InetAddress.getByName(dstHost).getCanonicalHostName()
- } catch {
- case e: UnknownHostException =>
- return false
- }
- if (!srcHost.equals(dstHost)) {
- return false
- }
- } else if (srcHost == null && dstHost != null) {
- return false
- } else if (srcHost != null && dstHost == null) {
- return false
- }
- //check for ports
- if (srcUri.getPort() != dstUri.getPort()) {
- return false
- }
-
- true
- }
-
- /** Copy the file into HDFS if needed. */
- private def copyRemoteFile(
- dstDir: Path,
- originalPath: Path,
- replication: Short,
- setPerms: Boolean = false): Path = {
- val fs = FileSystem.get(conf)
- val remoteFs = originalPath.getFileSystem(conf)
- var newPath = originalPath
- if (! compareFs(remoteFs, fs)) {
- newPath = new Path(dstDir, originalPath.getName())
- logInfo("Uploading " + originalPath + " to " + newPath)
- FileUtil.copy(remoteFs, originalPath, fs, newPath, false, conf)
- fs.setReplication(newPath, replication)
- if (setPerms) fs.setPermission(newPath, new FsPermission(APP_FILE_PERMISSION))
- }
- // Resolve any symlinks in the URI path so using a "current" symlink to point to a specific
- // version shows the specific version in the distributed cache configuration
- val qualPath = fs.makeQualified(newPath)
- val fc = FileContext.getFileContext(qualPath.toUri(), conf)
- val destPath = fc.resolvePath(qualPath)
- destPath
- }
-
- def prepareLocalResources(appStagingDir: String): HashMap[String, LocalResource] = {
- logInfo("Preparing Local resources")
- // Upload Spark and the application JAR to the remote file system if necessary. Add them as
- // local resources to the application master.
- val fs = FileSystem.get(conf)
-
- val delegTokenRenewer = Master.getMasterPrincipal(conf)
- if (UserGroupInformation.isSecurityEnabled()) {
- if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) {
- logError("Can't get Master Kerberos principal for use as renewer")
- System.exit(1)
- }
- }
- val dst = new Path(fs.getHomeDirectory(), appStagingDir)
- val replication = sparkConf.getInt("spark.yarn.submit.file.replication", 3).toShort
-
- if (UserGroupInformation.isSecurityEnabled()) {
- val dstFs = dst.getFileSystem(conf)
- dstFs.addDelegationTokens(delegTokenRenewer, credentials)
- }
-
- val localResources = HashMap[String, LocalResource]()
- FileSystem.mkdirs(fs, dst, new FsPermission(STAGING_DIR_PERMISSION))
-
- val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]()
-
- Map(
- Client.SPARK_JAR -> System.getenv("SPARK_JAR"), Client.APP_JAR -> args.userJar,
- Client.LOG4J_PROP -> System.getenv("SPARK_LOG4J_CONF")
- ).foreach { case(destName, _localPath) =>
- val localPath: String = if (_localPath != null) _localPath.trim() else ""
- if (! localPath.isEmpty()) {
- var localURI = new URI(localPath)
- // If not specified assume these are in the local filesystem to keep behavior like Hadoop
- if (localURI.getScheme() == null) {
- localURI = new URI(FileSystem.getLocal(conf).makeQualified(new Path(localPath)).toString)
- }
- val setPermissions = if (destName.equals(Client.APP_JAR)) true else false
- val destPath = copyRemoteFile(dst, new Path(localURI), replication, setPermissions)
- distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
- destName, statCache)
- }
- }
-
- // Handle jars local to the ApplicationMaster.
- if ((args.addJars != null) && (!args.addJars.isEmpty())){
- args.addJars.split(',').foreach { case file: String =>
- val localURI = new URI(file.trim())
- val localPath = new Path(localURI)
- val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
- val destPath = copyRemoteFile(dst, localPath, replication)
- // Only add the resource to the Spark ApplicationMaster.
- val appMasterOnly = true
- distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
- linkname, statCache, appMasterOnly)
- }
- }
-
- // Handle any distributed cache files
- if ((args.files != null) && (!args.files.isEmpty())){
- args.files.split(',').foreach { case file: String =>
- val localURI = new URI(file.trim())
- val localPath = new Path(localURI)
- val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
- val destPath = copyRemoteFile(dst, localPath, replication)
- distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE,
- linkname, statCache)
- }
- }
-
- // Handle any distributed cache archives
- if ((args.archives != null) && (!args.archives.isEmpty())) {
- args.archives.split(',').foreach { case file:String =>
- val localURI = new URI(file.trim())
- val localPath = new Path(localURI)
- val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName())
- val destPath = copyRemoteFile(dst, localPath, replication)
- distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.ARCHIVE,
- linkname, statCache)
- }
- }
-
- UserGroupInformation.getCurrentUser().addCredentials(credentials)
- localResources
- }
-
- def setupLaunchEnv(
- localResources: HashMap[String, LocalResource],
- stagingDir: String): HashMap[String, String] = {
- logInfo("Setting up the launch environment")
- val log4jConfLocalRes = localResources.getOrElse(Client.LOG4J_PROP, null)
-
- val env = new HashMap[String, String]()
-
- Client.populateClasspath(yarnConf, sparkConf, log4jConfLocalRes != null, env)
- env("SPARK_YARN_MODE") = "true"
- env("SPARK_YARN_STAGING_DIR") = stagingDir
-
- // Set the environment variables to be passed on to the Workers.
- distCacheMgr.setDistFilesEnv(env)
- distCacheMgr.setDistArchivesEnv(env)
-
- // Allow users to specify some environment variables.
- Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"))
-
- // Add each SPARK_* key to the environment.
- System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v }
-
- env
- }
-
- def userArgsToString(clientArgs: ClientArguments): String = {
- val prefix = " --args "
- val args = clientArgs.userArgs
- val retval = new StringBuilder()
- for (arg <- args) {
- retval.append(prefix).append(" '").append(arg).append("' ")
- }
- retval.toString
- }
-
- def createContainerLaunchContext(
- newApp: GetNewApplicationResponse,
- localResources: HashMap[String, LocalResource],
- env: HashMap[String, String]): ContainerLaunchContext = {
- logInfo("Setting up container launch context")
- val amContainer = Records.newRecord(classOf[ContainerLaunchContext])
- amContainer.setLocalResources(localResources)
- amContainer.setEnvironment(env)
-
+ def calculateAMMemory(newApp: GetNewApplicationResponse) :Int = {
// TODO: Need a replacement for the following code to fix -Xmx?
// val minResMemory: Int = newApp.getMinimumResourceCapability().getMemory()
// var amMemory = ((args.amMemory / minResMemory) * minResMemory) +
// ((if ((args.amMemory % minResMemory) == 0) 0 else minResMemory) -
// YarnAllocationHandler.MEMORY_OVERHEAD)
+ args.amMemory
+ }
- // Extra options for the JVM
- var JAVA_OPTS = ""
-
- // Add Xmx for AM memory
- JAVA_OPTS += "-Xmx" + args.amMemory + "m"
-
- val tmpDir = new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR)
- JAVA_OPTS += " -Djava.io.tmpdir=" + tmpDir
-
- // TODO: Remove once cpuset version is pushed out.
- // The context is, default gc for server class machines ends up using all cores to do gc -
- // hence if there are multiple containers in same node, Spark GC affects all other containers'
- // performance (which can be that of other Spark containers)
- // Instead of using this, rely on cpusets by YARN to enforce "proper" Spark behavior in
- // multi-tenant environments. Not sure how default Java GC behaves if it is limited to subset
- // of cores on a node.
- val useConcurrentAndIncrementalGC = env.isDefinedAt("SPARK_USE_CONC_INCR_GC") &&
- java.lang.Boolean.parseBoolean(env("SPARK_USE_CONC_INCR_GC"))
- if (useConcurrentAndIncrementalGC) {
- // In our expts, using (default) throughput collector has severe perf ramifications in
- // multi-tenant machines
- JAVA_OPTS += " -XX:+UseConcMarkSweepGC "
- JAVA_OPTS += " -XX:+CMSIncrementalMode "
- JAVA_OPTS += " -XX:+CMSIncrementalPacing "
- JAVA_OPTS += " -XX:CMSIncrementalDutyCycleMin=0 "
- JAVA_OPTS += " -XX:CMSIncrementalDutyCycle=10 "
- }
-
- if (env.isDefinedAt("SPARK_JAVA_OPTS")) {
- JAVA_OPTS += " " + env("SPARK_JAVA_OPTS")
- }
-
- // Command for the ApplicationMaster
- var javaCommand = "java"
- val javaHome = System.getenv("JAVA_HOME")
- if ((javaHome != null && !javaHome.isEmpty()) || env.isDefinedAt("JAVA_HOME")) {
- javaCommand = Environment.JAVA_HOME.$() + "/bin/java"
- }
-
- val commands = List[String](
- javaCommand +
- " -server " +
- JAVA_OPTS +
- " " + args.amClass +
- " --class " + args.userClass +
- " --jar " + args.userJar +
- userArgsToString(args) +
- " --worker-memory " + args.workerMemory +
- " --worker-cores " + args.workerCores +
- " --num-workers " + args.numWorkers +
- " 1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout" +
- " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr")
-
- logInfo("Command for starting the Spark ApplicationMaster: " + commands(0))
- amContainer.setCommands(commands)
-
+ def setupSecurityToken(amContainer: ContainerLaunchContext) = {
// Setup security tokens.
val dob = new DataOutputBuffer()
credentials.writeTokenStorageToStream(dob)
amContainer.setTokens(ByteBuffer.wrap(dob.getData()))
-
- amContainer
}
def submitApp(appContext: ApplicationSubmissionContext) = {
@@ -465,7 +160,6 @@ class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf)
)
val state = report.getYarnApplicationState()
- val dsStatus = report.getFinalApplicationStatus()
if (state == YarnApplicationState.FINISHED ||
state == YarnApplicationState.FAILED ||
state == YarnApplicationState.KILLED) {
@@ -477,9 +171,6 @@ class Client(args: ClientArguments, conf: Configuration, sparkConf: SparkConf)
}
object Client {
- val SPARK_JAR: String = "spark.jar"
- val APP_JAR: String = "app.jar"
- val LOG4J_PROP: String = "log4j.properties"
def main(argStrings: Array[String]) {
// Set an env variable indicating we are running in YARN mode.
@@ -492,36 +183,4 @@ object Client {
new Client(args, sparkConf).run()
}
- // Based on code from org.apache.hadoop.mapreduce.v2.util.MRApps
- def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String]) {
- for (c <- conf.getStrings(YarnConfiguration.YARN_APPLICATION_CLASSPATH)) {
- Apps.addToEnvironment(env, Environment.CLASSPATH.name, c.trim)
- }
- }
-
- def populateClasspath(conf: Configuration, sparkConf: SparkConf, addLog4j: Boolean, env: HashMap[String, String]) {
- Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$())
- // If log4j present, ensure ours overrides all others
- if (addLog4j) {
- Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
- Path.SEPARATOR + LOG4J_PROP)
- }
- // Normally the users app.jar is last in case conflicts with spark jars
- val userClasspathFirst = sparkConf.get("spark.yarn.user.classpath.first", "false")
- .toBoolean
- if (userClasspathFirst) {
- Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
- Path.SEPARATOR + APP_JAR)
- }
- Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
- Path.SEPARATOR + SPARK_JAR)
- Client.populateHadoopClasspath(conf, env)
-
- if (!userClasspathFirst) {
- Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
- Path.SEPARATOR + APP_JAR)
- }
- Apps.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$() +
- Path.SEPARATOR + "*")
- }
}
diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
index b7699050bb..ab4a79be70 100644
--- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
+++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala
@@ -17,20 +17,16 @@
package org.apache.spark.deploy.yarn
-import java.net.URI
import java.nio.ByteBuffer
import java.security.PrivilegedExceptionAction
import scala.collection.JavaConversions._
-import scala.collection.mutable.HashMap
import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.Path
import org.apache.hadoop.io.DataOutputBuffer
import org.apache.hadoop.net.NetUtils
import org.apache.hadoop.security.UserGroupInformation
import org.apache.hadoop.yarn.api._
-import org.apache.hadoop.yarn.api.ApplicationConstants.Environment
import org.apache.hadoop.yarn.api.records._
import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils
import org.apache.hadoop.yarn.api.protocolrecords._
@@ -45,16 +41,17 @@ import org.apache.spark.{SparkConf, Logging}
class WorkerRunnable(
container: Container,
conf: Configuration,
- sparkConf: SparkConf,
+ spConf: SparkConf,
masterAddress: String,
slaveId: String,
hostname: String,
workerMemory: Int,
workerCores: Int)
- extends Runnable with Logging {
+ extends Runnable with WorkerRunnableUtil with Logging {
var rpc: YarnRPC = YarnRPC.create(conf)
var nmClient: NMClient = _
+ val sparkConf = spConf
val yarnConf: YarnConfiguration = new YarnConfiguration(conf)
def run = {
@@ -74,73 +71,15 @@ class WorkerRunnable(
val localResources = prepareLocalResources
ctx.setLocalResources(localResources)
- val env = prepareEnvironment
ctx.setEnvironment(env)
- // Extra options for the JVM
- var JAVA_OPTS = ""
- // Set the JVM memory
- val workerMemoryString = workerMemory + "m"
- JAVA_OPTS += "-Xms" + workerMemoryString + " -Xmx" + workerMemoryString + " "
- if (env.isDefinedAt("SPARK_JAVA_OPTS")) {
- JAVA_OPTS += env("SPARK_JAVA_OPTS") + " "
- }
-
- JAVA_OPTS += " -Djava.io.tmpdir=" +
- new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + " "
-
- // Commenting it out for now - so that people can refer to the properties if required. Remove
- // it once cpuset version is pushed out.
- // The context is, default gc for server class machines end up using all cores to do gc - hence
- // if there are multiple containers in same node, spark gc effects all other containers
- // performance (which can also be other spark containers)
- // Instead of using this, rely on cpusets by YARN to enforce spark behaves 'properly' in
- // multi-tenant environments. Not sure how default java gc behaves if it is limited to subset
- // of cores on a node.
-/*
- else {
- // If no java_opts specified, default to using -XX:+CMSIncrementalMode
- // It might be possible that other modes/config is being done in SPARK_JAVA_OPTS, so we dont
- // want to mess with it.
- // In our expts, using (default) throughput collector has severe perf ramnifications in
- // multi-tennent machines
- // The options are based on
- // http://www.oracle.com/technetwork/java/gc-tuning-5-138395.html#0.0.0.%20When%20to%20Use%20the%20Concurrent%20Low%20Pause%20Collector|outline
- JAVA_OPTS += " -XX:+UseConcMarkSweepGC "
- JAVA_OPTS += " -XX:+CMSIncrementalMode "
- JAVA_OPTS += " -XX:+CMSIncrementalPacing "
- JAVA_OPTS += " -XX:CMSIncrementalDutyCycleMin=0 "
- JAVA_OPTS += " -XX:CMSIncrementalDutyCycle=10 "
- }
-*/
-
val credentials = UserGroupInformation.getCurrentUser().getCredentials()
val dob = new DataOutputBuffer()
credentials.writeTokenStorageToStream(dob)
ctx.setTokens(ByteBuffer.wrap(dob.getData()))
- var javaCommand = "java"
- val javaHome = System.getenv("JAVA_HOME")
- if ((javaHome != null && !javaHome.isEmpty()) || env.isDefinedAt("JAVA_HOME")) {
- javaCommand = Environment.JAVA_HOME.$() + "/bin/java"
- }
-
- val commands = List[String](javaCommand +
- " -server " +
- // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling.
- // Not killing the task leaves various aspects of the worker and (to some extent) the jvm in
- // an inconsistent state.
- // TODO: If the OOM is not recoverable by rescheduling it on different node, then do
- // 'something' to fail job ... akin to blacklisting trackers in mapred ?
- " -XX:OnOutOfMemoryError='kill %p' " +
- JAVA_OPTS +
- " org.apache.spark.executor.CoarseGrainedExecutorBackend " +
- masterAddress + " " +
- slaveId + " " +
- hostname + " " +
- workerCores +
- " 1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout" +
- " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr")
+ val commands = prepareCommand(masterAddress, slaveId, hostname, workerMemory, workerCores)
+
logInfo("Setting up worker with commands: " + commands)
ctx.setCommands(commands)
@@ -148,63 +87,4 @@ class WorkerRunnable(
nmClient.startContainer(container, ctx)
}
- private def setupDistributedCache(
- file: String,
- rtype: LocalResourceType,
- localResources: HashMap[String, LocalResource],
- timestamp: String,
- size: String,
- vis: String) = {
- val uri = new URI(file)
- val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource]
- amJarRsrc.setType(rtype)
- amJarRsrc.setVisibility(LocalResourceVisibility.valueOf(vis))
- amJarRsrc.setResource(ConverterUtils.getYarnUrlFromURI(uri))
- amJarRsrc.setTimestamp(timestamp.toLong)
- amJarRsrc.setSize(size.toLong)
- localResources(uri.getFragment()) = amJarRsrc
- }
-
- def prepareLocalResources: HashMap[String, LocalResource] = {
- logInfo("Preparing Local resources")
- val localResources = HashMap[String, LocalResource]()
-
- if (System.getenv("SPARK_YARN_CACHE_FILES") != null) {
- val timeStamps = System.getenv("SPARK_YARN_CACHE_FILES_TIME_STAMPS").split(',')
- val fileSizes = System.getenv("SPARK_YARN_CACHE_FILES_FILE_SIZES").split(',')
- val distFiles = System.getenv("SPARK_YARN_CACHE_FILES").split(',')
- val visibilities = System.getenv("SPARK_YARN_CACHE_FILES_VISIBILITIES").split(',')
- for( i <- 0 to distFiles.length - 1) {
- setupDistributedCache(distFiles(i), LocalResourceType.FILE, localResources, timeStamps(i),
- fileSizes(i), visibilities(i))
- }
- }
-
- if (System.getenv("SPARK_YARN_CACHE_ARCHIVES") != null) {
- val timeStamps = System.getenv("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS").split(',')
- val fileSizes = System.getenv("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES").split(',')
- val distArchives = System.getenv("SPARK_YARN_CACHE_ARCHIVES").split(',')
- val visibilities = System.getenv("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES").split(',')
- for( i <- 0 to distArchives.length - 1) {
- setupDistributedCache(distArchives(i), LocalResourceType.ARCHIVE, localResources,
- timeStamps(i), fileSizes(i), visibilities(i))
- }
- }
-
- logInfo("Prepared Local resources " + localResources)
- localResources
- }
-
- def prepareEnvironment: HashMap[String, String] = {
- val env = new HashMap[String, String]()
-
- Client.populateClasspath(yarnConf, sparkConf, System.getenv("SPARK_YARN_LOG4J_PATH") != null, env)
-
- // Allow users to specify some environment variables
- Apps.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"))
-
- System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v }
- env
- }
-
}