From 136c77d8bbf48f7c45dd7c3fbe261a0476f455fe Mon Sep 17 00:00:00 2001
From: Liang-Chi Hsieh <viirya@appier.com>
Date: Thu, 17 Sep 2015 10:02:15 -0700
Subject: [PATCH] [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.
---
 python/pyspark/rdd.py | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)

diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index 9ef60a7e2c..ab5aab1e11 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -84,7 +84,7 @@ def portable_hash(x):
         h ^= len(x)
         if h == -1:
             h = -2
-        return h
+        return int(h)
     return hash(x)
 
 
@@ -2192,6 +2192,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 kv: kv[0] == key).values()
 
-- 
GitLab