Skip to content

Commit 4fa5362

Browse files
viiryadavies
authored andcommitted
[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 apache#8796 from viirya/fix-pyrdd-lookup. (cherry picked from commit 136c77d) Signed-off-by: Davies Liu <davies.liu@gmail.com>
1 parent 4862a80 commit 4fa5362

File tree

1 file changed

+4
-1
lines changed

1 file changed

+4
-1
lines changed

python/pyspark/rdd.py

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -76,7 +76,7 @@ def portable_hash(x):
7676
h ^= len(x)
7777
if h == -1:
7878
h = -2
79-
return h
79+
return int(h)
8080
return hash(x)
8181

8282

@@ -1968,6 +1968,9 @@ def lookup(self, key):
19681968
[42]
19691969
>>> sorted.lookup(1024)
19701970
[]
1971+
>>> rdd2 = sc.parallelize([(('a', 'b'), 'c')]).groupByKey()
1972+
>>> list(rdd2.lookup(('a', 'b'))[0])
1973+
['c']
19711974
"""
19721975
values = self.filter(lambda (k, v): k == key).values()
19731976

0 commit comments

Comments
 (0)