diff options
Diffstat (limited to 'python/pyspark/streaming')
-rw-r--r-- | python/pyspark/streaming/kafka.py | 3 | ||||
-rw-r--r-- | python/pyspark/streaming/tests.py | 12 |
2 files changed, 6 insertions, 9 deletions
diff --git a/python/pyspark/streaming/kafka.py b/python/pyspark/streaming/kafka.py index 2c1a667fc8..bf27d8047a 100644 --- a/python/pyspark/streaming/kafka.py +++ b/python/pyspark/streaming/kafka.py @@ -287,6 +287,9 @@ class TopicAndPartition(object): def __ne__(self, other): return not self.__eq__(other) + def __hash__(self): + return (self._topic, self._partition).__hash__() + class Broker(object): """ diff --git a/python/pyspark/streaming/tests.py b/python/pyspark/streaming/tests.py index 360ba1e716..5ac007cd59 100644 --- a/python/pyspark/streaming/tests.py +++ b/python/pyspark/streaming/tests.py @@ -41,6 +41,9 @@ if sys.version_info[:2] <= (2, 6): else: import unittest +if sys.version >= "3": + long = int + from pyspark.context import SparkConf, SparkContext, RDD from pyspark.storagelevel import StorageLevel from pyspark.streaming.context import StreamingContext @@ -1058,7 +1061,6 @@ class KafkaStreamTests(PySparkStreamingTestCase): stream = KafkaUtils.createDirectStream(self.ssc, [topic], kafkaParams) self._validateStreamResult(sendData, stream) - @unittest.skipIf(sys.version >= "3", "long type not support") def test_kafka_direct_stream_from_offset(self): """Test the Python direct Kafka stream API with start offset specified.""" topic = self._randomTopic() @@ -1072,7 +1074,6 @@ class KafkaStreamTests(PySparkStreamingTestCase): stream = KafkaUtils.createDirectStream(self.ssc, [topic], kafkaParams, fromOffsets) self._validateStreamResult(sendData, stream) - @unittest.skipIf(sys.version >= "3", "long type not support") def test_kafka_rdd(self): """Test the Python direct Kafka RDD API.""" topic = self._randomTopic() @@ -1085,7 +1086,6 @@ class KafkaStreamTests(PySparkStreamingTestCase): rdd = KafkaUtils.createRDD(self.sc, kafkaParams, offsetRanges) self._validateRddResult(sendData, rdd) - @unittest.skipIf(sys.version >= "3", "long type not support") def test_kafka_rdd_with_leaders(self): """Test the Python direct Kafka RDD API with leaders.""" topic = self._randomTopic() @@ -1100,7 +1100,6 @@ class KafkaStreamTests(PySparkStreamingTestCase): rdd = KafkaUtils.createRDD(self.sc, kafkaParams, offsetRanges, leaders) self._validateRddResult(sendData, rdd) - @unittest.skipIf(sys.version >= "3", "long type not support") def test_kafka_rdd_get_offsetRanges(self): """Test Python direct Kafka RDD get OffsetRanges.""" topic = self._randomTopic() @@ -1113,7 +1112,6 @@ class KafkaStreamTests(PySparkStreamingTestCase): rdd = KafkaUtils.createRDD(self.sc, kafkaParams, offsetRanges) self.assertEqual(offsetRanges, rdd.offsetRanges()) - @unittest.skipIf(sys.version >= "3", "long type not support") def test_kafka_direct_stream_foreach_get_offsetRanges(self): """Test the Python direct Kafka stream foreachRDD get offsetRanges.""" topic = self._randomTopic() @@ -1138,7 +1136,6 @@ class KafkaStreamTests(PySparkStreamingTestCase): self.assertEqual(offsetRanges, [OffsetRange(topic, 0, long(0), long(6))]) - @unittest.skipIf(sys.version >= "3", "long type not support") def test_kafka_direct_stream_transform_get_offsetRanges(self): """Test the Python direct Kafka stream transform get offsetRanges.""" topic = self._randomTopic() @@ -1176,7 +1173,6 @@ class KafkaStreamTests(PySparkStreamingTestCase): self.assertNotEqual(topic_and_partition_a, topic_and_partition_c) self.assertNotEqual(topic_and_partition_a, topic_and_partition_d) - @unittest.skipIf(sys.version >= "3", "long type not support") def test_kafka_direct_stream_transform_with_checkpoint(self): """Test the Python direct Kafka stream transform with checkpoint correctly recovered.""" topic = self._randomTopic() @@ -1225,7 +1221,6 @@ class KafkaStreamTests(PySparkStreamingTestCase): finally: shutil.rmtree(tmpdir) - @unittest.skipIf(sys.version >= "3", "long type not support") def test_kafka_rdd_message_handler(self): """Test Python direct Kafka RDD MessageHandler.""" topic = self._randomTopic() @@ -1242,7 +1237,6 @@ class KafkaStreamTests(PySparkStreamingTestCase): messageHandler=getKeyAndDoubleMessage) self._validateRddResult({"aa": 1, "bb": 1, "cc": 2}, rdd) - @unittest.skipIf(sys.version >= "3", "long type not support") def test_kafka_direct_stream_message_handler(self): """Test the Python direct Kafka stream MessageHandler.""" topic = self._randomTopic() |