Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-10642][PySpark] Fix crash when calling rdd.lookup() on tuple keys #8796

Closed
wants to merge 2 commits into from
Closed
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
5 changes: 4 additions & 1 deletion python/pyspark/rdd.py
Original file line number Diff line number Diff line change
Expand Up @@ -84,7 +84,7 @@ def portable_hash(x):
h ^= len(x)
if h == -1:
h = -2
return h
return int(h)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just asking the dumb question here, but is this intended to return an int? sys.maxsize does not appear to be the max positive 32-bit int.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should it use sys.maxint instead?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

CC @davies the real question is whether this hash is intended to be 32-bit or 64-bit, and my Python knowledge is too limited to reason about this. It appears that it's computing a 64-bit hash given the size of sys.maxsize but maybe that's platform dependent or something. Anyway: I kind of suspect you're right that it's 32-bit, but I think that has to be verified first.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The h become a long when h *= 1000003, even after h &= sys.maxsize or (maxint).

The fix looks good to me.

return hash(x)


Expand Down Expand Up @@ -2192,6 +2192,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 kv: kv[0] == key).values()

Expand Down