├── .gitignore ├── src ├── main │ ├── scala │ │ └── com │ │ │ └── github │ │ │ └── maelstrom │ │ │ ├── controller │ │ │ ├── IControllerKafka.scala │ │ │ ├── ControllerKafkaPartition.scala │ │ │ ├── ControllerKafkaTopic.scala │ │ │ └── ControllerKafkaTopics.scala │ │ │ ├── ProcessorRunner.scala │ │ │ ├── Logging.scala │ │ │ ├── StreamProcessor.scala │ │ │ └── KafkaRDD.scala │ └── java │ │ └── com │ │ └── github │ │ └── maelstrom │ │ └── consumer │ │ ├── InfiniteRetryStrategy.java │ │ ├── LeaderBrokerChecker.java │ │ ├── KafkaConsumerPoolFactory.java │ │ ├── OffsetManager.java │ │ ├── KafkaConsumerPool.java │ │ ├── KafkaConsumer.java │ │ └── KafkaMetaData.java └── test │ ├── resources │ └── log4j.properties │ ├── scala │ └── com │ │ └── github │ │ └── maelstrom │ │ └── test │ │ └── scala │ │ ├── StreamSingleTopic.scala │ │ ├── StreamMultiTopic.scala │ │ └── StreamMultiTopicMultiProc.scala │ └── java │ └── com │ └── github │ └── maelstrom │ └── test │ └── java │ ├── StreamMultiTopic.java │ ├── StreamSingleTopic.java │ └── StreamMultiTopicMultiProc.java ├── README.md ├── pom.xml └── LICENSE /.gitignore: -------------------------------------------------------------------------------- 1 | target/ 2 | .cache 3 | .idea 4 | .settings/ 5 | *.iml 6 | -------------------------------------------------------------------------------- /src/main/scala/com/github/maelstrom/controller/IControllerKafka.scala: -------------------------------------------------------------------------------- 1 | package com.github.maelstrom.controller 2 | 3 | import org.apache.spark.rdd.RDD 4 | 5 | trait IControllerKafka[K, V] { 6 | def close() 7 | def getLag: Long 8 | def commit() 9 | def getRDD: RDD[(K, V)] 10 | } 11 | -------------------------------------------------------------------------------- /src/test/resources/log4j.properties: -------------------------------------------------------------------------------- 1 | ### File to be added to the Spark Driver 2 | # Set root logger level to DEBUG and its only appender to console. 3 | log4j.rootLogger=INFO, console 4 | 5 | # console is set to be a ConsoleAppender/PatternLayout. 6 | log4j.appender.console=org.apache.log4j.ConsoleAppender 7 | log4j.appender.console.layout=org.apache.log4j.PatternLayout 8 | log4j.appender.console.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n 9 | 10 | log4j.logger.org.apache.spark=WARN 11 | log4j.logger.org.apache.spark.streaming=INFO 12 | 13 | # MAELSTROM TEST 14 | log4j.logger.com.github.maelstrom=DEBUG 15 | -------------------------------------------------------------------------------- /src/main/scala/com/github/maelstrom/ProcessorRunner.scala: -------------------------------------------------------------------------------- 1 | package com.github.maelstrom 2 | 3 | import java.util.concurrent.{CountDownLatch, ExecutorService, Executors} 4 | 5 | import scala.collection.mutable.ListBuffer 6 | 7 | /** 8 | * @author Jeoffrey Lim 9 | * @version 0.2 10 | */ 11 | 12 | class ProcessorRunner { 13 | final private val processors: ListBuffer[StreamProcessor[_, _]] = ListBuffer[StreamProcessor[_, _]]() 14 | 15 | final def addProcessor(processor: StreamProcessor[_, _]): ProcessorRunner = { 16 | processors += processor 17 | this 18 | } 19 | 20 | final def start() { 21 | val countDownLatch: CountDownLatch = new CountDownLatch(processors.size) 22 | val taskExecutor: ExecutorService = Executors.newFixedThreadPool(processors.size) 23 | 24 | for (p <- processors) taskExecutor.execute(p) 25 | 26 | try { 27 | countDownLatch.await() 28 | } catch { 29 | case e: InterruptedException => 30 | Thread.currentThread.interrupt() 31 | } 32 | } 33 | 34 | final def stop() { 35 | for (p <- processors) p.stop() 36 | } 37 | } -------------------------------------------------------------------------------- /src/test/scala/com/github/maelstrom/test/scala/StreamSingleTopic.scala: -------------------------------------------------------------------------------- 1 | package com.github.maelstrom.test.scala 2 | 3 | import com.github.maelstrom.StreamProcessor 4 | import com.github.maelstrom.consumer.{KafkaConsumerPoolFactory, OffsetManager} 5 | import com.github.maelstrom.controller.ControllerKafkaTopics 6 | import kafka.serializer.StringDecoder 7 | import org.apache.spark.rdd.RDD 8 | import org.apache.spark.{SparkConf, SparkContext} 9 | import org.slf4j.{Logger, LoggerFactory} 10 | 11 | 12 | object StreamSingleTopic { 13 | private val logger: Logger = LoggerFactory.getLogger(getClass) 14 | 15 | def main(args: Array[String]) { 16 | val sparkConf = new SparkConf().setMaster("local[4]").setAppName("StreamSingleTopic") 17 | val sc = new SparkContext(sparkConf) 18 | val curator = OffsetManager.createCurator("127.0.0.1:2181") 19 | val poolFactory = new KafkaConsumerPoolFactory[String, String]("127.0.0.1:9092", classOf[StringDecoder], classOf[StringDecoder]) 20 | val topics = new ControllerKafkaTopics[String, String](sc, curator, poolFactory) 21 | val topic = topics.registerTopic("test_group", "test") 22 | 23 | new StreamProcessor[String, String](topic) { 24 | final def process() { 25 | val rdd: RDD[(String, String)] = fetch() 26 | 27 | rdd.foreachPartition { partitionData => 28 | partitionData.foreach { record => 29 | logger.info(s"key=${record._1} val=${record._2}") 30 | } 31 | } 32 | 33 | commit() 34 | } 35 | }.run() 36 | 37 | sc.stop() 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /src/test/scala/com/github/maelstrom/test/scala/StreamMultiTopic.scala: -------------------------------------------------------------------------------- 1 | package com.github.maelstrom.test.scala 2 | 3 | import com.github.maelstrom.StreamProcessor 4 | import com.github.maelstrom.consumer.{KafkaConsumerPoolFactory, OffsetManager} 5 | import com.github.maelstrom.controller.ControllerKafkaTopics 6 | import kafka.serializer.StringDecoder 7 | import org.apache.spark.rdd.RDD 8 | import org.apache.spark.{SparkConf, SparkContext} 9 | import org.slf4j.{Logger, LoggerFactory} 10 | 11 | object StreamMultiTopic { 12 | private val logger: Logger = LoggerFactory.getLogger(getClass) 13 | 14 | def main(args: Array[String]) { 15 | val sparkConf = new SparkConf().setMaster("local[4]").setAppName("StreamMultiTopic") 16 | val sc = new SparkContext(sparkConf) 17 | val curator = OffsetManager.createCurator("127.0.0.1:2181") 18 | val poolFactory = new KafkaConsumerPoolFactory[String, String]("127.0.0.1:9092", classOf[StringDecoder], classOf[StringDecoder]) 19 | val topics = new ControllerKafkaTopics[String, String](sc, curator, poolFactory) 20 | topics.registerTopic("test_multi", "test") 21 | topics.registerTopic("test_multi", "test2") 22 | 23 | new StreamProcessor[String, String](topics) { 24 | final def process() { 25 | val rdd: RDD[(String, String)] = fetch() 26 | 27 | rdd.foreachPartition { partitionData => 28 | partitionData.foreach { record => 29 | logger.info(s"key=${record._1} val=]${record._2}") 30 | } 31 | } 32 | 33 | commit() 34 | } 35 | }.run() 36 | 37 | sc.stop() 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /src/test/scala/com/github/maelstrom/test/scala/StreamMultiTopicMultiProc.scala: -------------------------------------------------------------------------------- 1 | package com.github.maelstrom.test.scala 2 | 3 | import com.github.maelstrom.consumer.{KafkaConsumerPoolFactory, OffsetManager} 4 | import com.github.maelstrom.controller.ControllerKafkaTopics 5 | import com.github.maelstrom.{ProcessorRunner, StreamProcessor} 6 | import kafka.serializer.StringDecoder 7 | import org.apache.spark.rdd.RDD 8 | import org.apache.spark.{SparkConf, SparkContext} 9 | import org.slf4j.{Logger, LoggerFactory} 10 | 11 | object StreamMultiTopicMultiProc { 12 | private val logger: Logger = LoggerFactory.getLogger(getClass) 13 | 14 | def main(args: Array[String]) { 15 | val sparkConf = new SparkConf().setMaster("local[4]").setAppName("StreamMultiTopicMultiProc") 16 | val sc = new SparkContext(sparkConf) 17 | val curator = OffsetManager.createCurator("127.0.0.1:2181") 18 | val poolFactory = new KafkaConsumerPoolFactory[String, String]("127.0.0.1:9092", classOf[StringDecoder], classOf[StringDecoder]) 19 | val topics = new ControllerKafkaTopics[String, String](sc, curator, poolFactory) 20 | 21 | new ProcessorRunner().addProcessor(new StreamProcessor[String, String](topics.registerTopic("test_multi_proc", "test", 1000)) { 22 | final def process() { 23 | val rdd: RDD[(String, String)] = fetch() 24 | 25 | rdd.foreachPartition { partitionData => 26 | partitionData.foreach { record => 27 | logger.info(s"key=${record._1} val=${record._2}") 28 | } 29 | } 30 | 31 | commit() 32 | } 33 | }).addProcessor(new StreamProcessor[String, String](topics.registerTopic("test_multi_proc", "test2", 500)) { 34 | final def process() { 35 | val rdd: RDD[(String, String)] = fetch() 36 | 37 | rdd.foreachPartition { partitionData => 38 | partitionData.foreach { record => 39 | logger.info(s"key=${record._1} val=${record._2}") 40 | } 41 | } 42 | 43 | commit() 44 | } 45 | }).start() 46 | 47 | sc.stop() 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /src/test/java/com/github/maelstrom/test/java/StreamMultiTopic.java: -------------------------------------------------------------------------------- 1 | package com.github.maelstrom.test.java; 2 | 3 | import com.github.maelstrom.StreamProcessor; 4 | import com.github.maelstrom.consumer.KafkaConsumerPoolFactory; 5 | import com.github.maelstrom.consumer.OffsetManager; 6 | import com.github.maelstrom.controller.ControllerKafkaTopics; 7 | import kafka.serializer.StringDecoder; 8 | import org.apache.curator.framework.CuratorFramework; 9 | import org.apache.spark.SparkConf; 10 | import org.apache.spark.api.java.JavaRDD; 11 | import org.apache.spark.api.java.JavaSparkContext; 12 | import org.apache.spark.api.java.function.VoidFunction; 13 | import org.slf4j.Logger; 14 | import org.slf4j.LoggerFactory; 15 | import scala.Tuple2; 16 | 17 | import java.util.Iterator; 18 | 19 | public class StreamMultiTopic { 20 | private static final Logger LOG = LoggerFactory.getLogger(StreamMultiTopic.class); 21 | 22 | public static void main(String[] args) { 23 | SparkConf sparkConf = new SparkConf().setMaster("local[1]").setAppName("StreamMultiTopic"); 24 | JavaSparkContext sc = new JavaSparkContext(sparkConf); 25 | 26 | CuratorFramework curator = OffsetManager.createCurator("127.0.0.1:2181"); 27 | KafkaConsumerPoolFactory poolFactory = new KafkaConsumerPoolFactory<>("127.0.0.1:9092", StringDecoder.class, StringDecoder.class); 28 | 29 | ControllerKafkaTopics topics = new ControllerKafkaTopics<>(sc.sc(), curator, poolFactory); 30 | topics.registerTopic("test_multi", "test"); 31 | topics.registerTopic("test_multi", "test2"); 32 | 33 | new StreamProcessor(topics) { 34 | @Override 35 | public final void process() { 36 | JavaRDD> rdd = fetch().toJavaRDD(); 37 | 38 | rdd.foreachPartition(new VoidFunction>>() { 39 | @Override 40 | public final void call(final Iterator> it) { 41 | while (it.hasNext()) { 42 | Tuple2 e = it.next(); 43 | LOG.info("key=" + e._1 + " message=" + e._2()); 44 | } 45 | } 46 | }); 47 | 48 | commit(); 49 | } 50 | }.run(); 51 | 52 | sc.sc().stop(); 53 | } 54 | } 55 | -------------------------------------------------------------------------------- /src/test/java/com/github/maelstrom/test/java/StreamSingleTopic.java: -------------------------------------------------------------------------------- 1 | package com.github.maelstrom.test.java; 2 | 3 | import com.github.maelstrom.StreamProcessor; 4 | import com.github.maelstrom.consumer.KafkaConsumerPoolFactory; 5 | import com.github.maelstrom.consumer.OffsetManager; 6 | import com.github.maelstrom.controller.ControllerKafkaTopic; 7 | import com.github.maelstrom.controller.ControllerKafkaTopics; 8 | import kafka.serializer.StringDecoder; 9 | import org.apache.curator.framework.CuratorFramework; 10 | import org.apache.spark.SparkConf; 11 | import org.apache.spark.api.java.JavaRDD; 12 | import org.apache.spark.api.java.JavaSparkContext; 13 | import org.apache.spark.api.java.function.VoidFunction; 14 | import org.slf4j.Logger; 15 | import org.slf4j.LoggerFactory; 16 | import scala.Tuple2; 17 | 18 | import java.util.Iterator; 19 | 20 | public class StreamSingleTopic { 21 | private static final Logger LOG = LoggerFactory.getLogger(StreamSingleTopic.class); 22 | 23 | public static void main(String[] args) { 24 | SparkConf sparkConf = new SparkConf().setMaster("local[4]").setAppName("StreamSingleTopic"); 25 | JavaSparkContext sc = new JavaSparkContext(sparkConf); 26 | 27 | CuratorFramework curator = OffsetManager.createCurator("127.0.0.1:2181"); 28 | KafkaConsumerPoolFactory poolFactory = new KafkaConsumerPoolFactory<>("127.0.0.1:9092", StringDecoder.class, StringDecoder.class); 29 | 30 | ControllerKafkaTopics topics = new ControllerKafkaTopics<>(sc.sc(), curator, poolFactory); 31 | ControllerKafkaTopic topic = topics.registerTopic("test_group", "test"); 32 | 33 | new StreamProcessor(topic) { 34 | @Override 35 | public final void process() { 36 | JavaRDD> rdd = fetch().toJavaRDD(); 37 | 38 | rdd.foreachPartition(new VoidFunction>>() { 39 | @Override 40 | public final void call(final Iterator> it) { 41 | while (it.hasNext()) { 42 | Tuple2 e = it.next(); 43 | LOG.info("key=" + e._1 + " message=" + e._2()); 44 | } 45 | } 46 | }); 47 | 48 | commit(); 49 | } 50 | }.run(); 51 | 52 | sc.sc().stop(); 53 | } 54 | } 55 | -------------------------------------------------------------------------------- /src/main/scala/com/github/maelstrom/controller/ControllerKafkaPartition.scala: -------------------------------------------------------------------------------- 1 | package com.github.maelstrom.controller 2 | 3 | import com.github.maelstrom.{KafkaRDDUtils, Logging} 4 | import com.github.maelstrom.consumer.{KafkaConsumer, KafkaConsumerPoolFactory, OffsetManager} 5 | import kafka.serializer.Decoder 6 | import org.apache.curator.framework.CuratorFramework 7 | import org.apache.spark.rdd.RDD 8 | import org.apache.spark.storage.StorageLevel 9 | import org.apache.spark.SparkContext 10 | 11 | class ControllerKafkaPartition[K, V](sc: SparkContext, 12 | curator: CuratorFramework, 13 | poolFactory: KafkaConsumerPoolFactory[_,_], 14 | val consumerGroup: String, 15 | val topic: String, 16 | val partitionId: Int, 17 | val maxQueue: Int = 5000) extends IControllerKafka[K, V] with Logging { 18 | final private val consumer: KafkaConsumer[K, V] = new KafkaConsumer[K, V](poolFactory.getBrokerList, 19 | consumerGroup, poolFactory.createKeyDecoder().asInstanceOf[Decoder[K]], 20 | poolFactory.createValueDecoder().asInstanceOf[Decoder[V]], topic, partitionId) 21 | final private val offsetManager: OffsetManager = new OffsetManager(curator, consumer, consumerGroup, topic, partitionId) 22 | private var stopAtOffset: Long = -1 23 | 24 | final def close() { 25 | consumer.close() 26 | } 27 | 28 | final def getLag: Long = { 29 | getLatestOffset - getLastOffset 30 | } 31 | 32 | final def getLastOffset: Long = { 33 | offsetManager.getLastOffset 34 | } 35 | 36 | final def getLatestOffset: Long = { 37 | consumer.getLatestOffset 38 | } 39 | 40 | final def getStopAtOffset: Long = { 41 | stopAtOffset 42 | } 43 | 44 | final def setStopAtOffset(stopAtOffset: Long) { 45 | this.stopAtOffset = stopAtOffset 46 | } 47 | 48 | final def commit() { 49 | if (stopAtOffset <= 0) return 50 | offsetManager.setLastOffset(stopAtOffset) 51 | offsetManager.storeLastOffset() 52 | stopAtOffset = -1 53 | } 54 | 55 | final def getRDD(): RDD[(K, V)] = { 56 | val lag: Long = getLag 57 | val perEach: Long = Math.max(1, maxQueue) 58 | var offsets: Map[Int, (Long, Long)] = Map() 59 | 60 | if (lag > maxQueue) { 61 | setStopAtOffset(Math.min(getLastOffset + perEach, getLastOffset + getLag)) 62 | logDebug(s"REAP PARTIAL: [$topic]-[$partitionId] : @$getStopAtOffset") 63 | } else { 64 | setStopAtOffset(getLatestOffset) 65 | logDebug(s"REAP FULL: [$topic]-[$partitionId] : @$getLatestOffset") 66 | } 67 | 68 | logDebug(s"OFFSET RANGE: [$topic]-[$partitionId] : $getLastOffset->$getStopAtOffset = ${getStopAtOffset - getLastOffset}") 69 | offsets += (partitionId -> (getLastOffset, getStopAtOffset)) 70 | 71 | KafkaRDDUtils.createKafkaRDD(sc, poolFactory, topic, offsets) 72 | .persist(StorageLevel.MEMORY_ONLY) 73 | .setName("KafkaRDD-" + consumerGroup + ":" + topic + ":" + partitionId) 74 | .asInstanceOf[RDD[(K, V)]] 75 | } 76 | } 77 | -------------------------------------------------------------------------------- /src/main/scala/com/github/maelstrom/Logging.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | package com.github.maelstrom 18 | 19 | import org.slf4j.{Logger, LoggerFactory} 20 | 21 | // Taken from Spark's Logging utility 22 | 23 | /** 24 | * Utility trait for classes that want to log data. Creates a SLF4J logger for the class and allows 25 | * logging messages at different levels using methods that only evaluate parameters lazily if the 26 | * log level is enabled. 27 | */ 28 | trait Logging { 29 | 30 | // Make the log field transient so that objects with Logging can 31 | // be serialized and used on another machine 32 | @transient private var log_ : Logger = null 33 | 34 | // Method to get the logger name for this object 35 | protected def logName = { 36 | // Ignore trailing $'s in the class names for Scala objects 37 | this.getClass.getName.stripSuffix("$") 38 | } 39 | 40 | // Method to get or create the logger for this object 41 | protected def log: Logger = { 42 | if (log_ == null) { 43 | log_ = LoggerFactory.getLogger(logName) 44 | } 45 | log_ 46 | } 47 | 48 | // Log methods that take only a String 49 | protected def logInfo(msg: => String) { 50 | if (log.isInfoEnabled) log.info(msg) 51 | } 52 | 53 | protected def logDebug(msg: => String) { 54 | if (log.isDebugEnabled) log.debug(msg) 55 | } 56 | 57 | protected def logTrace(msg: => String) { 58 | if (log.isTraceEnabled) log.trace(msg) 59 | } 60 | 61 | protected def logWarning(msg: => String) { 62 | if (log.isWarnEnabled) log.warn(msg) 63 | } 64 | 65 | protected def logError(msg: => String) { 66 | if (log.isErrorEnabled) log.error(msg) 67 | } 68 | 69 | // Log methods that take Throwables (Exceptions/Errors) too 70 | protected def logInfo(msg: => String, throwable: Throwable) { 71 | if (log.isInfoEnabled) log.info(msg, throwable) 72 | } 73 | 74 | protected def logDebug(msg: => String, throwable: Throwable) { 75 | if (log.isDebugEnabled) log.debug(msg, throwable) 76 | } 77 | 78 | protected def logTrace(msg: => String, throwable: Throwable) { 79 | if (log.isTraceEnabled) log.trace(msg, throwable) 80 | } 81 | 82 | protected def logWarning(msg: => String, throwable: Throwable) { 83 | if (log.isWarnEnabled) log.warn(msg, throwable) 84 | } 85 | 86 | protected def logError(msg: => String, throwable: Throwable) { 87 | if (log.isErrorEnabled) log.error(msg, throwable) 88 | } 89 | 90 | protected def isTraceEnabled(): Boolean = { 91 | log.isTraceEnabled 92 | } 93 | } -------------------------------------------------------------------------------- /src/test/java/com/github/maelstrom/test/java/StreamMultiTopicMultiProc.java: -------------------------------------------------------------------------------- 1 | package com.github.maelstrom.test.java; 2 | 3 | import com.github.maelstrom.ProcessorRunner; 4 | import com.github.maelstrom.StreamProcessor; 5 | import com.github.maelstrom.consumer.KafkaConsumerPoolFactory; 6 | import com.github.maelstrom.consumer.OffsetManager; 7 | import com.github.maelstrom.controller.ControllerKafkaTopics; 8 | import kafka.serializer.StringDecoder; 9 | import org.apache.curator.framework.CuratorFramework; 10 | import org.apache.spark.SparkConf; 11 | import org.apache.spark.api.java.JavaRDD; 12 | import org.apache.spark.api.java.JavaSparkContext; 13 | import org.apache.spark.api.java.function.VoidFunction; 14 | import org.slf4j.Logger; 15 | import org.slf4j.LoggerFactory; 16 | import scala.Tuple2; 17 | 18 | import java.util.Iterator; 19 | 20 | /** 21 | * User: jeoffrey 22 | * Date: 8/23/16 23 | * Time: 5:35 PM. 24 | */ 25 | public class StreamMultiTopicMultiProc { 26 | private static final Logger LOG = LoggerFactory.getLogger(StreamMultiTopicMultiProc.class); 27 | 28 | public static void main(String[] args) { 29 | SparkConf sparkConf = new SparkConf().setMaster("local[4]").setAppName("StreamMultiTopicMultiProc"); 30 | JavaSparkContext sc = new JavaSparkContext(sparkConf); 31 | 32 | CuratorFramework curator = OffsetManager.createCurator("127.0.0.1:2181"); 33 | KafkaConsumerPoolFactory poolFactory = new KafkaConsumerPoolFactory<>("127.0.0.1:9092", StringDecoder.class, StringDecoder.class); 34 | 35 | ControllerKafkaTopics topics = new ControllerKafkaTopics<>(sc.sc(), curator, poolFactory); 36 | 37 | new ProcessorRunner().addProcessor(new StreamProcessor(topics.registerTopic("test_multi_proc", "test", 1000)) { 38 | @Override 39 | public final void process() { 40 | JavaRDD> rdd = fetch().toJavaRDD(); 41 | 42 | rdd.foreachPartition(new VoidFunction>>() { 43 | @Override 44 | public final void call(final Iterator> it) { 45 | while (it.hasNext()) { 46 | Tuple2 e = it.next(); 47 | LOG.info("test - " + "key=" + e._1 + " message=" + e._2()); 48 | } 49 | } 50 | }); 51 | 52 | commit(); 53 | } 54 | }).addProcessor(new StreamProcessor(topics.registerTopic("test_multi_proc", "test2", 500)) { 55 | @Override 56 | public final void process() { 57 | JavaRDD> rdd = fetch().toJavaRDD(); 58 | 59 | rdd.foreachPartition(new VoidFunction>>() { 60 | @Override 61 | public final void call(final Iterator> it) { 62 | while (it.hasNext()) { 63 | Tuple2 e = it.next(); 64 | LOG.info("test2 - " + "key=" + e._1 + " message=" + e._2()); 65 | } 66 | } 67 | }); 68 | 69 | commit(); 70 | } 71 | }).start(); 72 | 73 | sc.sc().stop(); 74 | } 75 | } 76 | -------------------------------------------------------------------------------- /src/main/java/com/github/maelstrom/consumer/InfiniteRetryStrategy.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.github.maelstrom.consumer; 19 | 20 | import com.github.rholder.retry.Retryer; 21 | import com.github.rholder.retry.RetryerBuilder; 22 | import com.github.rholder.retry.StopStrategies; 23 | import com.github.rholder.retry.WaitStrategies; 24 | import com.google.common.base.Predicates; 25 | 26 | import java.util.concurrent.Callable; 27 | import java.util.concurrent.TimeUnit; 28 | 29 | /** 30 | * @author Jeoffrey Lim 31 | * @version 0.2 32 | */ 33 | @SuppressWarnings("unused") 34 | public class InfiniteRetryStrategy { 35 | 36 | private static final InfiniteRetryStrategy intResponseRetryer = new InfiniteRetryStrategy<>(); 37 | 38 | private static final InfiniteRetryStrategy longResponseRetryer = new InfiniteRetryStrategy<>(); 39 | 40 | private static final InfiniteRetryStrategy booleanResponseRetryer = new InfiniteRetryStrategy() { 41 | @Override 42 | protected Retryer init(final RetryerBuilder builder) { 43 | builder.retryIfResult(Predicates.or(Predicates.isNull(), Predicates.equalTo(false))); 44 | return builder.build(); 45 | } 46 | }; 47 | 48 | public final Retryer retryer; 49 | 50 | public InfiniteRetryStrategy() { 51 | this.retryer = init(RetryerBuilder.newBuilder() 52 | .retryIfResult(Predicates.equalTo((V)null)) 53 | .retryIfException() 54 | .withWaitStrategy(WaitStrategies.fibonacciWait(100, 2, TimeUnit.MINUTES)) 55 | .withStopStrategy(StopStrategies.neverStop())); 56 | } 57 | 58 | protected Retryer init(final RetryerBuilder builder) { 59 | return builder.build(); 60 | } 61 | 62 | public V retryInfinitely(final Callable callable) { 63 | try { 64 | return retryer.call(callable); 65 | } catch (Exception t) { 66 | throw new IllegalStateException("Unable to perform infinite retry", t); 67 | } 68 | } 69 | 70 | /** 71 | * Integer Response Retryer 72 | */ 73 | public static long retryInfinitelyInt(final Callable callable) { 74 | return intResponseRetryer.retryInfinitely(callable); 75 | } 76 | 77 | /** 78 | * Long Response Retryer 79 | */ 80 | public static long retryInfinitelyLong(final Callable callable) { 81 | return longResponseRetryer.retryInfinitely(callable); 82 | } 83 | 84 | /** 85 | * Boolean Response Retryer 86 | */ 87 | public static boolean retryInfinitelyBoolean(final Callable callable) { 88 | return booleanResponseRetryer.retryInfinitely(callable); 89 | } 90 | } 91 | -------------------------------------------------------------------------------- /src/main/scala/com/github/maelstrom/controller/ControllerKafkaTopic.scala: -------------------------------------------------------------------------------- 1 | package com.github.maelstrom.controller 2 | 3 | import com.github.maelstrom.consumer.{KafkaConsumerPoolFactory, KafkaMetaData} 4 | import com.github.maelstrom.{KafkaRDDUtils, Logging} 5 | import kafka.javaapi.TopicMetadata 6 | import org.apache.curator.framework.CuratorFramework 7 | import org.apache.spark.SparkContext 8 | import org.apache.spark.rdd.RDD 9 | import org.apache.spark.storage.StorageLevel 10 | 11 | import scala.collection.mutable 12 | 13 | class ControllerKafkaTopic[K, V] (sc: SparkContext, 14 | curator: CuratorFramework, 15 | poolFactory: KafkaConsumerPoolFactory[_,_], 16 | val consumerGroup: String, 17 | val topic: String, 18 | val maxQueue: Int = 5000) extends IControllerKafka[K, V] with Logging { 19 | val partitionMap = mutable.Map[Integer, ControllerKafkaPartition[K, V]]() 20 | val metaData: java.util.Map[String, TopicMetadata] = KafkaMetaData.getTopicMetaData(poolFactory.getBrokerList, java.util.Collections.singletonList(topic)) 21 | 22 | if (metaData.containsKey(topic)) { 23 | val topicMetadata: TopicMetadata = metaData.get(topic) 24 | val partionCount: Int = topicMetadata.partitionsMetadata.size 25 | 26 | for(i <- 0 until partionCount) 27 | partitionMap.put(i, new ControllerKafkaPartition[K, V](sc, curator, poolFactory, consumerGroup, topic, i)) 28 | 29 | } else { 30 | throw new IllegalArgumentException("Topic " + topic + " not found") 31 | } 32 | 33 | override def hashCode: Int = { 34 | java.util.Objects.hash(consumerGroup, topic) 35 | } 36 | 37 | override def equals(other: Any): Boolean = { 38 | !(other == null || !other.isInstanceOf[ControllerKafkaTopic[_, _]]) && other.asInstanceOf[ControllerKafkaTopic[_, _]].topic == topic && other.asInstanceOf[ControllerKafkaTopic[_, _]].consumerGroup == consumerGroup 39 | } 40 | 41 | final def getPartitionCount: Int = { 42 | partitionMap.size 43 | } 44 | 45 | final def close() { 46 | for (kafkaPartition <- partitionMap.values) kafkaPartition.close() 47 | } 48 | 49 | final def getLag: Long = { 50 | var lag: Long = 0L 51 | for (kafkaPartition <- partitionMap.values) lag += kafkaPartition.getLatestOffset - kafkaPartition.getLastOffset 52 | lag 53 | } 54 | 55 | final def commit() { 56 | for (kafkaPartition <- partitionMap.values) kafkaPartition.commit() 57 | } 58 | 59 | def getPartitions: Iterable[ControllerKafkaPartition[K, V]] = { 60 | partitionMap.values 61 | } 62 | 63 | final def getRDD(): RDD[(K, V)] = { 64 | val lag: Long = getLag 65 | val perEach: Long = Math.max(1, maxQueue / getPartitionCount) 66 | var offsets: Map[Int, (Long, Long)] = Map() 67 | 68 | for (kafkaPartition <- getPartitions) { 69 | if (lag > maxQueue) { 70 | kafkaPartition.setStopAtOffset(Math.min(kafkaPartition.getLastOffset + perEach, kafkaPartition.getLastOffset + kafkaPartition.getLag)) 71 | logDebug(s"REAP PARTIAL: [${kafkaPartition.topic}]-[${kafkaPartition.partitionId}] : @${kafkaPartition.getStopAtOffset}") 72 | } else { 73 | kafkaPartition.setStopAtOffset(kafkaPartition.getLatestOffset) 74 | logDebug(s"REAP FULL: [${kafkaPartition.topic}]-[${kafkaPartition.partitionId}] : @${kafkaPartition.getLatestOffset}") 75 | } 76 | logDebug(s"OFFSET RANGE: [${kafkaPartition.topic}]-[${kafkaPartition.partitionId}] : ${kafkaPartition.getLastOffset}->${kafkaPartition.getStopAtOffset} = ${kafkaPartition.getStopAtOffset - kafkaPartition.getLastOffset}") 77 | offsets += (kafkaPartition.partitionId -> (kafkaPartition.getLastOffset, kafkaPartition.getStopAtOffset)) 78 | } 79 | 80 | KafkaRDDUtils.createKafkaRDD(sc, poolFactory, topic, offsets) 81 | .persist(StorageLevel.MEMORY_ONLY) 82 | .setName("KafkaRDD-" + consumerGroup + ":" + topic) 83 | .asInstanceOf[RDD[(K, V)]] 84 | } 85 | } 86 | -------------------------------------------------------------------------------- /src/main/scala/com/github/maelstrom/StreamProcessor.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | *//* 17 | * Licensed to the Apache Software Foundation (ASF) under one or more 18 | * contributor license agreements. See the NOTICE file distributed with 19 | * this work for additional information regarding copyright ownership. 20 | * The ASF licenses this file to You under the Apache License, Version 2.0 21 | * (the "License"); you may not use this file except in compliance with 22 | * the License. You may obtain a copy of the License at 23 | * 24 | * http://www.apache.org/licenses/LICENSE-2.0 25 | * 26 | * Unless required by applicable law or agreed to in writing, software 27 | * distributed under the License is distributed on an "AS IS" BASIS, 28 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 29 | * See the License for the specific language governing permissions and 30 | * limitations under the License. 31 | */ 32 | package com.github.maelstrom 33 | 34 | import java.io.Serializable 35 | import java.util.concurrent.atomic.AtomicBoolean 36 | 37 | import com.github.maelstrom.controller.IControllerKafka 38 | import org.apache.spark.rdd.RDD 39 | 40 | /** 41 | * Base class for stream processing as a single unit, whether it could be a single 42 | * or multi topic (which is unique in Kafka). Many combinations can be done in 43 | * Kafka depending on the intended purpose. For example, one could read on the same 44 | * topic but have 2 different consumer groups & processors: a processor for storing data to DB, 45 | * and another for filtering messages and sending emails. 46 | * 47 | * @author Jeoffrey Lim 48 | * @version 0.2 49 | */ 50 | abstract class StreamProcessor[K, V](@transient val controller: IControllerKafka[K, V]) extends Runnable with Serializable { 51 | final private val isStopCalled: AtomicBoolean = new AtomicBoolean(false) 52 | 53 | final def run() { 54 | while (!isStopCalled.getAndSet(false)) { 55 | if (getLag > 0 && shouldProcess) process() 56 | else Thread.sleep(100L) 57 | } 58 | controller.close() 59 | stopped() 60 | } 61 | 62 | /** 63 | * Stop the stream processor 64 | */ 65 | final def stop() { 66 | isStopCalled.getAndSet(true) 67 | } 68 | 69 | /** 70 | * Get the lag of the registered controller 71 | * @return 72 | */ 73 | final def getLag: Long = { 74 | controller.getLag 75 | } 76 | 77 | /** 78 | * Fetch messages from the registered controller 79 | * @return RDD 80 | */ 81 | def fetch(): RDD[(K, V)] = { 82 | controller.getRDD 83 | } 84 | 85 | /** 86 | * Extend this function to provide other conditions before triggering processing. 87 | * An example of this is to check maximum lag (bucket full), or trigger every X time elapsed. 88 | * This would be useful most especially for data aggregation. 89 | * 90 | * @return true if stream processing should proceed 91 | */ 92 | protected def shouldProcess: Boolean = { 93 | true 94 | } 95 | 96 | /** 97 | * Main processing routine. This is where you fetch data as KafkaRDDs and use Spark to operate on the data. 98 | */ 99 | def process() 100 | 101 | /** 102 | * Helper function to commit the offsets 103 | */ 104 | final def commit() { 105 | controller.commit() 106 | } 107 | 108 | /** 109 | * Extend this if you want to be notified if stream processing has halted. 110 | */ 111 | protected def stopped() { 112 | } 113 | } -------------------------------------------------------------------------------- /src/main/java/com/github/maelstrom/consumer/LeaderBrokerChecker.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.github.maelstrom.consumer; 19 | 20 | import kafka.cluster.Broker; 21 | import org.slf4j.Logger; 22 | import org.slf4j.LoggerFactory; 23 | 24 | import java.util.List; 25 | import java.util.concurrent.atomic.AtomicBoolean; 26 | 27 | /** 28 | * @author Jeoffrey Lim 29 | * @version 0.2 30 | */ 31 | final class LeaderBrokerChecker implements Runnable { 32 | private static final Logger LOG = LoggerFactory.getLogger(LeaderBrokerChecker.class); 33 | 34 | private final String topic; 35 | private final int partitionId; 36 | 37 | private final List brokerList; 38 | private final Thread internalThread; 39 | private final AtomicBoolean isStopped = new AtomicBoolean(false); 40 | private final AtomicBoolean hasChanged = new AtomicBoolean(false); 41 | 42 | private static final long CHECK_EVERY = 300000; //5 minutes || 30000; //30 secs 43 | 44 | LeaderBrokerChecker(final List brokerList, final String topic, final int partitionId) { 45 | this.brokerList = brokerList; 46 | this.topic = topic; 47 | this.partitionId = partitionId; 48 | 49 | internalThread = new Thread(this); 50 | internalThread.setDaemon(true); 51 | internalThread.start(); 52 | } 53 | 54 | final void stop() { 55 | if (!isStopped.getAndSet(true)) { 56 | try { 57 | internalThread.join(); 58 | } catch (Exception e) { 59 | LOG.warn(e.getMessage(), e); 60 | } 61 | } 62 | } 63 | 64 | final boolean hasChanged() { 65 | return this.hasChanged.getAndSet(false); 66 | } 67 | 68 | private void setHasChanged() { 69 | hasChanged.getAndSet(true); 70 | } 71 | 72 | @Override 73 | public void run() { 74 | long lastRun = System.currentTimeMillis(); 75 | Broker lastBroker = KafkaMetaData.findNewLeader(brokerList, null, topic, partitionId).leader(); 76 | 77 | while (!isStopped.getAndSet(false)) { 78 | if ((System.currentTimeMillis() - lastRun) > CHECK_EVERY) { 79 | Broker checkBroker = KafkaMetaData.findNewLeader(brokerList, lastBroker, topic, partitionId).leader(); 80 | LOG.debug("[{}:{}] CHECK BROKERS Latest: {} Current: {}", topic, partitionId, checkBroker.toString(), lastBroker.toString()); 81 | 82 | // if *NOT* the same... 83 | if (!isSameBroker(lastBroker, checkBroker)) { 84 | LOG.warn("[{}:{}] Broker changed. New: {} Old: {}", topic, partitionId, checkBroker.toString(), lastBroker.toString()); 85 | lastBroker = checkBroker; 86 | setHasChanged(); 87 | } 88 | 89 | lastRun = System.currentTimeMillis(); 90 | } else { 91 | try { 92 | Thread.sleep(1000L); 93 | } catch (InterruptedException e) { 94 | LOG.warn(e.getMessage(), e); 95 | Thread.currentThread().interrupt(); 96 | break; 97 | } 98 | } 99 | } 100 | } 101 | 102 | public static boolean isSameBroker(final Broker lastBroker, final Broker checkBroker) { 103 | if (lastBroker == null || checkBroker == null) 104 | return false; 105 | 106 | return lastBroker.host().equals(checkBroker.host()) && lastBroker.port() == checkBroker.port(); 107 | } 108 | } 109 | -------------------------------------------------------------------------------- /src/main/scala/com/github/maelstrom/controller/ControllerKafkaTopics.scala: -------------------------------------------------------------------------------- 1 | package com.github.maelstrom.controller 2 | 3 | import java.util.concurrent.Executors 4 | 5 | import com.github.maelstrom.{KafkaRDDUtils, Logging} 6 | import com.github.maelstrom.consumer.KafkaConsumerPoolFactory 7 | import org.apache.curator.framework.CuratorFramework 8 | import org.apache.spark.rdd.RDD 9 | import org.apache.spark.storage.StorageLevel 10 | import org.apache.spark.SparkContext 11 | 12 | import scala.collection.mutable 13 | import scala.concurrent.duration.Duration 14 | import scala.concurrent.{Await, ExecutionContext, Future} 15 | import scala.util.{Failure, Success} 16 | 17 | class ControllerKafkaTopics[K, V] (sc: SparkContext, 18 | curator: CuratorFramework, 19 | poolFactory: KafkaConsumerPoolFactory[_,_]) extends IControllerKafka[K, V] with Logging { 20 | final private val topicSet = mutable.Set[ControllerKafkaTopic[K, V]]() 21 | private lazy val executor = Executors.newFixedThreadPool(getTotalTopics) 22 | private implicit lazy val ec: ExecutionContext = ExecutionContext.fromExecutor(executor) 23 | 24 | final def registerTopic(consumerGroup: String, topic: String): ControllerKafkaTopic[K, V] = { 25 | registerTopic(consumerGroup, topic, 5000) 26 | } 27 | 28 | final def registerTopic(consumerGroup: String, topic: String, maxQueue: Int = 5000): ControllerKafkaTopic[K, V] = { 29 | val controllerKafkaTopic = new ControllerKafkaTopic[K, V](sc, curator, poolFactory, consumerGroup, topic, maxQueue) 30 | if (topicSet.contains(controllerKafkaTopic)) throw new IllegalArgumentException("Already registered this kind of topic") 31 | topicSet.add(controllerKafkaTopic) 32 | controllerKafkaTopic 33 | } 34 | 35 | final def getTotalTopics: Int = { 36 | topicSet.size 37 | } 38 | 39 | final def getTotalPartitions: Int = { 40 | var totalPartitions: Int = 0 41 | 42 | for (kafkaTopic <- topicSet) 43 | totalPartitions += kafkaTopic.getPartitionCount 44 | 45 | totalPartitions 46 | } 47 | 48 | final def close() { 49 | for (kafkaTopic <- topicSet) kafkaTopic.close() 50 | executor.shutdown() 51 | } 52 | 53 | final def getLag: Long = { 54 | var lag: Long = 0L 55 | for (kafkaTopic <- topicSet) lag += kafkaTopic.getLag 56 | lag 57 | } 58 | 59 | final def commit() { 60 | for (kafkaTopic <- topicSet) kafkaTopic.commit() 61 | } 62 | 63 | final def getAllRDDs: List[RDD[(K, V)]] = { 64 | val lag: Long = getLag 65 | var futures = mutable.ListBuffer[Future[RDD[(K,V)]]]() 66 | 67 | for (controllerKafkaTopic <- topicSet) { 68 | if (controllerKafkaTopic.getLag > 0) { 69 | var offsets: Map[Int, (Long, Long)] = Map() 70 | val perEach: Long = Math.max(1, controllerKafkaTopic.maxQueue / controllerKafkaTopic.getPartitionCount) 71 | 72 | for (kafkaPartition <- controllerKafkaTopic.getPartitions) { 73 | if (lag > perEach) { 74 | kafkaPartition.setStopAtOffset(Math.min(kafkaPartition.getLastOffset + perEach, kafkaPartition.getLastOffset + kafkaPartition.getLag)) 75 | logDebug(s"REAP PARTIAL: [${kafkaPartition.topic}]-[${kafkaPartition.partitionId}] : @${kafkaPartition.getStopAtOffset}") 76 | } else { 77 | kafkaPartition.setStopAtOffset(kafkaPartition.getLatestOffset) 78 | logDebug(s"REAP FULL: [${kafkaPartition.topic}]-[${kafkaPartition.partitionId}] : @${kafkaPartition.getLatestOffset}") 79 | } 80 | logDebug(s"OFFSET RANGE: [${kafkaPartition.topic}]-[${kafkaPartition.partitionId}] : ${kafkaPartition.getLastOffset}->${kafkaPartition.getStopAtOffset} = ${kafkaPartition.getStopAtOffset - kafkaPartition.getLastOffset}") 81 | offsets += (kafkaPartition.partitionId -> (kafkaPartition.getLastOffset, kafkaPartition.getStopAtOffset)) 82 | } 83 | 84 | futures += Future { 85 | logDebug("LOADING RDD - " + controllerKafkaTopic.consumerGroup + ":" + controllerKafkaTopic.topic) 86 | KafkaRDDUtils.createKafkaRDD(sc, poolFactory, controllerKafkaTopic.topic, offsets) 87 | .persist(StorageLevel.MEMORY_ONLY) 88 | .setName("KafkaRDD-" + controllerKafkaTopic.consumerGroup + ":" + controllerKafkaTopic.topic) 89 | .asInstanceOf[RDD[(K, V)]] 90 | } 91 | } 92 | } 93 | 94 | Await.result(Future.sequence(futures), Duration.Inf) 95 | 96 | var rDDs = mutable.ListBuffer[RDD[(K, V)]]() 97 | 98 | futures.foreach(f => 99 | f.value.get match { 100 | case Success(rdd) => 101 | rDDs += rdd 102 | case Failure (error) => 103 | throw error 104 | } 105 | ) 106 | 107 | rDDs.toList 108 | } 109 | 110 | final def getRDD(): RDD[(K, V)] = { 111 | getAllRDDs.reduce(_ union _) 112 | } 113 | } 114 | -------------------------------------------------------------------------------- /src/main/java/com/github/maelstrom/consumer/KafkaConsumerPoolFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.github.maelstrom.consumer; 19 | 20 | import kafka.cluster.Broker; 21 | import kafka.serializer.Decoder; 22 | import kafka.utils.VerifiableProperties; 23 | 24 | import java.io.Serializable; 25 | import java.util.List; 26 | import java.util.Objects; 27 | import java.util.Properties; 28 | 29 | /* 30 | * Kafka Consumer Pool Factory 31 | * 32 | * This class gets serialized over to the Spark Executors to initialize a Kafka Consumer Object Pool 33 | * 34 | * @author Jeoffrey Lim 35 | * @version 0.2 36 | * */ 37 | public final class KafkaConsumerPoolFactory implements Serializable { 38 | private static final int DEFAULT_POOL_SIZE = 1000; 39 | private static final int DEFAULT_EXPIRE_AFTER_MINUTES = 5; 40 | 41 | public final int maxSize; 42 | public final int expireAfterMins; 43 | public final String brokers; 44 | public final Class keyMapper; 45 | public final Class valueMapper; 46 | public final Properties verifiableProperties; 47 | private List brokerList; 48 | 49 | public KafkaConsumerPoolFactory(final String brokers, final Class keyMapper, final Class valueMapper) { 50 | this(DEFAULT_POOL_SIZE, DEFAULT_EXPIRE_AFTER_MINUTES, brokers, keyMapper, valueMapper, new Properties()); 51 | } 52 | 53 | public KafkaConsumerPoolFactory(final String brokers, final Class keyMapper, final Class valueMapper, final Properties verifiableProperties) { 54 | this(DEFAULT_POOL_SIZE, DEFAULT_EXPIRE_AFTER_MINUTES, brokers, keyMapper, valueMapper, verifiableProperties); 55 | } 56 | 57 | public KafkaConsumerPoolFactory(final int maxSize, final int expireAfterMins, final String brokers, final Class keyMapper, final Class valueMapper) { 58 | this(maxSize, expireAfterMins, brokers, keyMapper, valueMapper, new Properties()); 59 | } 60 | 61 | public KafkaConsumerPoolFactory(final int maxSize, final int expireAfterMins, final String brokers, final Class keyMapper, final Class valueMapper, final Properties verifiableProperties) { 62 | this.maxSize = maxSize; 63 | this.expireAfterMins = expireAfterMins; 64 | this.brokers = brokers; 65 | this.keyMapper = keyMapper; 66 | this.valueMapper = valueMapper; 67 | this.verifiableProperties = verifiableProperties; 68 | } 69 | 70 | @Override 71 | public final int hashCode() { 72 | return Objects.hash(maxSize, expireAfterMins, brokers, keyMapper, valueMapper); 73 | } 74 | 75 | @Override 76 | public final boolean equals(final Object obj) { 77 | if (obj == null || getClass() != obj.getClass()) 78 | return false; 79 | 80 | final KafkaConsumerPoolFactory other = (KafkaConsumerPoolFactory) obj; 81 | 82 | return this.maxSize == other.maxSize && 83 | this.expireAfterMins == other.expireAfterMins && 84 | this.keyMapper.getClass() == other.keyMapper.getClass() && 85 | this.valueMapper.getClass() == other.valueMapper.getClass(); 86 | } 87 | 88 | public final List getBrokerList() { 89 | if (brokerList == null) 90 | brokerList = KafkaMetaData.createBrokerList(brokers); 91 | return brokerList; 92 | } 93 | 94 | @SuppressWarnings("unchecked") 95 | public final Decoder createKeyDecoder() { 96 | try { 97 | return (Decoder)keyMapper.getDeclaredConstructor(VerifiableProperties.class).newInstance(verifiableProperties != null ? new VerifiableProperties(verifiableProperties) : null); 98 | } catch (Exception e) { 99 | throw new IllegalArgumentException("Unable to create key decoder: " + e.getMessage(), e); 100 | } 101 | } 102 | 103 | @SuppressWarnings("unchecked") 104 | public final Decoder createValueDecoder() { 105 | try { 106 | return (Decoder)valueMapper.getDeclaredConstructor(VerifiableProperties.class).newInstance(verifiableProperties != null ? new VerifiableProperties(verifiableProperties) : null); 107 | } catch (Exception e) { 108 | throw new IllegalArgumentException("Unable to create value decoder: " + e.getMessage(), e); 109 | } 110 | } 111 | 112 | public final KafkaConsumerPool createKafkaConsumerPool() { 113 | return new KafkaConsumerPool<>(maxSize, expireAfterMins, brokers, createKeyDecoder(), createValueDecoder()); 114 | 115 | } 116 | } 117 | -------------------------------------------------------------------------------- /src/main/scala/com/github/maelstrom/KafkaRDD.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.github.maelstrom 19 | 20 | import java.lang.{Integer => JInt, Long => JLong} 21 | import java.util.concurrent.ConcurrentHashMap 22 | import java.util.{Map => JMap} 23 | 24 | import com.github.maelstrom.consumer.{KafkaConsumerPool, KafkaConsumerPoolFactory} 25 | import kafka.message.MessageAndMetadata 26 | import org.apache.spark.api.java.JavaRDD 27 | import org.apache.spark.rdd.RDD 28 | import org.apache.spark.util.TaskCompletionListener 29 | import org.apache.spark.{Partition, SparkContext, TaskContext} 30 | 31 | import scala.collection.Iterator 32 | import scala.collection.JavaConversions._ 33 | import scala.reflect.ClassTag 34 | 35 | 36 | /** 37 | * Apache Spark + Kafka Integration 38 | * 39 | * @author Jeoffrey Lim 40 | * @version 0.2 41 | */ 42 | 43 | object KafkaRDDUtils { 44 | def createJavaKafkaRDD[K:ClassTag, V:ClassTag](sc: SparkContext, poolFactory: KafkaConsumerPoolFactory[K,V], topic: String, joffsets: JMap[JInt, (JLong, JLong)]): JavaRDD[(K,V)] = { 45 | createKafkaRDDJavaParams(sc, poolFactory, topic, joffsets).toJavaRDD() 46 | } 47 | 48 | def createKafkaRDDJavaParams[K:ClassTag, V:ClassTag](sc: SparkContext, poolFactory: KafkaConsumerPoolFactory[K,V], topic: String, joffsets: JMap[JInt, (JLong, JLong)]): RDD[(K,V)] = { 49 | var offsets: Map[Int, (Long, Long)] = Map() 50 | joffsets.foreach(kv => 51 | offsets += (kv._1.toInt ->(kv._2._1.toLong, kv._2._2.toLong)) 52 | ) 53 | new KafkaRDD(sc, poolFactory, topic, offsets) 54 | } 55 | 56 | def createKafkaRDD[K:ClassTag, V:ClassTag](sc: SparkContext, poolFactory: KafkaConsumerPoolFactory[K,V], topic: String, offsets: Map[Int, (Long,Long)]): RDD[(K,V)] = { 57 | new KafkaRDD[K,V](sc, poolFactory, topic, offsets) 58 | } 59 | } 60 | 61 | object KafkaConsumerPoolCache extends Logging { 62 | private val poolFactoryCache = 63 | new ConcurrentHashMap[KafkaConsumerPoolFactory[_, _], KafkaConsumerPool[_, _]]() 64 | 65 | def getKafkaConsumerPool(poolFactory: KafkaConsumerPoolFactory[_, _]) : KafkaConsumerPool[_, _] = { 66 | var consumerPool: KafkaConsumerPool[_, _] = poolFactoryCache.get(poolFactory) 67 | 68 | if (consumerPool == null) { 69 | consumerPool = poolFactory.createKafkaConsumerPool 70 | poolFactoryCache.put(poolFactory, consumerPool) 71 | logDebug(s"CREATED NEW KafkaConsumerPool=${poolFactory.hashCode()}") 72 | } else { 73 | logDebug(s"GOT CACHED KafkaConsumerPool=${poolFactory.hashCode()}") 74 | } 75 | 76 | consumerPool 77 | } 78 | } 79 | 80 | class KafkaRDD[K:ClassTag, V:ClassTag](sc: SparkContext, val poolFactory: KafkaConsumerPoolFactory[K,V], val topic: String, val offsets: Map[Int, (Long,Long)]) extends RDD[(K,V)](sc, Nil) { 81 | final def compute(partition: Partition, context: TaskContext): Iterator[(K,V)] = { 82 | val consumerPool = KafkaConsumerPoolCache.getKafkaConsumerPool(poolFactory).asInstanceOf[KafkaConsumerPool[K,V]] 83 | val iterator: KafkaIterator[K,V] = new KafkaIterator[K,V](consumerPool, partition.asInstanceOf[KafkaPartition]) 84 | context.addTaskCompletionListener(iterator) 85 | iterator 86 | } 87 | 88 | final def getPartitions: Array[Partition] = { 89 | val partitions = new Array[Partition](offsets.size) 90 | offsets.foreach { 91 | case (key, value) => 92 | partitions(key) = new KafkaPartition(key, topic, key, value._1, value._2) 93 | } 94 | partitions 95 | } 96 | } 97 | 98 | class KafkaPartition(override val index: Int, val topic: String, val partition: Int, val startOffset: Long, val stopOffset: Long) extends Partition { 99 | } 100 | 101 | //Iterator based from org/apache/spark/util/NextIterator.scala 102 | class KafkaIterator[K,V] 103 | (val consumerPool: KafkaConsumerPool[K,V], val kafkaPartition: KafkaPartition) 104 | extends Iterator[(K,V)] 105 | with TaskCompletionListener { 106 | 107 | private var gotNext = false 108 | private var nextValue: (K,V) = _ 109 | private var closed = false 110 | protected var finished = false 111 | private var offset = kafkaPartition.startOffset 112 | private val poolableConsumer = consumerPool.getConsumer(kafkaPartition.topic, kafkaPartition.index) 113 | private val consumer = poolableConsumer.take() 114 | private var it: Iterator[MessageAndMetadata[K, V]] = _ 115 | 116 | consumer.setCurrentOffset(kafkaPartition.startOffset) 117 | 118 | final def onTaskCompletion(context: TaskContext) { 119 | closeIfNeeded() 120 | } 121 | 122 | private def closeIfNeeded() { 123 | if (!closed) { 124 | closed = true 125 | consumerPool.returnConsumer(poolableConsumer, consumer) 126 | } 127 | } 128 | 129 | final def hasNext: Boolean = { 130 | if (!finished && !gotNext) { 131 | nextValue = getNext 132 | if (finished) { 133 | closeIfNeeded() 134 | } else { 135 | gotNext = true 136 | } 137 | } 138 | !finished 139 | } 140 | 141 | private def getNext: (K,V) = { 142 | if (it != null && it.hasNext) { 143 | val nv = it.next 144 | return (nv.key(), nv.message()) 145 | } 146 | 147 | if (offset < kafkaPartition.stopOffset) { 148 | if (it == null || !it.hasNext) { 149 | it = consumer.receive(kafkaPartition.stopOffset) 150 | offset = consumer.getCurrentOffset 151 | } 152 | val nv = it.next 153 | (nv.key(), nv.message()) 154 | } else { 155 | finished = true 156 | null.asInstanceOf[(K,V)] 157 | } 158 | } 159 | 160 | def next: (K,V) = { 161 | if (!hasNext) { 162 | throw new NoSuchElementException("End of stream") 163 | } 164 | gotNext = false 165 | nextValue 166 | } 167 | } -------------------------------------------------------------------------------- /src/main/java/com/github/maelstrom/consumer/OffsetManager.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.github.maelstrom.consumer; 19 | 20 | import kafka.common.KafkaException; 21 | import org.apache.curator.framework.CuratorFramework; 22 | import org.apache.curator.framework.CuratorFrameworkFactory; 23 | import org.apache.curator.retry.ExponentialBackoffRetry; 24 | import org.apache.zookeeper.CreateMode; 25 | import org.slf4j.Logger; 26 | import org.slf4j.LoggerFactory; 27 | 28 | import java.util.concurrent.Callable; 29 | import java.util.concurrent.ThreadFactory; 30 | 31 | /** 32 | * @author Jeoffrey Lim 33 | * @version 0.2 34 | */ 35 | public class OffsetManager { 36 | private static final Logger LOG = LoggerFactory.getLogger(OffsetManager.class); 37 | 38 | private final CuratorFramework curator; 39 | protected final String consumerGroup; 40 | protected final String topic; 41 | protected final int partitionId; 42 | 43 | private final String zkPath; 44 | private long lastOffset; 45 | private long lastCommitOffset; 46 | 47 | public OffsetManager(final CuratorFramework curator, final KafkaConsumer consumer, final String consumerGroup, final String topic, final int partitionId) { 48 | this.curator = curator; 49 | this.consumerGroup = consumerGroup; 50 | this.topic = topic; 51 | this.partitionId = partitionId; 52 | 53 | this.zkPath = "/consumers/" + consumerGroup + "/offsets/" + this.topic + "/" + this.partitionId; 54 | 55 | this.lastOffset = getStoredLastOffset(); 56 | this.lastCommitOffset = lastOffset; 57 | 58 | long currentOffset = getLastOffset(); 59 | long earliestOffset = consumer.getEarliestOffset(); 60 | long latestOffset = consumer.getLatestOffset(); 61 | 62 | if (latestOffset < 0 || earliestOffset < 0) 63 | throw new KafkaException("Unstable Kafka Connection"); 64 | 65 | if (currentOffset == 0) { 66 | LOG.warn("{}-[{}:{}] No offset found starting from latest offset.", consumerGroup, topic, partitionId); 67 | setLastOffset(latestOffset); 68 | storeLastOffset(); //force commit 69 | } else { 70 | if (currentOffset < earliestOffset) { 71 | LOG.warn("Offset Out of Lower Bound for: {}-[{}:{}]@{} latest: {}", consumerGroup, topic, partitionId, currentOffset, latestOffset); 72 | setLastOffset(earliestOffset); 73 | storeLastOffset(); //force commit 74 | } else if (currentOffset > latestOffset) { 75 | LOG.warn("Offset Out of Higher Bound for: {}-[{}:{}]@{} latest: {}", consumerGroup, topic, partitionId, currentOffset, latestOffset); 76 | setLastOffset(latestOffset); 77 | storeLastOffset(); //force commit 78 | } 79 | } 80 | 81 | consumer.setCurrentOffset(getLastOffset()); 82 | LOG.info("LAST OFFSET: {}-[{}:{}]@{}", consumerGroup, topic, partitionId, lastOffset); 83 | } 84 | 85 | public static CuratorFramework createCurator(final String zookeeper) { 86 | CuratorFramework curator = CuratorFrameworkFactory.builder(). 87 | connectString(zookeeper). 88 | sessionTimeoutMs(120000). 89 | connectionTimeoutMs(120000). 90 | retryPolicy(new ExponentialBackoffRetry(1000, 29)) 91 | .threadFactory(new ThreadFactory() { 92 | @Override 93 | public Thread newThread(@SuppressWarnings("NullableProblems") Runnable r) { 94 | Thread t = new Thread(r); 95 | t.setDaemon(true); 96 | return t; 97 | } 98 | }).build(); 99 | curator.start(); 100 | 101 | return curator; 102 | } 103 | 104 | private long getStoredLastOffset() { 105 | return InfiniteRetryStrategy.retryInfinitelyLong(new Callable() { 106 | @Override 107 | public Long call() throws Exception { 108 | try { 109 | if (curator.checkExists().forPath(zkPath) == null) { 110 | return 0L; 111 | } else { 112 | return Long.parseLong(new String(curator.getData().forPath(zkPath))); 113 | } 114 | } catch (Exception e) { 115 | LOG.warn("{}-[{}:{}] Error retrieving offset from ZooKeeper: {}", consumerGroup, topic, partitionId, e.getMessage(), e); 116 | return null; 117 | } 118 | } 119 | }); 120 | } 121 | 122 | public final long getLastOffset() { 123 | return lastOffset; 124 | } 125 | 126 | public final void setLastOffset(final long lastOffset) { 127 | this.lastOffset = lastOffset; 128 | } 129 | 130 | public final void storeLastOffset() { 131 | if (lastCommitOffset == lastOffset) 132 | return; 133 | 134 | InfiniteRetryStrategy.retryInfinitelyBoolean(new Callable() { 135 | @Override 136 | public Boolean call() throws Exception { 137 | try { 138 | LOG.debug("STORE LAST OFFSET: {}-[{}:{}] @ {}", consumerGroup, topic, partitionId, lastOffset); 139 | 140 | if (curator.checkExists().forPath(zkPath) == null) 141 | curator.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT).forPath(zkPath, String.valueOf(lastOffset).getBytes()); 142 | else 143 | curator.setData().forPath(zkPath, String.valueOf(lastOffset).getBytes()); 144 | 145 | lastCommitOffset = lastOffset; 146 | 147 | return true; 148 | } catch (Exception e) { 149 | LOG.warn("{}-[{}:{}] Error storing offset in ZooKeeper: {}", consumerGroup, topic, partitionId, e.getMessage(), e); 150 | 151 | return false; 152 | } 153 | } 154 | }); 155 | } 156 | } 157 | -------------------------------------------------------------------------------- /src/main/java/com/github/maelstrom/consumer/KafkaConsumerPool.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.github.maelstrom.consumer; 19 | 20 | import com.google.common.cache.*; 21 | import kafka.cluster.Broker; 22 | import kafka.serializer.Decoder; 23 | import org.slf4j.Logger; 24 | import org.slf4j.LoggerFactory; 25 | import org.vibur.objectpool.ConcurrentPool; 26 | import org.vibur.objectpool.PoolObjectFactory; 27 | import org.vibur.objectpool.PoolService; 28 | import org.vibur.objectpool.util.ConcurrentLinkedQueueCollection; 29 | import scala.Tuple2; 30 | 31 | import javax.annotation.Nonnull; 32 | import java.util.List; 33 | import java.util.concurrent.TimeUnit; 34 | 35 | /** 36 | * @author Jeoffrey Lim 37 | * @version 0.2 38 | */ 39 | public final class KafkaConsumerPool { 40 | private static final Logger LOG = LoggerFactory.getLogger(KafkaConsumerPool.class); 41 | 42 | private final int maxSize; 43 | private final int expireAfterMins; 44 | private final List brokerList; 45 | private final Decoder keyMapper; 46 | private final Decoder valueMapper; 47 | private final LoadingCache, PoolService>> pool; 48 | 49 | KafkaConsumerPool(final int maxSize, final int expireAfterMins, final String brokers, final Decoder keyMapper, final Decoder valueMapper) { 50 | this.maxSize = maxSize; 51 | this.expireAfterMins = expireAfterMins; 52 | this.brokerList = KafkaMetaData.createBrokerList(brokers); 53 | this.keyMapper = keyMapper; 54 | this.valueMapper = valueMapper; 55 | this.pool = createPool(); 56 | 57 | Runtime.getRuntime().addShutdownHook(new Thread() { 58 | @Override 59 | public void run() { 60 | try { 61 | LOG.info("Shutdown Hook: Shutting down KafkaConsumerPool"); 62 | shutdown(); 63 | } catch (Exception e) { 64 | LOG.error(e.getMessage(), e); 65 | } 66 | } 67 | }); 68 | } 69 | 70 | @SuppressWarnings("unused") 71 | public List getBrokerList() { 72 | return brokerList; 73 | } 74 | 75 | private LoadingCache, PoolService>> createPool() { 76 | return CacheBuilder.newBuilder() 77 | .maximumSize(10_000) 78 | .expireAfterAccess(expireAfterMins, TimeUnit.MINUTES) 79 | .removalListener(new RemovalListener, PoolService>>() { 80 | @Override 81 | public void onRemoval(@Nonnull final RemovalNotification, PoolService>> removalNotification) { 82 | LOG.info("CLOSE @ POOL: KafkaConsumer: {}", removalNotification.getKey()); 83 | if (removalNotification.getValue() != null) 84 | (removalNotification.getValue()).close(); 85 | } 86 | }) 87 | .build(new CacheLoader, PoolService>>() { 88 | @Override 89 | public PoolService> load(@Nonnull final Tuple2 key) throws Exception { 90 | LOG.info("CREATE @ POOL: new KafkaConsumer: {}", key); 91 | 92 | return new ConcurrentPool<>( 93 | new ConcurrentLinkedQueueCollection>(), 94 | new KafkaConsumerFactory<>( 95 | brokerList, keyMapper, valueMapper, key._1(), key._2() 96 | ), 97 | 1, 98 | maxSize, 99 | false); 100 | } 101 | }); 102 | } 103 | 104 | public final PoolService> getConsumer(final String topic, final int partition) { 105 | LOG.debug("GET CONSUMER: KafkaConsumer: [{}:{}]", topic, partition); 106 | return pool.getUnchecked(new Tuple2<>(topic, partition)); 107 | } 108 | 109 | public final void returnConsumer(final PoolService> pool, final KafkaConsumer consumer) { 110 | LOG.debug("RETURN CONSUMER: KafkaConsumer: [{}:{}]", consumer.topic, consumer.partitionId); 111 | pool.restore(consumer); 112 | } 113 | 114 | @SuppressWarnings("WeakerAccess") 115 | public final void shutdown() { 116 | LOG.info("SHUTTING DOWN POOL"); 117 | pool.invalidateAll(); 118 | pool.cleanUp(); 119 | } 120 | 121 | private static class KafkaConsumerFactory implements PoolObjectFactory> { 122 | private final List brokerList; 123 | private final Decoder keyMapper; 124 | private final Decoder valueMapper; 125 | private final String topic; 126 | private final Integer partition; 127 | 128 | KafkaConsumerFactory(final List brokerList, 129 | final Decoder keyMapper, 130 | final Decoder valueMapper, 131 | final String topic, 132 | final Integer partition) { 133 | this.brokerList = brokerList; 134 | this.keyMapper = keyMapper; 135 | this.valueMapper = valueMapper; 136 | this.topic = topic; 137 | this.partition = partition; 138 | } 139 | 140 | @Override 141 | public final KafkaConsumer create() { 142 | try { 143 | LOG.info("CREATE @ KafkaConsumer: {}:{}", topic, partition); 144 | return new KafkaConsumer<>(brokerList, "spark-kafka-consumer-pool", keyMapper, valueMapper, topic, partition); 145 | } catch (Exception e) { 146 | LOG.error(e.getMessage(), e); 147 | throw e; 148 | } 149 | } 150 | 151 | @Override 152 | public final boolean readyToTake(final KafkaConsumer obj) { 153 | return true; 154 | } 155 | 156 | @Override 157 | public final boolean readyToRestore(final KafkaConsumer obj) { 158 | return true; 159 | } 160 | 161 | @Override 162 | public final void destroy(final KafkaConsumer consumer) { 163 | LOG.info("CLOSE @ KafkaConsumer: {}:{}", topic, partition); 164 | consumer.close(); 165 | } 166 | } 167 | } 168 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Maelstrom 2 | 3 | Maelstrom is an open source Kafka integration with Spark that is designed to be developer friendly, high performance 4 | (millisecond stream processing), scalable (consumes messages at Spark worker nodes), and is extremely reliable. 5 | 6 | This library has been running stable in production environment and has been proven to be resilient to numerous 7 | production issues. 8 | 9 | Thanks to [Adlogica](http://www.adlogica.com/) for sharing to the open source community this project! 10 | 11 | ## Features 12 | 13 | - Simple framework which follows Kafka semantics and best practices. 14 | - High performance, with latencies down to milliseconds. 15 | - Scalable, where message consumption is received in the Spark worker nodes and not on the driver side. 16 | - Throttling by specifying maximum number of messages to process per each "bulk receive" 17 | - Built-in offset management stored in Zookeeper. Numerous Kafka monitoring tools should work out of the box. 18 | - Fault tolerant design, if in case stream processing fails: it would go back to the last processed offsets. 19 | - Resilient to Kafka problems (automatic leader election detection, rebalance, etc) 20 | - Kafka connection resource is pooled and re-used but always get validated if connected to the correct leader broker. 21 | 22 | ## The Nuts & Bolts 23 | 24 | - The Kafka Consumer is built from scratch and is heavily inspired by Yahoo's Pistachio Kafka Consumer. 25 | - With infinite retry strategy in the Kafka Consumer level (like in Pistachio), any issues related to Kafka 26 | gets rectified and resolved which makes the application automatically recover. 27 | - Utilizing Ben Mane's multi-way pool, Kafka Consumers gets reused using (topic+partition) as keys. 28 | - As message consumption runs in the Spark worker nodes, to achieve scalability simply add more partitions to your Kafka topic. 29 | 30 | ## Zero data loss 31 | 32 | - As offset commits is done per each successful processing, failure scenario in between could happen: like abruptly 33 | terminating the application or external systems become unavailable. 34 | - Ensure that your application has the ability to do graceful shutdown, this can be achieved by exposing an http 35 | endpoint and calling *stop* to the ProcessorRunner. 36 | - Implement your own retry strategy in case of processing failure due to external dependencies 37 | 38 | ## Exactly once semantics 39 | 40 | - Generate unique ID for every message. 41 | - For near real-time stream processing: processors should write to another Kafka topic, and utilize Memcache/Redis 42 | to achieve exactly once semantics (i.e. send e-mail functionality). 43 | - For Batch processing: use Spark mapToPair + reduceByKey to remove duplicates. (*Pro tip*: repartition RDDs 44 | to maximize Spark cluster hardware utilization) 45 | 46 | 47 | ## Building Maelstrom from Source 48 | 49 | Prerequisites for building Maelstrom: 50 | 51 | * Git 52 | * Maven 53 | * Java 7 or 8 54 | 55 | ``` 56 | git clone https://github.com/jeoffreylim/maelstrom.git 57 | cd maelstrom 58 | mvn install 59 | ``` 60 | 61 | ## Including Maelstrom in your project 62 | 63 | Add the library in your project maven dependencies: 64 | 65 | ``` 66 | 67 | com.github.maelstrom 68 | maelstrom 69 | 0.2 70 | 71 | ``` 72 | 73 | ## Kafka Consumer on Executors 74 | 75 | The KafkaConsumerPoolFactory gets serialized over the wire to the Spark executors and cache Kafka Consumers to 76 | establish a persistent connection to Kafka brokers. Future version of Maelstrom is to make these Kafka Consumers 77 | remain to a specific Spark executor to as much as possible: all registered Consumer Group + Topics + Partitions 78 | properly distributed to available Spark executors (Worker + Number of Executors). 79 | 80 | 81 | | Config | Description | Default | 82 | |-----------------|----------------------------------------------------------------------------------|-----------| 83 | | brokers | The Kafka Broker List | n/a | 84 | | keyMapper | Class for decoding Keys | n/a | 85 | | valueMapper | Class for decoding Values | n/a | 86 | | maxSize | Make this at least double the amount of all Consumer Group + Topics + Partitions | 1000 | 87 | | expireAfterMins | Kafka Consumer that is unused will expire after the defined minutes | 5 minutes | 88 | 89 | ## Java example 90 | 91 | ```java 92 | SparkConf sparkConf = new SparkConf().setMaster("local[4]").setAppName("StreamSingleTopic"); 93 | JavaSparkContext sc = new JavaSparkContext(sparkConf); 94 | 95 | CuratorFramework curator = OffsetManager.createCurator("127.0.0.1:2181"); 96 | KafkaConsumerPoolFactory poolFactory = new KafkaConsumerPoolFactory<>("127.0.0.1:9092", StringDecoder.class, StringDecoder.class); 97 | 98 | ControllerKafkaTopics topics = new ControllerKafkaTopics<>(sc.sc(), curator, poolFactory); 99 | ControllerKafkaTopic topic = topics.registerTopic("test_group", "test"); 100 | 101 | new StreamProcessor(topic) { 102 | @Override 103 | public final void process() { 104 | JavaRDD> rdd = fetch().toJavaRDD(); 105 | 106 | rdd.foreachPartition(new VoidFunction>>() { 107 | @Override 108 | public final void call(final Iterator> it) { 109 | while (it.hasNext()) { 110 | Tuple2 e = it.next(); 111 | LOG.info("key=" + e._1 + " message=" + e._2()); 112 | } 113 | } 114 | }); 115 | 116 | commit(); 117 | } 118 | }.run(); 119 | 120 | sc.sc().stop(); 121 | ``` 122 | 123 | ## Scala example 124 | 125 | ```scala 126 | val sparkConf = new SparkConf().setMaster("local[4]").setAppName("StreamSingleTopic") 127 | val sc = new SparkContext(sparkConf) 128 | val curator = OffsetManager.createCurator("127.0.0.1:2181") 129 | val poolFactory = new KafkaConsumerPoolFactory[String, String]("127.0.0.1:9092", classOf[StringDecoder], classOf[StringDecoder]) 130 | val topics = new ControllerKafkaTopics[String, String](sc, curator, poolFactory) 131 | val topic = topics.registerTopic("test_group", "test") 132 | 133 | new StreamProcessor[String, String](topic) { 134 | final def process() { 135 | val rdd: RDD[(String, String)] = fetch() 136 | 137 | rdd.foreachPartition { partitionData => 138 | partitionData.foreach { record => 139 | logger.info("key=" + record._1 + " val=" + record._2) 140 | } 141 | } 142 | 143 | commit() 144 | } 145 | }.run() 146 | 147 | sc.stop() 148 | ``` 149 | 150 | ## Example of a Multi-Topic + Multi Processor with throttling 151 | 152 | One stream processor consumes on *test* topic with maximum 1000 records per mini-batch. 153 | Second stream processor consumes on *test2* topic with maximum 500 records per mini-batch. 154 | 155 | ```scala 156 | val sparkConf = new SparkConf().setMaster("local[4]").setAppName("StreamMultiTopicMultiProc") 157 | val sc = new SparkContext(sparkConf) 158 | val curator = OffsetManager.createCurator("127.0.0.1:2181") 159 | val poolFactory = new KafkaConsumerPoolFactory[String, String]("127.0.0.1:9092", classOf[StringDecoder], classOf[StringDecoder]) 160 | val topics = new ControllerKafkaTopics[String, String](sc, curator, poolFactory) 161 | 162 | new ProcessorRunner().addProcessor(new StreamProcessor[String, String](topics.registerTopic("test_multi_proc", "test", 1000)) { 163 | final def process() { 164 | val rdd: RDD[(String, String)] = fetch() 165 | 166 | rdd.foreachPartition { partitionData => 167 | partitionData.foreach { record => 168 | logger.info("key=" + record._1 + " val=" + record._2) 169 | } 170 | } 171 | 172 | commit() 173 | } 174 | }).addProcessor(new StreamProcessor[String, String](topics.registerTopic("test_multi_proc", "test2", 500)) { 175 | final def process() { 176 | val rdd: RDD[(String, String)] = fetch() 177 | 178 | rdd.foreachPartition { partitionData => 179 | partitionData.foreach { record => 180 | logger.info("key=" + record._1 + " val=" + record._2) 181 | } 182 | } 183 | 184 | commit() 185 | } 186 | }).start() 187 | 188 | sc.stop() 189 | ``` 190 | 191 | 192 | ## TODO: 193 | 194 | - Message handler parameter for decoding K, V (to enable the developer to include Topic & Partition information) 195 | - Automatic partition resize detection 196 | - Improve Leader Broker checker to use only a single thread. 197 | - Custom offset storage (HBase, Mecache/Redis, MapDB - for performance junkies) 198 | - Offset Management Utility 199 | - Ability to store data in KafkaRDD 200 | 201 | ## License 202 | 203 | This code is distributed using the Apache license, Version 2.0. 204 | 205 | ## Author 206 | 207 | - Jeoffrey Lim -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 4.0.0 3 | com.github.maelstrom 4 | maelstrom 5 | Maelstrom 6 | jar 7 | 0.2 8 | 9 | http://github.com/jeoffreylim/maelstrom 10 | 11 | 12 | 13 | The Apache Software License, Version 2.0 14 | http://www.apache.org/licenses/LICENSE-2.0.txt 15 | repo 16 | 17 | 18 | 19 | 20 | http://github.com/jeoffreylim/maelstrom 21 | scm:git:git@github.com:jeoffreylim/maelstrom.git 22 | 23 | 24 | 25 | 26 | jeoffreylim 27 | Jeoffrey Lim 28 | 29 | 30 | 31 | 32 | 2.11.8 33 | 2.11 34 | 2.15.2 35 | 36 | 1.7 37 | 38 | 2.0.0 39 | 0.8.2.1 40 | 41 | 2.4.0 42 | UTF-8 43 | UTF-8 44 | 45 | 46 | 47 | 48 | 49 | net.alchim31.maven 50 | scala-maven-plugin 51 | 3.2.2 52 | 53 | 54 | eclipse-add-source 55 | 56 | add-source 57 | 58 | 59 | 60 | scala-compile-first 61 | process-resources 62 | 63 | compile 64 | 65 | 66 | 67 | scala-test-compile-first 68 | process-test-resources 69 | 70 | testCompile 71 | 72 | 73 | 74 | attach-scaladocs 75 | verify 76 | 77 | doc-jar 78 | 79 | 80 | 81 | 82 | ${scala.version} 83 | incremental 84 | true 85 | 86 | -unchecked 87 | -deprecation 88 | -feature 89 | 90 | 91 | -source 92 | ${java.version} 93 | -target 94 | ${java.version} 95 | -Xlint:all,-serial,-path 96 | 97 | 98 | 99 | 100 | org.apache.maven.plugins 101 | maven-compiler-plugin 102 | 3.1 103 | 104 | ${java.version} 105 | ${java.version} 106 | 107 | 108 | 109 | 110 | 111 | src/main/resources 112 | 113 | 114 | 115 | 116 | 117 | 118 | 119 | org.apache.spark 120 | spark-core_${scala.binary.version} 121 | ${spark.version} 122 | 123 | 124 | 125 | 126 | org.apache.kafka 127 | kafka_${scala.binary.version} 128 | ${kafka.version} 129 | 130 | 131 | com.sun.jmx 132 | jmxri 133 | 134 | 135 | com.sun.jdmk 136 | jmxtools 137 | 138 | 139 | net.sf.jopt-simple 140 | jopt-simple 141 | 142 | 143 | org.slf4j 144 | slf4j-simple 145 | 146 | 147 | org.scala-lang 148 | scala-compiler 149 | 150 | 151 | org.scala-lang 152 | scala-library 153 | 154 | 155 | org.scalatest 156 | scalatest_${scala.binary.version} 157 | 158 | 159 | org.apache.zookeeper 160 | zookeeper 161 | 162 | 163 | log4j 164 | log4j 165 | 166 | 167 | 168 | 169 | 170 | 171 | org.apache.curator 172 | curator-client 173 | ${curator.version} 174 | 175 | 176 | com.google.guava 177 | guava 178 | 179 | 180 | org.slf4j 181 | slf4j-api 182 | 183 | 184 | org.apache.zookeeper 185 | zookeeper 186 | 187 | 188 | log4j 189 | log4j 190 | 191 | 192 | 193 | 194 | 195 | org.scala-lang 196 | scala-reflect 197 | ${scala.version} 198 | compile 199 | 200 | 201 | 202 | 203 | com.github.rholder 204 | guava-retrying 205 | 2.0.0 206 | 207 | 208 | com.google.guava 209 | guava 210 | 211 | 212 | 213 | 214 | 215 | 216 | org.vibur 217 | vibur-object-pool 218 | 16.1 219 | 220 | 221 | 222 | 223 | org.apache.spark 224 | spark-streaming_${scala.binary.version} 225 | ${spark.version} 226 | test 227 | 228 | 229 | 230 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | Apache License 2 | Version 2.0, January 2004 3 | http://www.apache.org/licenses/ 4 | 5 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION 6 | 7 | 1. Definitions. 8 | 9 | "License" shall mean the terms and conditions for use, reproduction, 10 | and distribution as defined by Sections 1 through 9 of this document. 11 | 12 | "Licensor" shall mean the copyright owner or entity authorized by 13 | the copyright owner that is granting the License. 14 | 15 | "Legal Entity" shall mean the union of the acting entity and all 16 | other entities that control, are controlled by, or are under common 17 | control with that entity. For the purposes of this definition, 18 | "control" means (i) the power, direct or indirect, to cause the 19 | direction or management of such entity, whether by contract or 20 | otherwise, or (ii) ownership of fifty percent (50%) or more of the 21 | outstanding shares, or (iii) beneficial ownership of such entity. 22 | 23 | "You" (or "Your") shall mean an individual or Legal Entity 24 | exercising permissions granted by this License. 25 | 26 | "Source" form shall mean the preferred form for making modifications, 27 | including but not limited to software source code, documentation 28 | source, and configuration files. 29 | 30 | "Object" form shall mean any form resulting from mechanical 31 | transformation or translation of a Source form, including but 32 | not limited to compiled object code, generated documentation, 33 | and conversions to other media types. 34 | 35 | "Work" shall mean the work of authorship, whether in Source or 36 | Object form, made available under the License, as indicated by a 37 | copyright notice that is included in or attached to the work 38 | (an example is provided in the Appendix below). 39 | 40 | "Derivative Works" shall mean any work, whether in Source or Object 41 | form, that is based on (or derived from) the Work and for which the 42 | editorial revisions, annotations, elaborations, or other modifications 43 | represent, as a whole, an original work of authorship. For the purposes 44 | of this License, Derivative Works shall not include works that remain 45 | separable from, or merely link (or bind by name) to the interfaces of, 46 | the Work and Derivative Works thereof. 47 | 48 | "Contribution" shall mean any work of authorship, including 49 | the original version of the Work and any modifications or additions 50 | to that Work or Derivative Works thereof, that is intentionally 51 | submitted to Licensor for inclusion in the Work by the copyright owner 52 | or by an individual or Legal Entity authorized to submit on behalf of 53 | the copyright owner. For the purposes of this definition, "submitted" 54 | means any form of electronic, verbal, or written communication sent 55 | to the Licensor or its representatives, including but not limited to 56 | communication on electronic mailing lists, source code control systems, 57 | and issue tracking systems that are managed by, or on behalf of, the 58 | Licensor for the purpose of discussing and improving the Work, but 59 | excluding communication that is conspicuously marked or otherwise 60 | designated in writing by the copyright owner as "Not a Contribution." 61 | 62 | "Contributor" shall mean Licensor and any individual or Legal Entity 63 | on behalf of whom a Contribution has been received by Licensor and 64 | subsequently incorporated within the Work. 65 | 66 | 2. Grant of Copyright License. Subject to the terms and conditions of 67 | this License, each Contributor hereby grants to You a perpetual, 68 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 69 | copyright license to reproduce, prepare Derivative Works of, 70 | publicly display, publicly perform, sublicense, and distribute the 71 | Work and such Derivative Works in Source or Object form. 72 | 73 | 3. Grant of Patent License. Subject to the terms and conditions of 74 | this License, each Contributor hereby grants to You a perpetual, 75 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable 76 | (except as stated in this section) patent license to make, have made, 77 | use, offer to sell, sell, import, and otherwise transfer the Work, 78 | where such license applies only to those patent claims licensable 79 | by such Contributor that are necessarily infringed by their 80 | Contribution(s) alone or by combination of their Contribution(s) 81 | with the Work to which such Contribution(s) was submitted. If You 82 | institute patent litigation against any entity (including a 83 | cross-claim or counterclaim in a lawsuit) alleging that the Work 84 | or a Contribution incorporated within the Work constitutes direct 85 | or contributory patent infringement, then any patent licenses 86 | granted to You under this License for that Work shall terminate 87 | as of the date such litigation is filed. 88 | 89 | 4. Redistribution. You may reproduce and distribute copies of the 90 | Work or Derivative Works thereof in any medium, with or without 91 | modifications, and in Source or Object form, provided that You 92 | meet the following conditions: 93 | 94 | (a) You must give any other recipients of the Work or 95 | Derivative Works a copy of this License; and 96 | 97 | (b) You must cause any modified files to carry prominent notices 98 | stating that You changed the files; and 99 | 100 | (c) You must retain, in the Source form of any Derivative Works 101 | that You distribute, all copyright, patent, trademark, and 102 | attribution notices from the Source form of the Work, 103 | excluding those notices that do not pertain to any part of 104 | the Derivative Works; and 105 | 106 | (d) If the Work includes a "NOTICE" text file as part of its 107 | distribution, then any Derivative Works that You distribute must 108 | include a readable copy of the attribution notices contained 109 | within such NOTICE file, excluding those notices that do not 110 | pertain to any part of the Derivative Works, in at least one 111 | of the following places: within a NOTICE text file distributed 112 | as part of the Derivative Works; within the Source form or 113 | documentation, if provided along with the Derivative Works; or, 114 | within a display generated by the Derivative Works, if and 115 | wherever such third-party notices normally appear. The contents 116 | of the NOTICE file are for informational purposes only and 117 | do not modify the License. You may add Your own attribution 118 | notices within Derivative Works that You distribute, alongside 119 | or as an addendum to the NOTICE text from the Work, provided 120 | that such additional attribution notices cannot be construed 121 | as modifying the License. 122 | 123 | You may add Your own copyright statement to Your modifications and 124 | may provide additional or different license terms and conditions 125 | for use, reproduction, or distribution of Your modifications, or 126 | for any such Derivative Works as a whole, provided Your use, 127 | reproduction, and distribution of the Work otherwise complies with 128 | the conditions stated in this License. 129 | 130 | 5. Submission of Contributions. Unless You explicitly state otherwise, 131 | any Contribution intentionally submitted for inclusion in the Work 132 | by You to the Licensor shall be under the terms and conditions of 133 | this License, without any additional terms or conditions. 134 | Notwithstanding the above, nothing herein shall supersede or modify 135 | the terms of any separate license agreement you may have executed 136 | with Licensor regarding such Contributions. 137 | 138 | 6. Trademarks. This License does not grant permission to use the trade 139 | names, trademarks, service marks, or product names of the Licensor, 140 | except as required for reasonable and customary use in describing the 141 | origin of the Work and reproducing the content of the NOTICE file. 142 | 143 | 7. Disclaimer of Warranty. Unless required by applicable law or 144 | agreed to in writing, Licensor provides the Work (and each 145 | Contributor provides its Contributions) on an "AS IS" BASIS, 146 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 147 | implied, including, without limitation, any warranties or conditions 148 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A 149 | PARTICULAR PURPOSE. You are solely responsible for determining the 150 | appropriateness of using or redistributing the Work and assume any 151 | risks associated with Your exercise of permissions under this License. 152 | 153 | 8. Limitation of Liability. In no event and under no legal theory, 154 | whether in tort (including negligence), contract, or otherwise, 155 | unless required by applicable law (such as deliberate and grossly 156 | negligent acts) or agreed to in writing, shall any Contributor be 157 | liable to You for damages, including any direct, indirect, special, 158 | incidental, or consequential damages of any character arising as a 159 | result of this License or out of the use or inability to use the 160 | Work (including but not limited to damages for loss of goodwill, 161 | work stoppage, computer failure or malfunction, or any and all 162 | other commercial damages or losses), even if such Contributor 163 | has been advised of the possibility of such damages. 164 | 165 | 9. Accepting Warranty or Additional Liability. While redistributing 166 | the Work or Derivative Works thereof, You may choose to offer, 167 | and charge a fee for, acceptance of support, warranty, indemnity, 168 | or other liability obligations and/or rights consistent with this 169 | License. However, in accepting such obligations, You may act only 170 | on Your own behalf and on Your sole responsibility, not on behalf 171 | of any other Contributor, and only if You agree to indemnify, 172 | defend, and hold each Contributor harmless for any liability 173 | incurred by, or claims asserted against, such Contributor by reason 174 | of your accepting any such warranty or additional liability. 175 | 176 | END OF TERMS AND CONDITIONS 177 | 178 | APPENDIX: How to apply the Apache License to your work. 179 | 180 | To apply the Apache License to your work, attach the following 181 | boilerplate notice, with the fields enclosed by brackets "{}" 182 | replaced with your own identifying information. (Don't include 183 | the brackets!) The text should be enclosed in the appropriate 184 | comment syntax for the file format. We also recommend that a 185 | file or class name and description of purpose be included on the 186 | same "printed page" as the copyright notice for easier 187 | identification within third-party archives. 188 | 189 | Copyright {yyyy} {name of copyright owner} 190 | 191 | Licensed under the Apache License, Version 2.0 (the "License"); 192 | you may not use this file except in compliance with the License. 193 | You may obtain a copy of the License at 194 | 195 | http://www.apache.org/licenses/LICENSE-2.0 196 | 197 | Unless required by applicable law or agreed to in writing, software 198 | distributed under the License is distributed on an "AS IS" BASIS, 199 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 200 | See the License for the specific language governing permissions and 201 | limitations under the License. 202 | -------------------------------------------------------------------------------- /src/main/java/com/github/maelstrom/consumer/KafkaConsumer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.github.maelstrom.consumer; 19 | 20 | import kafka.api.FetchRequest; 21 | import kafka.api.FetchRequestBuilder; 22 | import kafka.api.PartitionOffsetRequestInfo; 23 | import kafka.cluster.Broker; 24 | import kafka.common.ErrorMapping; 25 | import kafka.common.KafkaException; 26 | import kafka.common.TopicAndPartition; 27 | import kafka.javaapi.FetchResponse; 28 | import kafka.javaapi.OffsetRequest; 29 | import kafka.javaapi.OffsetResponse; 30 | import kafka.javaapi.PartitionMetadata; 31 | import kafka.javaapi.consumer.SimpleConsumer; 32 | import kafka.message.Message; 33 | import kafka.message.MessageAndMetadata; 34 | import kafka.message.MessageAndOffset; 35 | import kafka.serializer.Decoder; 36 | import org.slf4j.Logger; 37 | import org.slf4j.LoggerFactory; 38 | 39 | import java.util.*; 40 | import java.util.concurrent.Callable; 41 | 42 | /** 43 | * Heavily inspired by Yahoo Pistachio's Kafka Consumer 44 | * - https://github.com/lyogavin/Pistachio/blob/master/src/main/java/com/yahoo/ads/pb/kafka/KafkaSimpleConsumer.java 45 | * 46 | * @author Jeoffrey Lim 47 | * @version 0.2 48 | */ 49 | public final class KafkaConsumer { 50 | private static final Logger LOG = LoggerFactory.getLogger(KafkaConsumer.class); 51 | 52 | private static final long EARLIEST_TIME = kafka.api.OffsetRequest.EarliestTime(); 53 | private static final long LATEST_TIME = kafka.api.OffsetRequest.LatestTime(); 54 | 55 | private static final InfiniteRetryStrategy offsetResponseRetryer = new InfiniteRetryStrategy<>(); 56 | private static final InfiniteRetryStrategy fetchResponseRetryer = new InfiniteRetryStrategy<>(); 57 | 58 | private final List brokersList; 59 | private final Decoder keyDecoder; 60 | private final Decoder valueDecoder; 61 | 62 | public final String consumerGroup; 63 | public final String topic; 64 | public final int partitionId; 65 | 66 | private final LeaderBrokerChecker leaderBrokerChecker; 67 | private SimpleConsumer consumer; 68 | private final String clientId; 69 | 70 | private long currentOffset; 71 | 72 | public KafkaConsumer(final List brokerList, final String consumerGroup, 73 | final Decoder keyDecoder, final Decoder valueDecoder, 74 | final String topic, final int partitionId) { 75 | this.brokersList = brokerList; 76 | this.consumerGroup = consumerGroup; 77 | this.keyDecoder = keyDecoder; 78 | this.valueDecoder = valueDecoder; 79 | this.topic = topic; 80 | this.partitionId = partitionId; 81 | 82 | this.clientId = KafkaMetaData.createClientId(consumerGroup, topic, partitionId); 83 | this.leaderBrokerChecker = new LeaderBrokerChecker(brokersList, topic, partitionId); 84 | 85 | connect(); 86 | } 87 | 88 | private void connect() { 89 | LOG.info("Create consumer: {}-[{}:{}]", consumerGroup,topic, partitionId); 90 | 91 | if (consumer != null) 92 | consumer.close(); 93 | 94 | consumer = KafkaMetaData.getLeaderBroker(brokersList, clientId, topic, partitionId); 95 | } 96 | 97 | public final void close() { 98 | LOG.info("Close consumer: {}-[{}:{}] Last offset: {})", consumerGroup, topic, partitionId, currentOffset); 99 | 100 | //clean up consumer 101 | if (consumer != null) 102 | consumer.close(); 103 | 104 | //clean up broker leader checker 105 | leaderBrokerChecker.stop(); 106 | } 107 | 108 | private void verifyBroker() { 109 | if (leaderBrokerChecker.hasChanged()) { 110 | LOG.warn("{}-[{}-{}] Broker changed. Creating new Consumer", consumerGroup, topic, partitionId); 111 | connect(); 112 | } 113 | } 114 | 115 | @SuppressWarnings("squid:MethodCyclomaticComplexity") 116 | private void checkNeedNewLeader(final short errorCode) { 117 | LOG.warn("Error fetching data from the Broker: [{}:{}] Topic: {}-[{}:{}]@{} Error: {}", consumer.host(), consumer.port(), consumerGroup, topic, partitionId, currentOffset, errorCode); 118 | 119 | boolean needNewLeader = false; 120 | 121 | if (errorCode == ErrorMapping.OffsetOutOfRangeCode()) { 122 | PartitionMetadata partitionMetadata = KafkaMetaData.getPartitionMetadata(consumer, Collections.singletonList(topic), partitionId); 123 | 124 | if (partitionMetadata == null || !LeaderBrokerChecker.isSameBroker( 125 | KafkaMetaData.findNewLeader(brokersList, partitionMetadata.leader(), topic, partitionId).leader(), 126 | partitionMetadata.leader())) { 127 | needNewLeader = true; 128 | } else { 129 | long earliestOffset = getEarliestOffset(); 130 | long latestOffset = getLatestOffset(); 131 | 132 | if (latestOffset < 0 || earliestOffset < 0) 133 | needNewLeader = true; 134 | else if (currentOffset > latestOffset) 135 | throw new KafkaException("Offset Out of Higher Bound for [" + topic + ":" + partitionId + "@" + currentOffset + "] latest:" + latestOffset); 136 | else if (currentOffset < earliestOffset) 137 | throw new KafkaException("Offset Out of Lower Bound for [" + topic + ":" + partitionId + "@" + currentOffset + "] earliest:" + earliestOffset); 138 | } 139 | } else { 140 | needNewLeader = true; 141 | } 142 | 143 | if (needNewLeader) 144 | connect(); 145 | } 146 | 147 | /** 148 | * Iterator of incoming messages halting at stopAtOffset 149 | * 150 | * @param stopAtOffset offset to stop retrieving data 151 | * @return an iterator of MessageAndMetadata key/value 152 | */ 153 | @SuppressWarnings({"squid:MethodCyclomaticComplexity", "squid:S1188"}) 154 | public Iterator> receive(final long stopAtOffset) { 155 | //always verify broker if changed... 156 | verifyBroker(); 157 | 158 | // The fetch API is used to fetch a chunk of one or more logs for some topic-partitions. 159 | // Logically one specifies the topics, partitions, and starting offset at which to begin the fetch and gets back a chunk of messages. 160 | // In general, the return messages will have offsets larger than or equal to the starting offset. 161 | final FetchRequest request = new FetchRequestBuilder().clientId(clientId).addFetch(topic, partitionId, currentOffset, 100000000).maxWait(1000).build(); 162 | final FetchResponse response = fetchResponseRetryer.retryInfinitely(new Callable() { 163 | @Override 164 | public FetchResponse call() { 165 | try { 166 | FetchResponse response = consumer.fetch(request); 167 | if (response != null && !response.hasError()) 168 | return response; 169 | 170 | short errorCode = response != null ? response.errorCode(topic, partitionId) : -1; 171 | 172 | if (errorCode == ErrorMapping.RequestTimedOutCode()) 173 | return response; 174 | else 175 | checkNeedNewLeader(errorCode); 176 | } catch (Exception e) { 177 | LOG.warn("Exception while fetch {}-[{}:{}]", consumerGroup, topic, partitionId, e); 178 | connect(); 179 | } 180 | 181 | return null; 182 | } 183 | }); 184 | 185 | if (response.hasError()) 186 | return Collections.emptyListIterator(); 187 | 188 | List> ret = new LinkedList<>(); 189 | 190 | long lastOffset = currentOffset; 191 | 192 | for (MessageAndOffset messageAndOffset : response.messageSet(topic, partitionId)) { 193 | // However, with compressed messages, it's possible for the returned messages to have offsets smaller than the starting offset. 194 | // The number of such messages is typically small and the caller is responsible for filtering out those messages. 195 | final long messageOffset = messageAndOffset.offset(); 196 | 197 | if (messageOffset >= lastOffset && messageOffset < stopAtOffset) { 198 | //set next offset 199 | currentOffset = messageAndOffset.nextOffset(); 200 | Message message = messageAndOffset.message(); 201 | MessageAndMetadata messageAndMetadata = new MessageAndMetadata<>(topic, partitionId, message, messageOffset, keyDecoder, valueDecoder); 202 | 203 | if (LOG.isDebugEnabled()) 204 | LOG.info("Kafka consumed message: [{}:{}]@{} - {}={}", topic, partitionId, messageOffset, messageAndMetadata.key(), messageAndMetadata.message()); 205 | 206 | ret.add(messageAndMetadata); 207 | } 208 | } 209 | 210 | return ret.iterator(); 211 | } 212 | 213 | public final long getCurrentOffset() { 214 | return currentOffset; 215 | } 216 | 217 | public final void setCurrentOffset(final long currentOffset) { 218 | this.currentOffset = currentOffset; 219 | } 220 | 221 | public long getEarliestOffset() { 222 | return getOffset(EARLIEST_TIME).offsets(topic, partitionId)[0]; 223 | } 224 | 225 | public long getLatestOffset() { 226 | return getOffset(LATEST_TIME).offsets(topic, partitionId)[0]; 227 | } 228 | 229 | private OffsetResponse getOffset(final long whichTime) { 230 | verifyBroker(); 231 | 232 | TopicAndPartition topicAndPartition = new TopicAndPartition(topic, partitionId); 233 | Map requestInfo = new HashMap<>(); 234 | requestInfo.put(topicAndPartition, new PartitionOffsetRequestInfo(whichTime, 1)); 235 | final OffsetRequest request = new OffsetRequest(requestInfo, kafka.api.OffsetRequest.CurrentVersion(), clientId); 236 | 237 | return offsetResponseRetryer.retryInfinitely(new Callable() { 238 | @Override 239 | public OffsetResponse call() throws Exception { 240 | try { 241 | OffsetResponse response = consumer.getOffsetsBefore(request); 242 | if (response != null) { 243 | long[] offsets = response.offsets(topic, partitionId); 244 | if (offsets != null && offsets.length > 0) 245 | return response; 246 | } 247 | LOG.warn("Error fetching offset data: {}-[{}:{}] Error code: {}", consumerGroup, topic, partitionId, response != null ? response.errorCode(topic, partitionId) : -1); 248 | } catch (Exception e) { 249 | LOG.warn("Error fetching offset data: {}-[{}:{}] Exception: {}", consumerGroup, topic, partitionId, e.getMessage(), e); 250 | } 251 | 252 | connect(); //always reconnect in case offset cannot be obtained 253 | 254 | return null; 255 | } 256 | }); 257 | } 258 | } 259 | -------------------------------------------------------------------------------- /src/main/java/com/github/maelstrom/consumer/KafkaMetaData.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.github.maelstrom.consumer; 19 | 20 | import kafka.cluster.Broker; 21 | import kafka.javaapi.PartitionMetadata; 22 | import kafka.javaapi.TopicMetadata; 23 | import kafka.javaapi.TopicMetadataRequest; 24 | import kafka.javaapi.TopicMetadataResponse; 25 | import kafka.javaapi.consumer.SimpleConsumer; 26 | import org.slf4j.Logger; 27 | import org.slf4j.LoggerFactory; 28 | 29 | import java.net.InetAddress; 30 | import java.net.UnknownHostException; 31 | import java.util.*; 32 | import java.util.concurrent.Callable; 33 | import java.util.concurrent.CopyOnWriteArrayList; 34 | 35 | /** 36 | * @author Jeoffrey Lim 37 | * @version 0.2 38 | */ 39 | public class KafkaMetaData { 40 | private static final Logger LOG = LoggerFactory.getLogger(KafkaMetaData.class); 41 | private static final String META_DATA_CLIENT_ID = "-metadata"; 42 | 43 | private static final InfiniteRetryStrategy> brokerListRetryer = new InfiniteRetryStrategy<>(); 44 | private static final InfiniteRetryStrategy> topicMetaResponseRetryer = new InfiniteRetryStrategy<>(); 45 | private static final InfiniteRetryStrategy pmResponseRetryer = new InfiniteRetryStrategy<>(); 46 | private static final InfiniteRetryStrategy consumerResponseRetryer = new InfiniteRetryStrategy<>(); 47 | 48 | private KafkaMetaData() { 49 | } 50 | 51 | public static List createBrokerList(final String seedBrokers) { 52 | return brokerListRetryer.retryInfinitely(new Callable>() { 53 | @Override 54 | public List call() { 55 | try { 56 | Set seedBrokersSet = new HashSet<>(); 57 | 58 | for (String seedBroker : seedBrokers.split(",")) { 59 | String[] hostAndPort = seedBroker.split(":", 2); 60 | String host = hostAndPort[0]; 61 | int port = Integer.parseInt(hostAndPort[1]); 62 | 63 | //brute-force way to obtain ConsumerMetadataKey 64 | //to support older kafka servers 65 | SimpleConsumer consumer = null; 66 | 67 | try { 68 | consumer = new SimpleConsumer(host, port, 100000, 64 * 1024, META_DATA_CLIENT_ID); 69 | final TopicMetadataRequest req = new TopicMetadataRequest(new ArrayList()); 70 | TopicMetadataResponse resp = consumer.send(req); 71 | 72 | List topicMetadataList = resp.topicsMetadata(); 73 | 74 | for (TopicMetadata metaData : topicMetadataList) { 75 | for (PartitionMetadata pm : metaData.partitionsMetadata()) { 76 | Broker leader = pm.leader(); 77 | //validate leader 78 | if (leader != null && leader.host() != null) 79 | seedBrokersSet.add(leader); 80 | } 81 | } 82 | } catch (Exception e) { 83 | LOG.error("Error communicating with Broker [{}:{}] to fetch topic meta data. Reason: {}", host, port, e.getMessage(), e); 84 | } finally { 85 | if (consumer != null) 86 | consumer.close(); 87 | } 88 | } 89 | 90 | List seedBrokersList = new CopyOnWriteArrayList<>(seedBrokersSet); 91 | 92 | Collections.shuffle(seedBrokersList); 93 | 94 | for (Broker broker: seedBrokersList) 95 | LOG.info("FOUND BROKER [{}-{}:{}]", broker.id(), broker.host(), broker.port()); 96 | 97 | return seedBrokersList; 98 | } catch (Exception e) { 99 | LOG.error(e.getMessage(), e); 100 | return null; 101 | } 102 | } 103 | }); 104 | } 105 | 106 | public static String createClientId(final String consumerGroup, final String topic, final int partition) { 107 | String hostName = System.getProperty("os.name").toLowerCase().contains("win") ? 108 | System.getenv("COMPUTERNAME") : 109 | System.getenv("HOSTNAME"); 110 | 111 | if (hostName == null || hostName.isEmpty()) { 112 | try { 113 | hostName = InetAddress.getLocalHost().getCanonicalHostName(); 114 | } catch (UnknownHostException e) { 115 | LOG.warn("[{}] Can't determine local hostname", partition, e); 116 | hostName = "unknown.host"; 117 | } 118 | } 119 | 120 | return consumerGroup + "-" + topic + "-" + partition + "-" + hostName; 121 | } 122 | 123 | @SuppressWarnings("squid:S1188") 124 | public static Map getTopicMetaData(final List brokerList, final List topics) { 125 | final TopicMetadataRequest req = new TopicMetadataRequest(topics); 126 | 127 | return topicMetaResponseRetryer.retryInfinitely(new Callable>() { 128 | @Override 129 | public Map call() throws Exception { 130 | for (Broker seed : brokerList) { 131 | SimpleConsumer consumer = null; 132 | 133 | try { 134 | consumer = new SimpleConsumer(seed.host(), seed.port(), 100000, 64 * 1024, META_DATA_CLIENT_ID); 135 | TopicMetadataResponse resp = consumer.send(req); 136 | 137 | List topicMetadataList = resp.topicsMetadata(); 138 | 139 | Map topicMetaData = new HashMap<>(); 140 | 141 | for (TopicMetadata metaData : topicMetadataList) 142 | topicMetaData.put(metaData.topic(), metaData); 143 | 144 | return topicMetaData; 145 | } catch (Exception e) { 146 | LOG.error("Error communicating with Broker [{}] to fetch topic meta data. Reason: {}", seed, e.getMessage(), e); 147 | } finally { 148 | if (consumer != null) 149 | consumer.close(); 150 | } 151 | } 152 | 153 | return null; 154 | } 155 | }); 156 | } 157 | 158 | public static PartitionMetadata getPartitionMetadata(final SimpleConsumer consumer, final List topics, final int partitionId) { 159 | try { 160 | TopicMetadataRequest req = new TopicMetadataRequest(topics); 161 | TopicMetadataResponse resp = consumer.send(req); 162 | 163 | List topicMetadataList = resp.topicsMetadata(); 164 | 165 | for (TopicMetadata metaData : topicMetadataList) { 166 | for (PartitionMetadata part : metaData.partitionsMetadata()) { 167 | if (part.partitionId() == partitionId) { 168 | return part; 169 | } 170 | } 171 | } 172 | } catch (Exception e) { 173 | LOG.warn("Unable to fetch partition meta data from host[{}:{}] [{}:{}]", consumer.host(), consumer.port(), topics, partitionId, e); 174 | } 175 | 176 | return null; 177 | } 178 | 179 | @SuppressWarnings({"squid:S1188", "squid:S135", "squid:MethodCyclomaticComplexity"}) 180 | public static PartitionMetadata findNewLeader(final List brokerList, final Broker currentBroker, final String topic, final int partitionId) { 181 | final List topics = Collections.singletonList(topic); 182 | 183 | return pmResponseRetryer.retryInfinitely(new Callable() { 184 | @Override 185 | public final PartitionMetadata call() throws Exception { 186 | for (final Broker broker : brokerList) { 187 | SimpleConsumer consumer = null; 188 | 189 | try { 190 | consumer = new SimpleConsumer(broker.host(), broker.port(), 100000, 64 * 1024, META_DATA_CLIENT_ID); 191 | PartitionMetadata partitionMetadata = getPartitionMetadata(consumer, topics, partitionId); 192 | 193 | //skip "unstable" workers not having information about the topic & partition 194 | if (partitionMetadata == null || partitionMetadata.leader() == null) 195 | continue; 196 | 197 | //attempt find within the list 198 | if (currentBroker == null && LeaderBrokerChecker.isSameBroker(broker, partitionMetadata.leader())) 199 | return partitionMetadata; 200 | 201 | //ignore "self" lookup, and only look at replica brokers 202 | if (LeaderBrokerChecker.isSameBroker(currentBroker, partitionMetadata.leader())) 203 | continue; 204 | //if reported from a replica broker that it is no longer the leader, return new leader broker information 205 | if (!LeaderBrokerChecker.isSameBroker(currentBroker, partitionMetadata.leader())) 206 | return partitionMetadata; 207 | } catch (Exception e) { 208 | LOG.error("Error communicating with Broker [{}] to find Leader for [{}:{}] Reason: {}", broker, topic, partitionId, e.getMessage(), e); 209 | } finally { 210 | if (consumer != null) 211 | consumer.close(); 212 | } 213 | } 214 | 215 | return null; 216 | } 217 | }); 218 | } 219 | 220 | @SuppressWarnings("squid:S1188") 221 | public static SimpleConsumer getLeaderBroker(final List brokerList, final String clientId, final String topic, final int partitionId) { 222 | final List topics = Collections.singletonList(topic); 223 | 224 | return consumerResponseRetryer.retryInfinitely(new Callable() { 225 | @Override 226 | public final SimpleConsumer call() throws Exception { 227 | 228 | for (final Broker broker : brokerList) { 229 | SimpleConsumer consumer = null; 230 | 231 | try { 232 | consumer = new SimpleConsumer(broker.host(), broker.port(), 100000, 64 * 1024, clientId); 233 | PartitionMetadata partitionMetadata = getPartitionMetadata(consumer, topics, partitionId); 234 | 235 | //skip "unstable" workers not having information about the topic & partition 236 | if (partitionMetadata == null || partitionMetadata.leader() == null) 237 | continue; 238 | 239 | LOG.info("broker [{}:{}] = consumer [{}:{}] = partition-leader: [{}:{}]", broker.host(), broker.port(), consumer.host(), consumer.port(), partitionMetadata.leader().host(), partitionMetadata.leader().port()); 240 | 241 | if (LeaderBrokerChecker.isSameBroker(broker, partitionMetadata.leader())) { 242 | SimpleConsumer leaderConsumer = consumer; 243 | consumer = null; 244 | return leaderConsumer; 245 | } 246 | } catch (Exception e) { 247 | LOG.error("Error communicating with Broker [{}] to find Leader for [{}:{}] Reason: {}", broker, topic, partitionId, e.getMessage(), e); 248 | } finally { 249 | if (consumer != null) 250 | consumer.close(); 251 | } 252 | } 253 | 254 | return null; 255 | } 256 | }); 257 | } 258 | } 259 | --------------------------------------------------------------------------------