aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--docs/configuration.md19
-rw-r--r--python/pyspark/accumulators.py15
-rw-r--r--python/pyspark/context.py39
-rw-r--r--python/pyspark/rdd.py10
-rw-r--r--python/pyspark/sql.py2
-rw-r--r--python/pyspark/tests.py30
-rw-r--r--python/pyspark/worker.py19
7 files changed, 127 insertions, 7 deletions
diff --git a/docs/configuration.md b/docs/configuration.md
index a6dd7245e1..791b6f2aa3 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -207,6 +207,25 @@ Apart from these, the following properties are also available, and may be useful
</td>
</tr>
<tr>
+ <td><code>spark.python.profile</code></td>
+ <td>false</td>
+ <td>
+ Enable profiling in Python worker, the profile result will show up by `sc.show_profiles()`,
+ or it will be displayed before the driver exiting. It also can be dumped into disk by
+ `sc.dump_profiles(path)`. If some of the profile results had been displayed maually,
+ they will not be displayed automatically before driver exiting.
+ </td>
+</tr>
+<tr>
+ <td><code>spark.python.profile.dump</code></td>
+ <td>(none)</td>
+ <td>
+ The directory which is used to dump the profile result before driver exiting.
+ The results will be dumped as separated file for each RDD. They can be loaded
+ by ptats.Stats(). If this is specified, the profile result will not be displayed
+ automatically.
+</tr>
+<tr>
<td><code>spark.python.worker.reuse</code></td>
<td>true</td>
<td>
diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py
index ccbca67656..b8cdbbe3cf 100644
--- a/python/pyspark/accumulators.py
+++ b/python/pyspark/accumulators.py
@@ -215,6 +215,21 @@ FLOAT_ACCUMULATOR_PARAM = AddingAccumulatorParam(0.0)
COMPLEX_ACCUMULATOR_PARAM = AddingAccumulatorParam(0.0j)
+class PStatsParam(AccumulatorParam):
+ """PStatsParam is used to merge pstats.Stats"""
+
+ @staticmethod
+ def zero(value):
+ return None
+
+ @staticmethod
+ def addInPlace(value1, value2):
+ if value1 is None:
+ return value2
+ value1.add(value2)
+ return value1
+
+
class _UpdateRequestHandler(SocketServer.StreamRequestHandler):
"""
diff --git a/python/pyspark/context.py b/python/pyspark/context.py
index 8e7b00469e..e9418320ff 100644
--- a/python/pyspark/context.py
+++ b/python/pyspark/context.py
@@ -20,6 +20,7 @@ import shutil
import sys
from threading import Lock
from tempfile import NamedTemporaryFile
+import atexit
from pyspark import accumulators
from pyspark.accumulators import Accumulator
@@ -30,7 +31,6 @@ from pyspark.java_gateway import launch_gateway
from pyspark.serializers import PickleSerializer, BatchedSerializer, UTF8Deserializer, \
PairDeserializer, CompressedSerializer
from pyspark.storagelevel import StorageLevel
-from pyspark import rdd
from pyspark.rdd import RDD
from pyspark.traceback_utils import CallSite, first_spark_call
@@ -192,6 +192,9 @@ class SparkContext(object):
self._temp_dir = \
self._jvm.org.apache.spark.util.Utils.createTempDir(local_dir).getAbsolutePath()
+ # profiling stats collected for each PythonRDD
+ self._profile_stats = []
+
def _initialize_context(self, jconf):
"""
Initialize SparkContext in function to allow subclass specific initialization
@@ -792,6 +795,40 @@ class SparkContext(object):
it = self._jvm.PythonRDD.runJob(self._jsc.sc(), mappedRDD._jrdd, javaPartitions, allowLocal)
return list(mappedRDD._collect_iterator_through_file(it))
+ def _add_profile(self, id, profileAcc):
+ if not self._profile_stats:
+ dump_path = self._conf.get("spark.python.profile.dump")
+ if dump_path:
+ atexit.register(self.dump_profiles, dump_path)
+ else:
+ atexit.register(self.show_profiles)
+
+ self._profile_stats.append([id, profileAcc, False])
+
+ def show_profiles(self):
+ """ Print the profile stats to stdout """
+ for i, (id, acc, showed) in enumerate(self._profile_stats):
+ stats = acc.value
+ if not showed and stats:
+ print "=" * 60
+ print "Profile of RDD<id=%d>" % id
+ print "=" * 60
+ stats.sort_stats("time", "cumulative").print_stats()
+ # mark it as showed
+ self._profile_stats[i][2] = True
+
+ def dump_profiles(self, path):
+ """ Dump the profile stats into directory `path`
+ """
+ if not os.path.exists(path):
+ os.makedirs(path)
+ for id, acc, _ in self._profile_stats:
+ stats = acc.value
+ if stats:
+ p = os.path.join(path, "rdd_%d.pstats" % id)
+ stats.dump_stats(p)
+ self._profile_stats = []
+
def _test():
import atexit
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index 680140d72d..8ed89e2f97 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -15,7 +15,6 @@
# limitations under the License.
#
-from base64 import standard_b64encode as b64enc
import copy
from collections import defaultdict
from itertools import chain, ifilter, imap
@@ -32,6 +31,7 @@ import bisect
from random import Random
from math import sqrt, log, isinf, isnan
+from pyspark.accumulators import PStatsParam
from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \
BatchedSerializer, CloudPickleSerializer, PairDeserializer, \
PickleSerializer, pack_long, AutoBatchedSerializer
@@ -2080,7 +2080,9 @@ class PipelinedRDD(RDD):
return self._jrdd_val
if self._bypass_serializer:
self._jrdd_deserializer = NoOpSerializer()
- command = (self.func, self._prev_jrdd_deserializer,
+ enable_profile = self.ctx._conf.get("spark.python.profile", "false") == "true"
+ profileStats = self.ctx.accumulator(None, PStatsParam) if enable_profile else None
+ command = (self.func, profileStats, self._prev_jrdd_deserializer,
self._jrdd_deserializer)
# the serialized command will be compressed by broadcast
ser = CloudPickleSerializer()
@@ -2102,6 +2104,10 @@ class PipelinedRDD(RDD):
self.ctx.pythonExec,
broadcast_vars, self.ctx._javaAccumulator)
self._jrdd_val = python_rdd.asJavaRDD()
+
+ if enable_profile:
+ self._id = self._jrdd_val.id()
+ self.ctx._add_profile(self._id, profileStats)
return self._jrdd_val
def id(self):
diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py
index f71d24c470..d8bdf22355 100644
--- a/python/pyspark/sql.py
+++ b/python/pyspark/sql.py
@@ -960,7 +960,7 @@ class SQLContext(object):
[Row(c0=4)]
"""
func = lambda _, it: imap(lambda x: f(*x), it)
- command = (func,
+ command = (func, None,
BatchedSerializer(PickleSerializer(), 1024),
BatchedSerializer(PickleSerializer(), 1024))
ser = CloudPickleSerializer()
diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py
index 29df754c6f..7e2bbc9cb6 100644
--- a/python/pyspark/tests.py
+++ b/python/pyspark/tests.py
@@ -632,6 +632,36 @@ class TestRDDFunctions(PySparkTestCase):
self.assertEquals(result.count(), 3)
+class TestProfiler(PySparkTestCase):
+
+ def setUp(self):
+ self._old_sys_path = list(sys.path)
+ class_name = self.__class__.__name__
+ conf = SparkConf().set("spark.python.profile", "true")
+ self.sc = SparkContext('local[4]', class_name, batchSize=2, conf=conf)
+
+ def test_profiler(self):
+
+ def heavy_foo(x):
+ for i in range(1 << 20):
+ x = 1
+ rdd = self.sc.parallelize(range(100))
+ rdd.foreach(heavy_foo)
+ profiles = self.sc._profile_stats
+ self.assertEqual(1, len(profiles))
+ id, acc, _ = profiles[0]
+ stats = acc.value
+ self.assertTrue(stats is not None)
+ width, stat_list = stats.get_print_list([])
+ func_names = [func_name for fname, n, func_name in stat_list]
+ self.assertTrue("heavy_foo" in func_names)
+
+ self.sc.show_profiles()
+ d = tempfile.gettempdir()
+ self.sc.dump_profiles(d)
+ self.assertTrue("rdd_%d.pstats" % id in os.listdir(d))
+
+
class TestSQL(PySparkTestCase):
def setUp(self):
diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py
index c1f6e3e4a1..8257dddfee 100644
--- a/python/pyspark/worker.py
+++ b/python/pyspark/worker.py
@@ -23,6 +23,8 @@ import sys
import time
import socket
import traceback
+import cProfile
+import pstats
from pyspark.accumulators import _accumulatorRegistry
from pyspark.broadcast import Broadcast, _broadcastRegistry
@@ -90,10 +92,21 @@ def main(infile, outfile):
command = pickleSer._read_with_length(infile)
if isinstance(command, Broadcast):
command = pickleSer.loads(command.value)
- (func, deserializer, serializer) = command
+ (func, stats, deserializer, serializer) = command
init_time = time.time()
- iterator = deserializer.load_stream(infile)
- serializer.dump_stream(func(split_index, iterator), outfile)
+
+ def process():
+ iterator = deserializer.load_stream(infile)
+ serializer.dump_stream(func(split_index, iterator), outfile)
+
+ if stats:
+ p = cProfile.Profile()
+ p.runcall(process)
+ st = pstats.Stats(p)
+ st.stream = None # make it picklable
+ stats.add(st.strip_dirs())
+ else:
+ process()
except Exception:
try:
write_int(SpecialLengths.PYTHON_EXCEPTION_THROWN, outfile)