aboutsummaryrefslogtreecommitdiff
path: root/python
diff options
context:
space:
mode:
authorLiang-Chi Hsieh <viirya@appier.com>2015-09-17 10:02:15 -0700
committerDavies Liu <davies.liu@gmail.com>2015-09-17 10:02:15 -0700
commit136c77d8bbf48f7c45dd7c3fbe261a0476f455fe (patch)
treeddad12bd74b684333d08ca2f697b2d8a0e134355 /python
parentc88bb5df94f9696677c3a429472114bc66f32a52 (diff)
downloadspark-136c77d8bbf48f7c45dd7c3fbe261a0476f455fe.tar.gz
spark-136c77d8bbf48f7c45dd7c3fbe261a0476f455fe.tar.bz2
spark-136c77d8bbf48f7c45dd7c3fbe261a0476f455fe.zip
[SPARK-10642] [PYSPARK] Fix crash when calling rdd.lookup() on tuple keys
JIRA: https://issues.apache.org/jira/browse/SPARK-10642 When calling `rdd.lookup()` on a RDD with tuple keys, `portable_hash` will return a long. That causes `DAGScheduler.submitJob` to throw `java.lang.ClassCastException: java.lang.Long cannot be cast to java.lang.Integer`. Author: Liang-Chi Hsieh <viirya@appier.com> Closes #8796 from viirya/fix-pyrdd-lookup.
Diffstat (limited to 'python')
-rw-r--r--python/pyspark/rdd.py5
1 files changed, 4 insertions, 1 deletions
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index 9ef60a7e2c..ab5aab1e11 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -84,7 +84,7 @@ def portable_hash(x):
h ^= len(x)
if h == -1:
h = -2
- return h
+ return int(h)
return hash(x)
@@ -2192,6 +2192,9 @@ class RDD(object):
[42]
>>> sorted.lookup(1024)
[]
+ >>> rdd2 = sc.parallelize([(('a', 'b'), 'c')]).groupByKey()
+ >>> list(rdd2.lookup(('a', 'b'))[0])
+ ['c']
"""
values = self.filter(lambda kv: kv[0] == key).values()