diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index 9ef60a7e2c84b4d316688e3019fef3707c848d2e..ab5aab1e115f7f8dcf606e9a25800be7aac42bfd 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()