Skip to content

Commit

Permalink
add approx API for RDD
Browse files Browse the repository at this point in the history
  • Loading branch information
davies committed Aug 22, 2014
1 parent 050f8d0 commit e8c252b
Show file tree
Hide file tree
Showing 2 changed files with 98 additions and 0 deletions.
17 changes: 17 additions & 0 deletions core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -749,6 +749,23 @@ private[spark] object PythonRDD extends Logging {
}
}
}

/**
* Convert an RDD of serialized Python objects to RDD of objects, that is usable by PySpark.
*/
def pythonToJava(pyRDD: JavaRDD[Array[Byte]], batched: Boolean): JavaRDD[Any] = {
pyRDD.rdd.mapPartitions { iter =>
val unpickle = new Unpickler
iter.flatMap { row =>
val obj = unpickle.loads(row)
if (batched) {
obj.asInstanceOf[JArrayList[_]]
} else {
Seq(obj)
}
}
}.toJavaRDD()
}
}

private
Expand Down
81 changes: 81 additions & 0 deletions python/pyspark/rdd.py
Original file line number Diff line number Diff line change
Expand Up @@ -131,6 +131,22 @@ def __exit__(self, type, value, tb):
self._context._jsc.setCallSite(None)


class BoundedFloat(float):
"""
Bounded value is generated by approximate job, with confidence and low
bound and high bound.
>>> BoundedFloat(100.0, 0.95, 95.0, 105.0)
100.0
"""
def __new__(cls, mean, confidence, low, high):
obj = float.__new__(cls, mean)
obj.confidence = confidence
obj.low = low
obj.high = high
return obj


class MaxHeapQ(object):

"""
Expand Down Expand Up @@ -1782,6 +1798,71 @@ def _defaultReducePartitions(self):
# keys in the pairs. This could be an expensive operation, since those
# hashes aren't retained.

def _is_pickled(self):
""" Return this RDD is serialized by Pickle or not. """
der = self._jrdd_deserializer
if isinstance(der, PickleSerializer):
return True
if isinstance(der, BatchedSerializer) and isinstance(der.serializer, PickleSerializer):
return True
return False

def _to_jrdd(self):
""" Return an JavaRDD of Object by unpickling
It will convert each Python object into Java object by Pyrolite, whenever the
RDD is serialized in batch or not.
"""
if not self._is_pickled():
self = self._reserialize(BatchedSerializer(PickleSerializer(), 1024))
batched = isinstance(self._jrdd_deserializer, BatchedSerializer)
return self.ctx._jvm.PythonRDD.pythonToJava(self._jrdd, batched)

def countApprox(self, timeout, confidence=0.95):
"""
:: Experimental ::
Approximate version of count() that returns a potentially incomplete
result within a timeout, even if not all tasks have finished.
>>> rdd = sc.parallelize(range(1000), 10)
>>> rdd.countApprox(1000, 1.0)
1000
"""
drdd = self.mapPartitions(lambda it: [float(sum(1 for i in it))])
return int(drdd.sumApprox(timeout, confidence))

def sumApprox(self, timeout, confidence=0.95):
"""
:: Experimental ::
Approximate operation to return the sum within a timeout
or meet the confidence.
>>> rdd = sc.parallelize(range(1000), 10)
>>> r = sum(xrange(1000))
>>> (rdd.sumApprox(1000) - r) / r < 0.05
True
"""
jrdd = self.mapPartitions(lambda it: [float(sum(it))])._to_jrdd()
jdrdd = self.ctx._jvm.JavaDoubleRDD.fromRDD(jrdd.rdd())
r = jdrdd.sumApprox(timeout, confidence).getFinalValue()
return BoundedFloat(r.mean(), r.confidence(), r.low(), r.high())

def meanApprox(self, timeout, confidence=0.95):
"""
:: Experimental ::
Approximate operation to return the mean within a timeout
or meet the confidence.
>>> rdd = sc.parallelize(range(1000), 10)
>>> r = sum(xrange(1000)) / 1000.0
>>> (rdd.meanApprox(1000) - r) / r < 0.05
True
"""
jrdd = self.map(float)._to_jrdd()
jdrdd = self.ctx._jvm.JavaDoubleRDD.fromRDD(jrdd.rdd())
r = jdrdd.meanApprox(timeout, confidence).getFinalValue()
return BoundedFloat(r.mean(), r.confidence(), r.low(), r.high())


class PipelinedRDD(RDD):

Expand Down

0 comments on commit e8c252b

Please sign in to comment.