-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-2494] [PySpark] make hash of None consistant cross machines #1371
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
Conversation
|
Can one of the admins verify this patch? |
|
This was already fixed before in another way -- see the code at line 1061 of rdd.py: I much prefer this to replacing the global "hash" function. Just do the same fix elsewhere if there are places where it's a problem. |
|
Actually I see you even deleted this code. What was the reason for this change? |
|
@mateiz If there is None in Tuple, such as (None, 3), the hash of it will be different cross machines. If user provide a partitionFunc, which uses hash() in it, then it will have problem. This hack does not look good for me. Maybe we just use this portable hash as default one? |
|
Ah, I see, that makes sense. In that case let's give it our own global name instead of replacing the built-in hash. We can just put it in the "pyspark" package. |
|
The original motivation to do in this way is that we hope it can fix the problem of structure with None in it automatically, but in fact, it does not work in some cases, such as tuple. It's also will help in some cases, such as user defined objects. |
|
Sure, but is it okay to not replace the global hash()? Just call it something like pyspark.hash and set the default to pyspark.hash. |
|
Jenkins, add to whitelist and test this please |
python/pyspark/tests.py
Outdated
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Won't this fail now? (Or if it passes it's because this runs on one machine somehow)
|
QA tests have started for PR 1371. This patch merges cleanly. |
|
QA results for PR 1371: |
|
Jenkins, test this please |
|
BTW for us to merge this you should also open a JIRA for it so we can track it. Just add one on https://issues.apache.org/jira/browse/SPARK. |
|
Create issue #SPARK-2494 to track this. |
|
i've confirmed that this patch addresses the reported issue... |
|
@mattf, Thanks! |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
My comment from before was deleted, but please add a link to where the implementation is from, or a reference to the Python source code for this
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Also explain what "consistent hash code means", this comment doesn't say anything about the hash code of None being different across machines by default
|
Hey @davies apart from the small comments above, please add a test in |
|
@matei, our tests only run in local mode, but this issue can only be On Sun, Jul 20, 2014 at 1:26 AM, Matei Zaharia notifications@github.com
|
|
Even in local mode, we launch multiple Python processes, one per core. Just set the master to local[4] or something like that. Some of our other tests do that. |
|
Even with multiprocess, the hash of None are the same, because they are On Sun, Jul 20, 2014 at 4:33 PM, Matei Zaharia notifications@github.com
|
|
Are you sure about that? They're forked from Java, not from the Python process. If this is the case, please suggest another way to test this. We can't add a bug fix without a test. |
|
Jenkins, test this please |
|
Actually I see there are some doctests that I missed earlier, maybe that's okay. Though last time it failed Jenkins... |
|
QA tests have started for PR 1371. This patch merges cleanly. |
|
QA results for PR 1371: |
|
Jenkins, test this please |
|
QA tests have started for PR 1371. This patch merges cleanly. |
|
QA results for PR 1371: |
|
The JVM fork one python daemon(daemon.py), then the daemon fork all the workers. |
In CPython, hash of None is different cross machines, it will cause wrong result during shuffle. This PR will fix this. Author: Davies Liu <davies.liu@gmail.com> Closes #1371 from davies/hash_of_none and squashes the following commits: d01745f [Davies Liu] add comments, remove outdated unit tests 5467141 [Davies Liu] disable hijack of hash, use it only for partitionBy() b7118aa [Davies Liu] use __builtin__ instead of __builtins__ 839e417 [Davies Liu] hijack hash to make hash of None consistant cross machines (cherry picked from commit 872538c) Signed-off-by: Matei Zaharia <matei@databricks.com>
In CPython, hash of None is different cross machines, it will cause wrong result during shuffle. This PR will fix this. Author: Davies Liu <davies.liu@gmail.com> Closes #1371 from davies/hash_of_none and squashes the following commits: d01745f [Davies Liu] add comments, remove outdated unit tests 5467141 [Davies Liu] disable hijack of hash, use it only for partitionBy() b7118aa [Davies Liu] use __builtin__ instead of __builtins__ 839e417 [Davies Liu] hijack hash to make hash of None consistant cross machines (cherry picked from commit 872538c) Signed-off-by: Matei Zaharia <matei@databricks.com>
In CPython, hash of None is different cross machines, it will cause wrong result during shuffle. This PR will fix this. Author: Davies Liu <davies.liu@gmail.com> Closes #1371 from davies/hash_of_none and squashes the following commits: d01745f [Davies Liu] add comments, remove outdated unit tests 5467141 [Davies Liu] disable hijack of hash, use it only for partitionBy() b7118aa [Davies Liu] use __builtin__ instead of __builtins__ 839e417 [Davies Liu] hijack hash to make hash of None consistant cross machines (cherry picked from commit 872538c) Signed-off-by: Matei Zaharia <matei@databricks.com>
|
Ah right, that makes sense. I've merged this in now. |
In CPython, hash of None is different cross machines, it will cause wrong result during shuffle. This PR will fix this. Author: Davies Liu <davies.liu@gmail.com> Closes apache#1371 from davies/hash_of_none and squashes the following commits: d01745f [Davies Liu] add comments, remove outdated unit tests 5467141 [Davies Liu] disable hijack of hash, use it only for partitionBy() b7118aa [Davies Liu] use __builtin__ instead of __builtins__ 839e417 [Davies Liu] hijack hash to make hash of None consistant cross machines
In CPython, hash of None is different cross machines, it will cause wrong result during shuffle. This PR will fix this.