spark-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From dav...@apache.org
Subject spark git commit: [SPARK-10642] [PYSPARK] Fix crash when calling rdd.lookup() on tuple keys
Date Thu, 17 Sep 2015 17:03:22 GMT
Repository: spark
Updated Branches:
  refs/heads/branch-1.2 4862a80d2 -> 4fa5362be


[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.

(cherry picked from commit 136c77d8bbf48f7c45dd7c3fbe261a0476f455fe)
Signed-off-by: Davies Liu <davies.liu@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4fa5362b
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4fa5362b
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4fa5362b

Branch: refs/heads/branch-1.2
Commit: 4fa5362bec9dcc0b785a7152a215481c3a43d4c7
Parents: 4862a80
Author: Liang-Chi Hsieh <viirya@appier.com>
Authored: Thu Sep 17 10:02:15 2015 -0700
Committer: Davies Liu <davies.liu@gmail.com>
Committed: Thu Sep 17 10:03:13 2015 -0700

----------------------------------------------------------------------
 python/pyspark/rdd.py | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/4fa5362b/python/pyspark/rdd.py
----------------------------------------------------------------------
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index 6903bc8..fa9de4e 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -76,7 +76,7 @@ def portable_hash(x):
         h ^= len(x)
         if h == -1:
             h = -2
-        return h
+        return int(h)
     return hash(x)
 
 
@@ -1968,6 +1968,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 (k, v): k == key).values()
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org


Mime
View raw message