├── .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 |
--------------------------------------------------------------------------------