From dd9aeebe21a83a2eb75320318f43e183909f48f1 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 23 Oct 2014 16:19:35 +0800 Subject: [PATCH 01/21] Initial commit for reliable Kafka receiver --- .../streaming/kafka/KafkaInputDStream.scala | 13 +- .../spark/streaming/kafka/KafkaReceiver.scala | 135 +++++++++++ .../spark/streaming/kafka/KafkaUtils.scala | 69 +++++- .../kafka/ReliableKafkaReceiver.scala | 211 ++++++++++++++++++ .../streaming/receiver/BlockGenerator.scala | 13 +- .../receiver/ReceiverSupervisorImpl.scala | 4 + .../spark/streaming/ReceiverSuite.scala | 4 + 7 files changed, 441 insertions(+), 8 deletions(-) create mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaReceiver.scala create mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala index 28ac5929df44a..92dd3f878a71f 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala @@ -18,12 +18,8 @@ package org.apache.spark.streaming.kafka import scala.collection.Map -import scala.reflect.{classTag, ClassTag} +import scala.reflect.ClassTag -import java.util.Properties -import java.util.concurrent.Executors - -import kafka.consumer._ import kafka.serializer.Decoder import kafka.utils.VerifiableProperties @@ -51,11 +47,16 @@ class KafkaInputDStream[ @transient ssc_ : StreamingContext, kafkaParams: Map[String, String], topics: Map[String, Int], + reliableStoreEnabled: Boolean, storageLevel: StorageLevel ) extends ReceiverInputDStream[(K, V)](ssc_) with Logging { def getReceiver(): Receiver[(K, V)] = { - new KafkaReceiver[K, V, U, T](kafkaParams, topics, storageLevel) + if (!reliableStoreEnabled) { + new KafkaReceiver[K, V, U, T](kafkaParams, topics, storageLevel) + .asInstanceOf[Receiver[(K, V)]] + } else { + new ReliableKafkaReceiver[K, V, U, T](kafkaParams, topics, storageLevel) .asInstanceOf[Receiver[(K, V)]] } } diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaReceiver.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaReceiver.scala new file mode 100644 index 0000000000000..fa3b2458904b6 --- /dev/null +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaReceiver.scala @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import scala.collection.Map +import scala.reflect.{classTag, ClassTag} + +import java.util.Properties +import java.util.concurrent.Executors + +import kafka.consumer._ +import kafka.serializer.Decoder +import kafka.utils.VerifiableProperties +import kafka.utils.ZKStringSerializer +import org.I0Itec.zkclient._ + +import org.apache.spark.Logging +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.receiver.Receiver + +private[streaming] +class KafkaReceiver[ + K: ClassTag, + V: ClassTag, + U <: Decoder[_]: ClassTag, + T <: Decoder[_]: ClassTag]( + kafkaParams: Map[String, String], + topics: Map[String, Int], + storageLevel: StorageLevel + ) extends Receiver[Any](storageLevel) with Logging { + + // Connection to Kafka + var consumerConnector: ConsumerConnector = null + + def onStop() { + if (consumerConnector != null) { + consumerConnector.shutdown() + } + } + + def onStart() { + + logInfo("Starting Kafka Consumer Stream with group: " + kafkaParams("group.id")) + + // Kafka connection properties + val props = new Properties() + kafkaParams.foreach(param => props.put(param._1, param._2)) + + val zkConnect = kafkaParams("zookeeper.connect") + // Create the connection to the cluster + logInfo("Connecting to Zookeeper: " + zkConnect) + val consumerConfig = new ConsumerConfig(props) + consumerConnector = Consumer.create(consumerConfig) + logInfo("Connected to " + zkConnect) + + // When auto.offset.reset is defined, it is our responsibility to try and whack the + // consumer group zk node. + if (kafkaParams.contains("auto.offset.reset")) { + tryZookeeperConsumerGroupCleanup(zkConnect, kafkaParams("group.id")) + } + + val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties]) + .newInstance(consumerConfig.props) + .asInstanceOf[Decoder[K]] + val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties]) + .newInstance(consumerConfig.props) + .asInstanceOf[Decoder[V]] + + // Create Threads for each Topic/Message Stream we are listening + val topicMessageStreams = consumerConnector.createMessageStreams( + topics, keyDecoder, valueDecoder) + + val executorPool = Executors.newFixedThreadPool(topics.values.sum) + try { + // Start the messages handler for each partition + topicMessageStreams.values.foreach { streams => + streams.foreach { stream => executorPool.submit(new MessageHandler(stream)) } + } + } finally { + executorPool.shutdown() // Just causes threads to terminate after work is done + } + } + + // Handles Kafka Messages + private class MessageHandler[K: ClassTag, V: ClassTag](stream: KafkaStream[K, V]) + extends Runnable { + def run() { + logInfo("Starting MessageHandler.") + try { + for (msgAndMetadata <- stream) { + store((msgAndMetadata.key, msgAndMetadata.message)) + } + } catch { + case e: Throwable => logError("Error handling message; exiting", e) + } + } + } + + // It is our responsibility to delete the consumer group when specifying auto.offset.reset. This + // is because Kafka 0.7.2 only honors this param when the group is not in zookeeper. + // + // The kafka high level consumer doesn't expose setting offsets currently, this is a trick copied + // from Kafka's ConsoleConsumer. See code related to 'auto.offset.reset' when it is set to + // 'smallest'/'largest': + // scalastyle:off + // https://github.com/apache/kafka/blob/0.7.2/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala + // scalastyle:on + private def tryZookeeperConsumerGroupCleanup(zkUrl: String, groupId: String) { + val dir = "/consumers/" + groupId + logInfo("Cleaning up temporary Zookeeper data under " + dir + ".") + val zk = new ZkClient(zkUrl, 30*1000, 30*1000, ZKStringSerializer) + try { + zk.deleteRecursive(dir) + } catch { + case e: Throwable => logWarning("Error cleaning up temporary Zookeeper data", e) + } finally { + zk.close() + } + } +} \ No newline at end of file diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala index ec812e1ef3b04..2eabdd170ddb6 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -70,7 +70,7 @@ object KafkaUtils { topics: Map[String, Int], storageLevel: StorageLevel ): ReceiverInputDStream[(K, V)] = { - new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, storageLevel) + new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, false, storageLevel) } /** @@ -144,4 +144,71 @@ object KafkaUtils { createStream[K, V, U, T]( jssc.ssc, kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) } + + def createReliableStream( + ssc: StreamingContext, + zkQuorum: String, + groupId: String, + topics: Map[String, Int], + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2) + : ReceiverInputDStream[(String, String)] = { + val kafkaParams = Map[String, String]( + "zookeeper.connect" -> zkQuorum, "group.id" -> groupId, + "zookeeper.connection.timeout.ms" -> "10000") + createReliableStream[String, String, StringDecoder, StringDecoder]( + ssc, kafkaParams, topics, storageLevel) + } + + def createReliableStream[ + K: ClassTag, + V: ClassTag, + U <: Decoder[_]: ClassTag, + T <: Decoder[_]: ClassTag]( + ssc: StreamingContext, + kafkaParams: Map[String, String], + topics: Map[String, Int], + storageLevel: StorageLevel + ): ReceiverInputDStream[(K, V)] = { + new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, true, storageLevel) + } + + def createReliableStream( + jssc: JavaStreamingContext, + zkQuorum: String, + groupId: String, + topics: JMap[String, JInt] + ): JavaPairReceiverInputDStream[String, String] = { + createReliableStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*)) + } + + def createReliableStream( + jssc: JavaStreamingContext, + zkQuorum: String, + groupId: String, + topics: JMap[String, JInt], + storageLevel: StorageLevel + ): JavaPairReceiverInputDStream[String, String] = { + createReliableStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), + storageLevel) + } + + def createReliableStream[K, V, U <: Decoder[_], T <: Decoder[_]]( + jssc: JavaStreamingContext, + keyTypeClass: Class[K], + valueTypeClass: Class[V], + keyDecoderClass: Class[U], + valueDecoderClass: Class[T], + kafkaParams: JMap[String, String], + topics: JMap[String, JInt], + storageLevel: StorageLevel + ): JavaPairReceiverInputDStream[K, V] = { + implicit val keyCmt: ClassTag[K] = ClassTag(keyTypeClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueTypeClass) + + implicit val keyCmd: ClassTag[U] = ClassTag(keyDecoderClass) + implicit val valueCmd: ClassTag[T] = ClassTag(valueDecoderClass) + + createReliableStream[K, V, U, T]( + jssc.ssc, kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) + } } diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala new file mode 100644 index 0000000000000..6773db67db24c --- /dev/null +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala @@ -0,0 +1,211 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import java.util.Properties +import java.util.concurrent.{ConcurrentHashMap, Executors} + +import scala.collection.Map +import scala.collection.mutable +import scala.reflect.{classTag, ClassTag} + +import kafka.common.TopicAndPartition +import kafka.consumer.{Consumer, ConsumerConfig, ConsumerConnector} +import kafka.serializer.Decoder +import kafka.utils.{ZkUtils, ZKGroupTopicDirs, ZKStringSerializer, VerifiableProperties} +import org.I0Itec.zkclient.ZkClient + +import org.apache.spark.{SparkEnv, Logging} +import org.apache.spark.storage.{StreamBlockId, StorageLevel} +import org.apache.spark.streaming.receiver.{BlockGeneratorListener, BlockGenerator, Receiver} + +private[streaming] +class ReliableKafkaReceiver[ + K: ClassTag, + V: ClassTag, + U <: Decoder[_]: ClassTag, + T <: Decoder[_]: ClassTag]( + kafkaParams: Map[String, String], + topics: Map[String, Int], + storageLevel: StorageLevel) + extends Receiver[Any](storageLevel) with Logging { + + /** High level consumer to connect to Kafka */ + private var consumerConnector: ConsumerConnector = null + + /** zkClient to connect to Zookeeper to commit the offsets */ + private var zkClient: ZkClient = null + + private val groupId = kafkaParams("group.id") + + private lazy val env = SparkEnv.get + + private val AUTO_OFFSET_COMMIT = "auto.commit.enable" + + /** A HashMap to manage the offset for each topic/partition, this HashMap is called in + * synchronized block, so mutable HashMap will not meet concurrency issue */ + private lazy val topicPartitionOffsetMap = new mutable.HashMap[TopicAndPartition, Long] + + /** A concurrent HashMap to store the stream block id and related offset snapshot */ + private lazy val blockOffsetMap = + new ConcurrentHashMap[StreamBlockId, Map[TopicAndPartition, Long]] + + private lazy val blockGeneratorListener = new BlockGeneratorListener { + override def onStoreData(data: Any, metadata: Any): Unit = { + if (metadata != null) { + val kafkaMetadata = metadata.asInstanceOf[(TopicAndPartition, Long)] + topicPartitionOffsetMap.put(kafkaMetadata._1, kafkaMetadata._2) + } + println(s"offset map: ${topicPartitionOffsetMap.mkString(":")}") + } + + override def onGenerateBlock(blockId: StreamBlockId): Unit = { + // Get a snapshot of current offset map and store with related block id. Since this hook + // function is called in synchronized block, so we can get the snapshot without explicit lock. + val offsetSnapshot = topicPartitionOffsetMap.toMap + blockOffsetMap.put(blockId, offsetSnapshot) + topicPartitionOffsetMap.clear() + println(s"block generated: $blockId, offset snapshot: ${offsetSnapshot.mkString(":")}") + } + + override def onPushBlock(blockId: StreamBlockId, arrayBuffer: mutable.ArrayBuffer[_]): Unit = { + // TODO. this should be replaced to reliable store after WAL is ready. + store(arrayBuffer.asInstanceOf[mutable.ArrayBuffer[Any]]) + + // Commit and remove the related offsets. + Option(blockOffsetMap.get(blockId)).foreach { offsetMap => + commitOffset(offsetMap) + } + blockOffsetMap.remove(blockId) + } + + override def onError(message: String, throwable: Throwable): Unit = { + reportError(message, throwable) + } + } + + /** Manage the BlockGenerator in receiver itself for better managing block store and offset + * commit */ + @volatile private lazy val blockGenerator = + new BlockGenerator(blockGeneratorListener, streamId, env.conf) + + override def onStop(): Unit = { + if (consumerConnector != null) { + consumerConnector.shutdown() + consumerConnector = null + } + + if (zkClient != null) { + zkClient.close() + zkClient = null + } + + blockGenerator.stop() + } + + override def onStart(): Unit = { + logInfo(s"Starting Kafka Consumer Stream with group: $groupId") + + if (kafkaParams.contains(AUTO_OFFSET_COMMIT) && kafkaParams(AUTO_OFFSET_COMMIT) == "true") { + logWarning(s"$AUTO_OFFSET_COMMIT should be set to false in ReliableKafkaReceiver, " + + "otherwise we cannot enable reliable offset commit mechanism") + } + + val props = new Properties() + kafkaParams.foreach(param => props.put(param._1, param._2)) + // Manually set "auto.commit.enable" to "false" no matter user explicitly set it to true, + // we have to make sure this property is set to false to turn off auto commit mechanism in + // Kafka. + props.setProperty(AUTO_OFFSET_COMMIT, "false") + + val consumerConfig = new ConsumerConfig(props) + logInfo(s"Connecting to Zookeeper: ${consumerConfig.zkConnect}") + consumerConnector = Consumer.create(consumerConfig) + logInfo(s"Connected to Zookeeper: ${consumerConfig.zkConnect}") + + zkClient = new ZkClient(consumerConfig.zkConnect, consumerConfig.zkSessionTimeoutMs, + consumerConfig.zkConnectionTimeoutMs, ZKStringSerializer) + + // start BlockGenerator + blockGenerator.start() + + val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties]) + .newInstance(consumerConfig.props) + .asInstanceOf[Decoder[K]] + + val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties]) + .newInstance(consumerConfig.props) + .asInstanceOf[Decoder[V]] + + val topicMessageStreams = consumerConnector.createMessageStreams( + topics, keyDecoder, valueDecoder) + + val executorPool = Executors.newFixedThreadPool(topics.values.sum) + + try { + topicMessageStreams.values.foreach { streams => + streams.foreach { stream => + executorPool.submit(new Runnable { + override def run(): Unit = { + logInfo(s"Starting message process thread ${Thread.currentThread().getId}.") + try { + for (msgAndMetadata <- stream) { + val topicAndPartition = TopicAndPartition( + msgAndMetadata.topic, msgAndMetadata.partition) + val metadata = (topicAndPartition, msgAndMetadata.offset) + + blockGenerator += ((msgAndMetadata.key, msgAndMetadata.message), metadata) + } + } catch { + case e: Throwable => logError("Error handling message; existing", e) + } + } + }) + } + } + } finally { + executorPool.shutdown() + } + } + + /** + * Commit the offset of Kafka's topic/partition, the commit mechanism follow Kafka 0.8.x's + * metadata schema in Zookeeper. + */ + private def commitOffset(offsetMap: Map[TopicAndPartition, Long]): Unit = { + if (zkClient == null) { + logError(s"zkClient $zkClient should be initialized at started") + return + } + + for ((topicAndPart, offset) <- offsetMap) { + try { + val topicDirs = new ZKGroupTopicDirs(groupId, topicAndPart.topic) + val zkPath = s"${topicDirs.consumerOffsetDir}/${topicAndPart.partition}" + + ZkUtils.updatePersistentPath(zkClient, zkPath, offset.toString) + } catch { + case t: Throwable => logWarning(s"Exception during commit offset $offset for topic" + + s"${topicAndPart.topic}, partition ${topicAndPart.partition}", t) + } + + println(s"Committed offset ${offset} for topic ${topicAndPart.topic}, " + + s"partition ${topicAndPart.partition}") + } + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala index 0316b6862f195..6e214d7d28a9a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala @@ -27,8 +27,17 @@ import org.apache.spark.streaming.util.{RecurringTimer, SystemClock} /** Listener object for BlockGenerator events */ private[streaming] trait BlockGeneratorListener { + /** Called when new data is added into BlockGenerator, this hook function will be called each + * time new data is added into BlockGenerator, any heavy or blocking operation will hurt the + * throughput */ + def onStoreData(data: Any, metadata: Any) + + /** Called when a new block is generated */ + def onGenerateBlock(blockId: StreamBlockId) + /** Called when a new block needs to be pushed */ def onPushBlock(blockId: StreamBlockId, arrayBuffer: ArrayBuffer[_]) + /** Called when an error has occurred in BlockGenerator */ def onError(message: String, throwable: Throwable) } @@ -80,9 +89,10 @@ private[streaming] class BlockGenerator( * Push a single data item into the buffer. All received data items * will be periodically pushed into BlockManager. */ - def += (data: Any): Unit = synchronized { + def += (data: Any, metadata: Any = null): Unit = synchronized { waitToPush() currentBuffer += data + listener.onStoreData(data, metadata) } /** Change the buffer to which single records are added to. */ @@ -93,6 +103,7 @@ private[streaming] class BlockGenerator( if (newBlockBuffer.size > 0) { val blockId = StreamBlockId(receiverId, time - blockInterval) val newBlock = new Block(blockId, newBlockBuffer) + listener.onGenerateBlock(blockId) blocksForPushing.put(newBlock) // put is blocking when queue is full logDebug("Last element in " + blockId + " is " + newBlockBuffer.last) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index 5360412330d37..5f1784a532573 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -99,6 +99,10 @@ private[streaming] class ReceiverSupervisorImpl( /** Divides received data records into data blocks for pushing in BlockManager. */ private val blockGenerator = new BlockGenerator(new BlockGeneratorListener { + def onStoreData(data: Any, metadata: Any): Unit = { } + + def onGenerateBlock(blockId: StreamBlockId): Unit = { } + def onError(message: String, throwable: Throwable) { reportError(message, throwable) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala index 0f6a9489dbe0d..5133590d36c2e 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala @@ -299,6 +299,10 @@ class ReceiverSuite extends FunSuite with Timeouts { val arrayBuffers = new ArrayBuffer[ArrayBuffer[Int]] val errors = new ArrayBuffer[Throwable] + def onStoreData(data: Any, metadata: Any) { } + + def onGenerateBlock(blockId: StreamBlockId) { } + def onPushBlock(blockId: StreamBlockId, arrayBuffer: ArrayBuffer[_]) { val bufferOfInts = arrayBuffer.map(_.asInstanceOf[Int]) arrayBuffers += bufferOfInts From 77c3e5064f72c259cc497aa06084da358145cf16 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Mon, 27 Oct 2014 16:55:13 +0800 Subject: [PATCH 02/21] Code refactor and add some unit tests --- .../streaming/kafka/KafkaInputDStream.scala | 4 +- .../kafka/ReliableKafkaReceiver.scala | 10 +- .../streaming/kafka/KafkaStreamSuite.scala | 27 ++- .../kafka/ReliableKafkaStreamSuite.scala | 184 ++++++++++++++++++ 4 files changed, 213 insertions(+), 12 deletions(-) create mode 100644 external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala index 92dd3f878a71f..5a8f4a07d0a3d 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala @@ -47,12 +47,12 @@ class KafkaInputDStream[ @transient ssc_ : StreamingContext, kafkaParams: Map[String, String], topics: Map[String, Int], - reliableStoreEnabled: Boolean, + reliableReceiveEnabled: Boolean, storageLevel: StorageLevel ) extends ReceiverInputDStream[(K, V)](ssc_) with Logging { def getReceiver(): Receiver[(K, V)] = { - if (!reliableStoreEnabled) { + if (!reliableReceiveEnabled) { new KafkaReceiver[K, V, U, T](kafkaParams, topics, storageLevel) .asInstanceOf[Receiver[(K, V)]] } else { diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala index 6773db67db24c..57a66f4e64fd5 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala @@ -71,7 +71,6 @@ class ReliableKafkaReceiver[ val kafkaMetadata = metadata.asInstanceOf[(TopicAndPartition, Long)] topicPartitionOffsetMap.put(kafkaMetadata._1, kafkaMetadata._2) } - println(s"offset map: ${topicPartitionOffsetMap.mkString(":")}") } override def onGenerateBlock(blockId: StreamBlockId): Unit = { @@ -80,7 +79,6 @@ class ReliableKafkaReceiver[ val offsetSnapshot = topicPartitionOffsetMap.toMap blockOffsetMap.put(blockId, offsetSnapshot) topicPartitionOffsetMap.clear() - println(s"block generated: $blockId, offset snapshot: ${offsetSnapshot.mkString(":")}") } override def onPushBlock(blockId: StreamBlockId, arrayBuffer: mutable.ArrayBuffer[_]): Unit = { @@ -101,8 +99,7 @@ class ReliableKafkaReceiver[ /** Manage the BlockGenerator in receiver itself for better managing block store and offset * commit */ - @volatile private lazy val blockGenerator = - new BlockGenerator(blockGeneratorListener, streamId, env.conf) + private lazy val blockGenerator = new BlockGenerator(blockGeneratorListener, streamId, env.conf) override def onStop(): Unit = { if (consumerConnector != null) { @@ -134,6 +131,9 @@ class ReliableKafkaReceiver[ props.setProperty(AUTO_OFFSET_COMMIT, "false") val consumerConfig = new ConsumerConfig(props) + + assert(consumerConfig.autoCommitEnable == false) + logInfo(s"Connecting to Zookeeper: ${consumerConfig.zkConnect}") consumerConnector = Consumer.create(consumerConfig) logInfo(s"Connected to Zookeeper: ${consumerConfig.zkConnect}") @@ -204,7 +204,7 @@ class ReliableKafkaReceiver[ s"${topicAndPart.topic}, partition ${topicAndPart.partition}", t) } - println(s"Committed offset ${offset} for topic ${topicAndPart.topic}, " + + logInfo(s"Committed offset ${offset} for topic ${topicAndPart.topic}, " + s"partition ${topicAndPart.partition}") } } diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala index 6943326eb750e..cf58b5cb70cd6 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala @@ -93,12 +93,27 @@ class KafkaStreamSuite extends TestSuiteBase { } override def afterFunction() { - producer.close() - server.shutdown() + if (producer != null) { + producer.close() + producer = null + } + + if (server != null) { + server.shutdown() + server = null + } + brokerConf.logDirs.foreach { f => Utils.deleteRecursively(new File(f)) } - zkClient.close() - zookeeper.shutdown() + if (zkClient != null) { + zkClient.close() + zkClient = null + } + + if (zookeeper != null) { + zookeeper.shutdown() + zookeeper = null + } super.afterFunction() } @@ -155,7 +170,9 @@ class KafkaStreamSuite extends TestSuiteBase { def produceAndSendMessage(topic: String, sent: Map[String, Int]) { val brokerAddr = brokerConf.hostName + ":" + brokerConf.port - producer = new Producer[String, String](new ProducerConfig(getProducerConfig(brokerAddr))) + if (producer == null) { + producer = new Producer[String, String](new ProducerConfig(getProducerConfig(brokerAddr))) + } producer.send(createTestMessage(topic, sent): _*) logInfo("==================== 6 ====================") } diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala new file mode 100644 index 0000000000000..0cf2752ebdb4d --- /dev/null +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import scala.collection.mutable + +import kafka.serializer.StringDecoder +import kafka.utils.{ZkUtils, ZKGroupTopicDirs} + +import org.apache.spark.SparkConf +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.StreamingContext + +class ReliableKafkaStreamSuite extends KafkaStreamSuite { + import KafkaTestUtils._ + + test("Reliable Kafka input stream") { + val ssc = new StreamingContext(master, framework, batchDuration) + val topic = "test" + val sent = Map("a" -> 1, "b" -> 1, "c" -> 1) + createTopic(topic) + produceAndSendMessage(topic, sent) + + val kafkaParams = Map("zookeeper.connect" -> s"$zkHost:$zkPort", + "group.id" -> s"test-consumer-${random.nextInt(10000)}", + "auto.offset.reset" -> "smallest") + + val stream = KafkaUtils.createReliableStream[String, String, StringDecoder, StringDecoder]( + ssc, + kafkaParams, + Map(topic -> 1), + StorageLevel.MEMORY_ONLY) + val result = new mutable.HashMap[String, Long]() + stream.map { case (k, v) => v } + .foreachRDD { r => + val ret = r.collect() + ret.foreach { v => + val count = result.getOrElseUpdate(v, 0) + 1 + result.put(v, count) + } + } + ssc.start() + ssc.awaitTermination(3000) + + assert(sent.size === result.size) + sent.keys.foreach { k => assert(sent(k) === result(k).toInt) } + + ssc.stop() + } + + test("Verify the offset commit") { + val ssc = new StreamingContext(master, framework, batchDuration) + val topic = "test" + val sent = Map("a" -> 10, "b" -> 10, "c" -> 10) + createTopic(topic) + produceAndSendMessage(topic, sent) + + val groupId = s"test-consumer-${random.nextInt(10000)}" + + val kafkaParams = Map("zookeeper.connect" -> s"$zkHost:$zkPort", + "group.id" -> groupId, + "auto.offset.reset" -> "smallest") + + assert(getCommitOffset(groupId, topic, 0) === 0L) + + val stream = KafkaUtils.createReliableStream[String, String, StringDecoder, StringDecoder]( + ssc, + kafkaParams, + Map(topic -> 1), + StorageLevel.MEMORY_ONLY) + stream.foreachRDD(_ => Unit) + ssc.start() + ssc.awaitTermination(3000) + ssc.stop() + + assert(getCommitOffset(groupId, topic, 0) === 29L) + } + + test("Verify multiple topics offset commit") { + val ssc = new StreamingContext(master, framework, batchDuration) + val topics = Map("topic1" -> 1, "topic2" -> 1, "topic3" -> 1) + val sent = Map("a" -> 10, "b" -> 10, "c" -> 10) + topics.foreach { case (t, _) => + createTopic(t) + produceAndSendMessage(t, sent) + } + + val groupId = s"test-consumer-${random.nextInt(10000)}" + + val kafkaParams = Map("zookeeper.connect" -> s"$zkHost:$zkPort", + "group.id" -> groupId, + "auto.offset.reset" -> "smallest") + + topics.foreach { case (t, _) => assert(getCommitOffset(groupId, t, 0) === 0L) } + + val stream = KafkaUtils.createReliableStream[String, String, StringDecoder, StringDecoder]( + ssc, + kafkaParams, + topics, + StorageLevel.MEMORY_ONLY) + stream.foreachRDD(_ => Unit) + ssc.start() + ssc.awaitTermination(3000) + ssc.stop() + + topics.foreach { case (t, _) => assert(getCommitOffset(groupId, t, 0) === 29L) } + } + + test("Verify offset commit when exception is met") { + val sparkConf = new SparkConf() + .setMaster(master) + .setAppName(framework) + var ssc = new StreamingContext( + sparkConf.clone.set("spark.streaming.blockInterval", "4000"), + batchDuration) + val topics = Map("topic1" -> 1, "topic2" -> 1, "topic3" -> 1) + val sent = Map("a" -> 10, "b" -> 10, "c" -> 10) + topics.foreach { case (t, _) => + createTopic(t) + produceAndSendMessage(t, sent) + } + + val groupId = s"test-consumer-${random.nextInt(10000)}" + + val kafkaParams = Map("zookeeper.connect" -> s"$zkHost:$zkPort", + "group.id" -> groupId, + "auto.offset.reset" -> "smallest") + + KafkaUtils.createReliableStream[String, String, StringDecoder, StringDecoder]( + ssc, + kafkaParams, + topics, + StorageLevel.MEMORY_ONLY).foreachRDD(_ => throw new Exception) + try { + ssc.start() + ssc.awaitTermination(1000) + } catch { + case e: Exception => + if (ssc != null) { + ssc.stop() + ssc = null + } + } + // Failed before putting to BM, so offset is not updated. + topics.foreach { case (t, _) => assert(getCommitOffset(groupId, t, 0) === 0L) } + + // Restart to see if data is consumed from last checkpoint. + ssc = new StreamingContext(sparkConf, batchDuration) + KafkaUtils.createReliableStream[String, String, StringDecoder, StringDecoder]( + ssc, + kafkaParams, + topics, + StorageLevel.MEMORY_ONLY).foreachRDD(_ => Unit) + ssc.start() + ssc.awaitTermination(3000) + ssc.stop() + + topics.foreach { case (t, _) => assert(getCommitOffset(groupId, t, 0) === 29L) } + } + + private def getCommitOffset(groupId: String, topic: String, partition: Int): Long = { + assert(zkClient != null, "Zookeeper client is not initialized") + + val topicDirs = new ZKGroupTopicDirs(groupId, topic) + val zkPath = s"${topicDirs.consumerOffsetDir}/$partition" + + ZkUtils.readDataMaybeNull(zkClient, zkPath)._1.map(_.toLong).getOrElse(0L) + } +} From 0894aefaccf3e095107896b86964dcf628b66792 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 28 Oct 2014 10:56:18 +0800 Subject: [PATCH 03/21] Add some comments --- .../spark/streaming/kafka/KafkaUtils.scala | 53 ++++++++++++++++++- 1 file changed, 51 insertions(+), 2 deletions(-) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala index 2eabdd170ddb6..9077d1fa0d549 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -99,7 +99,6 @@ object KafkaUtils { * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed * in its own thread. * @param storageLevel RDD storage level. - * */ def createStream( jssc: JavaStreamingContext, @@ -145,6 +144,16 @@ object KafkaUtils { jssc.ssc, kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) } + /** + * Create an reliable input stream that pulls messages from a Kafka Broker. + * @param ssc StreamingContext object + * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..) + * @param groupId The group id for this consumer + * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * in its own thread + * @param storageLevel Storage level to use for storing the received objects + * (default: StorageLevel.MEMORY_AND_DISK_SER_2) + */ def createReliableStream( ssc: StreamingContext, zkQuorum: String, @@ -159,6 +168,15 @@ object KafkaUtils { ssc, kafkaParams, topics, storageLevel) } + /** + * Create an reliable input stream that pulls messages from a Kafka Broker. + * @param ssc StreamingContext object + * @param kafkaParams Map of kafka configuration parameters, + * see http://kafka.apache.org/08/configuration.html + * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * in its own thread. + * @param storageLevel Storage level to use for storing the received objects + */ def createReliableStream[ K: ClassTag, V: ClassTag, @@ -172,7 +190,16 @@ object KafkaUtils { new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, true, storageLevel) } - def createReliableStream( + /** + * Create an reliable Java input stream that pulls messages form a Kafka Broker. + * Storage level of the data will be the default StorageLevel.MEMORY_AND_DISK_SER_2. + * @param jssc JavaStreamingContext object + * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..) + * @param groupId The group id for this consumer + * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * in its own thread + */ + def createReliableStream( jssc: JavaStreamingContext, zkQuorum: String, groupId: String, @@ -181,6 +208,15 @@ object KafkaUtils { createReliableStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*)) } + /** + * Create an reliable Java input stream that pulls messages form a Kafka Broker. + * @param jssc JavaStreamingContext object + * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..). + * @param groupId The group id for this consumer. + * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * in its own thread. + * @param storageLevel RDD storage level. + */ def createReliableStream( jssc: JavaStreamingContext, zkQuorum: String, @@ -192,6 +228,19 @@ object KafkaUtils { storageLevel) } + /** + * Create an reliable Java input stream that pulls messages form a Kafka Broker. + * @param jssc JavaStreamingContext object + * @param keyTypeClass Key type of RDD + * @param valueTypeClass value type of RDD + * @param keyDecoderClass Type of kafka key decoder + * @param valueDecoderClass Type of kafka value decoder + * @param kafkaParams Map of kafka configuration parameters, + * see http://kafka.apache.org/08/configuration.html + * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * in its own thread + * @param storageLevel RDD storage level. + */ def createReliableStream[K, V, U <: Decoder[_], T <: Decoder[_]]( jssc: JavaStreamingContext, keyTypeClass: Class[K], From 16bfe78e2d4de696c0b1634ec4e255a83dfca019 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 28 Oct 2014 13:38:23 +0800 Subject: [PATCH 04/21] Change the ordering of imports --- .../org/apache/spark/streaming/kafka/KafkaReceiver.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaReceiver.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaReceiver.scala index fa3b2458904b6..f686fbca1a509 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaReceiver.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaReceiver.scala @@ -17,12 +17,12 @@ package org.apache.spark.streaming.kafka -import scala.collection.Map -import scala.reflect.{classTag, ClassTag} - import java.util.Properties import java.util.concurrent.Executors +import scala.collection.Map +import scala.reflect.{classTag, ClassTag} + import kafka.consumer._ import kafka.serializer.Decoder import kafka.utils.VerifiableProperties From a9497415219d0083ffdd1e4bff71e05a8d3212d9 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Wed, 29 Oct 2014 13:08:50 +0800 Subject: [PATCH 05/21] Address the comments --- .../spark/streaming/kafka/KafkaUtils.scala | 120 +----------------- .../kafka/ReliableKafkaReceiver.scala | 3 +- .../kafka/ReliableKafkaStreamSuite.scala | 29 +++-- 3 files changed, 24 insertions(+), 128 deletions(-) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala index 9077d1fa0d549..c7b3c4d33f796 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -70,7 +70,8 @@ object KafkaUtils { topics: Map[String, Int], storageLevel: StorageLevel ): ReceiverInputDStream[(K, V)] = { - new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, false, storageLevel) + val WALEnabled = ssc.conf.getBoolean("spark.streaming.receiver.writeAheadLog.enable", false) + new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, WALEnabled, storageLevel) } /** @@ -143,121 +144,4 @@ object KafkaUtils { createStream[K, V, U, T]( jssc.ssc, kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) } - - /** - * Create an reliable input stream that pulls messages from a Kafka Broker. - * @param ssc StreamingContext object - * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..) - * @param groupId The group id for this consumer - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread - * @param storageLevel Storage level to use for storing the received objects - * (default: StorageLevel.MEMORY_AND_DISK_SER_2) - */ - def createReliableStream( - ssc: StreamingContext, - zkQuorum: String, - groupId: String, - topics: Map[String, Int], - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2) - : ReceiverInputDStream[(String, String)] = { - val kafkaParams = Map[String, String]( - "zookeeper.connect" -> zkQuorum, "group.id" -> groupId, - "zookeeper.connection.timeout.ms" -> "10000") - createReliableStream[String, String, StringDecoder, StringDecoder]( - ssc, kafkaParams, topics, storageLevel) - } - - /** - * Create an reliable input stream that pulls messages from a Kafka Broker. - * @param ssc StreamingContext object - * @param kafkaParams Map of kafka configuration parameters, - * see http://kafka.apache.org/08/configuration.html - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread. - * @param storageLevel Storage level to use for storing the received objects - */ - def createReliableStream[ - K: ClassTag, - V: ClassTag, - U <: Decoder[_]: ClassTag, - T <: Decoder[_]: ClassTag]( - ssc: StreamingContext, - kafkaParams: Map[String, String], - topics: Map[String, Int], - storageLevel: StorageLevel - ): ReceiverInputDStream[(K, V)] = { - new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, true, storageLevel) - } - - /** - * Create an reliable Java input stream that pulls messages form a Kafka Broker. - * Storage level of the data will be the default StorageLevel.MEMORY_AND_DISK_SER_2. - * @param jssc JavaStreamingContext object - * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..) - * @param groupId The group id for this consumer - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread - */ - def createReliableStream( - jssc: JavaStreamingContext, - zkQuorum: String, - groupId: String, - topics: JMap[String, JInt] - ): JavaPairReceiverInputDStream[String, String] = { - createReliableStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*)) - } - - /** - * Create an reliable Java input stream that pulls messages form a Kafka Broker. - * @param jssc JavaStreamingContext object - * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..). - * @param groupId The group id for this consumer. - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread. - * @param storageLevel RDD storage level. - */ - def createReliableStream( - jssc: JavaStreamingContext, - zkQuorum: String, - groupId: String, - topics: JMap[String, JInt], - storageLevel: StorageLevel - ): JavaPairReceiverInputDStream[String, String] = { - createReliableStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), - storageLevel) - } - - /** - * Create an reliable Java input stream that pulls messages form a Kafka Broker. - * @param jssc JavaStreamingContext object - * @param keyTypeClass Key type of RDD - * @param valueTypeClass value type of RDD - * @param keyDecoderClass Type of kafka key decoder - * @param valueDecoderClass Type of kafka value decoder - * @param kafkaParams Map of kafka configuration parameters, - * see http://kafka.apache.org/08/configuration.html - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread - * @param storageLevel RDD storage level. - */ - def createReliableStream[K, V, U <: Decoder[_], T <: Decoder[_]]( - jssc: JavaStreamingContext, - keyTypeClass: Class[K], - valueTypeClass: Class[V], - keyDecoderClass: Class[U], - valueDecoderClass: Class[T], - kafkaParams: JMap[String, String], - topics: JMap[String, JInt], - storageLevel: StorageLevel - ): JavaPairReceiverInputDStream[K, V] = { - implicit val keyCmt: ClassTag[K] = ClassTag(keyTypeClass) - implicit val valueCmt: ClassTag[V] = ClassTag(valueTypeClass) - - implicit val keyCmd: ClassTag[U] = ClassTag(keyDecoderClass) - implicit val valueCmd: ClassTag[T] = ClassTag(valueDecoderClass) - - createReliableStream[K, V, U, T]( - jssc.ssc, kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) - } } diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala index 57a66f4e64fd5..50bcd44ea5605 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala @@ -82,7 +82,6 @@ class ReliableKafkaReceiver[ } override def onPushBlock(blockId: StreamBlockId, arrayBuffer: mutable.ArrayBuffer[_]): Unit = { - // TODO. this should be replaced to reliable store after WAL is ready. store(arrayBuffer.asInstanceOf[mutable.ArrayBuffer[Any]]) // Commit and remove the related offsets. @@ -120,7 +119,7 @@ class ReliableKafkaReceiver[ if (kafkaParams.contains(AUTO_OFFSET_COMMIT) && kafkaParams(AUTO_OFFSET_COMMIT) == "true") { logWarning(s"$AUTO_OFFSET_COMMIT should be set to false in ReliableKafkaReceiver, " + - "otherwise we cannot enable reliable offset commit mechanism") + "otherwise we will manually set it to false to turn off auto offset commit in Kafka") } val props = new Properties() diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala index 0cf2752ebdb4d..0aa49987ea23a 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala @@ -30,7 +30,11 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuite { import KafkaTestUtils._ test("Reliable Kafka input stream") { - val ssc = new StreamingContext(master, framework, batchDuration) + val sparkConf = new SparkConf() + .setMaster(master) + .setAppName(framework) + .set("spark.streaming.receiver.writeAheadLog.enable", "true") + val ssc = new StreamingContext(sparkConf, batchDuration) val topic = "test" val sent = Map("a" -> 1, "b" -> 1, "c" -> 1) createTopic(topic) @@ -40,7 +44,7 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuite { "group.id" -> s"test-consumer-${random.nextInt(10000)}", "auto.offset.reset" -> "smallest") - val stream = KafkaUtils.createReliableStream[String, String, StringDecoder, StringDecoder]( + val stream = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( ssc, kafkaParams, Map(topic -> 1), @@ -64,7 +68,11 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuite { } test("Verify the offset commit") { - val ssc = new StreamingContext(master, framework, batchDuration) + val sparkConf = new SparkConf() + .setMaster(master) + .setAppName(framework) + .set("spark.streaming.receiver.writeAheadLog.enable", "true") + val ssc = new StreamingContext(sparkConf, batchDuration) val topic = "test" val sent = Map("a" -> 10, "b" -> 10, "c" -> 10) createTopic(topic) @@ -78,7 +86,7 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuite { assert(getCommitOffset(groupId, topic, 0) === 0L) - val stream = KafkaUtils.createReliableStream[String, String, StringDecoder, StringDecoder]( + val stream = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( ssc, kafkaParams, Map(topic -> 1), @@ -92,7 +100,11 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuite { } test("Verify multiple topics offset commit") { - val ssc = new StreamingContext(master, framework, batchDuration) + val sparkConf = new SparkConf() + .setMaster(master) + .setAppName(framework) + .set("spark.streaming.receiver.writeAheadLog.enable", "true") + val ssc = new StreamingContext(sparkConf, batchDuration) val topics = Map("topic1" -> 1, "topic2" -> 1, "topic3" -> 1) val sent = Map("a" -> 10, "b" -> 10, "c" -> 10) topics.foreach { case (t, _) => @@ -108,7 +120,7 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuite { topics.foreach { case (t, _) => assert(getCommitOffset(groupId, t, 0) === 0L) } - val stream = KafkaUtils.createReliableStream[String, String, StringDecoder, StringDecoder]( + val stream = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( ssc, kafkaParams, topics, @@ -125,6 +137,7 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuite { val sparkConf = new SparkConf() .setMaster(master) .setAppName(framework) + .set("spark.streaming.receiver.writeAheadLog.enable", "true") var ssc = new StreamingContext( sparkConf.clone.set("spark.streaming.blockInterval", "4000"), batchDuration) @@ -141,7 +154,7 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuite { "group.id" -> groupId, "auto.offset.reset" -> "smallest") - KafkaUtils.createReliableStream[String, String, StringDecoder, StringDecoder]( + KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( ssc, kafkaParams, topics, @@ -161,7 +174,7 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuite { // Restart to see if data is consumed from last checkpoint. ssc = new StreamingContext(sparkConf, batchDuration) - KafkaUtils.createReliableStream[String, String, StringDecoder, StringDecoder]( + KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( ssc, kafkaParams, topics, From 8135d31915053e9dc3218f5d242ce894407a61e3 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Wed, 29 Oct 2014 15:02:46 +0800 Subject: [PATCH 06/21] Fix flaky test --- .../kafka/ReliableKafkaReceiver.scala | 4 +++- .../kafka/ReliableKafkaStreamSuite.scala | 20 ++++++++++--------- 2 files changed, 14 insertions(+), 10 deletions(-) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala index 50bcd44ea5605..d832a82aacf09 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala @@ -98,7 +98,7 @@ class ReliableKafkaReceiver[ /** Manage the BlockGenerator in receiver itself for better managing block store and offset * commit */ - private lazy val blockGenerator = new BlockGenerator(blockGeneratorListener, streamId, env.conf) + private var blockGenerator: BlockGenerator = null override def onStop(): Unit = { if (consumerConnector != null) { @@ -117,6 +117,8 @@ class ReliableKafkaReceiver[ override def onStart(): Unit = { logInfo(s"Starting Kafka Consumer Stream with group: $groupId") + blockGenerator = new BlockGenerator(blockGeneratorListener, streamId, env.conf) + if (kafkaParams.contains(AUTO_OFFSET_COMMIT) && kafkaParams(AUTO_OFFSET_COMMIT) == "true") { logWarning(s"$AUTO_OFFSET_COMMIT should be set to false in ReliableKafkaReceiver, " + "otherwise we will manually set it to false to turn off auto offset commit in Kafka") diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala index 0aa49987ea23a..83a88597e57c7 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala @@ -139,7 +139,7 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuite { .setAppName(framework) .set("spark.streaming.receiver.writeAheadLog.enable", "true") var ssc = new StreamingContext( - sparkConf.clone.set("spark.streaming.blockInterval", "4000"), + sparkConf.clone.set("spark.streaming.blockInterval", "10000"), batchDuration) val topics = Map("topic1" -> 1, "topic2" -> 1, "topic3" -> 1) val sent = Map("a" -> 10, "b" -> 10, "c" -> 10) @@ -155,10 +155,11 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuite { "auto.offset.reset" -> "smallest") KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( - ssc, - kafkaParams, - topics, - StorageLevel.MEMORY_ONLY).foreachRDD(_ => throw new Exception) + ssc, + kafkaParams, + topics, + StorageLevel.MEMORY_ONLY) + .foreachRDD(_ => throw new Exception) try { ssc.start() ssc.awaitTermination(1000) @@ -175,10 +176,11 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuite { // Restart to see if data is consumed from last checkpoint. ssc = new StreamingContext(sparkConf, batchDuration) KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( - ssc, - kafkaParams, - topics, - StorageLevel.MEMORY_ONLY).foreachRDD(_ => Unit) + ssc, + kafkaParams, + topics, + StorageLevel.MEMORY_ONLY) + .foreachRDD(_ => Unit) ssc.start() ssc.awaitTermination(3000) ssc.stop() From 96c7a1dd97a93d2abb4daad99b7ec24e5114356a Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 31 Oct 2014 13:31:07 +0800 Subject: [PATCH 07/21] Update the ReliableKafkaReceiver unit test --- .../kafka/ReliableKafkaStreamSuite.scala | 73 +++++-------------- 1 file changed, 18 insertions(+), 55 deletions(-) diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala index 83a88597e57c7..c2f318efa61b8 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.streaming.kafka +import java.io.File + import scala.collection.mutable import kafka.serializer.StringDecoder @@ -25,6 +27,7 @@ import kafka.utils.{ZkUtils, ZKGroupTopicDirs} import org.apache.spark.SparkConf import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext +import org.apache.spark.util.Utils class ReliableKafkaStreamSuite extends KafkaStreamSuite { import KafkaTestUtils._ @@ -35,6 +38,11 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuite { .setAppName(framework) .set("spark.streaming.receiver.writeAheadLog.enable", "true") val ssc = new StreamingContext(sparkConf, batchDuration) + val checkpointDir = s"${System.getProperty("java.io.tmpdir", "/tmp")}/" + + s"test-checkpoint${random.nextInt(10000)}" + Utils.registerShutdownDeleteDir(new File(checkpointDir)) + ssc.checkpoint(checkpointDir) + val topic = "test" val sent = Map("a" -> 1, "b" -> 1, "c" -> 1) createTopic(topic) @@ -73,6 +81,11 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuite { .setAppName(framework) .set("spark.streaming.receiver.writeAheadLog.enable", "true") val ssc = new StreamingContext(sparkConf, batchDuration) + val checkpointDir = s"${System.getProperty("java.io.tmpdir", "/tmp")}/" + + s"test-checkpoint${random.nextInt(10000)}" + Utils.registerShutdownDeleteDir(new File(checkpointDir)) + ssc.checkpoint(checkpointDir) + val topic = "test" val sent = Map("a" -> 10, "b" -> 10, "c" -> 10) createTopic(topic) @@ -105,6 +118,11 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuite { .setAppName(framework) .set("spark.streaming.receiver.writeAheadLog.enable", "true") val ssc = new StreamingContext(sparkConf, batchDuration) + val checkpointDir = s"${System.getProperty("java.io.tmpdir", "/tmp")}/" + + s"test-checkpoint${random.nextInt(10000)}" + Utils.registerShutdownDeleteDir(new File(checkpointDir)) + ssc.checkpoint(checkpointDir) + val topics = Map("topic1" -> 1, "topic2" -> 1, "topic3" -> 1) val sent = Map("a" -> 10, "b" -> 10, "c" -> 10) topics.foreach { case (t, _) => @@ -133,61 +151,6 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuite { topics.foreach { case (t, _) => assert(getCommitOffset(groupId, t, 0) === 29L) } } - test("Verify offset commit when exception is met") { - val sparkConf = new SparkConf() - .setMaster(master) - .setAppName(framework) - .set("spark.streaming.receiver.writeAheadLog.enable", "true") - var ssc = new StreamingContext( - sparkConf.clone.set("spark.streaming.blockInterval", "10000"), - batchDuration) - val topics = Map("topic1" -> 1, "topic2" -> 1, "topic3" -> 1) - val sent = Map("a" -> 10, "b" -> 10, "c" -> 10) - topics.foreach { case (t, _) => - createTopic(t) - produceAndSendMessage(t, sent) - } - - val groupId = s"test-consumer-${random.nextInt(10000)}" - - val kafkaParams = Map("zookeeper.connect" -> s"$zkHost:$zkPort", - "group.id" -> groupId, - "auto.offset.reset" -> "smallest") - - KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( - ssc, - kafkaParams, - topics, - StorageLevel.MEMORY_ONLY) - .foreachRDD(_ => throw new Exception) - try { - ssc.start() - ssc.awaitTermination(1000) - } catch { - case e: Exception => - if (ssc != null) { - ssc.stop() - ssc = null - } - } - // Failed before putting to BM, so offset is not updated. - topics.foreach { case (t, _) => assert(getCommitOffset(groupId, t, 0) === 0L) } - - // Restart to see if data is consumed from last checkpoint. - ssc = new StreamingContext(sparkConf, batchDuration) - KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( - ssc, - kafkaParams, - topics, - StorageLevel.MEMORY_ONLY) - .foreachRDD(_ => Unit) - ssc.start() - ssc.awaitTermination(3000) - ssc.stop() - - topics.foreach { case (t, _) => assert(getCommitOffset(groupId, t, 0) === 29L) } - } - private def getCommitOffset(groupId: String, topic: String, partition: Int): Long = { assert(zkClient != null, "Zookeeper client is not initialized") From 4854ee95b959dfac3242ce67c050cb4c2bfa1bba Mon Sep 17 00:00:00 2001 From: jerryshao Date: Mon, 10 Nov 2014 14:42:22 +0800 Subject: [PATCH 08/21] Address all the comments --- .../kafka/ReliableKafkaReceiver.scala | 101 +++++++++++------- .../kafka/ReliableKafkaStreamSuite.scala | 11 ++ 2 files changed, 75 insertions(+), 37 deletions(-) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala index d832a82aacf09..417b0464c1e86 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala @@ -18,14 +18,14 @@ package org.apache.spark.streaming.kafka import java.util.Properties -import java.util.concurrent.{ConcurrentHashMap, Executors} +import java.util.concurrent.ConcurrentHashMap import scala.collection.Map import scala.collection.mutable import scala.reflect.{classTag, ClassTag} import kafka.common.TopicAndPartition -import kafka.consumer.{Consumer, ConsumerConfig, ConsumerConnector} +import kafka.consumer.{Consumer, ConsumerConfig, ConsumerConnector, KafkaStream} import kafka.serializer.Decoder import kafka.utils.{ZkUtils, ZKGroupTopicDirs, ZKStringSerializer, VerifiableProperties} import org.I0Itec.zkclient.ZkClient @@ -33,6 +33,7 @@ import org.I0Itec.zkclient.ZkClient import org.apache.spark.{SparkEnv, Logging} import org.apache.spark.storage.{StreamBlockId, StorageLevel} import org.apache.spark.streaming.receiver.{BlockGeneratorListener, BlockGenerator, Receiver} +import org.apache.spark.util.Utils private[streaming] class ReliableKafkaReceiver[ @@ -45,27 +46,33 @@ class ReliableKafkaReceiver[ storageLevel: StorageLevel) extends Receiver[Any](storageLevel) with Logging { - /** High level consumer to connect to Kafka */ + /** High level consumer to connect to Kafka. */ private var consumerConnector: ConsumerConnector = null - /** zkClient to connect to Zookeeper to commit the offsets */ + /** zkClient to connect to Zookeeper to commit the offsets. */ private var zkClient: ZkClient = null private val groupId = kafkaParams("group.id") - private lazy val env = SparkEnv.get + private def conf() = SparkEnv.get.conf private val AUTO_OFFSET_COMMIT = "auto.commit.enable" /** A HashMap to manage the offset for each topic/partition, this HashMap is called in - * synchronized block, so mutable HashMap will not meet concurrency issue */ - private lazy val topicPartitionOffsetMap = new mutable.HashMap[TopicAndPartition, Long] + * synchronized block, so mutable HashMap will not meet concurrency issue. + */ + private var topicPartitionOffsetMap: mutable.HashMap[TopicAndPartition, Long] = null - /** A concurrent HashMap to store the stream block id and related offset snapshot */ - private lazy val blockOffsetMap = - new ConcurrentHashMap[StreamBlockId, Map[TopicAndPartition, Long]] + /** A concurrent HashMap to store the stream block id and related offset snapshot. */ + private var blockOffsetMap: ConcurrentHashMap[StreamBlockId, Map[TopicAndPartition, Long]] = null - private lazy val blockGeneratorListener = new BlockGeneratorListener { + /** Manage the BlockGenerator in receiver itself for better managing block store and offset + * commit. + */ + private var blockGenerator: BlockGenerator = null + + /** Kafka offsets checkpoint listener to register into BlockGenerator for offsets checkpoint. */ + private final class OffsetCheckpointListener extends BlockGeneratorListener { override def onStoreData(data: Any, metadata: Any): Unit = { if (metadata != null) { val kafkaMetadata = metadata.asInstanceOf[(TopicAndPartition, Long)] @@ -96,10 +103,6 @@ class ReliableKafkaReceiver[ } } - /** Manage the BlockGenerator in receiver itself for better managing block store and offset - * commit */ - private var blockGenerator: BlockGenerator = null - override def onStop(): Unit = { if (consumerConnector != null) { consumerConnector.shutdown() @@ -111,13 +114,33 @@ class ReliableKafkaReceiver[ zkClient = null } - blockGenerator.stop() + if (blockGenerator != null) { + blockGenerator.stop() + blockGenerator = null + } + + if (topicPartitionOffsetMap != null) { + topicPartitionOffsetMap.clear() + topicPartitionOffsetMap = null + } + + if (blockOffsetMap != null) { + blockOffsetMap.clear() + blockOffsetMap = null + } } override def onStart(): Unit = { logInfo(s"Starting Kafka Consumer Stream with group: $groupId") - blockGenerator = new BlockGenerator(blockGeneratorListener, streamId, env.conf) + // Initialize the topic-partition / offset hash map. + topicPartitionOffsetMap = new mutable.HashMap[TopicAndPartition, Long] + + // Initialize the stream block id / offset snapshot hash map. + blockOffsetMap = new ConcurrentHashMap[StreamBlockId, Map[TopicAndPartition, Long]]() + + // Initialize the block generator for storing Kafka message. + blockGenerator = new BlockGenerator(new OffsetCheckpointListener, streamId, conf()) if (kafkaParams.contains(AUTO_OFFSET_COMMIT) && kafkaParams(AUTO_OFFSET_COMMIT) == "true") { logWarning(s"$AUTO_OFFSET_COMMIT should be set to false in ReliableKafkaReceiver, " + @@ -133,7 +156,7 @@ class ReliableKafkaReceiver[ val consumerConfig = new ConsumerConfig(props) - assert(consumerConfig.autoCommitEnable == false) + assert(!consumerConfig.autoCommitEnable) logInfo(s"Connecting to Zookeeper: ${consumerConfig.zkConnect}") consumerConnector = Consumer.create(consumerConfig) @@ -156,27 +179,12 @@ class ReliableKafkaReceiver[ val topicMessageStreams = consumerConnector.createMessageStreams( topics, keyDecoder, valueDecoder) - val executorPool = Executors.newFixedThreadPool(topics.values.sum) + val executorPool = Utils.newDaemonFixedThreadPool(topics.values.sum, "KafkaMessageHandler") try { topicMessageStreams.values.foreach { streams => streams.foreach { stream => - executorPool.submit(new Runnable { - override def run(): Unit = { - logInfo(s"Starting message process thread ${Thread.currentThread().getId}.") - try { - for (msgAndMetadata <- stream) { - val topicAndPartition = TopicAndPartition( - msgAndMetadata.topic, msgAndMetadata.partition) - val metadata = (topicAndPartition, msgAndMetadata.offset) - - blockGenerator += ((msgAndMetadata.key, msgAndMetadata.message), metadata) - } - } catch { - case e: Throwable => logError("Error handling message; existing", e) - } - } - }) + executorPool.submit(new MessageHandler(stream)) } } } finally { @@ -184,13 +192,32 @@ class ReliableKafkaReceiver[ } } + /** A inner class to handle received Kafka message. */ + private final class MessageHandler(stream: KafkaStream[K, V]) extends Runnable { + override def run(): Unit = { + logInfo(s"Starting message process thread ${Thread.currentThread().getId}.") + try { + for (msgAndMetadata <- stream) { + val topicAndPartition = TopicAndPartition( + msgAndMetadata.topic, msgAndMetadata.partition) + val metadata = (topicAndPartition, msgAndMetadata.offset) + + blockGenerator += ((msgAndMetadata.key, msgAndMetadata.message), metadata) + } + } catch { + case e: Throwable => logError("Error handling message; existing", e) + } + } + } + /** * Commit the offset of Kafka's topic/partition, the commit mechanism follow Kafka 0.8.x's * metadata schema in Zookeeper. */ private def commitOffset(offsetMap: Map[TopicAndPartition, Long]): Unit = { if (zkClient == null) { - logError(s"zkClient $zkClient should be initialized at started") + val thrown = new IllegalStateException("Zookeeper client is unexpectedly null") + stop("Zookeeper client is not initialized before commit offsets to ZK", thrown) return } @@ -205,7 +232,7 @@ class ReliableKafkaReceiver[ s"${topicAndPart.topic}, partition ${topicAndPart.partition}", t) } - logInfo(s"Committed offset ${offset} for topic ${topicAndPart.topic}, " + + logInfo(s"Committed offset $offset for topic ${topicAndPart.topic}, " + s"partition ${topicAndPart.partition}") } } diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala index c2f318efa61b8..9a8557e496295 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala @@ -69,13 +69,17 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuite { ssc.start() ssc.awaitTermination(3000) + // A basic process verification for ReliableKafkaReceiver. + // Verify whether received message number is equal to the sent message number. assert(sent.size === result.size) + // Verify whether each message is the same as the data to be verified. sent.keys.foreach { k => assert(sent(k) === result(k).toInt) } ssc.stop() } test("Verify the offset commit") { + // Verify the corretness of offset commit mechanism. val sparkConf = new SparkConf() .setMaster(master) .setAppName(framework) @@ -97,8 +101,10 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuite { "group.id" -> groupId, "auto.offset.reset" -> "smallest") + // Verify whether the offset of this group/topic/partition is 0 before starting. assert(getCommitOffset(groupId, topic, 0) === 0L) + // Do this to consume all the message of this group/topic. val stream = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( ssc, kafkaParams, @@ -109,6 +115,7 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuite { ssc.awaitTermination(3000) ssc.stop() + // Verify the offset number whether it is equal to the total message number. assert(getCommitOffset(groupId, topic, 0) === 29L) } @@ -136,8 +143,10 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuite { "group.id" -> groupId, "auto.offset.reset" -> "smallest") + // Before started, verify all the group/topic/partition offsets are 0. topics.foreach { case (t, _) => assert(getCommitOffset(groupId, t, 0) === 0L) } + // Consuming all the data sent to the broker which will potential commit the offsets internally. val stream = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( ssc, kafkaParams, @@ -148,9 +157,11 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuite { ssc.awaitTermination(3000) ssc.stop() + // Verify the offset for each group/topic to see whether they are equal to the expected one. topics.foreach { case (t, _) => assert(getCommitOffset(groupId, t, 0) === 29L) } } + /** Getting partition offset from Zookeeper. */ private def getCommitOffset(groupId: String, topic: String, partition: Int): Long = { assert(zkClient != null, "Zookeeper client is not initialized") From 98f3d071c90bdf290af66c1e27b171fb6f8e0cbf Mon Sep 17 00:00:00 2001 From: jerryshao Date: Mon, 10 Nov 2014 15:35:30 +0800 Subject: [PATCH 09/21] Fix comment style --- .../streaming/kafka/ReliableKafkaReceiver.scala | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala index 417b0464c1e86..938fcc1fc6a32 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala @@ -58,17 +58,19 @@ class ReliableKafkaReceiver[ private val AUTO_OFFSET_COMMIT = "auto.commit.enable" - /** A HashMap to manage the offset for each topic/partition, this HashMap is called in - * synchronized block, so mutable HashMap will not meet concurrency issue. - */ + /** + * A HashMap to manage the offset for each topic/partition, this HashMap is called in + * synchronized block, so mutable HashMap will not meet concurrency issue. + */ private var topicPartitionOffsetMap: mutable.HashMap[TopicAndPartition, Long] = null /** A concurrent HashMap to store the stream block id and related offset snapshot. */ private var blockOffsetMap: ConcurrentHashMap[StreamBlockId, Map[TopicAndPartition, Long]] = null - /** Manage the BlockGenerator in receiver itself for better managing block store and offset - * commit. - */ + /** + * Manage the BlockGenerator in receiver itself for better managing block store and offset + * commit. + */ private var blockGenerator: BlockGenerator = null /** Kafka offsets checkpoint listener to register into BlockGenerator for offsets checkpoint. */ From ea873e424424747c8807adbc0c217a5fbf80e060 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Wed, 12 Nov 2014 11:04:21 +0800 Subject: [PATCH 10/21] Further address the comments --- .../streaming/kafka/KafkaInputDStream.scala | 26 ++-- .../spark/streaming/kafka/KafkaReceiver.scala | 135 ------------------ .../spark/streaming/kafka/KafkaUtils.scala | 4 +- .../kafka/ReliableKafkaReceiver.scala | 42 +++--- .../streaming/receiver/BlockGenerator.scala | 6 - .../receiver/ReceiverSupervisorImpl.scala | 2 - .../spark/streaming/ReceiverSuite.scala | 2 - 7 files changed, 42 insertions(+), 175 deletions(-) delete mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaReceiver.scala diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala index 5a8f4a07d0a3d..15ba9c8212aa1 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala @@ -17,9 +17,12 @@ package org.apache.spark.streaming.kafka +import java.util.Properties + import scala.collection.Map -import scala.reflect.ClassTag +import scala.reflect.{classTag, ClassTag} +import kafka.consumer.{KafkaStream, Consumer, ConsumerConfig, ConsumerConnector} import kafka.serializer.Decoder import kafka.utils.VerifiableProperties @@ -28,6 +31,7 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.dstream._ import org.apache.spark.streaming.receiver.Receiver +import org.apache.spark.util.Utils /** * Input stream that pulls messages from a Kafka Broker. @@ -47,17 +51,16 @@ class KafkaInputDStream[ @transient ssc_ : StreamingContext, kafkaParams: Map[String, String], topics: Map[String, Int], - reliableReceiveEnabled: Boolean, + useReliableReceiver: Boolean, storageLevel: StorageLevel ) extends ReceiverInputDStream[(K, V)](ssc_) with Logging { def getReceiver(): Receiver[(K, V)] = { - if (!reliableReceiveEnabled) { + if (!useReliableReceiver) { new KafkaReceiver[K, V, U, T](kafkaParams, topics, storageLevel) - .asInstanceOf[Receiver[(K, V)]] } else { new ReliableKafkaReceiver[K, V, U, T](kafkaParams, topics, storageLevel) - .asInstanceOf[Receiver[(K, V)]] + } } } @@ -70,14 +73,15 @@ class KafkaReceiver[ kafkaParams: Map[String, String], topics: Map[String, Int], storageLevel: StorageLevel - ) extends Receiver[Any](storageLevel) with Logging { + ) extends Receiver[(K, V)](storageLevel) with Logging { // Connection to Kafka - var consumerConnector : ConsumerConnector = null + var consumerConnector: ConsumerConnector = null def onStop() { if (consumerConnector != null) { consumerConnector.shutdown() + consumerConnector = null } } @@ -103,11 +107,11 @@ class KafkaReceiver[ .newInstance(consumerConfig.props) .asInstanceOf[Decoder[V]] - // Create Threads for each Topic/Message Stream we are listening + // Create threads for each topic/message Stream we are listening val topicMessageStreams = consumerConnector.createMessageStreams( topics, keyDecoder, valueDecoder) - val executorPool = Executors.newFixedThreadPool(topics.values.sum) + val executorPool = Utils.newDaemonFixedThreadPool(topics.values.sum, "KafkaMessageHandler") try { // Start the messages handler for each partition topicMessageStreams.values.foreach { streams => @@ -118,8 +122,8 @@ class KafkaReceiver[ } } - // Handles Kafka Messages - private class MessageHandler[K: ClassTag, V: ClassTag](stream: KafkaStream[K, V]) + // Handles Kafka messages + private class MessageHandler(stream: KafkaStream[K, V]) extends Runnable { def run() { logInfo("Starting MessageHandler.") diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaReceiver.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaReceiver.scala deleted file mode 100644 index f686fbca1a509..0000000000000 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaReceiver.scala +++ /dev/null @@ -1,135 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kafka - -import java.util.Properties -import java.util.concurrent.Executors - -import scala.collection.Map -import scala.reflect.{classTag, ClassTag} - -import kafka.consumer._ -import kafka.serializer.Decoder -import kafka.utils.VerifiableProperties -import kafka.utils.ZKStringSerializer -import org.I0Itec.zkclient._ - -import org.apache.spark.Logging -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.receiver.Receiver - -private[streaming] -class KafkaReceiver[ - K: ClassTag, - V: ClassTag, - U <: Decoder[_]: ClassTag, - T <: Decoder[_]: ClassTag]( - kafkaParams: Map[String, String], - topics: Map[String, Int], - storageLevel: StorageLevel - ) extends Receiver[Any](storageLevel) with Logging { - - // Connection to Kafka - var consumerConnector: ConsumerConnector = null - - def onStop() { - if (consumerConnector != null) { - consumerConnector.shutdown() - } - } - - def onStart() { - - logInfo("Starting Kafka Consumer Stream with group: " + kafkaParams("group.id")) - - // Kafka connection properties - val props = new Properties() - kafkaParams.foreach(param => props.put(param._1, param._2)) - - val zkConnect = kafkaParams("zookeeper.connect") - // Create the connection to the cluster - logInfo("Connecting to Zookeeper: " + zkConnect) - val consumerConfig = new ConsumerConfig(props) - consumerConnector = Consumer.create(consumerConfig) - logInfo("Connected to " + zkConnect) - - // When auto.offset.reset is defined, it is our responsibility to try and whack the - // consumer group zk node. - if (kafkaParams.contains("auto.offset.reset")) { - tryZookeeperConsumerGroupCleanup(zkConnect, kafkaParams("group.id")) - } - - val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties]) - .newInstance(consumerConfig.props) - .asInstanceOf[Decoder[K]] - val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties]) - .newInstance(consumerConfig.props) - .asInstanceOf[Decoder[V]] - - // Create Threads for each Topic/Message Stream we are listening - val topicMessageStreams = consumerConnector.createMessageStreams( - topics, keyDecoder, valueDecoder) - - val executorPool = Executors.newFixedThreadPool(topics.values.sum) - try { - // Start the messages handler for each partition - topicMessageStreams.values.foreach { streams => - streams.foreach { stream => executorPool.submit(new MessageHandler(stream)) } - } - } finally { - executorPool.shutdown() // Just causes threads to terminate after work is done - } - } - - // Handles Kafka Messages - private class MessageHandler[K: ClassTag, V: ClassTag](stream: KafkaStream[K, V]) - extends Runnable { - def run() { - logInfo("Starting MessageHandler.") - try { - for (msgAndMetadata <- stream) { - store((msgAndMetadata.key, msgAndMetadata.message)) - } - } catch { - case e: Throwable => logError("Error handling message; exiting", e) - } - } - } - - // It is our responsibility to delete the consumer group when specifying auto.offset.reset. This - // is because Kafka 0.7.2 only honors this param when the group is not in zookeeper. - // - // The kafka high level consumer doesn't expose setting offsets currently, this is a trick copied - // from Kafka's ConsoleConsumer. See code related to 'auto.offset.reset' when it is set to - // 'smallest'/'largest': - // scalastyle:off - // https://github.com/apache/kafka/blob/0.7.2/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala - // scalastyle:on - private def tryZookeeperConsumerGroupCleanup(zkUrl: String, groupId: String) { - val dir = "/consumers/" + groupId - logInfo("Cleaning up temporary Zookeeper data under " + dir + ".") - val zk = new ZkClient(zkUrl, 30*1000, 30*1000, ZKStringSerializer) - try { - zk.deleteRecursive(dir) - } catch { - case e: Throwable => logWarning("Error cleaning up temporary Zookeeper data", e) - } finally { - zk.close() - } - } -} \ No newline at end of file diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala index c7b3c4d33f796..b4ac929e0c070 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -70,8 +70,8 @@ object KafkaUtils { topics: Map[String, Int], storageLevel: StorageLevel ): ReceiverInputDStream[(K, V)] = { - val WALEnabled = ssc.conf.getBoolean("spark.streaming.receiver.writeAheadLog.enable", false) - new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, WALEnabled, storageLevel) + val walEnabled = ssc.conf.getBoolean("spark.streaming.receiver.writeAheadLog.enable", false) + new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, walEnabled, storageLevel) } /** diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala index 938fcc1fc6a32..4a2e414365715 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala @@ -35,6 +35,19 @@ import org.apache.spark.storage.{StreamBlockId, StorageLevel} import org.apache.spark.streaming.receiver.{BlockGeneratorListener, BlockGenerator, Receiver} import org.apache.spark.util.Utils + +/** + * ReliableKafkaReceiver offers the ability to reliably store data into BlockManager without loss. + * It is turned off by default and will be enabled when + * spark.streaming.receiver.writeAheadLog.enable is true. The difference compared to KafkaReceiver + * is that this receiver manages topic-partition/offset itself and updates the offset information + * after data is reliably stored as write-ahead log. Offsets will only be updated when data is + * reliably stored, so the potential data loss problem of KafkaReceiver can be eliminated. + * + * Note: ReliableKafkaReceiver will set auto.commit.enable to false to turn off automatic offset + * commit mechanism in Kafka consumer. So setting this configuration manually within kafkaParams + * will not take effect. + */ private[streaming] class ReliableKafkaReceiver[ K: ClassTag, @@ -44,7 +57,13 @@ class ReliableKafkaReceiver[ kafkaParams: Map[String, String], topics: Map[String, Int], storageLevel: StorageLevel) - extends Receiver[Any](storageLevel) with Logging { + extends Receiver[(K, V)](storageLevel) with Logging { + + private val groupId = kafkaParams("group.id") + + private val AUTO_OFFSET_COMMIT = "auto.commit.enable" + + private def conf() = SparkEnv.get.conf /** High level consumer to connect to Kafka. */ private var consumerConnector: ConsumerConnector = null @@ -52,12 +71,6 @@ class ReliableKafkaReceiver[ /** zkClient to connect to Zookeeper to commit the offsets. */ private var zkClient: ZkClient = null - private val groupId = kafkaParams("group.id") - - private def conf() = SparkEnv.get.conf - - private val AUTO_OFFSET_COMMIT = "auto.commit.enable" - /** * A HashMap to manage the offset for each topic/partition, this HashMap is called in * synchronized block, so mutable HashMap will not meet concurrency issue. @@ -75,12 +88,6 @@ class ReliableKafkaReceiver[ /** Kafka offsets checkpoint listener to register into BlockGenerator for offsets checkpoint. */ private final class OffsetCheckpointListener extends BlockGeneratorListener { - override def onStoreData(data: Any, metadata: Any): Unit = { - if (metadata != null) { - val kafkaMetadata = metadata.asInstanceOf[(TopicAndPartition, Long)] - topicPartitionOffsetMap.put(kafkaMetadata._1, kafkaMetadata._2) - } - } override def onGenerateBlock(blockId: StreamBlockId): Unit = { // Get a snapshot of current offset map and store with related block id. Since this hook @@ -91,7 +98,7 @@ class ReliableKafkaReceiver[ } override def onPushBlock(blockId: StreamBlockId, arrayBuffer: mutable.ArrayBuffer[_]): Unit = { - store(arrayBuffer.asInstanceOf[mutable.ArrayBuffer[Any]]) + store(arrayBuffer.asInstanceOf[mutable.ArrayBuffer[(K, V)]]) // Commit and remove the related offsets. Option(blockOffsetMap.get(blockId)).foreach { offsetMap => @@ -202,9 +209,10 @@ class ReliableKafkaReceiver[ for (msgAndMetadata <- stream) { val topicAndPartition = TopicAndPartition( msgAndMetadata.topic, msgAndMetadata.partition) - val metadata = (topicAndPartition, msgAndMetadata.offset) - - blockGenerator += ((msgAndMetadata.key, msgAndMetadata.message), metadata) + blockGenerator.synchronized { + blockGenerator += ((msgAndMetadata.key, msgAndMetadata.message)) + topicPartitionOffsetMap.put(topicAndPartition, msgAndMetadata.offset) + } } } catch { case e: Throwable => logError("Error handling message; existing", e) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala index 6e214d7d28a9a..e563c89dc8115 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala @@ -27,11 +27,6 @@ import org.apache.spark.streaming.util.{RecurringTimer, SystemClock} /** Listener object for BlockGenerator events */ private[streaming] trait BlockGeneratorListener { - /** Called when new data is added into BlockGenerator, this hook function will be called each - * time new data is added into BlockGenerator, any heavy or blocking operation will hurt the - * throughput */ - def onStoreData(data: Any, metadata: Any) - /** Called when a new block is generated */ def onGenerateBlock(blockId: StreamBlockId) @@ -92,7 +87,6 @@ private[streaming] class BlockGenerator( def += (data: Any, metadata: Any = null): Unit = synchronized { waitToPush() currentBuffer += data - listener.onStoreData(data, metadata) } /** Change the buffer to which single records are added to. */ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index 5f1784a532573..a13689a9254a6 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -99,8 +99,6 @@ private[streaming] class ReceiverSupervisorImpl( /** Divides received data records into data blocks for pushing in BlockManager. */ private val blockGenerator = new BlockGenerator(new BlockGeneratorListener { - def onStoreData(data: Any, metadata: Any): Unit = { } - def onGenerateBlock(blockId: StreamBlockId): Unit = { } def onError(message: String, throwable: Throwable) { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala index 5133590d36c2e..c189f1fa9ef47 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala @@ -299,8 +299,6 @@ class ReceiverSuite extends FunSuite with Timeouts { val arrayBuffers = new ArrayBuffer[ArrayBuffer[Int]] val errors = new ArrayBuffer[Throwable] - def onStoreData(data: Any, metadata: Any) { } - def onGenerateBlock(blockId: StreamBlockId) { } def onPushBlock(blockId: StreamBlockId, arrayBuffer: ArrayBuffer[_]) { From e5e21c1b54222cc5595e42933f739ca3f078ed36 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Wed, 12 Nov 2014 13:57:50 +0800 Subject: [PATCH 11/21] Change to while loop --- .../org/apache/spark/streaming/kafka/KafkaInputDStream.scala | 4 +++- .../apache/spark/streaming/kafka/ReliableKafkaReceiver.scala | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala index 15ba9c8212aa1..4d26b640e8d74 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala @@ -128,7 +128,9 @@ class KafkaReceiver[ def run() { logInfo("Starting MessageHandler.") try { - for (msgAndMetadata <- stream) { + val streamIterator = stream.iterator() + while (streamIterator.hasNext()) { + val msgAndMetadata = streamIterator.next() store((msgAndMetadata.key, msgAndMetadata.message)) } } catch { diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala index 4a2e414365715..37a966fa302af 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala @@ -206,7 +206,9 @@ class ReliableKafkaReceiver[ override def run(): Unit = { logInfo(s"Starting message process thread ${Thread.currentThread().getId}.") try { - for (msgAndMetadata <- stream) { + val streamIterator = stream.iterator() + while (streamIterator.hasNext()) { + val msgAndMetadata = streamIterator.next() val topicAndPartition = TopicAndPartition( msgAndMetadata.topic, msgAndMetadata.partition) blockGenerator.synchronized { From b798535f1e6c8c981bfb700fe6083e881072d210 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Wed, 12 Nov 2014 14:14:53 +0800 Subject: [PATCH 12/21] Fix the missed issue --- .../org/apache/spark/streaming/receiver/BlockGenerator.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala index e563c89dc8115..2e3be392189ba 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala @@ -84,7 +84,7 @@ private[streaming] class BlockGenerator( * Push a single data item into the buffer. All received data items * will be periodically pushed into BlockManager. */ - def += (data: Any, metadata: Any = null): Unit = synchronized { + def += (data: Any): Unit = synchronized { waitToPush() currentBuffer += data } From e501b3ce04d43f514676c953799ba91963415227 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Wed, 12 Nov 2014 15:01:35 +0800 Subject: [PATCH 13/21] Add Mima excludes --- project/MimaExcludes.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index a94d09be3bec6..8a2a865867fc4 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -85,6 +85,10 @@ object MimaExcludes { "org.apache.hadoop.mapred.SparkHadoopMapRedUtil"), ProblemFilters.exclude[MissingTypesProblem]( "org.apache.spark.rdd.PairRDDFunctions") + ) ++ Seq( + // SPARK-4062 + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.streaming.kafka.KafkaReceiver#MessageHandler.this") ) case v if v.startsWith("1.1") => From b2b2f84b794ed0dba4dc69693d05836a92bc0156 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 12 Nov 2014 16:47:57 -0800 Subject: [PATCH 14/21] Refactored Kafka receiver logic and Kafka testsuites --- .../kafka/ReliableKafkaReceiver.scala | 201 ++++++++++-------- .../streaming/kafka/JavaKafkaStreamSuite.java | 23 +- .../streaming/kafka/KafkaStreamSuite.scala | 139 ++++++------ .../kafka/ReliableKafkaStreamSuite.scala | 123 +++++------ .../streaming/receiver/BlockGenerator.scala | 4 +- 5 files changed, 254 insertions(+), 236 deletions(-) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala index 37a966fa302af..bfee7603fc0ed 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala @@ -18,24 +18,23 @@ package org.apache.spark.streaming.kafka import java.util.Properties -import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.{ThreadPoolExecutor, ConcurrentHashMap} -import scala.collection.Map -import scala.collection.mutable -import scala.reflect.{classTag, ClassTag} +import scala.collection.{Map, mutable} +import scala.reflect.{ClassTag, classTag} import kafka.common.TopicAndPartition import kafka.consumer.{Consumer, ConsumerConfig, ConsumerConnector, KafkaStream} +import kafka.message.MessageAndMetadata import kafka.serializer.Decoder -import kafka.utils.{ZkUtils, ZKGroupTopicDirs, ZKStringSerializer, VerifiableProperties} +import kafka.utils.{VerifiableProperties, ZKGroupTopicDirs, ZKStringSerializer, ZkUtils} import org.I0Itec.zkclient.ZkClient -import org.apache.spark.{SparkEnv, Logging} -import org.apache.spark.storage.{StreamBlockId, StorageLevel} -import org.apache.spark.streaming.receiver.{BlockGeneratorListener, BlockGenerator, Receiver} +import org.apache.spark.{Logging, SparkEnv} +import org.apache.spark.storage.{StorageLevel, StreamBlockId} +import org.apache.spark.streaming.receiver.{BlockGenerator, BlockGeneratorListener, Receiver} import org.apache.spark.util.Utils - /** * ReliableKafkaReceiver offers the ability to reliably store data into BlockManager without loss. * It is turned off by default and will be enabled when @@ -60,10 +59,8 @@ class ReliableKafkaReceiver[ extends Receiver[(K, V)](storageLevel) with Logging { private val groupId = kafkaParams("group.id") - private val AUTO_OFFSET_COMMIT = "auto.commit.enable" - - private def conf() = SparkEnv.get.conf + private def conf = SparkEnv.get.conf /** High level consumer to connect to Kafka. */ private var consumerConnector: ConsumerConnector = null @@ -86,58 +83,8 @@ class ReliableKafkaReceiver[ */ private var blockGenerator: BlockGenerator = null - /** Kafka offsets checkpoint listener to register into BlockGenerator for offsets checkpoint. */ - private final class OffsetCheckpointListener extends BlockGeneratorListener { - - override def onGenerateBlock(blockId: StreamBlockId): Unit = { - // Get a snapshot of current offset map and store with related block id. Since this hook - // function is called in synchronized block, so we can get the snapshot without explicit lock. - val offsetSnapshot = topicPartitionOffsetMap.toMap - blockOffsetMap.put(blockId, offsetSnapshot) - topicPartitionOffsetMap.clear() - } - - override def onPushBlock(blockId: StreamBlockId, arrayBuffer: mutable.ArrayBuffer[_]): Unit = { - store(arrayBuffer.asInstanceOf[mutable.ArrayBuffer[(K, V)]]) - - // Commit and remove the related offsets. - Option(blockOffsetMap.get(blockId)).foreach { offsetMap => - commitOffset(offsetMap) - } - blockOffsetMap.remove(blockId) - } - - override def onError(message: String, throwable: Throwable): Unit = { - reportError(message, throwable) - } - } - - override def onStop(): Unit = { - if (consumerConnector != null) { - consumerConnector.shutdown() - consumerConnector = null - } - - if (zkClient != null) { - zkClient.close() - zkClient = null - } - - if (blockGenerator != null) { - blockGenerator.stop() - blockGenerator = null - } - - if (topicPartitionOffsetMap != null) { - topicPartitionOffsetMap.clear() - topicPartitionOffsetMap = null - } - - if (blockOffsetMap != null) { - blockOffsetMap.clear() - blockOffsetMap = null - } - } + /** Threadpool running the handlers for receiving message from multiple topics and partitions. */ + private var messageHandlerThreadPool: ThreadPoolExecutor = null override def onStart(): Unit = { logInfo(s"Starting Kafka Consumer Stream with group: $groupId") @@ -149,7 +96,7 @@ class ReliableKafkaReceiver[ blockOffsetMap = new ConcurrentHashMap[StreamBlockId, Map[TopicAndPartition, Long]]() // Initialize the block generator for storing Kafka message. - blockGenerator = new BlockGenerator(new OffsetCheckpointListener, streamId, conf()) + blockGenerator = new BlockGenerator(new GeneratedBlockHandler, streamId, conf) if (kafkaParams.contains(AUTO_OFFSET_COMMIT) && kafkaParams(AUTO_OFFSET_COMMIT) == "true") { logWarning(s"$AUTO_OFFSET_COMMIT should be set to false in ReliableKafkaReceiver, " + @@ -174,7 +121,9 @@ class ReliableKafkaReceiver[ zkClient = new ZkClient(consumerConfig.zkConnect, consumerConfig.zkSessionTimeoutMs, consumerConfig.zkConnectionTimeoutMs, ZKStringSerializer) - // start BlockGenerator + messageHandlerThreadPool = Utils.newDaemonFixedThreadPool( + topics.values.sum, "KafkaMessageHandler") + blockGenerator.start() val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties]) @@ -188,40 +137,70 @@ class ReliableKafkaReceiver[ val topicMessageStreams = consumerConnector.createMessageStreams( topics, keyDecoder, valueDecoder) - val executorPool = Utils.newDaemonFixedThreadPool(topics.values.sum, "KafkaMessageHandler") - - try { - topicMessageStreams.values.foreach { streams => - streams.foreach { stream => - executorPool.submit(new MessageHandler(stream)) - } + topicMessageStreams.values.foreach { streams => + streams.foreach { stream => + messageHandlerThreadPool.submit(new MessageHandler(stream)) } - } finally { - executorPool.shutdown() } + println("Starting") } - /** A inner class to handle received Kafka message. */ - private final class MessageHandler(stream: KafkaStream[K, V]) extends Runnable { - override def run(): Unit = { - logInfo(s"Starting message process thread ${Thread.currentThread().getId}.") - try { - val streamIterator = stream.iterator() - while (streamIterator.hasNext()) { - val msgAndMetadata = streamIterator.next() - val topicAndPartition = TopicAndPartition( - msgAndMetadata.topic, msgAndMetadata.partition) - blockGenerator.synchronized { - blockGenerator += ((msgAndMetadata.key, msgAndMetadata.message)) - topicPartitionOffsetMap.put(topicAndPartition, msgAndMetadata.offset) - } - } - } catch { - case e: Throwable => logError("Error handling message; existing", e) - } + override def onStop(): Unit = { + if (messageHandlerThreadPool != null) { + messageHandlerThreadPool.shutdown() + messageHandlerThreadPool = null + } + + if (consumerConnector != null) { + consumerConnector.shutdown() + consumerConnector = null + } + + if (zkClient != null) { + zkClient.close() + zkClient = null + } + + if (blockGenerator != null) { + blockGenerator.stop() + blockGenerator = null + } + + if (topicPartitionOffsetMap != null) { + topicPartitionOffsetMap.clear() + topicPartitionOffsetMap = null + } + + if (blockOffsetMap != null) { + blockOffsetMap.clear() + blockOffsetMap = null } } + /** Store a Kafka message and the associated metadata as a tuple */ + private def storeMessageAndMetadata( + msgAndMetadata: MessageAndMetadata[K, V]): Unit = synchronized { + val topicAndPartition = TopicAndPartition(msgAndMetadata.topic, msgAndMetadata.partition) + blockGenerator += ((msgAndMetadata.key, msgAndMetadata.message)) + topicPartitionOffsetMap.put(topicAndPartition, msgAndMetadata.offset) + } + + /** Remember the current offsets for each topic and partition. This is called when a block is generated */ + private def rememberBlockOffsets(blockId: StreamBlockId): Unit = synchronized { + // Get a snapshot of current offset map and store with related block id. Since this hook + // function is called in synchronized block, so we can get the snapshot without explicit lock. + val offsetSnapshot = topicPartitionOffsetMap.toMap + blockOffsetMap.put(blockId, offsetSnapshot) + topicPartitionOffsetMap.clear() + } + + /** Store the ready-to-be-stored block and commit the related offsets to zookeeper */ + private def storeBlockAndCommitOffset(blockId: StreamBlockId, arrayBuffer: mutable.ArrayBuffer[_]): Unit = { + store(arrayBuffer.asInstanceOf[mutable.ArrayBuffer[(K, V)]]) + Option(blockOffsetMap.get(blockId)).foreach(commitOffset) + blockOffsetMap.remove(blockId) + } + /** * Commit the offset of Kafka's topic/partition, the commit mechanism follow Kafka 0.8.x's * metadata schema in Zookeeper. @@ -248,4 +227,40 @@ class ReliableKafkaReceiver[ s"partition ${topicAndPart.partition}") } } + + /** Class to handle received Kafka message. */ + private final class MessageHandler(stream: KafkaStream[K, V]) extends Runnable { + override def run(): Unit = { + while (!isStopped) { + println(s"Starting message process thread ${Thread.currentThread().getId}.") + try { + val streamIterator = stream.iterator() + while (streamIterator.hasNext) { + storeMessageAndMetadata(streamIterator.next) + } + } catch { + case e: Exception => + logError("Error handling message", e) + } + } + } + } + + /** Class to handle blocks generated by the block generator. */ + private final class GeneratedBlockHandler extends BlockGeneratorListener { + + override def onGenerateBlock(blockId: StreamBlockId): Unit = { + // Remember the offsets of topics/partitions when a block has been generated + rememberBlockOffsets(blockId) + } + + override def onPushBlock(blockId: StreamBlockId, arrayBuffer: mutable.ArrayBuffer[_]): Unit = { + // Store block and commit the blocks offset + storeBlockAndCommitOffset(blockId, arrayBuffer) + } + + override def onError(message: String, throwable: Throwable): Unit = { + reportError(message, throwable) + } + } } diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java index efb0099c7c850..6dfb34424011e 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java @@ -20,6 +20,7 @@ import java.io.Serializable; import java.util.HashMap; import java.util.List; +import java.util.Random; import scala.Predef; import scala.Tuple2; @@ -42,25 +43,23 @@ import org.junit.After; import org.junit.Before; -public class JavaKafkaStreamSuite extends LocalJavaStreamingContext implements Serializable { - private transient KafkaStreamSuite testSuite = new KafkaStreamSuite(); +public class JavaKafkaStreamSuite extends KafkaStreamSuiteBase implements Serializable { + private transient JavaStreamingContext ssc = null; + private Random random = new Random(); @Before - @Override public void setUp() { - testSuite.beforeFunction(); + beforeFunction(); System.clearProperty("spark.driver.port"); - //System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock"); - ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); + ssc = new JavaStreamingContext(sparkConf(), batchDuration()); } @After - @Override public void tearDown() { ssc.stop(); ssc = null; System.clearProperty("spark.driver.port"); - testSuite.afterFunction(); + afterFunction(); } @Test @@ -74,15 +73,15 @@ public void testKafkaStream() throws InterruptedException { sent.put("b", 3); sent.put("c", 10); - testSuite.createTopic(topic); + createTopic(topic); HashMap tmp = new HashMap(sent); - testSuite.produceAndSendMessage(topic, + produceAndSendMessage(topic, JavaConverters.mapAsScalaMapConverter(tmp).asScala().toMap( Predef.>conforms())); HashMap kafkaParams = new HashMap(); - kafkaParams.put("zookeeper.connect", testSuite.zkHost() + ":" + testSuite.zkPort()); - kafkaParams.put("group.id", "test-consumer-" + KafkaTestUtils.random().nextInt(10000)); + kafkaParams.put("zookeeper.connect", zkAddress()); + kafkaParams.put("group.id", "test-consumer-" + random.nextInt(10000)); kafkaParams.put("auto.offset.reset", "smallest"); JavaPairDStream stream = KafkaUtils.createStream(ssc, diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala index cf58b5cb70cd6..d65f9b4ec7d20 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala @@ -19,51 +19,60 @@ package org.apache.spark.streaming.kafka import java.io.File import java.net.InetSocketAddress -import java.util.{Properties, Random} +import java.util.Properties import scala.collection.mutable +import scala.concurrent.duration._ +import scala.language.postfixOps +import scala.util.Random import kafka.admin.CreateTopicCommand import kafka.common.{KafkaException, TopicAndPartition} -import kafka.producer.{KeyedMessage, ProducerConfig, Producer} -import kafka.utils.ZKStringSerializer +import kafka.producer.{KeyedMessage, Producer, ProducerConfig} import kafka.serializer.{StringDecoder, StringEncoder} import kafka.server.{KafkaConfig, KafkaServer} - +import kafka.utils.ZKStringSerializer import org.I0Itec.zkclient.ZkClient +import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} +import org.scalatest.{BeforeAndAfter, FunSuite} +import org.scalatest.concurrent.Eventually -import org.apache.zookeeper.server.ZooKeeperServer -import org.apache.zookeeper.server.NIOServerCnxnFactory - -import org.apache.spark.streaming.{StreamingContext, TestSuiteBase} +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{Milliseconds, StreamingContext} import org.apache.spark.util.Utils -class KafkaStreamSuite extends TestSuiteBase { +abstract class KafkaStreamSuiteBase extends FunSuite with BeforeAndAfter with Logging { import KafkaTestUtils._ - val zkHost = "localhost" - var zkPort: Int = 0 - val zkConnectionTimeout = 6000 - val zkSessionTimeout = 6000 - - protected var brokerPort = 9092 - protected var brokerConf: KafkaConfig = _ - protected var zookeeper: EmbeddedZookeeper = _ - protected var zkClient: ZkClient = _ - protected var server: KafkaServer = _ - protected var producer: Producer[String, String] = _ - - override def useManualClock = false - - override def beforeFunction() { + val sparkConf = new SparkConf() + .setMaster("local[4]") + .setAppName(this.getClass.getSimpleName) + val batchDuration = Milliseconds(500) + var ssc: StreamingContext = _ + + var zkAddress: String = _ + var zkClient: ZkClient = _ + + private val zkHost = "localhost" + private val zkConnectionTimeout = 6000 + private val zkSessionTimeout = 6000 + private var zookeeper: EmbeddedZookeeper = _ + private var zkPort: Int = 0 + private var brokerPort = 9092 + private var brokerConf: KafkaConfig = _ + private var server: KafkaServer = _ + private var producer: Producer[String, String] = _ + + def beforeFunction() { // Zookeeper server startup zookeeper = new EmbeddedZookeeper(s"$zkHost:$zkPort") // Get the actual zookeeper binding port zkPort = zookeeper.actualPort + zkAddress = s"$zkHost:$zkPort" logInfo("==================== 0 ====================") - zkClient = new ZkClient(s"$zkHost:$zkPort", zkSessionTimeout, zkConnectionTimeout, + zkClient = new ZkClient(zkAddress, zkSessionTimeout, zkConnectionTimeout, ZKStringSerializer) logInfo("==================== 1 ====================") @@ -71,7 +80,7 @@ class KafkaStreamSuite extends TestSuiteBase { var bindSuccess: Boolean = false while(!bindSuccess) { try { - val brokerProps = getBrokerConfig(brokerPort, s"$zkHost:$zkPort") + val brokerProps = getBrokerConfig(brokerPort, zkAddress) brokerConf = new KafkaConfig(brokerProps) server = new KafkaServer(brokerConf) logInfo("==================== 2 ====================") @@ -89,10 +98,14 @@ class KafkaStreamSuite extends TestSuiteBase { Thread.sleep(2000) logInfo("==================== 4 ====================") - super.beforeFunction() } - override def afterFunction() { + def afterFunction() { + if (ssc != null) { + ssc.stop() + ssc = null + } + if (producer != null) { producer.close() producer = null @@ -114,19 +127,47 @@ class KafkaStreamSuite extends TestSuiteBase { zookeeper.shutdown() zookeeper = null } + } - super.afterFunction() + private def createTestMessage(topic: String, sent: Map[String, Int]) + : Seq[KeyedMessage[String, String]] = { + val messages = for ((s, freq) <- sent; i <- 0 until freq) yield { + new KeyedMessage[String, String](topic, s) + } + messages.toSeq + } + + def createTopic(topic: String) { + CreateTopicCommand.createTopic(zkClient, topic, 1, 1, "0") + logInfo("==================== 5 ====================") + // wait until metadata is propagated + waitUntilMetadataIsPropagated(Seq(server), topic, 0, 1000) + } + + def produceAndSendMessage(topic: String, sent: Map[String, Int]) { + val brokerAddr = brokerConf.hostName + ":" + brokerConf.port + if (producer == null) { + producer = new Producer[String, String](new ProducerConfig(getProducerConfig(brokerAddr))) + } + producer.send(createTestMessage(topic, sent): _*) + logInfo("==================== 6 ====================") } +} + +class KafkaStreamSuite extends KafkaStreamSuiteBase with Eventually { + + before { beforeFunction() } + after { afterFunction() } test("Kafka input stream") { - val ssc = new StreamingContext(master, framework, batchDuration) + ssc = new StreamingContext(sparkConf, batchDuration) val topic = "topic1" val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) createTopic(topic) produceAndSendMessage(topic, sent) - val kafkaParams = Map("zookeeper.connect" -> s"$zkHost:$zkPort", - "group.id" -> s"test-consumer-${random.nextInt(10000)}", + val kafkaParams = Map("zookeeper.connect" -> zkAddress, + "group.id" -> s"test-consumer-${Random.nextInt(10000)}", "auto.offset.reset" -> "smallest") val stream = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( @@ -145,41 +186,17 @@ class KafkaStreamSuite extends TestSuiteBase { } } ssc.start() - ssc.awaitTermination(3000) - - assert(sent.size === result.size) - sent.keys.foreach { k => assert(sent(k) === result(k).toInt) } - - ssc.stop() - } - - private def createTestMessage(topic: String, sent: Map[String, Int]) - : Seq[KeyedMessage[String, String]] = { - val messages = for ((s, freq) <- sent; i <- 0 until freq) yield { - new KeyedMessage[String, String](topic, s) + eventually(timeout(3000 milliseconds), interval(100 milliseconds)) { + assert(sent.size === result.size) + sent.keys.foreach { k => assert(sent(k) === result(k).toInt) } } - messages.toSeq - } - def createTopic(topic: String) { - CreateTopicCommand.createTopic(zkClient, topic, 1, 1, "0") - logInfo("==================== 5 ====================") - // wait until metadata is propagated - waitUntilMetadataIsPropagated(Seq(server), topic, 0, 1000) - } - - def produceAndSendMessage(topic: String, sent: Map[String, Int]) { - val brokerAddr = brokerConf.hostName + ":" + brokerConf.port - if (producer == null) { - producer = new Producer[String, String](new ProducerConfig(getProducerConfig(brokerAddr))) - } - producer.send(createTestMessage(topic, sent): _*) - logInfo("==================== 6 ====================") + ssc.stop() } } + object KafkaTestUtils { - val random = new Random() def getBrokerConfig(port: Int, zkConnect: String): Properties = { val props = new Properties() diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala index 9a8557e496295..7d90b567442ae 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala @@ -20,37 +20,47 @@ package org.apache.spark.streaming.kafka import java.io.File import scala.collection.mutable +import scala.concurrent.duration._ +import scala.language.postfixOps +import scala.util.Random import kafka.serializer.StringDecoder -import kafka.utils.{ZkUtils, ZKGroupTopicDirs} +import kafka.utils.{ZKGroupTopicDirs, ZkUtils} +import org.scalatest.concurrent.Eventually -import org.apache.spark.SparkConf import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.util.Utils -class ReliableKafkaStreamSuite extends KafkaStreamSuite { - import KafkaTestUtils._ +class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with Eventually { + val topic = "topic" + val data = Map("a" -> 10, "b" -> 10, "c" -> 10) + var groupId: String = _ + var kafkaParams: Map[String, String] = _ + + before { + beforeFunction() // call this first to start ZK and Kafka + groupId = s"test-consumer-${Random.nextInt(10000)}" + kafkaParams = Map( + "zookeeper.connect" -> zkAddress, + "group.id" -> groupId, + "auto.offset.reset" -> "smallest" + ) + } + + after { + afterFunction() + } test("Reliable Kafka input stream") { - val sparkConf = new SparkConf() - .setMaster(master) - .setAppName(framework) - .set("spark.streaming.receiver.writeAheadLog.enable", "true") - val ssc = new StreamingContext(sparkConf, batchDuration) + sparkConf.set("spark.streaming.receiver.writeAheadLog.enable", "true") + ssc = new StreamingContext(sparkConf, batchDuration) val checkpointDir = s"${System.getProperty("java.io.tmpdir", "/tmp")}/" + - s"test-checkpoint${random.nextInt(10000)}" + s"test-checkpoint${Random.nextInt(10000)}" Utils.registerShutdownDeleteDir(new File(checkpointDir)) ssc.checkpoint(checkpointDir) - - val topic = "test" - val sent = Map("a" -> 1, "b" -> 1, "c" -> 1) createTopic(topic) - produceAndSendMessage(topic, sent) - - val kafkaParams = Map("zookeeper.connect" -> s"$zkHost:$zkPort", - "group.id" -> s"test-consumer-${random.nextInt(10000)}", - "auto.offset.reset" -> "smallest") + produceAndSendMessage(topic, data) val stream = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( ssc, @@ -58,8 +68,7 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuite { Map(topic -> 1), StorageLevel.MEMORY_ONLY) val result = new mutable.HashMap[String, Long]() - stream.map { case (k, v) => v } - .foreachRDD { r => + stream.map { case (k, v) => v }.foreachRDD { r => val ret = r.collect() ret.foreach { v => val count = result.getOrElseUpdate(v, 0) + 1 @@ -67,39 +76,27 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuite { } } ssc.start() - ssc.awaitTermination(3000) - - // A basic process verification for ReliableKafkaReceiver. - // Verify whether received message number is equal to the sent message number. - assert(sent.size === result.size) - // Verify whether each message is the same as the data to be verified. - sent.keys.foreach { k => assert(sent(k) === result(k).toInt) } - + eventually(timeout(10000 milliseconds), interval(100 milliseconds)) { + // A basic process verification for ReliableKafkaReceiver. + // Verify whether received message number is equal to the sent message number. + assert(data.size === result.size) + // Verify whether each message is the same as the data to be verified. + data.keys.foreach { k => assert(data(k) === result(k).toInt) } + } ssc.stop() } - +/* test("Verify the offset commit") { - // Verify the corretness of offset commit mechanism. - val sparkConf = new SparkConf() - .setMaster(master) - .setAppName(framework) - .set("spark.streaming.receiver.writeAheadLog.enable", "true") - val ssc = new StreamingContext(sparkConf, batchDuration) + // Verify the correctness of offset commit mechanism. + sparkConf.set("spark.streaming.receiver.writeAheadLog.enable", "true") + ssc = new StreamingContext(sparkConf, batchDuration) val checkpointDir = s"${System.getProperty("java.io.tmpdir", "/tmp")}/" + - s"test-checkpoint${random.nextInt(10000)}" + s"test-checkpoint${Random.nextInt(10000)}" Utils.registerShutdownDeleteDir(new File(checkpointDir)) ssc.checkpoint(checkpointDir) - val topic = "test" - val sent = Map("a" -> 10, "b" -> 10, "c" -> 10) createTopic(topic) - produceAndSendMessage(topic, sent) - - val groupId = s"test-consumer-${random.nextInt(10000)}" - - val kafkaParams = Map("zookeeper.connect" -> s"$zkHost:$zkPort", - "group.id" -> groupId, - "auto.offset.reset" -> "smallest") + produceAndSendMessage(topic, data) // Verify whether the offset of this group/topic/partition is 0 before starting. assert(getCommitOffset(groupId, topic, 0) === 0L) @@ -112,37 +109,27 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuite { StorageLevel.MEMORY_ONLY) stream.foreachRDD(_ => Unit) ssc.start() - ssc.awaitTermination(3000) + eventually(timeout(3000 milliseconds), interval(100 milliseconds)) { + // Verify the offset number whether it is equal to the total message number. + assert(getCommitOffset(groupId, topic, 0) === 29L) + } ssc.stop() - - // Verify the offset number whether it is equal to the total message number. - assert(getCommitOffset(groupId, topic, 0) === 29L) } test("Verify multiple topics offset commit") { - val sparkConf = new SparkConf() - .setMaster(master) - .setAppName(framework) - .set("spark.streaming.receiver.writeAheadLog.enable", "true") - val ssc = new StreamingContext(sparkConf, batchDuration) + sparkConf.set("spark.streaming.receiver.writeAheadLog.enable", "true") + ssc = new StreamingContext(sparkConf, batchDuration) val checkpointDir = s"${System.getProperty("java.io.tmpdir", "/tmp")}/" + - s"test-checkpoint${random.nextInt(10000)}" + s"test-checkpoint${Random.nextInt(10000)}" Utils.registerShutdownDeleteDir(new File(checkpointDir)) ssc.checkpoint(checkpointDir) val topics = Map("topic1" -> 1, "topic2" -> 1, "topic3" -> 1) - val sent = Map("a" -> 10, "b" -> 10, "c" -> 10) topics.foreach { case (t, _) => createTopic(t) - produceAndSendMessage(t, sent) + produceAndSendMessage(t, data) } - val groupId = s"test-consumer-${random.nextInt(10000)}" - - val kafkaParams = Map("zookeeper.connect" -> s"$zkHost:$zkPort", - "group.id" -> groupId, - "auto.offset.reset" -> "smallest") - // Before started, verify all the group/topic/partition offsets are 0. topics.foreach { case (t, _) => assert(getCommitOffset(groupId, t, 0) === 0L) } @@ -154,13 +141,13 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuite { StorageLevel.MEMORY_ONLY) stream.foreachRDD(_ => Unit) ssc.start() - ssc.awaitTermination(3000) + eventually(timeout(3000 milliseconds), interval(100 milliseconds)) { + // Verify the offset for each group/topic to see whether they are equal to the expected one. + topics.foreach { case (t, _) => assert(getCommitOffset(groupId, t, 0) === 29L) } + } ssc.stop() - - // Verify the offset for each group/topic to see whether they are equal to the expected one. - topics.foreach { case (t, _) => assert(getCommitOffset(groupId, t, 0) === 29L) } } - +*/ /** Getting partition offset from Zookeeper. */ private def getCommitOffset(groupId: String, topic: String, partition: Int): Long = { assert(zkClient != null, "Zookeeper client is not initialized") diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala index 2e3be392189ba..b1e9cb7673f2c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala @@ -90,10 +90,10 @@ private[streaming] class BlockGenerator( } /** Change the buffer to which single records are added to. */ - private def updateCurrentBuffer(time: Long): Unit = synchronized { + private def updateCurrentBuffer(time: Long): Unit = { try { val newBlockBuffer = currentBuffer - currentBuffer = new ArrayBuffer[Any] + synchronized { currentBuffer = new ArrayBuffer[Any] } if (newBlockBuffer.size > 0) { val blockId = StreamBlockId(receiverId, time - blockInterval) val newBlock = new Block(blockId, newBlockBuffer) From 2a20a0107af78f9db85bea28acd6cd4730af84f7 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 13 Nov 2014 15:02:01 +0800 Subject: [PATCH 15/21] Address some comments --- .../kafka/ReliableKafkaReceiver.scala | 19 ++++++++-------- .../streaming/kafka/JavaKafkaStreamSuite.java | 22 +++++++++---------- .../streaming/kafka/KafkaStreamSuite.scala | 4 ++-- .../kafka/ReliableKafkaStreamSuite.scala | 6 ++--- 4 files changed, 26 insertions(+), 25 deletions(-) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala index bfee7603fc0ed..c207e95d5d337 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala @@ -83,7 +83,7 @@ class ReliableKafkaReceiver[ */ private var blockGenerator: BlockGenerator = null - /** Threadpool running the handlers for receiving message from multiple topics and partitions. */ + /** Thread pool running the handlers for receiving message from multiple topics and partitions. */ private var messageHandlerThreadPool: ThreadPoolExecutor = null override def onStart(): Unit = { @@ -142,7 +142,6 @@ class ReliableKafkaReceiver[ messageHandlerThreadPool.submit(new MessageHandler(stream)) } } - println("Starting") } override def onStop(): Unit = { @@ -177,7 +176,7 @@ class ReliableKafkaReceiver[ } } - /** Store a Kafka message and the associated metadata as a tuple */ + /** Store a Kafka message and the associated metadata as a tuple. */ private def storeMessageAndMetadata( msgAndMetadata: MessageAndMetadata[K, V]): Unit = synchronized { val topicAndPartition = TopicAndPartition(msgAndMetadata.topic, msgAndMetadata.partition) @@ -185,17 +184,20 @@ class ReliableKafkaReceiver[ topicPartitionOffsetMap.put(topicAndPartition, msgAndMetadata.offset) } - /** Remember the current offsets for each topic and partition. This is called when a block is generated */ + /** + * Remember the current offsets for each topic and partition. This is called when a block is + * generated. + */ private def rememberBlockOffsets(blockId: StreamBlockId): Unit = synchronized { - // Get a snapshot of current offset map and store with related block id. Since this hook - // function is called in synchronized block, so we can get the snapshot without explicit lock. + // Get a snapshot of current offset map and store with related block id. val offsetSnapshot = topicPartitionOffsetMap.toMap blockOffsetMap.put(blockId, offsetSnapshot) topicPartitionOffsetMap.clear() } - /** Store the ready-to-be-stored block and commit the related offsets to zookeeper */ - private def storeBlockAndCommitOffset(blockId: StreamBlockId, arrayBuffer: mutable.ArrayBuffer[_]): Unit = { + /** Store the ready-to-be-stored block and commit the related offsets to zookeeper. */ + private def storeBlockAndCommitOffset( + blockId: StreamBlockId, arrayBuffer: mutable.ArrayBuffer[_]): Unit = { store(arrayBuffer.asInstanceOf[mutable.ArrayBuffer[(K, V)]]) Option(blockOffsetMap.get(blockId)).foreach(commitOffset) blockOffsetMap.remove(blockId) @@ -232,7 +234,6 @@ class ReliableKafkaReceiver[ private final class MessageHandler(stream: KafkaStream[K, V]) extends Runnable { override def run(): Unit = { while (!isStopped) { - println(s"Starting message process thread ${Thread.currentThread().getId}.") try { val streamIterator = stream.iterator() while (streamIterator.hasNext) { diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java index 6dfb34424011e..6386602ef8a43 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java @@ -33,8 +33,6 @@ import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.function.Function; import org.apache.spark.storage.StorageLevel; -import org.apache.spark.streaming.Duration; -import org.apache.spark.streaming.LocalJavaStreamingContext; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; @@ -43,15 +41,17 @@ import org.junit.After; import org.junit.Before; -public class JavaKafkaStreamSuite extends KafkaStreamSuiteBase implements Serializable { - private transient JavaStreamingContext ssc = null; - private Random random = new Random(); +public class JavaKafkaStreamSuite implements Serializable { + private transient JavaStreamingContext ssc = null; + private Random random = new Random(); + private transient KafkaStreamSuiteBase suiteBase = null; @Before public void setUp() { - beforeFunction(); + suiteBase = new KafkaStreamSuiteBase() { }; + suiteBase.beforeFunction(); System.clearProperty("spark.driver.port"); - ssc = new JavaStreamingContext(sparkConf(), batchDuration()); + ssc = new JavaStreamingContext(suiteBase.sparkConf(), suiteBase.batchDuration()); } @After @@ -59,7 +59,7 @@ public void tearDown() { ssc.stop(); ssc = null; System.clearProperty("spark.driver.port"); - afterFunction(); + suiteBase.afterFunction(); } @Test @@ -73,14 +73,14 @@ public void testKafkaStream() throws InterruptedException { sent.put("b", 3); sent.put("c", 10); - createTopic(topic); + suiteBase.createTopic(topic); HashMap tmp = new HashMap(sent); - produceAndSendMessage(topic, + suiteBase.produceAndSendMessage(topic, JavaConverters.mapAsScalaMapConverter(tmp).asScala().toMap( Predef.>conforms())); HashMap kafkaParams = new HashMap(); - kafkaParams.put("zookeeper.connect", zkAddress()); + kafkaParams.put("zookeeper.connect", suiteBase.zkAddress()); kafkaParams.put("group.id", "test-consumer-" + random.nextInt(10000)); kafkaParams.put("auto.offset.reset", "smallest"); diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala index d65f9b4ec7d20..1bb8e0175b97c 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala @@ -42,7 +42,7 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Milliseconds, StreamingContext} import org.apache.spark.util.Utils -abstract class KafkaStreamSuiteBase extends FunSuite with BeforeAndAfter with Logging { +abstract class KafkaStreamSuiteBase extends FunSuite with Logging { import KafkaTestUtils._ val sparkConf = new SparkConf() @@ -154,7 +154,7 @@ abstract class KafkaStreamSuiteBase extends FunSuite with BeforeAndAfter with Lo } } -class KafkaStreamSuite extends KafkaStreamSuiteBase with Eventually { +class KafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter with Eventually { before { beforeFunction() } after { afterFunction() } diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala index 7d90b567442ae..b546d22ca6c38 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala @@ -26,13 +26,14 @@ import scala.util.Random import kafka.serializer.StringDecoder import kafka.utils.{ZKGroupTopicDirs, ZkUtils} +import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.Eventually import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.util.Utils -class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with Eventually { +class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter with Eventually { val topic = "topic" val data = Map("a" -> 10, "b" -> 10, "c" -> 10) var groupId: String = _ @@ -85,7 +86,6 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with Eventually { } ssc.stop() } -/* test("Verify the offset commit") { // Verify the correctness of offset commit mechanism. sparkConf.set("spark.streaming.receiver.writeAheadLog.enable", "true") @@ -147,7 +147,7 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with Eventually { } ssc.stop() } -*/ + /** Getting partition offset from Zookeeper. */ private def getCommitOffset(groupId: String, topic: String, partition: Int): Long = { assert(zkClient != null, "Zookeeper client is not initialized") From ec2e95eb72d09bb36d83e28402769f636b04c745 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 13 Nov 2014 13:33:14 -0800 Subject: [PATCH 16/21] Removed the receiver's locks and essentially reverted to Saisai's original design. --- .../kafka/ReliableKafkaReceiver.scala | 28 +++-- .../streaming/kafka/JavaKafkaStreamSuite.java | 16 ++- .../streaming/kafka/KafkaStreamSuite.scala | 41 +++---- .../kafka/ReliableKafkaStreamSuite.scala | 106 +++++++++--------- .../streaming/receiver/BlockGenerator.scala | 52 +++++++-- .../receiver/ReceiverSupervisorImpl.scala | 6 +- .../spark/streaming/ReceiverSuite.scala | 6 +- 7 files changed, 157 insertions(+), 98 deletions(-) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala index c207e95d5d337..f85296efe8d60 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala @@ -178,17 +178,23 @@ class ReliableKafkaReceiver[ /** Store a Kafka message and the associated metadata as a tuple. */ private def storeMessageAndMetadata( - msgAndMetadata: MessageAndMetadata[K, V]): Unit = synchronized { + msgAndMetadata: MessageAndMetadata[K, V]): Unit = { val topicAndPartition = TopicAndPartition(msgAndMetadata.topic, msgAndMetadata.partition) - blockGenerator += ((msgAndMetadata.key, msgAndMetadata.message)) - topicPartitionOffsetMap.put(topicAndPartition, msgAndMetadata.offset) + val data = (msgAndMetadata.key, msgAndMetadata.message) + val metadata = (topicAndPartition, msgAndMetadata.offset) + blockGenerator.addDataWithCallback(data, metadata) + } + + /** Update stored offset */ + private def updateOffset(topicAndPartition: TopicAndPartition, offset: Long): Unit = { + topicPartitionOffsetMap.put(topicAndPartition, offset) } /** * Remember the current offsets for each topic and partition. This is called when a block is * generated. */ - private def rememberBlockOffsets(blockId: StreamBlockId): Unit = synchronized { + private def rememberBlockOffsets(blockId: StreamBlockId): Unit = { // Get a snapshot of current offset map and store with related block id. val offsetSnapshot = topicPartitionOffsetMap.toMap blockOffsetMap.put(blockId, offsetSnapshot) @@ -250,17 +256,25 @@ class ReliableKafkaReceiver[ /** Class to handle blocks generated by the block generator. */ private final class GeneratedBlockHandler extends BlockGeneratorListener { - override def onGenerateBlock(blockId: StreamBlockId): Unit = { + def onAddData(data: Any, metadata: Any): Unit = { + // Update the offset of the data that was added to the generator + if (metadata != null) { + val (topicAndPartition, offset) = metadata.asInstanceOf[(TopicAndPartition, Long)] + updateOffset(topicAndPartition, offset) + } + } + + def onGenerateBlock(blockId: StreamBlockId): Unit = { // Remember the offsets of topics/partitions when a block has been generated rememberBlockOffsets(blockId) } - override def onPushBlock(blockId: StreamBlockId, arrayBuffer: mutable.ArrayBuffer[_]): Unit = { + def onPushBlock(blockId: StreamBlockId, arrayBuffer: mutable.ArrayBuffer[_]): Unit = { // Store block and commit the blocks offset storeBlockAndCommitOffset(blockId, arrayBuffer) } - override def onError(message: String, throwable: Throwable): Unit = { + def onError(message: String, throwable: Throwable): Unit = { reportError(message, throwable) } } diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java index 6386602ef8a43..60e6f8deb0d13 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java @@ -22,6 +22,8 @@ import java.util.List; import java.util.Random; +import org.apache.spark.SparkConf; +import org.apache.spark.streaming.Duration; import scala.Predef; import scala.Tuple2; import scala.collection.JavaConverters; @@ -43,15 +45,17 @@ public class JavaKafkaStreamSuite implements Serializable { private transient JavaStreamingContext ssc = null; - private Random random = new Random(); + private transient Random random = new Random(); private transient KafkaStreamSuiteBase suiteBase = null; @Before public void setUp() { suiteBase = new KafkaStreamSuiteBase() { }; - suiteBase.beforeFunction(); + suiteBase.setupKafka(); System.clearProperty("spark.driver.port"); - ssc = new JavaStreamingContext(suiteBase.sparkConf(), suiteBase.batchDuration()); + SparkConf sparkConf = new SparkConf() + .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); + ssc = new JavaStreamingContext(sparkConf, new Duration(500)); } @After @@ -59,7 +63,7 @@ public void tearDown() { ssc.stop(); ssc = null; System.clearProperty("spark.driver.port"); - suiteBase.afterFunction(); + suiteBase.tearDownKafka(); } @Test @@ -76,8 +80,8 @@ public void testKafkaStream() throws InterruptedException { suiteBase.createTopic(topic); HashMap tmp = new HashMap(sent); suiteBase.produceAndSendMessage(topic, - JavaConverters.mapAsScalaMapConverter(tmp).asScala().toMap( - Predef.>conforms())); + JavaConverters.mapAsScalaMapConverter(tmp).asScala().toMap( + Predef.>conforms())); HashMap kafkaParams = new HashMap(); kafkaParams.put("zookeeper.connect", suiteBase.zkAddress()); diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala index 1bb8e0175b97c..6e24b6f7ffb3b 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala @@ -42,15 +42,13 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Milliseconds, StreamingContext} import org.apache.spark.util.Utils +/** + * This is an abstract base class for Kafka testsuites. This has the functionality to set up + * and tear down local Kafka servers, and to push data using Kafka producers. + */ abstract class KafkaStreamSuiteBase extends FunSuite with Logging { import KafkaTestUtils._ - val sparkConf = new SparkConf() - .setMaster("local[4]") - .setAppName(this.getClass.getSimpleName) - val batchDuration = Milliseconds(500) - var ssc: StreamingContext = _ - var zkAddress: String = _ var zkClient: ZkClient = _ @@ -64,7 +62,7 @@ abstract class KafkaStreamSuiteBase extends FunSuite with Logging { private var server: KafkaServer = _ private var producer: Producer[String, String] = _ - def beforeFunction() { + def setupKafka() { // Zookeeper server startup zookeeper = new EmbeddedZookeeper(s"$zkHost:$zkPort") // Get the actual zookeeper binding port @@ -100,12 +98,7 @@ abstract class KafkaStreamSuiteBase extends FunSuite with Logging { logInfo("==================== 4 ====================") } - def afterFunction() { - if (ssc != null) { - ssc.stop() - ssc = null - } - + def tearDownKafka() { if (producer != null) { producer.close() producer = null @@ -146,21 +139,31 @@ abstract class KafkaStreamSuiteBase extends FunSuite with Logging { def produceAndSendMessage(topic: String, sent: Map[String, Int]) { val brokerAddr = brokerConf.hostName + ":" + brokerConf.port - if (producer == null) { - producer = new Producer[String, String](new ProducerConfig(getProducerConfig(brokerAddr))) - } + producer = new Producer[String, String](new ProducerConfig(getProducerConfig(brokerAddr))) producer.send(createTestMessage(topic, sent): _*) + producer.close() logInfo("==================== 6 ====================") } } class KafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter with Eventually { + var ssc: StreamingContext = _ + + before { + setupKafka() + } - before { beforeFunction() } - after { afterFunction() } + after { + if (ssc != null) { + ssc.stop() + ssc = null + } + tearDownKafka() + } test("Kafka input stream") { - ssc = new StreamingContext(sparkConf, batchDuration) + val sparkConf = new SparkConf().setMaster("local[4]").setAppName(this.getClass.getSimpleName) + ssc = new StreamingContext(sparkConf, Milliseconds(500)) val topic = "topic1" val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) createTopic(topic) diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala index b546d22ca6c38..8489d64762a2b 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.streaming.kafka + import java.io.File import scala.collection.mutable @@ -24,50 +25,67 @@ import scala.concurrent.duration._ import scala.language.postfixOps import scala.util.Random +import com.google.common.io.Files import kafka.serializer.StringDecoder import kafka.utils.{ZKGroupTopicDirs, ZkUtils} +import org.apache.commons.io.FileUtils import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.Eventually +import org.apache.spark.SparkConf import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.util.Utils +import org.apache.spark.streaming.{Milliseconds, StreamingContext} class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter with Eventually { - val topic = "topic" + + val sparkConf = new SparkConf() + .setMaster("local[4]") + .setAppName(this.getClass.getSimpleName) + .set("spark.streaming.receiver.writeAheadLog.enable", "true") val data = Map("a" -> 10, "b" -> 10, "c" -> 10) + + var topic: String = _ var groupId: String = _ var kafkaParams: Map[String, String] = _ + var ssc: StreamingContext = _ + var tempDirectory: File = null before { - beforeFunction() // call this first to start ZK and Kafka + setupKafka() + topic = s"test-topic-${Random.nextInt(10000)}" groupId = s"test-consumer-${Random.nextInt(10000)}" kafkaParams = Map( "zookeeper.connect" -> zkAddress, "group.id" -> groupId, "auto.offset.reset" -> "smallest" ) + + ssc = new StreamingContext(sparkConf, Milliseconds(500)) + tempDirectory = Files.createTempDir() + ssc.checkpoint(tempDirectory.getAbsolutePath) } after { - afterFunction() + if (ssc != null) { + ssc.stop() + } + if (tempDirectory != null && tempDirectory.exists()) { + FileUtils.deleteDirectory(tempDirectory) + tempDirectory = null + } + tearDownKafka() } - test("Reliable Kafka input stream") { - sparkConf.set("spark.streaming.receiver.writeAheadLog.enable", "true") - ssc = new StreamingContext(sparkConf, batchDuration) - val checkpointDir = s"${System.getProperty("java.io.tmpdir", "/tmp")}/" + - s"test-checkpoint${Random.nextInt(10000)}" - Utils.registerShutdownDeleteDir(new File(checkpointDir)) - ssc.checkpoint(checkpointDir) + + test("Reliable Kafka input stream with single topic") { createTopic(topic) produceAndSendMessage(topic, data) + // Verify whether the offset of this group/topic/partition is 0 before starting. + assert(getCommitOffset(groupId, topic, 0) === None) + val stream = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( - ssc, - kafkaParams, - Map(topic -> 1), - StorageLevel.MEMORY_ONLY) + ssc, kafkaParams, Map(topic -> 1), StorageLevel.MEMORY_ONLY) val result = new mutable.HashMap[String, Long]() stream.map { case (k, v) => v }.foreachRDD { r => val ret = r.collect() @@ -77,53 +95,36 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter } } ssc.start() - eventually(timeout(10000 milliseconds), interval(100 milliseconds)) { + + eventually(timeout(20000 milliseconds), interval(200 milliseconds)) { // A basic process verification for ReliableKafkaReceiver. // Verify whether received message number is equal to the sent message number. assert(data.size === result.size) // Verify whether each message is the same as the data to be verified. data.keys.foreach { k => assert(data(k) === result(k).toInt) } + // Verify the offset number whether it is equal to the total message number. + assert(getCommitOffset(groupId, topic, 0) === Some(29L)) + } ssc.stop() } +/* test("Verify the offset commit") { // Verify the correctness of offset commit mechanism. - sparkConf.set("spark.streaming.receiver.writeAheadLog.enable", "true") - ssc = new StreamingContext(sparkConf, batchDuration) - val checkpointDir = s"${System.getProperty("java.io.tmpdir", "/tmp")}/" + - s"test-checkpoint${Random.nextInt(10000)}" - Utils.registerShutdownDeleteDir(new File(checkpointDir)) - ssc.checkpoint(checkpointDir) - createTopic(topic) produceAndSendMessage(topic, data) - // Verify whether the offset of this group/topic/partition is 0 before starting. - assert(getCommitOffset(groupId, topic, 0) === 0L) - // Do this to consume all the message of this group/topic. val stream = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( - ssc, - kafkaParams, - Map(topic -> 1), - StorageLevel.MEMORY_ONLY) + ssc, kafkaParams, Map(topic -> 1), StorageLevel.MEMORY_ONLY) stream.foreachRDD(_ => Unit) ssc.start() - eventually(timeout(3000 milliseconds), interval(100 milliseconds)) { - // Verify the offset number whether it is equal to the total message number. - assert(getCommitOffset(groupId, topic, 0) === 29L) + eventually(timeout(20000 milliseconds), interval(200 milliseconds)) { } ssc.stop() } - - test("Verify multiple topics offset commit") { - sparkConf.set("spark.streaming.receiver.writeAheadLog.enable", "true") - ssc = new StreamingContext(sparkConf, batchDuration) - val checkpointDir = s"${System.getProperty("java.io.tmpdir", "/tmp")}/" + - s"test-checkpoint${Random.nextInt(10000)}" - Utils.registerShutdownDeleteDir(new File(checkpointDir)) - ssc.checkpoint(checkpointDir) - +*/ + test("Reliable Kafka input stream with multiple topics") { val topics = Map("topic1" -> 1, "topic2" -> 1, "topic3" -> 1) topics.foreach { case (t, _) => createTopic(t) @@ -131,30 +132,27 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter } // Before started, verify all the group/topic/partition offsets are 0. - topics.foreach { case (t, _) => assert(getCommitOffset(groupId, t, 0) === 0L) } + topics.foreach { case (t, _) => assert(getCommitOffset(groupId, t, 0) === None) } // Consuming all the data sent to the broker which will potential commit the offsets internally. val stream = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( - ssc, - kafkaParams, - topics, - StorageLevel.MEMORY_ONLY) + ssc, kafkaParams, topics, StorageLevel.MEMORY_ONLY) stream.foreachRDD(_ => Unit) ssc.start() - eventually(timeout(3000 milliseconds), interval(100 milliseconds)) { + eventually(timeout(20000 milliseconds), interval(100 milliseconds)) { // Verify the offset for each group/topic to see whether they are equal to the expected one. - topics.foreach { case (t, _) => assert(getCommitOffset(groupId, t, 0) === 29L) } + topics.foreach { case (t, _) => assert(getCommitOffset(groupId, t, 0) === Some(29L)) } } ssc.stop() } + /** Getting partition offset from Zookeeper. */ - private def getCommitOffset(groupId: String, topic: String, partition: Int): Long = { + private def getCommitOffset(groupId: String, topic: String, partition: Int): Option[Long] = { assert(zkClient != null, "Zookeeper client is not initialized") - val topicDirs = new ZKGroupTopicDirs(groupId, topic) val zkPath = s"${topicDirs.consumerOffsetDir}/$partition" - - ZkUtils.readDataMaybeNull(zkClient, zkPath)._1.map(_.toLong).getOrElse(0L) + val offset = ZkUtils.readDataMaybeNull(zkClient, zkPath)._1.map(_.toLong) + offset } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala index b1e9cb7673f2c..d286fadd50096 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala @@ -27,13 +27,38 @@ import org.apache.spark.streaming.util.{RecurringTimer, SystemClock} /** Listener object for BlockGenerator events */ private[streaming] trait BlockGeneratorListener { - /** Called when a new block is generated */ + /** + * Called after a data item is added into the BlockGenerator. The data addition and this + * callback are synchronized with the block generation and its associated callback, + * so block generation waits for the active data addition+callback to complete. This is useful + * for updating metadata on successful buffering of a data item, specifically that metadata + * that will be useful when a block is generated. Any long blocking operation in this callback + * will hurt the throughput. + */ + def onAddData(data: Any, metadata: Any) + + /** + * Called when a new block of data is generated by the block generator. The block generation + * and this callback are synchronized with the data addition and its associated callback, so + * the data addition waits for the block generation+callback to complete. This is useful + * for updating metadata when a block has been generated, specifically metadata that will + * be useful when the block has been successfully stored. Any long blocking operation in this + * callback will hurt the throughput. + */ def onGenerateBlock(blockId: StreamBlockId) - /** Called when a new block needs to be pushed */ + /** + * Called when a new block is ready to be pushed. Callers are supposed to store the block into + * Spark in this method. Internally this is called from a single + * thread, that is not synchronized with any other callbacks. Hence it is okay to do long + * blocking operation in this callback. + */ def onPushBlock(blockId: StreamBlockId, arrayBuffer: ArrayBuffer[_]) - /** Called when an error has occurred in BlockGenerator */ + /** + * Called when an error has occurred in the BlockGenerator. Can be called form many places + * so better to not do any long block operation in this callback. + */ def onError(message: String, throwable: Throwable) } @@ -84,16 +109,27 @@ private[streaming] class BlockGenerator( * Push a single data item into the buffer. All received data items * will be periodically pushed into BlockManager. */ - def += (data: Any): Unit = synchronized { + def addData (data: Any): Unit = synchronized { waitToPush() currentBuffer += data } + /** + * Push a single data item into the buffer. After buffering the data, the + * `BlockGeneratorListnere.onAddData` callback will be called. All received data items + * will be periodically pushed into BlockManager. + */ + def addDataWithCallback(data: Any, metadata: Any) = synchronized { + waitToPush() + currentBuffer += data + listener.onAddData(data, metadata) + } + /** Change the buffer to which single records are added to. */ - private def updateCurrentBuffer(time: Long): Unit = { + private def updateCurrentBuffer(time: Long): Unit = synchronized { try { val newBlockBuffer = currentBuffer - synchronized { currentBuffer = new ArrayBuffer[Any] } + currentBuffer = new ArrayBuffer[Any] if (newBlockBuffer.size > 0) { val blockId = StreamBlockId(receiverId, time - blockInterval) val newBlock = new Block(blockId, newBlockBuffer) @@ -131,8 +167,8 @@ private[streaming] class BlockGenerator( } catch { case ie: InterruptedException => logInfo("Block pushing thread was interrupted") - case t: Throwable => - reportError("Error in block pushing thread", t) + case e: Exception => + reportError("Error in block updating thread", e) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index a13689a9254a6..704019a55aff3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -27,10 +27,10 @@ import akka.actor.{Actor, Props} import akka.pattern.ask import com.google.common.base.Throwables import org.apache.hadoop.conf.Configuration + import org.apache.spark.{Logging, SparkEnv, SparkException} import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.scheduler._ -import org.apache.spark.streaming.util.WriteAheadLogFileSegment import org.apache.spark.util.{AkkaUtils, Utils} /** @@ -99,6 +99,8 @@ private[streaming] class ReceiverSupervisorImpl( /** Divides received data records into data blocks for pushing in BlockManager. */ private val blockGenerator = new BlockGenerator(new BlockGeneratorListener { + def onAddData(data: Any, metadata: Any): Unit = { } + def onGenerateBlock(blockId: StreamBlockId): Unit = { } def onError(message: String, throwable: Throwable) { @@ -112,7 +114,7 @@ private[streaming] class ReceiverSupervisorImpl( /** Push a single record of received data into block generator. */ def pushSingle(data: Any) { - blockGenerator += (data) + blockGenerator addData (data) } /** Store an ArrayBuffer of received data as a data block into Spark's memory. */ diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala index c189f1fa9ef47..9bad105f5a83b 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala @@ -138,7 +138,7 @@ class ReceiverSuite extends FunSuite with Timeouts { blockGenerator.start() var count = 0 while(System.currentTimeMillis - startTime < waitTime) { - blockGenerator += count + blockGenerator addData count generatedData += count count += 1 Thread.sleep(10) @@ -168,7 +168,7 @@ class ReceiverSuite extends FunSuite with Timeouts { blockGenerator.start() var count = 0 while(System.currentTimeMillis - startTime < waitTime) { - blockGenerator += count + blockGenerator addData count generatedData += count count += 1 Thread.sleep(1) @@ -299,6 +299,8 @@ class ReceiverSuite extends FunSuite with Timeouts { val arrayBuffers = new ArrayBuffer[ArrayBuffer[Int]] val errors = new ArrayBuffer[Throwable] + def onAddData(data: Any, metadata: Any) { } + def onGenerateBlock(blockId: StreamBlockId) { } def onPushBlock(blockId: StreamBlockId, arrayBuffer: ArrayBuffer[_]) { From d9a452cd3ffefc390622c147816d18a57d9428e4 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 13 Nov 2014 13:39:30 -0800 Subject: [PATCH 17/21] Minor updates. --- .../streaming/kafka/JavaKafkaStreamSuite.java | 4 ++-- .../kafka/ReliableKafkaStreamSuite.scala | 24 +++---------------- 2 files changed, 5 insertions(+), 23 deletions(-) diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java index 60e6f8deb0d13..ce48dfbc0fcf1 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java @@ -80,8 +80,8 @@ public void testKafkaStream() throws InterruptedException { suiteBase.createTopic(topic); HashMap tmp = new HashMap(sent); suiteBase.produceAndSendMessage(topic, - JavaConverters.mapAsScalaMapConverter(tmp).asScala().toMap( - Predef.>conforms())); + JavaConverters.mapAsScalaMapConverter(tmp).asScala().toMap( + Predef.>conforms())); HashMap kafkaParams = new HashMap(); kafkaParams.put("zookeeper.connect", suiteBase.zkAddress()); diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala index 8489d64762a2b..64ccc92c81fa9 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala @@ -44,7 +44,7 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter .set("spark.streaming.receiver.writeAheadLog.enable", "true") val data = Map("a" -> 10, "b" -> 10, "c" -> 10) - var topic: String = _ + var groupId: String = _ var kafkaParams: Map[String, String] = _ var ssc: StreamingContext = _ @@ -52,7 +52,6 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter before { setupKafka() - topic = s"test-topic-${Random.nextInt(10000)}" groupId = s"test-consumer-${Random.nextInt(10000)}" kafkaParams = Map( "zookeeper.connect" -> zkAddress, @@ -78,6 +77,7 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter test("Reliable Kafka input stream with single topic") { + var topic = "test-topic" createTopic(topic) produceAndSendMessage(topic, data) @@ -95,7 +95,6 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter } } ssc.start() - eventually(timeout(20000 milliseconds), interval(200 milliseconds)) { // A basic process verification for ReliableKafkaReceiver. // Verify whether received message number is equal to the sent message number. @@ -104,26 +103,10 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter data.keys.foreach { k => assert(data(k) === result(k).toInt) } // Verify the offset number whether it is equal to the total message number. assert(getCommitOffset(groupId, topic, 0) === Some(29L)) - } ssc.stop() } -/* - test("Verify the offset commit") { - // Verify the correctness of offset commit mechanism. - createTopic(topic) - produceAndSendMessage(topic, data) - // Do this to consume all the message of this group/topic. - val stream = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( - ssc, kafkaParams, Map(topic -> 1), StorageLevel.MEMORY_ONLY) - stream.foreachRDD(_ => Unit) - ssc.start() - eventually(timeout(20000 milliseconds), interval(200 milliseconds)) { - } - ssc.stop() - } -*/ test("Reliable Kafka input stream with multiple topics") { val topics = Map("topic1" -> 1, "topic2" -> 1, "topic3" -> 1) topics.foreach { case (t, _) => @@ -152,7 +135,6 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter assert(zkClient != null, "Zookeeper client is not initialized") val topicDirs = new ZKGroupTopicDirs(groupId, topic) val zkPath = s"${topicDirs.consumerOffsetDir}/$partition" - val offset = ZkUtils.readDataMaybeNull(zkClient, zkPath)._1.map(_.toLong) - offset + ZkUtils.readDataMaybeNull(zkClient, zkPath)._1.map(_.toLong) } } From 14630aaf06892494ade308c93ab1d18b3d8d5207 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 13 Nov 2014 13:51:42 -0800 Subject: [PATCH 18/21] Minor updates. --- .../spark/streaming/kafka/ReliableKafkaReceiver.scala | 2 +- .../apache/spark/streaming/receiver/BlockGenerator.scala | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala index f85296efe8d60..09c3d1c4549e7 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala @@ -227,7 +227,7 @@ class ReliableKafkaReceiver[ ZkUtils.updatePersistentPath(zkClient, zkPath, offset.toString) } catch { - case t: Throwable => logWarning(s"Exception during commit offset $offset for topic" + + case e: Exception => logWarning(s"Exception during commit offset $offset for topic" + s"${topicAndPart.topic}, partition ${topicAndPart.partition}", t) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala index d286fadd50096..55765dc90698b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala @@ -140,8 +140,8 @@ private[streaming] class BlockGenerator( } catch { case ie: InterruptedException => logInfo("Block updating timer thread was interrupted") - case t: Throwable => - reportError("Error in block updating thread", t) + case e: Exception => + reportError("Error in block updating thread", e) } } @@ -168,7 +168,7 @@ private[streaming] class BlockGenerator( case ie: InterruptedException => logInfo("Block pushing thread was interrupted") case e: Exception => - reportError("Error in block updating thread", e) + reportError("Error in block pushing thread", e) } } From 149948b57de14ddacabc81c16a278ceff4cf927a Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 13 Nov 2014 13:55:09 -0800 Subject: [PATCH 19/21] Fixed mistake --- .../apache/spark/streaming/kafka/ReliableKafkaReceiver.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala index 09c3d1c4549e7..be734b80272d1 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala @@ -227,8 +227,9 @@ class ReliableKafkaReceiver[ ZkUtils.updatePersistentPath(zkClient, zkPath, offset.toString) } catch { - case e: Exception => logWarning(s"Exception during commit offset $offset for topic" + - s"${topicAndPart.topic}, partition ${topicAndPart.partition}", t) + case e: Exception => + logWarning(s"Exception during commit offset $offset for topic" + + s"${topicAndPart.topic}, partition ${topicAndPart.partition}", e) } logInfo(s"Committed offset $offset for topic ${topicAndPart.topic}, " + From fab14c79d3b1e7ced5b881359343366915a622b0 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 13 Nov 2014 14:21:19 -0800 Subject: [PATCH 20/21] minor update. --- .../spark/streaming/receiver/ReceiverSupervisorImpl.scala | 2 +- .../test/scala/org/apache/spark/streaming/ReceiverSuite.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index 704019a55aff3..3b1233e86c210 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -114,7 +114,7 @@ private[streaming] class ReceiverSupervisorImpl( /** Push a single record of received data into block generator. */ def pushSingle(data: Any) { - blockGenerator addData (data) + blockGenerator.addData(data) } /** Store an ArrayBuffer of received data as a data block into Spark's memory. */ diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala index 9bad105f5a83b..e26c0c6859e57 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala @@ -138,7 +138,7 @@ class ReceiverSuite extends FunSuite with Timeouts { blockGenerator.start() var count = 0 while(System.currentTimeMillis - startTime < waitTime) { - blockGenerator addData count + blockGenerator.addData(count) generatedData += count count += 1 Thread.sleep(10) @@ -168,7 +168,7 @@ class ReceiverSuite extends FunSuite with Timeouts { blockGenerator.start() var count = 0 while(System.currentTimeMillis - startTime < waitTime) { - blockGenerator addData count + blockGenerator.addData(count) generatedData += count count += 1 Thread.sleep(1) From eae4ad606e60f940a1537feea01e0c2cc4fb6ae8 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 13 Nov 2014 14:54:14 -0800 Subject: [PATCH 21/21] Refectored KafkaStreamSuiteBased to eliminate KafkaTestUtils and made Java more robust. --- .../streaming/kafka/JavaKafkaStreamSuite.java | 9 +- .../streaming/kafka/KafkaStreamSuite.scala | 151 ++++++++---------- 2 files changed, 73 insertions(+), 87 deletions(-) diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java index ce48dfbc0fcf1..6e1abf3f385ee 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java @@ -127,11 +127,16 @@ public Void call(JavaPairRDD rdd) throws Exception { ); ssc.start(); - ssc.awaitTermination(3000); - + long startTime = System.currentTimeMillis(); + boolean sizeMatches = false; + while (!sizeMatches && System.currentTimeMillis() - startTime < 20000) { + sizeMatches = sent.size() == result.size(); + Thread.sleep(200); + } Assert.assertEquals(sent.size(), result.size()); for (String k : sent.keySet()) { Assert.assertEquals(sent.get(k).intValue(), result.get(k).intValue()); } + ssc.stop(); } } diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala index 6e24b6f7ffb3b..b19c053ebfc44 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala @@ -46,8 +46,7 @@ import org.apache.spark.util.Utils * This is an abstract base class for Kafka testsuites. This has the functionality to set up * and tear down local Kafka servers, and to push data using Kafka producers. */ -abstract class KafkaStreamSuiteBase extends FunSuite with Logging { - import KafkaTestUtils._ +abstract class KafkaStreamSuiteBase extends FunSuite with Eventually with Logging { var zkAddress: String = _ var zkClient: ZkClient = _ @@ -78,7 +77,7 @@ abstract class KafkaStreamSuiteBase extends FunSuite with Logging { var bindSuccess: Boolean = false while(!bindSuccess) { try { - val brokerProps = getBrokerConfig(brokerPort, zkAddress) + val brokerProps = getBrokerConfig() brokerConf = new KafkaConfig(brokerProps) server = new KafkaServer(brokerConf) logInfo("==================== 2 ====================") @@ -134,111 +133,43 @@ abstract class KafkaStreamSuiteBase extends FunSuite with Logging { CreateTopicCommand.createTopic(zkClient, topic, 1, 1, "0") logInfo("==================== 5 ====================") // wait until metadata is propagated - waitUntilMetadataIsPropagated(Seq(server), topic, 0, 1000) + waitUntilMetadataIsPropagated(topic, 0) } def produceAndSendMessage(topic: String, sent: Map[String, Int]) { - val brokerAddr = brokerConf.hostName + ":" + brokerConf.port - producer = new Producer[String, String](new ProducerConfig(getProducerConfig(brokerAddr))) + producer = new Producer[String, String](new ProducerConfig(getProducerConfig())) producer.send(createTestMessage(topic, sent): _*) producer.close() logInfo("==================== 6 ====================") } -} - -class KafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter with Eventually { - var ssc: StreamingContext = _ - - before { - setupKafka() - } - - after { - if (ssc != null) { - ssc.stop() - ssc = null - } - tearDownKafka() - } - - test("Kafka input stream") { - val sparkConf = new SparkConf().setMaster("local[4]").setAppName(this.getClass.getSimpleName) - ssc = new StreamingContext(sparkConf, Milliseconds(500)) - val topic = "topic1" - val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) - createTopic(topic) - produceAndSendMessage(topic, sent) - - val kafkaParams = Map("zookeeper.connect" -> zkAddress, - "group.id" -> s"test-consumer-${Random.nextInt(10000)}", - "auto.offset.reset" -> "smallest") - - val stream = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( - ssc, - kafkaParams, - Map(topic -> 1), - StorageLevel.MEMORY_ONLY) - val result = new mutable.HashMap[String, Long]() - stream.map { case (k, v) => v } - .countByValue() - .foreachRDD { r => - val ret = r.collect() - ret.toMap.foreach { kv => - val count = result.getOrElseUpdate(kv._1, 0) + kv._2 - result.put(kv._1, count) - } - } - ssc.start() - eventually(timeout(3000 milliseconds), interval(100 milliseconds)) { - assert(sent.size === result.size) - sent.keys.foreach { k => assert(sent(k) === result(k).toInt) } - } - - ssc.stop() - } -} - -object KafkaTestUtils { - - def getBrokerConfig(port: Int, zkConnect: String): Properties = { + private def getBrokerConfig(): Properties = { val props = new Properties() props.put("broker.id", "0") props.put("host.name", "localhost") - props.put("port", port.toString) + props.put("port", brokerPort.toString) props.put("log.dir", Utils.createTempDir().getAbsolutePath) - props.put("zookeeper.connect", zkConnect) + props.put("zookeeper.connect", zkAddress) props.put("log.flush.interval.messages", "1") props.put("replica.socket.timeout.ms", "1500") props } - def getProducerConfig(brokerList: String): Properties = { + private def getProducerConfig(): Properties = { + val brokerAddr = brokerConf.hostName + ":" + brokerConf.port val props = new Properties() - props.put("metadata.broker.list", brokerList) + props.put("metadata.broker.list", brokerAddr) props.put("serializer.class", classOf[StringEncoder].getName) props } - def waitUntilTrue(condition: () => Boolean, waitTime: Long): Boolean = { - val startTime = System.currentTimeMillis() - while (true) { - if (condition()) - return true - if (System.currentTimeMillis() > startTime + waitTime) - return false - Thread.sleep(waitTime.min(100L)) + private def waitUntilMetadataIsPropagated(topic: String, partition: Int) { + eventually(timeout(1000 milliseconds), interval(100 milliseconds)) { + assert( + server.apis.leaderCache.keySet.contains(TopicAndPartition(topic, partition)), + s"Partition [$topic, $partition] metadata not propagated after timeout" + ) } - // Should never go to here - throw new RuntimeException("unexpected error") - } - - def waitUntilMetadataIsPropagated(servers: Seq[KafkaServer], topic: String, partition: Int, - timeout: Long) { - assert(waitUntilTrue(() => - servers.foldLeft(true)(_ && _.apis.leaderCache.keySet.contains( - TopicAndPartition(topic, partition))), timeout), - s"Partition [$topic, $partition] metadata not propagated after timeout") } class EmbeddedZookeeper(val zkConnect: String) { @@ -264,3 +195,53 @@ object KafkaTestUtils { } } } + + +class KafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter { + var ssc: StreamingContext = _ + + before { + setupKafka() + } + + after { + if (ssc != null) { + ssc.stop() + ssc = null + } + tearDownKafka() + } + + test("Kafka input stream") { + val sparkConf = new SparkConf().setMaster("local[4]").setAppName(this.getClass.getSimpleName) + ssc = new StreamingContext(sparkConf, Milliseconds(500)) + val topic = "topic1" + val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) + createTopic(topic) + produceAndSendMessage(topic, sent) + + val kafkaParams = Map("zookeeper.connect" -> zkAddress, + "group.id" -> s"test-consumer-${Random.nextInt(10000)}", + "auto.offset.reset" -> "smallest") + + val stream = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( + ssc, kafkaParams, Map(topic -> 1), StorageLevel.MEMORY_ONLY) + val result = new mutable.HashMap[String, Long]() + stream.map(_._2).countByValue().foreachRDD { r => + val ret = r.collect() + ret.toMap.foreach { kv => + val count = result.getOrElseUpdate(kv._1, 0) + kv._2 + result.put(kv._1, count) + } + } + ssc.start() + eventually(timeout(10000 milliseconds), interval(100 milliseconds)) { + assert(sent.size === result.size) + sent.keys.foreach { k => + assert(sent(k) === result(k).toInt) + } + } + ssc.stop() + } +} +