diff options
-rw-r--r-- | core/src/main/scala/spark/PairRDDFunctions.scala | 27 | ||||
-rw-r--r-- | core/src/main/scala/spark/api/java/JavaPairRDD.scala | 24 | ||||
-rw-r--r-- | core/src/main/scala/spark/deploy/worker/Worker.scala | 14 | ||||
-rw-r--r-- | core/src/main/scala/spark/rdd/CoGroupedRDD.scala | 43 | ||||
-rw-r--r-- | core/src/main/scala/spark/rdd/CoalescedRDD.scala | 4 | ||||
-rw-r--r-- | core/src/main/scala/spark/rdd/HadoopRDD.scala | 7 | ||||
-rw-r--r-- | core/src/main/scala/spark/rdd/NewHadoopRDD.scala | 8 | ||||
-rw-r--r-- | core/src/test/scala/spark/JavaAPISuite.java | 22 | ||||
-rw-r--r-- | core/src/test/scala/spark/RDDSuite.scala | 32 | ||||
-rw-r--r-- | examples/src/main/java/spark/examples/JavaKMeans.java | 114 | ||||
-rw-r--r-- | examples/src/main/java/spark/examples/JavaLogQuery.java | 114 | ||||
-rw-r--r-- | examples/src/main/java/spark/examples/JavaSparkPi.java | 48 | ||||
-rw-r--r-- | examples/src/main/scala/spark/examples/SparkKMeans.scala | 1 | ||||
-rw-r--r-- | examples/src/main/scala/spark/examples/SparkPi.scala | 1 |
14 files changed, 437 insertions, 22 deletions
diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 3d1b1ca268..07efba9e8d 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -89,6 +89,33 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( } /** + * Merge the values for each key using an associative function and a neutral "zero value" which may + * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for + * list concatenation, 0 for addition, or 1 for multiplication.). + */ + def foldByKey(zeroValue: V, partitioner: Partitioner)(func: (V, V) => V): RDD[(K, V)] = { + combineByKey[V]({v: V => func(zeroValue, v)}, func, func, partitioner) + } + + /** + * Merge the values for each key using an associative function and a neutral "zero value" which may + * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for + * list concatenation, 0 for addition, or 1 for multiplication.). + */ + def foldByKey(zeroValue: V, numPartitions: Int)(func: (V, V) => V): RDD[(K, V)] = { + foldByKey(zeroValue, new HashPartitioner(numPartitions))(func) + } + + /** + * Merge the values for each key using an associative function and a neutral "zero value" which may + * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for + * list concatenation, 0 for addition, or 1 for multiplication.). + */ + def foldByKey(zeroValue: V)(func: (V, V) => V): RDD[(K, V)] = { + foldByKey(zeroValue, defaultPartitioner(self))(func) + } + + /** * Merge the values for each key using an associative reduce function. This will also perform * the merging locally on each mapper before sending results to a reducer, similarly to a * "combiner" in MapReduce. diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala index c1bd13c49a..49aaabf835 100644 --- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala @@ -161,6 +161,30 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif rdd.countByKeyApprox(timeout, confidence).map(mapAsJavaMap) /** + * Merge the values for each key using an associative function and a neutral "zero value" which may + * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for + * list concatenation, 0 for addition, or 1 for multiplication.). + */ + def foldByKey(zeroValue: V, partitioner: Partitioner, func: JFunction2[V, V, V]): JavaPairRDD[K, V] = + fromRDD(rdd.foldByKey(zeroValue, partitioner)(func)) + + /** + * Merge the values for each key using an associative function and a neutral "zero value" which may + * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for + * list concatenation, 0 for addition, or 1 for multiplication.). + */ + def foldByKey(zeroValue: V, numPartitions: Int, func: JFunction2[V, V, V]): JavaPairRDD[K, V] = + fromRDD(rdd.foldByKey(zeroValue, numPartitions)(func)) + + /** + * Merge the values for each key using an associative function and a neutral "zero value" which may + * be added to the result an arbitrary number of times, and must not change the result (e.g., Nil for + * list concatenation, 0 for addition, or 1 for multiplication.). + */ + def foldByKey(zeroValue: V, func: JFunction2[V, V, V]): JavaPairRDD[K, V] = + fromRDD(rdd.foldByKey(zeroValue)(func)) + + /** * Merge the values for each key using an associative reduce function. This will also perform * the merging locally on each mapper before sending results to a reducer, similarly to a * "combiner" in MapReduce. Output will be hash-partitioned with numPartitions partitions. diff --git a/core/src/main/scala/spark/deploy/worker/Worker.scala b/core/src/main/scala/spark/deploy/worker/Worker.scala index 2bbc931316..da3f4f636c 100644 --- a/core/src/main/scala/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/spark/deploy/worker/Worker.scala @@ -74,16 +74,10 @@ private[spark] class Worker( def connectToMaster() { logInfo("Connecting to master " + masterUrl) - try { - master = context.actorFor(Master.toAkkaUrl(masterUrl)) - master ! RegisterWorker(workerId, ip, port, cores, memory, webUiPort, publicAddress) - context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) - context.watch(master) // Doesn't work with remote actors, but useful for testing - } catch { - case e: Exception => - logError("Failed to connect to master", e) - System.exit(1) - } + master = context.actorFor(Master.toAkkaUrl(masterUrl)) + master ! RegisterWorker(workerId, ip, port, cores, memory, webUiPort, publicAddress) + context.system.eventStream.subscribe(self, classOf[RemoteClientLifeCycleEvent]) + context.watch(master) // Doesn't work with remote actors, but useful for testing } def startWebUi() { diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index 65b4621b87..9213513e80 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -2,10 +2,11 @@ package spark.rdd import java.io.{ObjectOutputStream, IOException} import java.util.{HashMap => JHashMap} + import scala.collection.JavaConversions import scala.collection.mutable.ArrayBuffer -import spark.{Aggregator, Logging, Partitioner, RDD, SparkEnv, Partition, TaskContext} +import spark.{Aggregator, Logging, Partition, Partitioner, RDD, SparkEnv, TaskContext} import spark.{Dependency, OneToOneDependency, ShuffleDependency} @@ -28,7 +29,8 @@ private[spark] case class NarrowCoGroupSplitDep( private[spark] case class ShuffleCoGroupSplitDep(shuffleId: Int) extends CoGroupSplitDep private[spark] -class CoGroupPartition(idx: Int, val deps: Seq[CoGroupSplitDep]) extends Partition with Serializable { +class CoGroupPartition(idx: Int, val deps: Seq[CoGroupSplitDep]) + extends Partition with Serializable { override val index: Int = idx override def hashCode(): Int = idx } @@ -40,7 +42,19 @@ private[spark] class CoGroupAggregator { (b1, b2) => b1 ++ b2 }) with Serializable -class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(K, _)]], part: Partitioner) + +/** + * A RDD that cogroups its parents. For each key k in parent RDDs, the resulting RDD contains a + * tuple with the list of values for that key. + * + * @param rdds parent RDDs. + * @param part partitioner used to partition the shuffle output. + * @param mapSideCombine flag indicating whether to merge values before shuffle step. + */ +class CoGroupedRDD[K]( + @transient var rdds: Seq[RDD[(K, _)]], + part: Partitioner, + val mapSideCombine: Boolean = true) extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) { private val aggr = new CoGroupAggregator @@ -52,8 +66,12 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(K, _)]], part: Partitioner) new OneToOneDependency(rdd) } else { logInfo("Adding shuffle dependency with " + rdd) - val mapSideCombinedRDD = rdd.mapPartitions(aggr.combineValuesByKey(_), true) - new ShuffleDependency[Any, ArrayBuffer[Any]](mapSideCombinedRDD, part) + if (mapSideCombine) { + val mapSideCombinedRDD = rdd.mapPartitions(aggr.combineValuesByKey(_), true) + new ShuffleDependency[Any, ArrayBuffer[Any]](mapSideCombinedRDD, part) + } else { + new ShuffleDependency[Any, Any](rdd.asInstanceOf[RDD[(Any, Any)]], part) + } } } } @@ -82,6 +100,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(K, _)]], part: Partitioner) val numRdds = split.deps.size // e.g. for `(k, a) cogroup (k, b)`, K -> Seq(ArrayBuffer as, ArrayBuffer bs) val map = new JHashMap[K, Seq[ArrayBuffer[Any]]] + def getSeq(k: K): Seq[ArrayBuffer[Any]] = { val seq = map.get(k) if (seq != null) { @@ -92,6 +111,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(K, _)]], part: Partitioner) seq } } + for ((dep, depNum) <- split.deps.zipWithIndex) dep match { case NarrowCoGroupSplitDep(rdd, _, itsSplit) => { // Read them from the parent @@ -102,9 +122,16 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[(K, _)]], part: Partitioner) case ShuffleCoGroupSplitDep(shuffleId) => { // Read map outputs of shuffle val fetcher = SparkEnv.get.shuffleFetcher - val fetchItr = fetcher.fetch[K, Seq[Any]](shuffleId, split.index, context.taskMetrics) - for ((k, vs) <- fetchItr) { - getSeq(k)(depNum) ++= vs + if (mapSideCombine) { + // With map side combine on, for each key, the shuffle fetcher returns a list of values. + fetcher.fetch[K, Seq[Any]](shuffleId, split.index, context.taskMetrics).foreach { + case (key, values) => getSeq(key)(depNum) ++= values + } + } else { + // With map side combine off, for each key the shuffle fetcher returns a single value. + fetcher.fetch[K, Any](shuffleId, split.index, context.taskMetrics).foreach { + case (key, value) => getSeq(key)(depNum) += value + } } } } diff --git a/core/src/main/scala/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/spark/rdd/CoalescedRDD.scala index 0d16cf6e85..6d862c0c28 100644 --- a/core/src/main/scala/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoalescedRDD.scala @@ -37,8 +37,8 @@ class CoalescedRDD[T: ClassManifest]( prevSplits.map(_.index).map{idx => new CoalescedRDDPartition(idx, prev, Array(idx)) } } else { (0 until maxPartitions).map { i => - val rangeStart = (i * prevSplits.length) / maxPartitions - val rangeEnd = ((i + 1) * prevSplits.length) / maxPartitions + val rangeStart = ((i.toLong * prevSplits.length) / maxPartitions).toInt + val rangeEnd = (((i.toLong + 1) * prevSplits.length) / maxPartitions).toInt new CoalescedRDDPartition(i, prev, (rangeStart until rangeEnd).toArray) }.toArray } diff --git a/core/src/main/scala/spark/rdd/HadoopRDD.scala b/core/src/main/scala/spark/rdd/HadoopRDD.scala index a6322dc58d..cbf5512e24 100644 --- a/core/src/main/scala/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/HadoopRDD.scala @@ -17,6 +17,7 @@ import org.apache.hadoop.util.ReflectionUtils import spark.{Dependency, Logging, Partition, RDD, SerializableWritable, SparkContext, TaskContext} import spark.util.NextIterator +import org.apache.hadoop.conf.Configurable /** @@ -50,6 +51,9 @@ class HadoopRDD[K, V]( override def getPartitions: Array[Partition] = { val inputFormat = createInputFormat(conf) + if (inputFormat.isInstanceOf[Configurable]) { + inputFormat.asInstanceOf[Configurable].setConf(conf) + } val inputSplits = inputFormat.getSplits(conf, minSplits) val array = new Array[Partition](inputSplits.size) for (i <- 0 until inputSplits.size) { @@ -69,6 +73,9 @@ class HadoopRDD[K, V]( val conf = confBroadcast.value.value val fmt = createInputFormat(conf) + if (fmt.isInstanceOf[Configurable]) { + fmt.asInstanceOf[Configurable].setConf(conf) + } reader = fmt.getRecordReader(split.inputSplit.value, conf, Reporter.NULL) // Register an on-task-completion callback to close the input stream. diff --git a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala index df2361025c..bdd974590a 100644 --- a/core/src/main/scala/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/spark/rdd/NewHadoopRDD.scala @@ -3,7 +3,7 @@ package spark.rdd import java.text.SimpleDateFormat import java.util.Date -import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ @@ -42,6 +42,9 @@ class NewHadoopRDD[K, V]( override def getPartitions: Array[Partition] = { val inputFormat = inputFormatClass.newInstance + if (inputFormat.isInstanceOf[Configurable]) { + inputFormat.asInstanceOf[Configurable].setConf(conf) + } val jobContext = newJobContext(conf, jobId) val rawSplits = inputFormat.getSplits(jobContext).toArray val result = new Array[Partition](rawSplits.size) @@ -57,6 +60,9 @@ class NewHadoopRDD[K, V]( val attemptId = new TaskAttemptID(jobtrackerId, id, true, split.index, 0) val hadoopAttemptContext = newTaskAttemptContext(conf, attemptId) val format = inputFormatClass.newInstance + if (format.isInstanceOf[Configurable]) { + format.asInstanceOf[Configurable].setConf(conf) + } val reader = format.createRecordReader( split.serializableHadoopSplit.value, hadoopAttemptContext) reader.initialize(split.serializableHadoopSplit.value, hadoopAttemptContext) diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index 26e3ab72c0..d3dcd3bbeb 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -197,6 +197,28 @@ public class JavaAPISuite implements Serializable { } @Test + public void foldByKey() { + List<Tuple2<Integer, Integer>> pairs = Arrays.asList( + new Tuple2<Integer, Integer>(2, 1), + new Tuple2<Integer, Integer>(2, 1), + new Tuple2<Integer, Integer>(1, 1), + new Tuple2<Integer, Integer>(3, 2), + new Tuple2<Integer, Integer>(3, 1) + ); + JavaPairRDD<Integer, Integer> rdd = sc.parallelizePairs(pairs); + JavaPairRDD<Integer, Integer> sums = rdd.foldByKey(0, + new Function2<Integer, Integer, Integer>() { + @Override + public Integer call(Integer a, Integer b) { + return a + b; + } + }); + Assert.assertEquals(1, sums.lookup(1).get(0).intValue()); + Assert.assertEquals(2, sums.lookup(2).get(0).intValue()); + Assert.assertEquals(3, sums.lookup(3).get(0).intValue()); + } + + @Test public void reduceByKey() { List<Tuple2<Integer, Integer>> pairs = Arrays.asList( new Tuple2<Integer, Integer>(2, 1), diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 9739ba869b..0ff4a47998 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -3,7 +3,7 @@ package spark import scala.collection.mutable.HashMap import org.scalatest.FunSuite import spark.SparkContext._ -import spark.rdd.{CoalescedRDD, PartitionPruningRDD} +import spark.rdd.{CoalescedRDD, CoGroupedRDD, PartitionPruningRDD} class RDDSuite extends FunSuite with LocalSparkContext { @@ -123,6 +123,36 @@ class RDDSuite extends FunSuite with LocalSparkContext { assert(rdd.collect().toList === List(1, 2, 3, 4)) } + test("cogrouped RDDs") { + sc = new SparkContext("local", "test") + val rdd1 = sc.makeRDD(Array((1, "one"), (1, "another one"), (2, "two"), (3, "three")), 2) + val rdd2 = sc.makeRDD(Array((1, "one1"), (1, "another one1"), (2, "two1")), 2) + + // Use cogroup function + val cogrouped = rdd1.cogroup(rdd2).collectAsMap() + assert(cogrouped(1) === (Seq("one", "another one"), Seq("one1", "another one1"))) + assert(cogrouped(2) === (Seq("two"), Seq("two1"))) + assert(cogrouped(3) === (Seq("three"), Seq())) + + // Construct CoGroupedRDD directly, with map side combine enabled + val cogrouped1 = new CoGroupedRDD[Int]( + Seq(rdd1.asInstanceOf[RDD[(Int, Any)]], rdd2.asInstanceOf[RDD[(Int, Any)]]), + new HashPartitioner(3), + true).collectAsMap() + assert(cogrouped1(1).toSeq === Seq(Seq("one", "another one"), Seq("one1", "another one1"))) + assert(cogrouped1(2).toSeq === Seq(Seq("two"), Seq("two1"))) + assert(cogrouped1(3).toSeq === Seq(Seq("three"), Seq())) + + // Construct CoGroupedRDD directly, with map side combine disabled + val cogrouped2 = new CoGroupedRDD[Int]( + Seq(rdd1.asInstanceOf[RDD[(Int, Any)]], rdd2.asInstanceOf[RDD[(Int, Any)]]), + new HashPartitioner(3), + false).collectAsMap() + assert(cogrouped2(1).toSeq === Seq(Seq("one", "another one"), Seq("one1", "another one1"))) + assert(cogrouped2(2).toSeq === Seq(Seq("two"), Seq("two1"))) + assert(cogrouped2(3).toSeq === Seq(Seq("three"), Seq())) + } + test("coalesced RDDs") { sc = new SparkContext("local", "test") val data = sc.parallelize(1 to 10, 10) diff --git a/examples/src/main/java/spark/examples/JavaKMeans.java b/examples/src/main/java/spark/examples/JavaKMeans.java new file mode 100644 index 0000000000..626034eb0d --- /dev/null +++ b/examples/src/main/java/spark/examples/JavaKMeans.java @@ -0,0 +1,114 @@ +package spark.examples; + +import scala.Tuple2; +import spark.api.java.JavaPairRDD; +import spark.api.java.JavaRDD; +import spark.api.java.JavaSparkContext; +import spark.api.java.function.Function; +import spark.api.java.function.PairFunction; +import spark.util.Vector; + +import java.util.List; +import java.util.Map; + +/** + * K-means clustering using Java API. + */ +public class JavaKMeans { + + /** Parses numbers split by whitespace to a vector */ + static Vector parseVector(String line) { + String[] splits = line.split(" "); + double[] data = new double[splits.length]; + int i = 0; + for (String s : splits) + data[i] = Double.parseDouble(splits[i++]); + return new Vector(data); + } + + /** Computes the vector to which the input vector is closest using squared distance */ + static int closestPoint(Vector p, List<Vector> centers) { + int bestIndex = 0; + double closest = Double.POSITIVE_INFINITY; + for (int i = 0; i < centers.size(); i++) { + double tempDist = p.squaredDist(centers.get(i)); + if (tempDist < closest) { + closest = tempDist; + bestIndex = i; + } + } + return bestIndex; + } + + /** Computes the mean across all vectors in the input set of vectors */ + static Vector average(List<Vector> ps) { + int numVectors = ps.size(); + Vector out = new Vector(ps.get(0).elements()); + // start from i = 1 since we already copied index 0 above + for (int i = 1; i < numVectors; i++) { + out.addInPlace(ps.get(i)); + } + return out.divide(numVectors); + } + + public static void main(String[] args) throws Exception { + if (args.length < 4) { + System.err.println("Usage: JavaKMeans <master> <file> <k> <convergeDist>"); + System.exit(1); + } + JavaSparkContext sc = new JavaSparkContext(args[0], "JavaKMeans", + System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + String path = args[1]; + int K = Integer.parseInt(args[2]); + double convergeDist = Double.parseDouble(args[3]); + + JavaRDD<Vector> data = sc.textFile(path).map( + new Function<String, Vector>() { + @Override + public Vector call(String line) throws Exception { + return parseVector(line); + } + } + ).cache(); + + final List<Vector> centroids = data.takeSample(false, K, 42); + + double tempDist; + do { + // allocate each vector to closest centroid + JavaPairRDD<Integer, Vector> closest = data.map( + new PairFunction<Vector, Integer, Vector>() { + @Override + public Tuple2<Integer, Vector> call(Vector vector) throws Exception { + return new Tuple2<Integer, Vector>( + closestPoint(vector, centroids), vector); + } + } + ); + + // group by cluster id and average the vectors within each cluster to compute centroids + JavaPairRDD<Integer, List<Vector>> pointsGroup = closest.groupByKey(); + Map<Integer, Vector> newCentroids = pointsGroup.mapValues( + new Function<List<Vector>, Vector>() { + public Vector call(List<Vector> ps) throws Exception { + return average(ps); + } + }).collectAsMap(); + tempDist = 0.0; + for (int i = 0; i < K; i++) { + tempDist += centroids.get(i).squaredDist(newCentroids.get(i)); + } + for (Map.Entry<Integer, Vector> t: newCentroids.entrySet()) { + centroids.set(t.getKey(), t.getValue()); + } + System.out.println("Finished iteration (delta = " + tempDist + ")"); + } while (tempDist > convergeDist); + + System.out.println("Final centers:"); + for (Vector c : centroids) + System.out.println(c); + + System.exit(0); + + } +} diff --git a/examples/src/main/java/spark/examples/JavaLogQuery.java b/examples/src/main/java/spark/examples/JavaLogQuery.java new file mode 100644 index 0000000000..6b22e7120c --- /dev/null +++ b/examples/src/main/java/spark/examples/JavaLogQuery.java @@ -0,0 +1,114 @@ +package spark.examples; + +import com.google.common.collect.Lists; +import scala.Tuple2; +import scala.Tuple3; +import spark.api.java.JavaPairRDD; +import spark.api.java.JavaRDD; +import spark.api.java.JavaSparkContext; +import spark.api.java.function.Function2; +import spark.api.java.function.PairFunction; + +import java.io.Serializable; +import java.util.Collections; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Executes a roll up-style query against Apache logs. + */ +public class JavaLogQuery { + + public static List<String> exampleApacheLogs = Lists.newArrayList( + "10.10.10.10 - \"FRED\" [18/Jan/2013:17:56:07 +1100] \"GET http://images.com/2013/Generic.jpg " + + "HTTP/1.1\" 304 315 \"http://referall.com/\" \"Mozilla/4.0 (compatible; MSIE 7.0; " + + "Windows NT 5.1; GTB7.4; .NET CLR 2.0.50727; .NET CLR 3.0.04506.30; .NET CLR 3.0.04506.648; " + + ".NET CLR 3.5.21022; .NET CLR 3.0.4506.2152; .NET CLR 1.0.3705; .NET CLR 1.1.4322; .NET CLR " + + "3.5.30729; Release=ARP)\" \"UD-1\" - \"image/jpeg\" \"whatever\" 0.350 \"-\" - \"\" 265 923 934 \"\" " + + "62.24.11.25 images.com 1358492167 - Whatup", + "10.10.10.10 - \"FRED\" [18/Jan/2013:18:02:37 +1100] \"GET http://images.com/2013/Generic.jpg " + + "HTTP/1.1\" 304 306 \"http:/referall.com\" \"Mozilla/4.0 (compatible; MSIE 7.0; Windows NT 5.1; " + + "GTB7.4; .NET CLR 2.0.50727; .NET CLR 3.0.04506.30; .NET CLR 3.0.04506.648; .NET CLR " + + "3.5.21022; .NET CLR 3.0.4506.2152; .NET CLR 1.0.3705; .NET CLR 1.1.4322; .NET CLR " + + "3.5.30729; Release=ARP)\" \"UD-1\" - \"image/jpeg\" \"whatever\" 0.352 \"-\" - \"\" 256 977 988 \"\" " + + "0 73.23.2.15 images.com 1358492557 - Whatup"); + + public static Pattern apacheLogRegex = Pattern.compile( + "^([\\d.]+) (\\S+) (\\S+) \\[([\\w\\d:/]+\\s[+\\-]\\d{4})\\] \"(.+?)\" (\\d{3}) ([\\d\\-]+) \"([^\"]+)\" \"([^\"]+)\".*"); + + /** Tracks the total query count and number of aggregate bytes for a particular group. */ + public static class Stats implements Serializable { + + private int count; + private int numBytes; + + public Stats(int count, int numBytes) { + this.count = count; + this.numBytes = numBytes; + } + public Stats merge(Stats other) { + return new Stats(count + other.count, numBytes + other.numBytes); + } + + public String toString() { + return String.format("bytes=%s\tn=%s", numBytes, count); + } + } + + public static Tuple3<String, String, String> extractKey(String line) { + Matcher m = apacheLogRegex.matcher(line); + List<String> key = Collections.emptyList(); + if (m.find()) { + String ip = m.group(1); + String user = m.group(3); + String query = m.group(5); + if (!user.equalsIgnoreCase("-")) { + return new Tuple3<String, String, String>(ip, user, query); + } + } + return new Tuple3<String, String, String>(null, null, null); + } + + public static Stats extractStats(String line) { + Matcher m = apacheLogRegex.matcher(line); + if (m.find()) { + int bytes = Integer.parseInt(m.group(7)); + return new Stats(1, bytes); + } + else + return new Stats(1, 0); + } + + public static void main(String[] args) throws Exception { + if (args.length == 0) { + System.err.println("Usage: JavaLogQuery <master> [logFile]"); + System.exit(1); + } + + JavaSparkContext jsc = new JavaSparkContext(args[0], "JavaLogQuery", + System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + + JavaRDD<String> dataSet = (args.length == 2) ? jsc.textFile(args[1]) : jsc.parallelize(exampleApacheLogs); + + JavaPairRDD<Tuple3<String, String, String>, Stats> extracted = dataSet.map(new PairFunction<String, Tuple3<String, String, String>, Stats>() { + @Override + public Tuple2<Tuple3<String, String, String>, Stats> call(String s) throws Exception { + return new Tuple2<Tuple3<String, String, String>, Stats>(extractKey(s), extractStats(s)); + } + }); + + JavaPairRDD<Tuple3<String, String, String>, Stats> counts = extracted.reduceByKey(new Function2<Stats, Stats, Stats>() { + @Override + public Stats call(Stats stats, Stats stats2) throws Exception { + return stats.merge(stats2); + } + }); + + List<Tuple2<Tuple3<String, String, String>, Stats>> output = counts.collect(); + for (Tuple2 t : output) { + System.out.println(t._1 + "\t" + t._2); + } + System.exit(0); + } +} diff --git a/examples/src/main/java/spark/examples/JavaSparkPi.java b/examples/src/main/java/spark/examples/JavaSparkPi.java new file mode 100644 index 0000000000..a15a967de8 --- /dev/null +++ b/examples/src/main/java/spark/examples/JavaSparkPi.java @@ -0,0 +1,48 @@ +package spark.examples; + +import spark.api.java.JavaRDD; +import spark.api.java.JavaSparkContext; +import spark.api.java.function.Function; +import spark.api.java.function.Function2; + +import java.util.ArrayList; +import java.util.List; + +/** Computes an approximation to pi */ +public class JavaSparkPi { + + + public static void main(String[] args) throws Exception { + if (args.length == 0) { + System.err.println("Usage: JavaLogQuery <master> [slices]"); + System.exit(1); + } + + JavaSparkContext jsc = new JavaSparkContext(args[0], "JavaLogQuery", + System.getenv("SPARK_HOME"), System.getenv("SPARK_EXAMPLES_JAR")); + + int slices = (args.length == 2) ? Integer.parseInt(args[1]) : 2; + int n = 100000 * slices; + List<Integer> l = new ArrayList<Integer>(n); + for (int i = 0; i < n; i++) + l.add(i); + + JavaRDD<Integer> dataSet = jsc.parallelize(l, slices); + + int count = dataSet.map(new Function<Integer, Integer>() { + @Override + public Integer call(Integer integer) throws Exception { + double x = Math.random() * 2 - 1; + double y = Math.random() * 2 - 1; + return (x * x + y * y < 1) ? 1 : 0; + } + }).reduce(new Function2<Integer, Integer, Integer>() { + @Override + public Integer call(Integer integer, Integer integer2) throws Exception { + return integer + integer2; + } + }); + + System.out.println("Pi is roughly " + 4.0 * count / n); + } +} diff --git a/examples/src/main/scala/spark/examples/SparkKMeans.scala b/examples/src/main/scala/spark/examples/SparkKMeans.scala index 7c21ea12fb..4161c59fea 100644 --- a/examples/src/main/scala/spark/examples/SparkKMeans.scala +++ b/examples/src/main/scala/spark/examples/SparkKMeans.scala @@ -64,6 +64,7 @@ object SparkKMeans { for (newP <- newPoints) { kPoints(newP._1) = newP._2 } + println("Finished iteration (delta = " + tempDist + ")") } println("Final centers:") diff --git a/examples/src/main/scala/spark/examples/SparkPi.scala b/examples/src/main/scala/spark/examples/SparkPi.scala index 5a31d74444..f598d2ff9c 100644 --- a/examples/src/main/scala/spark/examples/SparkPi.scala +++ b/examples/src/main/scala/spark/examples/SparkPi.scala @@ -4,6 +4,7 @@ import scala.math.random import spark._ import SparkContext._ +/** Computes an approximation to pi */ object SparkPi { def main(args: Array[String]) { if (args.length == 0) { |