Skip to content

Commit

Permalink
[SPARK-10642] [PYSPARK] Fix crash when calling rdd.lookup() on tuple …
Browse files Browse the repository at this point in the history
…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 <[email protected]>

Closes #8796 from viirya/fix-pyrdd-lookup.

(cherry picked from commit 136c77d)
Signed-off-by: Davies Liu <[email protected]>
  • Loading branch information
viirya authored and davies committed Sep 17, 2015
1 parent 8c8d7ab commit 7494034
Showing 1 changed file with 4 additions and 1 deletion.
5 changes: 4 additions & 1 deletion python/pyspark/rdd.py
Original file line number Diff line number Diff line change
Expand Up @@ -75,7 +75,7 @@ def portable_hash(x):
h ^= len(x)
if h == -1:
h = -2
return h
return int(h)
return hash(x)


Expand Down Expand Up @@ -2102,6 +2102,9 @@ def lookup(self, key):
[42]
>>> sorted.lookup(1024)
[]
>>> rdd2 = sc.parallelize([(('a', 'b'), 'c')]).groupByKey()
>>> list(rdd2.lookup(('a', 'b'))[0])
['c']
"""
values = self.filter(lambda (k, v): k == key).values()

Expand Down

0 comments on commit 7494034

Please sign in to comment.