From 886b39de557b4d5f54f5ca11559fca9799534280 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 10 Aug 2012 01:10:02 -0700 Subject: [PATCH 01/85] Add Python API. --- .../scala/spark/api/python/PythonRDD.scala | 147 +++++ pyspark/pyspark/__init__.py | 0 pyspark/pyspark/context.py | 69 +++ pyspark/pyspark/examples/__init__.py | 0 pyspark/pyspark/examples/kmeans.py | 56 ++ pyspark/pyspark/examples/pi.py | 20 + pyspark/pyspark/examples/tc.py | 49 ++ pyspark/pyspark/java_gateway.py | 20 + pyspark/pyspark/join.py | 104 ++++ pyspark/pyspark/rdd.py | 517 ++++++++++++++++++ pyspark/pyspark/serializers.py | 229 ++++++++ pyspark/pyspark/worker.py | 97 ++++ pyspark/requirements.txt | 9 + python/tc.py | 22 + 14 files changed, 1339 insertions(+) create mode 100644 core/src/main/scala/spark/api/python/PythonRDD.scala create mode 100644 pyspark/pyspark/__init__.py create mode 100644 pyspark/pyspark/context.py create mode 100644 pyspark/pyspark/examples/__init__.py create mode 100644 pyspark/pyspark/examples/kmeans.py create mode 100644 pyspark/pyspark/examples/pi.py create mode 100644 pyspark/pyspark/examples/tc.py create mode 100644 pyspark/pyspark/java_gateway.py create mode 100644 pyspark/pyspark/join.py create mode 100644 pyspark/pyspark/rdd.py create mode 100644 pyspark/pyspark/serializers.py create mode 100644 pyspark/pyspark/worker.py create mode 100644 pyspark/requirements.txt create mode 100644 python/tc.py diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala new file mode 100644 index 0000000000..660ad48afe --- /dev/null +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -0,0 +1,147 @@ +package spark.api.python + +import java.io.PrintWriter + +import scala.collection.Map +import scala.collection.JavaConversions._ +import scala.io.Source +import spark._ +import api.java.{JavaPairRDD, JavaRDD} +import scala.Some + +trait PythonRDDBase { + def compute[T](split: Split, envVars: Map[String, String], + command: Seq[String], parent: RDD[T], pythonExec: String): Iterator[String]= { + val currentEnvVars = new ProcessBuilder().environment() + val SPARK_HOME = currentEnvVars.get("SPARK_HOME") + + val pb = new ProcessBuilder(Seq(pythonExec, SPARK_HOME + "/pyspark/pyspark/worker.py")) + // Add the environmental variables to the process. + envVars.foreach { + case (variable, value) => currentEnvVars.put(variable, value) + } + + val proc = pb.start() + val env = SparkEnv.get + + // Start a thread to print the process's stderr to ours + new Thread("stderr reader for " + command) { + override def run() { + for (line <- Source.fromInputStream(proc.getErrorStream).getLines) { + System.err.println(line) + } + } + }.start() + + // Start a thread to feed the process input from our parent's iterator + new Thread("stdin writer for " + command) { + override def run() { + SparkEnv.set(env) + val out = new PrintWriter(proc.getOutputStream) + for (elem <- command) { + out.println(elem) + } + for (elem <- parent.iterator(split)) { + out.println(PythonRDD.pythonDump(elem)) + } + out.close() + } + }.start() + + // Return an iterator that read lines from the process's stdout + val lines: Iterator[String] = Source.fromInputStream(proc.getInputStream).getLines + wrapIterator(lines, proc) + } + + def wrapIterator[T](iter: Iterator[T], proc: Process): Iterator[T] = { + return new Iterator[T] { + def next() = iter.next() + + def hasNext = { + if (iter.hasNext) { + true + } else { + val exitStatus = proc.waitFor() + if (exitStatus != 0) { + throw new Exception("Subprocess exited with status " + exitStatus) + } + false + } + } + } + } +} + +class PythonRDD[T: ClassManifest]( + parent: RDD[T], command: Seq[String], envVars: Map[String, String], + preservePartitoning: Boolean, pythonExec: String) + extends RDD[String](parent.context) with PythonRDDBase { + + def this(parent: RDD[T], command: Seq[String], preservePartitoning: Boolean, pythonExec: String) = + this(parent, command, Map(), preservePartitoning, pythonExec) + + // Similar to Runtime.exec(), if we are given a single string, split it into words + // using a standard StringTokenizer (i.e. by spaces) + def this(parent: RDD[T], command: String, preservePartitoning: Boolean, pythonExec: String) = + this(parent, PipedRDD.tokenize(command), preservePartitoning, pythonExec) + + override def splits = parent.splits + + override val dependencies = List(new OneToOneDependency(parent)) + + override val partitioner = if (preservePartitoning) parent.partitioner else None + + override def compute(split: Split): Iterator[String] = + compute(split, envVars, command, parent, pythonExec) + + val asJavaRDD : JavaRDD[String] = JavaRDD.fromRDD(this) +} + +class PythonPairRDD[T: ClassManifest] ( + parent: RDD[T], command: Seq[String], envVars: Map[String, String], + preservePartitoning: Boolean, pythonExec: String) + extends RDD[(String, String)](parent.context) with PythonRDDBase { + + def this(parent: RDD[T], command: Seq[String], preservePartitoning: Boolean, pythonExec: String) = + this(parent, command, Map(), preservePartitoning, pythonExec) + + // Similar to Runtime.exec(), if we are given a single string, split it into words + // using a standard StringTokenizer (i.e. by spaces) + def this(parent: RDD[T], command: String, preservePartitoning: Boolean, pythonExec: String) = + this(parent, PipedRDD.tokenize(command), preservePartitoning, pythonExec) + + override def splits = parent.splits + + override val dependencies = List(new OneToOneDependency(parent)) + + override val partitioner = if (preservePartitoning) parent.partitioner else None + + override def compute(split: Split): Iterator[(String, String)] = { + compute(split, envVars, command, parent, pythonExec).grouped(2).map { + case Seq(a, b) => (a, b) + case x => throw new Exception("Unexpected value: " + x) + } + } + + val asJavaPairRDD : JavaPairRDD[String, String] = JavaPairRDD.fromRDD(this) +} + +object PythonRDD { + def pythonDump[T](x: T): String = { + if (x.isInstanceOf[scala.Option[_]]) { + val t = x.asInstanceOf[scala.Option[_]] + t match { + case None => "*" + case Some(z) => pythonDump(z) + } + } else if (x.isInstanceOf[scala.Tuple2[_, _]]) { + val t = x.asInstanceOf[scala.Tuple2[_, _]] + "(" + pythonDump(t._1) + "," + pythonDump(t._2) + ")" + } else if (x.isInstanceOf[java.util.List[_]]) { + val objs = asScalaBuffer(x.asInstanceOf[java.util.List[_]]).map(pythonDump) + "[" + objs.mkString("|") + "]" + } else { + x.toString + } + } +} diff --git a/pyspark/pyspark/__init__.py b/pyspark/pyspark/__init__.py new file mode 100644 index 0000000000..e69de29bb2 diff --git a/pyspark/pyspark/context.py b/pyspark/pyspark/context.py new file mode 100644 index 0000000000..587ab12b5f --- /dev/null +++ b/pyspark/pyspark/context.py @@ -0,0 +1,69 @@ +import os +import atexit +from tempfile import NamedTemporaryFile + +from pyspark.java_gateway import launch_gateway +from pyspark.serializers import JSONSerializer, NopSerializer +from pyspark.rdd import RDD, PairRDD + + +class SparkContext(object): + + gateway = launch_gateway() + jvm = gateway.jvm + python_dump = jvm.spark.api.python.PythonRDD.pythonDump + + def __init__(self, master, name, defaultSerializer=JSONSerializer, + defaultParallelism=None, pythonExec='python'): + self.master = master + self.name = name + self._jsc = self.jvm.JavaSparkContext(master, name) + self.defaultSerializer = defaultSerializer + self.defaultParallelism = \ + defaultParallelism or self._jsc.sc().defaultParallelism() + self.pythonExec = pythonExec + + def __del__(self): + if self._jsc: + self._jsc.stop() + + def stop(self): + self._jsc.stop() + self._jsc = None + + def parallelize(self, c, numSlices=None, serializer=None): + serializer = serializer or self.defaultSerializer + numSlices = numSlices or self.defaultParallelism + # Calling the Java parallelize() method with an ArrayList is too slow, + # because it sends O(n) Py4J commands. As an alternative, serialized + # objects are written to a file and loaded through textFile(). + tempFile = NamedTemporaryFile(delete=False) + tempFile.writelines(serializer.dumps(x) + '\n' for x in c) + tempFile.close() + atexit.register(lambda: os.unlink(tempFile.name)) + return self.textFile(tempFile.name, numSlices, serializer) + + def parallelizePairs(self, c, numSlices=None, keySerializer=None, + valSerializer=None): + """ + >>> sc = SparkContext("local", "test") + >>> rdd = sc.parallelizePairs([(1, 2), (3, 4)]) + >>> rdd.collect() + [(1, 2), (3, 4)] + """ + keySerializer = keySerializer or self.defaultSerializer + valSerializer = valSerializer or self.defaultSerializer + numSlices = numSlices or self.defaultParallelism + tempFile = NamedTemporaryFile(delete=False) + for (k, v) in c: + tempFile.write(keySerializer.dumps(k).rstrip('\r\n') + '\n') + tempFile.write(valSerializer.dumps(v).rstrip('\r\n') + '\n') + tempFile.close() + atexit.register(lambda: os.unlink(tempFile.name)) + jrdd = self.textFile(tempFile.name, numSlices)._pipePairs([], "echo") + return PairRDD(jrdd, self, keySerializer, valSerializer) + + def textFile(self, name, numSlices=None, serializer=NopSerializer): + numSlices = numSlices or self.defaultParallelism + jrdd = self._jsc.textFile(name, numSlices) + return RDD(jrdd, self, serializer) diff --git a/pyspark/pyspark/examples/__init__.py b/pyspark/pyspark/examples/__init__.py new file mode 100644 index 0000000000..e69de29bb2 diff --git a/pyspark/pyspark/examples/kmeans.py b/pyspark/pyspark/examples/kmeans.py new file mode 100644 index 0000000000..0761d6e395 --- /dev/null +++ b/pyspark/pyspark/examples/kmeans.py @@ -0,0 +1,56 @@ +import sys + +from pyspark.context import SparkContext + + +def parseVector(line): + return [float(x) for x in line.split(' ')] + + +def addVec(x, y): + return [a + b for (a, b) in zip(x, y)] + + +def squaredDist(x, y): + return sum((a - b) ** 2 for (a, b) in zip(x, y)) + + +def closestPoint(p, centers): + bestIndex = 0 + closest = float("+inf") + for i in range(len(centers)): + tempDist = squaredDist(p, centers[i]) + if tempDist < closest: + closest = tempDist + bestIndex = i + return bestIndex + + +if __name__ == "__main__": + if len(sys.argv) < 5: + print >> sys.stderr, \ + "Usage: PythonKMeans " + exit(-1) + sc = SparkContext(sys.argv[1], "PythonKMeans") + lines = sc.textFile(sys.argv[2]) + data = lines.map(parseVector).cache() + K = int(sys.argv[3]) + convergeDist = float(sys.argv[4]) + + kPoints = data.takeSample(False, K, 34) + tempDist = 1.0 + + while tempDist > convergeDist: + closest = data.mapPairs( + lambda p : (closestPoint(p, kPoints), (p, 1))) + pointStats = closest.reduceByKey( + lambda (x1, y1), (x2, y2): (addVec(x1, x2), y1 + y2)) + newPoints = pointStats.mapPairs( + lambda (x, (y, z)): (x, [a / z for a in y])).collect() + + tempDist = sum(squaredDist(kPoints[x], y) for (x, y) in newPoints) + + for (x, y) in newPoints: + kPoints[x] = y + + print "Final centers: " + str(kPoints) diff --git a/pyspark/pyspark/examples/pi.py b/pyspark/pyspark/examples/pi.py new file mode 100644 index 0000000000..ad77694c41 --- /dev/null +++ b/pyspark/pyspark/examples/pi.py @@ -0,0 +1,20 @@ +import sys +from random import random +from operator import add +from pyspark.context import SparkContext + + +if __name__ == "__main__": + if len(sys.argv) == 1: + print >> sys.stderr, \ + "Usage: PythonPi []" + exit(-1) + sc = SparkContext(sys.argv[1], "PythonKMeans") + slices = sys.argv[2] if len(sys.argv) > 2 else 2 + n = 100000 * slices + def f(_): + x = random() * 2 - 1 + y = random() * 2 - 1 + return 1 if x ** 2 + y ** 2 < 1 else 0 + count = sc.parallelize(xrange(1, n+1), slices).map(f).reduce(add) + print "Pi is roughly %f" % (4.0 * count / n) diff --git a/pyspark/pyspark/examples/tc.py b/pyspark/pyspark/examples/tc.py new file mode 100644 index 0000000000..2796fdc6ad --- /dev/null +++ b/pyspark/pyspark/examples/tc.py @@ -0,0 +1,49 @@ +import sys +from random import Random +from pyspark.context import SparkContext + +numEdges = 200 +numVertices = 100 +rand = Random(42) + + +def generateGraph(): + edges = set() + while len(edges) < numEdges: + src = rand.randrange(0, numEdges) + dst = rand.randrange(0, numEdges) + if src != dst: + edges.add((src, dst)) + return edges + + +if __name__ == "__main__": + if len(sys.argv) == 1: + print >> sys.stderr, \ + "Usage: PythonTC []" + exit(-1) + sc = SparkContext(sys.argv[1], "PythonKMeans") + slices = sys.argv[2] if len(sys.argv) > 2 else 2 + tc = sc.parallelizePairs(generateGraph(), slices).cache() + + # Linear transitive closure: each round grows paths by one edge, + # by joining the graph's edges with the already-discovered paths. + # e.g. join the path (y, z) from the TC with the edge (x, y) from + # the graph to obtain the path (x, z). + + # Because join() joins on keys, the edges are stored in reversed order. + edges = tc.mapPairs(lambda (x, y): (y, x)) + + oldCount = 0L + nextCount = tc.count() + while True: + oldCount = nextCount + # Perform the join, obtaining an RDD of (y, (z, x)) pairs, + # then project the result to obtain the new (x, z) paths. + new_edges = tc.join(edges).mapPairs(lambda (_, (a, b)): (b, a)) + tc = tc.union(new_edges).distinct().cache() + nextCount = tc.count() + if nextCount == oldCount: + break + + print "TC has %i edges" % tc.count() diff --git a/pyspark/pyspark/java_gateway.py b/pyspark/pyspark/java_gateway.py new file mode 100644 index 0000000000..2df80aee85 --- /dev/null +++ b/pyspark/pyspark/java_gateway.py @@ -0,0 +1,20 @@ +import glob +import os +from py4j.java_gateway import java_import, JavaGateway + + +SPARK_HOME = os.environ["SPARK_HOME"] + + +assembly_jar = glob.glob(os.path.join(SPARK_HOME, "core/target") + \ + "/spark-core-assembly-*-SNAPSHOT.jar")[0] + + +def launch_gateway(): + gateway = JavaGateway.launch_gateway(classpath=assembly_jar, + javaopts=["-Xmx256m"], die_on_exit=True) + java_import(gateway.jvm, "spark.api.java.*") + java_import(gateway.jvm, "spark.api.python.*") + java_import(gateway.jvm, "scala.Tuple2") + java_import(gateway.jvm, "spark.api.python.PythonRDD.pythonDump") + return gateway diff --git a/pyspark/pyspark/join.py b/pyspark/pyspark/join.py new file mode 100644 index 0000000000..c67520fce8 --- /dev/null +++ b/pyspark/pyspark/join.py @@ -0,0 +1,104 @@ +""" +Copyright (c) 2011, Douban Inc. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are +met: + + * Redistributions of source code must retain the above copyright +notice, this list of conditions and the following disclaimer. + + * Redistributions in binary form must reproduce the above +copyright notice, this list of conditions and the following disclaimer +in the documentation and/or other materials provided with the +distribution. + + * Neither the name of the Douban Inc. nor the names of its +contributors may be used to endorse or promote products derived from +this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +""" +from pyspark.serializers import PairSerializer, OptionSerializer, \ + ArraySerializer + + +def _do_python_join(rdd, other, numSplits, dispatch, valSerializer): + vs = rdd.mapPairs(lambda (k, v): (k, (1, v))) + ws = other.mapPairs(lambda (k, v): (k, (2, v))) + return vs.union(ws).groupByKey(numSplits) \ + .flatMapValues(dispatch, valSerializer) + + +def python_join(rdd, other, numSplits): + def dispatch(seq): + vbuf, wbuf = [], [] + for (n, v) in seq: + if n == 1: + vbuf.append(v) + elif n == 2: + wbuf.append(v) + return [(v, w) for v in vbuf for w in wbuf] + valSerializer = PairSerializer(rdd.valSerializer, other.valSerializer) + return _do_python_join(rdd, other, numSplits, dispatch, valSerializer) + + +def python_right_outer_join(rdd, other, numSplits): + def dispatch(seq): + vbuf, wbuf = [], [] + for (n, v) in seq: + if n == 1: + vbuf.append(v) + elif n == 2: + wbuf.append(v) + if not vbuf: + vbuf.append(None) + return [(v, w) for v in vbuf for w in wbuf] + valSerializer = PairSerializer(OptionSerializer(rdd.valSerializer), + other.valSerializer) + return _do_python_join(rdd, other, numSplits, dispatch, valSerializer) + + +def python_left_outer_join(rdd, other, numSplits): + def dispatch(seq): + vbuf, wbuf = [], [] + for (n, v) in seq: + if n == 1: + vbuf.append(v) + elif n == 2: + wbuf.append(v) + if not wbuf: + wbuf.append(None) + return [(v, w) for v in vbuf for w in wbuf] + valSerializer = PairSerializer(rdd.valSerializer, + OptionSerializer(other.valSerializer)) + return _do_python_join(rdd, other, numSplits, dispatch, valSerializer) + + +def python_cogroup(rdd, other, numSplits): + resultValSerializer = PairSerializer( + ArraySerializer(rdd.valSerializer), + ArraySerializer(other.valSerializer)) + vs = rdd.mapPairs(lambda (k, v): (k, (1, v))) + ws = other.mapPairs(lambda (k, v): (k, (2, v))) + def dispatch(seq): + vbuf, wbuf = [], [] + for (n, v) in seq: + if n == 1: + vbuf.append(v) + elif n == 2: + wbuf.append(v) + return (vbuf, wbuf) + return vs.union(ws).groupByKey(numSplits) \ + .mapValues(dispatch, resultValSerializer) diff --git a/pyspark/pyspark/rdd.py b/pyspark/pyspark/rdd.py new file mode 100644 index 0000000000..c892e86b93 --- /dev/null +++ b/pyspark/pyspark/rdd.py @@ -0,0 +1,517 @@ +from base64 import standard_b64encode as b64enc +from cloud.serialization import cloudpickle +from itertools import chain + +from pyspark.serializers import PairSerializer, NopSerializer, \ + OptionSerializer, ArraySerializer +from pyspark.join import python_join, python_left_outer_join, \ + python_right_outer_join, python_cogroup + + +class RDD(object): + + def __init__(self, jrdd, ctx, serializer=None): + self._jrdd = jrdd + self.is_cached = False + self.ctx = ctx + self.serializer = serializer or ctx.defaultSerializer + + def _builder(self, jrdd, ctx): + return RDD(jrdd, ctx, self.serializer) + + @property + def id(self): + return self._jrdd.id() + + @property + def splits(self): + return self._jrdd.splits() + + @classmethod + def _get_pipe_command(cls, command, functions): + if functions and not isinstance(functions, (list, tuple)): + functions = [functions] + worker_args = [command] + for f in functions: + worker_args.append(b64enc(cloudpickle.dumps(f))) + return " ".join(worker_args) + + def cache(self): + self.is_cached = True + self._jrdd.cache() + return self + + def map(self, f, serializer=None, preservesPartitioning=False): + return MappedRDD(self, f, serializer, preservesPartitioning) + + def mapPairs(self, f, keySerializer=None, valSerializer=None, + preservesPartitioning=False): + return PairMappedRDD(self, f, keySerializer, valSerializer, + preservesPartitioning) + + def flatMap(self, f, serializer=None): + """ + >>> rdd = sc.parallelize([2, 3, 4]) + >>> sorted(rdd.flatMap(lambda x: range(1, x)).collect()) + [1, 1, 1, 2, 2, 3] + """ + serializer = serializer or self.ctx.defaultSerializer + dumps = serializer.dumps + loads = self.serializer.loads + def func(x): + pickled_elems = (dumps(y) for y in f(loads(x))) + return "\n".join(pickled_elems) or None + pipe_command = RDD._get_pipe_command("map", [func]) + class_manifest = self._jrdd.classManifest() + jrdd = self.ctx.jvm.PythonRDD(self._jrdd.rdd(), pipe_command, + False, self.ctx.pythonExec, + class_manifest).asJavaRDD() + return RDD(jrdd, self.ctx, serializer) + + def flatMapPairs(self, f, keySerializer=None, valSerializer=None, + preservesPartitioning=False): + """ + >>> rdd = sc.parallelize([2, 3, 4]) + >>> sorted(rdd.flatMapPairs(lambda x: [(x, x), (x, x)]).collect()) + [(2, 2), (2, 2), (3, 3), (3, 3), (4, 4), (4, 4)] + """ + keySerializer = keySerializer or self.ctx.defaultSerializer + valSerializer = valSerializer or self.ctx.defaultSerializer + dumpk = keySerializer.dumps + dumpv = valSerializer.dumps + loads = self.serializer.loads + def func(x): + pairs = f(loads(x)) + pickled_pairs = ((dumpk(k), dumpv(v)) for (k, v) in pairs) + return "\n".join(chain.from_iterable(pickled_pairs)) or None + pipe_command = RDD._get_pipe_command("map", [func]) + class_manifest = self._jrdd.classManifest() + python_rdd = self.ctx.jvm.PythonPairRDD(self._jrdd.rdd(), pipe_command, + preservesPartitioning, self.ctx.pythonExec, class_manifest) + return PairRDD(python_rdd.asJavaPairRDD(), self.ctx, keySerializer, + valSerializer) + + def filter(self, f): + """ + >>> rdd = sc.parallelize([1, 2, 3, 4, 5]) + >>> rdd.filter(lambda x: x % 2 == 0).collect() + [2, 4] + """ + loads = self.serializer.loads + def filter_func(x): return x if f(loads(x)) else None + return self._builder(self._pipe(filter_func), self.ctx) + + def _pipe(self, functions, command="map"): + class_manifest = self._jrdd.classManifest() + pipe_command = RDD._get_pipe_command(command, functions) + python_rdd = self.ctx.jvm.PythonRDD(self._jrdd.rdd(), pipe_command, + False, self.ctx.pythonExec, class_manifest) + return python_rdd.asJavaRDD() + + def _pipePairs(self, functions, command="mapPairs", + preservesPartitioning=False): + class_manifest = self._jrdd.classManifest() + pipe_command = RDD._get_pipe_command(command, functions) + python_rdd = self.ctx.jvm.PythonPairRDD(self._jrdd.rdd(), pipe_command, + preservesPartitioning, self.ctx.pythonExec, class_manifest) + return python_rdd.asJavaPairRDD() + + def distinct(self): + """ + >>> sorted(sc.parallelize([1, 1, 2, 3]).distinct().collect()) + [1, 2, 3] + """ + if self.serializer.is_comparable: + return self._builder(self._jrdd.distinct(), self.ctx) + return self.mapPairs(lambda x: (x, "")) \ + .reduceByKey(lambda x, _: x) \ + .map(lambda (x, _): x) + + def sample(self, withReplacement, fraction, seed): + jrdd = self._jrdd.sample(withReplacement, fraction, seed) + return self._builder(jrdd, self.ctx) + + def takeSample(self, withReplacement, num, seed): + vals = self._jrdd.takeSample(withReplacement, num, seed) + return [self.serializer.loads(self.ctx.python_dump(x)) for x in vals] + + def union(self, other): + """ + >>> rdd = sc.parallelize([1, 1, 2, 3]) + >>> rdd.union(rdd).collect() + [1, 1, 2, 3, 1, 1, 2, 3] + """ + return self._builder(self._jrdd.union(other._jrdd), self.ctx) + + # TODO: sort + + # TODO: Overload __add___? + + # TODO: glom + + def cartesian(self, other): + """ + >>> rdd = sc.parallelize([1, 2]) + >>> sorted(rdd.cartesian(rdd).collect()) + [(1, 1), (1, 2), (2, 1), (2, 2)] + """ + return PairRDD(self._jrdd.cartesian(other._jrdd), self.ctx) + + # numsplits + def groupBy(self, f, numSplits=None): + """ + >>> rdd = sc.parallelize([1, 1, 2, 3, 5, 8]) + >>> sorted(rdd.groupBy(lambda x: x % 2).collect()) + [(0, [2, 8]), (1, [1, 1, 3, 5])] + """ + return self.mapPairs(lambda x: (f(x), x)).groupByKey(numSplits) + + # TODO: pipe + + # TODO: mapPartitions + + def foreach(self, f): + """ + >>> def f(x): print x + >>> sc.parallelize([1, 2, 3, 4, 5]).foreach(f) + """ + self.map(f).collect() # Force evaluation + + def collect(self): + vals = self._jrdd.collect() + return [self.serializer.loads(self.ctx.python_dump(x)) for x in vals] + + def reduce(self, f, serializer=None): + """ + >>> import operator + >>> sc.parallelize([1, 2, 3, 4, 5]).reduce(operator.add) + 15 + """ + serializer = serializer or self.ctx.defaultSerializer + loads = self.serializer.loads + dumps = serializer.dumps + def reduceFunction(x, acc): + if acc is None: + return loads(x) + else: + return f(loads(x), acc) + vals = self._pipe([reduceFunction, dumps], command="reduce").collect() + return reduce(f, (serializer.loads(x) for x in vals)) + + # TODO: fold + + # TODO: aggregate + + def count(self): + """ + >>> sc.parallelize([2, 3, 4]).count() + 3L + """ + return self._jrdd.count() + + # TODO: count approx methods + + def take(self, num): + """ + >>> sc.parallelize([2, 3, 4]).take(2) + [2, 3] + """ + vals = self._jrdd.take(num) + return [self.serializer.loads(self.ctx.python_dump(x)) for x in vals] + + def first(self): + """ + >>> sc.parallelize([2, 3, 4]).first() + 2 + """ + return self.serializer.loads(self.ctx.python_dump(self._jrdd.first())) + + # TODO: saveAsTextFile + + # TODO: saveAsObjectFile + + +class PairRDD(RDD): + + def __init__(self, jrdd, ctx, keySerializer=None, valSerializer=None): + RDD.__init__(self, jrdd, ctx) + self.keySerializer = keySerializer or ctx.defaultSerializer + self.valSerializer = valSerializer or ctx.defaultSerializer + self.serializer = \ + PairSerializer(self.keySerializer, self.valSerializer) + + def _builder(self, jrdd, ctx): + return PairRDD(jrdd, ctx, self.keySerializer, self.valSerializer) + + def reduceByKey(self, func, numSplits=None): + """ + >>> x = sc.parallelizePairs([("a", 1), ("b", 1), ("a", 1)]) + >>> sorted(x.reduceByKey(lambda a, b: a + b).collect()) + [('a', 2), ('b', 1)] + """ + return self.combineByKey(lambda x: x, func, func, numSplits) + + # TODO: reduceByKeyLocally() + + # TODO: countByKey() + + # TODO: partitionBy + + def join(self, other, numSplits=None): + """ + >>> x = sc.parallelizePairs([("a", 1), ("b", 4)]) + >>> y = sc.parallelizePairs([("a", 2), ("a", 3)]) + >>> x.join(y).collect() + [('a', (1, 2)), ('a', (1, 3))] + + Check that we get a PairRDD-like object back: + >>> assert x.join(y).join + """ + assert self.keySerializer.name == other.keySerializer.name + if self.keySerializer.is_comparable: + return PairRDD(self._jrdd.join(other._jrdd), + self.ctx, self.keySerializer, + PairSerializer(self.valSerializer, other.valSerializer)) + else: + return python_join(self, other, numSplits) + + def leftOuterJoin(self, other, numSplits=None): + """ + >>> x = sc.parallelizePairs([("a", 1), ("b", 4)]) + >>> y = sc.parallelizePairs([("a", 2)]) + >>> sorted(x.leftOuterJoin(y).collect()) + [('a', (1, 2)), ('b', (4, None))] + """ + assert self.keySerializer.name == other.keySerializer.name + if self.keySerializer.is_comparable: + return PairRDD(self._jrdd.leftOuterJoin(other._jrdd), + self.ctx, self.keySerializer, + PairSerializer(self.valSerializer, + OptionSerializer(other.valSerializer))) + else: + return python_left_outer_join(self, other, numSplits) + + def rightOuterJoin(self, other, numSplits=None): + """ + >>> x = sc.parallelizePairs([("a", 1), ("b", 4)]) + >>> y = sc.parallelizePairs([("a", 2)]) + >>> sorted(y.rightOuterJoin(x).collect()) + [('a', (2, 1)), ('b', (None, 4))] + """ + assert self.keySerializer.name == other.keySerializer.name + if self.keySerializer.is_comparable: + return PairRDD(self._jrdd.rightOuterJoin(other._jrdd), + self.ctx, self.keySerializer, + PairSerializer(OptionSerializer(self.valSerializer), + other.valSerializer)) + else: + return python_right_outer_join(self, other, numSplits) + + def combineByKey(self, createCombiner, mergeValue, mergeCombiners, + numSplits=None, serializer=None): + """ + >>> x = sc.parallelizePairs([("a", 1), ("b", 1), ("a", 1)]) + >>> def f(x): return x + >>> def add(a, b): return a + str(b) + >>> sorted(x.combineByKey(str, add, add).collect()) + [('a', '11'), ('b', '1')] + """ + serializer = serializer or self.ctx.defaultSerializer + if numSplits is None: + numSplits = self.ctx.defaultParallelism + # Use hash() to create keys that are comparable in Java. + loadkv = self.serializer.loads + def pairify(kv): + # TODO: add method to deserialize only the key or value from + # a PairSerializer? + key = loadkv(kv)[0] + return (str(hash(key)), kv) + partitioner = self.ctx.jvm.spark.HashPartitioner(numSplits) + jrdd = self._pipePairs(pairify).partitionBy(partitioner) + pairified = PairRDD(jrdd, self.ctx, NopSerializer, self.serializer) + + loads = PairSerializer(NopSerializer, self.serializer).loads + dumpk = self.keySerializer.dumps + dumpc = serializer.dumps + + functions = [createCombiner, mergeValue, mergeCombiners, loads, dumpk, + dumpc] + jpairs = pairified._pipePairs(functions, "combine_by_key", + preservesPartitioning=True) + return PairRDD(jpairs, self.ctx, self.keySerializer, serializer) + + def groupByKey(self, numSplits=None): + """ + >>> x = sc.parallelizePairs([("a", 1), ("b", 1), ("a", 1)]) + >>> sorted(x.groupByKey().collect()) + [('a', [1, 1]), ('b', [1])] + """ + + def createCombiner(x): + return [x] + + def mergeValue(xs, x): + xs.append(x) + return xs + + def mergeCombiners(a, b): + return a + b + + return self.combineByKey(createCombiner, mergeValue, mergeCombiners, + numSplits) + + def collectAsMap(self): + """ + >>> m = sc.parallelizePairs([(1, 2), (3, 4)]).collectAsMap() + >>> m[1] + 2 + >>> m[3] + 4 + """ + m = self._jrdd.collectAsMap() + def loads(x): + (k, v) = x + return (self.keySerializer.loads(k), self.valSerializer.loads(v)) + return dict(loads(x) for x in m.items()) + + def flatMapValues(self, f, valSerializer=None): + flat_map_fn = lambda (k, v): ((k, x) for x in f(v)) + return self.flatMapPairs(flat_map_fn, self.keySerializer, + valSerializer, True) + + def mapValues(self, f, valSerializer=None): + map_values_fn = lambda (k, v): (k, f(v)) + return self.mapPairs(map_values_fn, self.keySerializer, valSerializer, + True) + + # TODO: support varargs cogroup of several RDDs. + def groupWith(self, other): + return self.cogroup(other) + + def cogroup(self, other, numSplits=None): + """ + >>> x = sc.parallelizePairs([("a", 1), ("b", 4)]) + >>> y = sc.parallelizePairs([("a", 2)]) + >>> x.cogroup(y).collect() + [('a', ([1], [2])), ('b', ([4], []))] + """ + assert self.keySerializer.name == other.keySerializer.name + resultValSerializer = PairSerializer( + ArraySerializer(self.valSerializer), + ArraySerializer(other.valSerializer)) + if self.keySerializer.is_comparable: + return PairRDD(self._jrdd.cogroup(other._jrdd), + self.ctx, self.keySerializer, resultValSerializer) + else: + return python_cogroup(self, other, numSplits) + + # TODO: `lookup` is disabled because we can't make direct comparisons based + # on the key; we need to compare the hash of the key to the hash of the + # keys in the pairs. This could be an expensive operation, since those + # hashes aren't retained. + + # TODO: file saving + + +class MappedRDDBase(object): + def __init__(self, prev, func, serializer, preservesPartitioning=False): + if isinstance(prev, MappedRDDBase) and not prev.is_cached: + prev_func = prev.func + self.func = lambda x: func(prev_func(x)) + self.preservesPartitioning = \ + prev.preservesPartitioning and preservesPartitioning + self._prev_jrdd = prev._prev_jrdd + self._prev_serializer = prev._prev_serializer + else: + self.func = func + self.preservesPartitioning = preservesPartitioning + self._prev_jrdd = prev._jrdd + self._prev_serializer = prev.serializer + self.serializer = serializer or prev.ctx.defaultSerializer + self.is_cached = False + self.ctx = prev.ctx + self.prev = prev + self._jrdd_val = None + + +class MappedRDD(MappedRDDBase, RDD): + """ + >>> rdd = sc.parallelize([1, 2, 3, 4]) + >>> rdd.map(lambda x: 2 * x).cache().map(lambda x: 2 * x).collect() + [4, 8, 12, 16] + >>> rdd.map(lambda x: 2 * x).map(lambda x: 2 * x).collect() + [4, 8, 12, 16] + """ + + @property + def _jrdd(self): + if not self._jrdd_val: + udf = self.func + loads = self._prev_serializer.loads + dumps = self.serializer.dumps + func = lambda x: dumps(udf(loads(x))) + pipe_command = RDD._get_pipe_command("map", [func]) + class_manifest = self._prev_jrdd.classManifest() + python_rdd = self.ctx.jvm.PythonRDD(self._prev_jrdd.rdd(), + pipe_command, self.preservesPartitioning, self.ctx.pythonExec, + class_manifest) + self._jrdd_val = python_rdd.asJavaRDD() + return self._jrdd_val + + +class PairMappedRDD(MappedRDDBase, PairRDD): + """ + >>> rdd = sc.parallelize([1, 2, 3, 4]) + >>> rdd.mapPairs(lambda x: (x, x)) \\ + ... .mapPairs(lambda (x, y): (2*x, 2*y)) \\ + ... .collect() + [(2, 2), (4, 4), (6, 6), (8, 8)] + >>> rdd.mapPairs(lambda x: (x, x)) \\ + ... .mapPairs(lambda (x, y): (2*x, 2*y)) \\ + ... .map(lambda (x, _): x).collect() + [2, 4, 6, 8] + """ + + def __init__(self, prev, func, keySerializer=None, valSerializer=None, + preservesPartitioning=False): + self.keySerializer = keySerializer or prev.ctx.defaultSerializer + self.valSerializer = valSerializer or prev.ctx.defaultSerializer + serializer = PairSerializer(self.keySerializer, self.valSerializer) + MappedRDDBase.__init__(self, prev, func, serializer, + preservesPartitioning) + + @property + def _jrdd(self): + if not self._jrdd_val: + udf = self.func + loads = self._prev_serializer.loads + dumpk = self.keySerializer.dumps + dumpv = self.valSerializer.dumps + def func(x): + (k, v) = udf(loads(x)) + return (dumpk(k), dumpv(v)) + pipe_command = RDD._get_pipe_command("mapPairs", [func]) + class_manifest = self._prev_jrdd.classManifest() + self._jrdd_val = self.ctx.jvm.PythonPairRDD(self._prev_jrdd.rdd(), + pipe_command, self.preservesPartitioning, self.ctx.pythonExec, + class_manifest).asJavaPairRDD() + return self._jrdd_val + + +def _test(): + import doctest + from pyspark.context import SparkContext + from pyspark.serializers import PickleSerializer, JSONSerializer + globs = globals().copy() + globs['sc'] = SparkContext('local', 'PythonTest', + defaultSerializer=JSONSerializer) + doctest.testmod(globs=globs) + globs['sc'].stop() + globs['sc'] = SparkContext('local', 'PythonTest', + defaultSerializer=PickleSerializer) + doctest.testmod(globs=globs) + globs['sc'].stop() + + +if __name__ == "__main__": + _test() diff --git a/pyspark/pyspark/serializers.py b/pyspark/pyspark/serializers.py new file mode 100644 index 0000000000..b113f5656b --- /dev/null +++ b/pyspark/pyspark/serializers.py @@ -0,0 +1,229 @@ +""" +Data serialization methods. + +The Spark Python API is built on top of the Spark Java API. RDDs created in +Python are stored in Java as RDDs of Strings. Python objects are automatically +serialized/deserialized, so this representation is transparent to the end-user. + +------------------ +Serializer objects +------------------ + +`Serializer` objects are used to customize how an RDD's values are serialized. + +Each `Serializer` is a named tuple with four fields: + + - A `dumps` function, for serializing a Python object to a string. + + - A `loads` function, for deserializing a Python object from a string. + + - An `is_comparable` field, True if equal Python objects are serialized to + equal strings, and False otherwise. + + - A `name` field, used to identify the Serializer. Serializers are + compared for equality by comparing their names. + +The serializer's output should be base64-encoded. + +------------------------------------------------------------------ +`is_comparable`: comparing serialized representations for equality +------------------------------------------------------------------ + +If `is_comparable` is False, the serializer's representations of equal objects +are not required to be equal: + +>>> import pickle +>>> a = {1: 0, 9: 0} +>>> b = {9: 0, 1: 0} +>>> a == b +True +>>> pickle.dumps(a) == pickle.dumps(b) +False + +RDDs with comparable serializers can use native Java implementations of +operations like join() and distinct(), which may lead to better performance by +eliminating deserialization and Python comparisons. + +The default JSONSerializer produces comparable representations of common Python +data structures. + +-------------------------------------- +Examples of serialized representations +-------------------------------------- + +The RDD transformations that use Python UDFs are implemented in terms of +a modified `PipedRDD.pipe()` function. For each record `x` in the RDD, the +`pipe()` function pipes `x.toString()` to a Python worker process, which +deserializes the string into a Python object, executes user-defined functions, +and outputs serialized Python objects. + +The regular `toString()` method returns an ambiguous representation, due to the +way that Scala `Option` instances are printed: + +>>> from context import SparkContext +>>> sc = SparkContext("local", "SerializerDocs") +>>> x = sc.parallelizePairs([("a", 1), ("b", 4)]) +>>> y = sc.parallelizePairs([("a", 2)]) + +>>> print y.rightOuterJoin(x)._jrdd.first().toString() +(ImEi,(Some(Mg==),MQ==)) + +In Java, preprocessing is performed to handle Option instances, so the Python +process receives unambiguous input: + +>>> print sc.python_dump(y.rightOuterJoin(x)._jrdd.first()) +(ImEi,(Mg==,MQ==)) + +The base64-encoding eliminates the need to escape newlines, parentheses and +other special characters. + +---------------------- +Serializer composition +---------------------- + +In order to handle nested structures, which could contain object serialized +with different serializers, the RDD module composes serializers. For example, +the serializers in the previous example are: + +>>> print x.serializer.name +PairSerializer + +>>> print y.serializer.name +PairSerializer + +>>> print y.rightOuterJoin(x).serializer.name +PairSerializer, JSONSerializer>> +""" +from base64 import standard_b64encode, standard_b64decode +from collections import namedtuple +import cPickle +import simplejson + + +Serializer = namedtuple("Serializer", + ["dumps","loads", "is_comparable", "name"]) + + +NopSerializer = Serializer(str, str, True, "NopSerializer") + + +JSONSerializer = Serializer( + lambda obj: standard_b64encode(simplejson.dumps(obj, sort_keys=True, + separators=(',', ':'))), + lambda s: simplejson.loads(standard_b64decode(s)), + True, + "JSONSerializer" +) + + +PickleSerializer = Serializer( + lambda obj: standard_b64encode(cPickle.dumps(obj)), + lambda s: cPickle.loads(standard_b64decode(s)), + False, + "PickleSerializer" +) + + +def OptionSerializer(serializer): + """ + >>> ser = OptionSerializer(NopSerializer) + >>> ser.loads(ser.dumps("Hello, World!")) + 'Hello, World!' + >>> ser.loads(ser.dumps(None)) is None + True + """ + none_placeholder = '*' + + def dumps(x): + if x is None: + return none_placeholder + else: + return serializer.dumps(x) + + def loads(x): + if x == none_placeholder: + return None + else: + return serializer.loads(x) + + name = "OptionSerializer<%s>" % serializer.name + return Serializer(dumps, loads, serializer.is_comparable, name) + + +def PairSerializer(keySerializer, valSerializer): + """ + Returns a Serializer for a (key, value) pair. + + >>> ser = PairSerializer(JSONSerializer, JSONSerializer) + >>> ser.loads(ser.dumps((1, 2))) + (1, 2) + + >>> ser = PairSerializer(JSONSerializer, ser) + >>> ser.loads(ser.dumps((1, (2, 3)))) + (1, (2, 3)) + """ + def loads(kv): + try: + (key, val) = kv[1:-1].split(',', 1) + key = keySerializer.loads(key) + val = valSerializer.loads(val) + return (key, val) + except: + print "Error in deserializing pair from '%s'" % str(kv) + raise + + def dumps(kv): + (key, val) = kv + return"(%s,%s)" % (keySerializer.dumps(key), valSerializer.dumps(val)) + is_comparable = \ + keySerializer.is_comparable and valSerializer.is_comparable + name = "PairSerializer<%s, %s>" % (keySerializer.name, valSerializer.name) + return Serializer(dumps, loads, is_comparable, name) + + +def ArraySerializer(serializer): + """ + >>> ser = ArraySerializer(JSONSerializer) + >>> ser.loads(ser.dumps([1, 2, 3, 4])) + [1, 2, 3, 4] + >>> ser = ArraySerializer(PairSerializer(JSONSerializer, PickleSerializer)) + >>> ser.loads(ser.dumps([('a', 1), ('b', 2)])) + [('a', 1), ('b', 2)] + >>> ser.loads(ser.dumps([('a', 1)])) + [('a', 1)] + >>> ser.loads(ser.dumps([])) + [] + """ + def dumps(arr): + if arr == []: + return '[]' + else: + return '[' + '|'.join(serializer.dumps(x) for x in arr) + ']' + + def loads(s): + if s == '[]': + return [] + items = s[1:-1] + if '|' in items: + items = items.split('|') + else: + items = [items] + return [serializer.loads(x) for x in items] + + name = "ArraySerializer<%s>" % serializer.name + return Serializer(dumps, loads, serializer.is_comparable, name) + + +# TODO: IntegerSerializer + + +# TODO: DoubleSerializer + + +def _test(): + import doctest + doctest.testmod() + + +if __name__ == "__main__": + _test() diff --git a/pyspark/pyspark/worker.py b/pyspark/pyspark/worker.py new file mode 100644 index 0000000000..4d4cc939c3 --- /dev/null +++ b/pyspark/pyspark/worker.py @@ -0,0 +1,97 @@ +""" +Worker that receives input from Piped RDD. +""" +import sys +from base64 import standard_b64decode +# CloudPickler needs to be imported so that depicklers are registered using the +# copy_reg module. +from cloud.serialization.cloudpickle import CloudPickler +import cPickle + + +# Redirect stdout to stderr so that users must return values from functions. +old_stdout = sys.stdout +sys.stdout = sys.stderr + + +def load_function(): + return cPickle.loads(standard_b64decode(sys.stdin.readline().strip())) + + +def output(x): + for line in x.split("\n"): + old_stdout.write(line.rstrip("\r\n") + "\n") + + +def read_input(): + for line in sys.stdin: + yield line.rstrip("\r\n") + + +def do_combine_by_key(): + create_combiner = load_function() + merge_value = load_function() + merge_combiners = load_function() # TODO: not used. + depickler = load_function() + key_pickler = load_function() + combiner_pickler = load_function() + combiners = {} + for line in read_input(): + # Discard the hashcode added in the Python combineByKey() method. + (key, value) = depickler(line)[1] + if key not in combiners: + combiners[key] = create_combiner(value) + else: + combiners[key] = merge_value(combiners[key], value) + for (key, combiner) in combiners.iteritems(): + output(key_pickler(key)) + output(combiner_pickler(combiner)) + + +def do_map(map_pairs=False): + f = load_function() + for line in read_input(): + try: + out = f(line) + if out is not None: + if map_pairs: + for x in out: + output(x) + else: + output(out) + except: + sys.stderr.write("Error processing line '%s'\n" % line) + raise + + +def do_reduce(): + f = load_function() + dumps = load_function() + acc = None + for line in read_input(): + acc = f(line, acc) + output(dumps(acc)) + + +def do_echo(): + old_stdout.writelines(sys.stdin.readlines()) + + +def main(): + command = sys.stdin.readline().strip() + if command == "map": + do_map(map_pairs=False) + elif command == "mapPairs": + do_map(map_pairs=True) + elif command == "combine_by_key": + do_combine_by_key() + elif command == "reduce": + do_reduce() + elif command == "echo": + do_echo() + else: + raise Exception("Unsupported command %s" % command) + + +if __name__ == '__main__': + main() diff --git a/pyspark/requirements.txt b/pyspark/requirements.txt new file mode 100644 index 0000000000..d9b3fe40bd --- /dev/null +++ b/pyspark/requirements.txt @@ -0,0 +1,9 @@ +# The Python API relies on some new features from the Py4J development branch. +# pip can't install Py4J from git because the setup.py file for the Python +# package is not at the root of the git repository. It may be possible to +# install Py4J from git once https://github.com/pypa/pip/pull/526 is merged. + +# git+git://github.com/bartdag/py4j.git@3dbf380d3d2cdeb9aab394454ea74d80c4aba1ea + +simplejson==2.6.1 +cloud==2.5.5 diff --git a/python/tc.py b/python/tc.py new file mode 100644 index 0000000000..5dcc4317e0 --- /dev/null +++ b/python/tc.py @@ -0,0 +1,22 @@ +from rdd import SparkContext + +sc = SparkContext("local", "PythonWordCount") +e = [(1, 2), (2, 3), (4, 1)] + +tc = sc.parallelizePairs(e) + +edges = tc.mapPairs(lambda (x, y): (y, x)) + +oldCount = 0 +nextCount = tc.count() + +def project(x): + return (x[1][1], x[1][0]) + +while nextCount != oldCount: + oldCount = nextCount + tc = tc.union(tc.join(edges).mapPairs(project)).distinct() + nextCount = tc.count() + +print "TC has %i edges" % tc.count() +print tc.collect() From 13b9514966a423f80f672f23f42ec3f0113936fd Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 19 Aug 2012 17:12:51 -0700 Subject: [PATCH 02/85] Bundle cloudpickle with pyspark. --- pyspark/pyspark/cloudpickle.py | 974 +++++++++++++++++++++++++++++++++ pyspark/pyspark/rdd.py | 2 +- pyspark/pyspark/worker.py | 2 +- pyspark/requirements.txt | 3 - 4 files changed, 976 insertions(+), 5 deletions(-) create mode 100644 pyspark/pyspark/cloudpickle.py diff --git a/pyspark/pyspark/cloudpickle.py b/pyspark/pyspark/cloudpickle.py new file mode 100644 index 0000000000..6a7c23a069 --- /dev/null +++ b/pyspark/pyspark/cloudpickle.py @@ -0,0 +1,974 @@ +""" +This class is defined to override standard pickle functionality + +The goals of it follow: +-Serialize lambdas and nested functions to compiled byte code +-Deal with main module correctly +-Deal with other non-serializable objects + +It does not include an unpickler, as standard python unpickling suffices. + +This module was extracted from the `cloud` package, developed by `PiCloud, Inc. +`_. + +Copyright (c) 2012, Regents of the University of California. +Copyright (c) 2009 `PiCloud, Inc. `_. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + * Neither the name of the University of California, Berkeley nor the + names of its contributors may be used to endorse or promote + products derived from this software without specific prior written + permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED +TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR +PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING +NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +""" + + +import operator +import os +import pickle +import struct +import sys +import types +from functools import partial +import itertools +from copy_reg import _extension_registry, _inverted_registry, _extension_cache +import new +import dis +import traceback + +#relevant opcodes +STORE_GLOBAL = chr(dis.opname.index('STORE_GLOBAL')) +DELETE_GLOBAL = chr(dis.opname.index('DELETE_GLOBAL')) +LOAD_GLOBAL = chr(dis.opname.index('LOAD_GLOBAL')) +GLOBAL_OPS = [STORE_GLOBAL, DELETE_GLOBAL, LOAD_GLOBAL] + +HAVE_ARGUMENT = chr(dis.HAVE_ARGUMENT) +EXTENDED_ARG = chr(dis.EXTENDED_ARG) + +import logging +cloudLog = logging.getLogger("Cloud.Transport") + +try: + import ctypes +except (MemoryError, ImportError): + logging.warning('Exception raised on importing ctypes. Likely python bug.. some functionality will be disabled', exc_info = True) + ctypes = None + PyObject_HEAD = None +else: + + # for reading internal structures + PyObject_HEAD = [ + ('ob_refcnt', ctypes.c_size_t), + ('ob_type', ctypes.c_void_p), + ] + + +try: + from cStringIO import StringIO +except ImportError: + from StringIO import StringIO + +# These helper functions were copied from PiCloud's util module. +def islambda(func): + return getattr(func,'func_name') == '' + +def xrange_params(xrangeobj): + """Returns a 3 element tuple describing the xrange start, step, and len + respectively + + Note: Only guarentees that elements of xrange are the same. parameters may + be different. + e.g. xrange(1,1) is interpretted as xrange(0,0); both behave the same + though w/ iteration + """ + + xrange_len = len(xrangeobj) + if not xrange_len: #empty + return (0,1,0) + start = xrangeobj[0] + if xrange_len == 1: #one element + return start, 1, 1 + return (start, xrangeobj[1] - xrangeobj[0], xrange_len) + +#debug variables intended for developer use: +printSerialization = False +printMemoization = False + +useForcedImports = True #Should I use forced imports for tracking? + + + +class CloudPickler(pickle.Pickler): + + dispatch = pickle.Pickler.dispatch.copy() + savedForceImports = False + savedDjangoEnv = False #hack tro transport django environment + + def __init__(self, file, protocol=None, min_size_to_save= 0): + pickle.Pickler.__init__(self,file,protocol) + self.modules = set() #set of modules needed to depickle + self.globals_ref = {} # map ids to dictionary. used to ensure that functions can share global env + + def dump(self, obj): + # note: not thread safe + # minimal side-effects, so not fixing + recurse_limit = 3000 + base_recurse = sys.getrecursionlimit() + if base_recurse < recurse_limit: + sys.setrecursionlimit(recurse_limit) + self.inject_addons() + try: + return pickle.Pickler.dump(self, obj) + except RuntimeError, e: + if 'recursion' in e.args[0]: + msg = """Could not pickle object as excessively deep recursion required. + Try _fast_serialization=2 or contact PiCloud support""" + raise pickle.PicklingError(msg) + finally: + new_recurse = sys.getrecursionlimit() + if new_recurse == recurse_limit: + sys.setrecursionlimit(base_recurse) + + def save_buffer(self, obj): + """Fallback to save_string""" + pickle.Pickler.save_string(self,str(obj)) + dispatch[buffer] = save_buffer + + #block broken objects + def save_unsupported(self, obj, pack=None): + raise pickle.PicklingError("Cannot pickle objects of type %s" % type(obj)) + dispatch[types.GeneratorType] = save_unsupported + + #python2.6+ supports slice pickling. some py2.5 extensions might as well. We just test it + try: + slice(0,1).__reduce__() + except TypeError: #can't pickle - + dispatch[slice] = save_unsupported + + #itertools objects do not pickle! + for v in itertools.__dict__.values(): + if type(v) is type: + dispatch[v] = save_unsupported + + + def save_dict(self, obj): + """hack fix + If the dict is a global, deal with it in a special way + """ + #print 'saving', obj + if obj is __builtins__: + self.save_reduce(_get_module_builtins, (), obj=obj) + else: + pickle.Pickler.save_dict(self, obj) + dispatch[pickle.DictionaryType] = save_dict + + + def save_module(self, obj, pack=struct.pack): + """ + Save a module as an import + """ + #print 'try save import', obj.__name__ + self.modules.add(obj) + self.save_reduce(subimport,(obj.__name__,), obj=obj) + dispatch[types.ModuleType] = save_module #new type + + def save_codeobject(self, obj, pack=struct.pack): + """ + Save a code object + """ + #print 'try to save codeobj: ', obj + args = ( + obj.co_argcount, obj.co_nlocals, obj.co_stacksize, obj.co_flags, obj.co_code, + obj.co_consts, obj.co_names, obj.co_varnames, obj.co_filename, obj.co_name, + obj.co_firstlineno, obj.co_lnotab, obj.co_freevars, obj.co_cellvars + ) + self.save_reduce(types.CodeType, args, obj=obj) + dispatch[types.CodeType] = save_codeobject #new type + + def save_function(self, obj, name=None, pack=struct.pack): + """ Registered with the dispatch to handle all function types. + + Determines what kind of function obj is (e.g. lambda, defined at + interactive prompt, etc) and handles the pickling appropriately. + """ + write = self.write + + name = obj.__name__ + modname = pickle.whichmodule(obj, name) + #print 'which gives %s %s %s' % (modname, obj, name) + try: + themodule = sys.modules[modname] + except KeyError: # eval'd items such as namedtuple give invalid items for their function __module__ + modname = '__main__' + + if modname == '__main__': + themodule = None + + if themodule: + self.modules.add(themodule) + + if not self.savedDjangoEnv: + #hack for django - if we detect the settings module, we transport it + django_settings = os.environ.get('DJANGO_SETTINGS_MODULE', '') + if django_settings: + django_mod = sys.modules.get(django_settings) + if django_mod: + cloudLog.debug('Transporting django settings %s during save of %s', django_mod, name) + self.savedDjangoEnv = True + self.modules.add(django_mod) + write(pickle.MARK) + self.save_reduce(django_settings_load, (django_mod.__name__,), obj=django_mod) + write(pickle.POP_MARK) + + + # if func is lambda, def'ed at prompt, is in main, or is nested, then + # we'll pickle the actual function object rather than simply saving a + # reference (as is done in default pickler), via save_function_tuple. + if islambda(obj) or obj.func_code.co_filename == '' or themodule == None: + #Force server to import modules that have been imported in main + modList = None + if themodule == None and not self.savedForceImports: + mainmod = sys.modules['__main__'] + if useForcedImports and hasattr(mainmod,'___pyc_forcedImports__'): + modList = list(mainmod.___pyc_forcedImports__) + self.savedForceImports = True + self.save_function_tuple(obj, modList) + return + else: # func is nested + klass = getattr(themodule, name, None) + if klass is None or klass is not obj: + self.save_function_tuple(obj, [themodule]) + return + + if obj.__dict__: + # essentially save_reduce, but workaround needed to avoid recursion + self.save(_restore_attr) + write(pickle.MARK + pickle.GLOBAL + modname + '\n' + name + '\n') + self.memoize(obj) + self.save(obj.__dict__) + write(pickle.TUPLE + pickle.REDUCE) + else: + write(pickle.GLOBAL + modname + '\n' + name + '\n') + self.memoize(obj) + dispatch[types.FunctionType] = save_function + + def save_function_tuple(self, func, forced_imports): + """ Pickles an actual func object. + + A func comprises: code, globals, defaults, closure, and dict. We + extract and save these, injecting reducing functions at certain points + to recreate the func object. Keep in mind that some of these pieces + can contain a ref to the func itself. Thus, a naive save on these + pieces could trigger an infinite loop of save's. To get around that, + we first create a skeleton func object using just the code (this is + safe, since this won't contain a ref to the func), and memoize it as + soon as it's created. The other stuff can then be filled in later. + """ + save = self.save + write = self.write + + # save the modules (if any) + if forced_imports: + write(pickle.MARK) + save(_modules_to_main) + #print 'forced imports are', forced_imports + + forced_names = map(lambda m: m.__name__, forced_imports) + save((forced_names,)) + + #save((forced_imports,)) + write(pickle.REDUCE) + write(pickle.POP_MARK) + + code, f_globals, defaults, closure, dct, base_globals = self.extract_func_data(func) + + save(_fill_function) # skeleton function updater + write(pickle.MARK) # beginning of tuple that _fill_function expects + + # create a skeleton function object and memoize it + save(_make_skel_func) + save((code, len(closure), base_globals)) + write(pickle.REDUCE) + self.memoize(func) + + # save the rest of the func data needed by _fill_function + save(f_globals) + save(defaults) + save(closure) + save(dct) + write(pickle.TUPLE) + write(pickle.REDUCE) # applies _fill_function on the tuple + + @staticmethod + def extract_code_globals(co): + """ + Find all globals names read or written to by codeblock co + """ + code = co.co_code + names = co.co_names + out_names = set() + + n = len(code) + i = 0 + extended_arg = 0 + while i < n: + op = code[i] + + i = i+1 + if op >= HAVE_ARGUMENT: + oparg = ord(code[i]) + ord(code[i+1])*256 + extended_arg + extended_arg = 0 + i = i+2 + if op == EXTENDED_ARG: + extended_arg = oparg*65536L + if op in GLOBAL_OPS: + out_names.add(names[oparg]) + #print 'extracted', out_names, ' from ', names + return out_names + + def extract_func_data(self, func): + """ + Turn the function into a tuple of data necessary to recreate it: + code, globals, defaults, closure, dict + """ + code = func.func_code + + # extract all global ref's + func_global_refs = CloudPickler.extract_code_globals(code) + if code.co_consts: # see if nested function have any global refs + for const in code.co_consts: + if type(const) is types.CodeType and const.co_names: + func_global_refs = func_global_refs.union( CloudPickler.extract_code_globals(const)) + # process all variables referenced by global environment + f_globals = {} + for var in func_global_refs: + #Some names, such as class functions are not global - we don't need them + if func.func_globals.has_key(var): + f_globals[var] = func.func_globals[var] + + # defaults requires no processing + defaults = func.func_defaults + + def get_contents(cell): + try: + return cell.cell_contents + except ValueError, e: #cell is empty error on not yet assigned + raise pickle.PicklingError('Function to be pickled has free variables that are referenced before assignment in enclosing scope') + + + # process closure + if func.func_closure: + closure = map(get_contents, func.func_closure) + else: + closure = [] + + # save the dict + dct = func.func_dict + + if printSerialization: + outvars = ['code: ' + str(code) ] + outvars.append('globals: ' + str(f_globals)) + outvars.append('defaults: ' + str(defaults)) + outvars.append('closure: ' + str(closure)) + print 'function ', func, 'is extracted to: ', ', '.join(outvars) + + base_globals = self.globals_ref.get(id(func.func_globals), {}) + self.globals_ref[id(func.func_globals)] = base_globals + + return (code, f_globals, defaults, closure, dct, base_globals) + + def save_global(self, obj, name=None, pack=struct.pack): + write = self.write + memo = self.memo + + if name is None: + name = obj.__name__ + + modname = getattr(obj, "__module__", None) + if modname is None: + modname = pickle.whichmodule(obj, name) + + try: + __import__(modname) + themodule = sys.modules[modname] + except (ImportError, KeyError, AttributeError): #should never occur + raise pickle.PicklingError( + "Can't pickle %r: Module %s cannot be found" % + (obj, modname)) + + if modname == '__main__': + themodule = None + + if themodule: + self.modules.add(themodule) + + sendRef = True + typ = type(obj) + #print 'saving', obj, typ + try: + try: #Deal with case when getattribute fails with exceptions + klass = getattr(themodule, name) + except (AttributeError): + if modname == '__builtin__': #new.* are misrepeported + modname = 'new' + __import__(modname) + themodule = sys.modules[modname] + try: + klass = getattr(themodule, name) + except AttributeError, a: + #print themodule, name, obj, type(obj) + raise pickle.PicklingError("Can't pickle builtin %s" % obj) + else: + raise + + except (ImportError, KeyError, AttributeError): + if typ == types.TypeType or typ == types.ClassType: + sendRef = False + else: #we can't deal with this + raise + else: + if klass is not obj and (typ == types.TypeType or typ == types.ClassType): + sendRef = False + if not sendRef: + #note: Third party types might crash this - add better checks! + d = dict(obj.__dict__) #copy dict proxy to a dict + if not isinstance(d.get('__dict__', None), property): # don't extract dict that are properties + d.pop('__dict__',None) + d.pop('__weakref__',None) + + # hack as __new__ is stored differently in the __dict__ + new_override = d.get('__new__', None) + if new_override: + d['__new__'] = obj.__new__ + + self.save_reduce(type(obj),(obj.__name__,obj.__bases__, + d),obj=obj) + #print 'internal reduce dask %s %s' % (obj, d) + return + + if self.proto >= 2: + code = _extension_registry.get((modname, name)) + if code: + assert code > 0 + if code <= 0xff: + write(pickle.EXT1 + chr(code)) + elif code <= 0xffff: + write("%c%c%c" % (pickle.EXT2, code&0xff, code>>8)) + else: + write(pickle.EXT4 + pack("= 2 and getattr(func, "__name__", "") == "__newobj__": + #Added fix to allow transient + cls = args[0] + if not hasattr(cls, "__new__"): + raise pickle.PicklingError( + "args[0] from __newobj__ args has no __new__") + if obj is not None and cls is not obj.__class__: + raise pickle.PicklingError( + "args[0] from __newobj__ args has the wrong class") + args = args[1:] + save(cls) + + #Don't pickle transient entries + if hasattr(obj, '__transient__'): + transient = obj.__transient__ + state = state.copy() + + for k in list(state.keys()): + if k in transient: + del state[k] + + save(args) + write(pickle.NEWOBJ) + else: + save(func) + save(args) + write(pickle.REDUCE) + + if obj is not None: + self.memoize(obj) + + # More new special cases (that work with older protocols as + # well): when __reduce__ returns a tuple with 4 or 5 items, + # the 4th and 5th item should be iterators that provide list + # items and dict items (as (key, value) tuples), or None. + + if listitems is not None: + self._batch_appends(listitems) + + if dictitems is not None: + self._batch_setitems(dictitems) + + if state is not None: + #print 'obj %s has state %s' % (obj, state) + save(state) + write(pickle.BUILD) + + + def save_xrange(self, obj): + """Save an xrange object in python 2.5 + Python 2.6 supports this natively + """ + range_params = xrange_params(obj) + self.save_reduce(_build_xrange,range_params) + + #python2.6+ supports xrange pickling. some py2.5 extensions might as well. We just test it + try: + xrange(0).__reduce__() + except TypeError: #can't pickle -- use PiCloud pickler + dispatch[xrange] = save_xrange + + def save_partial(self, obj): + """Partial objects do not serialize correctly in python2.x -- this fixes the bugs""" + self.save_reduce(_genpartial, (obj.func, obj.args, obj.keywords)) + + if sys.version_info < (2,7): #2.7 supports partial pickling + dispatch[partial] = save_partial + + + def save_file(self, obj): + """Save a file""" + import StringIO as pystringIO #we can't use cStringIO as it lacks the name attribute + from ..transport.adapter import SerializingAdapter + + if not hasattr(obj, 'name') or not hasattr(obj, 'mode'): + raise pickle.PicklingError("Cannot pickle files that do not map to an actual file") + if obj.name == '': + return self.save_reduce(getattr, (sys,'stdout'), obj=obj) + if obj.name == '': + return self.save_reduce(getattr, (sys,'stderr'), obj=obj) + if obj.name == '': + raise pickle.PicklingError("Cannot pickle standard input") + if hasattr(obj, 'isatty') and obj.isatty(): + raise pickle.PicklingError("Cannot pickle files that map to tty objects") + if 'r' not in obj.mode: + raise pickle.PicklingError("Cannot pickle files that are not opened for reading") + name = obj.name + try: + fsize = os.stat(name).st_size + except OSError: + raise pickle.PicklingError("Cannot pickle file %s as it cannot be stat" % name) + + if obj.closed: + #create an empty closed string io + retval = pystringIO.StringIO("") + retval.close() + elif not fsize: #empty file + retval = pystringIO.StringIO("") + try: + tmpfile = file(name) + tst = tmpfile.read(1) + except IOError: + raise pickle.PicklingError("Cannot pickle file %s as it cannot be read" % name) + tmpfile.close() + if tst != '': + raise pickle.PicklingError("Cannot pickle file %s as it does not appear to map to a physical, real file" % name) + elif fsize > SerializingAdapter.max_transmit_data: + raise pickle.PicklingError("Cannot pickle file %s as it exceeds cloudconf.py's max_transmit_data of %d" % + (name,SerializingAdapter.max_transmit_data)) + else: + try: + tmpfile = file(name) + contents = tmpfile.read(SerializingAdapter.max_transmit_data) + tmpfile.close() + except IOError: + raise pickle.PicklingError("Cannot pickle file %s as it cannot be read" % name) + retval = pystringIO.StringIO(contents) + curloc = obj.tell() + retval.seek(curloc) + + retval.name = name + self.save(retval) #save stringIO + self.memoize(obj) + + dispatch[file] = save_file + """Special functions for Add-on libraries""" + + def inject_numpy(self): + numpy = sys.modules.get('numpy') + if not numpy or not hasattr(numpy, 'ufunc'): + return + self.dispatch[numpy.ufunc] = self.__class__.save_ufunc + + numpy_tst_mods = ['numpy', 'scipy.special'] + def save_ufunc(self, obj): + """Hack function for saving numpy ufunc objects""" + name = obj.__name__ + for tst_mod_name in self.numpy_tst_mods: + tst_mod = sys.modules.get(tst_mod_name, None) + if tst_mod: + if name in tst_mod.__dict__: + self.save_reduce(_getobject, (tst_mod_name, name)) + return + raise pickle.PicklingError('cannot save %s. Cannot resolve what module it is defined in' % str(obj)) + + def inject_timeseries(self): + """Handle bugs with pickling scikits timeseries""" + tseries = sys.modules.get('scikits.timeseries.tseries') + if not tseries or not hasattr(tseries, 'Timeseries'): + return + self.dispatch[tseries.Timeseries] = self.__class__.save_timeseries + + def save_timeseries(self, obj): + import scikits.timeseries.tseries as ts + + func, reduce_args, state = obj.__reduce__() + if func != ts._tsreconstruct: + raise pickle.PicklingError('timeseries using unexpected reconstruction function %s' % str(func)) + state = (1, + obj.shape, + obj.dtype, + obj.flags.fnc, + obj._data.tostring(), + ts.getmaskarray(obj).tostring(), + obj._fill_value, + obj._dates.shape, + obj._dates.__array__().tostring(), + obj._dates.dtype, #added -- preserve type + obj.freq, + obj._optinfo, + ) + return self.save_reduce(_genTimeSeries, (reduce_args, state)) + + def inject_email(self): + """Block email LazyImporters from being saved""" + email = sys.modules.get('email') + if not email: + return + self.dispatch[email.LazyImporter] = self.__class__.save_unsupported + + def inject_addons(self): + """Plug in system. Register additional pickling functions if modules already loaded""" + self.inject_numpy() + self.inject_timeseries() + self.inject_email() + + """Python Imaging Library""" + def save_image(self, obj): + if not obj.im and obj.fp and 'r' in obj.fp.mode and obj.fp.name \ + and not obj.fp.closed and (not hasattr(obj, 'isatty') or not obj.isatty()): + #if image not loaded yet -- lazy load + self.save_reduce(_lazyloadImage,(obj.fp,), obj=obj) + else: + #image is loaded - just transmit it over + self.save_reduce(_generateImage, (obj.size, obj.mode, obj.tostring()), obj=obj) + + """ + def memoize(self, obj): + pickle.Pickler.memoize(self, obj) + if printMemoization: + print 'memoizing ' + str(obj) + """ + + + +# Shorthands for legacy support + +def dump(obj, file, protocol=2): + CloudPickler(file, protocol).dump(obj) + +def dumps(obj, protocol=2): + file = StringIO() + + cp = CloudPickler(file,protocol) + cp.dump(obj) + + #print 'cloud dumped', str(obj), str(cp.modules) + + return file.getvalue() + + +#hack for __import__ not working as desired +def subimport(name): + __import__(name) + return sys.modules[name] + +#hack to load django settings: +def django_settings_load(name): + modified_env = False + + if 'DJANGO_SETTINGS_MODULE' not in os.environ: + os.environ['DJANGO_SETTINGS_MODULE'] = name # must set name first due to circular deps + modified_env = True + try: + module = subimport(name) + except Exception, i: + print >> sys.stderr, 'Cloud not import django settings %s:' % (name) + print_exec(sys.stderr) + if modified_env: + del os.environ['DJANGO_SETTINGS_MODULE'] + else: + #add project directory to sys,path: + if hasattr(module,'__file__'): + dirname = os.path.split(module.__file__)[0] + '/' + sys.path.append(dirname) + +# restores function attributes +def _restore_attr(obj, attr): + for key, val in attr.items(): + setattr(obj, key, val) + return obj + +def _get_module_builtins(): + return pickle.__builtins__ + +def print_exec(stream): + ei = sys.exc_info() + traceback.print_exception(ei[0], ei[1], ei[2], None, stream) + +def _modules_to_main(modList): + """Force every module in modList to be placed into main""" + if not modList: + return + + main = sys.modules['__main__'] + for modname in modList: + if type(modname) is str: + try: + mod = __import__(modname) + except Exception, i: #catch all... + sys.stderr.write('warning: could not import %s\n. Your function may unexpectedly error due to this import failing; \ +A version mismatch is likely. Specific error was:\n' % modname) + print_exec(sys.stderr) + else: + setattr(main,mod.__name__, mod) + else: + #REVERSE COMPATIBILITY FOR CLOUD CLIENT 1.5 (WITH EPD) + #In old version actual module was sent + setattr(main,modname.__name__, modname) + +#object generators: +def _build_xrange(start, step, len): + """Built xrange explicitly""" + return xrange(start, start + step*len, step) + +def _genpartial(func, args, kwds): + if not args: + args = () + if not kwds: + kwds = {} + return partial(func, *args, **kwds) + + +def _fill_function(func, globals, defaults, closure, dict): + """ Fills in the rest of function data into the skeleton function object + that were created via _make_skel_func(). + """ + func.func_globals.update(globals) + func.func_defaults = defaults + func.func_dict = dict + + if len(closure) != len(func.func_closure): + raise pickle.UnpicklingError("closure lengths don't match up") + for i in range(len(closure)): + _change_cell_value(func.func_closure[i], closure[i]) + + return func + +def _make_skel_func(code, num_closures, base_globals = None): + """ Creates a skeleton function object that contains just the provided + code and the correct number of cells in func_closure. All other + func attributes (e.g. func_globals) are empty. + """ + #build closure (cells): + if not ctypes: + raise Exception('ctypes failed to import; cannot build function') + + cellnew = ctypes.pythonapi.PyCell_New + cellnew.restype = ctypes.py_object + cellnew.argtypes = (ctypes.py_object,) + dummy_closure = tuple(map(lambda i: cellnew(None), range(num_closures))) + + if base_globals is None: + base_globals = {} + base_globals['__builtins__'] = __builtins__ + + return types.FunctionType(code, base_globals, + None, None, dummy_closure) + +# this piece of opaque code is needed below to modify 'cell' contents +cell_changer_code = new.code( + 1, 1, 2, 0, + ''.join([ + chr(dis.opmap['LOAD_FAST']), '\x00\x00', + chr(dis.opmap['DUP_TOP']), + chr(dis.opmap['STORE_DEREF']), '\x00\x00', + chr(dis.opmap['RETURN_VALUE']) + ]), + (), (), ('newval',), '', 'cell_changer', 1, '', ('c',), () +) + +def _change_cell_value(cell, newval): + """ Changes the contents of 'cell' object to newval """ + return new.function(cell_changer_code, {}, None, (), (cell,))(newval) + +"""Constructors for 3rd party libraries +Note: These can never be renamed due to client compatibility issues""" + +def _getobject(modname, attribute): + mod = __import__(modname) + return mod.__dict__[attribute] + +def _generateImage(size, mode, str_rep): + """Generate image from string representation""" + import Image + i = Image.new(mode, size) + i.fromstring(str_rep) + return i + +def _lazyloadImage(fp): + import Image + fp.seek(0) #works in almost any case + return Image.open(fp) + +"""Timeseries""" +def _genTimeSeries(reduce_args, state): + import scikits.timeseries.tseries as ts + from numpy import ndarray + from numpy.ma import MaskedArray + + + time_series = ts._tsreconstruct(*reduce_args) + + #from setstate modified + (ver, shp, typ, isf, raw, msk, flv, dsh, dtm, dtyp, frq, infodict) = state + #print 'regenerating %s' % dtyp + + MaskedArray.__setstate__(time_series, (ver, shp, typ, isf, raw, msk, flv)) + _dates = time_series._dates + #_dates.__setstate__((ver, dsh, typ, isf, dtm, frq)) #use remote typ + ndarray.__setstate__(_dates,(dsh,dtyp, isf, dtm)) + _dates.freq = frq + _dates._cachedinfo.update(dict(full=None, hasdups=None, steps=None, + toobj=None, toord=None, tostr=None)) + # Update the _optinfo dictionary + time_series._optinfo.update(infodict) + return time_series + diff --git a/pyspark/pyspark/rdd.py b/pyspark/pyspark/rdd.py index c892e86b93..5579c56de3 100644 --- a/pyspark/pyspark/rdd.py +++ b/pyspark/pyspark/rdd.py @@ -1,5 +1,5 @@ from base64 import standard_b64encode as b64enc -from cloud.serialization import cloudpickle +from pyspark import cloudpickle from itertools import chain from pyspark.serializers import PairSerializer, NopSerializer, \ diff --git a/pyspark/pyspark/worker.py b/pyspark/pyspark/worker.py index 4d4cc939c3..4c4b02fce4 100644 --- a/pyspark/pyspark/worker.py +++ b/pyspark/pyspark/worker.py @@ -5,7 +5,7 @@ import sys from base64 import standard_b64decode # CloudPickler needs to be imported so that depicklers are registered using the # copy_reg module. -from cloud.serialization.cloudpickle import CloudPickler +from pyspark.cloudpickle import CloudPickler import cPickle diff --git a/pyspark/requirements.txt b/pyspark/requirements.txt index d9b3fe40bd..71e2bc2b89 100644 --- a/pyspark/requirements.txt +++ b/pyspark/requirements.txt @@ -4,6 +4,3 @@ # install Py4J from git once https://github.com/pypa/pip/pull/526 is merged. # git+git://github.com/bartdag/py4j.git@3dbf380d3d2cdeb9aab394454ea74d80c4aba1ea - -simplejson==2.6.1 -cloud==2.5.5 From fd94e5443c99775bfad1928729f5075c900ad0f9 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 18 Aug 2012 16:07:10 -0700 Subject: [PATCH 03/85] Use only cPickle for serialization in Python API. Objects serialized with JSON can be compared for equality, but JSON can be slow to serialize and only supports a limited range of data types. --- .../scala/spark/api/python/PythonRDD.scala | 192 ++++++-- pyspark/pyspark/context.py | 53 +-- pyspark/pyspark/java_gateway.py | 1 - pyspark/pyspark/join.py | 32 +- pyspark/pyspark/rdd.py | 422 +++++++----------- pyspark/pyspark/serializers.py | 233 +--------- pyspark/pyspark/worker.py | 64 +-- 7 files changed, 387 insertions(+), 610 deletions(-) diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 660ad48afe..b9a0168d18 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -1,22 +1,26 @@ package spark.api.python -import java.io.PrintWriter +import java.io._ import scala.collection.Map import scala.collection.JavaConversions._ import scala.io.Source import spark._ -import api.java.{JavaPairRDD, JavaRDD} +import api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} +import scala.{collection, Some} +import collection.parallel.mutable +import scala.collection import scala.Some trait PythonRDDBase { def compute[T](split: Split, envVars: Map[String, String], - command: Seq[String], parent: RDD[T], pythonExec: String): Iterator[String]= { - val currentEnvVars = new ProcessBuilder().environment() - val SPARK_HOME = currentEnvVars.get("SPARK_HOME") + command: Seq[String], parent: RDD[T], pythonExec: String): Iterator[Array[Byte]] = { + val SPARK_HOME = new ProcessBuilder().environment().get("SPARK_HOME") val pb = new ProcessBuilder(Seq(pythonExec, SPARK_HOME + "/pyspark/pyspark/worker.py")) // Add the environmental variables to the process. + val currentEnvVars = pb.environment() + envVars.foreach { case (variable, value) => currentEnvVars.put(variable, value) } @@ -41,33 +45,70 @@ trait PythonRDDBase { for (elem <- command) { out.println(elem) } + out.flush() + val dOut = new DataOutputStream(proc.getOutputStream) for (elem <- parent.iterator(split)) { - out.println(PythonRDD.pythonDump(elem)) + if (elem.isInstanceOf[Array[Byte]]) { + val arr = elem.asInstanceOf[Array[Byte]] + dOut.writeInt(arr.length) + dOut.write(arr) + } else if (elem.isInstanceOf[scala.Tuple2[_, _]]) { + val t = elem.asInstanceOf[scala.Tuple2[_, _]] + val t1 = t._1.asInstanceOf[Array[Byte]] + val t2 = t._2.asInstanceOf[Array[Byte]] + val length = t1.length + t2.length - 3 - 3 + 4 // stripPickle() removes 3 bytes + dOut.writeInt(length) + dOut.writeByte(Pickle.PROTO) + dOut.writeByte(Pickle.TWO) + dOut.write(PythonRDD.stripPickle(t1)) + dOut.write(PythonRDD.stripPickle(t2)) + dOut.writeByte(Pickle.TUPLE2) + dOut.writeByte(Pickle.STOP) + } else if (elem.isInstanceOf[String]) { + // For uniformity, strings are wrapped into Pickles. + val s = elem.asInstanceOf[String].getBytes("UTF-8") + val length = 2 + 1 + 4 + s.length + 1 + dOut.writeInt(length) + dOut.writeByte(Pickle.PROTO) + dOut.writeByte(Pickle.TWO) + dOut.writeByte(Pickle.BINUNICODE) + dOut.writeInt(Integer.reverseBytes(s.length)) + dOut.write(s) + dOut.writeByte(Pickle.STOP) + } else { + throw new Exception("Unexpected RDD type") + } } - out.close() + dOut.flush() + out.flush() + proc.getOutputStream.close() } }.start() // Return an iterator that read lines from the process's stdout - val lines: Iterator[String] = Source.fromInputStream(proc.getInputStream).getLines - wrapIterator(lines, proc) - } + val stream = new DataInputStream(proc.getInputStream) + return new Iterator[Array[Byte]] { + def next() = { + val obj = _nextObj + _nextObj = read() + obj + } - def wrapIterator[T](iter: Iterator[T], proc: Process): Iterator[T] = { - return new Iterator[T] { - def next() = iter.next() - - def hasNext = { - if (iter.hasNext) { - true - } else { - val exitStatus = proc.waitFor() - if (exitStatus != 0) { - throw new Exception("Subprocess exited with status " + exitStatus) - } - false + private def read() = { + try { + val length = stream.readInt() + val obj = new Array[Byte](length) + stream.readFully(obj) + obj + } catch { + case eof: EOFException => { new Array[Byte](0) } + case e => throw e } } + + var _nextObj = read() + + def hasNext = _nextObj.length != 0 } } } @@ -75,7 +116,7 @@ trait PythonRDDBase { class PythonRDD[T: ClassManifest]( parent: RDD[T], command: Seq[String], envVars: Map[String, String], preservePartitoning: Boolean, pythonExec: String) - extends RDD[String](parent.context) with PythonRDDBase { + extends RDD[Array[Byte]](parent.context) with PythonRDDBase { def this(parent: RDD[T], command: Seq[String], preservePartitoning: Boolean, pythonExec: String) = this(parent, command, Map(), preservePartitoning, pythonExec) @@ -91,16 +132,16 @@ class PythonRDD[T: ClassManifest]( override val partitioner = if (preservePartitoning) parent.partitioner else None - override def compute(split: Split): Iterator[String] = + override def compute(split: Split): Iterator[Array[Byte]] = compute(split, envVars, command, parent, pythonExec) - val asJavaRDD : JavaRDD[String] = JavaRDD.fromRDD(this) + val asJavaRDD : JavaRDD[Array[Byte]] = JavaRDD.fromRDD(this) } class PythonPairRDD[T: ClassManifest] ( parent: RDD[T], command: Seq[String], envVars: Map[String, String], preservePartitoning: Boolean, pythonExec: String) - extends RDD[(String, String)](parent.context) with PythonRDDBase { + extends RDD[(Array[Byte], Array[Byte])](parent.context) with PythonRDDBase { def this(parent: RDD[T], command: Seq[String], preservePartitoning: Boolean, pythonExec: String) = this(parent, command, Map(), preservePartitoning, pythonExec) @@ -116,32 +157,95 @@ class PythonPairRDD[T: ClassManifest] ( override val partitioner = if (preservePartitoning) parent.partitioner else None - override def compute(split: Split): Iterator[(String, String)] = { + override def compute(split: Split): Iterator[(Array[Byte], Array[Byte])] = { compute(split, envVars, command, parent, pythonExec).grouped(2).map { case Seq(a, b) => (a, b) - case x => throw new Exception("Unexpected value: " + x) + case x => throw new Exception("PythonPairRDD: unexpected value: " + x) } } - val asJavaPairRDD : JavaPairRDD[String, String] = JavaPairRDD.fromRDD(this) + val asJavaPairRDD : JavaPairRDD[Array[Byte], Array[Byte]] = JavaPairRDD.fromRDD(this) } + object PythonRDD { - def pythonDump[T](x: T): String = { - if (x.isInstanceOf[scala.Option[_]]) { - val t = x.asInstanceOf[scala.Option[_]] - t match { - case None => "*" - case Some(z) => pythonDump(z) - } - } else if (x.isInstanceOf[scala.Tuple2[_, _]]) { - val t = x.asInstanceOf[scala.Tuple2[_, _]] - "(" + pythonDump(t._1) + "," + pythonDump(t._2) + ")" - } else if (x.isInstanceOf[java.util.List[_]]) { - val objs = asScalaBuffer(x.asInstanceOf[java.util.List[_]]).map(pythonDump) - "[" + objs.mkString("|") + "]" + + /** Strips the pickle PROTO and STOP opcodes from the start and end of a pickle */ + def stripPickle(arr: Array[Byte]) : Array[Byte] = { + arr.slice(2, arr.length - 1) + } + + def asPickle(elem: Any) : Array[Byte] = { + val baos = new ByteArrayOutputStream(); + val dOut = new DataOutputStream(baos); + if (elem.isInstanceOf[Array[Byte]]) { + elem.asInstanceOf[Array[Byte]] + } else if (elem.isInstanceOf[scala.Tuple2[_, _]]) { + val t = elem.asInstanceOf[scala.Tuple2[_, _]] + val t1 = t._1.asInstanceOf[Array[Byte]] + val t2 = t._2.asInstanceOf[Array[Byte]] + dOut.writeByte(Pickle.PROTO) + dOut.writeByte(Pickle.TWO) + dOut.write(PythonRDD.stripPickle(t1)) + dOut.write(PythonRDD.stripPickle(t2)) + dOut.writeByte(Pickle.TUPLE2) + dOut.writeByte(Pickle.STOP) + baos.toByteArray() + } else if (elem.isInstanceOf[String]) { + // For uniformity, strings are wrapped into Pickles. + val s = elem.asInstanceOf[String].getBytes("UTF-8") + dOut.writeByte(Pickle.PROTO) + dOut.writeByte(Pickle.TWO) + dOut.write(Pickle.BINUNICODE) + dOut.writeInt(Integer.reverseBytes(s.length)) + dOut.write(s) + dOut.writeByte(Pickle.STOP) + baos.toByteArray() } else { - x.toString + throw new Exception("Unexpected RDD type") } } + + def pickleFile(sc: JavaSparkContext, filename: String, parallelism: Int) : + JavaRDD[Array[Byte]] = { + val file = new DataInputStream(new FileInputStream(filename)) + val objs = new collection.mutable.ArrayBuffer[Array[Byte]] + try { + while (true) { + val length = file.readInt() + val obj = new Array[Byte](length) + file.readFully(obj) + objs.append(obj) + } + } catch { + case eof: EOFException => {} + case e => throw e + } + JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism)) + } + + def arrayAsPickle(arr : Any) : Array[Byte] = { + val pickles : Array[Byte] = arr.asInstanceOf[Array[Any]].map(asPickle).map(stripPickle).flatten + + Array[Byte](Pickle.PROTO, Pickle.TWO, Pickle.EMPTY_LIST, Pickle.MARK) ++ pickles ++ + Array[Byte] (Pickle.APPENDS, Pickle.STOP) + } +} + +private object Pickle { + def b(x: Int): Byte = x.asInstanceOf[Byte] + val PROTO: Byte = b(0x80) + val TWO: Byte = b(0x02) + val BINUNICODE : Byte = 'X' + val STOP : Byte = '.' + val TUPLE2 : Byte = b(0x86) + val EMPTY_LIST : Byte = ']' + val MARK : Byte = '(' + val APPENDS : Byte = 'e' +} +class ExtractValue extends spark.api.java.function.Function[(Array[Byte], + Array[Byte]), Array[Byte]] { + + override def call(pair: (Array[Byte], Array[Byte])) : Array[Byte] = pair._2 + } diff --git a/pyspark/pyspark/context.py b/pyspark/pyspark/context.py index 587ab12b5f..ac7e4057e9 100644 --- a/pyspark/pyspark/context.py +++ b/pyspark/pyspark/context.py @@ -3,22 +3,24 @@ import atexit from tempfile import NamedTemporaryFile from pyspark.java_gateway import launch_gateway -from pyspark.serializers import JSONSerializer, NopSerializer -from pyspark.rdd import RDD, PairRDD +from pyspark.serializers import PickleSerializer, dumps +from pyspark.rdd import RDD class SparkContext(object): gateway = launch_gateway() jvm = gateway.jvm - python_dump = jvm.spark.api.python.PythonRDD.pythonDump + pickleFile = jvm.spark.api.python.PythonRDD.pickleFile + asPickle = jvm.spark.api.python.PythonRDD.asPickle + arrayAsPickle = jvm.spark.api.python.PythonRDD.arrayAsPickle - def __init__(self, master, name, defaultSerializer=JSONSerializer, - defaultParallelism=None, pythonExec='python'): + + def __init__(self, master, name, defaultParallelism=None, + pythonExec='python'): self.master = master self.name = name self._jsc = self.jvm.JavaSparkContext(master, name) - self.defaultSerializer = defaultSerializer self.defaultParallelism = \ defaultParallelism or self._jsc.sc().defaultParallelism() self.pythonExec = pythonExec @@ -31,39 +33,26 @@ class SparkContext(object): self._jsc.stop() self._jsc = None - def parallelize(self, c, numSlices=None, serializer=None): - serializer = serializer or self.defaultSerializer + def parallelize(self, c, numSlices=None): + """ + >>> sc = SparkContext("local", "test") + >>> rdd = sc.parallelize([(1, 2), (3, 4)]) + >>> rdd.collect() + [(1, 2), (3, 4)] + """ numSlices = numSlices or self.defaultParallelism # Calling the Java parallelize() method with an ArrayList is too slow, # because it sends O(n) Py4J commands. As an alternative, serialized # objects are written to a file and loaded through textFile(). tempFile = NamedTemporaryFile(delete=False) - tempFile.writelines(serializer.dumps(x) + '\n' for x in c) + for x in c: + dumps(PickleSerializer.dumps(x), tempFile) tempFile.close() atexit.register(lambda: os.unlink(tempFile.name)) - return self.textFile(tempFile.name, numSlices, serializer) + jrdd = self.pickleFile(self._jsc, tempFile.name, numSlices) + return RDD(jrdd, self) - def parallelizePairs(self, c, numSlices=None, keySerializer=None, - valSerializer=None): - """ - >>> sc = SparkContext("local", "test") - >>> rdd = sc.parallelizePairs([(1, 2), (3, 4)]) - >>> rdd.collect() - [(1, 2), (3, 4)] - """ - keySerializer = keySerializer or self.defaultSerializer - valSerializer = valSerializer or self.defaultSerializer - numSlices = numSlices or self.defaultParallelism - tempFile = NamedTemporaryFile(delete=False) - for (k, v) in c: - tempFile.write(keySerializer.dumps(k).rstrip('\r\n') + '\n') - tempFile.write(valSerializer.dumps(v).rstrip('\r\n') + '\n') - tempFile.close() - atexit.register(lambda: os.unlink(tempFile.name)) - jrdd = self.textFile(tempFile.name, numSlices)._pipePairs([], "echo") - return PairRDD(jrdd, self, keySerializer, valSerializer) - - def textFile(self, name, numSlices=None, serializer=NopSerializer): + def textFile(self, name, numSlices=None): numSlices = numSlices or self.defaultParallelism jrdd = self._jsc.textFile(name, numSlices) - return RDD(jrdd, self, serializer) + return RDD(jrdd, self) diff --git a/pyspark/pyspark/java_gateway.py b/pyspark/pyspark/java_gateway.py index 2df80aee85..bcb405ba72 100644 --- a/pyspark/pyspark/java_gateway.py +++ b/pyspark/pyspark/java_gateway.py @@ -16,5 +16,4 @@ def launch_gateway(): java_import(gateway.jvm, "spark.api.java.*") java_import(gateway.jvm, "spark.api.python.*") java_import(gateway.jvm, "scala.Tuple2") - java_import(gateway.jvm, "spark.api.python.PythonRDD.pythonDump") return gateway diff --git a/pyspark/pyspark/join.py b/pyspark/pyspark/join.py index c67520fce8..7036c47980 100644 --- a/pyspark/pyspark/join.py +++ b/pyspark/pyspark/join.py @@ -30,15 +30,12 @@ THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. """ -from pyspark.serializers import PairSerializer, OptionSerializer, \ - ArraySerializer -def _do_python_join(rdd, other, numSplits, dispatch, valSerializer): - vs = rdd.mapPairs(lambda (k, v): (k, (1, v))) - ws = other.mapPairs(lambda (k, v): (k, (2, v))) - return vs.union(ws).groupByKey(numSplits) \ - .flatMapValues(dispatch, valSerializer) +def _do_python_join(rdd, other, numSplits, dispatch): + vs = rdd.map(lambda (k, v): (k, (1, v))) + ws = other.map(lambda (k, v): (k, (2, v))) + return vs.union(ws).groupByKey(numSplits).flatMapValues(dispatch) def python_join(rdd, other, numSplits): @@ -50,8 +47,7 @@ def python_join(rdd, other, numSplits): elif n == 2: wbuf.append(v) return [(v, w) for v in vbuf for w in wbuf] - valSerializer = PairSerializer(rdd.valSerializer, other.valSerializer) - return _do_python_join(rdd, other, numSplits, dispatch, valSerializer) + return _do_python_join(rdd, other, numSplits, dispatch) def python_right_outer_join(rdd, other, numSplits): @@ -65,9 +61,7 @@ def python_right_outer_join(rdd, other, numSplits): if not vbuf: vbuf.append(None) return [(v, w) for v in vbuf for w in wbuf] - valSerializer = PairSerializer(OptionSerializer(rdd.valSerializer), - other.valSerializer) - return _do_python_join(rdd, other, numSplits, dispatch, valSerializer) + return _do_python_join(rdd, other, numSplits, dispatch) def python_left_outer_join(rdd, other, numSplits): @@ -81,17 +75,12 @@ def python_left_outer_join(rdd, other, numSplits): if not wbuf: wbuf.append(None) return [(v, w) for v in vbuf for w in wbuf] - valSerializer = PairSerializer(rdd.valSerializer, - OptionSerializer(other.valSerializer)) - return _do_python_join(rdd, other, numSplits, dispatch, valSerializer) + return _do_python_join(rdd, other, numSplits, dispatch) def python_cogroup(rdd, other, numSplits): - resultValSerializer = PairSerializer( - ArraySerializer(rdd.valSerializer), - ArraySerializer(other.valSerializer)) - vs = rdd.mapPairs(lambda (k, v): (k, (1, v))) - ws = other.mapPairs(lambda (k, v): (k, (2, v))) + vs = rdd.map(lambda (k, v): (k, (1, v))) + ws = other.map(lambda (k, v): (k, (2, v))) def dispatch(seq): vbuf, wbuf = [], [] for (n, v) in seq: @@ -100,5 +89,4 @@ def python_cogroup(rdd, other, numSplits): elif n == 2: wbuf.append(v) return (vbuf, wbuf) - return vs.union(ws).groupByKey(numSplits) \ - .mapValues(dispatch, resultValSerializer) + return vs.union(ws).groupByKey(numSplits).mapValues(dispatch) diff --git a/pyspark/pyspark/rdd.py b/pyspark/pyspark/rdd.py index 5579c56de3..8eccddc0a2 100644 --- a/pyspark/pyspark/rdd.py +++ b/pyspark/pyspark/rdd.py @@ -1,31 +1,17 @@ from base64 import standard_b64encode as b64enc -from pyspark import cloudpickle -from itertools import chain -from pyspark.serializers import PairSerializer, NopSerializer, \ - OptionSerializer, ArraySerializer +from pyspark import cloudpickle +from pyspark.serializers import PickleSerializer from pyspark.join import python_join, python_left_outer_join, \ python_right_outer_join, python_cogroup class RDD(object): - def __init__(self, jrdd, ctx, serializer=None): + def __init__(self, jrdd, ctx): self._jrdd = jrdd self.is_cached = False self.ctx = ctx - self.serializer = serializer or ctx.defaultSerializer - - def _builder(self, jrdd, ctx): - return RDD(jrdd, ctx, self.serializer) - - @property - def id(self): - return self._jrdd.id() - - @property - def splits(self): - return self._jrdd.splits() @classmethod def _get_pipe_command(cls, command, functions): @@ -41,55 +27,18 @@ class RDD(object): self._jrdd.cache() return self - def map(self, f, serializer=None, preservesPartitioning=False): - return MappedRDD(self, f, serializer, preservesPartitioning) + def map(self, f, preservesPartitioning=False): + return MappedRDD(self, f, preservesPartitioning) - def mapPairs(self, f, keySerializer=None, valSerializer=None, - preservesPartitioning=False): - return PairMappedRDD(self, f, keySerializer, valSerializer, - preservesPartitioning) - - def flatMap(self, f, serializer=None): + def flatMap(self, f): """ >>> rdd = sc.parallelize([2, 3, 4]) >>> sorted(rdd.flatMap(lambda x: range(1, x)).collect()) [1, 1, 1, 2, 2, 3] - """ - serializer = serializer or self.ctx.defaultSerializer - dumps = serializer.dumps - loads = self.serializer.loads - def func(x): - pickled_elems = (dumps(y) for y in f(loads(x))) - return "\n".join(pickled_elems) or None - pipe_command = RDD._get_pipe_command("map", [func]) - class_manifest = self._jrdd.classManifest() - jrdd = self.ctx.jvm.PythonRDD(self._jrdd.rdd(), pipe_command, - False, self.ctx.pythonExec, - class_manifest).asJavaRDD() - return RDD(jrdd, self.ctx, serializer) - - def flatMapPairs(self, f, keySerializer=None, valSerializer=None, - preservesPartitioning=False): - """ - >>> rdd = sc.parallelize([2, 3, 4]) - >>> sorted(rdd.flatMapPairs(lambda x: [(x, x), (x, x)]).collect()) + >>> sorted(rdd.flatMap(lambda x: [(x, x), (x, x)]).collect()) [(2, 2), (2, 2), (3, 3), (3, 3), (4, 4), (4, 4)] """ - keySerializer = keySerializer or self.ctx.defaultSerializer - valSerializer = valSerializer or self.ctx.defaultSerializer - dumpk = keySerializer.dumps - dumpv = valSerializer.dumps - loads = self.serializer.loads - def func(x): - pairs = f(loads(x)) - pickled_pairs = ((dumpk(k), dumpv(v)) for (k, v) in pairs) - return "\n".join(chain.from_iterable(pickled_pairs)) or None - pipe_command = RDD._get_pipe_command("map", [func]) - class_manifest = self._jrdd.classManifest() - python_rdd = self.ctx.jvm.PythonPairRDD(self._jrdd.rdd(), pipe_command, - preservesPartitioning, self.ctx.pythonExec, class_manifest) - return PairRDD(python_rdd.asJavaPairRDD(), self.ctx, keySerializer, - valSerializer) + return MappedRDD(self, f, preservesPartitioning=False, command='flatmap') def filter(self, f): """ @@ -97,9 +46,8 @@ class RDD(object): >>> rdd.filter(lambda x: x % 2 == 0).collect() [2, 4] """ - loads = self.serializer.loads - def filter_func(x): return x if f(loads(x)) else None - return self._builder(self._pipe(filter_func), self.ctx) + def filter_func(x): return x if f(x) else None + return RDD(self._pipe(filter_func), self.ctx) def _pipe(self, functions, command="map"): class_manifest = self._jrdd.classManifest() @@ -108,32 +56,22 @@ class RDD(object): False, self.ctx.pythonExec, class_manifest) return python_rdd.asJavaRDD() - def _pipePairs(self, functions, command="mapPairs", - preservesPartitioning=False): - class_manifest = self._jrdd.classManifest() - pipe_command = RDD._get_pipe_command(command, functions) - python_rdd = self.ctx.jvm.PythonPairRDD(self._jrdd.rdd(), pipe_command, - preservesPartitioning, self.ctx.pythonExec, class_manifest) - return python_rdd.asJavaPairRDD() - def distinct(self): """ >>> sorted(sc.parallelize([1, 1, 2, 3]).distinct().collect()) [1, 2, 3] """ - if self.serializer.is_comparable: - return self._builder(self._jrdd.distinct(), self.ctx) - return self.mapPairs(lambda x: (x, "")) \ + return self.map(lambda x: (x, "")) \ .reduceByKey(lambda x, _: x) \ .map(lambda (x, _): x) def sample(self, withReplacement, fraction, seed): jrdd = self._jrdd.sample(withReplacement, fraction, seed) - return self._builder(jrdd, self.ctx) + return RDD(jrdd, self.ctx) def takeSample(self, withReplacement, num, seed): vals = self._jrdd.takeSample(withReplacement, num, seed) - return [self.serializer.loads(self.ctx.python_dump(x)) for x in vals] + return [PickleSerializer.loads(x) for x in vals] def union(self, other): """ @@ -141,7 +79,7 @@ class RDD(object): >>> rdd.union(rdd).collect() [1, 1, 2, 3, 1, 1, 2, 3] """ - return self._builder(self._jrdd.union(other._jrdd), self.ctx) + return RDD(self._jrdd.union(other._jrdd), self.ctx) # TODO: sort @@ -155,16 +93,17 @@ class RDD(object): >>> sorted(rdd.cartesian(rdd).collect()) [(1, 1), (1, 2), (2, 1), (2, 2)] """ - return PairRDD(self._jrdd.cartesian(other._jrdd), self.ctx) + return RDD(self._jrdd.cartesian(other._jrdd), self.ctx) # numsplits def groupBy(self, f, numSplits=None): """ >>> rdd = sc.parallelize([1, 1, 2, 3, 5, 8]) - >>> sorted(rdd.groupBy(lambda x: x % 2).collect()) + >>> result = rdd.groupBy(lambda x: x % 2).collect() + >>> sorted([(x, sorted(y)) for (x, y) in result]) [(0, [2, 8]), (1, [1, 1, 3, 5])] """ - return self.mapPairs(lambda x: (f(x), x)).groupByKey(numSplits) + return self.map(lambda x: (f(x), x)).groupByKey(numSplits) # TODO: pipe @@ -178,25 +117,19 @@ class RDD(object): self.map(f).collect() # Force evaluation def collect(self): - vals = self._jrdd.collect() - return [self.serializer.loads(self.ctx.python_dump(x)) for x in vals] + pickle = self.ctx.arrayAsPickle(self._jrdd.rdd().collect()) + return PickleSerializer.loads(bytes(pickle)) - def reduce(self, f, serializer=None): + def reduce(self, f): """ - >>> import operator - >>> sc.parallelize([1, 2, 3, 4, 5]).reduce(operator.add) + >>> from operator import add + >>> sc.parallelize([1, 2, 3, 4, 5]).reduce(add) 15 + >>> sc.parallelize((2 for _ in range(10))).map(lambda x: 1).cache().reduce(add) + 10 """ - serializer = serializer or self.ctx.defaultSerializer - loads = self.serializer.loads - dumps = serializer.dumps - def reduceFunction(x, acc): - if acc is None: - return loads(x) - else: - return f(loads(x), acc) - vals = self._pipe([reduceFunction, dumps], command="reduce").collect() - return reduce(f, (serializer.loads(x) for x in vals)) + vals = MappedRDD(self, f, command="reduce", preservesPartitioning=False).collect() + return reduce(f, vals) # TODO: fold @@ -216,36 +149,35 @@ class RDD(object): >>> sc.parallelize([2, 3, 4]).take(2) [2, 3] """ - vals = self._jrdd.take(num) - return [self.serializer.loads(self.ctx.python_dump(x)) for x in vals] + pickle = self.ctx.arrayAsPickle(self._jrdd.rdd().take(num)) + return PickleSerializer.loads(bytes(pickle)) def first(self): """ >>> sc.parallelize([2, 3, 4]).first() 2 """ - return self.serializer.loads(self.ctx.python_dump(self._jrdd.first())) + return PickleSerializer.loads(bytes(self.ctx.asPickle(self._jrdd.first()))) # TODO: saveAsTextFile # TODO: saveAsObjectFile + # Pair functions -class PairRDD(RDD): - - def __init__(self, jrdd, ctx, keySerializer=None, valSerializer=None): - RDD.__init__(self, jrdd, ctx) - self.keySerializer = keySerializer or ctx.defaultSerializer - self.valSerializer = valSerializer or ctx.defaultSerializer - self.serializer = \ - PairSerializer(self.keySerializer, self.valSerializer) - - def _builder(self, jrdd, ctx): - return PairRDD(jrdd, ctx, self.keySerializer, self.valSerializer) + def collectAsMap(self): + """ + >>> m = sc.parallelize([(1, 2), (3, 4)]).collectAsMap() + >>> m[1] + 2 + >>> m[3] + 4 + """ + return dict(self.collect()) def reduceByKey(self, func, numSplits=None): """ - >>> x = sc.parallelizePairs([("a", 1), ("b", 1), ("a", 1)]) + >>> x = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) >>> sorted(x.reduceByKey(lambda a, b: a + b).collect()) [('a', 2), ('b', 1)] """ @@ -259,90 +191,67 @@ class PairRDD(RDD): def join(self, other, numSplits=None): """ - >>> x = sc.parallelizePairs([("a", 1), ("b", 4)]) - >>> y = sc.parallelizePairs([("a", 2), ("a", 3)]) - >>> x.join(y).collect() + >>> x = sc.parallelize([("a", 1), ("b", 4)]) + >>> y = sc.parallelize([("a", 2), ("a", 3)]) + >>> sorted(x.join(y).collect()) [('a', (1, 2)), ('a', (1, 3))] - - Check that we get a PairRDD-like object back: - >>> assert x.join(y).join """ - assert self.keySerializer.name == other.keySerializer.name - if self.keySerializer.is_comparable: - return PairRDD(self._jrdd.join(other._jrdd), - self.ctx, self.keySerializer, - PairSerializer(self.valSerializer, other.valSerializer)) - else: - return python_join(self, other, numSplits) + return python_join(self, other, numSplits) def leftOuterJoin(self, other, numSplits=None): """ - >>> x = sc.parallelizePairs([("a", 1), ("b", 4)]) - >>> y = sc.parallelizePairs([("a", 2)]) + >>> x = sc.parallelize([("a", 1), ("b", 4)]) + >>> y = sc.parallelize([("a", 2)]) >>> sorted(x.leftOuterJoin(y).collect()) [('a', (1, 2)), ('b', (4, None))] """ - assert self.keySerializer.name == other.keySerializer.name - if self.keySerializer.is_comparable: - return PairRDD(self._jrdd.leftOuterJoin(other._jrdd), - self.ctx, self.keySerializer, - PairSerializer(self.valSerializer, - OptionSerializer(other.valSerializer))) - else: - return python_left_outer_join(self, other, numSplits) + return python_left_outer_join(self, other, numSplits) def rightOuterJoin(self, other, numSplits=None): """ - >>> x = sc.parallelizePairs([("a", 1), ("b", 4)]) - >>> y = sc.parallelizePairs([("a", 2)]) + >>> x = sc.parallelize([("a", 1), ("b", 4)]) + >>> y = sc.parallelize([("a", 2)]) >>> sorted(y.rightOuterJoin(x).collect()) [('a', (2, 1)), ('b', (None, 4))] """ - assert self.keySerializer.name == other.keySerializer.name - if self.keySerializer.is_comparable: - return PairRDD(self._jrdd.rightOuterJoin(other._jrdd), - self.ctx, self.keySerializer, - PairSerializer(OptionSerializer(self.valSerializer), - other.valSerializer)) - else: - return python_right_outer_join(self, other, numSplits) + return python_right_outer_join(self, other, numSplits) + + # TODO: pipelining + # TODO: optimizations + def shuffle(self, numSplits): + if numSplits is None: + numSplits = self.ctx.defaultParallelism + pipe_command = RDD._get_pipe_command('shuffle_map_step', []) + class_manifest = self._jrdd.classManifest() + python_rdd = self.ctx.jvm.PythonPairRDD(self._jrdd.rdd(), + pipe_command, False, self.ctx.pythonExec, class_manifest) + partitioner = self.ctx.jvm.spark.HashPartitioner(numSplits) + jrdd = python_rdd.asJavaPairRDD().partitionBy(partitioner) + jrdd = jrdd.map(self.ctx.jvm.ExtractValue()) + # TODO: extract second value. + return RDD(jrdd, self.ctx) + + def combineByKey(self, createCombiner, mergeValue, mergeCombiners, - numSplits=None, serializer=None): + numSplits=None): """ - >>> x = sc.parallelizePairs([("a", 1), ("b", 1), ("a", 1)]) + >>> x = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) >>> def f(x): return x >>> def add(a, b): return a + str(b) >>> sorted(x.combineByKey(str, add, add).collect()) [('a', '11'), ('b', '1')] """ - serializer = serializer or self.ctx.defaultSerializer if numSplits is None: numSplits = self.ctx.defaultParallelism - # Use hash() to create keys that are comparable in Java. - loadkv = self.serializer.loads - def pairify(kv): - # TODO: add method to deserialize only the key or value from - # a PairSerializer? - key = loadkv(kv)[0] - return (str(hash(key)), kv) - partitioner = self.ctx.jvm.spark.HashPartitioner(numSplits) - jrdd = self._pipePairs(pairify).partitionBy(partitioner) - pairified = PairRDD(jrdd, self.ctx, NopSerializer, self.serializer) - - loads = PairSerializer(NopSerializer, self.serializer).loads - dumpk = self.keySerializer.dumps - dumpc = serializer.dumps - - functions = [createCombiner, mergeValue, mergeCombiners, loads, dumpk, - dumpc] - jpairs = pairified._pipePairs(functions, "combine_by_key", - preservesPartitioning=True) - return PairRDD(jpairs, self.ctx, self.keySerializer, serializer) + shuffled = self.shuffle(numSplits) + functions = [createCombiner, mergeValue, mergeCombiners] + jpairs = shuffled._pipe(functions, "combine_by_key") + return RDD(jpairs, self.ctx) def groupByKey(self, numSplits=None): """ - >>> x = sc.parallelizePairs([("a", 1), ("b", 1), ("a", 1)]) + >>> x = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) >>> sorted(x.groupByKey().collect()) [('a', [1, 1]), ('b', [1])] """ @@ -360,29 +269,15 @@ class PairRDD(RDD): return self.combineByKey(createCombiner, mergeValue, mergeCombiners, numSplits) - def collectAsMap(self): - """ - >>> m = sc.parallelizePairs([(1, 2), (3, 4)]).collectAsMap() - >>> m[1] - 2 - >>> m[3] - 4 - """ - m = self._jrdd.collectAsMap() - def loads(x): - (k, v) = x - return (self.keySerializer.loads(k), self.valSerializer.loads(v)) - return dict(loads(x) for x in m.items()) - - def flatMapValues(self, f, valSerializer=None): + def flatMapValues(self, f): flat_map_fn = lambda (k, v): ((k, x) for x in f(v)) - return self.flatMapPairs(flat_map_fn, self.keySerializer, - valSerializer, True) + return self.flatMap(flat_map_fn) - def mapValues(self, f, valSerializer=None): + def mapValues(self, f): map_values_fn = lambda (k, v): (k, f(v)) - return self.mapPairs(map_values_fn, self.keySerializer, valSerializer, - True) + return self.map(map_values_fn, preservesPartitioning=True) + + # TODO: implement shuffle. # TODO: support varargs cogroup of several RDDs. def groupWith(self, other): @@ -390,20 +285,12 @@ class PairRDD(RDD): def cogroup(self, other, numSplits=None): """ - >>> x = sc.parallelizePairs([("a", 1), ("b", 4)]) - >>> y = sc.parallelizePairs([("a", 2)]) + >>> x = sc.parallelize([("a", 1), ("b", 4)]) + >>> y = sc.parallelize([("a", 2)]) >>> x.cogroup(y).collect() [('a', ([1], [2])), ('b', ([4], []))] """ - assert self.keySerializer.name == other.keySerializer.name - resultValSerializer = PairSerializer( - ArraySerializer(self.valSerializer), - ArraySerializer(other.valSerializer)) - if self.keySerializer.is_comparable: - return PairRDD(self._jrdd.cogroup(other._jrdd), - self.ctx, self.keySerializer, resultValSerializer) - else: - return python_cogroup(self, other, numSplits) + return python_cogroup(self, other, numSplits) # TODO: `lookup` is disabled because we can't make direct comparisons based # on the key; we need to compare the hash of the key to the hash of the @@ -413,44 +300,84 @@ class PairRDD(RDD): # TODO: file saving -class MappedRDDBase(object): - def __init__(self, prev, func, serializer, preservesPartitioning=False): - if isinstance(prev, MappedRDDBase) and not prev.is_cached: - prev_func = prev.func - self.func = lambda x: func(prev_func(x)) - self.preservesPartitioning = \ - prev.preservesPartitioning and preservesPartitioning - self._prev_jrdd = prev._prev_jrdd - self._prev_serializer = prev._prev_serializer - else: - self.func = func - self.preservesPartitioning = preservesPartitioning - self._prev_jrdd = prev._jrdd - self._prev_serializer = prev.serializer - self.serializer = serializer or prev.ctx.defaultSerializer - self.is_cached = False - self.ctx = prev.ctx - self.prev = prev - self._jrdd_val = None - - -class MappedRDD(MappedRDDBase, RDD): +class MappedRDD(RDD): """ + Pipelined maps: >>> rdd = sc.parallelize([1, 2, 3, 4]) >>> rdd.map(lambda x: 2 * x).cache().map(lambda x: 2 * x).collect() [4, 8, 12, 16] >>> rdd.map(lambda x: 2 * x).map(lambda x: 2 * x).collect() [4, 8, 12, 16] + + Pipelined reduces: + >>> from operator import add + >>> rdd.map(lambda x: 2 * x).reduce(add) + 20 + >>> rdd.flatMap(lambda x: [x, x]).reduce(add) + 20 """ + def __init__(self, prev, func, preservesPartitioning=False, command='map'): + if isinstance(prev, MappedRDD) and not prev.is_cached: + prev_func = prev.func + if command == 'reduce': + if prev.command == 'flatmap': + def flatmap_reduce_func(x, acc): + values = prev_func(x) + if values is None: + return acc + if not acc: + if len(values) == 1: + return values[0] + else: + return reduce(func, values[1:], values[0]) + else: + return reduce(func, values, acc) + self.func = flatmap_reduce_func + else: + def reduce_func(x, acc): + val = prev_func(x) + if not val: + return acc + if acc is None: + return val + else: + return func(val, acc) + self.func = reduce_func + else: + if prev.command == 'flatmap': + command = 'flatmap' + self.func = lambda x: (func(y) for y in prev_func(x)) + else: + self.func = lambda x: func(prev_func(x)) + + self.preservesPartitioning = \ + prev.preservesPartitioning and preservesPartitioning + self._prev_jrdd = prev._prev_jrdd + self.is_pipelined = True + else: + if command == 'reduce': + def reduce_func(val, acc): + if acc is None: + return val + else: + return func(val, acc) + self.func = reduce_func + else: + self.func = func + self.preservesPartitioning = preservesPartitioning + self._prev_jrdd = prev._jrdd + self.is_pipelined = False + self.is_cached = False + self.ctx = prev.ctx + self.prev = prev + self._jrdd_val = None + self.command = command @property def _jrdd(self): if not self._jrdd_val: - udf = self.func - loads = self._prev_serializer.loads - dumps = self.serializer.dumps - func = lambda x: dumps(udf(loads(x))) - pipe_command = RDD._get_pipe_command("map", [func]) + funcs = [self.func] + pipe_command = RDD._get_pipe_command(self.command, funcs) class_manifest = self._prev_jrdd.classManifest() python_rdd = self.ctx.jvm.PythonRDD(self._prev_jrdd.rdd(), pipe_command, self.preservesPartitioning, self.ctx.pythonExec, @@ -459,56 +386,11 @@ class MappedRDD(MappedRDDBase, RDD): return self._jrdd_val -class PairMappedRDD(MappedRDDBase, PairRDD): - """ - >>> rdd = sc.parallelize([1, 2, 3, 4]) - >>> rdd.mapPairs(lambda x: (x, x)) \\ - ... .mapPairs(lambda (x, y): (2*x, 2*y)) \\ - ... .collect() - [(2, 2), (4, 4), (6, 6), (8, 8)] - >>> rdd.mapPairs(lambda x: (x, x)) \\ - ... .mapPairs(lambda (x, y): (2*x, 2*y)) \\ - ... .map(lambda (x, _): x).collect() - [2, 4, 6, 8] - """ - - def __init__(self, prev, func, keySerializer=None, valSerializer=None, - preservesPartitioning=False): - self.keySerializer = keySerializer or prev.ctx.defaultSerializer - self.valSerializer = valSerializer or prev.ctx.defaultSerializer - serializer = PairSerializer(self.keySerializer, self.valSerializer) - MappedRDDBase.__init__(self, prev, func, serializer, - preservesPartitioning) - - @property - def _jrdd(self): - if not self._jrdd_val: - udf = self.func - loads = self._prev_serializer.loads - dumpk = self.keySerializer.dumps - dumpv = self.valSerializer.dumps - def func(x): - (k, v) = udf(loads(x)) - return (dumpk(k), dumpv(v)) - pipe_command = RDD._get_pipe_command("mapPairs", [func]) - class_manifest = self._prev_jrdd.classManifest() - self._jrdd_val = self.ctx.jvm.PythonPairRDD(self._prev_jrdd.rdd(), - pipe_command, self.preservesPartitioning, self.ctx.pythonExec, - class_manifest).asJavaPairRDD() - return self._jrdd_val - - def _test(): import doctest from pyspark.context import SparkContext - from pyspark.serializers import PickleSerializer, JSONSerializer globs = globals().copy() - globs['sc'] = SparkContext('local', 'PythonTest', - defaultSerializer=JSONSerializer) - doctest.testmod(globs=globs) - globs['sc'].stop() - globs['sc'] = SparkContext('local', 'PythonTest', - defaultSerializer=PickleSerializer) + globs['sc'] = SparkContext('local', 'PythonTest') doctest.testmod(globs=globs) globs['sc'].stop() diff --git a/pyspark/pyspark/serializers.py b/pyspark/pyspark/serializers.py index b113f5656b..7b3e6966e1 100644 --- a/pyspark/pyspark/serializers.py +++ b/pyspark/pyspark/serializers.py @@ -2,228 +2,35 @@ Data serialization methods. The Spark Python API is built on top of the Spark Java API. RDDs created in -Python are stored in Java as RDDs of Strings. Python objects are automatically -serialized/deserialized, so this representation is transparent to the end-user. - ------------------- -Serializer objects ------------------- - -`Serializer` objects are used to customize how an RDD's values are serialized. - -Each `Serializer` is a named tuple with four fields: - - - A `dumps` function, for serializing a Python object to a string. - - - A `loads` function, for deserializing a Python object from a string. - - - An `is_comparable` field, True if equal Python objects are serialized to - equal strings, and False otherwise. - - - A `name` field, used to identify the Serializer. Serializers are - compared for equality by comparing their names. - -The serializer's output should be base64-encoded. - ------------------------------------------------------------------- -`is_comparable`: comparing serialized representations for equality ------------------------------------------------------------------- - -If `is_comparable` is False, the serializer's representations of equal objects -are not required to be equal: - ->>> import pickle ->>> a = {1: 0, 9: 0} ->>> b = {9: 0, 1: 0} ->>> a == b -True ->>> pickle.dumps(a) == pickle.dumps(b) -False - -RDDs with comparable serializers can use native Java implementations of -operations like join() and distinct(), which may lead to better performance by -eliminating deserialization and Python comparisons. - -The default JSONSerializer produces comparable representations of common Python -data structures. - --------------------------------------- -Examples of serialized representations --------------------------------------- - -The RDD transformations that use Python UDFs are implemented in terms of -a modified `PipedRDD.pipe()` function. For each record `x` in the RDD, the -`pipe()` function pipes `x.toString()` to a Python worker process, which -deserializes the string into a Python object, executes user-defined functions, -and outputs serialized Python objects. - -The regular `toString()` method returns an ambiguous representation, due to the -way that Scala `Option` instances are printed: - ->>> from context import SparkContext ->>> sc = SparkContext("local", "SerializerDocs") ->>> x = sc.parallelizePairs([("a", 1), ("b", 4)]) ->>> y = sc.parallelizePairs([("a", 2)]) - ->>> print y.rightOuterJoin(x)._jrdd.first().toString() -(ImEi,(Some(Mg==),MQ==)) - -In Java, preprocessing is performed to handle Option instances, so the Python -process receives unambiguous input: - ->>> print sc.python_dump(y.rightOuterJoin(x)._jrdd.first()) -(ImEi,(Mg==,MQ==)) - -The base64-encoding eliminates the need to escape newlines, parentheses and -other special characters. - ----------------------- -Serializer composition ----------------------- - -In order to handle nested structures, which could contain object serialized -with different serializers, the RDD module composes serializers. For example, -the serializers in the previous example are: - ->>> print x.serializer.name -PairSerializer - ->>> print y.serializer.name -PairSerializer - ->>> print y.rightOuterJoin(x).serializer.name -PairSerializer, JSONSerializer>> +Python are stored in Java as RDD[Array[Byte]]. Python objects are +automatically serialized/deserialized, so this representation is transparent to +the end-user. """ -from base64 import standard_b64encode, standard_b64decode from collections import namedtuple import cPickle -import simplejson +import struct -Serializer = namedtuple("Serializer", - ["dumps","loads", "is_comparable", "name"]) - - -NopSerializer = Serializer(str, str, True, "NopSerializer") - - -JSONSerializer = Serializer( - lambda obj: standard_b64encode(simplejson.dumps(obj, sort_keys=True, - separators=(',', ':'))), - lambda s: simplejson.loads(standard_b64decode(s)), - True, - "JSONSerializer" -) +Serializer = namedtuple("Serializer", ["dumps","loads"]) PickleSerializer = Serializer( - lambda obj: standard_b64encode(cPickle.dumps(obj)), - lambda s: cPickle.loads(standard_b64decode(s)), - False, - "PickleSerializer" -) + lambda obj: cPickle.dumps(obj, -1), + cPickle.loads) -def OptionSerializer(serializer): - """ - >>> ser = OptionSerializer(NopSerializer) - >>> ser.loads(ser.dumps("Hello, World!")) - 'Hello, World!' - >>> ser.loads(ser.dumps(None)) is None - True - """ - none_placeholder = '*' - - def dumps(x): - if x is None: - return none_placeholder - else: - return serializer.dumps(x) - - def loads(x): - if x == none_placeholder: - return None - else: - return serializer.loads(x) - - name = "OptionSerializer<%s>" % serializer.name - return Serializer(dumps, loads, serializer.is_comparable, name) +def dumps(obj, stream): + # TODO: determining the length of non-byte objects. + stream.write(struct.pack("!i", len(obj))) + stream.write(obj) -def PairSerializer(keySerializer, valSerializer): - """ - Returns a Serializer for a (key, value) pair. - - >>> ser = PairSerializer(JSONSerializer, JSONSerializer) - >>> ser.loads(ser.dumps((1, 2))) - (1, 2) - - >>> ser = PairSerializer(JSONSerializer, ser) - >>> ser.loads(ser.dumps((1, (2, 3)))) - (1, (2, 3)) - """ - def loads(kv): - try: - (key, val) = kv[1:-1].split(',', 1) - key = keySerializer.loads(key) - val = valSerializer.loads(val) - return (key, val) - except: - print "Error in deserializing pair from '%s'" % str(kv) - raise - - def dumps(kv): - (key, val) = kv - return"(%s,%s)" % (keySerializer.dumps(key), valSerializer.dumps(val)) - is_comparable = \ - keySerializer.is_comparable and valSerializer.is_comparable - name = "PairSerializer<%s, %s>" % (keySerializer.name, valSerializer.name) - return Serializer(dumps, loads, is_comparable, name) - - -def ArraySerializer(serializer): - """ - >>> ser = ArraySerializer(JSONSerializer) - >>> ser.loads(ser.dumps([1, 2, 3, 4])) - [1, 2, 3, 4] - >>> ser = ArraySerializer(PairSerializer(JSONSerializer, PickleSerializer)) - >>> ser.loads(ser.dumps([('a', 1), ('b', 2)])) - [('a', 1), ('b', 2)] - >>> ser.loads(ser.dumps([('a', 1)])) - [('a', 1)] - >>> ser.loads(ser.dumps([])) - [] - """ - def dumps(arr): - if arr == []: - return '[]' - else: - return '[' + '|'.join(serializer.dumps(x) for x in arr) + ']' - - def loads(s): - if s == '[]': - return [] - items = s[1:-1] - if '|' in items: - items = items.split('|') - else: - items = [items] - return [serializer.loads(x) for x in items] - - name = "ArraySerializer<%s>" % serializer.name - return Serializer(dumps, loads, serializer.is_comparable, name) - - -# TODO: IntegerSerializer - - -# TODO: DoubleSerializer - - -def _test(): - import doctest - doctest.testmod() - - -if __name__ == "__main__": - _test() +def loads(stream): + length = stream.read(4) + if length == "": + raise EOFError + length = struct.unpack("!i", length)[0] + obj = stream.read(length) + if obj == "": + raise EOFError + return obj diff --git a/pyspark/pyspark/worker.py b/pyspark/pyspark/worker.py index 4c4b02fce4..21ff84fb17 100644 --- a/pyspark/pyspark/worker.py +++ b/pyspark/pyspark/worker.py @@ -6,9 +6,9 @@ from base64 import standard_b64decode # CloudPickler needs to be imported so that depicklers are registered using the # copy_reg module. from pyspark.cloudpickle import CloudPickler +from pyspark.serializers import dumps, loads, PickleSerializer import cPickle - # Redirect stdout to stderr so that users must return values from functions. old_stdout = sys.stdout sys.stdout = sys.stderr @@ -19,58 +19,64 @@ def load_function(): def output(x): - for line in x.split("\n"): - old_stdout.write(line.rstrip("\r\n") + "\n") + dumps(x, old_stdout) def read_input(): - for line in sys.stdin: - yield line.rstrip("\r\n") - + try: + while True: + yield loads(sys.stdin) + except EOFError: + return def do_combine_by_key(): create_combiner = load_function() merge_value = load_function() merge_combiners = load_function() # TODO: not used. - depickler = load_function() - key_pickler = load_function() - combiner_pickler = load_function() combiners = {} - for line in read_input(): - # Discard the hashcode added in the Python combineByKey() method. - (key, value) = depickler(line)[1] + for obj in read_input(): + (key, value) = PickleSerializer.loads(obj) if key not in combiners: combiners[key] = create_combiner(value) else: combiners[key] = merge_value(combiners[key], value) for (key, combiner) in combiners.iteritems(): - output(key_pickler(key)) - output(combiner_pickler(combiner)) + output(PickleSerializer.dumps((key, combiner))) -def do_map(map_pairs=False): +def do_map(flat=False): f = load_function() - for line in read_input(): + for obj in read_input(): try: - out = f(line) + #from pickletools import dis + #print repr(obj) + #print dis(obj) + out = f(PickleSerializer.loads(obj)) if out is not None: - if map_pairs: + if flat: for x in out: - output(x) + output(PickleSerializer.dumps(x)) else: - output(out) + output(PickleSerializer.dumps(out)) except: - sys.stderr.write("Error processing line '%s'\n" % line) + sys.stderr.write("Error processing obj %s\n" % repr(obj)) raise +def do_shuffle_map_step(): + for obj in read_input(): + key = PickleSerializer.loads(obj)[1] + output(str(hash(key))) + output(obj) + + def do_reduce(): f = load_function() - dumps = load_function() acc = None - for line in read_input(): - acc = f(line, acc) - output(dumps(acc)) + for obj in read_input(): + acc = f(PickleSerializer.loads(obj), acc) + if acc is not None: + output(PickleSerializer.dumps(acc)) def do_echo(): @@ -80,13 +86,15 @@ def do_echo(): def main(): command = sys.stdin.readline().strip() if command == "map": - do_map(map_pairs=False) - elif command == "mapPairs": - do_map(map_pairs=True) + do_map(flat=False) + elif command == "flatmap": + do_map(flat=True) elif command == "combine_by_key": do_combine_by_key() elif command == "reduce": do_reduce() + elif command == "shuffle_map_step": + do_shuffle_map_step() elif command == "echo": do_echo() else: From 607b53abfca049e7d9139e2d29893a3bb252de19 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 22 Aug 2012 00:43:55 -0700 Subject: [PATCH 04/85] Use numpy in Python k-means example. --- .../scala/spark/api/python/PythonRDD.scala | 8 ++++++- pyspark/pyspark/examples/kmeans.py | 23 +++++++------------ pyspark/pyspark/rdd.py | 9 +++----- pyspark/pyspark/worker.py | 8 +++---- 4 files changed, 21 insertions(+), 27 deletions(-) diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index b9a0168d18..93847e2f14 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -101,7 +101,13 @@ trait PythonRDDBase { stream.readFully(obj) obj } catch { - case eof: EOFException => { new Array[Byte](0) } + case eof: EOFException => { + val exitStatus = proc.waitFor() + if (exitStatus != 0) { + throw new Exception("Subprocess exited with status " + exitStatus) + } + new Array[Byte](0) + } case e => throw e } } diff --git a/pyspark/pyspark/examples/kmeans.py b/pyspark/pyspark/examples/kmeans.py index 0761d6e395..9cc366f03c 100644 --- a/pyspark/pyspark/examples/kmeans.py +++ b/pyspark/pyspark/examples/kmeans.py @@ -1,25 +1,18 @@ import sys from pyspark.context import SparkContext +from numpy import array, sum as np_sum def parseVector(line): - return [float(x) for x in line.split(' ')] - - -def addVec(x, y): - return [a + b for (a, b) in zip(x, y)] - - -def squaredDist(x, y): - return sum((a - b) ** 2 for (a, b) in zip(x, y)) + return array([float(x) for x in line.split(' ')]) def closestPoint(p, centers): bestIndex = 0 closest = float("+inf") for i in range(len(centers)): - tempDist = squaredDist(p, centers[i]) + tempDist = np_sum((p - centers[i]) ** 2) if tempDist < closest: closest = tempDist bestIndex = i @@ -41,14 +34,14 @@ if __name__ == "__main__": tempDist = 1.0 while tempDist > convergeDist: - closest = data.mapPairs( + closest = data.map( lambda p : (closestPoint(p, kPoints), (p, 1))) pointStats = closest.reduceByKey( - lambda (x1, y1), (x2, y2): (addVec(x1, x2), y1 + y2)) - newPoints = pointStats.mapPairs( - lambda (x, (y, z)): (x, [a / z for a in y])).collect() + lambda (x1, y1), (x2, y2): (x1 + x2, y1 + y2)) + newPoints = pointStats.map( + lambda (x, (y, z)): (x, y / z)).collect() - tempDist = sum(squaredDist(kPoints[x], y) for (x, y) in newPoints) + tempDist = sum(np_sum((kPoints[x] - y) ** 2) for (x, y) in newPoints) for (x, y) in newPoints: kPoints[x] = y diff --git a/pyspark/pyspark/rdd.py b/pyspark/pyspark/rdd.py index 8eccddc0a2..ff9c483032 100644 --- a/pyspark/pyspark/rdd.py +++ b/pyspark/pyspark/rdd.py @@ -71,7 +71,7 @@ class RDD(object): def takeSample(self, withReplacement, num, seed): vals = self._jrdd.takeSample(withReplacement, num, seed) - return [PickleSerializer.loads(x) for x in vals] + return [PickleSerializer.loads(bytes(x)) for x in vals] def union(self, other): """ @@ -218,17 +218,16 @@ class RDD(object): # TODO: pipelining # TODO: optimizations - def shuffle(self, numSplits): + def shuffle(self, numSplits, hashFunc=hash): if numSplits is None: numSplits = self.ctx.defaultParallelism - pipe_command = RDD._get_pipe_command('shuffle_map_step', []) + pipe_command = RDD._get_pipe_command('shuffle_map_step', [hashFunc]) class_manifest = self._jrdd.classManifest() python_rdd = self.ctx.jvm.PythonPairRDD(self._jrdd.rdd(), pipe_command, False, self.ctx.pythonExec, class_manifest) partitioner = self.ctx.jvm.spark.HashPartitioner(numSplits) jrdd = python_rdd.asJavaPairRDD().partitionBy(partitioner) jrdd = jrdd.map(self.ctx.jvm.ExtractValue()) - # TODO: extract second value. return RDD(jrdd, self.ctx) @@ -277,8 +276,6 @@ class RDD(object): map_values_fn = lambda (k, v): (k, f(v)) return self.map(map_values_fn, preservesPartitioning=True) - # TODO: implement shuffle. - # TODO: support varargs cogroup of several RDDs. def groupWith(self, other): return self.cogroup(other) diff --git a/pyspark/pyspark/worker.py b/pyspark/pyspark/worker.py index 21ff84fb17..b13ed5699a 100644 --- a/pyspark/pyspark/worker.py +++ b/pyspark/pyspark/worker.py @@ -48,9 +48,6 @@ def do_map(flat=False): f = load_function() for obj in read_input(): try: - #from pickletools import dis - #print repr(obj) - #print dis(obj) out = f(PickleSerializer.loads(obj)) if out is not None: if flat: @@ -64,9 +61,10 @@ def do_map(flat=False): def do_shuffle_map_step(): + hashFunc = load_function() for obj in read_input(): - key = PickleSerializer.loads(obj)[1] - output(str(hash(key))) + key = PickleSerializer.loads(obj)[0] + output(str(hashFunc(key))) output(obj) From 4b523004877cf94152225484de7683e9d17cdb56 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 23 Aug 2012 15:54:15 -0700 Subject: [PATCH 05/85] Fix options parsing in Python pi example. --- pyspark/pyspark/examples/pi.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pyspark/pyspark/examples/pi.py b/pyspark/pyspark/examples/pi.py index ad77694c41..fe63d2c952 100644 --- a/pyspark/pyspark/examples/pi.py +++ b/pyspark/pyspark/examples/pi.py @@ -10,7 +10,7 @@ if __name__ == "__main__": "Usage: PythonPi []" exit(-1) sc = SparkContext(sys.argv[1], "PythonKMeans") - slices = sys.argv[2] if len(sys.argv) > 2 else 2 + slices = int(sys.argv[2]) if len(sys.argv) > 2 else 2 n = 100000 * slices def f(_): x = random() * 2 - 1 From f3b852ce66d193e3421eeecef71ea27bff73a94b Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 24 Aug 2012 19:38:50 -0700 Subject: [PATCH 06/85] Refactor Python MappedRDD to use iterator pipelines. --- pyspark/pyspark/rdd.py | 83 ++++++++++++--------------------------- pyspark/pyspark/worker.py | 55 +++++++------------------- 2 files changed, 41 insertions(+), 97 deletions(-) diff --git a/pyspark/pyspark/rdd.py b/pyspark/pyspark/rdd.py index ff9c483032..7d280d8844 100644 --- a/pyspark/pyspark/rdd.py +++ b/pyspark/pyspark/rdd.py @@ -1,4 +1,5 @@ from base64 import standard_b64encode as b64enc +from itertools import chain, ifilter, imap from pyspark import cloudpickle from pyspark.serializers import PickleSerializer @@ -15,8 +16,6 @@ class RDD(object): @classmethod def _get_pipe_command(cls, command, functions): - if functions and not isinstance(functions, (list, tuple)): - functions = [functions] worker_args = [command] for f in functions: worker_args.append(b64enc(cloudpickle.dumps(f))) @@ -28,7 +27,8 @@ class RDD(object): return self def map(self, f, preservesPartitioning=False): - return MappedRDD(self, f, preservesPartitioning) + def func(iterator): return imap(f, iterator) + return PipelinedRDD(self, func, preservesPartitioning) def flatMap(self, f): """ @@ -38,7 +38,8 @@ class RDD(object): >>> sorted(rdd.flatMap(lambda x: [(x, x), (x, x)]).collect()) [(2, 2), (2, 2), (3, 3), (3, 3), (4, 4), (4, 4)] """ - return MappedRDD(self, f, preservesPartitioning=False, command='flatmap') + def func(iterator): return chain.from_iterable(imap(f, iterator)) + return PipelinedRDD(self, func) def filter(self, f): """ @@ -46,10 +47,10 @@ class RDD(object): >>> rdd.filter(lambda x: x % 2 == 0).collect() [2, 4] """ - def filter_func(x): return x if f(x) else None - return RDD(self._pipe(filter_func), self.ctx) + def func(iterator): return ifilter(f, iterator) + return PipelinedRDD(self, func) - def _pipe(self, functions, command="map"): + def _pipe(self, functions, command): class_manifest = self._jrdd.classManifest() pipe_command = RDD._get_pipe_command(command, functions) python_rdd = self.ctx.jvm.PythonRDD(self._jrdd.rdd(), pipe_command, @@ -128,7 +129,16 @@ class RDD(object): >>> sc.parallelize((2 for _ in range(10))).map(lambda x: 1).cache().reduce(add) 10 """ - vals = MappedRDD(self, f, command="reduce", preservesPartitioning=False).collect() + def func(iterator): + acc = None + for obj in iterator: + if acc is None: + acc = obj + else: + acc = f(obj, acc) + if acc is not None: + yield acc + vals = PipelinedRDD(self, func).collect() return reduce(f, vals) # TODO: fold @@ -230,8 +240,6 @@ class RDD(object): jrdd = jrdd.map(self.ctx.jvm.ExtractValue()) return RDD(jrdd, self.ctx) - - def combineByKey(self, createCombiner, mergeValue, mergeCombiners, numSplits=None): """ @@ -297,7 +305,7 @@ class RDD(object): # TODO: file saving -class MappedRDD(RDD): +class PipelinedRDD(RDD): """ Pipelined maps: >>> rdd = sc.parallelize([1, 2, 3, 4]) @@ -313,68 +321,29 @@ class MappedRDD(RDD): >>> rdd.flatMap(lambda x: [x, x]).reduce(add) 20 """ - def __init__(self, prev, func, preservesPartitioning=False, command='map'): - if isinstance(prev, MappedRDD) and not prev.is_cached: + def __init__(self, prev, func, preservesPartitioning=False): + if isinstance(prev, PipelinedRDD) and not prev.is_cached: prev_func = prev.func - if command == 'reduce': - if prev.command == 'flatmap': - def flatmap_reduce_func(x, acc): - values = prev_func(x) - if values is None: - return acc - if not acc: - if len(values) == 1: - return values[0] - else: - return reduce(func, values[1:], values[0]) - else: - return reduce(func, values, acc) - self.func = flatmap_reduce_func - else: - def reduce_func(x, acc): - val = prev_func(x) - if not val: - return acc - if acc is None: - return val - else: - return func(val, acc) - self.func = reduce_func - else: - if prev.command == 'flatmap': - command = 'flatmap' - self.func = lambda x: (func(y) for y in prev_func(x)) - else: - self.func = lambda x: func(prev_func(x)) - + def pipeline_func(iterator): + return func(prev_func(iterator)) + self.func = pipeline_func self.preservesPartitioning = \ prev.preservesPartitioning and preservesPartitioning self._prev_jrdd = prev._prev_jrdd - self.is_pipelined = True else: - if command == 'reduce': - def reduce_func(val, acc): - if acc is None: - return val - else: - return func(val, acc) - self.func = reduce_func - else: - self.func = func + self.func = func self.preservesPartitioning = preservesPartitioning self._prev_jrdd = prev._jrdd - self.is_pipelined = False self.is_cached = False self.ctx = prev.ctx self.prev = prev self._jrdd_val = None - self.command = command @property def _jrdd(self): if not self._jrdd_val: funcs = [self.func] - pipe_command = RDD._get_pipe_command(self.command, funcs) + pipe_command = RDD._get_pipe_command("pipeline", funcs) class_manifest = self._prev_jrdd.classManifest() python_rdd = self.ctx.jvm.PythonRDD(self._prev_jrdd.rdd(), pipe_command, self.preservesPartitioning, self.ctx.pythonExec, diff --git a/pyspark/pyspark/worker.py b/pyspark/pyspark/worker.py index b13ed5699a..76b09918e7 100644 --- a/pyspark/pyspark/worker.py +++ b/pyspark/pyspark/worker.py @@ -25,17 +25,17 @@ def output(x): def read_input(): try: while True: - yield loads(sys.stdin) + yield cPickle.loads(loads(sys.stdin)) except EOFError: return + def do_combine_by_key(): create_combiner = load_function() merge_value = load_function() merge_combiners = load_function() # TODO: not used. combiners = {} - for obj in read_input(): - (key, value) = PickleSerializer.loads(obj) + for (key, value) in read_input(): if key not in combiners: combiners[key] = create_combiner(value) else: @@ -44,57 +44,32 @@ def do_combine_by_key(): output(PickleSerializer.dumps((key, combiner))) -def do_map(flat=False): +def do_pipeline(): f = load_function() - for obj in read_input(): - try: - out = f(PickleSerializer.loads(obj)) - if out is not None: - if flat: - for x in out: - output(PickleSerializer.dumps(x)) - else: - output(PickleSerializer.dumps(out)) - except: - sys.stderr.write("Error processing obj %s\n" % repr(obj)) - raise + for obj in f(read_input()): + output(PickleSerializer.dumps(obj)) def do_shuffle_map_step(): hashFunc = load_function() - for obj in read_input(): - key = PickleSerializer.loads(obj)[0] + while True: + try: + pickled = loads(sys.stdin) + except EOFError: + return + key = cPickle.loads(pickled)[0] output(str(hashFunc(key))) - output(obj) - - -def do_reduce(): - f = load_function() - acc = None - for obj in read_input(): - acc = f(PickleSerializer.loads(obj), acc) - if acc is not None: - output(PickleSerializer.dumps(acc)) - - -def do_echo(): - old_stdout.writelines(sys.stdin.readlines()) + output(pickled) def main(): command = sys.stdin.readline().strip() - if command == "map": - do_map(flat=False) - elif command == "flatmap": - do_map(flat=True) + if command == "pipeline": + do_pipeline() elif command == "combine_by_key": do_combine_by_key() - elif command == "reduce": - do_reduce() elif command == "shuffle_map_step": do_shuffle_map_step() - elif command == "echo": - do_echo() else: raise Exception("Unsupported command %s" % command) From 65e8406029a0fe1e1c5c5d033d335b43f6743a04 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 24 Aug 2012 21:07:26 -0700 Subject: [PATCH 07/85] Implement fold() in Python API. --- pyspark/pyspark/rdd.py | 20 +++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/pyspark/pyspark/rdd.py b/pyspark/pyspark/rdd.py index 7d280d8844..af7703fdfc 100644 --- a/pyspark/pyspark/rdd.py +++ b/pyspark/pyspark/rdd.py @@ -141,7 +141,25 @@ class RDD(object): vals = PipelinedRDD(self, func).collect() return reduce(f, vals) - # TODO: fold + def fold(self, zeroValue, op): + """ + Aggregate the elements of each partition, and then the results for all + the partitions, using a given associative function and a neutral "zero + value." The function op(t1, t2) is allowed to modify t1 and return it + as its result value to avoid object allocation; however, it should not + modify t2. + + >>> from operator import add + >>> sc.parallelize([1, 2, 3, 4, 5]).fold(0, add) + 15 + """ + def func(iterator): + acc = zeroValue + for obj in iterator: + acc = op(obj, acc) + yield acc + vals = PipelinedRDD(self, func).collect() + return reduce(op, vals, zeroValue) # TODO: aggregate From f79a1e4d2a8643157136de69b8d7de84f0034712 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 25 Aug 2012 13:59:01 -0700 Subject: [PATCH 08/85] Add broadcast variables to Python API. --- .../scala/spark/api/python/PythonRDD.scala | 43 ++++++++++------- pyspark/pyspark/broadcast.py | 46 +++++++++++++++++++ pyspark/pyspark/context.py | 17 +++++-- pyspark/pyspark/rdd.py | 27 +++++++---- pyspark/pyspark/worker.py | 6 +++ 5 files changed, 110 insertions(+), 29 deletions(-) create mode 100644 pyspark/pyspark/broadcast.py diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 93847e2f14..5163812df4 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -7,14 +7,13 @@ import scala.collection.JavaConversions._ import scala.io.Source import spark._ import api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} -import scala.{collection, Some} -import collection.parallel.mutable +import broadcast.Broadcast import scala.collection -import scala.Some trait PythonRDDBase { def compute[T](split: Split, envVars: Map[String, String], - command: Seq[String], parent: RDD[T], pythonExec: String): Iterator[Array[Byte]] = { + command: Seq[String], parent: RDD[T], pythonExec: String, + broadcastVars: java.util.List[Broadcast[Array[Byte]]]): Iterator[Array[Byte]] = { val SPARK_HOME = new ProcessBuilder().environment().get("SPARK_HOME") val pb = new ProcessBuilder(Seq(pythonExec, SPARK_HOME + "/pyspark/pyspark/worker.py")) @@ -42,11 +41,18 @@ trait PythonRDDBase { override def run() { SparkEnv.set(env) val out = new PrintWriter(proc.getOutputStream) + val dOut = new DataOutputStream(proc.getOutputStream) + out.println(broadcastVars.length) + for (broadcast <- broadcastVars) { + out.print(broadcast.uuid.toString) + dOut.writeInt(broadcast.value.length) + dOut.write(broadcast.value) + dOut.flush() + } for (elem <- command) { out.println(elem) } out.flush() - val dOut = new DataOutputStream(proc.getOutputStream) for (elem <- parent.iterator(split)) { if (elem.isInstanceOf[Array[Byte]]) { val arr = elem.asInstanceOf[Array[Byte]] @@ -121,16 +127,17 @@ trait PythonRDDBase { class PythonRDD[T: ClassManifest]( parent: RDD[T], command: Seq[String], envVars: Map[String, String], - preservePartitoning: Boolean, pythonExec: String) + preservePartitoning: Boolean, pythonExec: String, broadcastVars: java.util.List[Broadcast[Array[Byte]]]) extends RDD[Array[Byte]](parent.context) with PythonRDDBase { - def this(parent: RDD[T], command: Seq[String], preservePartitoning: Boolean, pythonExec: String) = - this(parent, command, Map(), preservePartitoning, pythonExec) + def this(parent: RDD[T], command: Seq[String], preservePartitoning: Boolean, + pythonExec: String, broadcastVars: java.util.List[Broadcast[Array[Byte]]]) = + this(parent, command, Map(), preservePartitoning, pythonExec, broadcastVars) // Similar to Runtime.exec(), if we are given a single string, split it into words // using a standard StringTokenizer (i.e. by spaces) - def this(parent: RDD[T], command: String, preservePartitoning: Boolean, pythonExec: String) = - this(parent, PipedRDD.tokenize(command), preservePartitoning, pythonExec) + def this(parent: RDD[T], command: String, preservePartitoning: Boolean, pythonExec: String, broadcastVars: java.util.List[Broadcast[Array[Byte]]]) = + this(parent, PipedRDD.tokenize(command), preservePartitoning, pythonExec, broadcastVars) override def splits = parent.splits @@ -139,23 +146,25 @@ class PythonRDD[T: ClassManifest]( override val partitioner = if (preservePartitoning) parent.partitioner else None override def compute(split: Split): Iterator[Array[Byte]] = - compute(split, envVars, command, parent, pythonExec) + compute(split, envVars, command, parent, pythonExec, broadcastVars) val asJavaRDD : JavaRDD[Array[Byte]] = JavaRDD.fromRDD(this) } class PythonPairRDD[T: ClassManifest] ( parent: RDD[T], command: Seq[String], envVars: Map[String, String], - preservePartitoning: Boolean, pythonExec: String) + preservePartitoning: Boolean, pythonExec: String, broadcastVars: java.util.List[Broadcast[Array[Byte]]]) extends RDD[(Array[Byte], Array[Byte])](parent.context) with PythonRDDBase { - def this(parent: RDD[T], command: Seq[String], preservePartitoning: Boolean, pythonExec: String) = - this(parent, command, Map(), preservePartitoning, pythonExec) + def this(parent: RDD[T], command: Seq[String], preservePartitoning: Boolean, + pythonExec: String, broadcastVars: java.util.List[Broadcast[Array[Byte]]]) = + this(parent, command, Map(), preservePartitoning, pythonExec, broadcastVars) // Similar to Runtime.exec(), if we are given a single string, split it into words // using a standard StringTokenizer (i.e. by spaces) - def this(parent: RDD[T], command: String, preservePartitoning: Boolean, pythonExec: String) = - this(parent, PipedRDD.tokenize(command), preservePartitoning, pythonExec) + def this(parent: RDD[T], command: String, preservePartitoning: Boolean, pythonExec: String, + broadcastVars: java.util.List[Broadcast[Array[Byte]]]) = + this(parent, PipedRDD.tokenize(command), preservePartitoning, pythonExec, broadcastVars) override def splits = parent.splits @@ -164,7 +173,7 @@ class PythonPairRDD[T: ClassManifest] ( override val partitioner = if (preservePartitoning) parent.partitioner else None override def compute(split: Split): Iterator[(Array[Byte], Array[Byte])] = { - compute(split, envVars, command, parent, pythonExec).grouped(2).map { + compute(split, envVars, command, parent, pythonExec, broadcastVars).grouped(2).map { case Seq(a, b) => (a, b) case x => throw new Exception("PythonPairRDD: unexpected value: " + x) } diff --git a/pyspark/pyspark/broadcast.py b/pyspark/pyspark/broadcast.py new file mode 100644 index 0000000000..1ea17d59af --- /dev/null +++ b/pyspark/pyspark/broadcast.py @@ -0,0 +1,46 @@ +""" +>>> from pyspark.context import SparkContext +>>> sc = SparkContext('local', 'test') +>>> b = sc.broadcast([1, 2, 3, 4, 5]) +>>> b.value +[1, 2, 3, 4, 5] + +>>> from pyspark.broadcast import _broadcastRegistry +>>> _broadcastRegistry[b.uuid] = b +>>> from cPickle import dumps, loads +>>> loads(dumps(b)).value +[1, 2, 3, 4, 5] + +>>> sc.parallelize([0, 0]).flatMap(lambda x: b.value).collect() +[1, 2, 3, 4, 5, 1, 2, 3, 4, 5] +""" +# Holds broadcasted data received from Java, keyed by UUID. +_broadcastRegistry = {} + + +def _from_uuid(uuid): + from pyspark.broadcast import _broadcastRegistry + if uuid not in _broadcastRegistry: + raise Exception("Broadcast variable '%s' not loaded!" % uuid) + return _broadcastRegistry[uuid] + + +class Broadcast(object): + def __init__(self, uuid, value, java_broadcast=None, pickle_registry=None): + self.value = value + self.uuid = uuid + self._jbroadcast = java_broadcast + self._pickle_registry = pickle_registry + + def __reduce__(self): + self._pickle_registry.add(self) + return (_from_uuid, (self.uuid, )) + + +def _test(): + import doctest + doctest.testmod() + + +if __name__ == "__main__": + _test() diff --git a/pyspark/pyspark/context.py b/pyspark/pyspark/context.py index ac7e4057e9..6f87206665 100644 --- a/pyspark/pyspark/context.py +++ b/pyspark/pyspark/context.py @@ -2,6 +2,7 @@ import os import atexit from tempfile import NamedTemporaryFile +from pyspark.broadcast import Broadcast from pyspark.java_gateway import launch_gateway from pyspark.serializers import PickleSerializer, dumps from pyspark.rdd import RDD @@ -24,6 +25,11 @@ class SparkContext(object): self.defaultParallelism = \ defaultParallelism or self._jsc.sc().defaultParallelism() self.pythonExec = pythonExec + # Broadcast's __reduce__ method stores Broadcast instances here. + # This allows other code to determine which Broadcast instances have + # been pickled, so it can determine which Java broadcast objects to + # send. + self._pickled_broadcast_vars = set() def __del__(self): if self._jsc: @@ -52,7 +58,12 @@ class SparkContext(object): jrdd = self.pickleFile(self._jsc, tempFile.name, numSlices) return RDD(jrdd, self) - def textFile(self, name, numSlices=None): - numSlices = numSlices or self.defaultParallelism - jrdd = self._jsc.textFile(name, numSlices) + def textFile(self, name, minSplits=None): + minSplits = minSplits or min(self.defaultParallelism, 2) + jrdd = self._jsc.textFile(name, minSplits) return RDD(jrdd, self) + + def broadcast(self, value): + jbroadcast = self._jsc.broadcast(bytearray(PickleSerializer.dumps(value))) + return Broadcast(jbroadcast.uuid().toString(), value, jbroadcast, + self._pickled_broadcast_vars) diff --git a/pyspark/pyspark/rdd.py b/pyspark/pyspark/rdd.py index af7703fdfc..4459095391 100644 --- a/pyspark/pyspark/rdd.py +++ b/pyspark/pyspark/rdd.py @@ -6,6 +6,8 @@ from pyspark.serializers import PickleSerializer from pyspark.join import python_join, python_left_outer_join, \ python_right_outer_join, python_cogroup +from py4j.java_collections import ListConverter + class RDD(object): @@ -15,11 +17,15 @@ class RDD(object): self.ctx = ctx @classmethod - def _get_pipe_command(cls, command, functions): + def _get_pipe_command(cls, ctx, command, functions): worker_args = [command] for f in functions: worker_args.append(b64enc(cloudpickle.dumps(f))) - return " ".join(worker_args) + broadcast_vars = [x._jbroadcast for x in ctx._pickled_broadcast_vars] + broadcast_vars = ListConverter().convert(broadcast_vars, + ctx.gateway._gateway_client) + ctx._pickled_broadcast_vars.clear() + return (" ".join(worker_args), broadcast_vars) def cache(self): self.is_cached = True @@ -52,9 +58,10 @@ class RDD(object): def _pipe(self, functions, command): class_manifest = self._jrdd.classManifest() - pipe_command = RDD._get_pipe_command(command, functions) + (pipe_command, broadcast_vars) = \ + RDD._get_pipe_command(self.ctx, command, functions) python_rdd = self.ctx.jvm.PythonRDD(self._jrdd.rdd(), pipe_command, - False, self.ctx.pythonExec, class_manifest) + False, self.ctx.pythonExec, broadcast_vars, class_manifest) return python_rdd.asJavaRDD() def distinct(self): @@ -249,10 +256,12 @@ class RDD(object): def shuffle(self, numSplits, hashFunc=hash): if numSplits is None: numSplits = self.ctx.defaultParallelism - pipe_command = RDD._get_pipe_command('shuffle_map_step', [hashFunc]) + (pipe_command, broadcast_vars) = \ + RDD._get_pipe_command(self.ctx, 'shuffle_map_step', [hashFunc]) class_manifest = self._jrdd.classManifest() python_rdd = self.ctx.jvm.PythonPairRDD(self._jrdd.rdd(), - pipe_command, False, self.ctx.pythonExec, class_manifest) + pipe_command, False, self.ctx.pythonExec, broadcast_vars, + class_manifest) partitioner = self.ctx.jvm.spark.HashPartitioner(numSplits) jrdd = python_rdd.asJavaPairRDD().partitionBy(partitioner) jrdd = jrdd.map(self.ctx.jvm.ExtractValue()) @@ -360,12 +369,12 @@ class PipelinedRDD(RDD): @property def _jrdd(self): if not self._jrdd_val: - funcs = [self.func] - pipe_command = RDD._get_pipe_command("pipeline", funcs) + (pipe_command, broadcast_vars) = \ + RDD._get_pipe_command(self.ctx, "pipeline", [self.func]) class_manifest = self._prev_jrdd.classManifest() python_rdd = self.ctx.jvm.PythonRDD(self._prev_jrdd.rdd(), pipe_command, self.preservesPartitioning, self.ctx.pythonExec, - class_manifest) + broadcast_vars, class_manifest) self._jrdd_val = python_rdd.asJavaRDD() return self._jrdd_val diff --git a/pyspark/pyspark/worker.py b/pyspark/pyspark/worker.py index 76b09918e7..7402897ac8 100644 --- a/pyspark/pyspark/worker.py +++ b/pyspark/pyspark/worker.py @@ -5,6 +5,7 @@ import sys from base64 import standard_b64decode # CloudPickler needs to be imported so that depicklers are registered using the # copy_reg module. +from pyspark.broadcast import Broadcast, _broadcastRegistry from pyspark.cloudpickle import CloudPickler from pyspark.serializers import dumps, loads, PickleSerializer import cPickle @@ -63,6 +64,11 @@ def do_shuffle_map_step(): def main(): + num_broadcast_variables = int(sys.stdin.readline().strip()) + for _ in range(num_broadcast_variables): + uuid = sys.stdin.read(36) + value = loads(sys.stdin) + _broadcastRegistry[uuid] = Broadcast(uuid, cPickle.loads(value)) command = sys.stdin.readline().strip() if command == "pipeline": do_pipeline() From 08b201d810c0dc0933d00d78ec2c1d9135e100c3 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 24 Aug 2012 22:51:45 -0700 Subject: [PATCH 09/85] Add mapPartitions(), glom(), countByValue() to Python API. --- pyspark/pyspark/rdd.py | 32 ++++++++++++++++++++++++++++---- 1 file changed, 28 insertions(+), 4 deletions(-) diff --git a/pyspark/pyspark/rdd.py b/pyspark/pyspark/rdd.py index 4459095391..f0d665236a 100644 --- a/pyspark/pyspark/rdd.py +++ b/pyspark/pyspark/rdd.py @@ -1,4 +1,5 @@ from base64 import standard_b64encode as b64enc +from collections import Counter from itertools import chain, ifilter, imap from pyspark import cloudpickle @@ -47,6 +48,15 @@ class RDD(object): def func(iterator): return chain.from_iterable(imap(f, iterator)) return PipelinedRDD(self, func) + def mapPartitions(self, f): + """ + >>> rdd = sc.parallelize([1, 2, 3, 4], 2) + >>> def f(iterator): yield sum(iterator) + >>> rdd.mapPartitions(f).collect() + [3, 7] + """ + return PipelinedRDD(self, f) + def filter(self, f): """ >>> rdd = sc.parallelize([1, 2, 3, 4, 5]) @@ -93,7 +103,14 @@ class RDD(object): # TODO: Overload __add___? - # TODO: glom + def glom(self): + """ + >>> rdd = sc.parallelize([1, 2, 3, 4], 2) + >>> rdd.glom().first() + [1, 2] + """ + def func(iterator): yield list(iterator) + return PipelinedRDD(self, func) def cartesian(self, other): """ @@ -115,8 +132,6 @@ class RDD(object): # TODO: pipe - # TODO: mapPartitions - def foreach(self, f): """ >>> def f(x): print x @@ -177,7 +192,16 @@ class RDD(object): """ return self._jrdd.count() - # TODO: count approx methods + def countByValue(self): + """ + >>> sc.parallelize([1, 2, 1, 2, 2]).countByValue().most_common() + [(2, 3), (1, 2)] + """ + def countPartition(iterator): + yield Counter(iterator) + def mergeMaps(m1, m2): + return m1 + m2 + return self.mapPartitions(countPartition).reduce(mergeMaps) def take(self, num): """ From 8b64b7ecd80c52f2f09a517f1517c0ece7a3d57f Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 24 Aug 2012 23:09:15 -0700 Subject: [PATCH 10/85] Add countByKey(), reduceByKeyLocally() to Python API --- pyspark/pyspark/rdd.py | 52 +++++++++++++++++++++++++++++++----------- 1 file changed, 39 insertions(+), 13 deletions(-) diff --git a/pyspark/pyspark/rdd.py b/pyspark/pyspark/rdd.py index f0d665236a..fd41ea0b17 100644 --- a/pyspark/pyspark/rdd.py +++ b/pyspark/pyspark/rdd.py @@ -99,9 +99,17 @@ class RDD(object): """ return RDD(self._jrdd.union(other._jrdd), self.ctx) - # TODO: sort + def __add__(self, other): + """ + >>> rdd = sc.parallelize([1, 1, 2, 3]) + >>> (rdd + rdd).collect() + [1, 1, 2, 3, 1, 1, 2, 3] + """ + if not isinstance(other, RDD): + raise TypeError + return self.union(other) - # TODO: Overload __add___? + # TODO: sort def glom(self): """ @@ -120,7 +128,6 @@ class RDD(object): """ return RDD(self._jrdd.cartesian(other._jrdd), self.ctx) - # numsplits def groupBy(self, f, numSplits=None): """ >>> rdd = sc.parallelize([1, 1, 2, 3, 5, 8]) @@ -236,17 +243,38 @@ class RDD(object): def reduceByKey(self, func, numSplits=None): """ - >>> x = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) - >>> sorted(x.reduceByKey(lambda a, b: a + b).collect()) + >>> from operator import add + >>> rdd = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) + >>> sorted(rdd.reduceByKey(add).collect()) [('a', 2), ('b', 1)] """ return self.combineByKey(lambda x: x, func, func, numSplits) - # TODO: reduceByKeyLocally() + def reduceByKeyLocally(self, func): + """ + >>> from operator import add + >>> rdd = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) + >>> sorted(rdd.reduceByKeyLocally(add).items()) + [('a', 2), ('b', 1)] + """ + def reducePartition(iterator): + m = {} + for (k, v) in iterator: + m[k] = v if k not in m else func(m[k], v) + yield m + def mergeMaps(m1, m2): + for (k, v) in m2.iteritems(): + m1[k] = v if k not in m1 else func(m1[k], v) + return m1 + return self.mapPartitions(reducePartition).reduce(mergeMaps) - # TODO: countByKey() - - # TODO: partitionBy + def countByKey(self): + """ + >>> rdd = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) + >>> rdd.countByKey().most_common() + [('a', 2), ('b', 1)] + """ + return self.map(lambda x: x[0]).countByValue() def join(self, other, numSplits=None): """ @@ -277,7 +305,7 @@ class RDD(object): # TODO: pipelining # TODO: optimizations - def shuffle(self, numSplits, hashFunc=hash): + def partitionBy(self, numSplits, hashFunc=hash): if numSplits is None: numSplits = self.ctx.defaultParallelism (pipe_command, broadcast_vars) = \ @@ -302,7 +330,7 @@ class RDD(object): """ if numSplits is None: numSplits = self.ctx.defaultParallelism - shuffled = self.shuffle(numSplits) + shuffled = self.partitionBy(numSplits) functions = [createCombiner, mergeValue, mergeCombiners] jpairs = shuffled._pipe(functions, "combine_by_key") return RDD(jpairs, self.ctx) @@ -353,8 +381,6 @@ class RDD(object): # keys in the pairs. This could be an expensive operation, since those # hashes aren't retained. - # TODO: file saving - class PipelinedRDD(RDD): """ From 6904cb77d4306a14891cc71338c8f9f966d009f1 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 25 Aug 2012 14:19:07 -0700 Subject: [PATCH 11/85] Use local combiners in Python API combineByKey(). --- pyspark/pyspark/rdd.py | 33 ++++++++++++++++++++++++--------- pyspark/pyspark/worker.py | 16 ---------------- 2 files changed, 24 insertions(+), 25 deletions(-) diff --git a/pyspark/pyspark/rdd.py b/pyspark/pyspark/rdd.py index fd41ea0b17..3528b8f308 100644 --- a/pyspark/pyspark/rdd.py +++ b/pyspark/pyspark/rdd.py @@ -46,7 +46,7 @@ class RDD(object): [(2, 2), (2, 2), (3, 3), (3, 3), (4, 4), (4, 4)] """ def func(iterator): return chain.from_iterable(imap(f, iterator)) - return PipelinedRDD(self, func) + return self.mapPartitions(func) def mapPartitions(self, f): """ @@ -64,7 +64,7 @@ class RDD(object): [2, 4] """ def func(iterator): return ifilter(f, iterator) - return PipelinedRDD(self, func) + return self.mapPartitions(func) def _pipe(self, functions, command): class_manifest = self._jrdd.classManifest() @@ -118,7 +118,7 @@ class RDD(object): [1, 2] """ def func(iterator): yield list(iterator) - return PipelinedRDD(self, func) + return self.mapPartitions(func) def cartesian(self, other): """ @@ -167,7 +167,7 @@ class RDD(object): acc = f(obj, acc) if acc is not None: yield acc - vals = PipelinedRDD(self, func).collect() + vals = self.mapPartitions(func).collect() return reduce(f, vals) def fold(self, zeroValue, op): @@ -187,7 +187,7 @@ class RDD(object): for obj in iterator: acc = op(obj, acc) yield acc - vals = PipelinedRDD(self, func).collect() + vals = self.mapPartitions(func).collect() return reduce(op, vals, zeroValue) # TODO: aggregate @@ -330,10 +330,25 @@ class RDD(object): """ if numSplits is None: numSplits = self.ctx.defaultParallelism - shuffled = self.partitionBy(numSplits) - functions = [createCombiner, mergeValue, mergeCombiners] - jpairs = shuffled._pipe(functions, "combine_by_key") - return RDD(jpairs, self.ctx) + def combineLocally(iterator): + combiners = {} + for (k, v) in iterator: + if k not in combiners: + combiners[k] = createCombiner(v) + else: + combiners[k] = mergeValue(combiners[k], v) + return combiners.iteritems() + locally_combined = self.mapPartitions(combineLocally) + shuffled = locally_combined.partitionBy(numSplits) + def _mergeCombiners(iterator): + combiners = {} + for (k, v) in iterator: + if not k in combiners: + combiners[k] = v + else: + combiners[k] = mergeCombiners(combiners[k], v) + return combiners.iteritems() + return shuffled.mapPartitions(_mergeCombiners) def groupByKey(self, numSplits=None): """ diff --git a/pyspark/pyspark/worker.py b/pyspark/pyspark/worker.py index 7402897ac8..0f90c6ff46 100644 --- a/pyspark/pyspark/worker.py +++ b/pyspark/pyspark/worker.py @@ -31,20 +31,6 @@ def read_input(): return -def do_combine_by_key(): - create_combiner = load_function() - merge_value = load_function() - merge_combiners = load_function() # TODO: not used. - combiners = {} - for (key, value) in read_input(): - if key not in combiners: - combiners[key] = create_combiner(value) - else: - combiners[key] = merge_value(combiners[key], value) - for (key, combiner) in combiners.iteritems(): - output(PickleSerializer.dumps((key, combiner))) - - def do_pipeline(): f = load_function() for obj in f(read_input()): @@ -72,8 +58,6 @@ def main(): command = sys.stdin.readline().strip() if command == "pipeline": do_pipeline() - elif command == "combine_by_key": - do_combine_by_key() elif command == "shuffle_map_step": do_shuffle_map_step() else: From 200d248dcc5903295296bf897211cf543b37f8c1 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 25 Aug 2012 16:46:07 -0700 Subject: [PATCH 12/85] Simplify Python worker; pipeline the map step of partitionBy(). --- .../scala/spark/api/python/PythonRDD.scala | 34 ++------- pyspark/pyspark/context.py | 9 ++- pyspark/pyspark/rdd.py | 70 +++++++------------ pyspark/pyspark/serializers.py | 23 ++---- pyspark/pyspark/worker.py | 50 ++++--------- 5 files changed, 59 insertions(+), 127 deletions(-) diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 5163812df4..b9091fd436 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -151,38 +151,18 @@ class PythonRDD[T: ClassManifest]( val asJavaRDD : JavaRDD[Array[Byte]] = JavaRDD.fromRDD(this) } -class PythonPairRDD[T: ClassManifest] ( - parent: RDD[T], command: Seq[String], envVars: Map[String, String], - preservePartitoning: Boolean, pythonExec: String, broadcastVars: java.util.List[Broadcast[Array[Byte]]]) - extends RDD[(Array[Byte], Array[Byte])](parent.context) with PythonRDDBase { - - def this(parent: RDD[T], command: Seq[String], preservePartitoning: Boolean, - pythonExec: String, broadcastVars: java.util.List[Broadcast[Array[Byte]]]) = - this(parent, command, Map(), preservePartitoning, pythonExec, broadcastVars) - - // Similar to Runtime.exec(), if we are given a single string, split it into words - // using a standard StringTokenizer (i.e. by spaces) - def this(parent: RDD[T], command: String, preservePartitoning: Boolean, pythonExec: String, - broadcastVars: java.util.List[Broadcast[Array[Byte]]]) = - this(parent, PipedRDD.tokenize(command), preservePartitoning, pythonExec, broadcastVars) - - override def splits = parent.splits - - override val dependencies = List(new OneToOneDependency(parent)) - - override val partitioner = if (preservePartitoning) parent.partitioner else None - - override def compute(split: Split): Iterator[(Array[Byte], Array[Byte])] = { - compute(split, envVars, command, parent, pythonExec, broadcastVars).grouped(2).map { +private class PairwiseRDD(prev: RDD[Array[Byte]]) extends + RDD[(Array[Byte], Array[Byte])](prev.context) { + override def splits = prev.splits + override val dependencies = List(new OneToOneDependency(prev)) + override def compute(split: Split) = + prev.iterator(split).grouped(2).map { case Seq(a, b) => (a, b) - case x => throw new Exception("PythonPairRDD: unexpected value: " + x) + case x => throw new Exception("PairwiseRDD: unexpected value: " + x) } - } - val asJavaPairRDD : JavaPairRDD[Array[Byte], Array[Byte]] = JavaPairRDD.fromRDD(this) } - object PythonRDD { /** Strips the pickle PROTO and STOP opcodes from the start and end of a pickle */ diff --git a/pyspark/pyspark/context.py b/pyspark/pyspark/context.py index 6f87206665..b8490019e3 100644 --- a/pyspark/pyspark/context.py +++ b/pyspark/pyspark/context.py @@ -4,7 +4,7 @@ from tempfile import NamedTemporaryFile from pyspark.broadcast import Broadcast from pyspark.java_gateway import launch_gateway -from pyspark.serializers import PickleSerializer, dumps +from pyspark.serializers import dump_pickle, write_with_length from pyspark.rdd import RDD @@ -16,9 +16,8 @@ class SparkContext(object): asPickle = jvm.spark.api.python.PythonRDD.asPickle arrayAsPickle = jvm.spark.api.python.PythonRDD.arrayAsPickle - def __init__(self, master, name, defaultParallelism=None, - pythonExec='python'): + pythonExec='python'): self.master = master self.name = name self._jsc = self.jvm.JavaSparkContext(master, name) @@ -52,7 +51,7 @@ class SparkContext(object): # objects are written to a file and loaded through textFile(). tempFile = NamedTemporaryFile(delete=False) for x in c: - dumps(PickleSerializer.dumps(x), tempFile) + write_with_length(dump_pickle(x), tempFile) tempFile.close() atexit.register(lambda: os.unlink(tempFile.name)) jrdd = self.pickleFile(self._jsc, tempFile.name, numSlices) @@ -64,6 +63,6 @@ class SparkContext(object): return RDD(jrdd, self) def broadcast(self, value): - jbroadcast = self._jsc.broadcast(bytearray(PickleSerializer.dumps(value))) + jbroadcast = self._jsc.broadcast(bytearray(dump_pickle(value))) return Broadcast(jbroadcast.uuid().toString(), value, jbroadcast, self._pickled_broadcast_vars) diff --git a/pyspark/pyspark/rdd.py b/pyspark/pyspark/rdd.py index 3528b8f308..21e822ba9f 100644 --- a/pyspark/pyspark/rdd.py +++ b/pyspark/pyspark/rdd.py @@ -3,7 +3,7 @@ from collections import Counter from itertools import chain, ifilter, imap from pyspark import cloudpickle -from pyspark.serializers import PickleSerializer +from pyspark.serializers import dump_pickle, load_pickle from pyspark.join import python_join, python_left_outer_join, \ python_right_outer_join, python_cogroup @@ -17,17 +17,6 @@ class RDD(object): self.is_cached = False self.ctx = ctx - @classmethod - def _get_pipe_command(cls, ctx, command, functions): - worker_args = [command] - for f in functions: - worker_args.append(b64enc(cloudpickle.dumps(f))) - broadcast_vars = [x._jbroadcast for x in ctx._pickled_broadcast_vars] - broadcast_vars = ListConverter().convert(broadcast_vars, - ctx.gateway._gateway_client) - ctx._pickled_broadcast_vars.clear() - return (" ".join(worker_args), broadcast_vars) - def cache(self): self.is_cached = True self._jrdd.cache() @@ -66,14 +55,6 @@ class RDD(object): def func(iterator): return ifilter(f, iterator) return self.mapPartitions(func) - def _pipe(self, functions, command): - class_manifest = self._jrdd.classManifest() - (pipe_command, broadcast_vars) = \ - RDD._get_pipe_command(self.ctx, command, functions) - python_rdd = self.ctx.jvm.PythonRDD(self._jrdd.rdd(), pipe_command, - False, self.ctx.pythonExec, broadcast_vars, class_manifest) - return python_rdd.asJavaRDD() - def distinct(self): """ >>> sorted(sc.parallelize([1, 1, 2, 3]).distinct().collect()) @@ -89,7 +70,7 @@ class RDD(object): def takeSample(self, withReplacement, num, seed): vals = self._jrdd.takeSample(withReplacement, num, seed) - return [PickleSerializer.loads(bytes(x)) for x in vals] + return [load_pickle(bytes(x)) for x in vals] def union(self, other): """ @@ -148,7 +129,7 @@ class RDD(object): def collect(self): pickle = self.ctx.arrayAsPickle(self._jrdd.rdd().collect()) - return PickleSerializer.loads(bytes(pickle)) + return load_pickle(bytes(pickle)) def reduce(self, f): """ @@ -216,19 +197,17 @@ class RDD(object): [2, 3] """ pickle = self.ctx.arrayAsPickle(self._jrdd.rdd().take(num)) - return PickleSerializer.loads(bytes(pickle)) + return load_pickle(bytes(pickle)) def first(self): """ >>> sc.parallelize([2, 3, 4]).first() 2 """ - return PickleSerializer.loads(bytes(self.ctx.asPickle(self._jrdd.first()))) + return load_pickle(bytes(self.ctx.asPickle(self._jrdd.first()))) # TODO: saveAsTextFile - # TODO: saveAsObjectFile - # Pair functions def collectAsMap(self): @@ -303,19 +282,18 @@ class RDD(object): """ return python_right_outer_join(self, other, numSplits) - # TODO: pipelining - # TODO: optimizations def partitionBy(self, numSplits, hashFunc=hash): if numSplits is None: numSplits = self.ctx.defaultParallelism - (pipe_command, broadcast_vars) = \ - RDD._get_pipe_command(self.ctx, 'shuffle_map_step', [hashFunc]) - class_manifest = self._jrdd.classManifest() - python_rdd = self.ctx.jvm.PythonPairRDD(self._jrdd.rdd(), - pipe_command, False, self.ctx.pythonExec, broadcast_vars, - class_manifest) + def add_shuffle_key(iterator): + for (k, v) in iterator: + yield str(hashFunc(k)) + yield dump_pickle((k, v)) + keyed = PipelinedRDD(self, add_shuffle_key) + keyed._bypass_serializer = True + pairRDD = self.ctx.jvm.PairwiseRDD(keyed._jrdd.rdd()).asJavaPairRDD() partitioner = self.ctx.jvm.spark.HashPartitioner(numSplits) - jrdd = python_rdd.asJavaPairRDD().partitionBy(partitioner) + jrdd = pairRDD.partitionBy(partitioner) jrdd = jrdd.map(self.ctx.jvm.ExtractValue()) return RDD(jrdd, self.ctx) @@ -430,17 +408,23 @@ class PipelinedRDD(RDD): self.ctx = prev.ctx self.prev = prev self._jrdd_val = None + self._bypass_serializer = False @property def _jrdd(self): - if not self._jrdd_val: - (pipe_command, broadcast_vars) = \ - RDD._get_pipe_command(self.ctx, "pipeline", [self.func]) - class_manifest = self._prev_jrdd.classManifest() - python_rdd = self.ctx.jvm.PythonRDD(self._prev_jrdd.rdd(), - pipe_command, self.preservesPartitioning, self.ctx.pythonExec, - broadcast_vars, class_manifest) - self._jrdd_val = python_rdd.asJavaRDD() + if self._jrdd_val: + return self._jrdd_val + funcs = [self.func, self._bypass_serializer] + pipe_command = ' '.join(b64enc(cloudpickle.dumps(f)) for f in funcs) + broadcast_vars = ListConverter().convert( + [x._jbroadcast for x in self.ctx._pickled_broadcast_vars], + self.ctx.gateway._gateway_client) + self.ctx._pickled_broadcast_vars.clear() + class_manifest = self._prev_jrdd.classManifest() + python_rdd = self.ctx.jvm.PythonRDD(self._prev_jrdd.rdd(), + pipe_command, self.preservesPartitioning, self.ctx.pythonExec, + broadcast_vars, class_manifest) + self._jrdd_val = python_rdd.asJavaRDD() return self._jrdd_val diff --git a/pyspark/pyspark/serializers.py b/pyspark/pyspark/serializers.py index 7b3e6966e1..faa1e683c7 100644 --- a/pyspark/pyspark/serializers.py +++ b/pyspark/pyspark/serializers.py @@ -1,31 +1,20 @@ -""" -Data serialization methods. - -The Spark Python API is built on top of the Spark Java API. RDDs created in -Python are stored in Java as RDD[Array[Byte]]. Python objects are -automatically serialized/deserialized, so this representation is transparent to -the end-user. -""" -from collections import namedtuple -import cPickle import struct +import cPickle -Serializer = namedtuple("Serializer", ["dumps","loads"]) +def dump_pickle(obj): + return cPickle.dumps(obj, 2) -PickleSerializer = Serializer( - lambda obj: cPickle.dumps(obj, -1), - cPickle.loads) +load_pickle = cPickle.loads -def dumps(obj, stream): - # TODO: determining the length of non-byte objects. +def write_with_length(obj, stream): stream.write(struct.pack("!i", len(obj))) stream.write(obj) -def loads(stream): +def read_with_length(stream): length = stream.read(4) if length == "": raise EOFError diff --git a/pyspark/pyspark/worker.py b/pyspark/pyspark/worker.py index 0f90c6ff46..a9ed71892f 100644 --- a/pyspark/pyspark/worker.py +++ b/pyspark/pyspark/worker.py @@ -7,61 +7,41 @@ from base64 import standard_b64decode # copy_reg module. from pyspark.broadcast import Broadcast, _broadcastRegistry from pyspark.cloudpickle import CloudPickler -from pyspark.serializers import dumps, loads, PickleSerializer -import cPickle +from pyspark.serializers import write_with_length, read_with_length, \ + dump_pickle, load_pickle + # Redirect stdout to stderr so that users must return values from functions. old_stdout = sys.stdout sys.stdout = sys.stderr -def load_function(): - return cPickle.loads(standard_b64decode(sys.stdin.readline().strip())) - - -def output(x): - dumps(x, old_stdout) +def load_obj(): + return load_pickle(standard_b64decode(sys.stdin.readline().strip())) def read_input(): try: while True: - yield cPickle.loads(loads(sys.stdin)) + yield load_pickle(read_with_length(sys.stdin)) except EOFError: return -def do_pipeline(): - f = load_function() - for obj in f(read_input()): - output(PickleSerializer.dumps(obj)) - - -def do_shuffle_map_step(): - hashFunc = load_function() - while True: - try: - pickled = loads(sys.stdin) - except EOFError: - return - key = cPickle.loads(pickled)[0] - output(str(hashFunc(key))) - output(pickled) - - def main(): num_broadcast_variables = int(sys.stdin.readline().strip()) for _ in range(num_broadcast_variables): uuid = sys.stdin.read(36) - value = loads(sys.stdin) - _broadcastRegistry[uuid] = Broadcast(uuid, cPickle.loads(value)) - command = sys.stdin.readline().strip() - if command == "pipeline": - do_pipeline() - elif command == "shuffle_map_step": - do_shuffle_map_step() + value = read_with_length(sys.stdin) + _broadcastRegistry[uuid] = Broadcast(uuid, load_pickle(value)) + func = load_obj() + bypassSerializer = load_obj() + if bypassSerializer: + dumps = lambda x: x else: - raise Exception("Unsupported command %s" % command) + dumps = dump_pickle + for obj in func(read_input()): + write_with_length(dumps(obj), old_stdout) if __name__ == '__main__': From bff6a46359131a8f9bc38b93149b22baa7c711cd Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 25 Aug 2012 18:00:25 -0700 Subject: [PATCH 13/85] Add pipe(), saveAsTextFile(), sc.union() to Python API. --- .../scala/spark/api/python/PythonRDD.scala | 10 +++++--- pyspark/pyspark/context.py | 14 ++++++----- pyspark/pyspark/rdd.py | 25 +++++++++++++++++-- 3 files changed, 38 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index b9091fd436..4d3bdb3963 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -9,6 +9,7 @@ import spark._ import api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} import broadcast.Broadcast import scala.collection +import java.nio.charset.Charset trait PythonRDDBase { def compute[T](split: Split, envVars: Map[String, String], @@ -238,9 +239,12 @@ private object Pickle { val MARK : Byte = '(' val APPENDS : Byte = 'e' } -class ExtractValue extends spark.api.java.function.Function[(Array[Byte], + +private class ExtractValue extends spark.api.java.function.Function[(Array[Byte], Array[Byte]), Array[Byte]] { - override def call(pair: (Array[Byte], Array[Byte])) : Array[Byte] = pair._2 - +} + +private class BytesToString extends spark.api.java.function.Function[Array[Byte], String] { + override def call(arr: Array[Byte]) : String = new String(arr, "UTF-8") } diff --git a/pyspark/pyspark/context.py b/pyspark/pyspark/context.py index b8490019e3..04932c93f2 100644 --- a/pyspark/pyspark/context.py +++ b/pyspark/pyspark/context.py @@ -7,6 +7,8 @@ from pyspark.java_gateway import launch_gateway from pyspark.serializers import dump_pickle, write_with_length from pyspark.rdd import RDD +from py4j.java_collections import ListConverter + class SparkContext(object): @@ -39,12 +41,6 @@ class SparkContext(object): self._jsc = None def parallelize(self, c, numSlices=None): - """ - >>> sc = SparkContext("local", "test") - >>> rdd = sc.parallelize([(1, 2), (3, 4)]) - >>> rdd.collect() - [(1, 2), (3, 4)] - """ numSlices = numSlices or self.defaultParallelism # Calling the Java parallelize() method with an ArrayList is too slow, # because it sends O(n) Py4J commands. As an alternative, serialized @@ -62,6 +58,12 @@ class SparkContext(object): jrdd = self._jsc.textFile(name, minSplits) return RDD(jrdd, self) + def union(self, rdds): + first = rdds[0]._jrdd + rest = [x._jrdd for x in rdds[1:]] + rest = ListConverter().convert(rest, self.gateway._gateway_client) + return RDD(self._jsc.union(first, rest), self) + def broadcast(self, value): jbroadcast = self._jsc.broadcast(bytearray(dump_pickle(value))) return Broadcast(jbroadcast.uuid().toString(), value, jbroadcast, diff --git a/pyspark/pyspark/rdd.py b/pyspark/pyspark/rdd.py index 21e822ba9f..8477f6dd02 100644 --- a/pyspark/pyspark/rdd.py +++ b/pyspark/pyspark/rdd.py @@ -1,6 +1,9 @@ from base64 import standard_b64encode as b64enc from collections import Counter from itertools import chain, ifilter, imap +import shlex +from subprocess import Popen, PIPE +from threading import Thread from pyspark import cloudpickle from pyspark.serializers import dump_pickle, load_pickle @@ -118,7 +121,20 @@ class RDD(object): """ return self.map(lambda x: (f(x), x)).groupByKey(numSplits) - # TODO: pipe + def pipe(self, command, env={}): + """ + >>> sc.parallelize([1, 2, 3]).pipe('cat').collect() + ['1', '2', '3'] + """ + def func(iterator): + pipe = Popen(shlex.split(command), env=env, stdin=PIPE, stdout=PIPE) + def pipe_objs(out): + for obj in iterator: + out.write(str(obj).rstrip('\n') + '\n') + out.close() + Thread(target=pipe_objs, args=[pipe.stdin]).start() + return (x.rstrip('\n') for x in pipe.stdout) + return self.mapPartitions(func) def foreach(self, f): """ @@ -206,7 +222,12 @@ class RDD(object): """ return load_pickle(bytes(self.ctx.asPickle(self._jrdd.first()))) - # TODO: saveAsTextFile + def saveAsTextFile(self, path): + def func(iterator): + return (str(x).encode("utf-8") for x in iterator) + keyed = PipelinedRDD(self, func) + keyed._bypass_serializer = True + keyed._jrdd.map(self.ctx.jvm.BytesToString()).saveAsTextFile(path) # Pair functions From 414367850982c4f8fc5e63cc94caa422eb736db5 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 27 Aug 2012 00:13:19 -0700 Subject: [PATCH 14/85] Fix minor bugs in Python API examples. --- pyspark/pyspark/examples/pi.py | 2 +- pyspark/pyspark/examples/tc.py | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/pyspark/pyspark/examples/pi.py b/pyspark/pyspark/examples/pi.py index fe63d2c952..348bbc5dce 100644 --- a/pyspark/pyspark/examples/pi.py +++ b/pyspark/pyspark/examples/pi.py @@ -9,7 +9,7 @@ if __name__ == "__main__": print >> sys.stderr, \ "Usage: PythonPi []" exit(-1) - sc = SparkContext(sys.argv[1], "PythonKMeans") + sc = SparkContext(sys.argv[1], "PythonPi") slices = int(sys.argv[2]) if len(sys.argv) > 2 else 2 n = 100000 * slices def f(_): diff --git a/pyspark/pyspark/examples/tc.py b/pyspark/pyspark/examples/tc.py index 2796fdc6ad..9630e72b47 100644 --- a/pyspark/pyspark/examples/tc.py +++ b/pyspark/pyspark/examples/tc.py @@ -22,9 +22,9 @@ if __name__ == "__main__": print >> sys.stderr, \ "Usage: PythonTC []" exit(-1) - sc = SparkContext(sys.argv[1], "PythonKMeans") + sc = SparkContext(sys.argv[1], "PythonTC") slices = sys.argv[2] if len(sys.argv) > 2 else 2 - tc = sc.parallelizePairs(generateGraph(), slices).cache() + tc = sc.parallelize(generateGraph(), slices).cache() # Linear transitive closure: each round grows paths by one edge, # by joining the graph's edges with the already-discovered paths. @@ -32,7 +32,7 @@ if __name__ == "__main__": # the graph to obtain the path (x, z). # Because join() joins on keys, the edges are stored in reversed order. - edges = tc.mapPairs(lambda (x, y): (y, x)) + edges = tc.map(lambda (x, y): (y, x)) oldCount = 0L nextCount = tc.count() @@ -40,7 +40,7 @@ if __name__ == "__main__": oldCount = nextCount # Perform the join, obtaining an RDD of (y, (z, x)) pairs, # then project the result to obtain the new (x, z) paths. - new_edges = tc.join(edges).mapPairs(lambda (_, (a, b)): (b, a)) + new_edges = tc.join(edges).map(lambda (_, (a, b)): (b, a)) tc = tc.union(new_edges).distinct().cache() nextCount = tc.count() if nextCount == oldCount: From 9abdfa663360252d2edb346e6b3df4ff94ce78d7 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 17 Sep 2012 00:08:50 -0700 Subject: [PATCH 15/85] Fix Python 2.6 compatibility in Python API. --- pyspark/pyspark/rdd.py | 17 +++++++++++------ python/tc.py | 22 ---------------------- 2 files changed, 11 insertions(+), 28 deletions(-) delete mode 100644 python/tc.py diff --git a/pyspark/pyspark/rdd.py b/pyspark/pyspark/rdd.py index 8477f6dd02..e2137fe06c 100644 --- a/pyspark/pyspark/rdd.py +++ b/pyspark/pyspark/rdd.py @@ -1,5 +1,5 @@ from base64 import standard_b64encode as b64enc -from collections import Counter +from collections import defaultdict from itertools import chain, ifilter, imap import shlex from subprocess import Popen, PIPE @@ -198,13 +198,18 @@ class RDD(object): def countByValue(self): """ - >>> sc.parallelize([1, 2, 1, 2, 2]).countByValue().most_common() - [(2, 3), (1, 2)] + >>> sorted(sc.parallelize([1, 2, 1, 2, 2], 2).countByValue().items()) + [(1, 2), (2, 3)] """ def countPartition(iterator): - yield Counter(iterator) + counts = defaultdict(int) + for obj in iterator: + counts[obj] += 1 + yield counts def mergeMaps(m1, m2): - return m1 + m2 + for (k, v) in m2.iteritems(): + m1[k] += v + return m1 return self.mapPartitions(countPartition).reduce(mergeMaps) def take(self, num): @@ -271,7 +276,7 @@ class RDD(object): def countByKey(self): """ >>> rdd = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) - >>> rdd.countByKey().most_common() + >>> sorted(rdd.countByKey().items()) [('a', 2), ('b', 1)] """ return self.map(lambda x: x[0]).countByValue() diff --git a/python/tc.py b/python/tc.py deleted file mode 100644 index 5dcc4317e0..0000000000 --- a/python/tc.py +++ /dev/null @@ -1,22 +0,0 @@ -from rdd import SparkContext - -sc = SparkContext("local", "PythonWordCount") -e = [(1, 2), (2, 3), (4, 1)] - -tc = sc.parallelizePairs(e) - -edges = tc.mapPairs(lambda (x, y): (y, x)) - -oldCount = 0 -nextCount = tc.count() - -def project(x): - return (x[1][1], x[1][0]) - -while nextCount != oldCount: - oldCount = nextCount - tc = tc.union(tc.join(edges).mapPairs(project)).distinct() - nextCount = tc.count() - -print "TC has %i edges" % tc.count() -print tc.collect() From 52989c8a2c8c10d7f5610c033f6782e58fd3abc2 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 19 Oct 2012 10:24:49 -0700 Subject: [PATCH 16/85] Update Python API for v0.6.0 compatibility. --- .../scala/spark/api/python/PythonRDD.scala | 18 +++++++++++------- .../main/scala/spark/broadcast/Broadcast.scala | 2 +- pyspark/pyspark/broadcast.py | 18 +++++++++--------- pyspark/pyspark/context.py | 2 +- pyspark/pyspark/java_gateway.py | 3 ++- pyspark/pyspark/serializers.py | 18 ++++++++++++++---- pyspark/pyspark/worker.py | 8 ++++---- 7 files changed, 42 insertions(+), 27 deletions(-) diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 4d3bdb3963..528885fe5c 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -5,11 +5,15 @@ import java.io._ import scala.collection.Map import scala.collection.JavaConversions._ import scala.io.Source -import spark._ -import api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} -import broadcast.Broadcast -import scala.collection -import java.nio.charset.Charset + +import spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} +import spark.broadcast.Broadcast +import spark.SparkEnv +import spark.Split +import spark.RDD +import spark.OneToOneDependency +import spark.rdd.PipedRDD + trait PythonRDDBase { def compute[T](split: Split, envVars: Map[String, String], @@ -43,9 +47,9 @@ trait PythonRDDBase { SparkEnv.set(env) val out = new PrintWriter(proc.getOutputStream) val dOut = new DataOutputStream(proc.getOutputStream) - out.println(broadcastVars.length) + dOut.writeInt(broadcastVars.length) for (broadcast <- broadcastVars) { - out.print(broadcast.uuid.toString) + dOut.writeLong(broadcast.id) dOut.writeInt(broadcast.value.length) dOut.write(broadcast.value) dOut.flush() diff --git a/core/src/main/scala/spark/broadcast/Broadcast.scala b/core/src/main/scala/spark/broadcast/Broadcast.scala index 6055bfd045..2ffe7f741d 100644 --- a/core/src/main/scala/spark/broadcast/Broadcast.scala +++ b/core/src/main/scala/spark/broadcast/Broadcast.scala @@ -5,7 +5,7 @@ import java.util.concurrent.atomic.AtomicLong import spark._ -abstract class Broadcast[T](id: Long) extends Serializable { +abstract class Broadcast[T](private[spark] val id: Long) extends Serializable { def value: T // We cannot have an abstract readObject here due to some weird issues with diff --git a/pyspark/pyspark/broadcast.py b/pyspark/pyspark/broadcast.py index 1ea17d59af..4cff02b36d 100644 --- a/pyspark/pyspark/broadcast.py +++ b/pyspark/pyspark/broadcast.py @@ -6,7 +6,7 @@ [1, 2, 3, 4, 5] >>> from pyspark.broadcast import _broadcastRegistry ->>> _broadcastRegistry[b.uuid] = b +>>> _broadcastRegistry[b.bid] = b >>> from cPickle import dumps, loads >>> loads(dumps(b)).value [1, 2, 3, 4, 5] @@ -14,27 +14,27 @@ >>> sc.parallelize([0, 0]).flatMap(lambda x: b.value).collect() [1, 2, 3, 4, 5, 1, 2, 3, 4, 5] """ -# Holds broadcasted data received from Java, keyed by UUID. +# Holds broadcasted data received from Java, keyed by its id. _broadcastRegistry = {} -def _from_uuid(uuid): +def _from_id(bid): from pyspark.broadcast import _broadcastRegistry - if uuid not in _broadcastRegistry: - raise Exception("Broadcast variable '%s' not loaded!" % uuid) - return _broadcastRegistry[uuid] + if bid not in _broadcastRegistry: + raise Exception("Broadcast variable '%s' not loaded!" % bid) + return _broadcastRegistry[bid] class Broadcast(object): - def __init__(self, uuid, value, java_broadcast=None, pickle_registry=None): + def __init__(self, bid, value, java_broadcast=None, pickle_registry=None): self.value = value - self.uuid = uuid + self.bid = bid self._jbroadcast = java_broadcast self._pickle_registry = pickle_registry def __reduce__(self): self._pickle_registry.add(self) - return (_from_uuid, (self.uuid, )) + return (_from_id, (self.bid, )) def _test(): diff --git a/pyspark/pyspark/context.py b/pyspark/pyspark/context.py index 04932c93f2..3f4db26644 100644 --- a/pyspark/pyspark/context.py +++ b/pyspark/pyspark/context.py @@ -66,5 +66,5 @@ class SparkContext(object): def broadcast(self, value): jbroadcast = self._jsc.broadcast(bytearray(dump_pickle(value))) - return Broadcast(jbroadcast.uuid().toString(), value, jbroadcast, + return Broadcast(jbroadcast.id(), value, jbroadcast, self._pickled_broadcast_vars) diff --git a/pyspark/pyspark/java_gateway.py b/pyspark/pyspark/java_gateway.py index bcb405ba72..3726bcbf17 100644 --- a/pyspark/pyspark/java_gateway.py +++ b/pyspark/pyspark/java_gateway.py @@ -7,7 +7,8 @@ SPARK_HOME = os.environ["SPARK_HOME"] assembly_jar = glob.glob(os.path.join(SPARK_HOME, "core/target") + \ - "/spark-core-assembly-*-SNAPSHOT.jar")[0] + "/spark-core-assembly-*.jar")[0] + # TODO: what if multiple assembly jars are found? def launch_gateway(): diff --git a/pyspark/pyspark/serializers.py b/pyspark/pyspark/serializers.py index faa1e683c7..21ef8b106c 100644 --- a/pyspark/pyspark/serializers.py +++ b/pyspark/pyspark/serializers.py @@ -9,16 +9,26 @@ def dump_pickle(obj): load_pickle = cPickle.loads +def read_long(stream): + length = stream.read(8) + if length == "": + raise EOFError + return struct.unpack("!q", length)[0] + + +def read_int(stream): + length = stream.read(4) + if length == "": + raise EOFError + return struct.unpack("!i", length)[0] + def write_with_length(obj, stream): stream.write(struct.pack("!i", len(obj))) stream.write(obj) def read_with_length(stream): - length = stream.read(4) - if length == "": - raise EOFError - length = struct.unpack("!i", length)[0] + length = read_int(stream) obj = stream.read(length) if obj == "": raise EOFError diff --git a/pyspark/pyspark/worker.py b/pyspark/pyspark/worker.py index a9ed71892f..62824a1c9b 100644 --- a/pyspark/pyspark/worker.py +++ b/pyspark/pyspark/worker.py @@ -8,7 +8,7 @@ from base64 import standard_b64decode from pyspark.broadcast import Broadcast, _broadcastRegistry from pyspark.cloudpickle import CloudPickler from pyspark.serializers import write_with_length, read_with_length, \ - dump_pickle, load_pickle + read_long, read_int, dump_pickle, load_pickle # Redirect stdout to stderr so that users must return values from functions. @@ -29,11 +29,11 @@ def read_input(): def main(): - num_broadcast_variables = int(sys.stdin.readline().strip()) + num_broadcast_variables = read_int(sys.stdin) for _ in range(num_broadcast_variables): - uuid = sys.stdin.read(36) + bid = read_long(sys.stdin) value = read_with_length(sys.stdin) - _broadcastRegistry[uuid] = Broadcast(uuid, load_pickle(value)) + _broadcastRegistry[bid] = Broadcast(bid, load_pickle(value)) func = load_obj() bypassSerializer = load_obj() if bypassSerializer: From c23bf1aff4b9a1faf9d32c7b64acad2213f9515c Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 20 Oct 2012 00:16:41 +0000 Subject: [PATCH 17/85] Add PySpark README and run scripts. --- core/src/main/scala/spark/SparkContext.scala | 2 +- pyspark/README | 58 ++++++++++++++++++++ pyspark/pyspark-shell | 3 + pyspark/pyspark/context.py | 5 +- pyspark/pyspark/examples/wordcount.py | 17 ++++++ pyspark/pyspark/shell.py | 21 +++++++ pyspark/run-pyspark | 23 ++++++++ 7 files changed, 125 insertions(+), 4 deletions(-) create mode 100644 pyspark/README create mode 100755 pyspark/pyspark-shell create mode 100644 pyspark/pyspark/examples/wordcount.py create mode 100644 pyspark/pyspark/shell.py create mode 100755 pyspark/run-pyspark diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index becf737597..acb38ae33d 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -113,7 +113,7 @@ class SparkContext( // Environment variables to pass to our executors private[spark] val executorEnvs = HashMap[String, String]() for (key <- Seq("SPARK_MEM", "SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS", - "SPARK_TESTING")) { + "SPARK_TESTING", "PYTHONPATH")) { val value = System.getenv(key) if (value != null) { executorEnvs(key) = value diff --git a/pyspark/README b/pyspark/README new file mode 100644 index 0000000000..63a1def141 --- /dev/null +++ b/pyspark/README @@ -0,0 +1,58 @@ +# PySpark + +PySpark is a Python API for Spark. + +PySpark jobs are writen in Python and executed using a standard Python +interpreter; this supports modules that use Python C extensions. The +API is based on the Spark Scala API and uses regular Python functions +and lambdas to support user-defined functions. PySpark supports +interactive use through a standard Python interpreter; it can +automatically serialize closures and ship them to worker processes. + +PySpark is built on top of the Spark Java API. Data is uniformly +represented as serialized Python objects and stored in Spark Java +processes, which communicate with PySpark worker processes over pipes. + +## Features + +PySpark supports most of the Spark API, including broadcast variables. +RDDs are dynamically typed and can hold any Python object. + +PySpark does not support: + +- Special functions on RDDs of doubles +- Accumulators + +## Examples and Documentation + +The PySpark source contains docstrings and doctests that document its +API. The public classes are in `context.py` and `rdd.py`. + +The `pyspark/pyspark/examples` directory contains a few complete +examples. + +## Installing PySpark + +PySpark requires a development version of Py4J, a Python library for +interacting with Java processes. It can be installed from +https://github.com/bartdag/py4j; make sure to install a version that +contains at least the commits through 3dbf380d3d. + +PySpark uses the `PYTHONPATH` environment variable to search for Python +classes; Py4J should be on this path, along with any libraries used by +PySpark programs. `PYTHONPATH` will be automatically shipped to worker +machines, but the files that it points to must be present on each +machine. + +PySpark requires the Spark assembly JAR, which can be created by running +`sbt/sbt assembly` in the Spark directory. + +Additionally, `SPARK_HOME` should be set to the location of the Spark +package. + +## Running PySpark + +The easiest way to run PySpark is to use the `run-pyspark` and +`pyspark-shell` scripts, which are included in the `pyspark` directory. +These scripts automatically load the `spark-conf.sh` file, set +`SPARK_HOME`, and add the `pyspark` package to the `PYTHONPATH`. diff --git a/pyspark/pyspark-shell b/pyspark/pyspark-shell new file mode 100755 index 0000000000..4ed3e6010c --- /dev/null +++ b/pyspark/pyspark-shell @@ -0,0 +1,3 @@ +#!/bin/sh +FWDIR="`dirname $0`" +exec $FWDIR/run-pyspark $FWDIR/pyspark/shell.py "$@" diff --git a/pyspark/pyspark/context.py b/pyspark/pyspark/context.py index 3f4db26644..50d57e5317 100644 --- a/pyspark/pyspark/context.py +++ b/pyspark/pyspark/context.py @@ -18,14 +18,13 @@ class SparkContext(object): asPickle = jvm.spark.api.python.PythonRDD.asPickle arrayAsPickle = jvm.spark.api.python.PythonRDD.arrayAsPickle - def __init__(self, master, name, defaultParallelism=None, - pythonExec='python'): + def __init__(self, master, name, defaultParallelism=None): self.master = master self.name = name self._jsc = self.jvm.JavaSparkContext(master, name) self.defaultParallelism = \ defaultParallelism or self._jsc.sc().defaultParallelism() - self.pythonExec = pythonExec + self.pythonExec = os.environ.get("PYSPARK_PYTHON_EXEC", 'python') # Broadcast's __reduce__ method stores Broadcast instances here. # This allows other code to determine which Broadcast instances have # been pickled, so it can determine which Java broadcast objects to diff --git a/pyspark/pyspark/examples/wordcount.py b/pyspark/pyspark/examples/wordcount.py new file mode 100644 index 0000000000..8365c070e8 --- /dev/null +++ b/pyspark/pyspark/examples/wordcount.py @@ -0,0 +1,17 @@ +import sys +from operator import add +from pyspark.context import SparkContext + +if __name__ == "__main__": + if len(sys.argv) < 3: + print >> sys.stderr, \ + "Usage: PythonWordCount " + exit(-1) + sc = SparkContext(sys.argv[1], "PythonWordCount") + lines = sc.textFile(sys.argv[2], 1) + counts = lines.flatMap(lambda x: x.split(' ')) \ + .map(lambda x: (x, 1)) \ + .reduceByKey(add) + output = counts.collect() + for (word, count) in output: + print "%s : %i" % (word, count) diff --git a/pyspark/pyspark/shell.py b/pyspark/pyspark/shell.py new file mode 100644 index 0000000000..7ef30894cb --- /dev/null +++ b/pyspark/pyspark/shell.py @@ -0,0 +1,21 @@ +""" +An interactive shell. +""" +import code +import sys + +from pyspark.context import SparkContext + + +def main(master='local'): + sc = SparkContext(master, 'PySparkShell') + print "Spark context available as sc." + code.interact(local={'sc': sc}) + + +if __name__ == '__main__': + if len(sys.argv) > 1: + master = sys.argv[1] + else: + master = 'local' + main(master) diff --git a/pyspark/run-pyspark b/pyspark/run-pyspark new file mode 100755 index 0000000000..9c5e027962 --- /dev/null +++ b/pyspark/run-pyspark @@ -0,0 +1,23 @@ +#!/bin/bash + +# Figure out where the Scala framework is installed +FWDIR="$(cd `dirname $0`; cd ../; 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 which Python executable to use +if [ -z "$PYSPARK_PYTHON" ] ; then + PYSPARK_PYTHON="python" +fi +export PYSPARK_PYTHON + +# Add the PySpark classes to the Python path: +export PYTHONPATH=$SPARK_HOME/pyspark/:$PYTHONPATH + +exec "$PYSPARK_PYTHON" "$@" From d4f2e5b0ef38db9d42bb0d5fbbbe6103ce047efe Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 22 Oct 2012 10:28:59 -0700 Subject: [PATCH 18/85] Remove PYTHONPATH from SparkContext's executorEnvs. It makes more sense to pass it in the dictionary of environment variables that is used to construct PythonRDD. --- core/src/main/scala/spark/SparkContext.scala | 2 +- .../main/scala/spark/api/python/PythonRDD.scala | 15 +++++++-------- pyspark/pyspark/rdd.py | 8 ++++++-- 3 files changed, 14 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index acb38ae33d..becf737597 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -113,7 +113,7 @@ class SparkContext( // Environment variables to pass to our executors private[spark] val executorEnvs = HashMap[String, String]() for (key <- Seq("SPARK_MEM", "SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS", - "SPARK_TESTING", "PYTHONPATH")) { + "SPARK_TESTING")) { val value = System.getenv(key) if (value != null) { executorEnvs(key) = value diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 528885fe5c..a593e53efd 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -131,18 +131,17 @@ trait PythonRDDBase { } class PythonRDD[T: ClassManifest]( - parent: RDD[T], command: Seq[String], envVars: Map[String, String], + parent: RDD[T], command: Seq[String], envVars: java.util.Map[String, String], preservePartitoning: Boolean, pythonExec: String, broadcastVars: java.util.List[Broadcast[Array[Byte]]]) extends RDD[Array[Byte]](parent.context) with PythonRDDBase { - def this(parent: RDD[T], command: Seq[String], preservePartitoning: Boolean, - pythonExec: String, broadcastVars: java.util.List[Broadcast[Array[Byte]]]) = - this(parent, command, Map(), preservePartitoning, pythonExec, broadcastVars) - // Similar to Runtime.exec(), if we are given a single string, split it into words // using a standard StringTokenizer (i.e. by spaces) - def this(parent: RDD[T], command: String, preservePartitoning: Boolean, pythonExec: String, broadcastVars: java.util.List[Broadcast[Array[Byte]]]) = - this(parent, PipedRDD.tokenize(command), preservePartitoning, pythonExec, broadcastVars) + def this(parent: RDD[T], command: String, envVars: java.util.Map[String, String], + preservePartitoning: Boolean, pythonExec: String, + broadcastVars: java.util.List[Broadcast[Array[Byte]]]) = + this(parent, PipedRDD.tokenize(command), envVars, preservePartitoning, pythonExec, + broadcastVars) override def splits = parent.splits @@ -151,7 +150,7 @@ class PythonRDD[T: ClassManifest]( override val partitioner = if (preservePartitoning) parent.partitioner else None override def compute(split: Split): Iterator[Array[Byte]] = - compute(split, envVars, command, parent, pythonExec, broadcastVars) + compute(split, envVars.toMap, command, parent, pythonExec, broadcastVars) val asJavaRDD : JavaRDD[Array[Byte]] = JavaRDD.fromRDD(this) } diff --git a/pyspark/pyspark/rdd.py b/pyspark/pyspark/rdd.py index e2137fe06c..e4878c08ba 100644 --- a/pyspark/pyspark/rdd.py +++ b/pyspark/pyspark/rdd.py @@ -1,6 +1,7 @@ from base64 import standard_b64encode as b64enc from collections import defaultdict from itertools import chain, ifilter, imap +import os import shlex from subprocess import Popen, PIPE from threading import Thread @@ -10,7 +11,7 @@ from pyspark.serializers import dump_pickle, load_pickle from pyspark.join import python_join, python_left_outer_join, \ python_right_outer_join, python_cogroup -from py4j.java_collections import ListConverter +from py4j.java_collections import ListConverter, MapConverter class RDD(object): @@ -447,8 +448,11 @@ class PipelinedRDD(RDD): self.ctx.gateway._gateway_client) self.ctx._pickled_broadcast_vars.clear() class_manifest = self._prev_jrdd.classManifest() + env = MapConverter().convert( + {'PYTHONPATH' : os.environ.get("PYTHONPATH", "")}, + self.ctx.gateway._gateway_client) python_rdd = self.ctx.jvm.PythonRDD(self._prev_jrdd.rdd(), - pipe_command, self.preservesPartitioning, self.ctx.pythonExec, + pipe_command, env, self.preservesPartitioning, self.ctx.pythonExec, broadcast_vars, class_manifest) self._jrdd_val = python_rdd.asJavaRDD() return self._jrdd_val From 7859879aaa1860ff6b383e32a18fd9a410a97416 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 28 Oct 2012 16:46:31 -0700 Subject: [PATCH 19/85] Bump required Py4J version and add test for large broadcast variables. --- pyspark/README | 2 +- pyspark/pyspark/broadcast.py | 2 ++ pyspark/requirements.txt | 2 +- 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/pyspark/README b/pyspark/README index 63a1def141..55490e1a83 100644 --- a/pyspark/README +++ b/pyspark/README @@ -36,7 +36,7 @@ examples. PySpark requires a development version of Py4J, a Python library for interacting with Java processes. It can be installed from https://github.com/bartdag/py4j; make sure to install a version that -contains at least the commits through 3dbf380d3d. +contains at least the commits through b7924aabe9. PySpark uses the `PYTHONPATH` environment variable to search for Python classes; Py4J should be on this path, along with any libraries used by diff --git a/pyspark/pyspark/broadcast.py b/pyspark/pyspark/broadcast.py index 4cff02b36d..93876fa738 100644 --- a/pyspark/pyspark/broadcast.py +++ b/pyspark/pyspark/broadcast.py @@ -13,6 +13,8 @@ >>> sc.parallelize([0, 0]).flatMap(lambda x: b.value).collect() [1, 2, 3, 4, 5, 1, 2, 3, 4, 5] + +>>> large_broadcast = sc.broadcast(list(range(10000))) """ # Holds broadcasted data received from Java, keyed by its id. _broadcastRegistry = {} diff --git a/pyspark/requirements.txt b/pyspark/requirements.txt index 71e2bc2b89..48fa2ab105 100644 --- a/pyspark/requirements.txt +++ b/pyspark/requirements.txt @@ -3,4 +3,4 @@ # package is not at the root of the git repository. It may be possible to # install Py4J from git once https://github.com/pypa/pip/pull/526 is merged. -# git+git://github.com/bartdag/py4j.git@3dbf380d3d2cdeb9aab394454ea74d80c4aba1ea +# git+git://github.com/bartdag/py4j.git@b7924aabe9c5e63f0a4d8bbd17019534c7ec014e From 2ccf3b665280bf5b0919e3801d028126cb070dbd Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 28 Oct 2012 22:30:28 -0700 Subject: [PATCH 20/85] Fix PySpark hash partitioning bug. A Java array's hashCode is based on its object identify, not its elements, so this was causing serialized keys to be hashed incorrectly. This commit adds a PySpark-specific workaround and adds more tests. --- .../spark/api/python/PythonPartitioner.scala | 41 +++++++++++++++++++ .../scala/spark/api/python/PythonRDD.scala | 10 ++--- pyspark/pyspark/rdd.py | 12 ++++-- 3 files changed, 54 insertions(+), 9 deletions(-) create mode 100644 core/src/main/scala/spark/api/python/PythonPartitioner.scala diff --git a/core/src/main/scala/spark/api/python/PythonPartitioner.scala b/core/src/main/scala/spark/api/python/PythonPartitioner.scala new file mode 100644 index 0000000000..ef9f808fb2 --- /dev/null +++ b/core/src/main/scala/spark/api/python/PythonPartitioner.scala @@ -0,0 +1,41 @@ +package spark.api.python + +import spark.Partitioner + +import java.util.Arrays + +/** + * A [[spark.Partitioner]] that performs handling of byte arrays, for use by the Python API. + */ +class PythonPartitioner(override val numPartitions: Int) extends Partitioner { + + override def getPartition(key: Any): Int = { + if (key == null) { + return 0 + } + else { + val hashCode = { + if (key.isInstanceOf[Array[Byte]]) { + System.err.println("Dumping a byte array!" + Arrays.hashCode(key.asInstanceOf[Array[Byte]]) + ) + Arrays.hashCode(key.asInstanceOf[Array[Byte]]) + } + else + key.hashCode() + } + val mod = hashCode % numPartitions + if (mod < 0) { + mod + numPartitions + } else { + mod // Guard against negative hash codes + } + } + } + + override def equals(other: Any): Boolean = other match { + case h: PythonPartitioner => + h.numPartitions == numPartitions + case _ => + false + } +} diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index a593e53efd..50094d6b0f 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -179,14 +179,12 @@ object PythonRDD { val dOut = new DataOutputStream(baos); if (elem.isInstanceOf[Array[Byte]]) { elem.asInstanceOf[Array[Byte]] - } else if (elem.isInstanceOf[scala.Tuple2[_, _]]) { - val t = elem.asInstanceOf[scala.Tuple2[_, _]] - val t1 = t._1.asInstanceOf[Array[Byte]] - val t2 = t._2.asInstanceOf[Array[Byte]] + } else if (elem.isInstanceOf[scala.Tuple2[Array[Byte], Array[Byte]]]) { + val t = elem.asInstanceOf[scala.Tuple2[Array[Byte], Array[Byte]]] dOut.writeByte(Pickle.PROTO) dOut.writeByte(Pickle.TWO) - dOut.write(PythonRDD.stripPickle(t1)) - dOut.write(PythonRDD.stripPickle(t2)) + dOut.write(PythonRDD.stripPickle(t._1)) + dOut.write(PythonRDD.stripPickle(t._2)) dOut.writeByte(Pickle.TUPLE2) dOut.writeByte(Pickle.STOP) baos.toByteArray() diff --git a/pyspark/pyspark/rdd.py b/pyspark/pyspark/rdd.py index e4878c08ba..85a24c6854 100644 --- a/pyspark/pyspark/rdd.py +++ b/pyspark/pyspark/rdd.py @@ -310,6 +310,12 @@ class RDD(object): return python_right_outer_join(self, other, numSplits) def partitionBy(self, numSplits, hashFunc=hash): + """ + >>> pairs = sc.parallelize([1, 2, 3, 4, 2, 4, 1]).map(lambda x: (x, x)) + >>> sets = pairs.partitionBy(2).glom().collect() + >>> set(sets[0]).intersection(set(sets[1])) + set([]) + """ if numSplits is None: numSplits = self.ctx.defaultParallelism def add_shuffle_key(iterator): @@ -319,7 +325,7 @@ class RDD(object): keyed = PipelinedRDD(self, add_shuffle_key) keyed._bypass_serializer = True pairRDD = self.ctx.jvm.PairwiseRDD(keyed._jrdd.rdd()).asJavaPairRDD() - partitioner = self.ctx.jvm.spark.HashPartitioner(numSplits) + partitioner = self.ctx.jvm.spark.api.python.PythonPartitioner(numSplits) jrdd = pairRDD.partitionBy(partitioner) jrdd = jrdd.map(self.ctx.jvm.ExtractValue()) return RDD(jrdd, self.ctx) @@ -391,7 +397,7 @@ class RDD(object): """ >>> x = sc.parallelize([("a", 1), ("b", 4)]) >>> y = sc.parallelize([("a", 2)]) - >>> x.cogroup(y).collect() + >>> sorted(x.cogroup(y).collect()) [('a', ([1], [2])), ('b', ([4], []))] """ return python_cogroup(self, other, numSplits) @@ -462,7 +468,7 @@ def _test(): import doctest from pyspark.context import SparkContext globs = globals().copy() - globs['sc'] = SparkContext('local', 'PythonTest') + globs['sc'] = SparkContext('local[4]', 'PythonTest') doctest.testmod(globs=globs) globs['sc'].stop() From ccd075cf960df6c6c449b709515cdd81499a52be Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 24 Dec 2012 15:01:13 -0800 Subject: [PATCH 21/85] Reduce object overhead in Pyspark shuffle and collect --- pyspark/pyspark/rdd.py | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/pyspark/pyspark/rdd.py b/pyspark/pyspark/rdd.py index 85a24c6854..708ea6eb55 100644 --- a/pyspark/pyspark/rdd.py +++ b/pyspark/pyspark/rdd.py @@ -145,8 +145,10 @@ class RDD(object): self.map(f).collect() # Force evaluation def collect(self): - pickle = self.ctx.arrayAsPickle(self._jrdd.rdd().collect()) - return load_pickle(bytes(pickle)) + def asList(iterator): + yield list(iterator) + pickles = self.mapPartitions(asList)._jrdd.rdd().collect() + return list(chain.from_iterable(load_pickle(bytes(p)) for p in pickles)) def reduce(self, f): """ @@ -319,16 +321,23 @@ class RDD(object): if numSplits is None: numSplits = self.ctx.defaultParallelism def add_shuffle_key(iterator): + buckets = defaultdict(list) for (k, v) in iterator: - yield str(hashFunc(k)) - yield dump_pickle((k, v)) + buckets[hashFunc(k) % numSplits].append((k, v)) + for (split, items) in buckets.iteritems(): + yield str(split) + yield dump_pickle(items) keyed = PipelinedRDD(self, add_shuffle_key) keyed._bypass_serializer = True pairRDD = self.ctx.jvm.PairwiseRDD(keyed._jrdd.rdd()).asJavaPairRDD() partitioner = self.ctx.jvm.spark.api.python.PythonPartitioner(numSplits) + # Transferring O(n) objects to Java is too expensive. Instead, we'll + # form the hash buckets in Python, transferring O(numSplits) objects + # to Java. Each object is a (splitNumber, [objects]) pair. jrdd = pairRDD.partitionBy(partitioner) jrdd = jrdd.map(self.ctx.jvm.ExtractValue()) - return RDD(jrdd, self.ctx) + # Flatten the resulting RDD: + return RDD(jrdd, self.ctx).flatMap(lambda items: items) def combineByKey(self, createCombiner, mergeValue, mergeCombiners, numSplits=None): From 4608902fb87af64a15b97ab21fe6382cd6e5a644 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 24 Dec 2012 17:20:10 -0800 Subject: [PATCH 22/85] Use filesystem to collect RDDs in PySpark. Passing large volumes of data through Py4J seems to be slow. It appears to be faster to write the data to the local filesystem and read it back from Python. --- .../scala/spark/api/python/PythonRDD.scala | 66 +++++++------------ pyspark/pyspark/context.py | 9 ++- pyspark/pyspark/rdd.py | 34 ++++++++-- pyspark/pyspark/serializers.py | 8 +++ pyspark/pyspark/worker.py | 12 +--- 5 files changed, 66 insertions(+), 63 deletions(-) diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 50094d6b0f..4f870e837a 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -1,6 +1,7 @@ package spark.api.python import java.io._ +import java.util.{List => JList} import scala.collection.Map import scala.collection.JavaConversions._ @@ -59,36 +60,7 @@ trait PythonRDDBase { } out.flush() for (elem <- parent.iterator(split)) { - if (elem.isInstanceOf[Array[Byte]]) { - val arr = elem.asInstanceOf[Array[Byte]] - dOut.writeInt(arr.length) - dOut.write(arr) - } else if (elem.isInstanceOf[scala.Tuple2[_, _]]) { - val t = elem.asInstanceOf[scala.Tuple2[_, _]] - val t1 = t._1.asInstanceOf[Array[Byte]] - val t2 = t._2.asInstanceOf[Array[Byte]] - val length = t1.length + t2.length - 3 - 3 + 4 // stripPickle() removes 3 bytes - dOut.writeInt(length) - dOut.writeByte(Pickle.PROTO) - dOut.writeByte(Pickle.TWO) - dOut.write(PythonRDD.stripPickle(t1)) - dOut.write(PythonRDD.stripPickle(t2)) - dOut.writeByte(Pickle.TUPLE2) - dOut.writeByte(Pickle.STOP) - } else if (elem.isInstanceOf[String]) { - // For uniformity, strings are wrapped into Pickles. - val s = elem.asInstanceOf[String].getBytes("UTF-8") - val length = 2 + 1 + 4 + s.length + 1 - dOut.writeInt(length) - dOut.writeByte(Pickle.PROTO) - dOut.writeByte(Pickle.TWO) - dOut.writeByte(Pickle.BINUNICODE) - dOut.writeInt(Integer.reverseBytes(s.length)) - dOut.write(s) - dOut.writeByte(Pickle.STOP) - } else { - throw new Exception("Unexpected RDD type") - } + PythonRDD.writeAsPickle(elem, dOut) } dOut.flush() out.flush() @@ -174,36 +146,45 @@ object PythonRDD { arr.slice(2, arr.length - 1) } - def asPickle(elem: Any) : Array[Byte] = { - val baos = new ByteArrayOutputStream(); - val dOut = new DataOutputStream(baos); + /** + * Write strings, pickled Python objects, or pairs of pickled objects to a data output stream. + * The data format is a 32-bit integer representing the pickled object's length (in bytes), + * followed by the pickled data. + * @param elem the object to write + * @param dOut a data output stream + */ + def writeAsPickle(elem: Any, dOut: DataOutputStream) { if (elem.isInstanceOf[Array[Byte]]) { - elem.asInstanceOf[Array[Byte]] + val arr = elem.asInstanceOf[Array[Byte]] + dOut.writeInt(arr.length) + dOut.write(arr) } else if (elem.isInstanceOf[scala.Tuple2[Array[Byte], Array[Byte]]]) { val t = elem.asInstanceOf[scala.Tuple2[Array[Byte], Array[Byte]]] + val length = t._1.length + t._2.length - 3 - 3 + 4 // stripPickle() removes 3 bytes + dOut.writeInt(length) dOut.writeByte(Pickle.PROTO) dOut.writeByte(Pickle.TWO) dOut.write(PythonRDD.stripPickle(t._1)) dOut.write(PythonRDD.stripPickle(t._2)) dOut.writeByte(Pickle.TUPLE2) dOut.writeByte(Pickle.STOP) - baos.toByteArray() } else if (elem.isInstanceOf[String]) { // For uniformity, strings are wrapped into Pickles. val s = elem.asInstanceOf[String].getBytes("UTF-8") + val length = 2 + 1 + 4 + s.length + 1 + dOut.writeInt(length) dOut.writeByte(Pickle.PROTO) dOut.writeByte(Pickle.TWO) dOut.write(Pickle.BINUNICODE) dOut.writeInt(Integer.reverseBytes(s.length)) dOut.write(s) dOut.writeByte(Pickle.STOP) - baos.toByteArray() } else { throw new Exception("Unexpected RDD type") } } - def pickleFile(sc: JavaSparkContext, filename: String, parallelism: Int) : + def readRDDFromPickleFile(sc: JavaSparkContext, filename: String, parallelism: Int) : JavaRDD[Array[Byte]] = { val file = new DataInputStream(new FileInputStream(filename)) val objs = new collection.mutable.ArrayBuffer[Array[Byte]] @@ -221,11 +202,12 @@ object PythonRDD { JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism)) } - def arrayAsPickle(arr : Any) : Array[Byte] = { - val pickles : Array[Byte] = arr.asInstanceOf[Array[Any]].map(asPickle).map(stripPickle).flatten - - Array[Byte](Pickle.PROTO, Pickle.TWO, Pickle.EMPTY_LIST, Pickle.MARK) ++ pickles ++ - Array[Byte] (Pickle.APPENDS, Pickle.STOP) + def writeArrayToPickleFile[T](items: Array[T], filename: String) { + val file = new DataOutputStream(new FileOutputStream(filename)) + for (item <- items) { + writeAsPickle(item, file) + } + file.close() } } diff --git a/pyspark/pyspark/context.py b/pyspark/pyspark/context.py index 50d57e5317..19f9f9e133 100644 --- a/pyspark/pyspark/context.py +++ b/pyspark/pyspark/context.py @@ -14,9 +14,8 @@ class SparkContext(object): gateway = launch_gateway() jvm = gateway.jvm - pickleFile = jvm.spark.api.python.PythonRDD.pickleFile - asPickle = jvm.spark.api.python.PythonRDD.asPickle - arrayAsPickle = jvm.spark.api.python.PythonRDD.arrayAsPickle + readRDDFromPickleFile = jvm.PythonRDD.readRDDFromPickleFile + writeArrayToPickleFile = jvm.PythonRDD.writeArrayToPickleFile def __init__(self, master, name, defaultParallelism=None): self.master = master @@ -45,11 +44,11 @@ class SparkContext(object): # because it sends O(n) Py4J commands. As an alternative, serialized # objects are written to a file and loaded through textFile(). tempFile = NamedTemporaryFile(delete=False) + atexit.register(lambda: os.unlink(tempFile.name)) for x in c: write_with_length(dump_pickle(x), tempFile) tempFile.close() - atexit.register(lambda: os.unlink(tempFile.name)) - jrdd = self.pickleFile(self._jsc, tempFile.name, numSlices) + jrdd = self.readRDDFromPickleFile(self._jsc, tempFile.name, numSlices) return RDD(jrdd, self) def textFile(self, name, minSplits=None): diff --git a/pyspark/pyspark/rdd.py b/pyspark/pyspark/rdd.py index 708ea6eb55..01908cff96 100644 --- a/pyspark/pyspark/rdd.py +++ b/pyspark/pyspark/rdd.py @@ -1,13 +1,15 @@ +import atexit from base64 import standard_b64encode as b64enc from collections import defaultdict from itertools import chain, ifilter, imap import os import shlex from subprocess import Popen, PIPE +from tempfile import NamedTemporaryFile from threading import Thread from pyspark import cloudpickle -from pyspark.serializers import dump_pickle, load_pickle +from pyspark.serializers import dump_pickle, load_pickle, read_from_pickle_file from pyspark.join import python_join, python_left_outer_join, \ python_right_outer_join, python_cogroup @@ -145,10 +147,30 @@ class RDD(object): self.map(f).collect() # Force evaluation def collect(self): + # To minimize the number of transfers between Python and Java, we'll + # flatten each partition into a list before collecting it. Due to + # pipelining, this should add minimal overhead. def asList(iterator): yield list(iterator) - pickles = self.mapPartitions(asList)._jrdd.rdd().collect() - return list(chain.from_iterable(load_pickle(bytes(p)) for p in pickles)) + picklesInJava = self.mapPartitions(asList)._jrdd.rdd().collect() + return list(chain.from_iterable(self._collect_array_through_file(picklesInJava))) + + def _collect_array_through_file(self, array): + # Transferring lots of data through Py4J can be slow because + # socket.readline() is inefficient. Instead, we'll dump the data to a + # file and read it back. + tempFile = NamedTemporaryFile(delete=False) + tempFile.close() + def clean_up_file(): + try: os.unlink(tempFile.name) + except: pass + atexit.register(clean_up_file) + self.ctx.writeArrayToPickleFile(array, tempFile.name) + # Read the data into Python and deserialize it: + with open(tempFile.name, 'rb') as tempFile: + for item in read_from_pickle_file(tempFile): + yield item + os.unlink(tempFile.name) def reduce(self, f): """ @@ -220,15 +242,15 @@ class RDD(object): >>> sc.parallelize([2, 3, 4]).take(2) [2, 3] """ - pickle = self.ctx.arrayAsPickle(self._jrdd.rdd().take(num)) - return load_pickle(bytes(pickle)) + picklesInJava = self._jrdd.rdd().take(num) + return list(self._collect_array_through_file(picklesInJava)) def first(self): """ >>> sc.parallelize([2, 3, 4]).first() 2 """ - return load_pickle(bytes(self.ctx.asPickle(self._jrdd.first()))) + return self.take(1)[0] def saveAsTextFile(self, path): def func(iterator): diff --git a/pyspark/pyspark/serializers.py b/pyspark/pyspark/serializers.py index 21ef8b106c..bfcdda8f12 100644 --- a/pyspark/pyspark/serializers.py +++ b/pyspark/pyspark/serializers.py @@ -33,3 +33,11 @@ def read_with_length(stream): if obj == "": raise EOFError return obj + + +def read_from_pickle_file(stream): + try: + while True: + yield load_pickle(read_with_length(stream)) + except EOFError: + return diff --git a/pyspark/pyspark/worker.py b/pyspark/pyspark/worker.py index 62824a1c9b..9f6b507dbd 100644 --- a/pyspark/pyspark/worker.py +++ b/pyspark/pyspark/worker.py @@ -8,7 +8,7 @@ from base64 import standard_b64decode from pyspark.broadcast import Broadcast, _broadcastRegistry from pyspark.cloudpickle import CloudPickler from pyspark.serializers import write_with_length, read_with_length, \ - read_long, read_int, dump_pickle, load_pickle + read_long, read_int, dump_pickle, load_pickle, read_from_pickle_file # Redirect stdout to stderr so that users must return values from functions. @@ -20,14 +20,6 @@ def load_obj(): return load_pickle(standard_b64decode(sys.stdin.readline().strip())) -def read_input(): - try: - while True: - yield load_pickle(read_with_length(sys.stdin)) - except EOFError: - return - - def main(): num_broadcast_variables = read_int(sys.stdin) for _ in range(num_broadcast_variables): @@ -40,7 +32,7 @@ def main(): dumps = lambda x: x else: dumps = dump_pickle - for obj in func(read_input()): + for obj in func(read_from_pickle_file(sys.stdin)): write_with_length(dumps(obj), old_stdout) From e2dad15621f5dc15275b300df05483afde5025a0 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 26 Dec 2012 17:34:24 -0800 Subject: [PATCH 23/85] Add support for batched serialization of Python objects in PySpark. --- pyspark/pyspark/context.py | 3 +- pyspark/pyspark/rdd.py | 57 +++++++++++++++++++++++----------- pyspark/pyspark/serializers.py | 34 +++++++++++++++++++- 3 files changed, 74 insertions(+), 20 deletions(-) diff --git a/pyspark/pyspark/context.py b/pyspark/pyspark/context.py index 19f9f9e133..032619693a 100644 --- a/pyspark/pyspark/context.py +++ b/pyspark/pyspark/context.py @@ -17,13 +17,14 @@ class SparkContext(object): readRDDFromPickleFile = jvm.PythonRDD.readRDDFromPickleFile writeArrayToPickleFile = jvm.PythonRDD.writeArrayToPickleFile - def __init__(self, master, name, defaultParallelism=None): + def __init__(self, master, name, defaultParallelism=None, batchSize=-1): self.master = master self.name = name self._jsc = self.jvm.JavaSparkContext(master, name) self.defaultParallelism = \ defaultParallelism or self._jsc.sc().defaultParallelism() self.pythonExec = os.environ.get("PYSPARK_PYTHON_EXEC", 'python') + self.batchSize = batchSize # -1 represents a unlimited batch size # Broadcast's __reduce__ method stores Broadcast instances here. # This allows other code to determine which Broadcast instances have # been pickled, so it can determine which Java broadcast objects to diff --git a/pyspark/pyspark/rdd.py b/pyspark/pyspark/rdd.py index 01908cff96..d7081dffd2 100644 --- a/pyspark/pyspark/rdd.py +++ b/pyspark/pyspark/rdd.py @@ -2,6 +2,7 @@ import atexit from base64 import standard_b64encode as b64enc from collections import defaultdict from itertools import chain, ifilter, imap +import operator import os import shlex from subprocess import Popen, PIPE @@ -9,7 +10,8 @@ from tempfile import NamedTemporaryFile from threading import Thread from pyspark import cloudpickle -from pyspark.serializers import dump_pickle, load_pickle, read_from_pickle_file +from pyspark.serializers import batched, Batch, dump_pickle, load_pickle, \ + read_from_pickle_file from pyspark.join import python_join, python_left_outer_join, \ python_right_outer_join, python_cogroup @@ -83,6 +85,11 @@ class RDD(object): >>> rdd = sc.parallelize([1, 1, 2, 3]) >>> rdd.union(rdd).collect() [1, 1, 2, 3, 1, 1, 2, 3] + + # Union of batched and unbatched RDDs: + >>> batchedRDD = sc.parallelize([Batch([1, 2, 3, 4, 5])]) + >>> rdd.union(batchedRDD).collect() + [1, 1, 2, 3, 1, 2, 3, 4, 5] """ return RDD(self._jrdd.union(other._jrdd), self.ctx) @@ -147,13 +154,8 @@ class RDD(object): self.map(f).collect() # Force evaluation def collect(self): - # To minimize the number of transfers between Python and Java, we'll - # flatten each partition into a list before collecting it. Due to - # pipelining, this should add minimal overhead. - def asList(iterator): - yield list(iterator) - picklesInJava = self.mapPartitions(asList)._jrdd.rdd().collect() - return list(chain.from_iterable(self._collect_array_through_file(picklesInJava))) + picklesInJava = self._jrdd.rdd().collect() + return list(self._collect_array_through_file(picklesInJava)) def _collect_array_through_file(self, array): # Transferring lots of data through Py4J can be slow because @@ -214,12 +216,21 @@ class RDD(object): # TODO: aggregate + def sum(self): + """ + >>> sc.parallelize([1.0, 2.0, 3.0]).sum() + 6.0 + """ + return self.mapPartitions(lambda x: [sum(x)]).reduce(operator.add) + def count(self): """ >>> sc.parallelize([2, 3, 4]).count() - 3L + 3 + >>> sc.parallelize([Batch([2, 3, 4])]).count() + 3 """ - return self._jrdd.count() + return self.mapPartitions(lambda i: [sum(1 for _ in i)]).sum() def countByValue(self): """ @@ -342,24 +353,23 @@ class RDD(object): """ if numSplits is None: numSplits = self.ctx.defaultParallelism + # Transferring O(n) objects to Java is too expensive. Instead, we'll + # form the hash buckets in Python, transferring O(numSplits) objects + # to Java. Each object is a (splitNumber, [objects]) pair. def add_shuffle_key(iterator): buckets = defaultdict(list) for (k, v) in iterator: buckets[hashFunc(k) % numSplits].append((k, v)) for (split, items) in buckets.iteritems(): yield str(split) - yield dump_pickle(items) + yield dump_pickle(Batch(items)) keyed = PipelinedRDD(self, add_shuffle_key) keyed._bypass_serializer = True pairRDD = self.ctx.jvm.PairwiseRDD(keyed._jrdd.rdd()).asJavaPairRDD() partitioner = self.ctx.jvm.spark.api.python.PythonPartitioner(numSplits) - # Transferring O(n) objects to Java is too expensive. Instead, we'll - # form the hash buckets in Python, transferring O(numSplits) objects - # to Java. Each object is a (splitNumber, [objects]) pair. jrdd = pairRDD.partitionBy(partitioner) jrdd = jrdd.map(self.ctx.jvm.ExtractValue()) - # Flatten the resulting RDD: - return RDD(jrdd, self.ctx).flatMap(lambda items: items) + return RDD(jrdd, self.ctx) def combineByKey(self, createCombiner, mergeValue, mergeCombiners, numSplits=None): @@ -478,8 +488,19 @@ class PipelinedRDD(RDD): def _jrdd(self): if self._jrdd_val: return self._jrdd_val - funcs = [self.func, self._bypass_serializer] - pipe_command = ' '.join(b64enc(cloudpickle.dumps(f)) for f in funcs) + func = self.func + if not self._bypass_serializer and self.ctx.batchSize != 1: + oldfunc = self.func + batchSize = self.ctx.batchSize + if batchSize == -1: # unlimited batch size + def batched_func(iterator): + yield Batch(list(oldfunc(iterator))) + else: + def batched_func(iterator): + return batched(oldfunc(iterator), batchSize) + func = batched_func + cmds = [func, self._bypass_serializer] + pipe_command = ' '.join(b64enc(cloudpickle.dumps(f)) for f in cmds) broadcast_vars = ListConverter().convert( [x._jbroadcast for x in self.ctx._pickled_broadcast_vars], self.ctx.gateway._gateway_client) diff --git a/pyspark/pyspark/serializers.py b/pyspark/pyspark/serializers.py index bfcdda8f12..4ed925697c 100644 --- a/pyspark/pyspark/serializers.py +++ b/pyspark/pyspark/serializers.py @@ -2,6 +2,33 @@ import struct import cPickle +class Batch(object): + """ + Used to store multiple RDD entries as a single Java object. + + This relieves us from having to explicitly track whether an RDD + is stored as batches of objects and avoids problems when processing + the union() of batched and unbatched RDDs (e.g. the union() of textFile() + with another RDD). + """ + def __init__(self, items): + self.items = items + + +def batched(iterator, batchSize): + items = [] + count = 0 + for item in iterator: + items.append(item) + count += 1 + if count == batchSize: + yield Batch(items) + items = [] + count = [] + if items: + yield Batch(items) + + def dump_pickle(obj): return cPickle.dumps(obj, 2) @@ -38,6 +65,11 @@ def read_with_length(stream): def read_from_pickle_file(stream): try: while True: - yield load_pickle(read_with_length(stream)) + obj = load_pickle(read_with_length(stream)) + if type(obj) == Batch: # We don't care about inheritance + for item in obj.items: + yield item + else: + yield obj except EOFError: return From 1dca0c51804b9c94709ec9cc0544b8dfb7afe59f Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 26 Dec 2012 18:23:06 -0800 Subject: [PATCH 24/85] Remove debug output from PythonPartitioner. --- core/src/main/scala/spark/api/python/PythonPartitioner.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/main/scala/spark/api/python/PythonPartitioner.scala b/core/src/main/scala/spark/api/python/PythonPartitioner.scala index ef9f808fb2..606a80d1eb 100644 --- a/core/src/main/scala/spark/api/python/PythonPartitioner.scala +++ b/core/src/main/scala/spark/api/python/PythonPartitioner.scala @@ -16,8 +16,6 @@ class PythonPartitioner(override val numPartitions: Int) extends Partitioner { else { val hashCode = { if (key.isInstanceOf[Array[Byte]]) { - System.err.println("Dumping a byte array!" + Arrays.hashCode(key.asInstanceOf[Array[Byte]]) - ) Arrays.hashCode(key.asInstanceOf[Array[Byte]]) } else From 2d98fff0651af4d527f41ba50c01f453fa049464 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 27 Dec 2012 10:13:29 -0800 Subject: [PATCH 25/85] Add IPython support to pyspark-shell. Suggested by / based on code from @MLnick --- pyspark/README | 3 +++ pyspark/pyspark/shell.py | 25 +++++++++++++++++-------- pyspark/requirements.txt | 1 + 3 files changed, 21 insertions(+), 8 deletions(-) diff --git a/pyspark/README b/pyspark/README index 55490e1a83..461176de7d 100644 --- a/pyspark/README +++ b/pyspark/README @@ -38,6 +38,9 @@ interacting with Java processes. It can be installed from https://github.com/bartdag/py4j; make sure to install a version that contains at least the commits through b7924aabe9. +PySpark requires the `argparse` module, which is included in Python 2.7 +and is is available for Python 2.6 through `pip` or `easy_install`. + PySpark uses the `PYTHONPATH` environment variable to search for Python classes; Py4J should be on this path, along with any libraries used by PySpark programs. `PYTHONPATH` will be automatically shipped to worker diff --git a/pyspark/pyspark/shell.py b/pyspark/pyspark/shell.py index 7ef30894cb..7012884abc 100644 --- a/pyspark/pyspark/shell.py +++ b/pyspark/pyspark/shell.py @@ -1,21 +1,30 @@ """ An interactive shell. """ +import argparse # argparse is avaiable for Python < 2.7 through easy_install. import code import sys from pyspark.context import SparkContext -def main(master='local'): +def main(master='local', ipython=False): sc = SparkContext(master, 'PySparkShell') - print "Spark context available as sc." - code.interact(local={'sc': sc}) + user_ns = {'sc' : sc} + banner = "Spark context avaiable as sc." + if ipython: + import IPython + IPython.embed(user_ns=user_ns, banner2=banner) + else: + print banner + code.interact(local=user_ns) if __name__ == '__main__': - if len(sys.argv) > 1: - master = sys.argv[1] - else: - master = 'local' - main(master) + parser = argparse.ArgumentParser() + parser.add_argument("master", help="Spark master host (default='local')", + nargs='?', type=str, default="local") + parser.add_argument("-i", "--ipython", help="Run IPython shell", + action="store_true") + args = parser.parse_args() + main(args.master, args.ipython) diff --git a/pyspark/requirements.txt b/pyspark/requirements.txt index 48fa2ab105..2464ca0074 100644 --- a/pyspark/requirements.txt +++ b/pyspark/requirements.txt @@ -4,3 +4,4 @@ # install Py4J from git once https://github.com/pypa/pip/pull/526 is merged. # git+git://github.com/bartdag/py4j.git@b7924aabe9c5e63f0a4d8bbd17019534c7ec014e +argparse From 85b8f2c64f0fc4be5645d8736629fc082cb3587b Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 27 Dec 2012 17:55:33 -0800 Subject: [PATCH 26/85] Add epydoc API documentation for PySpark. --- docs/README.md | 8 +- docs/_layouts/global.html | 10 +- docs/_plugins/copy_api_dirs.rb | 17 +++ pyspark/epydoc.conf | 19 ++++ pyspark/pyspark/context.py | 24 ++++ pyspark/pyspark/rdd.py | 195 ++++++++++++++++++++++++++++++--- 6 files changed, 254 insertions(+), 19 deletions(-) create mode 100644 pyspark/epydoc.conf diff --git a/docs/README.md b/docs/README.md index 092153070e..887f407f18 100644 --- a/docs/README.md +++ b/docs/README.md @@ -25,10 +25,12 @@ To mark a block of code in your markdown to be syntax highlighted by jekyll duri // supported languages too. {% endhighlight %} -## Scaladoc +## API Docs (Scaladoc and Epydoc) You can build just the Spark scaladoc by running `sbt/sbt doc` from the SPARK_PROJECT_ROOT directory. -When you run `jekyll` in the docs directory, it will also copy over the scala doc for the various Spark subprojects into the docs directory (and then also into the _site directory). We use a jekyll plugin to run `sbt/sbt doc` before building the site so if you haven't run it (recently) it may take some time as it generates all of the scaladoc. +Similarly, you can build just the PySpark epydoc by running `epydoc --config epydoc.conf` from the SPARK_PROJECT_ROOT/pyspark directory. -NOTE: To skip the step of building and copying over the scaladoc when you build the docs, run `SKIP_SCALADOC=1 jekyll`. +When you run `jekyll` in the docs directory, it will also copy over the scaladoc for the various Spark subprojects into the docs directory (and then also into the _site directory). We use a jekyll plugin to run `sbt/sbt doc` before building the site so if you haven't run it (recently) it may take some time as it generates all of the scaladoc. The jekyll plugin also generates the PySpark docs using [epydoc](http://epydoc.sourceforge.net/). + +NOTE: To skip the step of building and copying over the scaladoc when you build the docs, run `SKIP_SCALADOC=1 jekyll`. Similarly, `SKIP_EPYDOC=1 jekyll` will skip PySpark API doc generation. diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 41ad5242c9..43a5fa3e1c 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -49,8 +49,14 @@
  • Java
  • - -
  • API (Scaladoc)
  • + +
  • Quick Start
  • Scala
  • Java
  • +
  • Python
  • diff --git a/docs/api.md b/docs/api.md index 43548b223c..b9c93ac5e8 100644 --- a/docs/api.md +++ b/docs/api.md @@ -8,3 +8,4 @@ Here you can find links to the Scaladoc generated for the Spark sbt subprojects. - [Core](api/core/index.html) - [Examples](api/examples/index.html) - [Bagel](api/bagel/index.html) +- [PySpark](api/pyspark/index.html) diff --git a/docs/index.md b/docs/index.md index ed9953a590..33ab58a962 100644 --- a/docs/index.md +++ b/docs/index.md @@ -7,11 +7,11 @@ title: Spark Overview TODO(andyk): Rewrite to make the Java API a first class part of the story. {% endcomment %} -Spark is a MapReduce-like cluster computing framework designed for low-latency iterative jobs and interactive use from an -interpreter. It provides clean, language-integrated APIs in Scala and Java, with a rich array of parallel operators. Spark can -run on top of the [Apache Mesos](http://incubator.apache.org/mesos/) cluster manager, +Spark is a MapReduce-like cluster computing framework designed for low-latency iterative jobs and interactive use from an interpreter. +It provides clean, language-integrated APIs in Scala, Java, and Python, with a rich array of parallel operators. +Spark can run on top of the [Apache Mesos](http://incubator.apache.org/mesos/) cluster manager, [Hadoop YARN](http://hadoop.apache.org/docs/r2.0.1-alpha/hadoop-yarn/hadoop-yarn-site/YARN.html), -Amazon EC2, or without an independent resource manager ("standalone mode"). +Amazon EC2, or without an independent resource manager ("standalone mode"). # Downloading @@ -59,6 +59,7 @@ of `project/SparkBuild.scala`, then rebuilding Spark (`sbt/sbt clean compile`). * [Quick Start](quick-start.html): a quick introduction to the Spark API; start here! * [Spark Programming Guide](scala-programming-guide.html): an overview of Spark concepts, and details on the Scala API * [Java Programming Guide](java-programming-guide.html): using Spark from Java +* [Python Programming Guide](python-programming-guide.html): using Spark from Python **Deployment guides:** @@ -72,7 +73,7 @@ of `project/SparkBuild.scala`, then rebuilding Spark (`sbt/sbt clean compile`). * [Configuration](configuration.html): customize Spark via its configuration system * [Tuning Guide](tuning.html): best practices to optimize performance and memory use -* [API Docs (Scaladoc)](api/core/index.html) +* API Docs: [Java/Scala (Scaladoc)](api/core/index.html) and [Python (Epydoc)](api/pyspark/index.html) * [Bagel](bagel-programming-guide.html): an implementation of Google's Pregel on Spark * [Contributing to Spark](contributing-to-spark.html) diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md new file mode 100644 index 0000000000..b7c747f905 --- /dev/null +++ b/docs/python-programming-guide.md @@ -0,0 +1,74 @@ +--- +layout: global +title: Python Programming Guide +--- + + +The Spark Python API (PySpark) exposes most of the Spark features available in the Scala version to Python. +To learn the basics of Spark, we recommend reading through the +[Scala programming guide](scala-programming-guide.html) first; it should be +easy to follow even if you don't know Scala. +This guide will show how to use the Spark features described there in Python. + +# Key Differences in the Python API + +There are a few key differences between the Python and Scala APIs: + +* Python is dynamically typed, so RDDs can hold objects of different types. +* PySpark does not currently support the following Spark features: + - Accumulators + - Special functions on RRDs of doubles, such as `mean` and `stdev` + - Approximate jobs / functions, such as `countApprox` and `sumApprox`. + - `lookup` + - `mapPartitionsWithSplit` + - `persist` at storage levels other than `MEMORY_ONLY` + - `sample` + - `sort` + + +# Installing and Configuring PySpark + +PySpark requires Python 2.6 or higher. +PySpark jobs 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/). +By default, PySpark's scripts will run programs using `python`; an alternate Python executable may be specified by setting the `PYSPARK_PYTHON` environment variable in `conf/spark-env.sh`. + +All of PySpark's library dependencies, including [Py4J](http://py4j.sourceforge.net/), are bundled with PySpark and automatically imported. + +Standalone PySpark jobs should be run using the `run-pyspark` script, which automatically configures the Java and Python environmnt using the settings in `conf/spark-env.sh`. +The script automatically adds the `pyspark` package to the `PYTHONPATH`. + + +# Interactive Use + +PySpark's `pyspark-shell` script provides a simple way to learn the API: + +{% highlight python %} +>>> words = sc.textFile("/usr/share/dict/words") +>>> words.filter(lambda w: w.startswith("spar")).take(5) +[u'spar', u'sparable', u'sparada', u'sparadrap', u'sparagrass'] +{% endhighlight %} + +# Standalone Use + +PySpark can also be used from standalone Python scripts by creating a SparkContext in the script and running the script using the `run-pyspark` script in the `pyspark` directory. +The Quick Start guide includes a [complete example](quick-start.html#a-standalone-job-in-python) of a standalone Python job. + +Code dependencies can be deployed by listing them in the `pyFiles` option in the SparkContext constructor: + +{% highlight python %} +from pyspark import SparkContext +sc = SparkContext("local", "Job Name", pyFiles=['MyFile.py', 'lib.zip', 'app.egg']) +{% endhighlight %} + +Files listed here will be added to the `PYTHONPATH` and shipped to remote worker machines. +Code dependencies can be added to an existing SparkContext using its `addPyFile()` method. + +# Where to Go from Here + +PySpark includes several sample programs using the Python API in `pyspark/examples`. +You can run them by passing the files to the `pyspark-run` script included in PySpark -- for example `./pyspark-run examples/wordcount.py`. +Each example program prints usage help when run without any arguments. + +We currently provide [API documentation](api/pyspark/index.html) for the Python API as Epydoc. +Many of the RDD method descriptions contain [doctests](http://docs.python.org/2/library/doctest.html) that provide additional usage examples. diff --git a/docs/quick-start.md b/docs/quick-start.md index defdb34836..c859c31b09 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -6,7 +6,8 @@ title: Quick Start * This will become a table of contents (this text will be scraped). {:toc} -This tutorial provides a quick introduction to using Spark. We will first introduce the API through Spark's interactive Scala shell (don't worry if you don't know Scala -- you will need much for this), then show how to write standalone jobs in Scala and Java. See the [programming guide](scala-programming-guide.html) for a fuller reference. +This tutorial provides a quick introduction to using Spark. We will first introduce the API through Spark's interactive Scala shell (don't worry if you don't know Scala -- you will need much for this), then show how to write standalone jobs in Scala, Java, and Python. +See the [programming guide](scala-programming-guide.html) for a more complete reference. To follow along with this guide, you only need to have successfully built Spark on one machine. Simply go into your Spark directory and run: @@ -230,3 +231,40 @@ Lines with a: 8422, Lines with b: 1836 {% endhighlight %} This example only runs the job locally; for a tutorial on running jobs across several machines, see the [Standalone Mode](spark-standalone.html) documentation, and consider using a distributed input source, such as HDFS. + +# A Standalone Job In Python +Now we will show how to write a standalone job using the Python API (PySpark). + +As an example, we'll create a simple Spark job, `SimpleJob.py`: + +{% highlight python %} +"""SimpleJob.py""" +from pyspark import SparkContext + +logFile = "/var/log/syslog" # Should be some file on your system +sc = SparkContext("local", "Simple job") +logData = sc.textFile(logFile).cache() + +numAs = logData.filter(lambda s: 'a' in s).count() +numBs = logData.filter(lambda s: 'b' in s).count() + +print "Lines with a: %i, lines with b: %i" % (numAs, numBs) +{% endhighlight %} + + +This job simply counts the number of lines containing 'a' and the number containing 'b' in a system log file. +Like in the Scala and Java examples, we use a SparkContext to create RDDs. +We can pass Python functions to Spark, which are automatically serialized along with any variables that they reference. +For jobs that use custom classes or third-party libraries, we can add those code dependencies to SparkContext to ensure that they will be available on remote machines; this is described in more detail in the [Python programming guide](python-programming-guide). +`SimpleJob` is simple enough that we do not need to specify any code dependencies. + +We can run this job using the `run-pyspark` script in `$SPARK_HOME/pyspark`: + +{% highlight python %} +$ cd $SPARK_HOME +$ ./pyspark/run-pyspark SimpleJob.py +... +Lines with a: 8422, Lines with b: 1836 +{% endhighlight python %} + +This example only runs the job locally; for a tutorial on running jobs across several machines, see the [Standalone Mode](spark-standalone.html) documentation, and consider using a distributed input source, such as HDFS. diff --git a/pyspark/README b/pyspark/README deleted file mode 100644 index d8d521c72c..0000000000 --- a/pyspark/README +++ /dev/null @@ -1,42 +0,0 @@ -# PySpark - -PySpark is a Python API for Spark. - -PySpark jobs are writen in Python and executed using a standard Python -interpreter; this supports modules that use Python C extensions. The -API is based on the Spark Scala API and uses regular Python functions -and lambdas to support user-defined functions. PySpark supports -interactive use through a standard Python interpreter; it can -automatically serialize closures and ship them to worker processes. - -PySpark is built on top of the Spark Java API. Data is uniformly -represented as serialized Python objects and stored in Spark Java -processes, which communicate with PySpark worker processes over pipes. - -## Features - -PySpark supports most of the Spark API, including broadcast variables. -RDDs are dynamically typed and can hold any Python object. - -PySpark does not support: - -- Special functions on RDDs of doubles -- Accumulators - -## Examples and Documentation - -The PySpark source contains docstrings and doctests that document its -API. The public classes are in `context.py` and `rdd.py`. - -The `pyspark/pyspark/examples` directory contains a few complete -examples. - -## Installing PySpark -# -To use PySpark, `SPARK_HOME` should be set to the location of the Spark -package. - -## Running PySpark - -The easiest way to run PySpark is to use the `run-pyspark` and -`pyspark-shell` scripts, which are included in the `pyspark` directory. diff --git a/pyspark/pyspark/examples/kmeans.py b/pyspark/examples/kmeans.py similarity index 100% rename from pyspark/pyspark/examples/kmeans.py rename to pyspark/examples/kmeans.py diff --git a/pyspark/pyspark/examples/pi.py b/pyspark/examples/pi.py similarity index 100% rename from pyspark/pyspark/examples/pi.py rename to pyspark/examples/pi.py diff --git a/pyspark/pyspark/examples/tc.py b/pyspark/examples/tc.py similarity index 100% rename from pyspark/pyspark/examples/tc.py rename to pyspark/examples/tc.py diff --git a/pyspark/pyspark/examples/wordcount.py b/pyspark/examples/wordcount.py similarity index 100% rename from pyspark/pyspark/examples/wordcount.py rename to pyspark/examples/wordcount.py diff --git a/pyspark/pyspark/__init__.py b/pyspark/pyspark/__init__.py index 549c2d2711..8f8402b62b 100644 --- a/pyspark/pyspark/__init__.py +++ b/pyspark/pyspark/__init__.py @@ -1,3 +1,9 @@ import sys import os sys.path.insert(0, os.path.join(os.environ["SPARK_HOME"], "pyspark/lib/py4j0.7.egg")) + + +from pyspark.context import SparkContext + + +__all__ = ["SparkContext"] diff --git a/pyspark/pyspark/examples/__init__.py b/pyspark/pyspark/examples/__init__.py deleted file mode 100644 index e69de29bb2..0000000000 From 6ee1ff2663cf1f776dd33e448548a8ddcf974dc6 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 29 Dec 2012 22:22:56 +0000 Subject: [PATCH 32/85] Fix bug in pyspark.serializers.batch; add .gitignore. --- pyspark/.gitignore | 2 ++ pyspark/pyspark/rdd.py | 4 +++- pyspark/pyspark/serializers.py | 2 +- 3 files changed, 6 insertions(+), 2 deletions(-) create mode 100644 pyspark/.gitignore diff --git a/pyspark/.gitignore b/pyspark/.gitignore new file mode 100644 index 0000000000..5c56e638f9 --- /dev/null +++ b/pyspark/.gitignore @@ -0,0 +1,2 @@ +*.pyc +docs/ diff --git a/pyspark/pyspark/rdd.py b/pyspark/pyspark/rdd.py index 111476d274..20f84b2dd0 100644 --- a/pyspark/pyspark/rdd.py +++ b/pyspark/pyspark/rdd.py @@ -695,7 +695,9 @@ def _test(): import doctest from pyspark.context import SparkContext globs = globals().copy() - globs['sc'] = SparkContext('local[4]', 'PythonTest') + # The small batch size here ensures that we see multiple batches, + # even in these small test examples: + globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) doctest.testmod(globs=globs) globs['sc'].stop() diff --git a/pyspark/pyspark/serializers.py b/pyspark/pyspark/serializers.py index 4ed925697c..8b08f7ef0f 100644 --- a/pyspark/pyspark/serializers.py +++ b/pyspark/pyspark/serializers.py @@ -24,7 +24,7 @@ def batched(iterator, batchSize): if count == batchSize: yield Batch(items) items = [] - count = [] + count = 0 if items: yield Batch(items) From 26186e2d259f3aa2db9c8594097fd342107ce147 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 29 Dec 2012 15:34:57 -0800 Subject: [PATCH 33/85] Use batching in pyspark parallelize(); fix cartesian() --- pyspark/pyspark/context.py | 4 +++- pyspark/pyspark/rdd.py | 31 +++++++++++++++---------------- pyspark/pyspark/serializers.py | 23 +++++++++++++---------- 3 files changed, 31 insertions(+), 27 deletions(-) diff --git a/pyspark/pyspark/context.py b/pyspark/pyspark/context.py index b90596ecc2..6172d69dcf 100644 --- a/pyspark/pyspark/context.py +++ b/pyspark/pyspark/context.py @@ -4,7 +4,7 @@ from tempfile import NamedTemporaryFile from pyspark.broadcast import Broadcast from pyspark.java_gateway import launch_gateway -from pyspark.serializers import dump_pickle, write_with_length +from pyspark.serializers import dump_pickle, write_with_length, batched from pyspark.rdd import RDD from py4j.java_collections import ListConverter @@ -91,6 +91,8 @@ class SparkContext(object): # objects are written to a file and loaded through textFile(). tempFile = NamedTemporaryFile(delete=False) atexit.register(lambda: os.unlink(tempFile.name)) + if self.batchSize != 1: + c = batched(c, self.batchSize) for x in c: write_with_length(dump_pickle(x), tempFile) tempFile.close() diff --git a/pyspark/pyspark/rdd.py b/pyspark/pyspark/rdd.py index 20f84b2dd0..203f7377d2 100644 --- a/pyspark/pyspark/rdd.py +++ b/pyspark/pyspark/rdd.py @@ -2,7 +2,7 @@ import atexit from base64 import standard_b64encode as b64enc import copy from collections import defaultdict -from itertools import chain, ifilter, imap +from itertools import chain, ifilter, imap, product import operator import os import shlex @@ -123,12 +123,6 @@ class RDD(object): >>> rdd = sc.parallelize([1, 1, 2, 3]) >>> rdd.union(rdd).collect() [1, 1, 2, 3, 1, 1, 2, 3] - - Union of batched and unbatched RDDs (internal test): - - >>> batchedRDD = sc.parallelize([Batch([1, 2, 3, 4, 5])]) - >>> rdd.union(batchedRDD).collect() - [1, 1, 2, 3, 1, 2, 3, 4, 5] """ return RDD(self._jrdd.union(other._jrdd), self.ctx) @@ -168,7 +162,18 @@ class RDD(object): >>> sorted(rdd.cartesian(rdd).collect()) [(1, 1), (1, 2), (2, 1), (2, 2)] """ - return RDD(self._jrdd.cartesian(other._jrdd), self.ctx) + # Due to batching, we can't use the Java cartesian method. + java_cartesian = RDD(self._jrdd.cartesian(other._jrdd), self.ctx) + def unpack_batches(pair): + (x, y) = pair + if type(x) == Batch or type(y) == Batch: + xs = x.items if type(x) == Batch else [x] + ys = y.items if type(y) == Batch else [y] + for pair in product(xs, ys): + yield pair + else: + yield pair + return java_cartesian.flatMap(unpack_batches) def groupBy(self, f, numSplits=None): """ @@ -293,8 +298,6 @@ class RDD(object): >>> sc.parallelize([2, 3, 4]).count() 3 - >>> sc.parallelize([Batch([2, 3, 4])]).count() - 3 """ return self.mapPartitions(lambda i: [sum(1 for _ in i)]).sum() @@ -667,12 +670,8 @@ class PipelinedRDD(RDD): if not self._bypass_serializer and self.ctx.batchSize != 1: oldfunc = self.func batchSize = self.ctx.batchSize - if batchSize == -1: # unlimited batch size - def batched_func(iterator): - yield Batch(list(oldfunc(iterator))) - else: - def batched_func(iterator): - return batched(oldfunc(iterator), batchSize) + def batched_func(iterator): + return batched(oldfunc(iterator), batchSize) func = batched_func cmds = [func, self._bypass_serializer] pipe_command = ' '.join(b64enc(cloudpickle.dumps(f)) for f in cmds) diff --git a/pyspark/pyspark/serializers.py b/pyspark/pyspark/serializers.py index 8b08f7ef0f..9a5151ea00 100644 --- a/pyspark/pyspark/serializers.py +++ b/pyspark/pyspark/serializers.py @@ -16,17 +16,20 @@ class Batch(object): def batched(iterator, batchSize): - items = [] - count = 0 - for item in iterator: - items.append(item) - count += 1 - if count == batchSize: + if batchSize == -1: # unlimited batch size + yield Batch(list(iterator)) + else: + items = [] + count = 0 + for item in iterator: + items.append(item) + count += 1 + if count == batchSize: + yield Batch(items) + items = [] + count = 0 + if items: yield Batch(items) - items = [] - count = 0 - if items: - yield Batch(items) def dump_pickle(obj): From 59195c68ec37acf20d527189ed757397b273a207 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 29 Dec 2012 16:01:03 -0800 Subject: [PATCH 34/85] Update PySpark for compatibility with TaskContext. --- .../src/main/scala/spark/api/python/PythonRDD.scala | 13 +++++-------- pyspark/pyspark/rdd.py | 3 ++- 2 files changed, 7 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index f76616a4c4..dc48378fdc 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -8,10 +8,7 @@ import scala.io.Source import spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} import spark.broadcast.Broadcast -import spark.SparkEnv -import spark.Split -import spark.RDD -import spark.OneToOneDependency +import spark._ import spark.rdd.PipedRDD @@ -34,7 +31,7 @@ private[spark] class PythonRDD[T: ClassManifest]( override val partitioner = if (preservePartitoning) parent.partitioner else None - override def compute(split: Split): Iterator[Array[Byte]] = { + override def compute(split: Split, context: TaskContext): Iterator[Array[Byte]] = { val SPARK_HOME = new ProcessBuilder().environment().get("SPARK_HOME") val pb = new ProcessBuilder(Seq(pythonExec, SPARK_HOME + "/pyspark/pyspark/worker.py")) @@ -74,7 +71,7 @@ private[spark] class PythonRDD[T: ClassManifest]( out.println(elem) } out.flush() - for (elem <- parent.iterator(split)) { + for (elem <- parent.iterator(split, context)) { PythonRDD.writeAsPickle(elem, dOut) } dOut.flush() @@ -123,8 +120,8 @@ private class PairwiseRDD(prev: RDD[Array[Byte]]) extends RDD[(Array[Byte], Array[Byte])](prev.context) { override def splits = prev.splits override val dependencies = List(new OneToOneDependency(prev)) - override def compute(split: Split) = - prev.iterator(split).grouped(2).map { + override def compute(split: Split, context: TaskContext) = + prev.iterator(split, context).grouped(2).map { case Seq(a, b) => (a, b) case x => throw new Exception("PairwiseRDD: unexpected value: " + x) } diff --git a/pyspark/pyspark/rdd.py b/pyspark/pyspark/rdd.py index 203f7377d2..21dda31c4e 100644 --- a/pyspark/pyspark/rdd.py +++ b/pyspark/pyspark/rdd.py @@ -335,9 +335,10 @@ class RDD(object): """ items = [] splits = self._jrdd.splits() + taskContext = self.ctx.jvm.spark.TaskContext(0, 0, 0) while len(items) < num and splits: split = splits.pop(0) - iterator = self._jrdd.iterator(split) + iterator = self._jrdd.iterator(split, taskContext) items.extend(self._collect_iterator_through_file(iterator)) return items[:num] From 39dd953fd88e9aa7335603ab452d9c1bed4ba67a Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 29 Dec 2012 17:06:50 -0800 Subject: [PATCH 35/85] Add test for pyspark.RDD.saveAsTextFile(). --- pyspark/pyspark/rdd.py | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/pyspark/pyspark/rdd.py b/pyspark/pyspark/rdd.py index 21dda31c4e..cbffb6cc1f 100644 --- a/pyspark/pyspark/rdd.py +++ b/pyspark/pyspark/rdd.py @@ -351,10 +351,17 @@ class RDD(object): """ return self.take(1)[0] - # TODO: add test and fix for use with Batch def saveAsTextFile(self, path): """ Save this RDD as a text file, using string representations of elements. + + >>> tempFile = NamedTemporaryFile(delete=True) + >>> tempFile.close() + >>> sc.parallelize(range(10)).saveAsTextFile(tempFile.name) + >>> from fileinput import input + >>> from glob import glob + >>> ''.join(input(glob(tempFile.name + "/part-0000*"))) + '0\\n1\\n2\\n3\\n4\\n5\\n6\\n7\\n8\\n9\\n' """ def func(iterator): return (str(x).encode("utf-8") for x in iterator) From 099898b43955d99351ec94d4a373de854bf7edf7 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 29 Dec 2012 17:52:47 -0800 Subject: [PATCH 36/85] Port LR example to PySpark using numpy. This version of the example crashes after the first iteration with "OverflowError: math range error" because Python's math.exp() behaves differently than Scala's; see SPARK-646. --- pyspark/examples/lr.py | 57 ++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 57 insertions(+) create mode 100755 pyspark/examples/lr.py diff --git a/pyspark/examples/lr.py b/pyspark/examples/lr.py new file mode 100755 index 0000000000..5fca0266b8 --- /dev/null +++ b/pyspark/examples/lr.py @@ -0,0 +1,57 @@ +""" +This example requires numpy (http://www.numpy.org/) +""" +from collections import namedtuple +from math import exp +from os.path import realpath +import sys + +import numpy as np +from pyspark.context import SparkContext + + +N = 100000 # Number of data points +D = 10 # Number of dimensions +R = 0.7 # Scaling factor +ITERATIONS = 5 +np.random.seed(42) + + +DataPoint = namedtuple("DataPoint", ['x', 'y']) +from lr import DataPoint # So that DataPoint is properly serialized + + +def generateData(): + def generatePoint(i): + y = -1 if i % 2 == 0 else 1 + x = np.random.normal(size=D) + (y * R) + return DataPoint(x, y) + return [generatePoint(i) for i in range(N)] + + +if __name__ == "__main__": + if len(sys.argv) == 1: + print >> sys.stderr, \ + "Usage: PythonLR []" + exit(-1) + sc = SparkContext(sys.argv[1], "PythonLR", pyFiles=[realpath(__file__)]) + slices = int(sys.argv[2]) if len(sys.argv) > 2 else 2 + points = sc.parallelize(generateData(), slices).cache() + + # Initialize w to a random value + w = 2 * np.random.ranf(size=D) - 1 + print "Initial w: " + str(w) + + def add(x, y): + x += y + return x + + for i in range(1, ITERATIONS + 1): + print "On iteration %i" % i + + gradient = points.map(lambda p: + (1.0 / (1.0 + exp(-p.y * np.dot(w, p.x)))) * p.y * p.x + ).reduce(add) + w -= gradient + + print "Final w: " + str(w) From 6f6a6b79c4c3f3555f8ff427c91e714d02afe8fa Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 31 Dec 2012 14:56:23 -0800 Subject: [PATCH 37/85] Launch with `scala` by default in run-pyspark --- pyspark/run-pyspark | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/pyspark/run-pyspark b/pyspark/run-pyspark index f8039b8038..4d10fbea8b 100755 --- a/pyspark/run-pyspark +++ b/pyspark/run-pyspark @@ -20,4 +20,9 @@ export PYSPARK_PYTHON # Add the PySpark classes to the Python path: export PYTHONPATH=$SPARK_HOME/pyspark/:$PYTHONPATH +# Launch with `scala` by default: +if [[ "$SPARK_LAUNCH_WITH_SCALA" != "0" ]] ; then + export SPARK_LAUNCH_WITH_SCALA=1 +fi + exec "$PYSPARK_PYTHON" "$@" From 170e451fbdd308ae77065bd9c0f2bd278abf0cb7 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 1 Jan 2013 13:52:14 -0800 Subject: [PATCH 38/85] Minor documentation and style fixes for PySpark. --- .../spark/api/python/PythonPartitioner.scala | 4 +- .../scala/spark/api/python/PythonRDD.scala | 43 +++++++++++++------ docs/index.md | 8 +++- docs/python-programming-guide.md | 3 +- pyspark/examples/kmeans.py | 13 +++--- .../{lr.py => logistic_regression.py} | 4 +- pyspark/examples/pi.py | 5 ++- .../examples/{tc.py => transitive_closure.py} | 5 ++- pyspark/examples/wordcount.py | 4 +- pyspark/pyspark/__init__.py | 13 +++++- 10 files changed, 70 insertions(+), 32 deletions(-) rename pyspark/examples/{lr.py => logistic_regression.py} (93%) rename pyspark/examples/{tc.py => transitive_closure.py} (94%) diff --git a/core/src/main/scala/spark/api/python/PythonPartitioner.scala b/core/src/main/scala/spark/api/python/PythonPartitioner.scala index 2c829508e5..648d9402b0 100644 --- a/core/src/main/scala/spark/api/python/PythonPartitioner.scala +++ b/core/src/main/scala/spark/api/python/PythonPartitioner.scala @@ -17,9 +17,9 @@ private[spark] class PythonPartitioner(override val numPartitions: Int) extends val hashCode = { if (key.isInstanceOf[Array[Byte]]) { Arrays.hashCode(key.asInstanceOf[Array[Byte]]) - } - else + } else { key.hashCode() + } } val mod = hashCode % numPartitions if (mod < 0) { diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index dc48378fdc..19a039e330 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -13,8 +13,12 @@ import spark.rdd.PipedRDD private[spark] class PythonRDD[T: ClassManifest]( - parent: RDD[T], command: Seq[String], envVars: java.util.Map[String, String], - preservePartitoning: Boolean, pythonExec: String, broadcastVars: java.util.List[Broadcast[Array[Byte]]]) + parent: RDD[T], + command: Seq[String], + envVars: java.util.Map[String, String], + preservePartitoning: Boolean, + pythonExec: String, + broadcastVars: java.util.List[Broadcast[Array[Byte]]]) extends RDD[Array[Byte]](parent.context) { // Similar to Runtime.exec(), if we are given a single string, split it into words @@ -38,8 +42,8 @@ private[spark] class PythonRDD[T: ClassManifest]( // Add the environmental variables to the process. val currentEnvVars = pb.environment() - envVars.foreach { - case (variable, value) => currentEnvVars.put(variable, value) + for ((variable, value) <- envVars) { + currentEnvVars.put(variable, value) } val proc = pb.start() @@ -116,6 +120,10 @@ private[spark] class PythonRDD[T: ClassManifest]( val asJavaRDD : JavaRDD[Array[Byte]] = JavaRDD.fromRDD(this) } +/** + * Form an RDD[(Array[Byte], Array[Byte])] from key-value pairs returned from Python. + * This is used by PySpark's shuffle operations. + */ private class PairwiseRDD(prev: RDD[Array[Byte]]) extends RDD[(Array[Byte], Array[Byte])](prev.context) { override def splits = prev.splits @@ -139,6 +147,16 @@ private[spark] object PythonRDD { * Write strings, pickled Python objects, or pairs of pickled objects to a data output stream. * The data format is a 32-bit integer representing the pickled object's length (in bytes), * followed by the pickled data. + * + * Pickle module: + * + * http://docs.python.org/2/library/pickle.html + * + * The pickle protocol is documented in the source of the `pickle` and `pickletools` modules: + * + * http://hg.python.org/cpython/file/2.6/Lib/pickle.py + * http://hg.python.org/cpython/file/2.6/Lib/pickletools.py + * * @param elem the object to write * @param dOut a data output stream */ @@ -201,15 +219,14 @@ private[spark] object PythonRDD { } private object Pickle { - def b(x: Int): Byte = x.asInstanceOf[Byte] - val PROTO: Byte = b(0x80) - val TWO: Byte = b(0x02) - val BINUNICODE : Byte = 'X' - val STOP : Byte = '.' - val TUPLE2 : Byte = b(0x86) - val EMPTY_LIST : Byte = ']' - val MARK : Byte = '(' - val APPENDS : Byte = 'e' + val PROTO: Byte = 0x80.toByte + val TWO: Byte = 0x02.toByte + val BINUNICODE: Byte = 'X' + val STOP: Byte = '.' + val TUPLE2: Byte = 0x86.toByte + val EMPTY_LIST: Byte = ']' + val MARK: Byte = '(' + val APPENDS: Byte = 'e' } private class ExtractValue extends spark.api.java.function.Function[(Array[Byte], diff --git a/docs/index.md b/docs/index.md index 33ab58a962..848b585333 100644 --- a/docs/index.md +++ b/docs/index.md @@ -8,7 +8,7 @@ TODO(andyk): Rewrite to make the Java API a first class part of the story. {% endcomment %} Spark is a MapReduce-like cluster computing framework designed for low-latency iterative jobs and interactive use from an interpreter. -It provides clean, language-integrated APIs in Scala, Java, and Python, with a rich array of parallel operators. +It provides clean, language-integrated APIs in [Scala](scala-programming-guide.html), [Java](java-programming-guide.html), and [Python](python-programming-guide.html), with a rich array of parallel operators. Spark can run on top of the [Apache Mesos](http://incubator.apache.org/mesos/) cluster manager, [Hadoop YARN](http://hadoop.apache.org/docs/r2.0.1-alpha/hadoop-yarn/hadoop-yarn-site/YARN.html), Amazon EC2, or without an independent resource manager ("standalone mode"). @@ -61,6 +61,11 @@ of `project/SparkBuild.scala`, then rebuilding Spark (`sbt/sbt clean compile`). * [Java Programming Guide](java-programming-guide.html): using Spark from Java * [Python Programming Guide](python-programming-guide.html): using Spark from Python +**API Docs:** + +* [Java/Scala (Scaladoc)](api/core/index.html) +* [Python (Epydoc)](api/pyspark/index.html) + **Deployment guides:** * [Running Spark on Amazon EC2](ec2-scripts.html): scripts that let you launch a cluster on EC2 in about 5 minutes @@ -73,7 +78,6 @@ of `project/SparkBuild.scala`, then rebuilding Spark (`sbt/sbt clean compile`). * [Configuration](configuration.html): customize Spark via its configuration system * [Tuning Guide](tuning.html): best practices to optimize performance and memory use -* API Docs: [Java/Scala (Scaladoc)](api/core/index.html) and [Python (Epydoc)](api/pyspark/index.html) * [Bagel](bagel-programming-guide.html): an implementation of Google's Pregel on Spark * [Contributing to Spark](contributing-to-spark.html) diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index b7c747f905..d88d4eb42d 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -17,8 +17,7 @@ There are a few key differences between the Python and Scala APIs: * Python is dynamically typed, so RDDs can hold objects of different types. * PySpark does not currently support the following Spark features: - Accumulators - - Special functions on RRDs of doubles, such as `mean` and `stdev` - - Approximate jobs / functions, such as `countApprox` and `sumApprox`. + - Special functions on RDDs of doubles, such as `mean` and `stdev` - `lookup` - `mapPartitionsWithSplit` - `persist` at storage levels other than `MEMORY_ONLY` diff --git a/pyspark/examples/kmeans.py b/pyspark/examples/kmeans.py index 9cc366f03c..ad2be21178 100644 --- a/pyspark/examples/kmeans.py +++ b/pyspark/examples/kmeans.py @@ -1,18 +1,21 @@ +""" +This example requires numpy (http://www.numpy.org/) +""" import sys -from pyspark.context import SparkContext -from numpy import array, sum as np_sum +import numpy as np +from pyspark import SparkContext def parseVector(line): - return array([float(x) for x in line.split(' ')]) + return np.array([float(x) for x in line.split(' ')]) def closestPoint(p, centers): bestIndex = 0 closest = float("+inf") for i in range(len(centers)): - tempDist = np_sum((p - centers[i]) ** 2) + tempDist = np.sum((p - centers[i]) ** 2) if tempDist < closest: closest = tempDist bestIndex = i @@ -41,7 +44,7 @@ if __name__ == "__main__": newPoints = pointStats.map( lambda (x, (y, z)): (x, y / z)).collect() - tempDist = sum(np_sum((kPoints[x] - y) ** 2) for (x, y) in newPoints) + tempDist = sum(np.sum((kPoints[x] - y) ** 2) for (x, y) in newPoints) for (x, y) in newPoints: kPoints[x] = y diff --git a/pyspark/examples/lr.py b/pyspark/examples/logistic_regression.py similarity index 93% rename from pyspark/examples/lr.py rename to pyspark/examples/logistic_regression.py index 5fca0266b8..f13698a86f 100755 --- a/pyspark/examples/lr.py +++ b/pyspark/examples/logistic_regression.py @@ -7,7 +7,7 @@ from os.path import realpath import sys import numpy as np -from pyspark.context import SparkContext +from pyspark import SparkContext N = 100000 # Number of data points @@ -32,7 +32,7 @@ def generateData(): if __name__ == "__main__": if len(sys.argv) == 1: print >> sys.stderr, \ - "Usage: PythonLR []" + "Usage: PythonLR []" exit(-1) sc = SparkContext(sys.argv[1], "PythonLR", pyFiles=[realpath(__file__)]) slices = int(sys.argv[2]) if len(sys.argv) > 2 else 2 diff --git a/pyspark/examples/pi.py b/pyspark/examples/pi.py index 348bbc5dce..127cba029b 100644 --- a/pyspark/examples/pi.py +++ b/pyspark/examples/pi.py @@ -1,13 +1,14 @@ import sys from random import random from operator import add -from pyspark.context import SparkContext + +from pyspark import SparkContext if __name__ == "__main__": if len(sys.argv) == 1: print >> sys.stderr, \ - "Usage: PythonPi []" + "Usage: PythonPi []" exit(-1) sc = SparkContext(sys.argv[1], "PythonPi") slices = int(sys.argv[2]) if len(sys.argv) > 2 else 2 diff --git a/pyspark/examples/tc.py b/pyspark/examples/transitive_closure.py similarity index 94% rename from pyspark/examples/tc.py rename to pyspark/examples/transitive_closure.py index 9630e72b47..73f7f8fbaf 100644 --- a/pyspark/examples/tc.py +++ b/pyspark/examples/transitive_closure.py @@ -1,6 +1,7 @@ import sys from random import Random -from pyspark.context import SparkContext + +from pyspark import SparkContext numEdges = 200 numVertices = 100 @@ -20,7 +21,7 @@ def generateGraph(): if __name__ == "__main__": if len(sys.argv) == 1: print >> sys.stderr, \ - "Usage: PythonTC []" + "Usage: PythonTC []" exit(-1) sc = SparkContext(sys.argv[1], "PythonTC") slices = sys.argv[2] if len(sys.argv) > 2 else 2 diff --git a/pyspark/examples/wordcount.py b/pyspark/examples/wordcount.py index 8365c070e8..857160624b 100644 --- a/pyspark/examples/wordcount.py +++ b/pyspark/examples/wordcount.py @@ -1,6 +1,8 @@ import sys from operator import add -from pyspark.context import SparkContext + +from pyspark import SparkContext + if __name__ == "__main__": if len(sys.argv) < 3: diff --git a/pyspark/pyspark/__init__.py b/pyspark/pyspark/__init__.py index 8f8402b62b..1ab360a666 100644 --- a/pyspark/pyspark/__init__.py +++ b/pyspark/pyspark/__init__.py @@ -1,9 +1,20 @@ +""" +PySpark is a Python API for Spark. + +Public classes: + + - L{SparkContext} + Main entry point for Spark functionality. + - L{RDD} + A Resilient Distributed Dataset (RDD), the basic abstraction in Spark. +""" import sys import os sys.path.insert(0, os.path.join(os.environ["SPARK_HOME"], "pyspark/lib/py4j0.7.egg")) from pyspark.context import SparkContext +from pyspark.rdd import RDD -__all__ = ["SparkContext"] +__all__ = ["SparkContext", "RDD"] From b58340dbd9a741331fc4c3829b08c093560056c2 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 1 Jan 2013 14:48:45 -0800 Subject: [PATCH 39/85] Rename top-level 'pyspark' directory to 'python' --- .../src/main/scala/spark/api/python/PythonRDD.scala | 2 +- docs/_plugins/copy_api_dirs.rb | 8 ++++---- pyspark-shell | 3 +++ pyspark/pyspark-shell | 3 --- {pyspark => python}/.gitignore | 0 {pyspark => python}/epydoc.conf | 0 {pyspark => python}/examples/kmeans.py | 0 {pyspark => python}/examples/logistic_regression.py | 0 {pyspark => python}/examples/pi.py | 0 {pyspark => python}/examples/transitive_closure.py | 0 {pyspark => python}/examples/wordcount.py | 0 {pyspark => python}/lib/PY4J_LICENSE.txt | 0 {pyspark => python}/lib/PY4J_VERSION.txt | 0 {pyspark => python}/lib/py4j0.7.egg | Bin {pyspark => python}/lib/py4j0.7.jar | Bin {pyspark => python}/pyspark/__init__.py | 2 +- {pyspark => python}/pyspark/broadcast.py | 0 {pyspark => python}/pyspark/cloudpickle.py | 0 {pyspark => python}/pyspark/context.py | 0 {pyspark => python}/pyspark/java_gateway.py | 0 {pyspark => python}/pyspark/join.py | 0 {pyspark => python}/pyspark/rdd.py | 0 {pyspark => python}/pyspark/serializers.py | 0 {pyspark => python}/pyspark/shell.py | 0 {pyspark => python}/pyspark/worker.py | 0 run | 2 +- pyspark/run-pyspark => run-pyspark | 4 ++-- run2.cmd | 2 +- 28 files changed, 13 insertions(+), 13 deletions(-) create mode 100755 pyspark-shell delete mode 100755 pyspark/pyspark-shell rename {pyspark => python}/.gitignore (100%) rename {pyspark => python}/epydoc.conf (100%) rename {pyspark => python}/examples/kmeans.py (100%) rename {pyspark => python}/examples/logistic_regression.py (100%) rename {pyspark => python}/examples/pi.py (100%) rename {pyspark => python}/examples/transitive_closure.py (100%) rename {pyspark => python}/examples/wordcount.py (100%) rename {pyspark => python}/lib/PY4J_LICENSE.txt (100%) rename {pyspark => python}/lib/PY4J_VERSION.txt (100%) rename {pyspark => python}/lib/py4j0.7.egg (100%) rename {pyspark => python}/lib/py4j0.7.jar (100%) rename {pyspark => python}/pyspark/__init__.py (82%) rename {pyspark => python}/pyspark/broadcast.py (100%) rename {pyspark => python}/pyspark/cloudpickle.py (100%) rename {pyspark => python}/pyspark/context.py (100%) rename {pyspark => python}/pyspark/java_gateway.py (100%) rename {pyspark => python}/pyspark/join.py (100%) rename {pyspark => python}/pyspark/rdd.py (100%) rename {pyspark => python}/pyspark/serializers.py (100%) rename {pyspark => python}/pyspark/shell.py (100%) rename {pyspark => python}/pyspark/worker.py (100%) rename pyspark/run-pyspark => run-pyspark (86%) diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 19a039e330..cf60d14f03 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -38,7 +38,7 @@ private[spark] class PythonRDD[T: ClassManifest]( override def compute(split: Split, context: TaskContext): Iterator[Array[Byte]] = { val SPARK_HOME = new ProcessBuilder().environment().get("SPARK_HOME") - val pb = new ProcessBuilder(Seq(pythonExec, SPARK_HOME + "/pyspark/pyspark/worker.py")) + val pb = new ProcessBuilder(Seq(pythonExec, SPARK_HOME + "/python/pyspark/worker.py")) // Add the environmental variables to the process. val currentEnvVars = pb.environment() diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index 577f3ebe70..c9ce589c1b 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -30,8 +30,8 @@ if ENV['SKIP_SCALADOC'] != '1' end if ENV['SKIP_EPYDOC'] != '1' - puts "Moving to pyspark directory and building epydoc." - cd("../pyspark") + puts "Moving to python directory and building epydoc." + cd("../python") puts `epydoc --config epydoc.conf` puts "Moving back into docs dir." @@ -40,8 +40,8 @@ if ENV['SKIP_EPYDOC'] != '1' puts "echo making directory pyspark" mkdir_p "pyspark" - puts "cp -r ../pyspark/docs/. api/pyspark" - cp_r("../pyspark/docs/.", "api/pyspark") + puts "cp -r ../python/docs/. api/pyspark" + cp_r("../python/docs/.", "api/pyspark") cd("..") end diff --git a/pyspark-shell b/pyspark-shell new file mode 100755 index 0000000000..27aaac3a26 --- /dev/null +++ b/pyspark-shell @@ -0,0 +1,3 @@ +#!/usr/bin/env bash +FWDIR="`dirname $0`" +exec $FWDIR/run-pyspark $FWDIR/python/pyspark/shell.py "$@" diff --git a/pyspark/pyspark-shell b/pyspark/pyspark-shell deleted file mode 100755 index e3736826e8..0000000000 --- a/pyspark/pyspark-shell +++ /dev/null @@ -1,3 +0,0 @@ -#!/usr/bin/env bash -FWDIR="`dirname $0`" -exec $FWDIR/run-pyspark $FWDIR/pyspark/shell.py "$@" diff --git a/pyspark/.gitignore b/python/.gitignore similarity index 100% rename from pyspark/.gitignore rename to python/.gitignore diff --git a/pyspark/epydoc.conf b/python/epydoc.conf similarity index 100% rename from pyspark/epydoc.conf rename to python/epydoc.conf diff --git a/pyspark/examples/kmeans.py b/python/examples/kmeans.py similarity index 100% rename from pyspark/examples/kmeans.py rename to python/examples/kmeans.py diff --git a/pyspark/examples/logistic_regression.py b/python/examples/logistic_regression.py similarity index 100% rename from pyspark/examples/logistic_regression.py rename to python/examples/logistic_regression.py diff --git a/pyspark/examples/pi.py b/python/examples/pi.py similarity index 100% rename from pyspark/examples/pi.py rename to python/examples/pi.py diff --git a/pyspark/examples/transitive_closure.py b/python/examples/transitive_closure.py similarity index 100% rename from pyspark/examples/transitive_closure.py rename to python/examples/transitive_closure.py diff --git a/pyspark/examples/wordcount.py b/python/examples/wordcount.py similarity index 100% rename from pyspark/examples/wordcount.py rename to python/examples/wordcount.py diff --git a/pyspark/lib/PY4J_LICENSE.txt b/python/lib/PY4J_LICENSE.txt similarity index 100% rename from pyspark/lib/PY4J_LICENSE.txt rename to python/lib/PY4J_LICENSE.txt diff --git a/pyspark/lib/PY4J_VERSION.txt b/python/lib/PY4J_VERSION.txt similarity index 100% rename from pyspark/lib/PY4J_VERSION.txt rename to python/lib/PY4J_VERSION.txt diff --git a/pyspark/lib/py4j0.7.egg b/python/lib/py4j0.7.egg similarity index 100% rename from pyspark/lib/py4j0.7.egg rename to python/lib/py4j0.7.egg diff --git a/pyspark/lib/py4j0.7.jar b/python/lib/py4j0.7.jar similarity index 100% rename from pyspark/lib/py4j0.7.jar rename to python/lib/py4j0.7.jar diff --git a/pyspark/pyspark/__init__.py b/python/pyspark/__init__.py similarity index 82% rename from pyspark/pyspark/__init__.py rename to python/pyspark/__init__.py index 1ab360a666..c595ae0842 100644 --- a/pyspark/pyspark/__init__.py +++ b/python/pyspark/__init__.py @@ -10,7 +10,7 @@ Public classes: """ import sys import os -sys.path.insert(0, os.path.join(os.environ["SPARK_HOME"], "pyspark/lib/py4j0.7.egg")) +sys.path.insert(0, os.path.join(os.environ["SPARK_HOME"], "python/lib/py4j0.7.egg")) from pyspark.context import SparkContext diff --git a/pyspark/pyspark/broadcast.py b/python/pyspark/broadcast.py similarity index 100% rename from pyspark/pyspark/broadcast.py rename to python/pyspark/broadcast.py diff --git a/pyspark/pyspark/cloudpickle.py b/python/pyspark/cloudpickle.py similarity index 100% rename from pyspark/pyspark/cloudpickle.py rename to python/pyspark/cloudpickle.py diff --git a/pyspark/pyspark/context.py b/python/pyspark/context.py similarity index 100% rename from pyspark/pyspark/context.py rename to python/pyspark/context.py diff --git a/pyspark/pyspark/java_gateway.py b/python/pyspark/java_gateway.py similarity index 100% rename from pyspark/pyspark/java_gateway.py rename to python/pyspark/java_gateway.py diff --git a/pyspark/pyspark/join.py b/python/pyspark/join.py similarity index 100% rename from pyspark/pyspark/join.py rename to python/pyspark/join.py diff --git a/pyspark/pyspark/rdd.py b/python/pyspark/rdd.py similarity index 100% rename from pyspark/pyspark/rdd.py rename to python/pyspark/rdd.py diff --git a/pyspark/pyspark/serializers.py b/python/pyspark/serializers.py similarity index 100% rename from pyspark/pyspark/serializers.py rename to python/pyspark/serializers.py diff --git a/pyspark/pyspark/shell.py b/python/pyspark/shell.py similarity index 100% rename from pyspark/pyspark/shell.py rename to python/pyspark/shell.py diff --git a/pyspark/pyspark/worker.py b/python/pyspark/worker.py similarity index 100% rename from pyspark/pyspark/worker.py rename to python/pyspark/worker.py diff --git a/run b/run index ed788c4db3..08e2b2434b 100755 --- a/run +++ b/run @@ -63,7 +63,7 @@ CORE_DIR="$FWDIR/core" REPL_DIR="$FWDIR/repl" EXAMPLES_DIR="$FWDIR/examples" BAGEL_DIR="$FWDIR/bagel" -PYSPARK_DIR="$FWDIR/pyspark" +PYSPARK_DIR="$FWDIR/python" # Build up classpath CLASSPATH="$SPARK_CLASSPATH" diff --git a/pyspark/run-pyspark b/run-pyspark similarity index 86% rename from pyspark/run-pyspark rename to run-pyspark index 4d10fbea8b..deb0d708b3 100755 --- a/pyspark/run-pyspark +++ b/run-pyspark @@ -1,7 +1,7 @@ #!/usr/bin/env bash # Figure out where the Scala framework is installed -FWDIR="$(cd `dirname $0`; cd ../; pwd)" +FWDIR="$(cd `dirname $0`; pwd)" # Export this as SPARK_HOME export SPARK_HOME="$FWDIR" @@ -18,7 +18,7 @@ fi export PYSPARK_PYTHON # Add the PySpark classes to the Python path: -export PYTHONPATH=$SPARK_HOME/pyspark/:$PYTHONPATH +export PYTHONPATH=$SPARK_HOME/python/:$PYTHONPATH # Launch with `scala` by default: if [[ "$SPARK_LAUNCH_WITH_SCALA" != "0" ]] ; then diff --git a/run2.cmd b/run2.cmd index 9c50804e69..83464b1166 100644 --- a/run2.cmd +++ b/run2.cmd @@ -34,7 +34,7 @@ set CORE_DIR=%FWDIR%core set REPL_DIR=%FWDIR%repl set EXAMPLES_DIR=%FWDIR%examples set BAGEL_DIR=%FWDIR%bagel -set PYSPARK_DIR=%FWDIR%pyspark +set PYSPARK_DIR=%FWDIR%python rem Build up classpath set CLASSPATH=%SPARK_CLASSPATH%;%MESOS_CLASSPATH%;%FWDIR%conf;%CORE_DIR%\target\scala-%SCALA_VERSION%\classes From ce9f1bbe20eff794cd1d588dc88f109d32588cfe Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 1 Jan 2013 21:25:49 -0800 Subject: [PATCH 40/85] Add `pyspark` script to replace the other scripts. Expand the PySpark programming guide. --- docs/python-programming-guide.md | 49 ++++++++++++++++++++++++++++---- docs/quick-start.md | 4 +-- run-pyspark => pyspark | 4 +++ pyspark-shell | 3 -- python/pyspark/shell.py | 36 +++++++---------------- python/run-tests | 9 ++++++ 6 files changed, 69 insertions(+), 36 deletions(-) rename run-pyspark => pyspark (80%) delete mode 100755 pyspark-shell create mode 100755 python/run-tests diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index d88d4eb42d..d963551296 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -24,6 +24,35 @@ There are a few key differences between the Python and Scala APIs: - `sample` - `sort` +In PySpark, RDDs support the same methods as their Scala counterparts but take Python functions and return Python collection types. +Short functions can be passed to RDD methods using Python's [`lambda`](http://www.diveintopython.net/power_of_introspection/lambda_functions.html) syntax: + +{% highlight python %} +logData = sc.textFile(logFile).cache() +errors = logData.filter(lambda s: 'ERROR' in s.split()) +{% endhighlight %} + +You can also pass functions that are defined using the `def` keyword; this is useful for more complicated functions that cannot be expressed using `lambda`: + +{% highlight python %} +def is_error(line): + return 'ERROR' in line.split() +errors = logData.filter(is_error) +{% endhighlight %} + +Functions can access objects in enclosing scopes, although modifications to those objects within RDD methods will not be propagated to other tasks: + +{% highlight python %} +error_keywords = ["Exception", "Error"] +def is_error(line): + words = line.split() + return any(keyword in words for keyword in error_keywords) +errors = logData.filter(is_error) +{% endhighlight %} + +PySpark will automatically ship these functions to workers, along with any objects that they reference. +Instances of classes will be serialized and shipped to workers by PySpark, but classes themselves cannot be automatically distributed to workers. +The [Standalone Use](#standalone-use) section describes how to ship code dependencies to workers. # Installing and Configuring PySpark @@ -34,13 +63,14 @@ By default, PySpark's scripts will run programs using `python`; an alternate Pyt All of PySpark's library dependencies, including [Py4J](http://py4j.sourceforge.net/), are bundled with PySpark and automatically imported. -Standalone PySpark jobs should be run using the `run-pyspark` script, which automatically configures the Java and Python environmnt using the settings in `conf/spark-env.sh`. +Standalone PySpark jobs should be run using the `pyspark` script, which automatically configures the Java and Python environment using the settings in `conf/spark-env.sh`. The script automatically adds the `pyspark` package to the `PYTHONPATH`. # Interactive Use -PySpark's `pyspark-shell` script provides a simple way to learn the API: +The `pyspark` script launches a Python interpreter that is configured to run PySpark jobs. +When run without any input files, `pyspark` launches a shell that can be used explore data interactively, which is a simple way to learn the API: {% highlight python %} >>> words = sc.textFile("/usr/share/dict/words") @@ -48,9 +78,18 @@ PySpark's `pyspark-shell` script provides a simple way to learn the API: [u'spar', u'sparable', u'sparada', u'sparadrap', u'sparagrass'] {% endhighlight %} +By default, the `pyspark` shell creates SparkContext that runs jobs locally. +To connect to a non-local cluster, set the `MASTER` environment variable. +For example, to use the `pyspark` shell with a [standalone Spark cluster](spark-standalone.html): + +{% highlight shell %} +$ MASTER=spark://IP:PORT ./pyspark +{% endhighlight %} + + # Standalone Use -PySpark can also be used from standalone Python scripts by creating a SparkContext in the script and running the script using the `run-pyspark` script in the `pyspark` directory. +PySpark can also be used from standalone Python scripts by creating a SparkContext in your script and running the script using `pyspark`. The Quick Start guide includes a [complete example](quick-start.html#a-standalone-job-in-python) of a standalone Python job. Code dependencies can be deployed by listing them in the `pyFiles` option in the SparkContext constructor: @@ -65,8 +104,8 @@ Code dependencies can be added to an existing SparkContext using its `addPyFile( # Where to Go from Here -PySpark includes several sample programs using the Python API in `pyspark/examples`. -You can run them by passing the files to the `pyspark-run` script included in PySpark -- for example `./pyspark-run examples/wordcount.py`. +PySpark includes several sample programs using the Python API in `python/examples`. +You can run them by passing the files to the `pyspark` script -- for example `./pyspark python/examples/wordcount.py`. Each example program prints usage help when run without any arguments. We currently provide [API documentation](api/pyspark/index.html) for the Python API as Epydoc. diff --git a/docs/quick-start.md b/docs/quick-start.md index 8c25df5486..2c7cfbed25 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -258,11 +258,11 @@ We can pass Python functions to Spark, which are automatically serialized along For jobs that use custom classes or third-party libraries, we can add those code dependencies to SparkContext to ensure that they will be available on remote machines; this is described in more detail in the [Python programming guide](python-programming-guide). `SimpleJob` is simple enough that we do not need to specify any code dependencies. -We can run this job using the `run-pyspark` script in `$SPARK_HOME/pyspark`: +We can run this job using the `pyspark` script: {% highlight python %} $ cd $SPARK_HOME -$ ./pyspark/run-pyspark SimpleJob.py +$ ./pyspark SimpleJob.py ... Lines with a: 8422, Lines with b: 1836 {% endhighlight python %} diff --git a/run-pyspark b/pyspark similarity index 80% rename from run-pyspark rename to pyspark index deb0d708b3..9e89d51ba2 100755 --- a/run-pyspark +++ b/pyspark @@ -20,6 +20,10 @@ export PYSPARK_PYTHON # Add the PySpark classes to the Python path: export PYTHONPATH=$SPARK_HOME/python/:$PYTHONPATH +# Load the PySpark shell.py script when ./pyspark is used interactively: +export OLD_PYTHONSTARTUP=$PYTHONSTARTUP +export PYTHONSTARTUP=$FWDIR/python/pyspark/shell.py + # Launch with `scala` by default: if [[ "$SPARK_LAUNCH_WITH_SCALA" != "0" ]] ; then export SPARK_LAUNCH_WITH_SCALA=1 diff --git a/pyspark-shell b/pyspark-shell deleted file mode 100755 index 27aaac3a26..0000000000 --- a/pyspark-shell +++ /dev/null @@ -1,3 +0,0 @@ -#!/usr/bin/env bash -FWDIR="`dirname $0`" -exec $FWDIR/run-pyspark $FWDIR/python/pyspark/shell.py "$@" diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index bd39b0283f..7e6ad3aa76 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -1,33 +1,17 @@ """ An interactive shell. -""" -import optparse # I prefer argparse, but it's not included with Python < 2.7 -import code -import sys +This fle is designed to be launched as a PYTHONSTARTUP script. +""" +import os from pyspark.context import SparkContext -def main(master='local', ipython=False): - sc = SparkContext(master, 'PySparkShell') - user_ns = {'sc' : sc} - banner = "Spark context avaiable as sc." - if ipython: - import IPython - IPython.embed(user_ns=user_ns, banner2=banner) - else: - print banner - code.interact(local=user_ns) +sc = SparkContext(os.environ.get("MASTER", "local"), "PySparkShell") +print "Spark context avaiable as sc." - -if __name__ == '__main__': - usage = "usage: %prog [options] master" - parser = optparse.OptionParser(usage=usage) - parser.add_option("-i", "--ipython", help="Run IPython shell", - action="store_true") - (options, args) = parser.parse_args() - if len(sys.argv) > 1: - master = args[0] - else: - master = 'local' - main(master, options.ipython) +# The ./pyspark script stores the old PYTHONSTARTUP value in OLD_PYTHONSTARTUP, +# which allows us to execute the user's PYTHONSTARTUP file: +_pythonstartup = os.environ.get('OLD_PYTHONSTARTUP') +if _pythonstartup and os.path.isfile(_pythonstartup): + execfile(_pythonstartup) diff --git a/python/run-tests b/python/run-tests new file mode 100755 index 0000000000..da9e24cb1f --- /dev/null +++ b/python/run-tests @@ -0,0 +1,9 @@ +#!/usr/bin/env bash + +# Figure out where the Scala framework is installed +FWDIR="$(cd `dirname $0`; cd ../; pwd)" + +$FWDIR/pyspark pyspark/rdd.py +$FWDIR/pyspark -m doctest pyspark/broadcast.py + +# TODO: in the long-run, it would be nice to use a test runner like `nose`. From 33beba39656fc64984db09a82fc69ca4edcc02d4 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 3 Jan 2013 14:52:21 -0800 Subject: [PATCH 41/85] Change PySpark RDD.take() to not call iterator(). --- core/src/main/scala/spark/api/python/PythonRDD.scala | 4 ++++ python/pyspark/context.py | 1 + python/pyspark/rdd.py | 11 +++++------ 3 files changed, 10 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index cf60d14f03..79d824d494 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -10,6 +10,7 @@ import spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} import spark.broadcast.Broadcast import spark._ import spark.rdd.PipedRDD +import java.util private[spark] class PythonRDD[T: ClassManifest]( @@ -216,6 +217,9 @@ private[spark] object PythonRDD { } file.close() } + + def takePartition[T](rdd: RDD[T], partition: Int): java.util.Iterator[T] = + rdd.context.runJob(rdd, ((x: Iterator[T]) => x), Seq(partition), true).head } private object Pickle { diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 6172d69dcf..4439356c1f 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -21,6 +21,7 @@ class SparkContext(object): jvm = gateway.jvm _readRDDFromPickleFile = jvm.PythonRDD.readRDDFromPickleFile _writeIteratorToPickleFile = jvm.PythonRDD.writeIteratorToPickleFile + _takePartition = jvm.PythonRDD.takePartition def __init__(self, master, jobName, sparkHome=None, pyFiles=None, environment=None, batchSize=1024): diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index cbffb6cc1f..4ba417b2a2 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -328,18 +328,17 @@ class RDD(object): a lot of partitions are required. In that case, use L{collect} to get the whole RDD instead. - >>> sc.parallelize([2, 3, 4, 5, 6]).take(2) + >>> sc.parallelize([2, 3, 4, 5, 6]).cache().take(2) [2, 3] >>> sc.parallelize([2, 3, 4, 5, 6]).take(10) [2, 3, 4, 5, 6] """ items = [] - splits = self._jrdd.splits() - taskContext = self.ctx.jvm.spark.TaskContext(0, 0, 0) - while len(items) < num and splits: - split = splits.pop(0) - iterator = self._jrdd.iterator(split, taskContext) + for partition in range(self._jrdd.splits().size()): + iterator = self.ctx._takePartition(self._jrdd.rdd(), partition) items.extend(self._collect_iterator_through_file(iterator)) + if len(items) >= num: + break return items[:num] def first(self): From 8d57c78c83f74e45ce3c119e2e3915d5eac264e7 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Sat, 5 Jan 2013 10:54:05 -0600 Subject: [PATCH 42/85] Add PairRDDFunctions.keys and values. --- core/src/main/scala/spark/PairRDDFunctions.scala | 10 ++++++++++ core/src/test/scala/spark/ShuffleSuite.scala | 7 +++++++ 2 files changed, 17 insertions(+) diff --git a/core/src/main/scala/spark/PairRDDFunctions.scala b/core/src/main/scala/spark/PairRDDFunctions.scala index 413c944a66..ce48cea903 100644 --- a/core/src/main/scala/spark/PairRDDFunctions.scala +++ b/core/src/main/scala/spark/PairRDDFunctions.scala @@ -615,6 +615,16 @@ class PairRDDFunctions[K: ClassManifest, V: ClassManifest]( writer.cleanup() } + /** + * Return an RDD with the keys of each tuple. + */ + def keys: RDD[K] = self.map(_._1) + + /** + * Return an RDD with the values of each tuple. + */ + def values: RDD[V] = self.map(_._2) + private[spark] def getKeyClass() = implicitly[ClassManifest[K]].erasure private[spark] def getValueClass() = implicitly[ClassManifest[V]].erasure diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index 8170100f1d..5a867016f2 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -216,6 +216,13 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with BeforeAndAfter { // Test that a shuffle on the file works, because this used to be a bug assert(file.map(line => (line, 1)).reduceByKey(_ + _).collect().toList === Nil) } + + test("kesy and values") { + sc = new SparkContext("local", "test") + val rdd = sc.parallelize(Array((1, "a"), (2, "b"))) + assert(rdd.keys.collect().toList === List(1, 2)) + assert(rdd.values.collect().toList === List("a", "b")) + } } object ShuffleSuite { From f4e6b9361ffeec1018d5834f09db9fd86f2ba7bd Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Fri, 4 Jan 2013 22:43:22 -0600 Subject: [PATCH 43/85] Add RDD.collect(PartialFunction). --- core/src/main/scala/spark/RDD.scala | 7 +++++++ core/src/test/scala/spark/RDDSuite.scala | 1 + 2 files changed, 8 insertions(+) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 7e38583391..5163c80134 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -329,6 +329,13 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial */ def toArray(): Array[T] = collect() + /** + * Return an RDD that contains all matching values by applying `f`. + */ + def collect[U: ClassManifest](f: PartialFunction[T, U]): RDD[U] = { + filter(f.isDefinedAt).map(f) + } + /** * Reduces the elements of this RDD using the specified associative binary operator. */ diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 45e6c5f840..872b06fd08 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -35,6 +35,7 @@ class RDDSuite extends FunSuite with BeforeAndAfter { assert(nums.flatMap(x => 1 to x).collect().toList === List(1, 1, 2, 1, 2, 3, 1, 2, 3, 4)) assert(nums.union(nums).collect().toList === List(1, 2, 3, 4, 1, 2, 3, 4)) assert(nums.glom().map(_.toList).collect().toList === List(List(1, 2), List(3, 4))) + assert(nums.collect({ case i if i >= 3 => i.toString }).collect().toList === List("3", "4")) val partitionSums = nums.mapPartitions(iter => Iterator(iter.reduceLeft(_ + _))) assert(partitionSums.collect().toList === List(3, 7)) From 6a0db3b449a829f3e5cdf7229f6ee564268be1df Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Sat, 5 Jan 2013 12:56:17 -0600 Subject: [PATCH 44/85] Fix typo. --- core/src/test/scala/spark/ShuffleSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/spark/ShuffleSuite.scala b/core/src/test/scala/spark/ShuffleSuite.scala index 5a867016f2..bebb8ebe86 100644 --- a/core/src/test/scala/spark/ShuffleSuite.scala +++ b/core/src/test/scala/spark/ShuffleSuite.scala @@ -217,7 +217,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with BeforeAndAfter { assert(file.map(line => (line, 1)).reduceByKey(_ + _).collect().toList === Nil) } - test("kesy and values") { + test("keys and values") { sc = new SparkContext("local", "test") val rdd = sc.parallelize(Array((1, "a"), (2, "b"))) assert(rdd.keys.collect().toList === List(1, 2)) From 1fdb6946b5d076ed0f1b4d2bca2a20b6cd22cbc3 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Sat, 5 Jan 2013 13:07:59 -0600 Subject: [PATCH 45/85] Add RDD.tupleBy. --- core/src/main/scala/spark/RDD.scala | 7 +++++++ core/src/test/scala/spark/RDDSuite.scala | 1 + 2 files changed, 8 insertions(+) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 7e38583391..7aa4b0a173 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -510,6 +510,13 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial .saveAsSequenceFile(path) } + /** + * Tuples the elements of this RDD by applying `f`. + */ + def tupleBy[K](f: T => K): RDD[(K, T)] = { + map(x => (f(x), x)) + } + /** A private method for tests, to look at the contents of each partition */ private[spark] def collectPartitions(): Array[Array[T]] = { sc.runJob(this, (iter: Iterator[T]) => iter.toArray) diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 45e6c5f840..7832884224 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -35,6 +35,7 @@ class RDDSuite extends FunSuite with BeforeAndAfter { assert(nums.flatMap(x => 1 to x).collect().toList === List(1, 1, 2, 1, 2, 3, 1, 2, 3, 4)) assert(nums.union(nums).collect().toList === List(1, 2, 3, 4, 1, 2, 3, 4)) assert(nums.glom().map(_.toList).collect().toList === List(List(1, 2), List(3, 4))) + assert(nums.tupleBy(_.toString).collect().toList === List(("1", 1), ("2", 2), ("3", 3), ("4", 4))) val partitionSums = nums.mapPartitions(iter => Iterator(iter.reduceLeft(_ + _))) assert(partitionSums.collect().toList === List(3, 7)) From 86af64b0a6fde5a6418727a77b43bdfeda1b81cd Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 5 Jan 2013 20:54:08 -0500 Subject: [PATCH 46/85] Fix Accumulators in Java, and add a test for them --- core/src/main/scala/spark/Accumulators.scala | 18 +++++++- core/src/main/scala/spark/SparkContext.scala | 7 +-- .../spark/api/java/JavaSparkContext.scala | 23 ++++++---- core/src/test/scala/spark/JavaAPISuite.java | 44 +++++++++++++++++++ 4 files changed, 79 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/spark/Accumulators.scala b/core/src/main/scala/spark/Accumulators.scala index bacd0ace37..6280f25391 100644 --- a/core/src/main/scala/spark/Accumulators.scala +++ b/core/src/main/scala/spark/Accumulators.scala @@ -38,14 +38,28 @@ class Accumulable[R, T] ( */ def += (term: T) { value_ = param.addAccumulator(value_, term) } + /** + * Add more data to this accumulator / accumulable + * @param term the data to add + */ + def add(term: T) { value_ = param.addAccumulator(value_, term) } + /** * Merge two accumulable objects together - * + * * Normally, a user will not want to use this version, but will instead call `+=`. - * @param term the other Accumulable that will get merged with this + * @param term the other `R` that will get merged with this */ def ++= (term: R) { value_ = param.addInPlace(value_, term)} + /** + * Merge two accumulable objects together + * + * Normally, a user will not want to use this version, but will instead call `add`. + * @param term the other `R` that will get merged with this + */ + def merge(term: R) { value_ = param.addInPlace(value_, term)} + /** * Access the accumulator's current value; only allowed on master. */ diff --git a/core/src/main/scala/spark/SparkContext.scala b/core/src/main/scala/spark/SparkContext.scala index 4fd81bc63b..bbf8272eb3 100644 --- a/core/src/main/scala/spark/SparkContext.scala +++ b/core/src/main/scala/spark/SparkContext.scala @@ -382,11 +382,12 @@ class SparkContext( new Accumulator(initialValue, param) /** - * Create an [[spark.Accumulable]] shared variable, with a `+=` method + * Create an [[spark.Accumulable]] shared variable, to which tasks can add values with `+=`. + * Only the master can access the accumuable's `value`. * @tparam T accumulator type * @tparam R type that can be added to the accumulator */ - def accumulable[T,R](initialValue: T)(implicit param: AccumulableParam[T,R]) = + def accumulable[T, R](initialValue: T)(implicit param: AccumulableParam[T, R]) = new Accumulable(initialValue, param) /** @@ -404,7 +405,7 @@ class SparkContext( * Broadcast a read-only variable to the cluster, returning a [[spark.Broadcast]] object for * reading it in distributed functions. The variable will be sent to each cluster only once. */ - def broadcast[T](value: T) = env.broadcastManager.newBroadcast[T] (value, isLocal) + def broadcast[T](value: T) = env.broadcastManager.newBroadcast[T](value, isLocal) /** * Add a file to be downloaded into the working directory of this Spark job on every node. diff --git a/core/src/main/scala/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/spark/api/java/JavaSparkContext.scala index b7725313c4..bf9ad7a200 100644 --- a/core/src/main/scala/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/spark/api/java/JavaSparkContext.scala @@ -10,7 +10,7 @@ import org.apache.hadoop.mapred.InputFormat import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} -import spark.{Accumulator, AccumulatorParam, RDD, SparkContext} +import spark.{Accumulable, AccumulableParam, Accumulator, AccumulatorParam, RDD, SparkContext} import spark.SparkContext.IntAccumulatorParam import spark.SparkContext.DoubleAccumulatorParam import spark.broadcast.Broadcast @@ -265,25 +265,32 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork /** * Create an [[spark.Accumulator]] integer variable, which tasks can "add" values - * to using the `+=` method. Only the master can access the accumulator's `value`. + * to using the `add` method. Only the master can access the accumulator's `value`. */ - def intAccumulator(initialValue: Int): Accumulator[Int] = - sc.accumulator(initialValue)(IntAccumulatorParam) + def intAccumulator(initialValue: Int): Accumulator[java.lang.Integer] = + sc.accumulator(initialValue)(IntAccumulatorParam).asInstanceOf[Accumulator[java.lang.Integer]] /** * Create an [[spark.Accumulator]] double variable, which tasks can "add" values - * to using the `+=` method. Only the master can access the accumulator's `value`. + * to using the `add` method. Only the master can access the accumulator's `value`. */ - def doubleAccumulator(initialValue: Double): Accumulator[Double] = - sc.accumulator(initialValue)(DoubleAccumulatorParam) + def doubleAccumulator(initialValue: Double): Accumulator[java.lang.Double] = + sc.accumulator(initialValue)(DoubleAccumulatorParam).asInstanceOf[Accumulator[java.lang.Double]] /** * Create an [[spark.Accumulator]] variable of a given type, which tasks can "add" values - * to using the `+=` method. Only the master can access the accumulator's `value`. + * to using the `add` method. Only the master can access the accumulator's `value`. */ def accumulator[T](initialValue: T, accumulatorParam: AccumulatorParam[T]): Accumulator[T] = sc.accumulator(initialValue)(accumulatorParam) + /** + * Create an [[spark.Accumulable]] shared variable of the given type, to which tasks can + * "add" values with `add`. Only the master can access the accumuable's `value`. + */ + def accumulable[T, R](initialValue: T, param: AccumulableParam[T, R]): Accumulable[T, R] = + sc.accumulable(initialValue)(param) + /** * Broadcast a read-only variable to the cluster, returning a [[spark.Broadcast]] object for * reading it in distributed functions. The variable will be sent to each cluster only once. diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index 33d5fc2d89..b99e790093 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -581,4 +581,48 @@ public class JavaAPISuite implements Serializable { JavaPairRDD zipped = rdd.zip(doubles); zipped.count(); } + + @Test + public void accumulators() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); + + final Accumulator intAccum = sc.intAccumulator(10); + rdd.foreach(new VoidFunction() { + public void call(Integer x) { + intAccum.add(x); + } + }); + Assert.assertEquals((Integer) 25, intAccum.value()); + + final Accumulator doubleAccum = sc.doubleAccumulator(10.0); + rdd.foreach(new VoidFunction() { + public void call(Integer x) { + doubleAccum.add((double) x); + } + }); + Assert.assertEquals((Double) 25.0, doubleAccum.value()); + + // Try a custom accumulator type + AccumulatorParam floatAccumulatorParam = new AccumulatorParam() { + public Float addInPlace(Float r, Float t) { + return r + t; + } + + public Float addAccumulator(Float r, Float t) { + return r + t; + } + + public Float zero(Float initialValue) { + return 0.0f; + } + }; + + final Accumulator floatAccum = sc.accumulator((Float) 10.0f, floatAccumulatorParam); + rdd.foreach(new VoidFunction() { + public void call(Integer x) { + floatAccum.add((float) x); + } + }); + Assert.assertEquals((Float) 25.0f, floatAccum.value()); + } } From 0982572519655354b10987de4f68e29b8331bd2a Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 5 Jan 2013 22:11:28 -0500 Subject: [PATCH 47/85] Add methods called just 'accumulator' for int/double in Java API --- .../scala/spark/api/java/JavaSparkContext.scala | 13 +++++++++++++ core/src/test/scala/spark/JavaAPISuite.java | 4 ++-- 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/spark/api/java/JavaSparkContext.scala index bf9ad7a200..88ab2846be 100644 --- a/core/src/main/scala/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/spark/api/java/JavaSparkContext.scala @@ -277,6 +277,19 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork def doubleAccumulator(initialValue: Double): Accumulator[java.lang.Double] = sc.accumulator(initialValue)(DoubleAccumulatorParam).asInstanceOf[Accumulator[java.lang.Double]] + /** + * Create an [[spark.Accumulator]] integer variable, which tasks can "add" values + * to using the `add` method. Only the master can access the accumulator's `value`. + */ + def accumulator(initialValue: Int): Accumulator[java.lang.Integer] = intAccumulator(initialValue) + + /** + * Create an [[spark.Accumulator]] double variable, which tasks can "add" values + * to using the `add` method. Only the master can access the accumulator's `value`. + */ + def accumulator(initialValue: Double): Accumulator[java.lang.Double] = + doubleAccumulator(initialValue) + /** * Create an [[spark.Accumulator]] variable of a given type, which tasks can "add" values * to using the `add` method. Only the master can access the accumulator's `value`. diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index b99e790093..912f8de05d 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -586,7 +586,7 @@ public class JavaAPISuite implements Serializable { public void accumulators() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); - final Accumulator intAccum = sc.intAccumulator(10); + final Accumulator intAccum = sc.accumulator(10); rdd.foreach(new VoidFunction() { public void call(Integer x) { intAccum.add(x); @@ -594,7 +594,7 @@ public class JavaAPISuite implements Serializable { }); Assert.assertEquals((Integer) 25, intAccum.value()); - final Accumulator doubleAccum = sc.doubleAccumulator(10.0); + final Accumulator doubleAccum = sc.accumulator(10.0); rdd.foreach(new VoidFunction() { public void call(Integer x) { doubleAccum.add((double) x); From 8fd3a70c188182105f81f5143ec65e74663582d5 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 5 Jan 2013 22:46:45 -0500 Subject: [PATCH 48/85] Add PairRDD.keys() and values() to Java API --- core/src/main/scala/spark/api/java/JavaPairRDD.scala | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/core/src/main/scala/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/spark/api/java/JavaPairRDD.scala index 5c2be534ff..8ce32e0e2f 100644 --- a/core/src/main/scala/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/spark/api/java/JavaPairRDD.scala @@ -471,6 +471,16 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)])(implicit val kManifest: ClassManif implicit def toOrdered(x: K): Ordered[K] = new KeyOrdering(x) fromRDD(new OrderedRDDFunctions(rdd).sortByKey(ascending)) } + + /** + * Return an RDD with the keys of each tuple. + */ + def keys(): JavaRDD[K] = JavaRDD.fromRDD[K](rdd.map(_._1)) + + /** + * Return an RDD with the values of each tuple. + */ + def values(): JavaRDD[V] = JavaRDD.fromRDD[V](rdd.map(_._2)) } object JavaPairRDD { From 8dc06069fe2330c3ee0fcaaeb0ae6e627a5887c3 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Sun, 6 Jan 2013 15:21:45 -0600 Subject: [PATCH 49/85] Rename RDD.tupleBy to keyBy. --- core/src/main/scala/spark/RDD.scala | 4 ++-- core/src/test/scala/spark/RDDSuite.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/RDD.scala b/core/src/main/scala/spark/RDD.scala index 7aa4b0a173..5ce524c0e7 100644 --- a/core/src/main/scala/spark/RDD.scala +++ b/core/src/main/scala/spark/RDD.scala @@ -511,9 +511,9 @@ abstract class RDD[T: ClassManifest](@transient sc: SparkContext) extends Serial } /** - * Tuples the elements of this RDD by applying `f`. + * Creates tuples of the elements in this RDD by applying `f`. */ - def tupleBy[K](f: T => K): RDD[(K, T)] = { + def keyBy[K](f: T => K): RDD[(K, T)] = { map(x => (f(x), x)) } diff --git a/core/src/test/scala/spark/RDDSuite.scala b/core/src/test/scala/spark/RDDSuite.scala index 7832884224..77bff8aba1 100644 --- a/core/src/test/scala/spark/RDDSuite.scala +++ b/core/src/test/scala/spark/RDDSuite.scala @@ -35,7 +35,7 @@ class RDDSuite extends FunSuite with BeforeAndAfter { assert(nums.flatMap(x => 1 to x).collect().toList === List(1, 1, 2, 1, 2, 3, 1, 2, 3, 4)) assert(nums.union(nums).collect().toList === List(1, 2, 3, 4, 1, 2, 3, 4)) assert(nums.glom().map(_.toList).collect().toList === List(List(1, 2), List(3, 4))) - assert(nums.tupleBy(_.toString).collect().toList === List(("1", 1), ("2", 2), ("3", 3), ("4", 4))) + assert(nums.keyBy(_.toString).collect().toList === List(("1", 1), ("2", 2), ("3", 3), ("4", 4))) val partitionSums = nums.mapPartitions(iter => Iterator(iter.reduceLeft(_ + _))) assert(partitionSums.collect().toList === List(3, 7)) From 9c32f300fb4151a2b563bf3d2e46469722e016e1 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 7 Jan 2013 16:50:23 -0500 Subject: [PATCH 50/85] Add Accumulable.setValue for easier use in Java --- core/src/main/scala/spark/Accumulators.scala | 20 +++++++++++++++----- core/src/test/scala/spark/JavaAPISuite.java | 4 ++++ 2 files changed, 19 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/spark/Accumulators.scala b/core/src/main/scala/spark/Accumulators.scala index 6280f25391..b644aba5f8 100644 --- a/core/src/main/scala/spark/Accumulators.scala +++ b/core/src/main/scala/spark/Accumulators.scala @@ -63,9 +63,12 @@ class Accumulable[R, T] ( /** * Access the accumulator's current value; only allowed on master. */ - def value = { - if (!deserialized) value_ - else throw new UnsupportedOperationException("Can't read accumulator value in task") + def value: R = { + if (!deserialized) { + value_ + } else { + throw new UnsupportedOperationException("Can't read accumulator value in task") + } } /** @@ -82,10 +85,17 @@ class Accumulable[R, T] ( /** * Set the accumulator's value; only allowed on master. */ - def value_= (r: R) { - if (!deserialized) value_ = r + def value_= (newValue: R) { + if (!deserialized) value_ = newValue else throw new UnsupportedOperationException("Can't assign accumulator value in task") } + + /** + * Set the accumulator's value; only allowed on master + */ + def setValue(newValue: R) { + this.value = newValue + } // Called by Java when deserializing an object private def readObject(in: ObjectInputStream) { diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index 912f8de05d..0817d1146c 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -624,5 +624,9 @@ public class JavaAPISuite implements Serializable { } }); Assert.assertEquals((Float) 25.0f, floatAccum.value()); + + // Test the setValue method + floatAccum.setValue(5.0f); + Assert.assertEquals((Float) 5.0f, floatAccum.value()); } } From f8d579a0c05b7d29b59e541b483ded471d14ec17 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 27 Dec 2012 13:30:07 -0800 Subject: [PATCH 51/85] Remove dependencies on sun jvm classes. Instead use reflection to infer HotSpot options and total physical memory size --- core/src/main/scala/spark/SizeEstimator.scala | 13 ++++++++--- .../spark/deploy/worker/WorkerArguments.scala | 22 ++++++++++++++++--- 2 files changed, 29 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/SizeEstimator.scala b/core/src/main/scala/spark/SizeEstimator.scala index 7c3e8640e9..d4e1157250 100644 --- a/core/src/main/scala/spark/SizeEstimator.scala +++ b/core/src/main/scala/spark/SizeEstimator.scala @@ -9,7 +9,6 @@ import java.util.Random import javax.management.MBeanServer import java.lang.management.ManagementFactory -import com.sun.management.HotSpotDiagnosticMXBean import scala.collection.mutable.ArrayBuffer @@ -76,12 +75,20 @@ private[spark] object SizeEstimator extends Logging { if (System.getProperty("spark.test.useCompressedOops") != null) { return System.getProperty("spark.test.useCompressedOops").toBoolean } + try { val hotSpotMBeanName = "com.sun.management:type=HotSpotDiagnostic" val server = ManagementFactory.getPlatformMBeanServer() + + // NOTE: This should throw an exception in non-Sun JVMs + val hotSpotMBeanClass = Class.forName("com.sun.management.HotSpotDiagnosticMXBean") + val getVMMethod = hotSpotMBeanClass.getDeclaredMethod("getVMOption", + Class.forName("java.lang.String")) + val bean = ManagementFactory.newPlatformMXBeanProxy(server, - hotSpotMBeanName, classOf[HotSpotDiagnosticMXBean]) - return bean.getVMOption("UseCompressedOops").getValue.toBoolean + hotSpotMBeanName, hotSpotMBeanClass) + // TODO: We could use reflection on the VMOption returned ? + return getVMMethod.invoke(bean, "UseCompressedOops").toString.contains("true") } catch { case e: Exception => { // Guess whether they've enabled UseCompressedOops based on whether maxMemory < 32 GB diff --git a/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala index 340920025b..37524a7c82 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerArguments.scala @@ -104,9 +104,25 @@ private[spark] class WorkerArguments(args: Array[String]) { } def inferDefaultMemory(): Int = { - val bean = ManagementFactory.getOperatingSystemMXBean - .asInstanceOf[com.sun.management.OperatingSystemMXBean] - val totalMb = (bean.getTotalPhysicalMemorySize / 1024 / 1024).toInt + val ibmVendor = System.getProperty("java.vendor").contains("IBM") + var totalMb = 0 + try { + val bean = ManagementFactory.getOperatingSystemMXBean() + if (ibmVendor) { + val beanClass = Class.forName("com.ibm.lang.management.OperatingSystemMXBean") + val method = beanClass.getDeclaredMethod("getTotalPhysicalMemory") + totalMb = (method.invoke(bean).asInstanceOf[Long] / 1024 / 1024).toInt + } else { + val beanClass = Class.forName("com.sun.management.OperatingSystemMXBean") + val method = beanClass.getDeclaredMethod("getTotalPhysicalMemorySize") + totalMb = (method.invoke(bean).asInstanceOf[Long] / 1024 / 1024).toInt + } + } catch { + case e: Exception => { + totalMb = 2*1024 + System.out.println("Failed to get total physical memory. Using " + totalMb + " MB") + } + } // Leave out 1 GB for the operating system, but don't return a negative memory size math.max(totalMb - 1024, 512) } From aed368a970bbaee4bdf297ba3f6f1b0fa131452c Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sat, 29 Dec 2012 16:23:43 -0800 Subject: [PATCH 52/85] Update Hadoop dependency to 1.0.3 as 0.20 has Sun specific dependencies. Also fix SequenceFileRDDFunctions to pick the right type conversion across Hadoop versions --- core/src/main/scala/spark/SequenceFileRDDFunctions.scala | 8 +++++++- project/SparkBuild.scala | 2 +- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala index a34aee69c1..6b4a11d6d3 100644 --- a/core/src/main/scala/spark/SequenceFileRDDFunctions.scala +++ b/core/src/main/scala/spark/SequenceFileRDDFunctions.scala @@ -42,7 +42,13 @@ class SequenceFileRDDFunctions[K <% Writable: ClassManifest, V <% Writable : Cla if (classOf[Writable].isAssignableFrom(classManifest[T].erasure)) { classManifest[T].erasure } else { - implicitly[T => Writable].getClass.getMethods()(0).getReturnType + // We get the type of the Writable class by looking at the apply method which converts + // from T to Writable. Since we have two apply methods we filter out the one which + // is of the form "java.lang.Object apply(java.lang.Object)" + implicitly[T => Writable].getClass.getDeclaredMethods().filter( + m => m.getReturnType().toString != "java.lang.Object" && + m.getName() == "apply")(0).getReturnType + } // TODO: use something like WritableConverter to avoid reflection } diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 842d0fa96b..7c7c33131a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -10,7 +10,7 @@ import twirl.sbt.TwirlPlugin._ object SparkBuild extends Build { // Hadoop version to build against. For example, "0.20.2", "0.20.205.0", or // "1.0.3" for Apache releases, or "0.20.2-cdh3u5" for Cloudera Hadoop. - val HADOOP_VERSION = "0.20.205.0" + val HADOOP_VERSION = "1.0.3" val HADOOP_MAJOR_VERSION = "1" // For Hadoop 2 versions such as "2.0.0-mr1-cdh4.1.1", set the HADOOP_MAJOR_VERSION to "2" From 77d751731ccd06e161e3ef10540f8165d964282f Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sat, 29 Dec 2012 18:28:00 -0800 Subject: [PATCH 53/85] Remove unused BoundedMemoryCache file and associated test case. --- .../main/scala/spark/BoundedMemoryCache.scala | 118 ------------------ .../scala/spark/BoundedMemoryCacheSuite.scala | 58 --------- 2 files changed, 176 deletions(-) delete mode 100644 core/src/main/scala/spark/BoundedMemoryCache.scala delete mode 100644 core/src/test/scala/spark/BoundedMemoryCacheSuite.scala diff --git a/core/src/main/scala/spark/BoundedMemoryCache.scala b/core/src/main/scala/spark/BoundedMemoryCache.scala deleted file mode 100644 index e8392a194f..0000000000 --- a/core/src/main/scala/spark/BoundedMemoryCache.scala +++ /dev/null @@ -1,118 +0,0 @@ -package spark - -import java.util.LinkedHashMap - -/** - * An implementation of Cache that estimates the sizes of its entries and attempts to limit its - * total memory usage to a fraction of the JVM heap. Objects' sizes are estimated using - * SizeEstimator, which has limitations; most notably, we will overestimate total memory used if - * some cache entries have pointers to a shared object. Nonetheless, this Cache should work well - * when most of the space is used by arrays of primitives or of simple classes. - */ -private[spark] class BoundedMemoryCache(maxBytes: Long) extends Cache with Logging { - logInfo("BoundedMemoryCache.maxBytes = " + maxBytes) - - def this() { - this(BoundedMemoryCache.getMaxBytes) - } - - private var currentBytes = 0L - private val map = new LinkedHashMap[(Any, Int), Entry](32, 0.75f, true) - - override def get(datasetId: Any, partition: Int): Any = { - synchronized { - val entry = map.get((datasetId, partition)) - if (entry != null) { - entry.value - } else { - null - } - } - } - - override def put(datasetId: Any, partition: Int, value: Any): CachePutResponse = { - val key = (datasetId, partition) - logInfo("Asked to add key " + key) - val size = estimateValueSize(key, value) - synchronized { - if (size > getCapacity) { - return CachePutFailure() - } else if (ensureFreeSpace(datasetId, size)) { - logInfo("Adding key " + key) - map.put(key, new Entry(value, size)) - currentBytes += size - logInfo("Number of entries is now " + map.size) - return CachePutSuccess(size) - } else { - logInfo("Didn't add key " + key + " because we would have evicted part of same dataset") - return CachePutFailure() - } - } - } - - override def getCapacity: Long = maxBytes - - /** - * Estimate sizeOf 'value' - */ - private def estimateValueSize(key: (Any, Int), value: Any) = { - val startTime = System.currentTimeMillis - val size = SizeEstimator.estimate(value.asInstanceOf[AnyRef]) - val timeTaken = System.currentTimeMillis - startTime - logInfo("Estimated size for key %s is %d".format(key, size)) - logInfo("Size estimation for key %s took %d ms".format(key, timeTaken)) - size - } - - /** - * Remove least recently used entries from the map until at least space bytes are free, in order - * to make space for a partition from the given dataset ID. If this cannot be done without - * evicting other data from the same dataset, returns false; otherwise, returns true. Assumes - * that a lock is held on the BoundedMemoryCache. - */ - private def ensureFreeSpace(datasetId: Any, space: Long): Boolean = { - logInfo("ensureFreeSpace(%s, %d) called with curBytes=%d, maxBytes=%d".format( - datasetId, space, currentBytes, maxBytes)) - val iter = map.entrySet.iterator // Will give entries in LRU order - while (maxBytes - currentBytes < space && iter.hasNext) { - val mapEntry = iter.next() - val (entryDatasetId, entryPartition) = mapEntry.getKey - if (entryDatasetId == datasetId) { - // Cannot make space without removing part of the same dataset, or a more recently used one - return false - } - reportEntryDropped(entryDatasetId, entryPartition, mapEntry.getValue) - currentBytes -= mapEntry.getValue.size - iter.remove() - } - return true - } - - protected def reportEntryDropped(datasetId: Any, partition: Int, entry: Entry) { - logInfo("Dropping key (%s, %d) of size %d to make space".format(datasetId, partition, entry.size)) - // TODO: remove BoundedMemoryCache - - val (keySpaceId, innerDatasetId) = datasetId.asInstanceOf[(Any, Any)] - innerDatasetId match { - case rddId: Int => - SparkEnv.get.cacheTracker.dropEntry(rddId, partition) - case broadcastUUID: java.util.UUID => - // TODO: Maybe something should be done if the broadcasted variable falls out of cache - case _ => - } - } -} - -// An entry in our map; stores a cached object and its size in bytes -private[spark] case class Entry(value: Any, size: Long) - -private[spark] object BoundedMemoryCache { - /** - * Get maximum cache capacity from system configuration - */ - def getMaxBytes: Long = { - val memoryFractionToUse = System.getProperty("spark.boundedMemoryCache.memoryFraction", "0.66").toDouble - (Runtime.getRuntime.maxMemory * memoryFractionToUse).toLong - } -} - diff --git a/core/src/test/scala/spark/BoundedMemoryCacheSuite.scala b/core/src/test/scala/spark/BoundedMemoryCacheSuite.scala deleted file mode 100644 index 37cafd1e8e..0000000000 --- a/core/src/test/scala/spark/BoundedMemoryCacheSuite.scala +++ /dev/null @@ -1,58 +0,0 @@ -package spark - -import org.scalatest.FunSuite -import org.scalatest.PrivateMethodTester -import org.scalatest.matchers.ShouldMatchers - -// TODO: Replace this with a test of MemoryStore -class BoundedMemoryCacheSuite extends FunSuite with PrivateMethodTester with ShouldMatchers { - test("constructor test") { - val cache = new BoundedMemoryCache(60) - expect(60)(cache.getCapacity) - } - - test("caching") { - // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case - val oldArch = System.setProperty("os.arch", "amd64") - val oldOops = System.setProperty("spark.test.useCompressedOops", "true") - val initialize = PrivateMethod[Unit]('initialize) - SizeEstimator invokePrivate initialize() - - val cache = new BoundedMemoryCache(60) { - //TODO sorry about this, but there is not better way how to skip 'cacheTracker.dropEntry' - override protected def reportEntryDropped(datasetId: Any, partition: Int, entry: Entry) { - logInfo("Dropping key (%s, %d) of size %d to make space".format(datasetId, partition, entry.size)) - } - } - - // NOTE: The String class definition changed in JDK 7 to exclude the int fields count and length - // This means that the size of strings will be lesser by 8 bytes in JDK 7 compared to JDK 6. - // http://mail.openjdk.java.net/pipermail/core-libs-dev/2012-May/010257.html - // Work around to check for either. - - //should be OK - cache.put("1", 0, "Meh") should (equal (CachePutSuccess(56)) or equal (CachePutSuccess(48))) - - //we cannot add this to cache (there is not enough space in cache) & we cannot evict the only value from - //cache because it's from the same dataset - expect(CachePutFailure())(cache.put("1", 1, "Meh")) - - //should be OK, dataset '1' can be evicted from cache - cache.put("2", 0, "Meh") should (equal (CachePutSuccess(56)) or equal (CachePutSuccess(48))) - - //should fail, cache should obey it's capacity - expect(CachePutFailure())(cache.put("3", 0, "Very_long_and_useless_string")) - - if (oldArch != null) { - System.setProperty("os.arch", oldArch) - } else { - System.clearProperty("os.arch") - } - - if (oldOops != null) { - System.setProperty("spark.test.useCompressedOops", oldOops) - } else { - System.clearProperty("spark.test.useCompressedOops") - } - } -} From 55c66d365f76f3e5ecc6b850ba81c84b320f6772 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 7 Jan 2013 15:19:33 -0800 Subject: [PATCH 54/85] Use a dummy string class in Size Estimator tests to make it resistant to jdk versions --- .../test/scala/spark/SizeEstimatorSuite.scala | 33 ++++++++++++------- 1 file changed, 21 insertions(+), 12 deletions(-) diff --git a/core/src/test/scala/spark/SizeEstimatorSuite.scala b/core/src/test/scala/spark/SizeEstimatorSuite.scala index 17f366212b..bf3b2e1eed 100644 --- a/core/src/test/scala/spark/SizeEstimatorSuite.scala +++ b/core/src/test/scala/spark/SizeEstimatorSuite.scala @@ -20,6 +20,15 @@ class DummyClass4(val d: DummyClass3) { val x: Int = 0 } +object DummyString { + def apply(str: String) : DummyString = new DummyString(str.toArray) +} +class DummyString(val arr: Array[Char]) { + override val hashCode: Int = 0 + // JDK-7 has an extra hash32 field http://hg.openjdk.java.net/jdk7u/jdk7u6/jdk/rev/11987e85555f + @transient val hash32: Int = 0 +} + class SizeEstimatorSuite extends FunSuite with BeforeAndAfterAll with PrivateMethodTester with ShouldMatchers { @@ -50,10 +59,10 @@ class SizeEstimatorSuite // http://mail.openjdk.java.net/pipermail/core-libs-dev/2012-May/010257.html // Work around to check for either. test("strings") { - SizeEstimator.estimate("") should (equal (48) or equal (40)) - SizeEstimator.estimate("a") should (equal (56) or equal (48)) - SizeEstimator.estimate("ab") should (equal (56) or equal (48)) - SizeEstimator.estimate("abcdefgh") should (equal(64) or equal(56)) + SizeEstimator.estimate(DummyString("")) should (equal (48) or equal (40)) + SizeEstimator.estimate(DummyString("a")) should (equal (56) or equal (48)) + SizeEstimator.estimate(DummyString("ab")) should (equal (56) or equal (48)) + SizeEstimator.estimate(DummyString("abcdefgh")) should (equal(64) or equal(56)) } test("primitive arrays") { @@ -105,10 +114,10 @@ class SizeEstimatorSuite val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() - expect(40)(SizeEstimator.estimate("")) - expect(48)(SizeEstimator.estimate("a")) - expect(48)(SizeEstimator.estimate("ab")) - expect(56)(SizeEstimator.estimate("abcdefgh")) + expect(40)(SizeEstimator.estimate(DummyString(""))) + expect(48)(SizeEstimator.estimate(DummyString("a"))) + expect(48)(SizeEstimator.estimate(DummyString("ab"))) + expect(56)(SizeEstimator.estimate(DummyString("abcdefgh"))) resetOrClear("os.arch", arch) } @@ -124,10 +133,10 @@ class SizeEstimatorSuite val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() - SizeEstimator.estimate("") should (equal (64) or equal (56)) - SizeEstimator.estimate("a") should (equal (72) or equal (64)) - SizeEstimator.estimate("ab") should (equal (72) or equal (64)) - SizeEstimator.estimate("abcdefgh") should (equal (80) or equal (72)) + SizeEstimator.estimate(DummyString("")) should (equal (64) or equal (56)) + SizeEstimator.estimate(DummyString("a")) should (equal (72) or equal (64)) + SizeEstimator.estimate(DummyString("ab")) should (equal (72) or equal (64)) + SizeEstimator.estimate(DummyString("abcdefgh")) should (equal (80) or equal (72)) resetOrClear("os.arch", arch) resetOrClear("spark.test.useCompressedOops", oops) From fb3d4d5e85cd4b094411bb08a32ab50cc62dc151 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 7 Jan 2013 16:46:06 -0800 Subject: [PATCH 55/85] Make default hadoop version 1.0.3 in pom.xml --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index b33cee26b8..fe5b1d0ee4 100644 --- a/pom.xml +++ b/pom.xml @@ -489,7 +489,7 @@ org.apache.hadoop hadoop-core - 0.20.205.0 + 1.0.3 From b1336e2fe458b92dcf60dcd249c41c7bdcc8be6d Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 7 Jan 2013 17:00:32 -0800 Subject: [PATCH 56/85] Update expected size of strings to match our dummy string class --- .../test/scala/spark/SizeEstimatorSuite.scala | 31 ++++++++----------- 1 file changed, 13 insertions(+), 18 deletions(-) diff --git a/core/src/test/scala/spark/SizeEstimatorSuite.scala b/core/src/test/scala/spark/SizeEstimatorSuite.scala index bf3b2e1eed..e235ef2f67 100644 --- a/core/src/test/scala/spark/SizeEstimatorSuite.scala +++ b/core/src/test/scala/spark/SizeEstimatorSuite.scala @@ -3,7 +3,6 @@ package spark import org.scalatest.FunSuite import org.scalatest.BeforeAndAfterAll import org.scalatest.PrivateMethodTester -import org.scalatest.matchers.ShouldMatchers class DummyClass1 {} @@ -30,7 +29,7 @@ class DummyString(val arr: Array[Char]) { } class SizeEstimatorSuite - extends FunSuite with BeforeAndAfterAll with PrivateMethodTester with ShouldMatchers { + extends FunSuite with BeforeAndAfterAll with PrivateMethodTester { var oldArch: String = _ var oldOops: String = _ @@ -54,15 +53,13 @@ class SizeEstimatorSuite expect(48)(SizeEstimator.estimate(new DummyClass4(new DummyClass3))) } - // NOTE: The String class definition changed in JDK 7 to exclude the int fields count and length. - // This means that the size of strings will be lesser by 8 bytes in JDK 7 compared to JDK 6. - // http://mail.openjdk.java.net/pipermail/core-libs-dev/2012-May/010257.html - // Work around to check for either. + // NOTE: The String class definition varies across JDK versions (1.6 vs. 1.7) and vendors + // (Sun vs IBM). Use a DummyString class to make tests deterministic. test("strings") { - SizeEstimator.estimate(DummyString("")) should (equal (48) or equal (40)) - SizeEstimator.estimate(DummyString("a")) should (equal (56) or equal (48)) - SizeEstimator.estimate(DummyString("ab")) should (equal (56) or equal (48)) - SizeEstimator.estimate(DummyString("abcdefgh")) should (equal(64) or equal(56)) + expect(40)(SizeEstimator.estimate(DummyString(""))) + expect(48)(SizeEstimator.estimate(DummyString("a"))) + expect(48)(SizeEstimator.estimate(DummyString("ab"))) + expect(56)(SizeEstimator.estimate(DummyString("abcdefgh"))) } test("primitive arrays") { @@ -122,10 +119,8 @@ class SizeEstimatorSuite resetOrClear("os.arch", arch) } - // NOTE: The String class definition changed in JDK 7 to exclude the int fields count and length. - // This means that the size of strings will be lesser by 8 bytes in JDK 7 compared to JDK 6. - // http://mail.openjdk.java.net/pipermail/core-libs-dev/2012-May/010257.html - // Work around to check for either. + // NOTE: The String class definition varies across JDK versions (1.6 vs. 1.7) and vendors + // (Sun vs IBM). Use a DummyString class to make tests deterministic. test("64-bit arch with no compressed oops") { val arch = System.setProperty("os.arch", "amd64") val oops = System.setProperty("spark.test.useCompressedOops", "false") @@ -133,10 +128,10 @@ class SizeEstimatorSuite val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() - SizeEstimator.estimate(DummyString("")) should (equal (64) or equal (56)) - SizeEstimator.estimate(DummyString("a")) should (equal (72) or equal (64)) - SizeEstimator.estimate(DummyString("ab")) should (equal (72) or equal (64)) - SizeEstimator.estimate(DummyString("abcdefgh")) should (equal (80) or equal (72)) + expect(56)(SizeEstimator.estimate(DummyString(""))) + expect(64)(SizeEstimator.estimate(DummyString("a"))) + expect(64)(SizeEstimator.estimate(DummyString("ab"))) + expect(72)(SizeEstimator.estimate(DummyString("abcdefgh"))) resetOrClear("os.arch", arch) resetOrClear("spark.test.useCompressedOops", oops) From 4bbe07e5ece81fa874d2412bcc165179313a7619 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 7 Jan 2013 17:46:22 -0800 Subject: [PATCH 57/85] Activate hadoop1 profile by default for maven builds --- bagel/pom.xml | 3 +++ core/pom.xml | 5 ++++- examples/pom.xml | 3 +++ pom.xml | 3 +++ repl-bin/pom.xml | 3 +++ repl/pom.xml | 3 +++ 6 files changed, 19 insertions(+), 1 deletion(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index a8256a6e8b..4ca643bbb7 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -45,6 +45,9 @@ hadoop1 + + true + org.spark-project diff --git a/core/pom.xml b/core/pom.xml index ae52c20657..cd789a7db0 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -159,6 +159,9 @@ hadoop1 + + true + org.apache.hadoop @@ -267,4 +270,4 @@ - \ No newline at end of file + diff --git a/examples/pom.xml b/examples/pom.xml index 782c026d73..9e638c8284 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -45,6 +45,9 @@ hadoop1 + + true + org.spark-project diff --git a/pom.xml b/pom.xml index fe5b1d0ee4..0e2d93c170 100644 --- a/pom.xml +++ b/pom.xml @@ -481,6 +481,9 @@ hadoop1 + + true + 1 diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 0667b71cc7..aa9895eda2 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -70,6 +70,9 @@ hadoop1 + + true + hadoop1 diff --git a/repl/pom.xml b/repl/pom.xml index 114e3e9932..ba7a051310 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -72,6 +72,9 @@ hadoop1 + + true + hadoop1 From c41042c816c2d6299aa7d93529b7c39db5d5c03a Mon Sep 17 00:00:00 2001 From: Mikhail Bautin Date: Wed, 26 Dec 2012 15:52:51 -0800 Subject: [PATCH 58/85] Log preferred hosts --- .../main/scala/spark/scheduler/cluster/TaskSetManager.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index cf4aae03a7..dda7a6c64a 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -201,7 +201,9 @@ private[spark] class TaskSetManager( val taskId = sched.newTaskId() // Figure out whether this should count as a preferred launch val preferred = isPreferredLocation(task, host) - val prefStr = if (preferred) "preferred" else "non-preferred" + val prefStr = if (preferred) "preferred" else + "non-preferred, not one of " + + task.preferredLocations.mkString(", ") logInfo("Starting task %s:%d as TID %s on slave %s: %s (%s)".format( taskSet.id, index, taskId, slaveId, host, prefStr)) // Do various bookkeeping From 4725b0f6439337c7a0f5f6fc7034c6f6b9488ae9 Mon Sep 17 00:00:00 2001 From: Mikhail Bautin Date: Mon, 7 Jan 2013 20:07:08 -0800 Subject: [PATCH 59/85] Fixing if/else coding style for preferred hosts logging --- .../main/scala/spark/scheduler/cluster/TaskSetManager.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index dda7a6c64a..a842afcdeb 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -201,9 +201,8 @@ private[spark] class TaskSetManager( val taskId = sched.newTaskId() // Figure out whether this should count as a preferred launch val preferred = isPreferredLocation(task, host) - val prefStr = if (preferred) "preferred" else - "non-preferred, not one of " + - task.preferredLocations.mkString(", ") + val prefStr = if (preferred) "preferred" + else "non-preferred, not one of " + task.preferredLocations.mkString(", ") logInfo("Starting task %s:%d as TID %s on slave %s: %s (%s)".format( taskSet.id, index, taskId, slaveId, host, prefStr)) // Do various bookkeeping From f7adb382ace7f54c5093bf90574b3f9dd0d35534 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Tue, 8 Jan 2013 03:19:43 -0800 Subject: [PATCH 60/85] Activate hadoop1 if property hadoop is missing. hadoop2 can be activated now by using -Dhadoop -Phadoop2. --- bagel/pom.xml | 4 +++- core/pom.xml | 4 +++- examples/pom.xml | 4 +++- pom.xml | 4 +++- repl-bin/pom.xml | 4 +++- repl/pom.xml | 4 +++- 6 files changed, 18 insertions(+), 6 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index 4ca643bbb7..85b2077026 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -46,7 +46,9 @@ hadoop1 - true + + !hadoop + diff --git a/core/pom.xml b/core/pom.xml index cd789a7db0..005d8fe498 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -160,7 +160,9 @@ hadoop1 - true + + !hadoop + diff --git a/examples/pom.xml b/examples/pom.xml index 9e638c8284..3f738a3f8c 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -46,7 +46,9 @@ hadoop1 - true + + !hadoop + diff --git a/pom.xml b/pom.xml index 0e2d93c170..ea5b9c9d05 100644 --- a/pom.xml +++ b/pom.xml @@ -482,7 +482,9 @@ hadoop1 - true + + !hadoop + 1 diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index aa9895eda2..fecb01f3cd 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -71,7 +71,9 @@ hadoop1 - true + + !hadoop + hadoop1 diff --git a/repl/pom.xml b/repl/pom.xml index ba7a051310..04b2c35beb 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -73,7 +73,9 @@ hadoop1 - true + + !hadoop + hadoop1 From e4cb72da8a5428c6b9097e92ddbdf4ceee087b85 Mon Sep 17 00:00:00 2001 From: shane-huang Date: Tue, 8 Jan 2013 22:40:58 +0800 Subject: [PATCH 61/85] Fix an issue in ConnectionManager where sendingMessage may create too many unnecessary SendingConnections. --- .../main/scala/spark/network/Connection.scala | 7 +++++-- .../spark/network/ConnectionManager.scala | 17 +++++++++-------- .../spark/network/ConnectionManagerTest.scala | 18 +++++++++--------- 3 files changed, 23 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/spark/network/Connection.scala b/core/src/main/scala/spark/network/Connection.scala index 80262ab7b4..95096fd0ba 100644 --- a/core/src/main/scala/spark/network/Connection.scala +++ b/core/src/main/scala/spark/network/Connection.scala @@ -135,8 +135,11 @@ extends Connection(SocketChannel.open, selector_) { val chunk = message.getChunkForSending(defaultChunkSize) if (chunk.isDefined) { messages += message // this is probably incorrect, it wont work as fifo - if (!message.started) logDebug("Starting to send [" + message + "]") - message.started = true + if (!message.started) { + logDebug("Starting to send [" + message + "]") + message.started = true + message.startTime = System.currentTimeMillis + } return chunk } else { /*logInfo("Finished sending [" + message + "] to [" + remoteConnectionManagerId + "]")*/ diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala index 642fa4b525..e7bd2d3bbd 100644 --- a/core/src/main/scala/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/spark/network/ConnectionManager.scala @@ -43,12 +43,12 @@ private[spark] class ConnectionManager(port: Int) extends Logging { } val selector = SelectorProvider.provider.openSelector() - val handleMessageExecutor = Executors.newFixedThreadPool(4) + val handleMessageExecutor = Executors.newFixedThreadPool(20) val serverChannel = ServerSocketChannel.open() val connectionsByKey = new HashMap[SelectionKey, Connection] with SynchronizedMap[SelectionKey, Connection] val connectionsById = new HashMap[ConnectionManagerId, SendingConnection] with SynchronizedMap[ConnectionManagerId, SendingConnection] val messageStatuses = new HashMap[Int, MessageStatus] - val connectionRequests = new SynchronizedQueue[SendingConnection] + val connectionRequests = new HashMap[ConnectionManagerId, SendingConnection] with SynchronizedMap[ConnectionManagerId, SendingConnection] val keyInterestChangeRequests = new SynchronizedQueue[(SelectionKey, Int)] val sendMessageRequests = new Queue[(Message, SendingConnection)] @@ -78,11 +78,12 @@ private[spark] class ConnectionManager(port: Int) extends Logging { def run() { try { - while(!selectorThread.isInterrupted) { - while(!connectionRequests.isEmpty) { - val sendingConnection = connectionRequests.dequeue + while(!selectorThread.isInterrupted) { + for( (connectionManagerId, sendingConnection) <- connectionRequests) { + //val sendingConnection = connectionRequests.dequeue sendingConnection.connect() addConnection(sendingConnection) + connectionRequests -= connectionManagerId } sendMessageRequests.synchronized { while(!sendMessageRequests.isEmpty) { @@ -300,8 +301,7 @@ private[spark] class ConnectionManager(port: Int) extends Logging { private def sendMessage(connectionManagerId: ConnectionManagerId, message: Message) { def startNewConnection(): SendingConnection = { val inetSocketAddress = new InetSocketAddress(connectionManagerId.host, connectionManagerId.port) - val newConnection = new SendingConnection(inetSocketAddress, selector) - connectionRequests += newConnection + val newConnection = connectionRequests.getOrElseUpdate(connectionManagerId, new SendingConnection(inetSocketAddress, selector)) newConnection } val lookupKey = ConnectionManagerId.fromSocketAddress(connectionManagerId.toSocketAddress) @@ -465,7 +465,7 @@ private[spark] object ConnectionManager { val bufferMessage = Message.createBufferMessage(buffer.duplicate) manager.sendMessageReliably(manager.id, bufferMessage) }).foreach(f => { - val g = Await.result(f, 1 second) + val g = Await.result(f, 10 second) if (!g.isDefined) println("Failed") }) val finishTime = System.currentTimeMillis @@ -473,6 +473,7 @@ private[spark] object ConnectionManager { val mb = size * count / 1024.0 / 1024.0 val ms = finishTime - startTime val tput = mb * 1000.0 / ms + println("Sent " + mb + " MB in " + ms + " ms (" + tput + " MB/s)") println("--------------------------") println() } diff --git a/core/src/main/scala/spark/network/ConnectionManagerTest.scala b/core/src/main/scala/spark/network/ConnectionManagerTest.scala index 47ceaf3c07..0e79c518e0 100644 --- a/core/src/main/scala/spark/network/ConnectionManagerTest.scala +++ b/core/src/main/scala/spark/network/ConnectionManagerTest.scala @@ -13,8 +13,8 @@ import akka.util.duration._ private[spark] object ConnectionManagerTest extends Logging{ def main(args: Array[String]) { - if (args.length < 2) { - println("Usage: ConnectionManagerTest ") + if (args.length < 5) { + println("Usage: ConnectionManagerTest ") System.exit(1) } @@ -29,16 +29,16 @@ private[spark] object ConnectionManagerTest extends Logging{ /*println("Slaves")*/ /*slaves.foreach(println)*/ - - val slaveConnManagerIds = sc.parallelize(0 until slaves.length, slaves.length).map( + val tasknum = args(2).toInt + val slaveConnManagerIds = sc.parallelize(0 until tasknum, tasknum).map( i => SparkEnv.get.connectionManager.id).collect() println("\nSlave ConnectionManagerIds") slaveConnManagerIds.foreach(println) println - val count = 10 + val count = args(4).toInt (0 until count).foreach(i => { - val resultStrs = sc.parallelize(0 until slaves.length, slaves.length).map(i => { + val resultStrs = sc.parallelize(0 until tasknum, tasknum).map(i => { val connManager = SparkEnv.get.connectionManager val thisConnManagerId = connManager.id connManager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { @@ -46,7 +46,7 @@ private[spark] object ConnectionManagerTest extends Logging{ None }) - val size = 100 * 1024 * 1024 + val size = (args(3).toInt) * 1024 * 1024 val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) buffer.flip @@ -56,13 +56,13 @@ private[spark] object ConnectionManagerTest extends Logging{ logInfo("Sending [" + bufferMessage + "] to [" + slaveConnManagerId + "]") connManager.sendMessageReliably(slaveConnManagerId, bufferMessage) }) - val results = futures.map(f => Await.result(f, 1.second)) + val results = futures.map(f => Await.result(f, 999.second)) val finishTime = System.currentTimeMillis Thread.sleep(5000) val mb = size * results.size / 1024.0 / 1024.0 val ms = finishTime - startTime - val resultStr = "Sent " + mb + " MB in " + ms + " ms at " + (mb / ms * 1000.0) + " MB/s" + val resultStr = thisConnManagerId + " Sent " + mb + " MB in " + ms + " ms at " + (mb / ms * 1000.0) + " MB/s" logInfo(resultStr) resultStr }).collect() From 8ac0f35be42765fcd6f02dcf0f070f2ef2377a85 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Tue, 8 Jan 2013 09:57:45 -0600 Subject: [PATCH 62/85] Add JavaRDDLike.keyBy. --- core/src/main/scala/spark/api/java/JavaRDDLike.scala | 8 ++++++++ core/src/test/scala/spark/JavaAPISuite.java | 12 ++++++++++++ 2 files changed, 20 insertions(+) diff --git a/core/src/main/scala/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/spark/api/java/JavaRDDLike.scala index 81d3a94466..d15f6dd02f 100644 --- a/core/src/main/scala/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/spark/api/java/JavaRDDLike.scala @@ -298,4 +298,12 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Save this RDD as a SequenceFile of serialized objects. */ def saveAsObjectFile(path: String) = rdd.saveAsObjectFile(path) + + /** + * Creates tuples of the elements in this RDD by applying `f`. + */ + def keyBy[K](f: JFunction[T, K]): JavaPairRDD[K, T] = { + implicit val kcm: ClassManifest[K] = implicitly[ClassManifest[AnyRef]].asInstanceOf[ClassManifest[K]] + JavaPairRDD.fromRDD(rdd.keyBy(f)) + } } diff --git a/core/src/test/scala/spark/JavaAPISuite.java b/core/src/test/scala/spark/JavaAPISuite.java index 0817d1146c..c61913fc82 100644 --- a/core/src/test/scala/spark/JavaAPISuite.java +++ b/core/src/test/scala/spark/JavaAPISuite.java @@ -629,4 +629,16 @@ public class JavaAPISuite implements Serializable { floatAccum.setValue(5.0f); Assert.assertEquals((Float) 5.0f, floatAccum.value()); } + + @Test + public void keyBy() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2)); + List> s = rdd.keyBy(new Function() { + public String call(Integer t) throws Exception { + return t.toString(); + } + }).collect(); + Assert.assertEquals(new Tuple2("1", 1), s.get(0)); + Assert.assertEquals(new Tuple2("2", 2), s.get(1)); + } } From c3f1675f9c4a1be9eebf9512795abc968ac29ba2 Mon Sep 17 00:00:00 2001 From: Stephen Haberman Date: Tue, 8 Jan 2013 14:44:33 -0600 Subject: [PATCH 63/85] Retrieve jars to a flat directory so * can be used for the classpath. --- project/SparkBuild.scala | 1 + run | 12 +++--------- 2 files changed, 4 insertions(+), 9 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 7c7c33131a..518c4130f0 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -38,6 +38,7 @@ object SparkBuild extends Build { scalacOptions := Seq(/*"-deprecation",*/ "-unchecked", "-optimize"), // -deprecation is too noisy due to usage of old Hadoop API, enable it once that's no longer an issue unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath }, retrieveManaged := true, + retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", transitiveClassifiers in Scope.GlobalScope := Seq("sources"), testListeners <<= target.map(t => Seq(new eu.henkelmann.sbt.JUnitXmlTestsListener(t.getAbsolutePath))), diff --git a/run b/run index 1528f83534..6cfe9631af 100755 --- a/run +++ b/run @@ -75,16 +75,10 @@ CLASSPATH+=":$CORE_DIR/src/main/resources" CLASSPATH+=":$REPL_DIR/target/scala-$SCALA_VERSION/classes" CLASSPATH+=":$EXAMPLES_DIR/target/scala-$SCALA_VERSION/classes" if [ -e "$FWDIR/lib_managed" ]; then - for jar in `find "$FWDIR/lib_managed/jars" -name '*jar'`; do - CLASSPATH+=":$jar" - done - for jar in `find "$FWDIR/lib_managed/bundles" -name '*jar'`; do - CLASSPATH+=":$jar" - done + CLASSPATH+=":$FWDIR/lib_managed/jars/*" + CLASSPATH+=":$FWDIR/lib_managed/bundles/*" fi -for jar in `find "$REPL_DIR/lib" -name '*jar'`; do - CLASSPATH+=":$jar" -done +CLASSPATH+=":$REPL_DIR/lib/*" for jar in `find "$REPL_DIR/target" -name 'spark-repl-*-shaded-hadoop*.jar'`; do CLASSPATH+=":$jar" done From b57dd0f16024a82dfc223e69528b9908b931f068 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 8 Jan 2013 16:04:41 -0800 Subject: [PATCH 64/85] Add mapPartitionsWithSplit() to PySpark. --- .../scala/spark/api/python/PythonRDD.scala | 5 +++ docs/python-programming-guide.md | 1 - python/pyspark/rdd.py | 33 ++++++++++++------- python/pyspark/worker.py | 4 ++- 4 files changed, 30 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/spark/api/python/PythonRDD.scala b/core/src/main/scala/spark/api/python/PythonRDD.scala index 79d824d494..f431ef28d3 100644 --- a/core/src/main/scala/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/spark/api/python/PythonRDD.scala @@ -65,6 +65,9 @@ private[spark] class PythonRDD[T: ClassManifest]( SparkEnv.set(env) val out = new PrintWriter(proc.getOutputStream) val dOut = new DataOutputStream(proc.getOutputStream) + // Split index + dOut.writeInt(split.index) + // Broadcast variables dOut.writeInt(broadcastVars.length) for (broadcast <- broadcastVars) { dOut.writeLong(broadcast.id) @@ -72,10 +75,12 @@ private[spark] class PythonRDD[T: ClassManifest]( dOut.write(broadcast.value) dOut.flush() } + // Serialized user code for (elem <- command) { out.println(elem) } out.flush() + // Data values for (elem <- parent.iterator(split, context)) { PythonRDD.writeAsPickle(elem, dOut) } diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index d963551296..78ef310a00 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -19,7 +19,6 @@ There are a few key differences between the Python and Scala APIs: - Accumulators - Special functions on RDDs of doubles, such as `mean` and `stdev` - `lookup` - - `mapPartitionsWithSplit` - `persist` at storage levels other than `MEMORY_ONLY` - `sample` - `sort` diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 4ba417b2a2..1d36da42b0 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -55,7 +55,7 @@ class RDD(object): """ Return a new RDD containing the distinct elements in this RDD. """ - def func(iterator): return imap(f, iterator) + def func(split, iterator): return imap(f, iterator) return PipelinedRDD(self, func, preservesPartitioning) def flatMap(self, f, preservesPartitioning=False): @@ -69,8 +69,8 @@ class RDD(object): >>> sorted(rdd.flatMap(lambda x: [(x, x), (x, x)]).collect()) [(2, 2), (2, 2), (3, 3), (3, 3), (4, 4), (4, 4)] """ - def func(iterator): return chain.from_iterable(imap(f, iterator)) - return self.mapPartitions(func, preservesPartitioning) + def func(s, iterator): return chain.from_iterable(imap(f, iterator)) + return self.mapPartitionsWithSplit(func, preservesPartitioning) def mapPartitions(self, f, preservesPartitioning=False): """ @@ -81,9 +81,20 @@ class RDD(object): >>> rdd.mapPartitions(f).collect() [3, 7] """ - return PipelinedRDD(self, f, preservesPartitioning) + def func(s, iterator): return f(iterator) + return self.mapPartitionsWithSplit(func) - # TODO: mapPartitionsWithSplit + def mapPartitionsWithSplit(self, f, preservesPartitioning=False): + """ + Return a new RDD by applying a function to each partition of this RDD, + while tracking the index of the original partition. + + >>> rdd = sc.parallelize([1, 2, 3, 4], 4) + >>> def f(splitIndex, iterator): yield splitIndex + >>> rdd.mapPartitionsWithSplit(f).sum() + 6 + """ + return PipelinedRDD(self, f, preservesPartitioning) def filter(self, f): """ @@ -362,7 +373,7 @@ class RDD(object): >>> ''.join(input(glob(tempFile.name + "/part-0000*"))) '0\\n1\\n2\\n3\\n4\\n5\\n6\\n7\\n8\\n9\\n' """ - def func(iterator): + def func(split, iterator): return (str(x).encode("utf-8") for x in iterator) keyed = PipelinedRDD(self, func) keyed._bypass_serializer = True @@ -500,7 +511,7 @@ class RDD(object): # Transferring O(n) objects to Java is too expensive. Instead, we'll # form the hash buckets in Python, transferring O(numSplits) objects # to Java. Each object is a (splitNumber, [objects]) pair. - def add_shuffle_key(iterator): + def add_shuffle_key(split, iterator): buckets = defaultdict(list) for (k, v) in iterator: buckets[hashFunc(k) % numSplits].append((k, v)) @@ -653,8 +664,8 @@ class PipelinedRDD(RDD): def __init__(self, prev, func, preservesPartitioning=False): if isinstance(prev, PipelinedRDD) and not prev.is_cached: prev_func = prev.func - def pipeline_func(iterator): - return func(prev_func(iterator)) + def pipeline_func(split, iterator): + return func(split, prev_func(split, iterator)) self.func = pipeline_func self.preservesPartitioning = \ prev.preservesPartitioning and preservesPartitioning @@ -677,8 +688,8 @@ class PipelinedRDD(RDD): if not self._bypass_serializer and self.ctx.batchSize != 1: oldfunc = self.func batchSize = self.ctx.batchSize - def batched_func(iterator): - return batched(oldfunc(iterator), batchSize) + def batched_func(split, iterator): + return batched(oldfunc(split, iterator), batchSize) func = batched_func cmds = [func, self._bypass_serializer] pipe_command = ' '.join(b64enc(cloudpickle.dumps(f)) for f in cmds) diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 9f6b507dbd..3d792bbaa2 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -21,6 +21,7 @@ def load_obj(): def main(): + split_index = read_int(sys.stdin) num_broadcast_variables = read_int(sys.stdin) for _ in range(num_broadcast_variables): bid = read_long(sys.stdin) @@ -32,7 +33,8 @@ def main(): dumps = lambda x: x else: dumps = dump_pickle - for obj in func(read_from_pickle_file(sys.stdin)): + iterator = read_from_pickle_file(sys.stdin) + for obj in func(split_index, iterator): write_with_length(dumps(obj), old_stdout) From 9cc764f52323baa3a218ce9e301d3cc98f1e8b20 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 8 Jan 2013 22:29:57 -0800 Subject: [PATCH 65/85] Code style --- .../scala/spark/scheduler/cluster/TaskSetManager.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala index a842afcdeb..a089b71644 100644 --- a/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala +++ b/core/src/main/scala/spark/scheduler/cluster/TaskSetManager.scala @@ -201,8 +201,11 @@ private[spark] class TaskSetManager( val taskId = sched.newTaskId() // Figure out whether this should count as a preferred launch val preferred = isPreferredLocation(task, host) - val prefStr = if (preferred) "preferred" - else "non-preferred, not one of " + task.preferredLocations.mkString(", ") + val prefStr = if (preferred) { + "preferred" + } else { + "non-preferred, not one of " + task.preferredLocations.mkString(", ") + } logInfo("Starting task %s:%d as TID %s on slave %s: %s (%s)".format( taskSet.id, index, taskId, slaveId, host, prefStr)) // Do various bookkeeping From 6e8c8f61c478ec5829677a38a624f17ac9609f74 Mon Sep 17 00:00:00 2001 From: Tyson Date: Wed, 9 Jan 2013 10:35:23 -0500 Subject: [PATCH 66/85] Added the spray implicit marshaller library Added the io.spray JSON library --- project/SparkBuild.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 2f67bb9921..f2b79d9ed8 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -133,6 +133,8 @@ object SparkBuild extends Build { "colt" % "colt" % "1.2.0", "cc.spray" % "spray-can" % "1.0-M2.1", "cc.spray" % "spray-server" % "1.0-M2.1", + "cc.spray" %% "spray-json" % "1.1.1", + "io.spray" %% "spray-json" % "1.2.3", "org.apache.mesos" % "mesos" % "0.9.0-incubating" ) ++ (if (HADOOP_MAJOR_VERSION == "2") Some("org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION) else None).toSeq, unmanagedSourceDirectories in Compile <+= baseDirectory{ _ / ("src/hadoop" + HADOOP_MAJOR_VERSION + "/scala") } From 269fe018c73a0d4e12a3c881dbd3bd807e504891 Mon Sep 17 00:00:00 2001 From: Tyson Date: Wed, 9 Jan 2013 10:35:59 -0500 Subject: [PATCH 67/85] JSON object definitions --- .../scala/spark/deploy/JsonProtocol.scala | 59 +++++++++++++++++++ 1 file changed, 59 insertions(+) create mode 100644 core/src/main/scala/spark/deploy/JsonProtocol.scala diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/spark/deploy/JsonProtocol.scala new file mode 100644 index 0000000000..dc7da85f9c --- /dev/null +++ b/core/src/main/scala/spark/deploy/JsonProtocol.scala @@ -0,0 +1,59 @@ +package spark.deploy + +import master.{JobInfo, WorkerInfo} +import spray.json._ + +/** + * spray-json helper class containing implicit conversion to json for marshalling responses + */ +private[spark] object JsonProtocol extends DefaultJsonProtocol { + import cc.spray.json._ + + implicit object WorkerInfoJsonFormat extends RootJsonWriter[WorkerInfo] { + def write(obj: WorkerInfo) = JsObject( + "id" -> JsString(obj.id), + "host" -> JsString(obj.host), + "webuiaddress" -> JsString(obj.webUiAddress), + "cores" -> JsNumber(obj.cores), + "coresused" -> JsNumber(obj.coresUsed), + "memory" -> JsNumber(obj.memory), + "memoryused" -> JsNumber(obj.memoryUsed) + ) + } + + implicit object JobInfoJsonFormat extends RootJsonWriter[JobInfo] { + def write(obj: JobInfo) = JsObject( + "starttime" -> JsNumber(obj.startTime), + "id" -> JsString(obj.id), + "name" -> JsString(obj.desc.name), + "cores" -> JsNumber(obj.desc.cores), + "user" -> JsString(obj.desc.user), + "memoryperslave" -> JsNumber(obj.desc.memoryPerSlave), + "submitdate" -> JsString(obj.submitDate.toString)) + } + + implicit object MasterStateJsonFormat extends RootJsonWriter[MasterState] { + def write(obj: MasterState) = JsObject( + "url" -> JsString("spark://" + obj.uri), + "workers" -> JsArray(obj.workers.toList.map(_.toJson)), + "cores" -> JsNumber(obj.workers.map(_.cores).sum), + "coresused" -> JsNumber(obj.workers.map(_.coresUsed).sum), + "memory" -> JsNumber(obj.workers.map(_.memory).sum), + "memoryused" -> JsNumber(obj.workers.map(_.memoryUsed).sum), + "activejobs" -> JsArray(obj.activeJobs.toList.map(_.toJson)), + "completedjobs" -> JsArray(obj.completedJobs.toList.map(_.toJson)) + ) + } + + implicit object WorkerStateJsonFormat extends RootJsonWriter[WorkerState] { + def write(obj: WorkerState) = JsObject( + "id" -> JsString(obj.workerId), + "masterurl" -> JsString(obj.masterUrl), + "masterwebuiurl" -> JsString(obj.masterWebUiUrl), + "cores" -> JsNumber(obj.cores), + "coresused" -> JsNumber(obj.coresUsed), + "memory" -> JsNumber(obj.memory), + "memoryused" -> JsNumber(obj.memoryUsed) + ) + } +} From 0da2ff102e1e8ac50059252a153a1b9b3e74b6b8 Mon Sep 17 00:00:00 2001 From: Tyson Date: Wed, 9 Jan 2013 10:36:56 -0500 Subject: [PATCH 68/85] Added url query parameter json and handler --- .../spark/deploy/master/MasterWebUI.scala | 19 +++++++++++++----- .../spark/deploy/worker/WorkerWebUI.scala | 20 ++++++++++++++----- 2 files changed, 29 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index 3cdd3721f5..dfec1d1dc5 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -9,6 +9,9 @@ import cc.spray.Directives import cc.spray.directives._ import cc.spray.typeconversion.TwirlSupport._ import spark.deploy._ +import cc.spray.http.MediaTypes +import JsonProtocol._ +import cc.spray.typeconversion.SprayJsonSupport._ private[spark] class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Directives { @@ -19,13 +22,19 @@ class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Direct val handler = { get { - path("") { - completeWith { + (path("") & parameters('json ?)) { + case Some(js) => val future = master ? RequestMasterState - future.map { - masterState => spark.deploy.master.html.index.render(masterState.asInstanceOf[MasterState]) + respondWithMediaType(MediaTypes.`application/json`) { ctx => + ctx.complete(future.mapTo[MasterState]) + } + case None => + completeWith { + val future = master ? RequestMasterState + future.map { + masterState => spark.deploy.master.html.index.render(masterState.asInstanceOf[MasterState]) + } } - } } ~ path("job") { parameter("jobId") { jobId => diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala index d06f4884ee..a168f54ca0 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala @@ -7,7 +7,10 @@ import akka.util.Timeout import akka.util.duration._ import cc.spray.Directives import cc.spray.typeconversion.TwirlSupport._ -import spark.deploy.{WorkerState, RequestWorkerState} +import spark.deploy.{JsonProtocol, WorkerState, RequestWorkerState} +import cc.spray.http.MediaTypes +import JsonProtocol._ +import cc.spray.typeconversion.SprayJsonSupport._ private[spark] class WorkerWebUI(val actorSystem: ActorSystem, worker: ActorRef) extends Directives { @@ -18,13 +21,20 @@ class WorkerWebUI(val actorSystem: ActorSystem, worker: ActorRef) extends Direct val handler = { get { - path("") { - completeWith{ + (path("") & parameters('json ?)) { + case Some(js) => { val future = worker ? RequestWorkerState - future.map { workerState => - spark.deploy.worker.html.index(workerState.asInstanceOf[WorkerState]) + respondWithMediaType(MediaTypes.`application/json`) { ctx => + ctx.complete(future.mapTo[WorkerState]) } } + case None => + completeWith{ + val future = worker ? RequestWorkerState + future.map { workerState => + spark.deploy.worker.html.index(workerState.asInstanceOf[WorkerState]) + } + } } ~ path("log") { parameters("jobId", "executorId", "logType") { (jobId, executorId, logType) => From bf9d9946f97782c9212420123b4a042918d7df5e Mon Sep 17 00:00:00 2001 From: Tyson Date: Wed, 9 Jan 2013 11:29:22 -0500 Subject: [PATCH 69/85] Query parameter reformatted to be more extensible and routing more robust --- core/src/main/scala/spark/deploy/master/MasterWebUI.scala | 6 +++--- core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index dfec1d1dc5..a96b55d6f3 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -22,13 +22,13 @@ class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Direct val handler = { get { - (path("") & parameters('json ?)) { - case Some(js) => + (path("") & parameters('format ?)) { + case Some(js) if js.equalsIgnoreCase("json") => val future = master ? RequestMasterState respondWithMediaType(MediaTypes.`application/json`) { ctx => ctx.complete(future.mapTo[MasterState]) } - case None => + case _ => completeWith { val future = master ? RequestMasterState future.map { diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala index a168f54ca0..84b6c16bd6 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala @@ -21,14 +21,14 @@ class WorkerWebUI(val actorSystem: ActorSystem, worker: ActorRef) extends Direct val handler = { get { - (path("") & parameters('json ?)) { - case Some(js) => { + (path("") & parameters('format ?)) { + case Some(js) if js.equalsIgnoreCase("json") => { val future = worker ? RequestWorkerState respondWithMediaType(MediaTypes.`application/json`) { ctx => ctx.complete(future.mapTo[WorkerState]) } } - case None => + case _ => completeWith{ val future = worker ? RequestWorkerState future.map { workerState => From 549ee388a125ac7014ae3dadfb16c582e250c654 Mon Sep 17 00:00:00 2001 From: Tyson Date: Wed, 9 Jan 2013 15:12:23 -0500 Subject: [PATCH 70/85] Removed io.spray spray-json dependency as it is not needed. --- core/src/main/scala/spark/deploy/JsonProtocol.scala | 4 +--- project/SparkBuild.scala | 1 - 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/spark/deploy/JsonProtocol.scala index dc7da85f9c..f14f804b3a 100644 --- a/core/src/main/scala/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/spark/deploy/JsonProtocol.scala @@ -1,14 +1,12 @@ package spark.deploy import master.{JobInfo, WorkerInfo} -import spray.json._ +import cc.spray.json._ /** * spray-json helper class containing implicit conversion to json for marshalling responses */ private[spark] object JsonProtocol extends DefaultJsonProtocol { - import cc.spray.json._ - implicit object WorkerInfoJsonFormat extends RootJsonWriter[WorkerInfo] { def write(obj: WorkerInfo) = JsObject( "id" -> JsString(obj.id), diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f2b79d9ed8..c63efbdd2a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -134,7 +134,6 @@ object SparkBuild extends Build { "cc.spray" % "spray-can" % "1.0-M2.1", "cc.spray" % "spray-server" % "1.0-M2.1", "cc.spray" %% "spray-json" % "1.1.1", - "io.spray" %% "spray-json" % "1.2.3", "org.apache.mesos" % "mesos" % "0.9.0-incubating" ) ++ (if (HADOOP_MAJOR_VERSION == "2") Some("org.apache.hadoop" % "hadoop-client" % HADOOP_VERSION) else None).toSeq, unmanagedSourceDirectories in Compile <+= baseDirectory{ _ / ("src/hadoop" + HADOOP_MAJOR_VERSION + "/scala") } From 1a64432ba50904c3933d8a9539a619fc94b3b30b Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 9 Jan 2013 20:30:36 -0800 Subject: [PATCH 71/85] Indicate success/failure in PySpark test script. --- python/run-tests | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/python/run-tests b/python/run-tests index da9e24cb1f..fcdd1e27a7 100755 --- a/python/run-tests +++ b/python/run-tests @@ -3,7 +3,24 @@ # Figure out where the Scala framework is installed FWDIR="$(cd `dirname $0`; cd ../; pwd)" +FAILED=0 + $FWDIR/pyspark pyspark/rdd.py +FAILED=$(($?||$FAILED)) + $FWDIR/pyspark -m doctest pyspark/broadcast.py +FAILED=$(($?||$FAILED)) + +if [[ $FAILED != 0 ]]; then + echo -en "\033[31m" # Red + echo "Had test failures; see logs." + echo -en "\033[0m" # No color + exit -1 +else + echo -en "\033[32m" # Green + echo "Tests passed." + echo -en "\033[0m" # No color +fi # TODO: in the long-run, it would be nice to use a test runner like `nose`. +# The doctest fixtures are the current barrier to doing this. From d55f2b98822faa7d71f5fce2bfa980f8265e0610 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 9 Jan 2013 21:21:23 -0800 Subject: [PATCH 72/85] Use take() instead of takeSample() in PySpark kmeans example. This is a temporary change until we port takeSample(). --- python/examples/kmeans.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/python/examples/kmeans.py b/python/examples/kmeans.py index ad2be21178..72cf9f88c6 100644 --- a/python/examples/kmeans.py +++ b/python/examples/kmeans.py @@ -33,7 +33,9 @@ if __name__ == "__main__": K = int(sys.argv[3]) convergeDist = float(sys.argv[4]) - kPoints = data.takeSample(False, K, 34) + # TODO: change this after we port takeSample() + #kPoints = data.takeSample(False, K, 34) + kPoints = data.take(K) tempDist = 1.0 while tempDist > convergeDist: From 9930a95d217045c4c22c2575080a03e4b0fd2426 Mon Sep 17 00:00:00 2001 From: shane-huang Date: Thu, 10 Jan 2013 20:09:34 +0800 Subject: [PATCH 73/85] Modified Patch according to comments --- .../main/scala/spark/network/Connection.scala | 8 ++++---- .../spark/network/ConnectionManager.scala | 9 ++++----- .../spark/network/ConnectionManagerTest.scala | 20 +++++++++++++------ 3 files changed, 22 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/spark/network/Connection.scala b/core/src/main/scala/spark/network/Connection.scala index 95096fd0ba..c193bf7c8d 100644 --- a/core/src/main/scala/spark/network/Connection.scala +++ b/core/src/main/scala/spark/network/Connection.scala @@ -136,10 +136,10 @@ extends Connection(SocketChannel.open, selector_) { if (chunk.isDefined) { messages += message // this is probably incorrect, it wont work as fifo if (!message.started) { - logDebug("Starting to send [" + message + "]") - message.started = true - message.startTime = System.currentTimeMillis - } + logDebug("Starting to send [" + message + "]") + message.started = true + message.startTime = System.currentTimeMillis + } return chunk } else { /*logInfo("Finished sending [" + message + "] to [" + remoteConnectionManagerId + "]")*/ diff --git a/core/src/main/scala/spark/network/ConnectionManager.scala b/core/src/main/scala/spark/network/ConnectionManager.scala index e7bd2d3bbd..36c01ad629 100644 --- a/core/src/main/scala/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/spark/network/ConnectionManager.scala @@ -43,12 +43,12 @@ private[spark] class ConnectionManager(port: Int) extends Logging { } val selector = SelectorProvider.provider.openSelector() - val handleMessageExecutor = Executors.newFixedThreadPool(20) + val handleMessageExecutor = Executors.newFixedThreadPool(System.getProperty("spark.core.connection.handler.threads","20").toInt) val serverChannel = ServerSocketChannel.open() val connectionsByKey = new HashMap[SelectionKey, Connection] with SynchronizedMap[SelectionKey, Connection] val connectionsById = new HashMap[ConnectionManagerId, SendingConnection] with SynchronizedMap[ConnectionManagerId, SendingConnection] val messageStatuses = new HashMap[Int, MessageStatus] - val connectionRequests = new HashMap[ConnectionManagerId, SendingConnection] with SynchronizedMap[ConnectionManagerId, SendingConnection] + val connectionRequests = new HashMap[ConnectionManagerId, SendingConnection] with SynchronizedMap[ConnectionManagerId, SendingConnection] val keyInterestChangeRequests = new SynchronizedQueue[(SelectionKey, Int)] val sendMessageRequests = new Queue[(Message, SendingConnection)] @@ -78,9 +78,8 @@ private[spark] class ConnectionManager(port: Int) extends Logging { def run() { try { - while(!selectorThread.isInterrupted) { + while(!selectorThread.isInterrupted) { for( (connectionManagerId, sendingConnection) <- connectionRequests) { - //val sendingConnection = connectionRequests.dequeue sendingConnection.connect() addConnection(sendingConnection) connectionRequests -= connectionManagerId @@ -465,7 +464,7 @@ private[spark] object ConnectionManager { val bufferMessage = Message.createBufferMessage(buffer.duplicate) manager.sendMessageReliably(manager.id, bufferMessage) }).foreach(f => { - val g = Await.result(f, 10 second) + val g = Await.result(f, 1 second) if (!g.isDefined) println("Failed") }) val finishTime = System.currentTimeMillis diff --git a/core/src/main/scala/spark/network/ConnectionManagerTest.scala b/core/src/main/scala/spark/network/ConnectionManagerTest.scala index 0e79c518e0..533e4610f3 100644 --- a/core/src/main/scala/spark/network/ConnectionManagerTest.scala +++ b/core/src/main/scala/spark/network/ConnectionManagerTest.scala @@ -13,8 +13,14 @@ import akka.util.duration._ private[spark] object ConnectionManagerTest extends Logging{ def main(args: Array[String]) { - if (args.length < 5) { - println("Usage: ConnectionManagerTest ") + // - the master URL + // - a list slaves to run connectionTest on + //[num of tasks] - the number of parallel tasks to be initiated default is number of slave hosts + //[size of msg in MB (integer)] - the size of messages to be sent in each task, default is 10 + //[count] - how many times to run, default is 3 + //[await time in seconds] : await time (in seconds), default is 600 + if (args.length < 2) { + println("Usage: ConnectionManagerTest [num of tasks] [size of msg in MB (integer)] [count] [await time in seconds)] ") System.exit(1) } @@ -29,14 +35,17 @@ private[spark] object ConnectionManagerTest extends Logging{ /*println("Slaves")*/ /*slaves.foreach(println)*/ - val tasknum = args(2).toInt + val tasknum = if (args.length > 2) args(2).toInt else slaves.length + val size = ( if (args.length > 3) (args(3).toInt) else 10 ) * 1024 * 1024 + val count = if (args.length > 4) args(4).toInt else 3 + val awaitTime = (if (args.length > 5) args(5).toInt else 600 ).second + println("Running "+count+" rounds of test: " + "parallel tasks = " + tasknum + ", msg size = " + size/1024/1024 + " MB, awaitTime = " + awaitTime) val slaveConnManagerIds = sc.parallelize(0 until tasknum, tasknum).map( i => SparkEnv.get.connectionManager.id).collect() println("\nSlave ConnectionManagerIds") slaveConnManagerIds.foreach(println) println - val count = args(4).toInt (0 until count).foreach(i => { val resultStrs = sc.parallelize(0 until tasknum, tasknum).map(i => { val connManager = SparkEnv.get.connectionManager @@ -46,7 +55,6 @@ private[spark] object ConnectionManagerTest extends Logging{ None }) - val size = (args(3).toInt) * 1024 * 1024 val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) buffer.flip @@ -56,7 +64,7 @@ private[spark] object ConnectionManagerTest extends Logging{ logInfo("Sending [" + bufferMessage + "] to [" + slaveConnManagerId + "]") connManager.sendMessageReliably(slaveConnManagerId, bufferMessage) }) - val results = futures.map(f => Await.result(f, 999.second)) + val results = futures.map(f => Await.result(f, awaitTime)) val finishTime = System.currentTimeMillis Thread.sleep(5000) From 49c74ba2af2ab6fe5eda16dbcd35b30b46072a3a Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 10 Jan 2013 07:45:12 -0800 Subject: [PATCH 74/85] Change PYSPARK_PYTHON_EXEC to PYSPARK_PYTHON. --- python/pyspark/context.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 4439356c1f..e486f206b0 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -52,7 +52,7 @@ class SparkContext(object): self._jsc = self.jvm.JavaSparkContext(master, jobName, sparkHome, empty_string_array) - self.pythonExec = os.environ.get("PYSPARK_PYTHON_EXEC", 'python') + self.pythonExec = os.environ.get("PYSPARK_PYTHON", 'python') # Broadcast's __reduce__ method stores Broadcast instances here. # This allows other code to determine which Broadcast instances have # been pickled, so it can determine which Java broadcast objects to From bd336f5f406386c929f2d1f9aecd7d5190a1a087 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 10 Jan 2013 17:13:04 -0800 Subject: [PATCH 75/85] Changed CoGroupRDD's hash map from Scala to Java. --- core/src/main/scala/spark/rdd/CoGroupedRDD.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index de0d9fad88..2e051c81c8 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -1,7 +1,8 @@ package spark.rdd +import java.util.{HashMap => JHashMap} import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap +import scala.collection.JavaConversions._ import spark.{Aggregator, Logging, Partitioner, RDD, SparkEnv, Split, TaskContext} import spark.{Dependency, OneToOneDependency, ShuffleDependency} @@ -71,7 +72,7 @@ class CoGroupedRDD[K](@transient rdds: Seq[RDD[(_, _)]], part: Partitioner) override def compute(s: Split, context: TaskContext): Iterator[(K, Seq[Seq[_]])] = { val split = s.asInstanceOf[CoGroupSplit] val numRdds = split.deps.size - val map = new HashMap[K, Seq[ArrayBuffer[Any]]] + val map = new JHashMap[K, Seq[ArrayBuffer[Any]]] def getSeq(k: K): Seq[ArrayBuffer[Any]] = { map.getOrElseUpdate(k, Array.fill(numRdds)(new ArrayBuffer[Any])) } From 2e914d99835487e867cac6add8be1dbd80dc693f Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 10 Jan 2013 19:13:08 -0800 Subject: [PATCH 76/85] Formatting --- core/src/main/scala/spark/deploy/master/MasterWebUI.scala | 5 +++-- core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala | 5 +++-- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index a96b55d6f3..580014ef3f 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -8,11 +8,12 @@ import akka.util.duration._ import cc.spray.Directives import cc.spray.directives._ import cc.spray.typeconversion.TwirlSupport._ -import spark.deploy._ import cc.spray.http.MediaTypes -import JsonProtocol._ import cc.spray.typeconversion.SprayJsonSupport._ +import spark.deploy._ +import spark.deploy.JsonProtocol._ + private[spark] class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Directives { val RESOURCE_DIR = "spark/deploy/master/webui" diff --git a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala index 84b6c16bd6..f9489d99fc 100644 --- a/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala +++ b/core/src/main/scala/spark/deploy/worker/WorkerWebUI.scala @@ -7,11 +7,12 @@ import akka.util.Timeout import akka.util.duration._ import cc.spray.Directives import cc.spray.typeconversion.TwirlSupport._ -import spark.deploy.{JsonProtocol, WorkerState, RequestWorkerState} import cc.spray.http.MediaTypes -import JsonProtocol._ import cc.spray.typeconversion.SprayJsonSupport._ +import spark.deploy.{WorkerState, RequestWorkerState} +import spark.deploy.JsonProtocol._ + private[spark] class WorkerWebUI(val actorSystem: ActorSystem, worker: ActorRef) extends Directives { val RESOURCE_DIR = "spark/deploy/worker/webui" From 92625223066a5c28553d7710c6b14af56f64b560 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 10 Jan 2013 22:07:34 -0800 Subject: [PATCH 77/85] Activate hadoop2 profile in pom.xml with -Dhadoop=2 --- bagel/pom.xml | 6 ++++++ core/pom.xml | 6 ++++++ examples/pom.xml | 6 ++++++ pom.xml | 6 ++++++ repl-bin/pom.xml | 6 ++++++ repl/pom.xml | 6 ++++++ 6 files changed, 36 insertions(+) diff --git a/bagel/pom.xml b/bagel/pom.xml index 85b2077026..c3461fb889 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -77,6 +77,12 @@ hadoop2 + + + hadoop + 2 + + org.spark-project diff --git a/core/pom.xml b/core/pom.xml index 005d8fe498..c8ff625774 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -216,6 +216,12 @@ hadoop2 + + + hadoop + 2 + + org.apache.hadoop diff --git a/examples/pom.xml b/examples/pom.xml index 3f738a3f8c..d0b1e97747 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -77,6 +77,12 @@ hadoop2 + + + hadoop + 2 + + org.spark-project diff --git a/pom.xml b/pom.xml index ea5b9c9d05..ae87813d4e 100644 --- a/pom.xml +++ b/pom.xml @@ -502,6 +502,12 @@ hadoop2 + + + hadoop + 2 + + 2 diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index fecb01f3cd..54ae20659e 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -115,6 +115,12 @@ hadoop2 + + + hadoop + 2 + + hadoop2 diff --git a/repl/pom.xml b/repl/pom.xml index 04b2c35beb..3e979b93a6 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -121,6 +121,12 @@ hadoop2 + + + hadoop + 2 + + hadoop2 From 480c4139bbd2711e99f3a819c9ef164d8b3dcac0 Mon Sep 17 00:00:00 2001 From: Michael Heuer Date: Fri, 11 Jan 2013 11:24:48 -0600 Subject: [PATCH 78/85] add repositories section to simple job pom.xml --- docs/quick-start.md | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/docs/quick-start.md b/docs/quick-start.md index 177cb14551..d46dc2da3f 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -200,6 +200,16 @@ To build the job, we also write a Maven `pom.xml` file that lists Spark as a dep Simple Project jar 1.0 + + + Spray.cc repository + http://repo.spray.cc + + + Typesafe repository + http://repo.typesafe.com/typesafe/releases + + org.spark-project From c063e8777ebaeb04056889064e9264edc019edbd Mon Sep 17 00:00:00 2001 From: Tyson Date: Fri, 11 Jan 2013 14:57:38 -0500 Subject: [PATCH 79/85] Added implicit json writers for JobDescription and ExecutorRunner --- .../scala/spark/deploy/JsonProtocol.scala | 23 ++++++++++++++++++- 1 file changed, 22 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/deploy/JsonProtocol.scala b/core/src/main/scala/spark/deploy/JsonProtocol.scala index f14f804b3a..732fa08064 100644 --- a/core/src/main/scala/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/spark/deploy/JsonProtocol.scala @@ -1,6 +1,7 @@ package spark.deploy import master.{JobInfo, WorkerInfo} +import worker.ExecutorRunner import cc.spray.json._ /** @@ -30,6 +31,24 @@ private[spark] object JsonProtocol extends DefaultJsonProtocol { "submitdate" -> JsString(obj.submitDate.toString)) } + implicit object JobDescriptionJsonFormat extends RootJsonWriter[JobDescription] { + def write(obj: JobDescription) = JsObject( + "name" -> JsString(obj.name), + "cores" -> JsNumber(obj.cores), + "memoryperslave" -> JsNumber(obj.memoryPerSlave), + "user" -> JsString(obj.user) + ) + } + + implicit object ExecutorRunnerJsonFormat extends RootJsonWriter[ExecutorRunner] { + def write(obj: ExecutorRunner) = JsObject( + "id" -> JsNumber(obj.execId), + "memory" -> JsNumber(obj.memory), + "jobid" -> JsString(obj.jobId), + "jobdesc" -> obj.jobDesc.toJson.asJsObject + ) + } + implicit object MasterStateJsonFormat extends RootJsonWriter[MasterState] { def write(obj: MasterState) = JsObject( "url" -> JsString("spark://" + obj.uri), @@ -51,7 +70,9 @@ private[spark] object JsonProtocol extends DefaultJsonProtocol { "cores" -> JsNumber(obj.cores), "coresused" -> JsNumber(obj.coresUsed), "memory" -> JsNumber(obj.memory), - "memoryused" -> JsNumber(obj.memoryUsed) + "memoryused" -> JsNumber(obj.memoryUsed), + "executors" -> JsArray(obj.executors.toList.map(_.toJson)), + "finishedexecutors" -> JsArray(obj.finishedExecutors.toList.map(_.toJson)) ) } } From 1731f1fed4f1369662b1a9fde850a3dcba738a59 Mon Sep 17 00:00:00 2001 From: Tyson Date: Fri, 11 Jan 2013 15:01:43 -0500 Subject: [PATCH 80/85] Added an optional format parameter for individual job queries and optimized the jobId query --- .../spark/deploy/master/MasterWebUI.scala | 38 +++++++++++++------ 1 file changed, 27 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala index 580014ef3f..458ee2d665 100644 --- a/core/src/main/scala/spark/deploy/master/MasterWebUI.scala +++ b/core/src/main/scala/spark/deploy/master/MasterWebUI.scala @@ -38,20 +38,36 @@ class MasterWebUI(val actorSystem: ActorSystem, master: ActorRef) extends Direct } } ~ path("job") { - parameter("jobId") { jobId => - completeWith { + parameters("jobId", 'format ?) { + case (jobId, Some(js)) if (js.equalsIgnoreCase("json")) => val future = master ? RequestMasterState - future.map { state => - val masterState = state.asInstanceOf[MasterState] - - // A bit ugly an inefficient, but we won't have a number of jobs - // so large that it will make a significant difference. - (masterState.activeJobs ++ masterState.completedJobs).find(_.id == jobId) match { - case Some(job) => spark.deploy.master.html.job_details.render(job) - case _ => null + val jobInfo = for (masterState <- future.mapTo[MasterState]) yield { + masterState.activeJobs.find(_.id == jobId) match { + case Some(job) => job + case _ => masterState.completedJobs.find(_.id == jobId) match { + case Some(job) => job + case _ => null + } + } + } + respondWithMediaType(MediaTypes.`application/json`) { ctx => + ctx.complete(jobInfo.mapTo[JobInfo]) + } + case (jobId, _) => + completeWith { + val future = master ? RequestMasterState + future.map { state => + val masterState = state.asInstanceOf[MasterState] + + masterState.activeJobs.find(_.id == jobId) match { + case Some(job) => spark.deploy.master.html.job_details.render(job) + case _ => masterState.completedJobs.find(_.id == jobId) match { + case Some(job) => spark.deploy.master.html.job_details.render(job) + case _ => null + } + } } } - } } } ~ pathPrefix("static") { From bbc56d85ed4eb4c3a09b20d5457f704f4b8a70c4 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sat, 12 Jan 2013 15:24:13 -0800 Subject: [PATCH 81/85] Rename environment variable for hadoop profiles to hadoopVersion --- bagel/pom.xml | 4 ++-- core/pom.xml | 4 ++-- examples/pom.xml | 4 ++-- pom.xml | 5 +++-- repl-bin/pom.xml | 4 ++-- repl/pom.xml | 4 ++-- 6 files changed, 13 insertions(+), 12 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index c3461fb889..5f58347204 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -47,7 +47,7 @@ hadoop1 - !hadoop + !hadoopVersion @@ -79,7 +79,7 @@ hadoop2 - hadoop + hadoopVersion 2 diff --git a/core/pom.xml b/core/pom.xml index c8ff625774..ad9fdcde2c 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -161,7 +161,7 @@ hadoop1 - !hadoop + !hadoopVersion @@ -218,7 +218,7 @@ hadoop2 - hadoop + hadoopVersion 2 diff --git a/examples/pom.xml b/examples/pom.xml index d0b1e97747..3355deb6b7 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -47,7 +47,7 @@ hadoop1 - !hadoop + !hadoopVersion @@ -79,7 +79,7 @@ hadoop2 - hadoop + hadoopVersion 2 diff --git a/pom.xml b/pom.xml index ae87813d4e..8f1af673a3 100644 --- a/pom.xml +++ b/pom.xml @@ -483,9 +483,10 @@ hadoop1 - !hadoop + !hadoopVersion + 1 @@ -504,7 +505,7 @@ hadoop2 - hadoop + hadoopVersion 2 diff --git a/repl-bin/pom.xml b/repl-bin/pom.xml index 54ae20659e..da91c0f3ab 100644 --- a/repl-bin/pom.xml +++ b/repl-bin/pom.xml @@ -72,7 +72,7 @@ hadoop1 - !hadoop + !hadoopVersion @@ -117,7 +117,7 @@ hadoop2 - hadoop + hadoopVersion 2 diff --git a/repl/pom.xml b/repl/pom.xml index 3e979b93a6..38e883c7f8 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -74,7 +74,7 @@ hadoop1 - !hadoop + !hadoopVersion @@ -123,7 +123,7 @@ hadoop2 - hadoop + hadoopVersion 2 From ba06e9c97cc3f8723ffdc3895182c529d3bb2fb3 Mon Sep 17 00:00:00 2001 From: Eric Zhang Date: Sun, 13 Jan 2013 15:33:11 +0800 Subject: [PATCH 82/85] Update examples/src/main/scala/spark/examples/LocalLR.scala MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit fix spelling mistake --- examples/src/main/scala/spark/examples/LocalLR.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/scala/spark/examples/LocalLR.scala b/examples/src/main/scala/spark/examples/LocalLR.scala index f2ac2b3e06..9553162004 100644 --- a/examples/src/main/scala/spark/examples/LocalLR.scala +++ b/examples/src/main/scala/spark/examples/LocalLR.scala @@ -5,7 +5,7 @@ import spark.util.Vector object LocalLR { val N = 10000 // Number of data points - val D = 10 // Numer of dimensions + val D = 10 // Number of dimensions val R = 0.7 // Scaling factor val ITERATIONS = 5 val rand = new Random(42) From 88d8f11365db84d46ff456495c07f664c91d1896 Mon Sep 17 00:00:00 2001 From: Mikhail Bautin Date: Sun, 13 Jan 2013 00:45:52 -0800 Subject: [PATCH 83/85] Add missing dependency spray-json to Maven build --- core/pom.xml | 4 ++++ pom.xml | 6 ++++++ 2 files changed, 10 insertions(+) diff --git a/core/pom.xml b/core/pom.xml index ad9fdcde2c..862d3ec37a 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -71,6 +71,10 @@ cc.spray spray-server + + cc.spray + spray-json_${scala.version} + org.tomdz.twirl twirl-api diff --git a/pom.xml b/pom.xml index 8f1af673a3..751189a9d8 100644 --- a/pom.xml +++ b/pom.xml @@ -54,6 +54,7 @@ 0.9.0-incubating 2.0.3 1.0-M2.1 + 1.1.1 1.6.1 4.1.2 @@ -222,6 +223,11 @@ spray-server ${spray.version} + + cc.spray + spray-json_${scala.version} + ${spray.json.version} + org.tomdz.twirl twirl-api From be7166146bf5692369272b85622d5316eccfd8e6 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 13 Jan 2013 15:27:28 -0800 Subject: [PATCH 84/85] Removed the use of getOrElse to avoid Scala wrapper for every call. --- core/src/main/scala/spark/rdd/CoGroupedRDD.scala | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala index 2e051c81c8..ce5f171911 100644 --- a/core/src/main/scala/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/spark/rdd/CoGroupedRDD.scala @@ -1,8 +1,8 @@ package spark.rdd import java.util.{HashMap => JHashMap} +import scala.collection.JavaConversions import scala.collection.mutable.ArrayBuffer -import scala.collection.JavaConversions._ import spark.{Aggregator, Logging, Partitioner, RDD, SparkEnv, Split, TaskContext} import spark.{Dependency, OneToOneDependency, ShuffleDependency} @@ -74,7 +74,14 @@ class CoGroupedRDD[K](@transient rdds: Seq[RDD[(_, _)]], part: Partitioner) val numRdds = split.deps.size val map = new JHashMap[K, Seq[ArrayBuffer[Any]]] def getSeq(k: K): Seq[ArrayBuffer[Any]] = { - map.getOrElseUpdate(k, Array.fill(numRdds)(new ArrayBuffer[Any])) + val seq = map.get(k) + if (seq != null) { + seq + } else { + val seq = Array.fill(numRdds)(new ArrayBuffer[Any]) + map.put(k, seq) + seq + } } for ((dep, depNum) <- split.deps.zipWithIndex) dep match { case NarrowCoGroupSplitDep(rdd, itsSplit) => { @@ -94,6 +101,6 @@ class CoGroupedRDD[K](@transient rdds: Seq[RDD[(_, _)]], part: Partitioner) fetcher.fetch[K, Seq[Any]](shuffleId, split.index).foreach(mergePair) } } - map.iterator + JavaConversions.mapAsScalaMap(map).iterator } } From 72408e8dfacc24652f376d1ee4dd6f04edb54804 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 13 Jan 2013 19:34:07 -0800 Subject: [PATCH 85/85] Make filter preserve partitioner info, since it can --- core/src/main/scala/spark/rdd/FilteredRDD.scala | 3 ++- core/src/test/scala/spark/PartitioningSuite.scala | 5 +++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/spark/rdd/FilteredRDD.scala b/core/src/main/scala/spark/rdd/FilteredRDD.scala index b148da28de..d46549b8b6 100644 --- a/core/src/main/scala/spark/rdd/FilteredRDD.scala +++ b/core/src/main/scala/spark/rdd/FilteredRDD.scala @@ -7,5 +7,6 @@ private[spark] class FilteredRDD[T: ClassManifest](prev: RDD[T], f: T => Boolean) extends RDD[T](prev.context) { override def splits = prev.splits override val dependencies = List(new OneToOneDependency(prev)) + override val partitioner = prev.partitioner // Since filter cannot change a partition's keys override def compute(split: Split, context: TaskContext) = prev.iterator(split, context).filter(f) -} \ No newline at end of file +} diff --git a/core/src/test/scala/spark/PartitioningSuite.scala b/core/src/test/scala/spark/PartitioningSuite.scala index f09b602a7b..eb3c8f238f 100644 --- a/core/src/test/scala/spark/PartitioningSuite.scala +++ b/core/src/test/scala/spark/PartitioningSuite.scala @@ -106,6 +106,11 @@ class PartitioningSuite extends FunSuite with BeforeAndAfter { assert(grouped2.leftOuterJoin(reduced2).partitioner === grouped2.partitioner) assert(grouped2.rightOuterJoin(reduced2).partitioner === grouped2.partitioner) assert(grouped2.cogroup(reduced2).partitioner === grouped2.partitioner) + + assert(grouped2.map(_ => 1).partitioner === None) + assert(grouped2.mapValues(_ => 1).partitioner === grouped2.partitioner) + assert(grouped2.flatMapValues(_ => Seq(1)).partitioner === grouped2.partitioner) + assert(grouped2.filter(_._1 > 4).partitioner === grouped2.partitioner) } test("partitioning Java arrays should fail") {