├── .gitignore ├── CONTRIBUTORS.md ├── LICENSE ├── README.md ├── pom.xml └── src ├── main ├── resources │ └── scalastyle_config.xml └── scala │ └── com │ └── groupon │ └── dse │ ├── configs │ ├── AppConfigs.scala │ ├── KafkaServerConfig.scala │ └── ReceiverConfigs.scala │ ├── example │ ├── SampleBaryonDriver.scala │ └── plugins │ │ └── EventLoggerPlugin.scala │ ├── kafka │ ├── cluster │ │ ├── Cluster.scala │ │ └── impl │ │ │ └── ClusterImpl.scala │ ├── common │ │ ├── ConsumerClientCache.scala │ │ ├── KafkaException.scala │ │ ├── KafkaExceptionHandler.scala │ │ ├── Outcome.scala │ │ ├── State.scala │ │ └── WrappedMessage.scala │ ├── controllers │ │ ├── InMemoryStateController.scala │ │ ├── StateController.scala │ │ ├── StateControllerBuilder.scala │ │ └── ZookeeperStateController.scala │ ├── partition │ │ ├── Partition.scala │ │ ├── PartitionUtils.scala │ │ └── impl │ │ │ └── PartitionImpl.scala │ ├── policy │ │ ├── FetchPolicy.scala │ │ ├── FetchPolicyBuilder.scala │ │ ├── OffsetBasedFetchPolicy.scala │ │ ├── TimeAndOffsetBasedFetchPolicy.scala │ │ └── TimeBasedFetchPolicy.scala │ └── topic │ │ ├── HdfsJsonTopicFetcher.scala │ │ ├── HttpJsonTopicFetcher.scala │ │ ├── LocalTopicFetcher.scala │ │ ├── TopicFetcher.scala │ │ ├── TopicFetcherBuilder.scala │ │ └── TopicUtils.scala │ ├── spark │ ├── dstreams │ │ ├── KafkaLowLevelConsumer.scala │ │ └── KafkaLowLevelDStream.scala │ ├── listeners │ │ ├── MetricsListener.scala │ │ └── StreamingMetricsListener.scala │ └── plugins │ │ ├── PluginExecutor.scala │ │ └── ReceiverPlugin.scala │ ├── util │ └── Utils.scala │ └── zookeeper │ └── ZkClientBuilder.scala └── test └── scala └── com └── groupon └── dse ├── configs ├── AppConfigsTest.scala ├── KafkaServerConfigTest.scala └── ReceiverConfigTest.scala ├── helper ├── EmbeddedKafka.scala ├── EmbeddedSpark.scala ├── EmbeddedZookeeper.scala ├── TestDefaults.scala └── TestUtils.scala ├── kafka ├── cluster │ └── impl │ │ └── ClusterImplTest.scala ├── common │ ├── ConsumerClientCacheTest.scala │ ├── KafkaExceptionHandlerTest.scala │ └── StateTest.scala ├── controllers │ ├── InMemoryStateControllerTest.scala │ ├── StateControllerTest.scala │ └── ZookeeperStateControllerTest.scala ├── partition │ ├── PartitionUtilsTest.scala │ └── impl │ │ └── PartitionImplTest.scala └── topic │ ├── HdfsJsonTopicFetcherTest.scala │ ├── HttpJsonTopicFetcherTest.scala │ ├── LocalTopicFetcherTest.scala │ ├── TopicFetcherBuilder$Test.scala │ └── TopicUtils$Test.scala ├── policy └── FetchPolicyTest.scala ├── spark └── dstreams │ ├── KafkaLowLevelConsumerTest.scala │ └── KafkaLowLevelDStreamTest.scala ├── util └── UtilTest.scala └── zookeeper └── ZkClientBuilderTest.scala /.gitignore: -------------------------------------------------------------------------------- 1 | dependency-reduced-pom.xml 2 | *.swp 3 | *.iml 4 | *.DS_Store 5 | .sonar/ 6 | .idea/ 7 | target/ 8 | build/ 9 | out/ 10 | *.iml 11 | .classpath 12 | .project 13 | scalastyle_output.xml 14 | -------------------------------------------------------------------------------- /CONTRIBUTORS.md: -------------------------------------------------------------------------------- 1 | # Contributors 2 | 3 | The following people have made source contributions to Baryon: 4 | 5 | * Atsunori Kakitani (atkakitani at groupon dot com) 6 | * Rakesh Nair (rakesh at groupon dot com) 7 | * Udit Mehta (umehta at groupon dot com) 8 | 9 | © Groupon Inc., 2016 10 | 11 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | Copyright (c) 2016, Groupon, Inc. 2 | All rights reserved. 3 | 4 | Redistribution and use in source and binary forms, with or without 5 | modification, are permitted provided that the following conditions are 6 | met: 7 | 8 | Redistributions of source code must retain the above copyright notice, 9 | this list of conditions and the following disclaimer. 10 | 11 | Redistributions in binary form must reproduce the above copyright 12 | notice, this list of conditions and the following disclaimer in the 13 | documentation and/or other materials provided with the distribution. 14 | 15 | Neither the name of GROUPON nor the names of its contributors may be 16 | used to endorse or promote products derived from this software without 17 | specific prior written permission. 18 | 19 | THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 20 | IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 21 | TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 22 | PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 23 | HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 24 | SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 25 | TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 26 | PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 27 | LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 28 | NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 29 | SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 30 | 31 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Baryon 2 | **Baryon is a library for building Spark streaming applications that consume data from Kafka.** 3 | 4 | Baryon abstracts away all the bookkeeping involved in reliably connecting to a Kafka cluster and fetching data from it, so that users only need to focus on the logic to process this data. 5 | 6 | For a detailed guide on getting started with Baryon, take a look at the [wiki](../../wiki). 7 | 8 | ## Why Baryon? 9 | Spark itself also has libraries for interacting with Kafka, as documented in its [Kafka integration guide](https://spark.apache.org/docs/latest/streaming-kafka-integration.html). These libraries are well-developed, but there are certain limitations there that Baryon intends to address: 10 | 11 | * [Code-independent checkpointing](Code-Independent-Checkpointing) 12 | 13 | Baryon's Kafka state management system allows Kafka consumption state to be stored across multiple runs of an application, even when there are code changes. Spark's checkpointing system does not support maintaining state across changes in code, so users of Spark's Kafka libraries must implement the offset management logic themselves. 14 | 15 | * Improved error handling 16 | 17 | Baryon handles errors related to Kafka much more thoroughly than Spark's Kafka libraries, so users don't need to worry about handling Kafka problems in their code. 18 | 19 | 20 | In addition to the above, there are a handful of additional features unique to Baryon: 21 | 22 | * [Multiple consumption modes](Consumption-Modes) 23 | 24 | Baryon has two modes of consumption, the blocking mode and the non-blocking mode, which can be changed without any code changes. The blocking mode more or less corresponds to the consumption behavior of the ["direct" approach](https://spark.apache.org/docs/latest/streaming-kafka-integration.html#approach-2-direct-approach-no-receivers), while the non-blocking mode has consumption behavior similar to the [receiver-based approach](https://spark.apache.org/docs/latest/streaming-kafka-integration.html#approach-1-receiver-based-approach). 25 | 26 | * [Dynamically configured topics](Dynamically-Configured-Topics) 27 | 28 | Baryon supports changes to the set of Kafka topics that are consumed while the application is running. Alongside this, configurations can be set at a per-topic level, which makes it easier to build a single application to process multiple, heterogeneous data streams. 29 | 30 | * [Aggregated metrics](Aggregated-Metrics) 31 | 32 | Baryon uses the [spark-metrics library](https://github.com/groupon/spark-metrics) to collect and aggregate useful metrics across the driver and executors. These include metrics like offset lag, throughput, error rates, as well as augmented versions of existing metrics that Spark provides. The metrics here are integrated with Spark's metrics system, so they are compatible with the reporting system that comes with Spark. 33 | 34 | 35 | ## Quick Start 36 | Add Baryon as a dependency: 37 | ```xml 38 | 39 | com.groupon.dse 40 | baryon 41 | 1.0 42 | 43 | ``` 44 | 45 | If you want to add custom metrics that are integrated with Spark, use the [spark-metrics](https://github.com/groupon/spark-metrics) that Baryon also uses: 46 | ```xml 47 | 48 | com.groupon.dse 49 | spark-metrics 50 | 1.0 51 | 52 | ``` 53 | 54 | Take a look at the [examples](src/main/scala/com/groupon/dse/example) to see how to write the driver and a `ReceiverPlugin`. 55 | 56 | -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/configs/KafkaServerConfig.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.configs 34 | 35 | import java.util.Properties 36 | 37 | /** 38 | * Kafka server specific configs 39 | */ 40 | case class KafkaServerConfig(brokerZk: String, 41 | socketTimeout: Int, 42 | socketBufferSize: Int, 43 | zkConnectionTimeout: Int, 44 | zkSessionTimeout: Int 45 | ) 46 | extends Serializable { 47 | 48 | override def toString: String = s"KafkaServerConfig = [brokerZk: $brokerZk, socketTimeout: $socketTimeout, " + 49 | s"socketBufferSize: $socketBufferSize, zkConnTimeOut: $zkConnectionTimeout, " + 50 | s"zkSessionTimeOut: $zkSessionTimeout]" 51 | } 52 | 53 | /** 54 | * Create an instance of [[KafkaServerConfig]] based on user params 55 | */ 56 | object KafkaServerConfigBuilder { 57 | val requiredProperties = Array(AppConfigs.KafkaBrokerConnect._1) 58 | 59 | def apply(properties: Properties): KafkaServerConfig = { 60 | AppConfigs.validate(properties, requiredProperties) 61 | 62 | val brokerConnect = properties.getProperty(AppConfigs.KafkaBrokerConnect._1) 63 | 64 | val socketTimeOut = properties.getProperty(AppConfigs.KafkaSocketTimeoutMs._1, 65 | AppConfigs.KafkaSocketTimeoutMs._2).toInt 66 | 67 | val socketBufferSize = properties.getProperty(AppConfigs.KafkaSocketBufferSizeBytes._1, 68 | AppConfigs.KafkaSocketBufferSizeBytes._2).toInt 69 | 70 | val zkConnectionTimeout = properties.getProperty(AppConfigs.KafkaZkConnectionTimeoutMs._1, 71 | AppConfigs.KafkaZkConnectionTimeoutMs._2).toInt 72 | 73 | val zkSessionTimeout = properties.getProperty(AppConfigs.KafkaZkSessionTimeoutMs._1, 74 | AppConfigs.KafkaZkSessionTimeoutMs._2).toInt 75 | 76 | KafkaServerConfig( 77 | brokerConnect, 78 | socketTimeOut, 79 | socketBufferSize, 80 | zkConnectionTimeout, 81 | zkSessionTimeout) 82 | } 83 | 84 | } 85 | -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/configs/ReceiverConfigs.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.configs 34 | 35 | import java.util.Properties 36 | 37 | import com.groupon.dse.configs.AppConfigs.InvalidConfigException 38 | import com.groupon.dse.kafka.controllers.{StateController, StateControllerBuilder} 39 | import com.groupon.dse.kafka.policy.{FetchPolicy, FetchPolicyBuilder} 40 | import com.groupon.dse.kafka.topic.{TopicFetcher, TopicFetcherBuilder} 41 | import org.slf4j.LoggerFactory 42 | 43 | /** 44 | * Configs that will be sent to each [[com.groupon.dse.spark.dstreams.KafkaLowLevelDStream]] 45 | */ 46 | case class ReceiverConfigs(isBlocking: Boolean, 47 | fetchPolicy: FetchPolicy, 48 | kafkaServerConfigs: KafkaServerConfig, 49 | stateController: StateController, 50 | partitionRefreshIntervalMs: Int, 51 | partitionWarmUpRefreshIntervalMs: Int, 52 | receiverRestIntervalOnFail: Int, 53 | receiverRestIntervalOnSuccess: Int, 54 | topicRepartitionFactor: Int, 55 | topicFetcher: TopicFetcher 56 | ) 57 | extends Serializable { 58 | 59 | override def toString: String = s"isBlocking: $isBlocking\n" + 60 | s"fetchPolicy: $fetchPolicy\n" + 61 | s"kafkaServerConfigs: $kafkaServerConfigs\n" + 62 | s"stateController: $stateController\n" + 63 | s"topicFetcher: $topicFetcher\n" + 64 | s"partitionRefreshIntervalMs: $partitionRefreshIntervalMs\n" + 65 | s"partitionWarmUpRefreshIntervalMs: $partitionWarmUpRefreshIntervalMs\n" + 66 | s"receiverRestIntervalOnFail: $receiverRestIntervalOnFail\n" + 67 | s"receiverRestIntervalOnSuccess: $receiverRestIntervalOnSuccess\n" + 68 | s"topicRepartitionFactor: $topicRepartitionFactor]" 69 | } 70 | 71 | /** 72 | * Parse a Properties object to construct a ReceiverConfigs object 73 | */ 74 | object ReceiverConfigBuilder { 75 | 76 | val logger = LoggerFactory.getLogger(getClass) 77 | 78 | def apply(properties: Properties, 79 | providedStateController: Option[StateController] = None, 80 | providedTopicFetcher: Option[TopicFetcher] = None 81 | ) 82 | : ReceiverConfigs = { 83 | val isBlocking = AppConfigs.validatedBooleanConfig( 84 | properties, 85 | AppConfigs.TopicsEnableBlockingConsumption._1, 86 | AppConfigs.TopicsEnableBlockingConsumption._2) 87 | 88 | val partitionRefreshInterval = properties.getProperty(AppConfigs.PartitionRefreshIntervalMs._1, 89 | AppConfigs.PartitionRefreshIntervalMs._2).toInt 90 | 91 | val partitionWarmUpInterval = properties.getProperty(AppConfigs.PartitionWarmUpRefreshIntervalMs._1, 92 | AppConfigs.PartitionWarmUpRefreshIntervalMs._2).toInt 93 | 94 | val receiverRestIntervalOnFail = properties.getProperty(AppConfigs.ReceiverRestIntervalOnFailMs._1, 95 | AppConfigs.ReceiverRestIntervalOnFailMs._2).toInt 96 | 97 | val receiverRestIntervalOnSuccess = properties.getProperty(AppConfigs.ReceiverRestIntervalOnSuccessMs._1, 98 | AppConfigs.ReceiverRestIntervalOnSuccessMs._2).toInt 99 | 100 | val kafkaServerConfigs = KafkaServerConfigBuilder(properties) 101 | 102 | val fetchPolicy = FetchPolicyBuilder(properties) 103 | 104 | val stateController = providedStateController match { 105 | case _: Some[StateController] => providedStateController.get 106 | case None => StateControllerBuilder(properties) 107 | } 108 | 109 | val topicRepartitionFactor = properties.getProperty(AppConfigs.TopicRepartitionFactor._1, 110 | AppConfigs.TopicRepartitionFactor._2).toInt 111 | if (topicRepartitionFactor < 1) { 112 | throw InvalidConfigException(s"${AppConfigs.TopicRepartitionFactor._1} must be >= 1") 113 | } 114 | 115 | val topicFetcher = providedTopicFetcher match { 116 | case _: Some[TopicFetcher] => providedTopicFetcher.get 117 | case None => TopicFetcherBuilder(properties) 118 | } 119 | 120 | val receiverConfigs = ReceiverConfigs( 121 | isBlocking, 122 | fetchPolicy, 123 | kafkaServerConfigs, 124 | stateController, 125 | partitionRefreshInterval, 126 | partitionWarmUpInterval, 127 | receiverRestIntervalOnFail, 128 | receiverRestIntervalOnSuccess, 129 | topicRepartitionFactor, 130 | topicFetcher 131 | ) 132 | 133 | logger.info(s"New ReceiverConfigs created with properties: $receiverConfigs") 134 | 135 | receiverConfigs 136 | } 137 | 138 | 139 | } -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/example/SampleBaryonDriver.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.example 34 | 35 | import java.util.Properties 36 | 37 | import com.groupon.dse.example.plugins.EventLoggerPlugin 38 | import com.groupon.dse.spark.plugins.{PluginExecutor, ReceiverPlugin} 39 | import org.apache.spark.SparkConf 40 | import org.apache.spark.streaming.{Seconds, StreamingContext} 41 | 42 | /** 43 | * Sample application that uses PluginExecutor to start consuming data 44 | * from Kafka. The PluginExecutor executes a single Plugin that prints the 45 | * very first message in an RDD fetched from the underlying receiver. 46 | */ 47 | object SampleBaryonDriver { 48 | 49 | def main(args: Array[String]): Unit = { 50 | val sparkConf = new SparkConf().setAppName("SampleBaryonDriverApp") 51 | val batchInterval = Seconds(2) 52 | 53 | // Set the wait time for NODE_LOCAL tasks to 0. This is to prevent executors from waiting a set amount of time until 54 | // it tries to read data from a Kafka receiver on a different worker node. 55 | sparkConf.set("spark.locality.wait.node", "0s") 56 | 57 | val streamingContext = new StreamingContext(sparkConf, batchInterval) 58 | 59 | // Create a new plugin executor and start the streaming process 60 | val executor = new PluginExecutor(streamingContext) 61 | .forTopics("test-topic") 62 | .withKafkaZKEndpoint("localhost:2181") 63 | .usingLocalTopicFetcher 64 | .usingZKStateController 65 | .withZKStateControllerEndpoint("localhost:2181") 66 | .withZKStateControllerPath("/test-app") 67 | .withProperties(appConfigs) 68 | 69 | // Plugin initializations 70 | val printHead: ReceiverPlugin = new EventLoggerPlugin 71 | val plugins = Seq(printHead) 72 | 73 | // Plugin execution 74 | executor.execute(plugins) 75 | 76 | streamingContext.start() 77 | streamingContext.awaitTermination() 78 | } 79 | 80 | /** 81 | * Properties required to instantiate Baryon receivers 82 | * 83 | * @return Properties object 84 | */ 85 | def appConfigs: Properties = { 86 | /** 87 | * Note: 88 | * 1. Core properties (topics, kafkaEndpoint, zkEndpoint, zkPath) can also be just added to this properties 89 | * object. In such scenarios, instantiate the PluginExecutor without calling the Fluent api's. For eg: 90 | * val pluginExecutor = new PluginExecutor(streamingContext).withProperties(properties) 91 | * 92 | * 2. Properties can also be loaded from a .properties file using the AppConfigs.loadFromFile(fileName) API. 93 | * While submitting a job, make sure the property file is included in the 94 | * classpath using the '--files' option. 95 | */ 96 | 97 | val properties = new Properties() 98 | properties.setProperty("spark.num.receivers", "2") 99 | properties.setProperty("topic.start.offset", "-1") 100 | properties.setProperty("spark.storage.level", "MEMORY_AND_DISK_SER_2") 101 | properties.setProperty("partition.refresh.interval.ms", "10000") 102 | properties 103 | } 104 | } 105 | 106 | -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/example/plugins/EventLoggerPlugin.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.example.plugins 34 | 35 | import com.groupon.dse.kafka.common.WrappedMessage 36 | import com.groupon.dse.spark.plugins.ReceiverPlugin 37 | import org.apache.spark.rdd.RDD 38 | import org.slf4j.LoggerFactory 39 | 40 | /** 41 | * A plugin that prints each event in the RDD to a log file 42 | */ 43 | class EventLoggerPlugin extends ReceiverPlugin { 44 | private val logger = LoggerFactory.getLogger(getClass) 45 | 46 | override def execute(messages: RDD[WrappedMessage]): Unit = { 47 | messages.take(2).foreach(wm => { 48 | logger.info(s"Event -> ${new String(wm.payload)}") 49 | }) 50 | } 51 | } 52 | -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/kafka/cluster/Cluster.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.cluster 34 | 35 | import com.groupon.dse.configs.KafkaServerConfig 36 | import com.groupon.dse.kafka.partition.Partition 37 | import kafka.utils.ZkUtils 38 | import org.I0Itec.zkclient.ZkClient 39 | 40 | /** 41 | * Class to interact with Kafka cluster 42 | * 43 | * @param serverConfigs [[KafkaServerConfig]] instance with server related configs 44 | */ 45 | abstract class Cluster(val serverConfigs: KafkaServerConfig) 46 | extends Serializable { 47 | 48 | /** 49 | * Obtain list of all topics in the Kafka cluster 50 | * @param zkClient Client to interact with the Kafka Zookeeper 51 | * @return Seq of Kafka topics 52 | */ 53 | def topics(zkClient: ZkClient): Seq[String] 54 | 55 | /** 56 | * Obtain seq of [[Partition]] objects for the provided set of topics 57 | * 58 | * @param topics Seq of topics for which [[Partition]] object should be created 59 | * @param zkClient Client to interact with the Kafka Zookeeper 60 | * @return Seq of [[Partition]] objects 61 | */ 62 | def partitions(topics: Seq[String], zkClient: ZkClient): Seq[Partition] 63 | 64 | /** 65 | * Zookeeper path which stores the list of id's for connected brokers 66 | * 67 | * @return Zookeeper path string 68 | */ 69 | def brokerIdPath: String = ZkUtils.BrokerIdsPath 70 | 71 | /** 72 | * Zookeeper path which stores the list of topics stored in the cluster 73 | * 74 | * @return Zookeeper path string 75 | */ 76 | def brokerTopicsPath: String = ZkUtils.BrokerTopicsPath 77 | } 78 | -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/kafka/cluster/impl/ClusterImpl.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.cluster.impl 34 | 35 | import com.groupon.dse.configs.KafkaServerConfig 36 | import com.groupon.dse.kafka.cluster.Cluster 37 | import com.groupon.dse.kafka.common.KafkaException._ 38 | import com.groupon.dse.kafka.partition.Partition 39 | import com.groupon.dse.kafka.partition.impl.PartitionImpl 40 | import kafka.utils.ZkUtils 41 | import org.I0Itec.zkclient.ZkClient 42 | import org.slf4j.LoggerFactory 43 | 44 | /** 45 | * Cluster class to interact with Kafka version 0.8.* cluster 46 | * 47 | * @param serverConfigs [[KafkaServerConfig]] instance with server related configs 48 | */ 49 | class ClusterImpl(serverConfigs: KafkaServerConfig) 50 | extends Cluster(serverConfigs) { 51 | 52 | private val logger = LoggerFactory.getLogger(classOf[ClusterImpl]) 53 | 54 | /** 55 | * Obtain list of all topics in the Kafka cluster 56 | * 57 | * @param zkClient Client to interact with the Kafka Zookeeper 58 | * @return Seq of Kafka topics 59 | */ 60 | override def topics(zkClient: ZkClient): Seq[String] = { 61 | ZkUtils.getAllTopics(zkClient) 62 | } 63 | 64 | /** 65 | * Obtain seq of [[Partition]] objects for the provided set of topics 66 | * 67 | * @param topics Seq of topics for which [[Partition]] object should be created 68 | * @param zkClient Client to interact with the Kafka Zookeeper 69 | * @return Seq of [[Partition]] objects 70 | */ 71 | override def partitions(topics: Seq[String], zkClient: ZkClient): Seq[Partition] = { 72 | val partitionList = scala.collection.mutable.ArrayBuffer.empty[Partition] 73 | val topicPartitionMap = ZkUtils.getPartitionsForTopics(zkClient, topics) 74 | topicPartitionMap.foreach(topicPartition => { 75 | val topic = topicPartition._1 76 | topicPartition._2.foreach(partition => { 77 | try { 78 | partitionList += new PartitionImpl(topic, partition) 79 | } catch { 80 | case e: LeaderNotAvailableException => logger.error(s"Could not find leader for $topicPartition") 81 | } 82 | }) 83 | }) 84 | logger.debug(s"Obtained a new list of partitions for the topics: $topics \n $partitionList") 85 | partitionList.toList 86 | } 87 | 88 | } 89 | -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/kafka/common/ConsumerClientCache.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.common 34 | 35 | import com.groupon.dse.configs.{AppConfigs, KafkaServerConfig} 36 | import com.groupon.dse.kafka.common.KafkaException.LeaderNotAvailableException 37 | import com.groupon.dse.kafka.partition.{Leader, Partition} 38 | import com.groupon.dse.zookeeper.ZkClientBuilder 39 | import kafka.consumer.SimpleConsumer 40 | import org.I0Itec.zkclient.exception.ZkException 41 | import org.slf4j.LoggerFactory 42 | 43 | /** 44 | * Cache for storing SimpleConsumer instances for [[Partition]] objects 45 | */ 46 | class ConsumerClientCache(serverConfig: KafkaServerConfig, clientName: String) extends Serializable { 47 | 48 | private val logger = LoggerFactory.getLogger(getClass) 49 | private val consumerClientCache = scala.collection.mutable.HashMap.empty[Partition, SimpleConsumer] 50 | 51 | /** 52 | * Get the SimpleConsumer client created for the Partition. 53 | * Insert new entry if it does not already exist 54 | * 55 | * @param partition [[Partition]] for which a consumer client is required 56 | * @return Cached SimpleConsumer instance 57 | */ 58 | def getWithLoad(partition: Partition): Option[SimpleConsumer] = { 59 | if (!consumerClientCache.contains(partition)) { 60 | val zkConnTimeout = AppConfigs.ZookeeperStateControllerConnTimeoutMs._2.toInt 61 | val zkSessionTimeout = AppConfigs.ZookeeperStateControllerSessionTimeoutMs._2.toInt 62 | val zkClient = ZkClientBuilder(serverConfig.brokerZk, zkConnTimeout, zkSessionTimeout) 63 | try { 64 | val leader = partition.leader(zkClient) 65 | val client = ConsumerClientBuilder.newInstance(leader, serverConfig, clientName) 66 | consumerClientCache += partition -> client 67 | } catch { 68 | case lna: LeaderNotAvailableException => { 69 | logger.error(s"Cannot create consumer object for partition: $partition. Leader unavailable.") 70 | None 71 | } 72 | case zke: ZkException => { 73 | logger.error(s"Zookeeper exception while creating consumer object for partition: $partition", zke) 74 | None 75 | } 76 | 77 | } finally { 78 | zkClient.close() 79 | } 80 | } 81 | Some(consumerClientCache(partition)) 82 | } 83 | 84 | /** 85 | * Get the SimpleConsumer client created for the Partition. Return None if no entry found 86 | * 87 | * @param partition [[Partition]] for which a consumer client is required 88 | * @return Cached SimpleConsumer instance 89 | */ 90 | def get(partition: Partition): Option[SimpleConsumer] = { 91 | if (consumerClientCache.contains(partition)) { 92 | Some(consumerClientCache(partition)) 93 | } else { 94 | None 95 | } 96 | } 97 | 98 | /** 99 | * Remove from cache the SimpleConsumer instance corresponding to the [[Partition]] 100 | * 101 | * @param partition [[Partition]] for which the the consumer client entry should be removed 102 | */ 103 | def remove(partition: Partition): Unit = { 104 | if (consumerClientCache.contains(partition)) { 105 | consumerClientCache(partition).close() 106 | consumerClientCache.remove(partition) 107 | } 108 | } 109 | 110 | /** 111 | * Check if a [[Partition]] object is present in cache 112 | * 113 | * @param partition [[Partition]] object key 114 | * @return True if the [[Partition]] object key exists 115 | */ 116 | def contains(partition: Partition): Boolean = consumerClientCache.contains(partition) 117 | } 118 | 119 | /** 120 | * Builder for Kafka SimpleConsumer clients 121 | */ 122 | object ConsumerClientBuilder { 123 | 124 | /** 125 | * Create a new Kafka SimpleConsumer instance 126 | * 127 | * @param leader [[Leader]] contains the Kafka broker:port to connect to 128 | * @param serverConfig Kafka server configs 129 | * @param clientName Name of the client 130 | * @return 131 | */ 132 | def newInstance(leader: Leader, 133 | serverConfig: KafkaServerConfig, 134 | clientName: String) 135 | : SimpleConsumer = new SimpleConsumer( 136 | leader.host, 137 | leader.port, 138 | serverConfig.socketTimeout, 139 | serverConfig.socketBufferSize, 140 | clientName 141 | ) 142 | } 143 | -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/kafka/common/KafkaException.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.common 34 | 35 | /** 36 | * Generic Kafka exception class 37 | * 38 | * @param message Error description 39 | */ 40 | abstract class KafkaException(message: String) extends Exception(message) 41 | 42 | /** 43 | * Singleton to resolve Kafka error codes 44 | */ 45 | object KafkaException { 46 | 47 | // Error codes 48 | val Unknown = -1 49 | val OffsetOutOfRange = 1 50 | val InvalidMessage = 2 51 | val UnknownTopicOrPartition = 3 52 | val InvalidMessageSize = 4 53 | val LeaderNotAvailable = 5 54 | val OffsetMetadataTooLarge = 12 55 | val OffsetLoadInProgress = 14 56 | 57 | /** 58 | * Raise appropriate exception based on error code returned by Kafka server 59 | * 60 | * @param code Kafka server error code 61 | * @return Instance of [[KafkaException]] 62 | */ 63 | def apply(code: Int): KafkaException = code match { 64 | case Unknown => new UnknownException("Unexpected error") 65 | case OffsetOutOfRange => new OffsetOutOfRangeException("Offset out of range") 66 | case InvalidMessage => new InvalidMessageException("Invalid message") 67 | case UnknownTopicOrPartition => new UnknownTopicOrPartitionException("Unknown topic or partition") 68 | case InvalidMessageSize => new InvalidMessageSizeException("Invalid message size") 69 | case LeaderNotAvailable => new LeaderNotAvailableException("Leader not available") 70 | case OffsetMetadataTooLarge => new OffsetMetadataTooLargeException("Offset metadata too large") 71 | case OffsetLoadInProgress => new OffsetsLoadInProgressException("Offset load in progress") 72 | } 73 | 74 | /** 75 | * An unexpected server error 76 | * 77 | * @param message Error description 78 | */ 79 | case class UnknownException(message: String) extends KafkaException(message) 80 | 81 | /** 82 | * The requested offset is outside the range of offsets maintained by the server for the given topic/partition 83 | * 84 | * @param message Error description 85 | */ 86 | case class OffsetOutOfRangeException(message: String) extends KafkaException(message) 87 | 88 | /** 89 | * Message contents does not match its CRC 90 | * 91 | * @param message Error description 92 | */ 93 | case class InvalidMessageException(message: String) extends KafkaException(message) 94 | 95 | /** 96 | * This request is for a topic or partition that does not exist on this broker 97 | * 98 | * @param message Error description 99 | */ 100 | case class UnknownTopicOrPartitionException(message: String) extends KafkaException(message) 101 | 102 | /** 103 | * The message has a negative size 104 | * 105 | * @param message Error description 106 | */ 107 | case class InvalidMessageSizeException(message: String) extends KafkaException(message) 108 | 109 | /** 110 | * Leader not available 111 | * 112 | * @param message Error description 113 | */ 114 | case class LeaderNotAvailableException(message: String) extends KafkaException(message) 115 | 116 | /** 117 | * If you specify a string larger than configured maximum for offset metadata 118 | * 119 | * @param message Error description 120 | */ 121 | case class OffsetMetadataTooLargeException(message: String) extends KafkaException(message) 122 | 123 | /** 124 | * The broker returns this error code for an offset fetch request if it is still loading offsets 125 | * 126 | * @param message Error description 127 | */ 128 | case class OffsetsLoadInProgressException(message: String) extends KafkaException(message) 129 | 130 | } 131 | -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/kafka/common/KafkaExceptionHandler.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.common 34 | 35 | import com.groupon.dse.configs.ReceiverConfigs 36 | import com.groupon.dse.kafka.common.KafkaException._ 37 | import com.groupon.dse.kafka.common.Outcome._ 38 | import com.groupon.dse.kafka.partition.Partition 39 | import com.groupon.dse.spark.dstreams.KafkaLowLevelReceiver 40 | import com.groupon.dse.util.Utils 41 | import org.slf4j.LoggerFactory 42 | 43 | object KafkaExceptionHandler { 44 | 45 | private val logger = LoggerFactory.getLogger(getClass) 46 | 47 | /** 48 | * Handle specific Kafka exceptions 49 | * 50 | * @param kafkaException Instance of [[KafkaException]] 51 | * @param partition Current [[Partition]] 52 | * @param localState Local [[State]] for the [[Partition]] 53 | * @param receiver Underlying [[KafkaLowLevelReceiver]] 54 | * @param clientName Receiver name 55 | * 56 | * @return Type of [[Outcome]] 57 | */ 58 | def handleException(kafkaException: KafkaException, 59 | partition: Partition, 60 | localState: State, 61 | consumerClientCache: ConsumerClientCache, 62 | receiver: KafkaLowLevelReceiver, 63 | clientName: String) 64 | : Outcome = { 65 | val stateController = receiver.receiverConfigs.stateController 66 | 67 | val partitionKey = stateController.generateStateKey(partition) 68 | 69 | kafkaException match { 70 | // Set to next best available offset when required 71 | case ims: InvalidMessageSizeException => { 72 | val state = State(localState.offset + 1, localState.timestamp) 73 | blockingSetState(partitionKey, state, receiver.receiverConfigs) 74 | } 75 | 76 | case im: InvalidMessageException => { 77 | val state = State(localState.offset + 1, localState.timestamp) 78 | blockingSetState(partitionKey, state, receiver.receiverConfigs) 79 | } 80 | 81 | // Ensure the requested offsets are within the range of stored Kafka offsets 82 | case oor: OffsetOutOfRangeException => { 83 | val consumer = consumerClientCache.getWithLoad(partition).get 84 | val offsets = partition.offsets(consumer) 85 | val safeOffset = if (localState.offset < offsets.min) offsets.min else offsets.max 86 | val state = State(safeOffset, localState.timestamp) 87 | blockingSetState(partitionKey, state, receiver.receiverConfigs) 88 | } 89 | 90 | // Wait for the error state to resolve 91 | case utp: UnknownTopicOrPartitionException => { 92 | logger.error("Wait for error to resolve.", utp) 93 | Thread.sleep(receiver.receiverConfigs.receiverRestIntervalOnFail) 94 | } 95 | 96 | // Update the partition->leader mapping 97 | case lna: LeaderNotAvailableException => { 98 | logger.error(s"Leader not available for partition: $partition. " + 99 | s"Will attempt fetch during next iteration", lna) 100 | 101 | // As part of handling this error, we are just removing the SimpleConsumer client associated 102 | // with the Partition. During the next fetch cycle, as part of re-populating the cache, the 103 | // Leader for the Partition will also be computed. 104 | consumerClientCache.remove(partition) 105 | } 106 | 107 | // Restart receiver, for any other exception 108 | case ke: KafkaException => receiver.restart(s"$ke. Restarting receiver") 109 | } 110 | 111 | Outcome.FetchFailure 112 | } 113 | 114 | /** 115 | * Continuously attempt to set the state 116 | * 117 | * @param key Key of the [[Partition]] which needs state update 118 | * @param state New [[State]] to set 119 | * @param receiverConfigs Configs for the receiver 120 | */ 121 | def blockingSetState(key: String, 122 | state: State, 123 | receiverConfigs: ReceiverConfigs) 124 | : Unit = { 125 | Utils.doActionWithRetry[(String, State)]( 126 | { 127 | receiverConfigs.stateController.setState(key, state) 128 | }, (f: Throwable) => { 129 | logger.error("StateController could not connect to state storage system. Will retry.", f) 130 | Thread.sleep(receiverConfigs.receiverRestIntervalOnFail) 131 | }, 132 | -1 133 | ) 134 | } 135 | 136 | } 137 | -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/kafka/common/Outcome.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.common 34 | 35 | /** 36 | * Various outcomes of a fetch operation 37 | */ 38 | object Outcome extends Enumeration { 39 | type Outcome = Value 40 | val NothingToFetch, FetchSuccess, FetchFailure, NoOutcome = Value 41 | } -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/kafka/common/State.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.common 34 | 35 | import org.json4s.jackson.JsonMethods._ 36 | import org.json4s.jackson.Serialization 37 | import org.json4s.jackson.Serialization.write 38 | import org.json4s.{DefaultFormats, NoTypeHints, _} 39 | 40 | /** 41 | * State of the system, defined locally for the receiver or by the stateController 42 | * 43 | * @param offset Kafka partition offset 44 | * @param timestamp Time of consumption 45 | */ 46 | case class State(offset: Long, timestamp: Long) { 47 | implicit val formats = Serialization.formats(NoTypeHints) 48 | 49 | override def toString = write(this) 50 | } 51 | 52 | /** 53 | * Convert a state string of the format {"offset":100,"timestamp":1432323232} to a State object 54 | */ 55 | object State { 56 | implicit lazy val formats = DefaultFormats 57 | 58 | def apply(jsonStr: String): State = { 59 | parse(jsonStr).extract[State] 60 | } 61 | } 62 | -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/kafka/common/WrappedMessage.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.common 34 | 35 | /** 36 | * Wrapper class for storing message payload and offset details 37 | * 38 | * @param topic The underlying topic to which the messages belong to 39 | * @param partition The kafka partition from which the messages were fetched 40 | * @param partitionStateKey The key used by the StateController to store state 41 | * @param payload Byte array comprising the message 42 | * @param offset Current offset 43 | * @param nextOffset Offset for the next message in line 44 | * @param batchEndOffset Offset after the last message of the Kafka fetch that this WrappedMessage was created from 45 | */ 46 | case class WrappedMessage(topic: String, 47 | partition: Int, 48 | partitionStateKey: String, 49 | payload: Array[Byte], 50 | offset: Long, 51 | nextOffset: Long, 52 | batchEndOffset: Long) 53 | extends Serializable 54 | -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/kafka/controllers/InMemoryStateController.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.controllers 34 | 35 | import com.groupon.dse.kafka.common.State 36 | import com.groupon.dse.kafka.partition.Partition 37 | 38 | class InMemoryStateController extends StateController { 39 | 40 | private lazy val stateMap = scala.collection.mutable.HashMap.empty[String, State] 41 | 42 | /** 43 | * Generate the key based on the provided seed 44 | * 45 | * @return Key to store state information 46 | */ 47 | override def generateStateKey(seed: Partition): String = seed.partitionIdentifier 48 | 49 | /** 50 | * Always return true since there is no outbound connection to outside systems 51 | * 52 | * @return true 53 | */ 54 | override def isConnected: Boolean = true 55 | 56 | /** 57 | * Given a key, obtain the state information 58 | * 59 | * @param key Key to fetch state 60 | * @return State information 61 | */ 62 | @throws[KeyDoesNotExistException] 63 | override def getState(key: String): State = { 64 | if (!isKeyPresent(key)) 65 | throw new KeyDoesNotExistException(key + " does not exist") 66 | stateMap(key) 67 | } 68 | 69 | /** 70 | * Check if the state key is already present 71 | * 72 | * @param key Key to store state 73 | * @return Boolean to indicate whether the key is already present or not 74 | */ 75 | override def isKeyPresent(key: String): Boolean = stateMap.contains(key) 76 | 77 | /** 78 | * Given the key, store appropriate state in HashMap 79 | * 80 | * @param key Key to store state 81 | * @param value State information 82 | * @return Tuple containing the key and updated [[State]] 83 | */ 84 | override def setState(key: String, value: State): (String, State) = { 85 | stateMap += (key -> value) 86 | (key, value) 87 | } 88 | 89 | /** 90 | * Remove all the elements from HashMap 91 | */ 92 | override def close(): Unit = stateMap.clear() 93 | } 94 | -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/kafka/controllers/StateController.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.controllers 34 | 35 | import com.groupon.dse.kafka.common.{State, WrappedMessage} 36 | import com.groupon.dse.kafka.partition.Partition 37 | import org.apache.spark.rdd.RDD 38 | 39 | /** 40 | * A [[StateController]] stores the application's consumed offsets to some storage system. Implementations should be 41 | * serializable, as the [[StateController]] can exist on both the driver and the executors. 42 | * 43 | * Any exceptions thrown by the underlying storage system should either be caught and handled, or a 44 | * [[StateControllerConnectionException]] should be thrown. 45 | */ 46 | trait StateController extends Serializable { 47 | 48 | /** 49 | * Given a Kafka [[Partition]], generate a [[String]] that uniquely identifies this partition for this storage system 50 | * 51 | * @param seed [[Partition]] to generate a key for 52 | * @return a [[String]] that uniquely identifies this partition 53 | */ 54 | def generateStateKey(seed: Partition): String 55 | 56 | /** 57 | * For a key generated by [[generateStateKey()]], determine if this key is already present in this storage system 58 | * 59 | * @param key [[String]] that uniquely identifies a [[Partition]] 60 | * @return true if [[State]] for this key exists in this storage system; false otherwise 61 | */ 62 | def isKeyPresent(key: String): Boolean 63 | 64 | /** 65 | * Determine if this [[StateController]] is connected to the underlying storage system 66 | * 67 | * @return true if connected; false otherwise 68 | */ 69 | def isConnected: Boolean 70 | 71 | /** 72 | * Close the connection to the underlying storage system for this [[StateController]] 73 | */ 74 | def close(): Unit 75 | 76 | /** 77 | * For a key generated by [[generateStateKey()]], obtained the stored [[State]]. 78 | * 79 | * If [[State]] for this key does not exist, throws a [[KeyDoesNotExistException]] 80 | * 81 | * @param key [[String]] that uniquely identifies a [[Partition]] 82 | * @return [[State]] for the [[Partition]] represented by the key 83 | */ 84 | def getState(key: String): State 85 | 86 | /** 87 | * Set the [[State]] for a key, which should uniquely identify a [[Partition]]. 88 | * 89 | * If the key already exists, [[setState()]] should overwrite the existing [[State]]. 90 | * 91 | * @param key [[String]] that uniquely identifies a [[Partition]] 92 | * @param value new [[State]] to store 93 | * @return a tuple of the key and the newly set [[State]] 94 | */ 95 | def setState(key: String, value: State): (String, State) 96 | 97 | /** 98 | * Set the [[State]] by using the [[WrappedMessage]] RDD. 99 | * 100 | * This method is to be used only in blocking mode. 101 | * 102 | * In blocking mode, the assumption is that every partition in an RDD uniquely corresponds to a Kafka partition. In 103 | * other words, the number of partitions in an RDD will always be <= the number of Kafka partitions we are consuming 104 | * from, and no two RDD partitions will contain data from the same Kafka partition. With this assumption, it's 105 | * possible to just take the first [[WrappedMessage]] of each partition and use the `batchEndOffset` field to 106 | * set the state for this RDD. 107 | * 108 | * @param rdd The RDD comprising of a list of [[WrappedMessage]] object, used to compute latest state 109 | * @return (partition key, [[State]]) tuple with the latest state 110 | */ 111 | def setState(rdd: RDD[WrappedMessage]): Seq[(String, State)] = { 112 | val currentTimeMillis = System.currentTimeMillis() 113 | val partitionAndOffset = rdd.mapPartitions(partition => { 114 | if (partition.hasNext) { 115 | val wm = partition.next() 116 | Iterator((wm.partitionStateKey, wm.batchEndOffset)) 117 | } else { 118 | Iterator.empty 119 | } 120 | }).reduceByKeyLocally((offsetA, offsetB) => math.max(offsetA, offsetB)) 121 | val partitionAndState = partitionAndOffset.map({ case (partitionStateKey, batchEndOffset) => 122 | (partitionStateKey, State(batchEndOffset, currentTimeMillis)) 123 | }) 124 | 125 | try { 126 | partitionAndState.foreach({ 127 | case (partitionStateKey, state) => setState(partitionStateKey, state) 128 | }) 129 | } catch { 130 | case sc: StateControllerConnectionException => throw sc 131 | } 132 | 133 | partitionAndState.toSeq 134 | } 135 | } 136 | 137 | case class KeyDoesNotExistException(message: String) extends Exception(message) 138 | 139 | case class StateControllerConnectionException(message: String) extends Exception(message) 140 | -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/kafka/controllers/StateControllerBuilder.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.controllers 34 | 35 | import java.util.Properties 36 | 37 | import com.groupon.dse.configs.AppConfigs 38 | import com.groupon.dse.configs.AppConfigs.InvalidConfigException 39 | 40 | object StateControllerBuilder { 41 | 42 | def apply(properties: Properties): StateController = { 43 | val stateControllerType = properties.getProperty( 44 | AppConfigs.StateControllerType._1, 45 | AppConfigs.StateControllerType._2) 46 | 47 | stateControllerType match { 48 | case "ZOOKEEPER" => { 49 | AppConfigs.validate(properties, 50 | Array(AppConfigs.ZookeeperStateControllerConnect._1, AppConfigs.ZookeeperStateControllerRoot._1)) 51 | 52 | val connectionString = properties.getProperty(AppConfigs.ZookeeperStateControllerConnect._1) 53 | 54 | val rootPath = properties.getProperty(AppConfigs.ZookeeperStateControllerRoot._1, 55 | AppConfigs.ZookeeperStateControllerRoot._2) 56 | 57 | val connTimeout = properties.getProperty(AppConfigs.ZookeeperStateControllerConnTimeoutMs._1, 58 | AppConfigs.ZookeeperStateControllerConnTimeoutMs._2).toInt 59 | 60 | val sessionTimeout = properties.getProperty(AppConfigs.ZookeeperStateControllerSessionTimeoutMs._1, 61 | AppConfigs.ZookeeperStateControllerSessionTimeoutMs._2).toInt 62 | 63 | new ZookeeperStateController(connectionString, rootPath, connTimeout, sessionTimeout) 64 | } 65 | 66 | case "MEMORY" => { 67 | val startOffset = properties.getProperty( 68 | AppConfigs.TopicStartOffset._1, 69 | AppConfigs.TopicStartOffset._2).toInt 70 | 71 | val isBlocking = AppConfigs.validatedBooleanConfig( 72 | properties, 73 | AppConfigs.TopicsEnableBlockingConsumption._1, 74 | AppConfigs.TopicsEnableBlockingConsumption._2) 75 | 76 | if (startOffset != -1) { 77 | throw InvalidConfigException("In-Memory StateController requires topic.start.offset to be set to -1") 78 | } 79 | 80 | if (isBlocking) { 81 | throw InvalidConfigException("Blocking consumption mode not supported for In-Memory StateController. " + 82 | "Set 'topic.consumption.blocking' to false.") 83 | } 84 | new InMemoryStateController 85 | } 86 | 87 | case _ => throw UnSupportedStateControllerException("Could not find StateController implementation") 88 | } 89 | } 90 | 91 | case class UnSupportedStateControllerException(message: String) extends Exception(message) 92 | 93 | } 94 | -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/kafka/controllers/ZookeeperStateController.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.controllers 34 | 35 | import com.groupon.dse.configs.AppConfigs 36 | import com.groupon.dse.kafka.common.State 37 | import com.groupon.dse.kafka.partition.Partition 38 | import com.groupon.dse.zookeeper.ZkClientBuilder 39 | import kafka.utils.ZkUtils 40 | import org.I0Itec.zkclient.ZkClient 41 | import org.I0Itec.zkclient.exception.ZkException 42 | import org.slf4j.LoggerFactory 43 | 44 | /** 45 | * Controller for Zookeeper based state storage 46 | * 47 | * @param zkEndpoint Zookeeper connection endpoint 48 | * @param pathPrefix Base path in zookeeper to store state information 49 | */ 50 | class ZookeeperStateController( 51 | zkEndpoint: String, 52 | pathPrefix: String, 53 | connectionTimeout: Int, 54 | sessionTimeout: Int) 55 | extends StateController { 56 | 57 | private val logger = LoggerFactory.getLogger(getClass) 58 | private var connected: Boolean = true 59 | @transient private var zkClient: ZkClient = null 60 | 61 | /** 62 | * Use default values for retryInterval and maxRetries when none provide 63 | * 64 | * @return [[ZookeeperStateController]] instance 65 | */ 66 | def this(zkEndpoint: String, pathPrefix: String) = 67 | this( 68 | zkEndpoint, 69 | pathPrefix, 70 | AppConfigs.ZookeeperStateControllerConnTimeoutMs._2.toInt, 71 | AppConfigs.ZookeeperStateControllerSessionTimeoutMs._2.toInt) 72 | 73 | /** 74 | * String representation of the ZookeeperStateController object 75 | * 76 | * @return String representation 77 | */ 78 | override def toString: String = s"ZookeeperStateController = [zkEndpoint: $zkEndpoint, zkRoot: $pathPrefix]" 79 | 80 | /** 81 | * Given a key, obtain the state information 82 | * 83 | * @param key Key to fetch state 84 | * @return State information 85 | */ 86 | @throws[KeyDoesNotExistException] 87 | @throws[StateControllerConnectionException] 88 | override def getState(key: String): State = { 89 | if (!isKeyPresent(key)) 90 | throw new KeyDoesNotExistException(key + " does not exist") 91 | try { 92 | State(ZkUtils.readData(getZkClient, key)._1) 93 | } catch { 94 | case zk: ZkException => throwControllerException("Zookeeper connection exception while getting state", zk) 95 | } 96 | } 97 | 98 | /** 99 | * Check if the state key is already present 100 | * 101 | * @param key Key to store state 102 | * @return Boolean to indicate whether the key is already present or not 103 | */ 104 | @throws[StateControllerConnectionException] 105 | override def isKeyPresent(key: String): Boolean = { 106 | try { 107 | ZkUtils.pathExists(getZkClient, key) 108 | } catch { 109 | case zk: ZkException => throwControllerException("Zookeeper connection exception while checking for key", zk) 110 | } 111 | } 112 | 113 | /** 114 | * Given the key, store appropriate state in zookeeper 115 | * 116 | * @param key Key to store state 117 | * @param value State information 118 | * @return Tuple containing the key and updated [[State]] 119 | */ 120 | @throws[StateControllerConnectionException] 121 | override def setState(key: String, value: State): (String, State) = { 122 | try { 123 | ZkUtils.updatePersistentPath(getZkClient, key, value.toString()) 124 | (key, value) 125 | } catch { 126 | case zk: ZkException => throwControllerException("Zookeeper connection exception while setting state", zk) 127 | } 128 | } 129 | 130 | /** 131 | * Throw a new [[StateControllerConnectionException]] 132 | * 133 | * @param msg Message to log 134 | * @param zk ZkException object 135 | */ 136 | private def throwControllerException(msg: String, zk: ZkException) = { 137 | logger.error(msg, zk) 138 | close 139 | throw StateControllerConnectionException(zk.getMessage) 140 | } 141 | 142 | /** 143 | * Close any resources opened for the controller 144 | */ 145 | override def close = { 146 | try { 147 | connected = false 148 | zkClient.close() 149 | } catch { 150 | case e: Exception => logger.error("Failed to close the ZookeeperStateController zkClient", e) 151 | } 152 | } 153 | 154 | /** 155 | * Check if the zkClient valid else create a new one 156 | * 157 | * @return ZkClient instance 158 | */ 159 | private def getZkClient: ZkClient = { 160 | if (zkClient == null || !connected) { 161 | logger.debug("Creating new zkClient.") 162 | zkClient = ZkClientBuilder(zkEndpoint, connectionTimeout, sessionTimeout) 163 | connected = true 164 | } 165 | zkClient 166 | } 167 | 168 | /** 169 | * Generate the key based on the provided seed 170 | * 171 | * @param seed Seed to generate key 172 | * @return Key to store state information 173 | */ 174 | override def generateStateKey(seed: Partition): String = pathPrefix + "/" + seed.partitionIdentifier 175 | 176 | /** 177 | * Indicate if the controller is still connected to Zookeeper 178 | * 179 | * @return Return true if connected 180 | */ 181 | override def isConnected: Boolean = connected 182 | 183 | } -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/kafka/partition/Partition.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.partition 34 | 35 | import com.groupon.dse.kafka.common.KafkaException.LeaderNotAvailableException 36 | import com.groupon.dse.kafka.common.{KafkaException, WrappedMessage} 37 | import kafka.consumer.SimpleConsumer 38 | import org.I0Itec.zkclient.ZkClient 39 | import org.I0Itec.zkclient.exception.ZkException 40 | 41 | /** 42 | * Representation of a single Kafka partition 43 | * 44 | * @param topic Name of the topic in the partition 45 | * @param partitionNum Partition identifier 46 | */ 47 | abstract class Partition(val topic: String, val partitionNum: Int) 48 | extends Serializable { 49 | 50 | /** 51 | * Obtain the min and max offset for a [[Partition]] 52 | * 53 | * @param consumer Kafka consumer client 54 | * @return [[Offsets]] object with min and max offsets 55 | */ 56 | @throws[KafkaException] 57 | def offsets(consumer: SimpleConsumer): Offsets 58 | 59 | /** 60 | * Provide the N latest valid offset values 61 | * 62 | * @param count Maximum valid offset values to be fetched 63 | * @param consumer Kafka consumer client 64 | * @return Sequence of the N latest valid offsets 65 | */ 66 | @throws[KafkaException] 67 | def offsetRange(count: Int, consumer: SimpleConsumer): Seq[Long] 68 | 69 | /** 70 | * Fetch messages for the particular [[Partition]] 71 | * 72 | * @param startOffset Offset to start fetching from 73 | * @param fetchSize Data to fetch in a single api call 74 | * @param key Partition key associated with the message set 75 | * @param consumer Kafka consumer client 76 | * @return Iterable list of [[WrappedMessage]] 77 | */ 78 | @throws[KafkaException] 79 | def fetchMessages(startOffset: Long, 80 | fetchSize: Int, 81 | key: String, 82 | consumer: SimpleConsumer) 83 | : Iterable[WrappedMessage] 84 | 85 | /** 86 | * Obtain the [[Leader]] for the current partition 87 | * 88 | * @param zkClient Client to interact with the Kafka Zookeeper 89 | * @return [[Leader]] for the current partition 90 | */ 91 | @throws[LeaderNotAvailableException] 92 | @throws[ZkException] 93 | def leader(zkClient: ZkClient): Leader 94 | 95 | /** 96 | * Unique [[Partition]] identifier 97 | * 98 | * @return [[Partition]] identifier 99 | */ 100 | def partitionIdentifier: String 101 | 102 | /** 103 | * String representation of [[Partition]] class 104 | * 105 | * @return String representing the [[Partition]] class 106 | */ 107 | override def toString: String = s"[$topic:$partitionNum]" 108 | 109 | /** 110 | * Check equality of two objects 111 | * 112 | * @param that Object whose equality with current [[Partition]] object needs to be checked 113 | * @return Result of object comparison 114 | */ 115 | override def equals(that: Any): Boolean = that match { 116 | case that: Partition => that.canEqual(this) && that.hashCode == this.hashCode 117 | case _ => false 118 | } 119 | 120 | /** 121 | * HashCode for the [[Partition]] object 122 | * 123 | * @return HashCode value 124 | */ 125 | override def hashCode: Int = partitionIdentifier.hashCode 126 | 127 | /** 128 | * Check if given object is of type [[Partition]] 129 | * 130 | * @param o Other object 131 | * @return Result of instance type check 132 | */ 133 | def canEqual(o: Any): Boolean = o.isInstanceOf[Partition] 134 | } 135 | 136 | /** 137 | * Wrapper class for storing the min and max offset for a [[Partition]] 138 | * 139 | * @param min Min available offset 140 | * @param max Max available offset 141 | */ 142 | case class Offsets( 143 | min: Long, 144 | max: Long) 145 | extends Serializable { 146 | 147 | override def toString: String = s"[$min, $max]" 148 | } 149 | 150 | /** 151 | * Broker currently serving the [[Partition]] 152 | * 153 | * @param host 154 | * @param port 155 | */ 156 | case class Leader( 157 | host: String, 158 | port: Int) 159 | extends Serializable { 160 | 161 | override def toString: String = s"[$host:$port]" 162 | } -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/kafka/policy/FetchPolicy.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.policy 34 | 35 | import com.groupon.dse.kafka.common.State 36 | 37 | /** 38 | * Define the policy to consume Kafka data 39 | */ 40 | abstract class FetchPolicy( 41 | val fetchSize: Int, 42 | val startOffset: Long) 43 | extends Serializable { 44 | 45 | def canFetch(local: State, global: State): Boolean 46 | } -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/kafka/policy/FetchPolicyBuilder.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.policy 34 | 35 | import java.util.Properties 36 | 37 | import com.groupon.dse.configs.AppConfigs 38 | import org.slf4j.LoggerFactory 39 | 40 | /** 41 | * Construct the appropriate [[FetchPolicy]] object based on user params 42 | */ 43 | object FetchPolicyBuilder { 44 | 45 | private val logger = LoggerFactory.getLogger(getClass) 46 | 47 | def apply(properties: Properties): FetchPolicy = { 48 | 49 | val fetchPolicyType = properties.getProperty( 50 | AppConfigs.TopicConsumptionPolicy._1, 51 | AppConfigs.TopicConsumptionPolicy._2) 52 | 53 | val offsetThreshold = properties.getProperty( 54 | AppConfigs.TopicConsumptionOffsetThreshold._1, 55 | AppConfigs.TopicConsumptionOffsetThreshold._2).toLong 56 | 57 | val timeThreshold = properties.getProperty( 58 | AppConfigs.TopicConsumptionTimeThresholdMs._1, 59 | AppConfigs.TopicConsumptionTimeThresholdMs._2).toLong 60 | 61 | val fetchSize = properties.getProperty( 62 | AppConfigs.TopicFetchSizeBytes._1, 63 | AppConfigs.TopicFetchSizeBytes._2).toInt 64 | 65 | val startOffset = properties.getProperty( 66 | AppConfigs.TopicStartOffset._1, 67 | AppConfigs.TopicStartOffset._2).toLong 68 | 69 | val isBlocking = AppConfigs.validatedBooleanConfig( 70 | properties, 71 | AppConfigs.TopicsEnableBlockingConsumption._1, 72 | AppConfigs.TopicsEnableBlockingConsumption._2) 73 | 74 | if (isBlocking) { 75 | // If user has chosen blocking consumption and has also explicitly set 76 | // fetch policy related configs, notify user that the later will be ignored 77 | if (properties.containsKey(AppConfigs.TopicConsumptionPolicy)) { 78 | logger.warn("Fetch policy configs will be ignored if consumption type is blocking") 79 | } 80 | return fetchPolicyForBlockingConsumption(fetchSize, startOffset) 81 | } 82 | 83 | fetchPolicyByType( 84 | fetchPolicyType, 85 | offsetThreshold, 86 | timeThreshold, 87 | fetchSize, 88 | startOffset 89 | ) 90 | } 91 | 92 | /** 93 | * FetchPolicy that should be used by default if consumption is blocking 94 | * 95 | * @param fetchSize Data to fetch, from Kafka, in a single api call 96 | * @param startOffset Offset value to start consuming from (-1: Max, -2: Min, Other: Actual offset value) 97 | * @return [[FetchPolicy]] instance 98 | */ 99 | def fetchPolicyForBlockingConsumption(fetchSize: Int, startOffset: Long) 100 | : FetchPolicy = new OffsetBasedFetchPolicy( 101 | AppConfigs.TopicConsumptionOffsetThreshold._2.toInt, 102 | fetchSize, 103 | startOffset) 104 | 105 | /** 106 | * Build appropriate [[FetchPolicy]] based on requested type 107 | * 108 | * @param fetchPolicyType Fetch policy type expressed as String 109 | * @param offsetThreshold Offset threshold for FetchPolicy 110 | * @param timeThreshold Time threshold for FetchPolicy 111 | * @param fetchSize Data to fetch, from Kafka, in a single api call 112 | * @param startOffset Offset value to start consuming from (-1: Max, -2: Min, Other: Actual offset value) 113 | * @return [[FetchPolicy]] instance 114 | */ 115 | def fetchPolicyByType( 116 | fetchPolicyType: String, 117 | offsetThreshold: Long, 118 | timeThreshold: Long, 119 | fetchSize: Int, 120 | startOffset: Long 121 | ) 122 | : FetchPolicy = fetchPolicyType match { 123 | case "TIME_AND_OFFSET" => new TimeAndOffsetBasedFetchPolicy( 124 | offsetThreshold, 125 | timeThreshold, 126 | fetchSize, 127 | startOffset) 128 | 129 | case "OFFSET" => new OffsetBasedFetchPolicy( 130 | offsetThreshold, 131 | fetchSize, 132 | startOffset) 133 | 134 | case "TIME" => new TimeBasedFetchPolicy( 135 | timeThreshold, 136 | fetchSize, 137 | startOffset) 138 | 139 | case _ => throw InvalidFetchPolicyException("Valid policies are: TIME_AND_OFFSET, OFFSET, TIME") 140 | } 141 | 142 | case class InvalidFetchPolicyException(message: String) extends Exception(message) 143 | 144 | } 145 | -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/kafka/policy/OffsetBasedFetchPolicy.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.policy 34 | 35 | import com.groupon.dse.kafka.common.State 36 | 37 | /** 38 | * Fetch policy purely defined by consumed offsets 39 | * 40 | * @param offsetThreshold Offset threshold to enable fetch 41 | * @param fetchSize Kafka fetch size 42 | * @param startOffset Kafka start offset 43 | */ 44 | class OffsetBasedFetchPolicy( 45 | offsetThreshold: Long, 46 | fetchSize: Int, 47 | startOffset: Long) 48 | extends FetchPolicy(fetchSize, startOffset) { 49 | 50 | override def toString: String = s"OffsetBasedFetchPolicy = [" + 51 | s"offsetThreshold: $offsetThreshold," + 52 | s"fetchSize: $fetchSize, " + 53 | s"startOffset: $startOffset]" 54 | 55 | override def canFetch(local: State, global: State): Boolean = 56 | (local.offset - global.offset) <= offsetThreshold 57 | } 58 | -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/kafka/policy/TimeAndOffsetBasedFetchPolicy.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.policy 34 | 35 | import com.groupon.dse.kafka.common.State 36 | 37 | /** 38 | * Fetch policy defined by a combination of offsets and last consumed time 39 | * 40 | * @param offsetThreshold Offset threshold to enable fetch 41 | * @param timeThreshold Time threshold to enable fetch 42 | * @param fetchSize Kafka fetch size 43 | * @param startOffset Kafka start offset 44 | */ 45 | class TimeAndOffsetBasedFetchPolicy( 46 | offsetThreshold: Long, 47 | timeThreshold: Long, 48 | fetchSize: Int, 49 | startOffset: Long) 50 | extends FetchPolicy(fetchSize, startOffset) { 51 | 52 | override def toString: String = s"TimeAndOffsetBasedFetchPolicy = [" + 53 | s"offsetThreshold: $offsetThreshold, " + 54 | s"timeThreshold: $timeThreshold," + 55 | s"fetchSize: $fetchSize," + 56 | s"startOffset: $startOffset]" 57 | 58 | override def canFetch(local: State, global: State): Boolean = 59 | (local.timestamp - global.timestamp) <= timeThreshold || (local.offset - global.offset) <= offsetThreshold 60 | } 61 | -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/kafka/policy/TimeBasedFetchPolicy.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.policy 34 | 35 | import com.groupon.dse.kafka.common.State 36 | 37 | /** 38 | * Fetch policy purely defined by last consumption time 39 | * 40 | * @param timeThreshold Time threshold to enable fetch 41 | * @param fetchSize Kafka fetch size 42 | * @param startOffset Kafka start offset 43 | */ 44 | class TimeBasedFetchPolicy( 45 | timeThreshold: Long, 46 | fetchSize: Int, 47 | startOffset: Long) 48 | extends FetchPolicy(fetchSize, startOffset) { 49 | 50 | override def toString: String = s"TimeBasedFetchPolicy = [" + 51 | s"timeThreshold: $timeThreshold," + 52 | s"fetchSize: $fetchSize, " + 53 | s"startOffset: $startOffset]" 54 | 55 | override def canFetch(local: State, global: State): Boolean = 56 | (local.timestamp - global.timestamp) <= timeThreshold 57 | } -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/kafka/topic/HdfsJsonTopicFetcher.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.topic 34 | 35 | import org.apache.hadoop.conf.Configuration 36 | import org.apache.hadoop.fs.{FileSystem, Path} 37 | import org.slf4j.LoggerFactory 38 | 39 | import scala.io.Source 40 | 41 | /** 42 | * This [[TopicFetcher]] implementation reads the contents of a JSON file stored in HDFS. 43 | This specific implementation requires the JSON file to adhere to a specific schema as indicated below: 44 | { 45 | "topics": [ 46 | { 47 | "name": "topic1", 48 | "metadata": { 49 | "type": "avro" 50 | } 51 | }, 52 | { 53 | "name": "topic2" 54 | } 55 | ] 56 | } 57 | The JSON file should contain a REQUIRED field called "topics" that contains an array of topic entries. Each entry should 58 | again contain a REQUIRED "name" field indicating the name of the desired topic. The "metadata" field is optional 59 | and is not used in the current implementation. However users can create their own [[TopicFetcher]] implementations to 60 | use the "metadata" value to determine valid topics to fetch from Kafka. 61 | * @param topicsLocation The location of the topic file in HDFS. 62 | */ 63 | class HdfsJsonTopicFetcher(topicsLocation: String) 64 | extends TopicFetcher with Serializable { 65 | 66 | lazy val path = new Path(topicsLocation) 67 | lazy val config = new Configuration() 68 | lazy val fileSystem = FileSystem.get(config) 69 | private val logger = LoggerFactory.getLogger(getClass) 70 | 71 | /** 72 | * Fetch the topics from the JSON file in the given HDFS location. 73 | * @return The list of [[TopicAndMetadata]] from HDFS 74 | */ 75 | override def fetchTopics: Seq[TopicAndMetadata] = { 76 | try { 77 | val contents = readTopicFile 78 | val topicList = TopicUtils.extractTopicListFromJson(contents) 79 | topicList.toSeq 80 | } catch { 81 | case e: Exception => { 82 | logger.error("Exception occurred while fetching topics from HDFS : ", e) 83 | Seq.empty 84 | } 85 | } 86 | } 87 | 88 | /** 89 | * Helper method to read the topic file from HDFS and return the contents as a String 90 | * @return The contents of the HDFS file 91 | */ 92 | def readTopicFile: String = { 93 | val inputStream = fileSystem.open(path) 94 | try { 95 | val data = Source.fromInputStream(inputStream).getLines().mkString("\n") 96 | data 97 | } finally { 98 | inputStream.close() 99 | } 100 | } 101 | 102 | } 103 | -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/kafka/topic/HttpJsonTopicFetcher.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.topic 34 | 35 | import org.slf4j.LoggerFactory 36 | import play.api.libs.ws.ning.NingWSClient 37 | 38 | import scala.concurrent.Await 39 | import scala.concurrent.ExecutionContext.Implicits.global 40 | import scala.concurrent.duration._ 41 | import scala.util.{Failure, Success} 42 | 43 | /** 44 | * This [[TopicFetcher]] implementation reads the contents of a JSON file obtained from a HTTP endpoint. 45 | This specific implementation requires the JSON file to adhere to a specific schema as indicated below: 46 | { 47 | "topics": [ 48 | { 49 | "name": "topic1", 50 | "metadata": { 51 | "type": "avro" 52 | } 53 | }, 54 | { 55 | "name": "topic2" 56 | } 57 | ] 58 | } 59 | The JSON file should contain a REQUIRED field called "topics" that contains an array of topic entries. Each entry should 60 | again contain a REQUIRED "name" field indicating the name of the desired topic. The "metadata" field is optional 61 | and is not used in the current implementation. However users can create their own [[TopicFetcher]] implementations to 62 | use the "metadata" value to determine valid topics to fetch from Kafka. 63 | * @param topicsUrlEndpoint THe HTTP endpoint to fetch the JSON from 64 | */ 65 | class HttpJsonTopicFetcher(topicsUrlEndpoint: String) 66 | extends TopicFetcher with Serializable { 67 | lazy val builder = new com.ning.http.client.AsyncHttpClientConfig.Builder() 68 | lazy val client = new NingWSClient(builder.build()) 69 | private val logger = LoggerFactory.getLogger(getClass) 70 | 71 | /** 72 | * Fetch the [[TopicAndMetadata]] list from the specified URL 73 | * @return the list of [[TopicAndMetadata]] 74 | */ 75 | override def fetchTopics: Seq[TopicAndMetadata] = { 76 | try { 77 | TopicUtils.extractTopicListFromJson(getContentFromUrl.getOrElse("")) 78 | } catch { 79 | case e: Exception => { 80 | logger.error("Exception occurred while fetching topics from the provided url : ", e) 81 | Seq.empty 82 | } 83 | } 84 | } 85 | 86 | /** 87 | * Fetches the content from the provided URL asynchronously. Blocks until the response is available for further processing. 88 | * The url response is a [[scala.concurrent.Future]] object which implies that the results would be available at some point in the future. 89 | * The max wait time for the response is currently set to 2 seconds but it would return earlier if the response is available. 90 | * @return The String representation of the JSON fetched from the URL. 91 | */ 92 | def getContentFromUrl: Option[String] = { 93 | val response = client.url(topicsUrlEndpoint).get().map { resp => 94 | resp.json.toString 95 | } 96 | val jsonString = Await.ready(response, 2.seconds).value.get //TODO: Change this duration. Should it be user provided? 97 | 98 | val finalResponse = jsonString match { 99 | case Success(str) => Some(str) 100 | case Failure(exc) => { 101 | logger.error("Error fetching json from the provided URL : ", exc) 102 | None 103 | } 104 | } 105 | finalResponse 106 | } 107 | } 108 | -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/kafka/topic/LocalTopicFetcher.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.topic 34 | 35 | /** 36 | * Topic fetcher that retrieves the static topic list from the Application configs provided in the driver. 37 | * The blacklist regex provided in the configs is applied to the topics list to filter the unwanted topics. 38 | * Does not support topic metadata 39 | * @param topics The list of topics provided in the config file 40 | * @param topicBlackListRegex The topic blacklist regex 41 | */ 42 | class LocalTopicFetcher( 43 | topics: Seq[String], 44 | topicBlackListRegex: Seq[String]) 45 | extends TopicFetcher with Serializable { 46 | 47 | val blacklistTopics = scala.collection.mutable.ListBuffer.empty[String] 48 | topicBlackListRegex.foreach(tbr => blacklistTopics.appendAll(topics.filter(_.matches(tbr)))) 49 | 50 | val validTopics = topics.filterNot(t => blacklistTopics.contains(t)) 51 | 52 | val topicsList = validTopics.map(topic => TopicAndMetadata(topic)) 53 | 54 | /** 55 | * Fetch the topics from the local properties object 56 | * @return The list of [[TopicAndMetadata]] from the config file 57 | */ 58 | override def fetchTopics: Seq[TopicAndMetadata] = topicsList 59 | } -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/kafka/topic/TopicFetcher.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.topic 34 | 35 | /** 36 | * Trait to fetch the topics from the user specified location. 37 | */ 38 | trait TopicFetcher { 39 | 40 | def fetchTopics: Seq[TopicAndMetadata] 41 | } 42 | 43 | case class TopicAndMetadata(name: String, metadata: Map[String, AnyRef] = Map.empty) -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/kafka/topic/TopicFetcherBuilder.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.topic 34 | 35 | import java.util.Properties 36 | 37 | import com.groupon.dse.configs.AppConfigs 38 | import org.slf4j.LoggerFactory 39 | 40 | /** 41 | * Helper to build a [[TopicFetcher]] object based on the provided configs 42 | */ 43 | object TopicFetcherBuilder { 44 | 45 | private val logger = LoggerFactory.getLogger(getClass) 46 | 47 | def apply(properties: Properties): TopicFetcher = { 48 | 49 | val fetcherType = properties.getProperty( 50 | AppConfigs.TopicFetcherType._1, 51 | AppConfigs.TopicFetcherType._2) 52 | 53 | val hdfsTopicSource = properties.getProperty( 54 | AppConfigs.HDFSTopicSource._1, 55 | AppConfigs.HDFSTopicSource._2) 56 | 57 | val httpTopicSource = properties.getProperty( 58 | AppConfigs.HTTPTopicSource._1, 59 | AppConfigs.HTTPTopicSource._2) 60 | 61 | val topicBlacklist = properties.getProperty( 62 | AppConfigs.TopicsBlackList._1, 63 | AppConfigs.TopicsBlackList._2).split(",").map(_.trim) 64 | 65 | fetcherType match { 66 | case "LOCAL" => { 67 | AppConfigs.validate(properties, Seq(AppConfigs.Topics._1)) 68 | val topics = properties.getProperty(AppConfigs.Topics._1).split(",").map(_.trim) 69 | new LocalTopicFetcher(topics, topicBlacklist) 70 | } 71 | 72 | case "HDFS" => { 73 | AppConfigs.validate(properties, Seq(AppConfigs.HDFSTopicSource._1)) 74 | new HdfsJsonTopicFetcher(hdfsTopicSource.trim) 75 | } 76 | 77 | case "HTTP" => { 78 | AppConfigs.validate(properties, Seq(AppConfigs.HTTPTopicSource._1)) 79 | new HttpJsonTopicFetcher(httpTopicSource) 80 | } 81 | case _ => throw InvalidTopicFetcherException("Valid topic fetchers are LOCAL, HDFS, HTTP") 82 | } 83 | } 84 | 85 | case class InvalidTopicFetcherException(message: String) extends Exception(message) 86 | 87 | } 88 | -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/kafka/topic/TopicUtils.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.topic 34 | 35 | import com.fasterxml.jackson.databind.ObjectMapper 36 | import com.fasterxml.jackson.module.scala.DefaultScalaModule 37 | 38 | /** 39 | * Helper object to extract the [[TopicAndMetadata]] from the provided JSON 40 | */ 41 | object TopicUtils { 42 | 43 | def extractTopicListFromJson(json: String): List[TopicAndMetadata] = { 44 | val mapper = new ObjectMapper() 45 | mapper.registerModule(DefaultScalaModule) 46 | //The top level "topics" key is mapped to a list of [[TopicAndMetadata]] objects 47 | val contentMap = mapper.readValue(json, classOf[Map[String, List[Map[String, AnyRef]]]]) 48 | 49 | val topics = contentMap.getOrElse("topics", List.empty) 50 | 51 | val topicList = topics.map(topicMap => { 52 | val topic = topicMap.getOrElse("name", "") 53 | val metadata = topicMap.getOrElse("metadata", Map.empty) 54 | TopicAndMetadata(topic.toString, metadata.asInstanceOf[Map[String, AnyRef]]) 55 | }).filterNot(_.name.isEmpty) 56 | topicList 57 | } 58 | } 59 | -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/spark/dstreams/KafkaLowLevelDStream.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.spark.dstreams 34 | 35 | import com.groupon.dse.configs.ReceiverConfigs 36 | import com.groupon.dse.kafka.common._ 37 | import org.apache.spark.SparkContext 38 | import org.apache.spark.storage.StorageLevel 39 | import org.apache.spark.streaming.StreamingContext 40 | import org.apache.spark.streaming.dstream.ReceiverInputDStream 41 | import org.apache.spark.streaming.receiver.Receiver 42 | import org.slf4j.LoggerFactory 43 | 44 | /** 45 | * DStream for consuming from Kafka 46 | * 47 | * @param ssc Current [[StreamingContext]] 48 | * @param receiverConfigs Configs for the receiver 49 | * @param receiverId Id of the current receiver 50 | * @param totalReceivers Total number of receivers 51 | * @param storageLevel [[StorageLevel]] for RDD's 52 | */ 53 | class KafkaLowLevelDStream(@transient ssc: StreamingContext, 54 | receiverConfigs: ReceiverConfigs, 55 | receiverId: Int, 56 | totalReceivers: Int, 57 | storageLevel: StorageLevel) 58 | extends ReceiverInputDStream[WrappedMessage](ssc) { 59 | 60 | override def getReceiver(): Receiver[WrappedMessage] = new KafkaLowLevelReceiver(ssc.sparkContext, 61 | receiverConfigs, 62 | receiverId, 63 | totalReceivers, 64 | storageLevel).asInstanceOf[Receiver[WrappedMessage]] 65 | } 66 | 67 | /** 68 | * Receiver for consuming from Kafka 69 | * 70 | * @param sc Current [[SparkContext]] 71 | * @param receiverConfigs Configs for the receiver 72 | * @param receiverId Id of the current receiver 73 | * @param totalReceivers Total number of receivers 74 | * @param storageLevel [[StorageLevel]] for RDD's 75 | */ 76 | class KafkaLowLevelReceiver(sc: SparkContext, 77 | val receiverConfigs: ReceiverConfigs, 78 | val receiverId: Int, 79 | val totalReceivers: Int, 80 | storageLevel: StorageLevel) 81 | extends Receiver[WrappedMessage](storageLevel) { 82 | 83 | private val logger = LoggerFactory.getLogger(classOf[KafkaLowLevelReceiver]) 84 | 85 | override def onStart(): Unit = { 86 | val workerThread = new KafkaLowLevelConsumer(this) 87 | workerThread.start() 88 | logger.info(s"Receiver with id $receiverId started") 89 | } 90 | 91 | override def onStop(): Unit = { 92 | logger.info(s"Shutting down receiver with id $receiverId") 93 | receiverConfigs.stateController.close 94 | } 95 | } -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/spark/listeners/MetricsListener.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.spark.listeners 34 | 35 | import org.apache.spark.groupon.metrics.UserMetricsSystem 36 | import org.apache.spark.scheduler.{SparkListener, SparkListenerExecutorRemoved, SparkListenerStageCompleted, SparkListenerTaskEnd} 37 | 38 | /** 39 | * [[SparkListener]] implementation that provides metrics about failures in Spark 40 | */ 41 | class MetricsListener extends SparkListener { 42 | private lazy val executorRemovedMeter = UserMetricsSystem.meter("baryon.executorRemoved.rate") 43 | private lazy val failedStagesMeter = UserMetricsSystem.meter("baryon.failedStages.rate") 44 | private lazy val failedTasksMeter = UserMetricsSystem.meter("baryon.failedTasks.rate") 45 | 46 | override def onExecutorRemoved(executorRemoved: SparkListenerExecutorRemoved): Unit = { 47 | executorRemovedMeter.mark() 48 | } 49 | 50 | override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { 51 | if (stageCompleted.stageInfo.failureReason.isDefined) { 52 | failedStagesMeter.mark() 53 | } 54 | } 55 | 56 | override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { 57 | if (!taskEnd.taskInfo.successful) { 58 | failedTasksMeter.mark() 59 | } 60 | } 61 | } 62 | -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/spark/listeners/StreamingMetricsListener.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.spark.listeners 34 | 35 | import org.apache.spark.groupon.metrics.UserMetricsSystem 36 | import org.apache.spark.streaming.scheduler.{StreamingListener, StreamingListenerBatchCompleted} 37 | 38 | /** 39 | * [[StreamingListener]] implementation that computes more detailed versions of streaming metrics provided by Spark 40 | */ 41 | class StreamingMetricsListener extends StreamingListener { 42 | private lazy val processingTimeHistogram = UserMetricsSystem.histogram("baryon.processingTime") 43 | private lazy val schedulingDelayHistogram = UserMetricsSystem.histogram("baryon.schedulingDelay") 44 | 45 | override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted): Unit = { 46 | if (batchCompleted.batchInfo.processingDelay.isDefined) { 47 | processingTimeHistogram.update(batchCompleted.batchInfo.processingDelay.get) 48 | } 49 | if (batchCompleted.batchInfo.schedulingDelay.isDefined) { 50 | schedulingDelayHistogram.update(batchCompleted.batchInfo.schedulingDelay.get) 51 | } 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/spark/plugins/ReceiverPlugin.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.spark.plugins 34 | 35 | import com.groupon.dse.kafka.common.WrappedMessage 36 | import org.apache.spark.rdd.RDD 37 | 38 | trait ReceiverPlugin extends Serializable { 39 | def execute(rdd: RDD[WrappedMessage]) 40 | } 41 | -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/util/Utils.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.util 34 | 35 | import scala.util.{Failure, Success, Try} 36 | 37 | /** 38 | * List of utility methods for Baryon 39 | */ 40 | object Utils { 41 | /** 42 | * Perform an action with bounded retries 43 | * 44 | * @param action Closure for the primary action to be performed 45 | * @param failureHandler Closure for the action to be performed on failure 46 | * @param availableRetries Total number of times the primary action should be retried 47 | * @tparam T Return type for primary action 48 | * @return Value returned by the primary action 49 | */ 50 | def doActionWithRetry[T](action: => T, failureHandler: Throwable => Unit, availableRetries: Int): T = { 51 | Try(action) match { 52 | case Success(ret) => ret 53 | case Failure(f) => { 54 | if (availableRetries == 0) throw f 55 | failureHandler(f) 56 | doActionWithRetry[T](action, failureHandler, availableRetries - 1) 57 | } 58 | } 59 | } 60 | } 61 | -------------------------------------------------------------------------------- /src/main/scala/com/groupon/dse/zookeeper/ZkClientBuilder.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.zookeeper 34 | 35 | import kafka.utils.ZKStringSerializer 36 | import org.I0Itec.zkclient.ZkClient 37 | import org.I0Itec.zkclient.serialize.ZkSerializer 38 | 39 | /** 40 | * Builder for [[ZkClient]] object 41 | */ 42 | object ZkClientBuilder { 43 | 44 | private lazy val DEFAULT_SERIALIZER: ZkSerializer = ZKStringSerializer 45 | 46 | /** 47 | * Create a new [[ZkClient]] object 48 | * 49 | * @param connectionString Zookeeper endpoint 50 | * @param connectionTimeOut [[ZkClient]] connectionTimeOut in milliseconds 51 | * @param sessionTimeOut [[ZkClient]] sessionTimeOut in milliseconds 52 | * @param zkSerializer [[ZkClient]] serializer 53 | * @return [[ZkClient]] object 54 | */ 55 | def apply(connectionString: String, 56 | connectionTimeOut: Int, 57 | sessionTimeOut: Int, 58 | zkSerializer: ZkSerializer = DEFAULT_SERIALIZER): ZkClient = { 59 | new ZkClient(connectionString, sessionTimeOut, connectionTimeOut, zkSerializer) 60 | } 61 | } 62 | 63 | -------------------------------------------------------------------------------- /src/test/scala/com/groupon/dse/configs/AppConfigsTest.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.configs 34 | 35 | import java.util.Properties 36 | 37 | import com.groupon.dse.configs.AppConfigs.{InvalidConfigException, MissingConfigException} 38 | import org.scalatest.FlatSpec 39 | 40 | class AppConfigsTest extends FlatSpec { 41 | 42 | "AppConfigs" should "have proper number of fields" in { 43 | val nonSyntheticFields = AppConfigs.getClass.getDeclaredFields.filter(f => !f.isSynthetic) 44 | assert(nonSyntheticFields.size == 29) // Always one more than the total attributes 45 | } 46 | 47 | "AppConfigs" should "have proper default values" in { 48 | assert(AppConfigs.SparkReceivers._2 == "1") 49 | assert(AppConfigs.SparkStorageLevel._2 == "MEMORY_AND_DISK_SER_2") 50 | assert(AppConfigs.Topics._2 == "") 51 | assert(AppConfigs.TopicsBlackList._2 == "") 52 | assert(AppConfigs.TopicsEnableBlockingConsumption._2 == "false") 53 | assert(AppConfigs.TopicConsumptionPolicy._2 == "OFFSET") 54 | assert(AppConfigs.TopicConsumptionOffsetThreshold._2 == "0") 55 | assert(AppConfigs.TopicConsumptionTimeThresholdMs._2 == "1000") 56 | assert(AppConfigs.TopicFetchSizeBytes._2 == "1048576") 57 | assert(AppConfigs.TopicRepartitionFactor._2 == "1") 58 | assert(AppConfigs.TopicStartOffset._2 == "-1") 59 | assert(AppConfigs.PartitionRefreshIntervalMs._2 == "30000") 60 | assert(AppConfigs.PartitionWarmUpRefreshIntervalMs._2 == "10000") 61 | assert(AppConfigs.ReceiverRestIntervalOnFailMs._2 == "2500") 62 | assert(AppConfigs.ReceiverRestIntervalOnSuccessMs._2 == "100") 63 | assert(AppConfigs.KafkaBrokerConnect._2 == "") 64 | assert(AppConfigs.KafkaSocketTimeoutMs._2 == "10000") 65 | assert(AppConfigs.KafkaSocketBufferSizeBytes._2 == "1048576") 66 | assert(AppConfigs.KafkaZkSessionTimeoutMs._2 == "10000") 67 | assert(AppConfigs.KafkaZkConnectionTimeoutMs._2 == "10000") 68 | assert(AppConfigs.StateControllerType._2 == "MEMORY") 69 | assert(AppConfigs.ZookeeperStateControllerConnect._2 == "") 70 | assert(AppConfigs.ZookeeperStateControllerRoot._2 == "/baryon") 71 | assert(AppConfigs.ZookeeperStateControllerConnTimeoutMs._2 == "120000") 72 | assert(AppConfigs.ZookeeperStateControllerSessionTimeoutMs._2 == "60000") 73 | assert(AppConfigs.TopicFetcherType._2 == "LOCAL") 74 | assert(AppConfigs.HDFSTopicSource._2 == "") 75 | assert(AppConfigs.HTTPTopicSource._2 == "") 76 | } 77 | 78 | "validate" should "not throw exception if all configs present" in { 79 | val properties = new Properties() 80 | properties.setProperty("key1", "value1") 81 | properties.setProperty("key2", "value2") 82 | properties.setProperty("key3", "value3") 83 | 84 | val requiredProps = Seq("key1", "key2") 85 | 86 | AppConfigs.validate(properties, requiredProps) 87 | } 88 | 89 | it should "throw MissingConfigException if required config missing" in 90 | intercept[MissingConfigException] { 91 | val properties = new Properties() 92 | properties.setProperty("key1", "value1") 93 | properties.setProperty("key2", "value2") 94 | properties.setProperty("key3", "value3") 95 | 96 | val requiredProps = Seq("key4") 97 | 98 | AppConfigs.validate(properties, requiredProps) 99 | } 100 | 101 | "validatedBooleanConfig" should "return true for a boolean config with value true" in { 102 | val properties = new Properties() 103 | properties.setProperty("key1", "true") 104 | properties.setProperty("key2", "false") 105 | 106 | assert(AppConfigs.validatedBooleanConfig(properties, "key1", "true") == true) 107 | } 108 | 109 | "validatedBooleanConfig" should "return false for a boolean config with value false" in { 110 | val properties = new Properties() 111 | properties.setProperty("key1", "true") 112 | properties.setProperty("key2", "false") 113 | 114 | assert(AppConfigs.validatedBooleanConfig(properties, "key2", "false") == false) 115 | } 116 | 117 | it should "throw InvalidConfigException if boolean config has value other than true/false" in 118 | intercept[InvalidConfigException] { 119 | val properties = new Properties() 120 | properties.setProperty("key1", "nonboolean") 121 | properties.setProperty("key2", "false") 122 | 123 | AppConfigs.validatedBooleanConfig(properties, "key1", "true") 124 | } 125 | } 126 | -------------------------------------------------------------------------------- /src/test/scala/com/groupon/dse/configs/KafkaServerConfigTest.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.configs 34 | 35 | import java.util.Properties 36 | 37 | import org.scalatest.FlatSpec 38 | 39 | class KafkaServerConfigTest extends FlatSpec { 40 | 41 | it should " throw exception if required configs missing" in { 42 | intercept[AppConfigs.MissingConfigException] { 43 | val properties = new Properties() 44 | KafkaServerConfigBuilder(properties) 45 | } 46 | } 47 | 48 | "KafkaServerConfigBuilder default values" should "be" in { 49 | val properties = new Properties() 50 | properties.setProperty("kafka.broker.zk.connect", "localhost:2181") 51 | 52 | val kafkaConfigs = KafkaServerConfigBuilder(properties) 53 | assert(kafkaConfigs.socketTimeout == 10000 && 54 | kafkaConfigs.socketBufferSize == 1048576 && 55 | kafkaConfigs.zkSessionTimeout == 10000 && 56 | kafkaConfigs.zkConnectionTimeout == 10000) 57 | } 58 | } 59 | -------------------------------------------------------------------------------- /src/test/scala/com/groupon/dse/configs/ReceiverConfigTest.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.configs 34 | 35 | import java.util.Properties 36 | 37 | import com.groupon.dse.configs.AppConfigs.InvalidConfigException 38 | import com.groupon.dse.kafka.controllers.{InMemoryStateController, StateController, ZookeeperStateController} 39 | import com.groupon.dse.kafka.policy._ 40 | import com.groupon.dse.kafka.topic.LocalTopicFetcher 41 | import org.scalatest.{BeforeAndAfter, FlatSpec} 42 | 43 | class ReceiverConfigTest extends FlatSpec with BeforeAndAfter { 44 | val zkConnTimeout = 10000 45 | val zkSessionTimeout = 10000 46 | var properties: Properties = _ 47 | 48 | before { 49 | properties = new Properties() 50 | properties.setProperty("topics", "sometopic") 51 | properties.setProperty("kafka.broker.zk.connect", "localhost:2181") 52 | } 53 | 54 | "ReceiverConfig" should "throw exception if required configs missing" in { 55 | intercept[AppConfigs.MissingConfigException] { 56 | val properties = new Properties() 57 | ReceiverConfigBuilder(properties) 58 | } 59 | } 60 | 61 | "ReceiverConfig default values" should "be" in { 62 | val receiverConfig = ReceiverConfigBuilder(properties) 63 | assert(receiverConfig.partitionRefreshIntervalMs == 30000) 64 | assert(receiverConfig.partitionWarmUpRefreshIntervalMs == 10000) 65 | assert(receiverConfig.receiverRestIntervalOnFail == 2500) 66 | assert(receiverConfig.receiverRestIntervalOnSuccess == 100) 67 | assert(receiverConfig.isBlocking == false) 68 | assert(receiverConfig.stateController.getClass == classOf[InMemoryStateController]) 69 | assert(receiverConfig.fetchPolicy.getClass == classOf[OffsetBasedFetchPolicy]) 70 | assert(receiverConfig.topicRepartitionFactor == 1) 71 | assert(receiverConfig.topicFetcher.getClass == classOf[LocalTopicFetcher]) 72 | } 73 | 74 | "ReceiverConfig" should "accept the custom StateController" in { 75 | var stateController: StateController = new InMemoryStateController 76 | var receiverConfig = ReceiverConfigBuilder(properties, Some(stateController)) 77 | assert(receiverConfig.stateController.getClass == classOf[InMemoryStateController]) 78 | 79 | stateController = new ZookeeperStateController("someendpoint", "/somepath", zkConnTimeout, zkSessionTimeout) 80 | receiverConfig = ReceiverConfigBuilder(properties, Some(stateController)) 81 | assert(receiverConfig.stateController.getClass == classOf[ZookeeperStateController]) 82 | } 83 | 84 | "ReceiverConfig" should "throw exception when blocking enabled for default stateController" in 85 | intercept[InvalidConfigException] { 86 | val currentProperties = properties.clone().asInstanceOf[Properties] 87 | currentProperties.setProperty("topic.consumption.blocking", "true") 88 | ReceiverConfigBuilder(currentProperties) 89 | } 90 | 91 | "ReceiverConfig" should "throw exception when invalid value provided for blocking config" in 92 | intercept[InvalidConfigException] { 93 | val currentProperties = properties.clone().asInstanceOf[Properties] 94 | currentProperties.setProperty("topic.consumption.blocking", "random") 95 | ReceiverConfigBuilder(currentProperties) 96 | } 97 | 98 | "ReceiverConfig" should "throw exception when invalid value provided for repartition hint config" in 99 | intercept[InvalidConfigException] { 100 | val currentProperties = properties.clone().asInstanceOf[Properties] 101 | currentProperties.setProperty("topic.repartition.factor", "-1") 102 | ReceiverConfigBuilder(currentProperties) 103 | } 104 | 105 | "ReceiverConfig" should "return appropriate configs when blocking enabled" in { 106 | val currentProperties = properties.clone().asInstanceOf[Properties] 107 | currentProperties.setProperty("topic.consumption.blocking", "true") 108 | currentProperties.setProperty("statecontroller.type", "ZOOKEEPER") 109 | currentProperties.setProperty("statecontroller.zk.connect", "localhost:2181") 110 | currentProperties.setProperty("statecontroller.zk.root", "/test") 111 | currentProperties.setProperty("topic.consumption.policy", "TIME") //This should get ignored 112 | val receiverConfig = ReceiverConfigBuilder(currentProperties) 113 | assert(receiverConfig.isBlocking == true) 114 | assert(receiverConfig.fetchPolicy.getClass == classOf[OffsetBasedFetchPolicy]) 115 | assert(receiverConfig.stateController.getClass == classOf[ZookeeperStateController]) 116 | assert(receiverConfig.partitionRefreshIntervalMs == 30000) 117 | assert(receiverConfig.partitionWarmUpRefreshIntervalMs == 10000) 118 | assert(receiverConfig.receiverRestIntervalOnFail == 2500) 119 | assert(receiverConfig.receiverRestIntervalOnSuccess == 100) 120 | assert(receiverConfig.topicRepartitionFactor == 1) 121 | } 122 | } -------------------------------------------------------------------------------- /src/test/scala/com/groupon/dse/helper/EmbeddedKafka.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.testutils 34 | 35 | import java.util.Properties 36 | 37 | import kafka.producer.{KeyedMessage, Producer} 38 | import kafka.server.{KafkaConfig, KafkaServer} 39 | 40 | 41 | /** 42 | * KafkaCluster is used to start/stop a kafka cluster. It starts an embedded zookeeper and a kafka broker for testing purposes. 43 | */ 44 | class EmbeddedKafka { 45 | var zkPort: Int = _ 46 | var kafkaPort: Int = _ 47 | var zkServer: EmbeddedZookeeper = _ 48 | var kafkaServer: KafkaServer = _ 49 | 50 | /** 51 | * Starts a zookeeper and kafka broker 52 | */ 53 | def startCluster(): Unit = { 54 | zkServer = startZkServer() 55 | kafkaServer = startKafkaServer(zkServer.connectString) 56 | } 57 | 58 | /** 59 | * Starts a local kafka broker using a randomly selected port 60 | * @param zkConnect The zookeeper connect string 61 | * @return the Kafka server object 62 | */ 63 | def startKafkaServer(zkConnect: String) = { 64 | kafkaPort = TestUtils.choosePorts(1)(0) 65 | val logDir = TestUtils.tempDir 66 | val kafkaBrokerProperties = new Properties() { 67 | put("broker.id", "0") 68 | put("log.dirs", String.valueOf(logDir)) 69 | put("zookeeper.connect", zkConnect) 70 | put("port", kafkaPort.toString) 71 | put("host.name", "localhost") 72 | } 73 | val server = new KafkaServer(new KafkaConfig(kafkaBrokerProperties)) 74 | server.startup() 75 | server 76 | } 77 | 78 | /** 79 | * Starts a local zookeeper using a randomly selected port 80 | * @return the Embedded Zookeeper object 81 | */ 82 | def startZkServer() = { 83 | zkPort = TestUtils.choosePorts(1)(0) 84 | val server = new EmbeddedZookeeper(zkPort) 85 | server 86 | } 87 | 88 | /** 89 | * Stops a zookeeper and kafka broker 90 | */ 91 | def stopCluster(): Unit = { 92 | if (kafkaServer != null) 93 | kafkaServer.shutdown() 94 | if (zkServer != null) 95 | zkServer.cleanShutdown() 96 | } 97 | 98 | /** 99 | * @return Properties required by the producer 100 | */ 101 | def kafkaProducerProperties = new Properties() { 102 | put("metadata.broker.list", s"127.0.0.1:${kafkaPort}") 103 | put("request.required.acks", "-1") 104 | put("producer.type", "sync") 105 | 106 | } 107 | 108 | /** 109 | * @return Properties required by the consumer 110 | */ 111 | def kafkaConsumerProperties = new Properties() { 112 | put("group.id", "unit-test-id") 113 | put("zookeeper.connect", s"127.0.0.1:${zkPort}") 114 | put("zookeeper.session.timeout.ms", "6000") 115 | put("auto.offset.reset", "smallest") 116 | put("auto.commit.interval.ms", "10") 117 | put("consumer.id", "consumerid") 118 | put("consumer.timeout.ms", "-1") 119 | put("rebalance.max.retries", "4") 120 | put("num.consumer.fetchers", "2") 121 | } 122 | 123 | def sleep(ms: Long) = Thread.sleep(ms) 124 | 125 | /** 126 | * Send a message to the kafka broker 127 | * @param cnt The number of time to send the message 128 | * @param producer The producer object to use to send 129 | * @param kafkaTopic The topic to send to 130 | * @param message Message to send 131 | */ 132 | def sendMessage(cnt: Int, 133 | producer: Producer[String, Array[Byte]], 134 | kafkaTopic: String, 135 | message: Array[Byte] = Array[Byte](1, 2, 3, 4, 5, 6, 7, 8, 9)): Unit = { 136 | for (x <- 1 to cnt) 137 | producer.send(new KeyedMessage[String, Array[Byte]](kafkaTopic, message)) 138 | } 139 | 140 | } 141 | -------------------------------------------------------------------------------- /src/test/scala/com/groupon/dse/helper/EmbeddedSpark.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.testutils 34 | 35 | import org.apache.spark.SparkConf 36 | import org.apache.spark.streaming.{Milliseconds, StreamingContext} 37 | 38 | /** 39 | * Creates an embedded spark instance along with the spark streaming context 40 | * @param name Name of the Spark App 41 | * @param parallelism Number of threads 42 | * @param interval Fetch time for spark streaming 43 | */ 44 | class EmbeddedSpark(name: String, parallelism: Int, interval: Int) { 45 | private var sparkConf: SparkConf = _ 46 | private var ssc: StreamingContext = _ 47 | createSparkEmbedded 48 | 49 | def getSparkConf: SparkConf = sparkConf 50 | 51 | def getStreamingContext: StreamingContext = ssc 52 | 53 | /** 54 | * Stops spark streaming 55 | */ 56 | def stop = { 57 | if (ssc != null) { 58 | ssc.stop() 59 | } 60 | } 61 | 62 | /** 63 | * Creates spark conf and spark streaming objects using the values provided while creating the class object 64 | */ 65 | private[this] def createSparkEmbedded = { 66 | sparkConf = new SparkConf().setMaster(s"local[$parallelism]").setAppName(name) 67 | ssc = new StreamingContext(sparkConf, Milliseconds(interval)) 68 | ssc.sparkContext.setLogLevel("OFF") 69 | } 70 | } -------------------------------------------------------------------------------- /src/test/scala/com/groupon/dse/helper/EmbeddedZookeeper.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.testutils 34 | 35 | import java.io.File 36 | import java.net.InetSocketAddress 37 | 38 | import kafka.utils.Utils 39 | import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} 40 | 41 | /** 42 | * Embedded zookeeper to run locally 43 | * @param zkPort The port to start the local zookeeper on 44 | */ 45 | class EmbeddedZookeeper(zkPort: Int, zkSnapshotDir: File, zkLogDir: File, autoStart: Boolean) { 46 | 47 | val connectString = s"127.0.0.1:${zkPort}" 48 | val tickTime = 500 49 | val factory = new NIOServerCnxnFactory() 50 | val maxZkConnections = 100 51 | factory.configure(new InetSocketAddress("127.0.0.1", zkPort), maxZkConnections) 52 | 53 | var zookeeper: ZooKeeperServer = null 54 | 55 | def this(port: Int, autoStart: Boolean = true) = this(port, TestUtils.tempDir, TestUtils.tempDir, autoStart) 56 | 57 | if (autoStart) { 58 | start() 59 | } 60 | 61 | def start(): Unit = { 62 | // With Zookeeper 3.4, the startup logic of the ZookeeperServer has changed where a sequence of: 63 | // zookeeper.start() -> zookeeper.shutdown() -> zookeeper.start() 64 | // will fail to restart the ZookeeperServer. Because of this, a new ZookeeperServer needs to be instantiated if 65 | // we want to simulate Zookeeper unavailability during tests 66 | zookeeper = new ZooKeeperServer(zkSnapshotDir, zkLogDir, tickTime) 67 | factory.startup(zookeeper) 68 | } 69 | 70 | def stop(): Unit = { 71 | zookeeper.shutdown() 72 | zookeeper = null 73 | } 74 | 75 | def snapshotDir: File = zkSnapshotDir 76 | 77 | def logDir: File = zkLogDir 78 | 79 | def cleanShutdown(): Unit = { 80 | shutdown() 81 | Utils.rm(zkLogDir) 82 | Utils.rm(zkSnapshotDir) 83 | } 84 | 85 | def shutdown(): Unit = { 86 | Utils.swallow(zookeeper.shutdown()) 87 | Utils.swallow(factory.shutdown()) 88 | zookeeper = null 89 | } 90 | } 91 | 92 | -------------------------------------------------------------------------------- /src/test/scala/com/groupon/dse/helper/TestDefaults.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.testutils 34 | 35 | import com.groupon.dse.configs.KafkaServerConfig 36 | 37 | object TestDefaults { 38 | 39 | val TestClientName = "baryon_test_client" 40 | val TestClientId = 1 41 | val TestTopicFetchSize = 1048576 42 | val TestPathPrefix = "/baryon_test" 43 | val TestTopic = "topic1" 44 | 45 | 46 | def testKafkaServerConfig(zkConnect: String) 47 | : KafkaServerConfig = KafkaServerConfig( 48 | zkConnect, 49 | 10000, 1048576, 50 | 10000, 10000) 51 | } -------------------------------------------------------------------------------- /src/test/scala/com/groupon/dse/helper/TestUtils.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.testutils 34 | 35 | import java.io.File 36 | import java.net.ServerSocket 37 | import java.util.Random 38 | 39 | import kafka.utils.Utils 40 | 41 | /** 42 | * Provides a set of utilities for the embedded zookeeper and kafka broker 43 | */ 44 | object TestUtils { 45 | private[this] val seededRandom = new Random(192348092834L) 46 | private[this] val random = new Random 47 | 48 | /** 49 | * Creates a temp directory required by the zk. It deleted the dir on shutdown 50 | * Returns a java File object 51 | */ 52 | def tempDir: File = { 53 | val ioDir = System.getProperty("java.io.tmpdir") 54 | val dir = new File(ioDir, "kafka-" + random.nextInt(10000000)) 55 | dir.mkdirs 56 | println(s"tmp file : ${dir.getCanonicalPath}") 57 | dir.deleteOnExit() 58 | Runtime.getRuntime().addShutdownHook(new Thread() { 59 | override def run() = { 60 | Utils.rm(dir) 61 | } 62 | }) 63 | dir 64 | } 65 | 66 | /** 67 | * Returns n random ports for the zookeeper and broker 68 | * @param count indicates the number of random ports to return 69 | * @return Array of n random ports 70 | */ 71 | def choosePorts(count: Int) = { 72 | val sockets = 73 | for (i <- 0 until count) 74 | yield new ServerSocket(0) 75 | val socketList = sockets.toList 76 | val ports = socketList.map(_.getLocalPort) 77 | socketList map (_.close()) 78 | ports 79 | } 80 | } 81 | -------------------------------------------------------------------------------- /src/test/scala/com/groupon/dse/kafka/cluster/impl/ClusterImplTest.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.cluster.impl 34 | 35 | import com.groupon.dse.configs.KafkaServerConfig 36 | import com.groupon.dse.testutils.{EmbeddedKafka, TestDefaults} 37 | import com.groupon.dse.zookeeper.ZkClientBuilder 38 | import kafka.producer.{Producer, ProducerConfig} 39 | import org.I0Itec.zkclient.ZkClient 40 | import org.scalatest.{BeforeAndAfter, FlatSpec} 41 | 42 | class ClusterImplTest extends FlatSpec with BeforeAndAfter { 43 | 44 | val kafkaTopic = TestDefaults.TestTopic 45 | val zkConnTimeout = 10000 46 | val zkSessionTimeout = 10000 47 | var producer: Producer[String, Array[Byte]] = _ 48 | var embeddedKafka: EmbeddedKafka = _ 49 | var cluster: ClusterImpl = _ 50 | var zkConnect: String = _ 51 | var kafkaServerConfigs: KafkaServerConfig = _ 52 | var zkClient: ZkClient = _ 53 | 54 | before { 55 | embeddedKafka = new EmbeddedKafka 56 | embeddedKafka.startCluster() 57 | producer = new Producer[String, Array[Byte]](new ProducerConfig(embeddedKafka.kafkaProducerProperties)) 58 | zkConnect = embeddedKafka.zkServer.connectString 59 | kafkaServerConfigs = TestDefaults.testKafkaServerConfig(zkConnect) 60 | cluster = new ClusterImpl(kafkaServerConfigs) 61 | zkClient = ZkClientBuilder(zkConnect, zkConnTimeout, zkSessionTimeout) 62 | } 63 | 64 | after { 65 | zkClient.close() 66 | embeddedKafka.stopCluster() 67 | } 68 | 69 | "The topic list" must "have size 0 before producing" in { 70 | assert(cluster.topics(zkClient).size == 0) 71 | } 72 | 73 | "The topic list" must "have size 1 after producing" in { 74 | embeddedKafka.sendMessage(4, producer, kafkaTopic) 75 | assert(cluster.topics(zkClient).size == 1) 76 | } 77 | 78 | "The number of partitions for a topic" should "be 1 for 1 valid topic" in { 79 | embeddedKafka.sendMessage(4, producer, kafkaTopic) 80 | assert(cluster.partitions(List(kafkaTopic), zkClient).size == 1) 81 | } 82 | 83 | "The number of partitions" should "be 0 for an invalid topic" in { 84 | embeddedKafka.sendMessage(4, producer, kafkaTopic) 85 | assert(cluster.partitions(List("invalid_topic"), zkClient).size == 0) 86 | } 87 | 88 | "The number of partitions" should "be 1 for a valid and invalid topic" in { 89 | embeddedKafka.sendMessage(4, producer, kafkaTopic) 90 | assert(cluster.partitions(List(kafkaTopic, "invalid_topic"), zkClient).size == 1) 91 | } 92 | 93 | } 94 | -------------------------------------------------------------------------------- /src/test/scala/com/groupon/dse/kafka/common/ConsumerClientCacheTest.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.common 34 | 35 | import com.groupon.dse.configs.KafkaServerConfig 36 | import com.groupon.dse.kafka.partition.impl.PartitionImpl 37 | import com.groupon.dse.testutils.{EmbeddedKafka, TestDefaults} 38 | import kafka.consumer.SimpleConsumer 39 | import kafka.producer.{Producer, ProducerConfig} 40 | import org.scalatest.{BeforeAndAfter, FlatSpec} 41 | 42 | class ConsumerClientCacheTest extends FlatSpec with BeforeAndAfter { 43 | 44 | val kafkaTopic = TestDefaults.TestTopic 45 | val clientId = TestDefaults.TestClientId 46 | val clientName = TestDefaults.TestClientName 47 | var kafkaServerConfigs: KafkaServerConfig = _ 48 | var partition: PartitionImpl = _ 49 | var embeddedKafka: EmbeddedKafka = _ 50 | var partitionKey: String = _ 51 | var consumerClientCache: ConsumerClientCache = _ 52 | 53 | before { 54 | embeddedKafka = new EmbeddedKafka 55 | embeddedKafka.startCluster() 56 | 57 | val producer = new Producer[String, Array[Byte]](new ProducerConfig(embeddedKafka.kafkaProducerProperties)) 58 | embeddedKafka.sendMessage(1, producer, kafkaTopic) 59 | 60 | kafkaServerConfigs = TestDefaults.testKafkaServerConfig(embeddedKafka.zkServer.connectString) 61 | partition = new PartitionImpl(kafkaTopic, 0) 62 | partitionKey = TestDefaults.TestPathPrefix + "/" + partition.partitionIdentifier 63 | consumerClientCache = new ConsumerClientCache(kafkaServerConfigs, "test_client") 64 | } 65 | 66 | after { 67 | embeddedKafka.stopCluster() 68 | } 69 | 70 | "Contains on start " should " return false " in { 71 | assert(!consumerClientCache.contains(partition)) 72 | } 73 | 74 | 75 | "Get " should " return None for initial run " in { 76 | assert(consumerClientCache.get(partition) == None) 77 | } 78 | 79 | "Get " should " return Some SimpleConsumer client if presetn in cache " in { 80 | consumerClientCache.getWithLoad(partition) 81 | assert(consumerClientCache.get(partition).isDefined) 82 | assert(consumerClientCache.get(partition).get.isInstanceOf[SimpleConsumer]) 83 | } 84 | 85 | "Get with load " should " always return a client " in { 86 | assert(consumerClientCache.getWithLoad(partition).get.isInstanceOf[SimpleConsumer]) 87 | } 88 | 89 | "Remove " should " remove the partition from the cache " in { 90 | consumerClientCache.getWithLoad(partition) 91 | consumerClientCache.remove(partition) 92 | assert(!consumerClientCache.contains(partition)) 93 | } 94 | } 95 | -------------------------------------------------------------------------------- /src/test/scala/com/groupon/dse/kafka/common/StateTest.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.common 34 | 35 | import org.scalatest.FlatSpec 36 | 37 | class StateTest extends FlatSpec { 38 | "State string " should " return State object " in { 39 | assert(State("{\"offset\":12,\"timestamp\":121212121}") == State(12, 121212121)) 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /src/test/scala/com/groupon/dse/kafka/controllers/InMemoryStateControllerTest.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.controllers 34 | 35 | import com.groupon.dse.configs.KafkaServerConfig 36 | import com.groupon.dse.kafka.common.{ConsumerClientBuilder, State, WrappedMessage} 37 | import com.groupon.dse.kafka.partition.Leader 38 | import com.groupon.dse.kafka.partition.impl.PartitionImpl 39 | import com.groupon.dse.testutils.{EmbeddedKafka, EmbeddedSpark, TestDefaults} 40 | import kafka.consumer.SimpleConsumer 41 | import kafka.producer.{Producer, ProducerConfig} 42 | import org.apache.spark.groupon.metrics.UserMetricsSystem 43 | import org.apache.spark.streaming.StreamingContext 44 | import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FlatSpec} 45 | 46 | class InMemoryStateControllerTest extends FlatSpec with BeforeAndAfter with BeforeAndAfterAll { 47 | 48 | val kafkaTopic = TestDefaults.TestTopic 49 | val clientName = TestDefaults.TestClientName 50 | val clazz = getClass.getName 51 | var kafkaServerConfigs: KafkaServerConfig = _ 52 | var producer: Producer[String, Array[Byte]] = _ 53 | var partition: PartitionImpl = _ 54 | var partitionKey: String = _ 55 | var embeddedKafka: EmbeddedKafka = _ 56 | var embeddedSpark: EmbeddedSpark = _ 57 | var ssc: StreamingContext = _ 58 | var stateController: InMemoryStateController = _ 59 | var leader: Leader = _ 60 | var consumer: SimpleConsumer = _ 61 | 62 | override def beforeAll() = { 63 | embeddedSpark = new EmbeddedSpark(clazz, 4, 500) 64 | ssc = embeddedSpark.getStreamingContext 65 | UserMetricsSystem.initialize(ssc.sparkContext) 66 | } 67 | 68 | override def afterAll() = { 69 | embeddedSpark.stop 70 | } 71 | 72 | before { 73 | embeddedKafka = new EmbeddedKafka 74 | embeddedKafka.startCluster() 75 | producer = new Producer[String, Array[Byte]](new ProducerConfig(embeddedKafka.kafkaProducerProperties)) 76 | kafkaServerConfigs = TestDefaults.testKafkaServerConfig(embeddedKafka.zkServer.connectString) 77 | partition = new PartitionImpl(kafkaTopic, 0) 78 | partitionKey = TestDefaults.TestPathPrefix + "/" + partition.partitionIdentifier 79 | 80 | stateController = new InMemoryStateController 81 | leader = Leader("localhost", embeddedKafka.kafkaPort) 82 | consumer = ConsumerClientBuilder.newInstance(leader, kafkaServerConfigs, clientName) 83 | } 84 | 85 | after { 86 | stateController.close 87 | embeddedKafka.stopCluster() 88 | } 89 | 90 | it should "throw KeyDoesNotExistException if state is not set yet for the key" in { 91 | intercept[KeyDoesNotExistException] { 92 | stateController.getState("/invalid_path") 93 | } 94 | } 95 | 96 | it should "return the correct state offset if key exists " in { 97 | val state = State(1234, System.currentTimeMillis()) 98 | val output = stateController.setState("/valid_path", state) 99 | assert(output ==("/valid_path", state)) 100 | assert(stateController.getState("/valid_path").offset == 1234) 101 | } 102 | 103 | "The state key" should "equal partitionIdentifier for a give partition" in { 104 | assert(stateController.generateStateKey(new PartitionImpl("Test_topic", 0)) == "Test_topic/0") 105 | } 106 | 107 | it should "return false if key does not exist in the cache" in { 108 | assert(!stateController.isKeyPresent("/invalid_key")) 109 | } 110 | 111 | it should "return true if key does exist in the cache" in { 112 | stateController.setState("/valid_path", State(1234, System.currentTimeMillis())) 113 | assert(stateController.isKeyPresent("/valid_path")) 114 | } 115 | 116 | "StateController" should "compute the max offset from the RDD list" in { 117 | embeddedKafka.sendMessage(1, producer, kafkaTopic) 118 | val messages = partition.fetchMessages(0, 119 | TestDefaults.TestTopicFetchSize, 120 | partitionKey, 121 | consumer) 122 | 123 | val msgRdd = ssc.sparkContext.parallelize[WrappedMessage](messages.toList) 124 | val state = stateController.setState(msgRdd)(0)._2 125 | assert(state.offset == 1) 126 | } 127 | } 128 | 129 | -------------------------------------------------------------------------------- /src/test/scala/com/groupon/dse/kafka/controllers/StateControllerTest.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.controllers 34 | 35 | import java.util.Properties 36 | 37 | import com.groupon.dse.configs.AppConfigs 38 | import com.groupon.dse.configs.AppConfigs.InvalidConfigException 39 | import org.scalatest.FlatSpec 40 | 41 | class StateControllerTest extends FlatSpec { 42 | 43 | "StateControllerBuilder for ZOOKEEPER" should " return appropriate class object" in { 44 | val properties = new Properties() 45 | properties.setProperty("statecontroller.type", "ZOOKEEPER") 46 | properties.setProperty("statecontroller.zk.connect", "localhost:2181") 47 | properties.setProperty("statecontroller.zk.root", "/test") 48 | assert(StateControllerBuilder(properties).getClass == classOf[ZookeeperStateController]) 49 | } 50 | 51 | "StateControllerBuilder" should "return InMemory controller if stateController type is missing " in { 52 | val properties = new Properties() 53 | assert(StateControllerBuilder(properties).getClass == classOf[InMemoryStateController]) 54 | } 55 | 56 | "StateControllerBuilder" should " throw exception if required zkstatecontroller connect config missing" in { 57 | intercept[AppConfigs.MissingConfigException] { 58 | val properties = new Properties() 59 | properties.setProperty("statecontroller.type", "ZOOKEEPER") 60 | StateControllerBuilder(properties) 61 | } 62 | } 63 | "StateControllerBuilder" should " throw exception if state controller is in-memory and blocking fetch policy set " in 64 | intercept[InvalidConfigException] { 65 | val properties = new Properties() 66 | properties.setProperty("statecontroller.type", "MEMORY") 67 | properties.setProperty("topic.consumption.blocking", "true") 68 | StateControllerBuilder(properties) 69 | } 70 | 71 | "StateControllerBuilder" should " throw exception if state controller is in-memory and start offset is not -1 " in 72 | intercept[InvalidConfigException] { 73 | val properties = new Properties() 74 | properties.setProperty("statecontroller.type", "MEMORY") 75 | properties.setProperty("topic.start.offset", "-2") 76 | StateControllerBuilder(properties) 77 | } 78 | } 79 | -------------------------------------------------------------------------------- /src/test/scala/com/groupon/dse/kafka/topic/HdfsJsonTopicFetcherTest.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.topic 34 | 35 | 36 | import java.nio.file.Files 37 | 38 | import org.apache.hadoop.conf.Configuration 39 | import org.apache.hadoop.fs.{FileSystem, Path} 40 | import org.apache.hadoop.io.SequenceFile.{Writer => SeqWriter} 41 | import org.scalatest.{BeforeAndAfterAll, FlatSpec} 42 | 43 | class HdfsJsonTopicFetcherTest extends FlatSpec with BeforeAndAfterAll { 44 | 45 | val testRoot: Path = new Path(Files.createTempDirectory("topic-fetcher-test").toString) 46 | var conf: Configuration = _ 47 | var fs: FileSystem = _ 48 | var path: Path = _ 49 | 50 | override def beforeAll(): Unit = { 51 | conf = new Configuration() 52 | fs = FileSystem.get(conf) 53 | } 54 | 55 | override def afterAll(): Unit = { 56 | fs.delete(testRoot, true) 57 | fs.close() 58 | } 59 | 60 | "The hdfs topic fetcher " should "return the list of topics specified in the json" in { 61 | val output = fs.create(new Path(testRoot, "test-json-file.txt")) 62 | val contents = "{\"topics\":[{\"name\":\"topic1\",\"metadata\":{\"m1\":\"v1\",\"m2\":\"v2\"}},{\"name\":\"topic2\",\"metadata\":{\"m3\":\"v3\",\"m4\":\"v4\"}},{\"name\":\"topic3\",\"metadata\":{\"m3\":\"v3\",\"m4\":\"v4\"}}]}" 63 | output.write(contents.getBytes) 64 | output.close() 65 | val hdfsTopicFetcher = new HdfsJsonTopicFetcher(testRoot.toString() + "/test-json-file.txt") 66 | assert(hdfsTopicFetcher.fetchTopics == Seq(TopicAndMetadata("topic1", Map("m1" -> "v1", "m2" -> "v2")), TopicAndMetadata("topic2", Map("m3" -> "v3", "m4" -> "v4")), TopicAndMetadata("topic3", Map("m3" -> "v3", "m4" -> "v4")))) 67 | } 68 | 69 | "The hdfs topic fetcher " should "return the filtered list if the name key is missing for any topic" in { 70 | val output = fs.create(new Path(testRoot, "test-json-file.txt")) 71 | val contents = "{\"topics\":[{\"name\":\"topic1\",\"metadata\":{\"m1\":\"v1\",\"m2\":\"v2\"}},{\"name1\":\"topic2\",\"enabled\":\"false\",\"metadata\":{\"m3\":\"v3\",\"m4\":\"v4\"}},{\"name\":\"topic3\",\"enabled\":\"true\",\"metadata\":{\"m3\":\"v3\",\"m4\":\"v4\"}}],\"extras\":\"test\"}" 72 | output.write(contents.getBytes) 73 | output.close() 74 | val hdfsTopicFetcher = new HdfsJsonTopicFetcher(testRoot.toString() + "/test-json-file.txt") 75 | assert(hdfsTopicFetcher.fetchTopics == Seq(TopicAndMetadata("topic1", Map("m1" -> "v1", "m2" -> "v2")), TopicAndMetadata("topic3", Map("m3" -> "v3", "m4" -> "v4")))) 76 | } 77 | 78 | "The hdfs topic fetcher " should "support nested metadata" in { 79 | val output = fs.create(new Path(testRoot, "test-json-file.txt")) 80 | val contents = "{\"topics\":[{\"name\":\"topic1\",\"metadata\":{\"m1\":\"v1\",\"m2\":\"v2\"}},{\"name\":\"topic2\",\"metadata\":{\"m3\":\"v3\",\"m4\":{\"m5\":\"v5\"},\"bool\":true}}]}" 81 | output.write(contents.getBytes) 82 | output.close() 83 | val hdfsTopicFetcher = new HdfsJsonTopicFetcher(testRoot.toString() + "/test-json-file.txt") 84 | assert(hdfsTopicFetcher.fetchTopics == Seq(TopicAndMetadata("topic1", Map("m1" -> "v1", "m2" -> "v2")), TopicAndMetadata("topic2", Map("m3" -> "v3", "m4" -> Map("m5" -> "v5").asInstanceOf[AnyRef], "bool" -> true.asInstanceOf[AnyRef])))) 85 | } 86 | 87 | "The hdfs topic fetcher " should "return an empty map if not metadata is provided" in { 88 | val output = fs.create(new Path(testRoot, "test-json-file.txt")) 89 | val contents = "{\"topics\":[{\"name\":\"topic1\"},{\"name\":\"topic2\",\"metadata\":{\"m3\":\"v3\",\"m4\":{\"m5\":\"v5\"},\"bool\":true}}]}" 90 | output.write(contents.getBytes) 91 | output.close() 92 | val hdfsTopicFetcher = new HdfsJsonTopicFetcher(testRoot.toString() + "/test-json-file.txt") 93 | assert(hdfsTopicFetcher.fetchTopics == Seq(TopicAndMetadata("topic1", Map()), TopicAndMetadata("topic2", Map("m3" -> "v3", "m4" -> Map("m5" -> "v5").asInstanceOf[AnyRef], "bool" -> true.asInstanceOf[AnyRef])))) 94 | } 95 | 96 | "The read file method" should "return the contents of the file" in { 97 | val output = fs.create(new Path(testRoot, "test-json-file.txt")) 98 | val contents = "test file contents" 99 | output.write(contents.getBytes) 100 | output.close() 101 | val hdfsTopicFetcher = new HdfsJsonTopicFetcher(testRoot.toString() + "/test-json-file.txt") 102 | assert(hdfsTopicFetcher.readTopicFile == contents) 103 | } 104 | } 105 | 106 | -------------------------------------------------------------------------------- /src/test/scala/com/groupon/dse/kafka/topic/HttpJsonTopicFetcherTest.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.topic 34 | 35 | import org.mockito.Mockito 36 | import org.scalatest.{BeforeAndAfter, FlatSpec} 37 | 38 | class HttpJsonTopicFetcherTest extends FlatSpec with BeforeAndAfter { 39 | 40 | "The topic fetcher" should "return the correct list of topics " in { 41 | val http = Mockito.spy(new HttpJsonTopicFetcher("http://test-url")) 42 | val content = "{\"topics\":[{\"name\":\"topic1\",\"metadata\":{\"m1\":\"v1\",\"m2\":true}},{\"name\":\"topic2\",\"metadata\":{\"m3\":\"v3\"}}],\"extras\":\"test\"}" 43 | Mockito.doReturn(Some(content)).when(http).getContentFromUrl 44 | assert(http.fetchTopics == Seq(TopicAndMetadata("topic1", Map("m1" -> "v1", "m2" -> true.asInstanceOf[AnyRef])), TopicAndMetadata("topic2", Map("m3" -> "v3")))) 45 | } 46 | 47 | "The topic fetcher" should "return an empty list if the Json in incorrect " in { 48 | val http = Mockito.spy(new HttpJsonTopicFetcher("http://test-url")) 49 | val content = "{\"topics\":[{\"name\":\"topic1\",\"metadata\":{\"m1\":\"v1\",\"m2\":true}}{\"name\":\"topic2\",\"metadata\":{\"m3\":\"v3\"}}],\"extras\":\"test\"}" 50 | Mockito.doReturn(Some(content)).when(http).getContentFromUrl 51 | assert(http.fetchTopics == Seq()) 52 | } 53 | 54 | "The topic fetcher " should "return the filtered list if the name field is missing for any topic" in { 55 | val http = Mockito.spy(new HttpJsonTopicFetcher("http://test-url")) 56 | val contents = "{\"topics\":[{\"name\":\"topic1\",\"metadata\":{\"m1\":\"v1\",\"m2\":\"v2\"}},{\"name1\":\"topic2\",\"enabled\":\"false\",\"metadata\":{\"m3\":\"v3\",\"m4\":\"v4\"}},{\"name\":\"topic3\",\"enabled\":\"true\",\"metadata\":{\"m3\":\"v3\",\"m4\":\"v4\"}}],\"extras\":\"test\"}" 57 | Mockito.doReturn(Some(contents)).when(http).getContentFromUrl 58 | assert(http.fetchTopics == Seq(TopicAndMetadata("topic1", Map("m1" -> "v1", "m2" -> "v2")), TopicAndMetadata("topic3", Map("m3" -> "v3", "m4" -> "v4")))) 59 | } 60 | 61 | "The topic fetcher" should "return an empty map if no metadata is provided" in { 62 | val http = Mockito.spy(new HttpJsonTopicFetcher("http://test-url")) 63 | val contents = "{\"topics\":[{\"name\":\"topic1\"},{\"name\":\"topic2\",\"metadata\":{\"m3\":\"v3\",\"m4\":{\"m5\":\"v5\"},\"bool\":true}}]}" 64 | Mockito.doReturn(Some(contents)).when(http).getContentFromUrl 65 | assert(http.fetchTopics == Seq(TopicAndMetadata("topic1", Map()), TopicAndMetadata("topic2", Map("m3" -> "v3", "m4" -> Map("m5" -> "v5").asInstanceOf[AnyRef], "bool" -> true.asInstanceOf[AnyRef])))) 66 | } 67 | 68 | "The topic fetcher" should "return an empty list of topics if there is any problem when fetching the Json" in { 69 | val http = Mockito.spy(new HttpJsonTopicFetcher("http://test-url")) 70 | Mockito.doReturn(None).when(http).getContentFromUrl 71 | assert(http.fetchTopics == Seq.empty) 72 | } 73 | 74 | } -------------------------------------------------------------------------------- /src/test/scala/com/groupon/dse/kafka/topic/LocalTopicFetcherTest.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.topic 34 | 35 | import org.scalatest.FlatSpec 36 | 37 | class LocalTopicFetcherTest extends FlatSpec { 38 | "The config file topic fetcher" should "filter blacklisted topics regex" in { 39 | val topics = Seq("test_json", "test1", "test_random", "some_other_json") 40 | val topicBlacklistRegex = Seq(".*json.*") 41 | val topicFetcher = new LocalTopicFetcher(topics, topicBlacklistRegex) 42 | assert(topicFetcher.fetchTopics == Seq(TopicAndMetadata("test1"), TopicAndMetadata("test_random"))) 43 | } 44 | 45 | "The topic fetcher" should "return all topics when no blacklist regex is provided" in { 46 | val topics = Seq("test_json", "test1", "test_random", "some_other_json") 47 | val topicBlacklistRegex = Seq("") 48 | val topicFetcher = new LocalTopicFetcher(topics, topicBlacklistRegex) 49 | assert(topicFetcher.fetchTopics == Seq(TopicAndMetadata("test_json"), TopicAndMetadata("test1"), TopicAndMetadata("test_random"), TopicAndMetadata("some_other_json"))) 50 | } 51 | 52 | "The topic fetcher" should "filter the blacklisted topics list" in { 53 | val topics = Seq("test_json", "test1", "test_random", "some_other_json") 54 | val topicBlacklistRegex = Seq("some_other_json", ".*json.*") 55 | val topicFetcher = new LocalTopicFetcher(topics, topicBlacklistRegex) 56 | assert(topicFetcher.fetchTopics == Seq(TopicAndMetadata("test1"), TopicAndMetadata("test_random"))) 57 | } 58 | } 59 | -------------------------------------------------------------------------------- /src/test/scala/com/groupon/dse/kafka/topic/TopicFetcherBuilder$Test.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.topic 34 | 35 | import java.util.Properties 36 | 37 | import com.groupon.dse.configs.AppConfigs 38 | import com.groupon.dse.kafka.topic.TopicFetcherBuilder.InvalidTopicFetcherException 39 | import org.scalatest.{BeforeAndAfter, FlatSpec} 40 | 41 | class TopicFetcherBuilder$Test extends FlatSpec with BeforeAndAfter { 42 | 43 | var properties: Properties = _ 44 | before { 45 | properties = new Properties() 46 | } 47 | 48 | "The topic Fetcher builder" should "return the appropriate TopicFetcher" in { 49 | properties.setProperty("topics", "sometopic") 50 | assert(TopicFetcherBuilder(properties).getClass == classOf[LocalTopicFetcher]) 51 | properties.setProperty("topics.fetcher.type", "HDFS") 52 | properties.setProperty("topics.fetcher.hdfs.source", "/path/in/hdfs") 53 | assert(TopicFetcherBuilder(properties).getClass == classOf[HdfsJsonTopicFetcher]) 54 | properties.setProperty("topics.fetcher.type", "LOCAL") 55 | assert(TopicFetcherBuilder(properties).getClass == classOf[LocalTopicFetcher]) 56 | } 57 | 58 | "the topic fetcher builder" should "throw an exception if invalid type is provided" in { 59 | intercept[InvalidTopicFetcherException] { 60 | properties.setProperty("topics.fetcher.type", "INVALID") 61 | TopicFetcherBuilder(properties) 62 | } 63 | } 64 | 65 | "the topic fetcher" should "throw an exception if there are any missing configs" in { 66 | intercept[AppConfigs.MissingConfigException] { 67 | TopicFetcherBuilder(properties) 68 | } 69 | 70 | intercept[AppConfigs.MissingConfigException] { 71 | properties.setProperty("topics.fetcher.type", "HDFS") 72 | TopicFetcherBuilder(properties) 73 | } 74 | } 75 | } 76 | -------------------------------------------------------------------------------- /src/test/scala/com/groupon/dse/kafka/topic/TopicUtils$Test.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.kafka.topic 34 | 35 | import com.fasterxml.jackson.core.JsonParseException 36 | import org.scalatest.FlatSpec 37 | 38 | class TopicUtils$Test extends FlatSpec { 39 | 40 | "The Topic Utils " should "return the list of topics specified in the json" in { 41 | val contents = "{\"topics\":[{\"name\":\"topic1\",\"metadata\":{\"m1\":\"v1\",\"m2\":\"v2\"}},{\"name\":\"topic2\",\"metadata\":{\"m3\":\"v3\",\"m4\":\"v4\"}},{\"name\":\"topic3\",\"metadata\":{\"m3\":\"v3\",\"m4\":\"v4\"}}]}" 42 | assert(TopicUtils.extractTopicListFromJson(contents) == Seq(TopicAndMetadata("topic1", Map("m1" -> "v1", "m2" -> "v2")), TopicAndMetadata("topic2", Map("m3" -> "v3", "m4" -> "v4")), TopicAndMetadata("topic3", Map("m3" -> "v3", "m4" -> "v4")))) 43 | } 44 | 45 | "The Topic Utils " should "return the filtered list if the name key is missing for any topic" in { 46 | val contents = "{\"topics\":[{\"name\":\"topic1\",\"metadata\":{\"m1\":\"v1\",\"m2\":\"v2\"}},{\"name1\":\"topic2\",\"enabled\":\"false\",\"metadata\":{\"m3\":\"v3\",\"m4\":\"v4\"}},{\"name\":\"topic3\",\"enabled\":\"true\",\"metadata\":{\"m3\":\"v3\",\"m4\":\"v4\"}}],\"extras\":\"test\"}" 47 | assert(TopicUtils.extractTopicListFromJson(contents) == Seq(TopicAndMetadata("topic1", Map("m1" -> "v1", "m2" -> "v2")), TopicAndMetadata("topic3", Map("m3" -> "v3", "m4" -> "v4")))) 48 | } 49 | 50 | intercept[JsonParseException] { 51 | val contents = "invalid_json" 52 | TopicUtils.extractTopicListFromJson(contents) 53 | } 54 | } 55 | -------------------------------------------------------------------------------- /src/test/scala/com/groupon/dse/policy/FetchPolicyTest.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.policy 34 | 35 | import java.util.Properties 36 | 37 | import com.groupon.dse.kafka.common.State 38 | import com.groupon.dse.kafka.policy.FetchPolicyBuilder.InvalidFetchPolicyException 39 | import com.groupon.dse.kafka.policy.{FetchPolicyBuilder, OffsetBasedFetchPolicy, TimeAndOffsetBasedFetchPolicy, TimeBasedFetchPolicy} 40 | import org.scalatest.{BeforeAndAfter, FlatSpec} 41 | 42 | class FetchPolicyTest extends FlatSpec with BeforeAndAfter { 43 | var local: State = _ 44 | var global: State = _ 45 | 46 | before { 47 | local = State(1, 9000) 48 | global = State(1, 9500) 49 | } 50 | 51 | "Offset fetch policy " should " return true " in { 52 | val policy = new OffsetBasedFetchPolicy(1, 1024, -1) 53 | assert(policy.canFetch(local, global)) 54 | } 55 | 56 | "Time fetch policy " should " return true " in { 57 | val policy = new TimeBasedFetchPolicy(100, 1024, -1) 58 | assert(policy.canFetch(local, global)) 59 | } 60 | 61 | "Time and offset fetch policy " should " return true " in { 62 | val policy = new TimeAndOffsetBasedFetchPolicy(1, 100, 1024, -1) 63 | assert(policy.canFetch(local, global)) 64 | } 65 | 66 | "FetchPolicyBuilder for TIME_AND_OFFSET" should " return appropriate class object" in { 67 | val properties = new Properties() 68 | properties.setProperty("topic.consumption.policy", "TIME_AND_OFFSET") 69 | val clazz = FetchPolicyBuilder(properties).getClass 70 | assert(clazz == classOf[TimeAndOffsetBasedFetchPolicy]) 71 | } 72 | 73 | "FetchPolicyBuilder for OFFSET" should " return appropriate class object" in { 74 | val properties = new Properties() 75 | properties.setProperty("topic.consumption.policy", "OFFSET") 76 | val clazz = FetchPolicyBuilder(properties).getClass 77 | assert(clazz == classOf[OffsetBasedFetchPolicy]) 78 | } 79 | 80 | "FetchPolicyBuilder for TIME" should " return appropriate class object" in { 81 | val properties = new Properties() 82 | properties.setProperty("topic.consumption.policy", "TIME") 83 | val clazz = FetchPolicyBuilder(properties).getClass 84 | assert(clazz == classOf[TimeBasedFetchPolicy]) 85 | } 86 | 87 | "FetchPolicyBuilder default config values" should "be" in { 88 | val properties = new Properties() 89 | val fetchPolicy = FetchPolicyBuilder(properties) 90 | assert(fetchPolicy.getClass == classOf[OffsetBasedFetchPolicy] && 91 | fetchPolicy.fetchSize == 1048576 && 92 | fetchPolicy.startOffset == -1) 93 | } 94 | 95 | it should " throw exception for invalid fetch policy" in { 96 | intercept[InvalidFetchPolicyException] { 97 | val properties = new Properties() 98 | properties.setProperty("topic.consumption.policy", "UNKNOWN") 99 | FetchPolicyBuilder(properties) 100 | } 101 | } 102 | 103 | "FetchPolicyBuilder with blocking consumption" should " return OffsetBasedFetchPolicy object" in { 104 | val properties = new Properties() 105 | properties.setProperty("topic.consumption.blocking", "true") 106 | val clazz = FetchPolicyBuilder(properties).getClass 107 | assert(clazz == classOf[OffsetBasedFetchPolicy]) 108 | } 109 | 110 | "FetchPolicyBuilder with blocking consumption and fetch policy" should " return OffsetBasedFetchPolicy object" in { 111 | val properties = new Properties() 112 | properties.setProperty("topic.consumption.blocking", "true") 113 | properties.setProperty("topic.consumption.policy", "TIME") 114 | val clazz = FetchPolicyBuilder(properties).getClass 115 | assert(clazz == classOf[OffsetBasedFetchPolicy]) 116 | } 117 | } 118 | -------------------------------------------------------------------------------- /src/test/scala/com/groupon/dse/spark/dstreams/KafkaLowLevelDStreamTest.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.spark.dstreams 34 | 35 | import java.util.Properties 36 | 37 | import com.groupon.dse.configs.ReceiverConfigBuilder 38 | import com.groupon.dse.testutils._ 39 | import kafka.producer.{Producer, ProducerConfig} 40 | import org.apache.spark.storage.StorageLevel 41 | import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FlatSpec} 42 | 43 | /** 44 | * The test starts a kafka cluster, sends messages and uses the kafka low level receiver to consume from a single partition. 45 | */ 46 | class KafkaLowLevelDStreamTest extends FlatSpec with BeforeAndAfter with BeforeAndAfterAll { 47 | 48 | // This single test case performs a complete end to end testing for the DStream Receiver 49 | "The number of fetched and sent messages/bytes" should "be the same" in { 50 | // Prepare the environment 51 | val kafkaTopic = TestDefaults.TestTopic 52 | val totalReceivers = 1 53 | val receiverId = 0 54 | val embeddedKafka = new EmbeddedKafka 55 | embeddedKafka.startCluster() 56 | val producer = new Producer[String, Array[Byte]](new ProducerConfig(embeddedKafka.kafkaProducerProperties)) 57 | val embeddedSpark = new EmbeddedSpark(getClass.getName, 4, 500) 58 | val ssc = embeddedSpark.getStreamingContext 59 | val receiverConfigs = ReceiverConfigBuilder(testEnvProperties(kafkaTopic, embeddedKafka.zkServer.connectString)) 60 | val stateController = receiverConfigs.stateController 61 | 62 | val dStream = new KafkaLowLevelDStream(ssc, 63 | receiverConfigs, 64 | receiverId, 65 | totalReceivers, 66 | StorageLevel.MEMORY_AND_DISK_SER_2 67 | ) 68 | 69 | // Processing layer 70 | var result = 0 71 | var sum = 0 72 | 73 | val numSend = 100 74 | embeddedKafka.sendMessage(numSend, producer, kafkaTopic) 75 | 76 | dStream.foreachRDD(rdd => { 77 | if (rdd.count() > 0) { 78 | //Gets all the messages in the RDD and calculates the total number of messages and the total number of bytes 79 | val ret = rdd.collect() 80 | result += ret.length 81 | ret.foreach { v => 82 | sum += v.payload.length 83 | } 84 | stateController.setState(rdd) 85 | } 86 | }) 87 | dStream.print() 88 | 89 | ssc.start() 90 | // TODO: 15 sec is just an arbitrary sleep interval based on trial and error. 91 | // We should find a better way to determine this 92 | Thread.sleep(15000) 93 | ssc.stop() 94 | 95 | //Check if we get the same number of messages we send 96 | assert(result == numSend) 97 | //Assert if sent num bytes equals received num bytes 98 | assert(numSend * 9 == sum) 99 | 100 | // Cleanup after the test is over 101 | embeddedKafka.stopCluster() 102 | stateController.close 103 | embeddedSpark.stop 104 | } 105 | 106 | private def testEnvProperties(kafkaTopic: String, brokerZkEndpoint: String): Properties = { 107 | val properties = new Properties() 108 | properties.setProperty("topics", kafkaTopic) 109 | properties.setProperty("kafka.broker.zk.connect", brokerZkEndpoint) 110 | properties.setProperty("topic.start.offset", "-2") 111 | properties.setProperty("topic.consumption.policy", "OFFSET") 112 | properties.setProperty("topic.consumption.blocking", "true") 113 | 114 | // We have to use Zookeeper based StateController since the Memory based one 115 | // does not allow you to consume from the beginning 116 | properties.setProperty("statecontroller.type", "ZOOKEEPER") 117 | properties.setProperty("statecontroller.zk.connect", brokerZkEndpoint) 118 | properties.setProperty("statecontroller.zk.root", TestDefaults.TestPathPrefix) 119 | 120 | // Keeping a low fetch size so that multiple fetches happen 121 | properties.setProperty("topic.fetch.size.bytes", "750") 122 | 123 | properties 124 | } 125 | } 126 | -------------------------------------------------------------------------------- /src/test/scala/com/groupon/dse/util/UtilTest.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.util 34 | 35 | import org.scalatest.FlatSpec 36 | 37 | class UtilTest extends FlatSpec { 38 | 39 | "doActionWithRetry" should "execute action successfully if no errors" in { 40 | val action = (i: Int) => { 41 | i * 2 42 | } 43 | val failureHandler = (t: Throwable) => { 44 | // do nothing 45 | } 46 | val maxRetries = 1 47 | 48 | val ret = Utils.doActionWithRetry[Int]( 49 | { 50 | action(2) 51 | }, { 52 | failureHandler 53 | }, 54 | maxRetries) 55 | assert(ret == 4) 56 | } 57 | 58 | "doActionWithRetry" should "get an exception if error exists after retries" in 59 | intercept[RuntimeException] { 60 | val action = (i: Int) => { 61 | throw new RuntimeException 62 | } 63 | val failureHandler = (t: Throwable) => { 64 | // do nothing 65 | } 66 | val maxRetries = 1 67 | 68 | Utils.doActionWithRetry[Int]( 69 | { 70 | action(2) 71 | }, { 72 | failureHandler 73 | }, 74 | maxRetries) 75 | } 76 | 77 | "doActionWithRetry" should "eventually succeed after 1 retry" in { 78 | val action = (time: Long) => { 79 | val entryTime = System.currentTimeMillis() 80 | if (entryTime - time < 1000) { 81 | throw new RuntimeException 82 | } 83 | } 84 | 85 | val failureHandler = (t: Throwable) => { 86 | Thread.sleep(1000) 87 | } 88 | 89 | val maxRetries = 1 90 | 91 | val inputTime = System.currentTimeMillis() 92 | Utils.doActionWithRetry[Unit]( 93 | { 94 | action(inputTime) 95 | }, { 96 | failureHandler 97 | }, 98 | maxRetries) 99 | } 100 | 101 | "doActionWithRetry" should "eventually succeed after 2 retry" in { 102 | val action = (time: Long) => { 103 | val entryTime = System.currentTimeMillis() 104 | if (entryTime - time < 1000) { 105 | throw new RuntimeException 106 | } 107 | } 108 | 109 | val failureHandler = (t: Throwable) => { 110 | Thread.sleep(500) 111 | } 112 | 113 | val maxRetries = 2 114 | 115 | val inputTime = System.currentTimeMillis() 116 | Utils.doActionWithRetry[Unit]( 117 | { 118 | action(inputTime) 119 | }, { 120 | failureHandler 121 | }, 122 | maxRetries) 123 | } 124 | } 125 | -------------------------------------------------------------------------------- /src/test/scala/com/groupon/dse/zookeeper/ZkClientBuilderTest.scala: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright (c) 2016, Groupon, Inc. 3 | * All rights reserved. 4 | * 5 | * Redistribution and use in source and binary forms, with or without 6 | * modification, are permitted provided that the following conditions are 7 | * met: 8 | * 9 | * Redistributions of source code must retain the above copyright notice, 10 | * this list of conditions and the following disclaimer. 11 | * 12 | * Redistributions in binary form must reproduce the above copyright 13 | * notice, this list of conditions and the following disclaimer in the 14 | * documentation and/or other materials provided with the distribution. 15 | * 16 | * Neither the name of GROUPON nor the names of its contributors may be 17 | * used to endorse or promote products derived from this software without 18 | * specific prior written permission. 19 | * 20 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS 21 | * IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED 22 | * TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A 23 | * PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT 24 | * HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, 25 | * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED 26 | * TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR 27 | * PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF 28 | * LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING 29 | * NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS 30 | * SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. 31 | */ 32 | 33 | package com.groupon.dse.zookeeper 34 | 35 | import com.groupon.dse.testutils.EmbeddedKafka 36 | import org.scalatest.{BeforeAndAfter, FlatSpec} 37 | 38 | class ZkClientBuilderTest extends FlatSpec with BeforeAndAfter { 39 | 40 | val zkConnTimeout = 10000 41 | val zkSessionTimeout = 10000 42 | var embeddedKafka: EmbeddedKafka = _ 43 | var zkConnect: String = _ 44 | 45 | before { 46 | embeddedKafka = new EmbeddedKafka 47 | embeddedKafka.startCluster() 48 | zkConnect = s"127.0.0.1:${embeddedKafka.zkPort}" 49 | } 50 | 51 | after { 52 | embeddedKafka.stopCluster() 53 | } 54 | 55 | "ZkClient" should "successfully create a new node " in { 56 | val zkClient = ZkClientBuilder(zkConnect, zkConnTimeout, zkSessionTimeout) 57 | try { 58 | zkClient.createPersistent("/testpath") 59 | assert(zkClient.exists("/testpath")) 60 | } 61 | finally { 62 | zkClient.close() 63 | } 64 | } 65 | 66 | "ZkClient" should "successfully create and delete a new node " in { 67 | val zkClient = ZkClientBuilder(zkConnect, zkConnTimeout, zkSessionTimeout) 68 | try { 69 | zkClient.createPersistent("/testpath") 70 | zkClient.delete("/testpath") 71 | assert(!zkClient.exists("/testpath")) 72 | } 73 | finally { 74 | zkClient.close() 75 | } 76 | } 77 | } 78 | --------------------------------------------------------------------------------