├── project
├── build.properties
└── plugins.sbt
├── app-common
└── src
│ ├── main
│ ├── resources
│ │ ├── default_app_setup.conf
│ │ └── logback.xml
│ └── scala
│ │ └── com
│ │ └── supersonic
│ │ ├── util
│ │ ├── SignalingUtil.scala
│ │ ├── URLUtil.scala
│ │ ├── DurationUtil.scala
│ │ └── ActorySystemUtil.scala
│ │ └── main
│ │ ├── AppState.scala
│ │ └── KafkaMirrorAppTemplate.scala
│ └── it
│ └── scala
│ └── com
│ └── supersonic
│ └── integration
│ └── KafkaMirrorIntegrationTemplate.scala
├── file-watcher-app
├── src
│ ├── main
│ │ ├── resources
│ │ │ ├── kafka-mirror-test-app.conf
│ │ │ └── application.conf
│ │ └── scala
│ │ │ └── com
│ │ │ └── supersonic
│ │ │ ├── file_watch
│ │ │ ├── ConfigFileSettingsProcessing.scala
│ │ │ └── DirectoryFilesSource.scala
│ │ │ └── main
│ │ │ ├── DirectoryAppSettings.scala
│ │ │ └── KafkaFileWatcherMirrorApp.scala
│ ├── test
│ │ └── scala
│ │ │ └── com
│ │ │ └── supersonic
│ │ │ └── file_watch
│ │ │ ├── ConfigFileSettingsProcessingTest.scala
│ │ │ ├── TestDirectory.scala
│ │ │ ├── DirectoryFilesSourceTestUtil.scala
│ │ │ └── DirectoryFilesSourceTest.scala
│ └── it
│ │ └── scala
│ │ └── com
│ │ └── supersonic
│ │ └── integration
│ │ └── DirectoryBackedKafkaMirrorTest.scala
└── README.md
├── consul-app
├── src
│ ├── main
│ │ ├── resources
│ │ │ ├── kafka-mirror-test-app.conf
│ │ │ └── application.conf
│ │ └── scala
│ │ │ └── com
│ │ │ └── supersonic
│ │ │ ├── consul
│ │ │ └── ConsulSettingsFlow.scala
│ │ │ └── main
│ │ │ ├── settings.scala
│ │ │ └── KafkaConsulMirrorApp.scala
│ ├── it
│ │ └── scala
│ │ │ └── com
│ │ │ └── supersonic
│ │ │ └── integration
│ │ │ └── ConsulBackedKafkaMirror.scala
│ └── test
│ │ └── scala
│ │ └── com
│ │ └── supersonic
│ │ └── consul
│ │ └── ConsulSettingsFlowTest.scala
└── README.md
├── .gitignore
├── core
└── src
│ ├── it
│ ├── resources
│ │ ├── reference.conf
│ │ └── logback-test.xml
│ └── scala
│ │ └── com
│ │ └── supersonic
│ │ └── kafka_mirror
│ │ ├── KafkaMirrorIntegrationTest.scala
│ │ └── AkkaStreamsKafkaIntegrationSpec.scala
│ ├── test
│ ├── resources
│ │ ├── reference.conf
│ │ └── logback-test.xml
│ └── scala
│ │ └── com
│ │ └── supersonic
│ │ └── kafka_mirror
│ │ ├── ExternalKafkaMirrorSettingsTest.scala
│ │ ├── SettingsProcessingFlowTest.scala
│ │ ├── MirrorManagerFlowTest.scala
│ │ ├── TestUtil.scala
│ │ ├── StaggeredEventsFilterGateTest.scala
│ │ ├── MirrorManagerHelpersTest.scala
│ │ ├── SettingsProcessingFlowHelpersTest.scala
│ │ └── KafkaMirrorTest.scala
│ └── main
│ └── scala
│ └── com
│ └── supersonic
│ ├── util
│ └── ValidatedUtil.scala
│ └── kafka_mirror
│ ├── mirror_commands.scala
│ ├── StaggeredEventsFilterGate.scala
│ ├── mirror_settings.scala
│ ├── SettingsProcessingFlow.scala
│ ├── KafkaMirror.scala
│ └── MirrorManager.scala
├── LICENSE.txt
├── .github
└── workflows
│ ├── clean.yml
│ └── ci.yml
└── README.md
/project/build.properties:
--------------------------------------------------------------------------------
1 | sbt.version = 1.5.0
2 |
--------------------------------------------------------------------------------
/app-common/src/main/resources/default_app_setup.conf:
--------------------------------------------------------------------------------
1 | kafka-mirror-settings {
2 | port = 8080
3 | port = ${?KAFKA_SNOW_WHITE_HEALTHCHECK_PORT}
4 | }
5 |
--------------------------------------------------------------------------------
/file-watcher-app/src/main/resources/kafka-mirror-test-app.conf:
--------------------------------------------------------------------------------
1 | kafka-mirror-settings = {
2 | port = 8080
3 | mirrors-directory = "/tmp/kafka-mirror-app"
4 | }
5 |
--------------------------------------------------------------------------------
/file-watcher-app/src/main/resources/application.conf:
--------------------------------------------------------------------------------
1 | include "default_app_setup"
2 |
3 | kafka-mirror-settings = {
4 | mirrors-directory = ${?KAFKA_SNOW_WHITE_MIRRORS_DIR}
5 | }
6 |
--------------------------------------------------------------------------------
/consul-app/src/main/resources/kafka-mirror-test-app.conf:
--------------------------------------------------------------------------------
1 | kafka-mirror-settings = {
2 | port = 8080
3 | consul-settings = {
4 | url = "http://localhost:8500"
5 | root-key = "kafkaMirrors"
6 | stagger-time = 3 seconds
7 | }
8 | }
9 |
--------------------------------------------------------------------------------
/.gitignore:
--------------------------------------------------------------------------------
1 | *.class
2 | *.log
3 |
4 | # sbt specific
5 | .cache
6 | .history
7 | .lib/
8 | dist/*
9 | target/
10 | lib_managed/
11 | src_managed/
12 | project/boot/
13 | project/plugins/project/
14 |
15 | # Scala-IDE specific
16 | .scala_dependencies
17 | .worksheet
18 |
19 | local.*
--------------------------------------------------------------------------------
/app-common/src/main/scala/com/supersonic/util/SignalingUtil.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.util
2 |
3 | import sun.misc.Signal
4 |
5 | object SignalingUtil {
6 | def registerHandler(block: => Unit): Unit = {
7 | val _ = Signal.handle(new Signal("INT"), _ => block)
8 | }
9 | }
10 |
--------------------------------------------------------------------------------
/consul-app/src/main/resources/application.conf:
--------------------------------------------------------------------------------
1 | include "default_app_setup"
2 |
3 | kafka-mirror-settings = {
4 | consul-settings = {
5 | url = "http://localhost:8500"
6 | url = ${?KAFKA_SNOW_WHITE_CONSUL_URL}
7 |
8 | root-key = ${?KAFKA_SNOW_WHITE_CONSUL_ROOT_KEY}
9 |
10 | stagger-time = 3 seconds
11 | }
12 | }
13 |
--------------------------------------------------------------------------------
/core/src/it/resources/reference.conf:
--------------------------------------------------------------------------------
1 | akka {
2 | loggers = ["akka.event.slf4j.Slf4jLogger"]
3 | loglevel = "DEBUG"
4 | logging-filter = "akka.event.slf4j.Slf4jLoggingFilter"
5 | logger-startup-timeout = 15s
6 | test {
7 | single-expect-default = 10s
8 | }
9 | kafka.consumer {
10 | max-wakeups = 3
11 | }
12 |
13 | stream.materializer.debug.fuzzing-mode = on
14 | }
15 |
--------------------------------------------------------------------------------
/core/src/test/resources/reference.conf:
--------------------------------------------------------------------------------
1 | akka {
2 | loggers = ["akka.event.slf4j.Slf4jLogger"]
3 | loglevel = "DEBUG"
4 | logging-filter = "akka.event.slf4j.Slf4jLoggingFilter"
5 | logger-startup-timeout = 15s
6 | test {
7 | single-expect-default = 10s
8 | }
9 | kafka.consumer {
10 | max-wakeups = 3
11 | }
12 |
13 | stream.materializer.debug.fuzzing-mode = on
14 | }
15 |
--------------------------------------------------------------------------------
/core/src/it/resources/logback-test.xml:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 | %d{HH:mm:ss.SSS} %-5level %logger{36} - %msg%n%rEx
5 |
6 |
7 |
8 |
9 |
10 |
11 |
12 |
13 |
--------------------------------------------------------------------------------
/core/src/test/resources/logback-test.xml:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 | %d{HH:mm:ss.SSS} %-5level %logger{36} - %msg%n%rEx
5 |
6 |
7 |
8 |
9 |
10 |
11 |
12 |
13 |
--------------------------------------------------------------------------------
/app-common/src/main/resources/logback.xml:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 | %d{HH:mm:ss.SSS} %-5level %logger{36} - %msg%n%rEx
5 |
6 |
7 |
8 |
9 |
10 |
11 |
12 |
13 |
--------------------------------------------------------------------------------
/project/plugins.sbt:
--------------------------------------------------------------------------------
1 | logLevel := Level.Warn
2 |
3 | resolvers += Classpaths.sbtPluginReleases
4 |
5 | addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.14.6")
6 |
7 | addSbtPlugin("com.typesafe.sbt" % "sbt-git" % "0.9.3")
8 |
9 | addSbtPlugin("com.eed3si9n" % "sbt-buildinfo" % "0.7.0")
10 |
11 | addSbtPlugin("com.geirsson" % "sbt-ci-release" % "1.5.7")
12 |
13 | addSbtPlugin("com.codecommit" % "sbt-github-actions" % "0.10.1")
14 |
--------------------------------------------------------------------------------
/LICENSE.txt:
--------------------------------------------------------------------------------
1 | Copyright 2018 ironSource (https://www.ironsrc.com)
2 |
3 | Licensed under the Apache License, Version 2.0 (the "License");
4 | you may not use this file except in compliance with the License.
5 | You may obtain a copy of the License at
6 |
7 | http://www.apache.org/licenses/LICENSE-2.0
8 |
9 | Unless required by applicable law or agreed to in writing, software
10 | distributed under the License is distributed on an "AS IS" BASIS,
11 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12 | See the License for the specific language governing permissions and
13 | limitations under the License.
--------------------------------------------------------------------------------
/app-common/src/main/scala/com/supersonic/util/URLUtil.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.util
2 |
3 | import java.net.URL
4 | import com.typesafe.config.Config
5 | import net.ceedubs.ficus.readers.ValueReader
6 | import spray.json._
7 |
8 | object URLUtil {
9 |
10 | implicit object UrlValueReader extends ValueReader[URL] {
11 | def read(config: Config, path: String) = {
12 | new URL(config.getString(path))
13 | }
14 | }
15 |
16 | implicit object URLFormat extends JsonFormat[URL] {
17 | def write(obj: URL) = JsString(obj.toString)
18 |
19 | def read(json: JsValue) = json match {
20 | case JsString(url) => new URL(url)
21 | case other => deserializationError(s"Expected URL, got: $other")
22 | }
23 | }
24 | }
25 |
--------------------------------------------------------------------------------
/app-common/src/main/scala/com/supersonic/util/DurationUtil.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.util
2 |
3 | import spray.json._
4 | import scala.concurrent.duration.{Duration, FiniteDuration}
5 |
6 | object DurationUtil {
7 |
8 | implicit object FiniteDurationFormat extends JsonFormat[FiniteDuration] {
9 | def write(fd: FiniteDuration) = JsString(fd.toString)
10 |
11 | def read(json: JsValue) = {
12 | def error() = deserializationError(s"Expected a finite duration but got: $json")
13 |
14 | json match {
15 | case JsString(duration) => Duration(duration) match {
16 | case finite: FiniteDuration => finite
17 | case _ => error()
18 | }
19 | case _ => error()
20 | }
21 | }
22 | }
23 | }
24 |
--------------------------------------------------------------------------------
/app-common/src/main/scala/com/supersonic/util/ActorySystemUtil.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.util
2 |
3 | import akka.actor.ActorSystem
4 | import scala.concurrent.Await
5 | import scala.concurrent.duration.Duration
6 |
7 | object ActorSystemUtil {
8 | /** Creates an actor system and runs the given function with it.
9 | *
10 | * Terminating the actor system when the function completes (meaning that you shouldn't use
11 | * Futures in this context as they will be returned almost immediately and the system will be shutdown).
12 | */
13 | def withActorSystem[A](name: String)
14 | (f: ActorSystem => A): A = {
15 | val actorSystem = ActorSystem(name)
16 |
17 | try f(actorSystem)
18 | finally {
19 | val _ = Await.result(actorSystem.terminate(), Duration.Inf)
20 | }
21 | }
22 | }
23 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/supersonic/util/ValidatedUtil.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.util
2 |
3 | import cats.data.{NonEmptyList, Validated}
4 | import com.typesafe.config.Config
5 | import net.ceedubs.ficus.Ficus._
6 | import net.ceedubs.ficus.readers.ValueReader
7 | import scala.util.Try
8 |
9 | object ValidatedUtil {
10 | type ValidOrErrors[A] = Validated[NonEmptyList[Throwable], A]
11 |
12 | implicit class ValidatedOps[E, A](validated: Validated[E, A]) {
13 | def flatten[B](implicit ev: A <:< Validated[E, B]) = validated.map(ev).andThen(identity)
14 | }
15 |
16 | implicit def validatedValueReader[A](implicit reader: ValueReader[A]): ValueReader[ValidOrErrors[A]] =
17 | new ValueReader[ValidOrErrors[A]] {
18 | override def read(config: Config, path: String) = Validated
19 | .fromTry(config.as[Try[A]](path))
20 | .toValidatedNel
21 | }
22 | }
23 |
--------------------------------------------------------------------------------
/core/src/test/scala/com/supersonic/kafka_mirror/ExternalKafkaMirrorSettingsTest.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.kafka_mirror
2 |
3 | import akka.actor.ActorSystem
4 | import org.scalatest.{Matchers, WordSpec}
5 |
6 | class ExternalKafkaMirrorSettingsTest extends WordSpec with Matchers {
7 | implicit val system = ActorSystem("ExternalKafkaMirrorSettingsTest")
8 |
9 | "The Kafka mirrors settings hashing function" should {
10 | val hashFunction = TestUtil.runnableSettings("test")
11 | .toKafkaMirrorSettings.get.hashKey
12 |
13 | def bytes = Array(1, 2, 3, 4).map(_.toByte)
14 |
15 | val bytes1 = bytes
16 | val bytes2 = bytes
17 |
18 | "not use object identity for hashing" in {
19 | bytes1 shouldNot be theSameInstanceAs bytes2
20 | hashFunction(bytes1) shouldEqual hashFunction(bytes2)
21 | }
22 |
23 | "compute the hash using the byte contents" in {
24 | hashFunction(bytes) shouldEqual 1698034171
25 | }
26 |
27 | "produce positive numbers" in {
28 | hashFunction(Array(1, 2).map(_.toByte)) should be >= 0
29 | }
30 |
31 | "not fail if the input is 'null'" in {
32 | noException should be thrownBy hashFunction(null)
33 | }
34 | }
35 | }
36 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/supersonic/kafka_mirror/mirror_commands.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.kafka_mirror
2 |
3 | import akka.kafka.scaladsl.Consumer.Control
4 |
5 | case class MirrorID(value: String)
6 |
7 | /** Commands to manage the state of Kafka mirrors in the system. */
8 | sealed trait MirrorCommand extends Product with Serializable
9 |
10 | object MirrorCommand {
11 | /** Indicates that a mirror with the given settings under the specified ID should be started.
12 | * This can potentially mean that an old mirror with the same ID should be stopped first.
13 | */
14 | case class Start(id: MirrorID,
15 | mirrorSettings: ExternalKafkaMirrorSettings) extends MirrorCommand
16 |
17 | /** Indicates that the mirror with the given ID should be stopped. */
18 | case class Stop(id: MirrorID) extends MirrorCommand
19 |
20 | /** Signals that the state of the mirrors should be verified.
21 | * E.g., check whether some mirrors were terminated unexpectedly.
22 | */
23 | case object VerifyState extends MirrorCommand
24 |
25 | case object Shutdown extends MirrorCommand
26 | }
27 |
28 | /** Encapsulates the data needed to maintain a running mirror. */
29 | case class RunningMirror(control: Control, mirrorSettings: ExternalKafkaMirrorSettings)
30 |
31 |
--------------------------------------------------------------------------------
/app-common/src/main/scala/com/supersonic/main/AppState.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.main
2 |
3 | import com.supersonic.kafka_mirror.{ExternalKafkaMirrorSettings, MirrorID}
4 | import com.typesafe.config.{Config, ConfigFactory, ConfigRenderOptions}
5 | import fommil.sjs.FamilyFormats
6 | import shapeless.Typeable
7 | import spray.json._
8 |
9 | case class AppState[AppSettings](appVersion: String,
10 | commitHash: String,
11 | settings: AppSettings,
12 | mirrors: Either[String, List[(MirrorID, ExternalKafkaMirrorSettings)]])
13 |
14 | object AppState extends DefaultJsonProtocol with FamilyFormats {
15 | // boilerplate because of bugs in implicit resolution
16 | // see here: https://github.com/fommil/spray-json-shapeless/issues/22
17 | implicit def either[A: JsonFormat, B: JsonFormat] =
18 | DefaultJsonProtocol.eitherFormat[A, B]
19 |
20 | implicit val configFormat = new JsonFormat[Config] {
21 | def write(config: Config): JsValue = JsonParser(config.root.render(ConfigRenderOptions.concise))
22 |
23 | def read(json: JsValue): Config = ConfigFactory.parseString(json.toString)
24 | }
25 |
26 | def format[AppSettings: JsonFormat : Typeable]: RootJsonFormat[AppState[AppSettings]] =
27 | implicitly[RootJsonFormat[AppState[AppSettings]]]
28 | }
29 |
--------------------------------------------------------------------------------
/file-watcher-app/src/main/scala/com/supersonic/file_watch/ConfigFileSettingsProcessing.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.file_watch
2 |
3 | import java.nio.file.{Files, Path}
4 | import akka.NotUsed
5 | import akka.stream.scaladsl.Flow
6 | import com.google.common.io.{Files => GFiles}
7 | import com.supersonic.kafka_mirror.MirrorID
8 |
9 | object ConfigFileSettingsProcessing {
10 | val confFileSuffix = "conf"
11 |
12 | /** Creates a flow that takes a the contents of a directory and filters it down
13 | * to only the '.conf' file in it. Each '.conf' file's name becomes a [[MirrorID]].
14 | *
15 | * E.g., if the directory contains the following files:
16 | * - a.conf
17 | * - b.txt
18 | * - c.conf
19 | *
20 | * Then the resulting mirrors will have the following IDs:
21 | * - a
22 | * - c
23 | */
24 | def flow: Flow[Map[Path, Option[String]], Map[MirrorID, Option[String]], NotUsed] =
25 | Flow[Map[Path, Option[String]]].map(convertPathsToIDs)
26 |
27 | private[file_watch] def convertPathsToIDs[A](data: Map[Path, A]): Map[MirrorID, A] = {
28 | data.flatMap { case (path, value) =>
29 | val fileName = path.toFile.getName
30 |
31 | val id = if (!Files.isDirectory(path) &&
32 | confFileSuffix == GFiles.getFileExtension(fileName)) Some {
33 | GFiles.getNameWithoutExtension(fileName)
34 | }
35 | else None
36 |
37 | id.map(MirrorID(_) -> value)
38 | }
39 | }
40 | }
41 |
--------------------------------------------------------------------------------
/file-watcher-app/src/test/scala/com/supersonic/file_watch/ConfigFileSettingsProcessingTest.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.file_watch
2 |
3 | import java.nio.file.Paths
4 | import com.supersonic.file_watch.ConfigFileSettingsProcessing._
5 | import com.supersonic.kafka_mirror.TestUtil.id
6 | import org.scalatest.{Matchers, WordSpec}
7 |
8 | class ConfigFileSettingsProcessingTest extends WordSpec with Matchers {
9 |
10 | "Converting paths to IDs" should {
11 | def path(name: String) = Paths.get("/a/b", name)
12 |
13 | "use the file-names of '.conf' files" in {
14 | val data = Map(
15 | path("a.conf") -> 1,
16 | path("b.conf") -> 2,
17 | path("c.conf") -> 3)
18 |
19 | val expected = Map(
20 | id("a") -> 1,
21 | id("b") -> 2,
22 | id("c") -> 3)
23 |
24 | convertPathsToIDs(data) shouldBe expected
25 | }
26 |
27 | "ignore directories" in {
28 | val data = Map(
29 | path("a.conf") -> 1,
30 | path("b/") -> 2,
31 | path("c.conf") -> 3)
32 |
33 | val expected = Map(
34 | id("a") -> 1,
35 | id("c") -> 3)
36 |
37 | convertPathsToIDs(data) shouldBe expected
38 | }
39 |
40 | "ignore non-conf files" in {
41 | val data = Map(
42 | path("a.conf") -> 1,
43 | path("b.txt") -> 2,
44 | path("c.conf") -> 3,
45 | path("d.txt") -> 4)
46 |
47 | val expected = Map(
48 | id("a") -> 1,
49 | id("c") -> 3)
50 |
51 | convertPathsToIDs(data) shouldBe expected
52 | }
53 | }
54 | }
55 |
--------------------------------------------------------------------------------
/file-watcher-app/src/test/scala/com/supersonic/file_watch/TestDirectory.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.file_watch
2 |
3 | import java.nio.file.{Files, Path, Paths}
4 | import java.util.Comparator
5 | import com.supersonic.file_watch.TestDirectory._
6 |
7 | /** A helper trait that facilitates working with temporary directories for test purposes. */
8 | trait TestDirectory {
9 | def withDirectory[A](f: FileHelper => A): A = {
10 | val tempDir = System.getProperty("java.io.tmpdir")
11 | val dir = Paths.get(tempDir, s"test-folder-${System.nanoTime()}")
12 |
13 | Files.createDirectory(dir)
14 |
15 | try f(new FileHelper(dir))
16 | finally delete(dir)
17 | }
18 | }
19 |
20 | object TestDirectory {
21 | class FileHelper(val dir: Path) {
22 | def path(name: String) = dir.resolve(name)
23 |
24 | def createFile(name: String, content: String): Unit = {
25 | val _ = Files.write(path(name), content.getBytes)
26 | }
27 |
28 | def deleteFile(name: String): Unit = delete(path(name))
29 |
30 | def modifyFile(name: String, content: String): Unit =
31 | createFile(name, content) // creating an existing file just overwrites it
32 |
33 | def createDir(name: String): Unit = {
34 | val _ = Files.createDirectory(path(name))
35 | }
36 |
37 | def deleteDir(name: String): Unit = delete(path(name))
38 | }
39 |
40 | private def delete(path: Path): Unit = {
41 | import scala.collection.JavaConverters._
42 |
43 | Files.walk(path)
44 | .sorted(Comparator.reverseOrder())
45 | .iterator().asScala
46 | .foreach(Files.delete)
47 | }
48 | }
49 |
--------------------------------------------------------------------------------
/file-watcher-app/README.md:
--------------------------------------------------------------------------------
1 | # The File-watcher App
2 |
3 | The Kafka Snow White file-watcher app lets you configure the Kafka mirrors via files in the file-system.
4 |
5 | The app is provided as an executable JAR (see how to obtain it in the [main README](../README.md)) and can be started with the following command:
6 | ```
7 | export KAFKA_SNOW_WHITE_MIRRORS_DIR=/some/dir/with/mirrors
8 | java -jar file-watcher-app.jar
9 | ```
10 |
11 | (where we assume that the app JAR was renamed to `file-watcher-app.jar`)
12 |
13 | This will start the service and will look for mirror definitions in the `/some/dir/with/mirrors` directory.
14 |
15 | Alternatively, we can start the application with a configuration file instead of using environment variables, like so:
16 | ```
17 | java -Dconfig.file=path/to/some-conf-file.conf -jar file-watcher-app.jar
18 | ```
19 |
20 | Where `some-conf-file.conf` has the following content (in [HOCON format](https://github.com/lightbend/config/blob/master/HOCON.md)):
21 | ```
22 | include classpath("application")
23 |
24 | kafka-mirror-settings {
25 | mirrors-directory = "/some/dir/with/mirrors"
26 | }
27 | ```
28 |
29 | Each mirror definition needs to reside in a single file with a `.conf` suffix (see for the configuration format [main README](../README.md)). So if we have this file structure:
30 | ```
31 | /some/dir/with/mirrors
32 | |- mirror1.conf
33 | |- mirror2.conf
34 | ```
35 |
36 | Once the service is started we will have two mirrors named `mirror1` and `mirror2` running.
37 |
38 | The service will listen to directory changes and if any of the mirror definitions are updated/removed/added the service will dynamically update its configuration accordingly.
39 |
--------------------------------------------------------------------------------
/file-watcher-app/src/test/scala/com/supersonic/file_watch/DirectoryFilesSourceTestUtil.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.file_watch
2 |
3 | import java.nio.file.Path
4 | import akka.NotUsed
5 | import akka.actor.ActorSystem
6 | import akka.event.LoggingAdapter
7 | import akka.stream.Materializer
8 | import akka.stream.scaladsl.{Keep, Source}
9 | import akka.stream.testkit.TestSubscriber.Probe
10 | import akka.stream.testkit.scaladsl.TestSink
11 | import com.supersonic.file_watch.TestDirectory.FileHelper
12 | import scala.concurrent.duration._
13 |
14 | trait DirectoryFilesSourceTestUtil extends TestDirectory {
15 | def withDirectoryProbe[A](f: (FileHelper, Probe[Map[Path, Option[String]]]) => A)
16 | (implicit logger: LoggingAdapter,
17 | system: ActorSystem,
18 | materializer: Materializer): A =
19 | withDirectoryProbe[Map[Path, Option[String]], A](identity)(f)
20 |
21 | def withDirectoryProbe[A, B](transform: Source[Map[Path, Option[String]], NotUsed] => Source[A, NotUsed])
22 | (f: (FileHelper, Probe[A]) => B)
23 | (implicit logger: LoggingAdapter,
24 | system: ActorSystem,
25 | materializer: Materializer): B = {
26 | withDirectory { fileHelper =>
27 | val source = transform {
28 | new DirectoryFilesSource()(logger)(fileHelper.dir, 5.seconds, 1000)
29 | }
30 |
31 | val probe = source.toMat(TestSink.probe)(Keep.right).run()
32 |
33 | try f(fileHelper, probe)
34 | finally {
35 | val _ = probe.cancel()
36 | }
37 | }
38 | }
39 | }
40 |
--------------------------------------------------------------------------------
/file-watcher-app/src/it/scala/com/supersonic/integration/DirectoryBackedKafkaMirrorTest.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.integration
2 |
3 | import akka.NotUsed
4 | import akka.actor.ActorSystem
5 | import akka.stream.testkit.TestSubscriber.Probe
6 | import akka.testkit.TestKit
7 | import com.supersonic.file_watch.{ConfigFileSettingsProcessing, DirectoryFilesSourceTestUtil}
8 | import com.supersonic.integration.KafkaMirrorIntegrationTemplate.MirrorConfigManager
9 | import com.supersonic.kafka_mirror.{MirrorID, RunningMirror}
10 | import org.scalatest.Matchers
11 |
12 | class DirectoryBackedKafkaMirrorTest extends TestKit(ActorSystem("DirectoryBackedKafkaMirrorTest"))
13 | with KafkaMirrorIntegrationTemplate
14 | with DirectoryFilesSourceTestUtil
15 | with Matchers {
16 | protected def configSourceName: String = "a directory"
17 |
18 | protected type ConfigMat = NotUsed
19 |
20 | protected def withProbe[B](transform: MirrorConfigSource => RunningMirrorSource)
21 | (f: (Probe[Map[MirrorID, RunningMirror]], MirrorConfigManager) => B): B = {
22 | withDirectoryProbe(source => transform(source.via(ConfigFileSettingsProcessing.flow))) { (fileHelper, probe) =>
23 | val mirrorConfigManager = new MirrorConfigManager {
24 | def addMirror(name: String, settings: String) = fileHelper.createFile(s"$name.conf", settings)
25 |
26 | def deleteMirror(name: String) = fileHelper.deleteFile(s"$name.conf")
27 | }
28 |
29 | f(probe, mirrorConfigManager)
30 | }
31 | }
32 |
33 | runKafkaMirrorIntegration()
34 | }
35 |
--------------------------------------------------------------------------------
/consul-app/src/it/scala/com/supersonic/integration/ConsulBackedKafkaMirror.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.integration
2 |
3 | import akka.actor.ActorSystem
4 | import akka.stream.testkit.TestSubscriber.Probe
5 | import akka.testkit.TestKit
6 | import com.supersonic.consul.{CancellationToken, ConsulIntegrationSpec, ConsulSettingsFlow}
7 | import com.supersonic.integration.KafkaMirrorIntegrationTemplate.MirrorConfigManager
8 | import com.supersonic.kafka_mirror._
9 | import org.scalatest.Matchers
10 |
11 | class ConsulBackedKafkaMirror extends TestKit(ActorSystem("ConsulBackedKafkaMirror"))
12 | with KafkaMirrorIntegrationTemplate
13 | with ConsulIntegrationSpec
14 | with Matchers {
15 |
16 | protected def configSourceName: String = "Consul"
17 |
18 | protected type ConfigMat = CancellationToken
19 |
20 | protected def withProbe[B](transform: MirrorConfigSource => RunningMirrorSource)
21 | (f: (Probe[Map[MirrorID, RunningMirror]], MirrorConfigManager) => B): B = {
22 | val root = "foo"
23 |
24 | val mirrorConfigManager = new MirrorConfigManager {
25 | def addMirror(name: String, settings: String): Unit = {
26 | val _ = keyValueClient.putValue(s"$root/$name", settings)
27 | }
28 |
29 | def deleteMirror(name: String): Unit = keyValueClient.deleteKey(s"$root/$name")
30 | }
31 |
32 |
33 | def withProbe(probe: Probe[Map[MirrorID, RunningMirror]]) = f(probe, mirrorConfigManager)
34 |
35 | withStreamFromConsulProbe(root, consul)(
36 | source => transform(source.via(ConsulSettingsFlow(root))))(withProbe)
37 | }
38 |
39 | runKafkaMirrorIntegration()
40 | }
41 |
--------------------------------------------------------------------------------
/consul-app/README.md:
--------------------------------------------------------------------------------
1 | # The Consul App
2 |
3 | The Kafka Snow White consul app lets you configure the Kafka mirrors via keys in the [Consul](https://www.consul.io/) KV-store.
4 |
5 | The app is provided as an executable JAR (see how to obtain it in the [main README](../README.md)) and can be started with the following command:
6 | ```
7 | export KAFKA_SNOW_WHITE_CONSUL_URL=http://localhost:8500
8 | export KAFKA_SNOW_WHITE_CONSUL_ROOT_KEY=some/consul/key
9 |
10 | java -jar consul-app.jar
11 | ```
12 |
13 | (where we assume that the app JAR was renamed to `consul-app.jar`)
14 |
15 | This will start the service and will look for mirror definitions in the `some/consul/key` key in the Consul instance that resides in `http://localhost:8500`.
16 |
17 | Alternatively, we can start the application with a configuration file instead of using environment variables, like so:
18 | ```
19 | java -Dconfig.file=path/to/some-conf-file.conf -jar consul-app.jar
20 | ```
21 |
22 | Where `some-conf-file.conf` has the following content (in [HOCON format](https://github.com/lightbend/config/blob/master/HOCON.md)):
23 | ```
24 | include classpath("application")
25 |
26 | kafka-mirror-settings = {
27 | consul-settings = {
28 | url = "http://localhost:8500"
29 |
30 | root-key = some/consul/key
31 | }
32 | }
33 | ```
34 |
35 | Each mirror definition needs to reside in a single key under the root key specifid above (see for the configuration format [main README](../README.md)). So if we have key structure:
36 | ```
37 | some/consul/key
38 | |- mirror1
39 | |- mirror2
40 | ```
41 |
42 | Once the service is started we will have two mirrors named `mirror1` and `mirror2` running.
43 |
44 | The service will listen to changes in the root key and if any of the mirror definitions are updated/removed/added the service will dynamically update its configuration accordingly.
45 |
--------------------------------------------------------------------------------
/consul-app/src/test/scala/com/supersonic/consul/ConsulSettingsFlowTest.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.consul
2 |
3 | import com.supersonic.consul.ConsulSettingsFlow._
4 | import com.supersonic.kafka_mirror.TestUtil.id
5 | import org.scalatest.{Matchers, WordSpec}
6 |
7 | class ConsulSettingsFlowTest extends WordSpec with Matchers {
8 |
9 | "Converting paths to IDs" should {
10 |
11 | "drop the root prefix" in {
12 | val data = Map(
13 | "foo/bar/a" -> 1,
14 | "foo/bar/b" -> 2,
15 | "foo/bar/c" -> 3)
16 |
17 | val expected = Map(
18 | id("a") -> 1,
19 | id("b") -> 2,
20 | id("c") -> 3)
21 |
22 | convertPathsToIDs("foo/bar")(data) shouldBe expected
23 | }
24 |
25 | "ignore keys not under the root" in {
26 | val data = Map(
27 | "foo/bar/a" -> 1,
28 | "foo/qux/b" -> 2,
29 | "baz/c" -> 3)
30 |
31 | val expected = Map(id("a") -> 1)
32 |
33 | convertPathsToIDs("foo/bar")(data) shouldBe expected
34 | }
35 |
36 | "ignore deeply nested keys under the root" in {
37 | val data = Map(
38 | "foo/bar/a/b" -> 1,
39 | "foo/bar/c/d/e" -> 2)
40 |
41 | convertPathsToIDs("foo/bar")(data) shouldBe empty
42 | }
43 |
44 | "remove trailing slashes" in {
45 | val data = Map("foo/bar/a/" -> 1)
46 |
47 | convertPathsToIDs("foo/bar")(data) shouldBe Map(id("a") -> 1)
48 | }
49 |
50 | "ignore the trailing slash on the root" in {
51 | val data = Map("foo/bar/a" -> 1)
52 |
53 | convertPathsToIDs("foo/bar/")(data) shouldBe Map(id("a") -> 1)
54 | }
55 |
56 | "ignore the root key" in {
57 | val data1 = Map("foo/bar/" -> 1)
58 | val data2 = Map("foo/bar" -> 1)
59 |
60 | convertPathsToIDs("foo/bar")(data1) shouldBe empty
61 | convertPathsToIDs("foo/bar")(data2) shouldBe empty
62 | }
63 | }
64 | }
65 |
--------------------------------------------------------------------------------
/consul-app/src/main/scala/com/supersonic/consul/ConsulSettingsFlow.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.consul
2 |
3 | import akka.NotUsed
4 | import akka.stream.scaladsl.Flow
5 | import com.supersonic.kafka_mirror.MirrorID
6 |
7 | object ConsulSettingsFlow {
8 | /** Creates a flow that maps raw data coming from a Consul keys into mirror-IDs.
9 | * The keys are all based at the given root, so that the resulting IDs are all under the root
10 | * and the root prefix is removed. See [[convertPathsToIDs]].
11 | */
12 | def apply(root: String): Flow[Map[String, Option[String]], Map[MirrorID, Option[String]], NotUsed] =
13 | Flow[Map[String, Option[String]]].map(convertPathsToIDs(root))
14 |
15 | val sep = "/"
16 |
17 | /** Converts hierarchical data coming from a key in Consul into flat data that is keyed
18 | * by unique IDs. The IDs are all of the keys that are under the given root key.
19 | *
20 | * E.g., if the root is 'foo/bar' then a map with:
21 | * - foo/bar/a -> 1
22 | * - foo/bar/b -> 2
23 | * - foo/bar/c -> 3
24 | *
25 | * Will be converted to:
26 | * - a -> 1
27 | * - b -> 2
28 | * - c -> 3
29 | */
30 | private[consul] def convertPathsToIDs[A](root: String)
31 | (data: Map[String, A]): Map[MirrorID, A] = {
32 | // fetches the paths that are directly beneath the root.
33 | def fetchUnderRoot(path: String): Option[List[String]] = {
34 | val parts = getParts(path)
35 | val rootParts = getParts(root)
36 |
37 | if (!parts.forall(_.isEmpty) && parts.startsWith(rootParts)) Some {
38 | parts.drop(rootParts.size)
39 | } else None
40 | }
41 |
42 | data.flatMap { case (path, value) =>
43 | val maybeID = fetchUnderRoot(path).flatMap {
44 | case List(id) => Some(id)
45 | case _ => None
46 | }
47 |
48 | maybeID.map(MirrorID(_) -> value)
49 | }
50 | }
51 |
52 | private def getParts(path: String) = path.split(sep).toList.filterNot(_.isEmpty)
53 | }
54 |
--------------------------------------------------------------------------------
/consul-app/src/main/scala/com/supersonic/main/settings.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.main
2 |
3 | import java.net.URL
4 | import akka.event.LoggingAdapter
5 | import com.orbitz.consul.Consul
6 | import com.supersonic.main.KafkaMirrorAppTemplate.BaseAppSettings
7 | import spray.json.RootJsonFormat
8 | import scala.concurrent.duration.FiniteDuration
9 | import net.ceedubs.ficus.readers.ValueReader
10 | import shapeless.Typeable
11 |
12 | case class ConsulAppSettings(port: Int,
13 | consulSettings: ConsulSettings,
14 | stateVerificationInterval: Option[FiniteDuration]) extends BaseAppSettings
15 |
16 | case class ConsulSettings(url: URL,
17 | rootKey: String,
18 | staggerTime: FiniteDuration)
19 |
20 | object ConsulSettings {
21 | def verifySettings(consulSettings: ConsulSettings, consul: Consul)
22 | (implicit logger: LoggingAdapter): Unit = {
23 | val key = consulSettings.rootKey
24 | val withTrailingSlash = if (key.endsWith("/")) key else s"$key/"
25 | val result = Option(consul.keyValueClient().getValues(withTrailingSlash))
26 |
27 | if (result.isEmpty)
28 | logger.warning(s"The provided Consul root key: [$key] is either missing or not a directory")
29 | }
30 | }
31 |
32 | object ConsulAppSettings {
33 | // the imports below are actually used
34 |
35 | import com.supersonic.util.URLUtil._
36 |
37 | implicit val valueReader = {
38 | import net.ceedubs.ficus.readers.namemappers.implicits.hyphenCase
39 | import net.ceedubs.ficus.Ficus._
40 | import net.ceedubs.ficus.readers.ArbitraryTypeReader._
41 |
42 | implicitly[ValueReader[ConsulAppSettings]]
43 | }
44 |
45 | implicit val format: RootJsonFormat[ConsulAppSettings] = {
46 | import com.supersonic.util.DurationUtil._
47 | import fommil.sjs.FamilyFormats._
48 | shapeless.cachedImplicit
49 | }
50 |
51 | // needed to help deriving JSON formats
52 | val typeable: Typeable[ConsulAppSettings] = implicitly[Typeable[ConsulAppSettings]]
53 | }
54 |
--------------------------------------------------------------------------------
/file-watcher-app/src/main/scala/com/supersonic/main/DirectoryAppSettings.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.main
2 |
3 | import java.nio.file.{Files, Path, Paths}
4 | import akka.event.LoggingAdapter
5 | import com.supersonic.main.KafkaMirrorAppTemplate.BaseAppSettings
6 | import spray.json.RootJsonFormat
7 | import scala.concurrent.duration.FiniteDuration
8 | import net.ceedubs.ficus.readers.ValueReader
9 | import shapeless.Typeable
10 | import scala.util.Try
11 |
12 | case class DirectoryAppSettings(port: Int,
13 | mirrorsDirectory: String,
14 | pollInterval: Option[FiniteDuration],
15 | maxBufferSize: Option[Int],
16 | stateVerificationInterval: Option[FiniteDuration]) extends BaseAppSettings {
17 | def rootKey: String = mirrorsDirectory
18 |
19 | /** Tries to convert the mirrors directory into a valid path. Throws an exception and
20 | * adds logging if it is not.
21 | */
22 | def getMirrorDirectoryPath(implicit logger: LoggingAdapter): Path = {
23 | val maybePath = Try(Paths.get(mirrorsDirectory))
24 |
25 | def logException(throwable: Throwable) = {
26 | logger.error(throwable, s"Failed to convert [$mirrorsDirectory] into a valid path")
27 | throw throwable
28 | }
29 |
30 | val path = maybePath.fold(logException, identity)
31 |
32 | if (Files.isDirectory(path)) path
33 | else sys.error(s"The provided path is not a directory: [$mirrorsDirectory]")
34 | }
35 | }
36 |
37 |
38 | object DirectoryAppSettings {
39 | // the imports below are actually used
40 |
41 | implicit val valueReader = {
42 | import net.ceedubs.ficus.readers.namemappers.implicits.hyphenCase
43 | import net.ceedubs.ficus.Ficus._
44 | import net.ceedubs.ficus.readers.ArbitraryTypeReader._
45 |
46 | implicitly[ValueReader[DirectoryAppSettings]]
47 | }
48 |
49 | implicit val format: RootJsonFormat[DirectoryAppSettings] = {
50 | import com.supersonic.util.DurationUtil._
51 | import fommil.sjs.FamilyFormats._
52 |
53 | shapeless.cachedImplicit
54 | }
55 |
56 | // needed to help deriving JSON formats
57 | val typeable: Typeable[DirectoryAppSettings] = implicitly[Typeable[DirectoryAppSettings]]
58 | }
59 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/supersonic/kafka_mirror/StaggeredEventsFilterGate.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.kafka_mirror
2 |
3 | import akka.NotUsed
4 | import akka.stream.scaladsl.Flow
5 | import akka.stream.stage.{GraphStage, GraphStageLogic, InHandler, TimerGraphStageLogic, _}
6 | import akka.stream.{Attributes, FlowShape, Inlet, Outlet}
7 | import scala.concurrent.duration.FiniteDuration
8 |
9 | /** Filters events in such a way that incoming events have to wait at least [[staggerTime]] until
10 | * they are fired.
11 | *
12 | * Every time an event occurs a timer is set for [[staggerTime]], if in the meantime there are no
13 | * new events, then the event is fired. If a new event arrives, then the old event is dropped
14 | * and the timer is reset. The cycle repeats until no new events arrive and then the last retained
15 | * event is fired.
16 | *
17 | * @param staggerTime The time to wait before firing an event.
18 | */
19 | class StaggeredEventsFilterGate[A](staggerTime: FiniteDuration) extends GraphStage[FlowShape[A, A]] {
20 |
21 | private val in = Inlet[A]("StaggeredEventsFilterGate.in")
22 | private val out = Outlet[A]("StaggeredEventsFilterGate.out")
23 |
24 | val shape = FlowShape.of(in, out)
25 |
26 | override def createLogic(inheritedAttributes: Attributes): GraphStageLogic =
27 | new TimerGraphStageLogic(shape) {
28 | var last = Option.empty[A]
29 |
30 | def pushLater() = scheduleOnce((), staggerTime)
31 |
32 | setHandler(in, new InHandler {
33 | def onPush(): Unit = {
34 | val a = grab(in)
35 | last = Some(a)
36 |
37 | pushLater()
38 | pull(in)
39 | }
40 | })
41 |
42 | setHandler(out, new OutHandler {
43 | def onPull() = {
44 | pushLater()
45 | if (!hasBeenPulled(in)) pull(in)
46 | }
47 | })
48 |
49 | override protected def onTimer(timerKey: Any): Unit = {
50 | last.foreach { a =>
51 | if (isAvailable(out)) {
52 | push(out, a)
53 | last = None
54 | }
55 | }
56 | }
57 | }
58 | }
59 |
60 | object StaggeredEventsFilterGate {
61 | def apply[A](staggerTime: FiniteDuration): Flow[A, A, NotUsed] =
62 | Flow.fromGraph(new StaggeredEventsFilterGate(staggerTime))
63 | }
64 |
65 |
--------------------------------------------------------------------------------
/file-watcher-app/src/main/scala/com/supersonic/file_watch/DirectoryFilesSource.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.file_watch
2 |
3 | import java.nio.file.{Files, Path}
4 | import akka.NotUsed
5 | import akka.event.LoggingAdapter
6 | import akka.stream.alpakka.file.DirectoryChange
7 | import akka.stream.alpakka.file.scaladsl.DirectoryChangesSource
8 | import akka.stream.scaladsl.Source
9 | import scala.concurrent.duration.FiniteDuration
10 | import scala.util.Try
11 |
12 | /** Creates a source that listens to changes in a directory and stores the directory's file contents
13 | * in a [[Map]].
14 | */
15 | class DirectoryFilesSource(implicit logger: LoggingAdapter) {
16 |
17 | /** Creates a source that listens to changes at the given directory and aggregates the contents
18 | * of the files in the directory into a map that is keyed by the file's path.
19 | * The other parameters are as per [[DirectoryChangesSource]]'s documentation.
20 | */
21 | def apply(directory: Path,
22 | pollInterval: FiniteDuration,
23 | maxBufferSize: Int): Source[Map[Path, Option[String]], NotUsed] = {
24 |
25 | val source =
26 | DirectoryChangesSource(directory, pollInterval = pollInterval, maxBufferSize = maxBufferSize)
27 |
28 | source.scan(readAllFiles(directory)) { case (acc, (path, change)) =>
29 | val newAcc = change match {
30 | case DirectoryChange.Modification =>
31 | acc.updated(path, readFile(path))
32 |
33 | case DirectoryChange.Creation =>
34 | acc.updated(path, readFile(path))
35 |
36 | case DirectoryChange.Deletion =>
37 | acc - path
38 | }
39 |
40 | newAcc
41 | }
42 | }
43 |
44 | private def readAllFiles(directory: Path): Map[Path, Option[String]] = {
45 | import scala.collection.JavaConverters._
46 |
47 | Files.list(directory)
48 | .iterator().asScala
49 | .map(path => path -> readFile(path))
50 | .toMap
51 | }
52 |
53 | private def readFile(path: Path): Option[String] = {
54 | def log(t: Throwable) = {
55 | logger.error(t, s"Failed reading a file at: [$path]")
56 | None
57 | }
58 |
59 | if (!Files.isDirectory(path) && Files.isReadable(path))
60 | Try(new String(Files.readAllBytes(path)))
61 | .fold(log, Some(_))
62 | else None
63 | }
64 | }
65 |
--------------------------------------------------------------------------------
/file-watcher-app/src/main/scala/com/supersonic/main/KafkaFileWatcherMirrorApp.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.main
2 |
3 | import akka.NotUsed
4 | import akka.event.LoggingAdapter
5 | import akka.http.scaladsl.server.Route
6 | import com.supersonic.file_watch.{ConfigFileSettingsProcessing, DirectoryFilesSource}
7 | import net.ceedubs.ficus.readers.ValueReader
8 | import shapeless.Typeable
9 | import spray.json.JsonFormat
10 | import scala.concurrent.ExecutionContext
11 | import scala.concurrent.duration._
12 |
13 | /** Launches an application that listens to a directory for Kafka mirror definitions and manages them.
14 | * The application provides a healthcheck route for its current state.
15 | */
16 | object KafkaFileWatcherMirrorApp extends FileWatcherMirrorApp {
17 | protected def customRoute(currentState: () => AppState[AppSettings]): Option[Route] = None
18 | }
19 |
20 | trait FileWatcherMirrorApp extends KafkaMirrorAppTemplate {
21 | protected type MirrorConfigSourceMat = NotUsed
22 |
23 | protected type AppSettings = DirectoryAppSettings
24 |
25 | protected implicit def appSettingsValueReader: ValueReader[AppSettings] = DirectoryAppSettings.valueReader
26 |
27 | protected implicit def appSettingsJSONFormat: JsonFormat[AppSettings] = DirectoryAppSettings.format
28 |
29 | protected implicit def appSettingsTypeable: Typeable[AppSettings] = DirectoryAppSettings.typeable
30 |
31 | protected def onCompletion(notUsed: NotUsed): Unit = ()
32 |
33 | protected def withConfigSource[A](appSettings: AppSettings)
34 | (continuation: MirrorConfigSource => A)
35 | (implicit logger: LoggingAdapter,
36 | executionContext: ExecutionContext): A = {
37 |
38 | val directory = appSettings.getMirrorDirectoryPath
39 | val pollInterval = appSettings.pollInterval.getOrElse(defaultPollInterval)
40 | val maxBufferSize = appSettings.maxBufferSize.getOrElse(defaultMaxBufferSize)
41 |
42 | val directoryFilesSource = new DirectoryFilesSource
43 |
44 | val source = directoryFilesSource(directory, pollInterval, maxBufferSize)
45 | .via(ConfigFileSettingsProcessing.flow)
46 |
47 | continuation(source)
48 | }
49 |
50 | private def defaultPollInterval = 1.second
51 | private def defaultMaxBufferSize = 1000
52 | }
53 |
--------------------------------------------------------------------------------
/consul-app/src/main/scala/com/supersonic/main/KafkaConsulMirrorApp.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.main
2 |
3 | import akka.event.LoggingAdapter
4 | import akka.http.scaladsl.server.Route
5 | import com.orbitz.consul.Consul
6 | import com.supersonic.consul.{CancellationToken, ConsulSettingsFlow, ConsulStream}
7 | import com.supersonic.kafka_mirror._
8 | import net.ceedubs.ficus.readers.ValueReader
9 | import shapeless.Typeable
10 | import spray.json.JsonFormat
11 | import scala.concurrent.ExecutionContext
12 |
13 | /** Launches an application that listens to Consul for Kafka mirror definitions and manages them.
14 | * The application provides a healthcheck route for its current state.
15 | */
16 | object KafkaConsulMirrorApp extends ConsulMirrorApp {
17 | protected def customRoute(currentState: () => AppState[AppSettings]): Option[Route] = None
18 | }
19 |
20 | trait ConsulMirrorApp extends KafkaMirrorAppTemplate {
21 | protected type MirrorConfigSourceMat = CancellationToken
22 |
23 | protected type AppSettings = ConsulAppSettings
24 |
25 | protected implicit def appSettingsValueReader: ValueReader[AppSettings] = ConsulAppSettings.valueReader
26 |
27 | protected implicit def appSettingsJSONFormat: JsonFormat[AppSettings] = ConsulAppSettings.format
28 |
29 | protected implicit def appSettingsTypeable: Typeable[AppSettings] = ConsulAppSettings.typeable
30 |
31 | protected def onCompletion(cancellationToken: CancellationToken): Unit =
32 | cancellationToken.cancel()
33 |
34 | protected def withConfigSource[A](appSettings: AppSettings)
35 | (continuation: MirrorConfigSource => A)
36 | (implicit logger: LoggingAdapter,
37 | executionContext: ExecutionContext): A = {
38 |
39 | val consulSettings = appSettings.consulSettings
40 |
41 | val consul: Consul = Consul.builder()
42 | .withUrl(consulSettings.url)
43 | .withReadTimeoutMillis(0L) // no timeout
44 | .build()
45 |
46 | ConsulSettings.verifySettings(consulSettings, consul)
47 |
48 | val root = consulSettings.rootKey
49 |
50 | val source = ConsulStream.consulKeySource(root, consul)
51 | .via(StaggeredEventsFilterGate(consulSettings.staggerTime))
52 | .via(ConsulSettingsFlow(root))
53 |
54 | continuation(source)
55 | }
56 | }
57 |
--------------------------------------------------------------------------------
/.github/workflows/clean.yml:
--------------------------------------------------------------------------------
1 | # This file was automatically generated by sbt-github-actions using the
2 | # githubWorkflowGenerate task. You should add and commit this file to
3 | # your git repository. It goes without saying that you shouldn't edit
4 | # this file by hand! Instead, if you wish to make changes, you should
5 | # change your sbt build configuration to revise the workflow description
6 | # to meet your needs, then regenerate this file.
7 |
8 | name: Clean
9 |
10 | on: push
11 |
12 | jobs:
13 | delete-artifacts:
14 | name: Delete Artifacts
15 | runs-on: ubuntu-latest
16 | env:
17 | GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }}
18 | steps:
19 | - name: Delete artifacts
20 | run: |
21 | # Customize those three lines with your repository and credentials:
22 | REPO=${GITHUB_API_URL}/repos/${{ github.repository }}
23 |
24 | # A shortcut to call GitHub API.
25 | ghapi() { curl --silent --location --user _:$GITHUB_TOKEN "$@"; }
26 |
27 | # A temporary file which receives HTTP response headers.
28 | TMPFILE=/tmp/tmp.$$
29 |
30 | # An associative array, key: artifact name, value: number of artifacts of that name.
31 | declare -A ARTCOUNT
32 |
33 | # Process all artifacts on this repository, loop on returned "pages".
34 | URL=$REPO/actions/artifacts
35 | while [[ -n "$URL" ]]; do
36 |
37 | # Get current page, get response headers in a temporary file.
38 | JSON=$(ghapi --dump-header $TMPFILE "$URL")
39 |
40 | # Get URL of next page. Will be empty if we are at the last page.
41 | URL=$(grep '^Link:' "$TMPFILE" | tr ',' '\n' | grep 'rel="next"' | head -1 | sed -e 's/.*/' -e 's/>.*//')
42 | rm -f $TMPFILE
43 |
44 | # Number of artifacts on this page:
45 | COUNT=$(( $(jq <<<$JSON -r '.artifacts | length') ))
46 |
47 | # Loop on all artifacts on this page.
48 | for ((i=0; $i < $COUNT; i++)); do
49 |
50 | # Get name of artifact and count instances of this name.
51 | name=$(jq <<<$JSON -r ".artifacts[$i].name?")
52 | ARTCOUNT[$name]=$(( $(( ${ARTCOUNT[$name]} )) + 1))
53 |
54 | id=$(jq <<<$JSON -r ".artifacts[$i].id?")
55 | size=$(( $(jq <<<$JSON -r ".artifacts[$i].size_in_bytes?") ))
56 | printf "Deleting '%s' #%d, %'d bytes\n" $name ${ARTCOUNT[$name]} $size
57 | ghapi -X DELETE $REPO/actions/artifacts/$id
58 | done
59 | done
--------------------------------------------------------------------------------
/core/src/test/scala/com/supersonic/kafka_mirror/SettingsProcessingFlowTest.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.kafka_mirror
2 |
3 | import akka.actor.ActorSystem
4 | import akka.event.Logging
5 | import akka.stream.ActorMaterializer
6 | import akka.testkit.TestKit
7 | import com.supersonic.kafka_mirror.MirrorCommand._
8 | import com.supersonic.kafka_mirror.TestUtil._
9 | import com.typesafe.config.ConfigFactory
10 | import org.scalatest.{Matchers, WordSpecLike}
11 |
12 | class SettingsProcessingFlowTest extends TestKit(ActorSystem("SettingsProcessingFlowTest"))
13 | with WordSpecLike
14 | with Matchers {
15 |
16 | implicit val ec = system.dispatcher
17 | implicit val materializer = ActorMaterializer()
18 | implicit val logger = Logging(system.eventStream, "SettingsProcessingFlowTest")
19 |
20 | val fallback = ConfigFactory.parseString("""
21 | akka.kafka.consumer = {a = 1}
22 | akka.kafka.producer = {b = 2}
23 | """)
24 | val root = "foo"
25 |
26 | def probes() = {
27 | val flow = SettingsProcessingFlow(fallback)
28 |
29 | TestUtil.probes(flow)
30 | }
31 |
32 | def settings(i: Int) = Some {
33 | s"""
34 | consumer = {a = $i}
35 | producer = {b = $i}
36 | mirror = {
37 | whitelist = ["topic1", "topic2"]
38 | commitBatchSize = 1
39 | commitParallelism = 4
40 | }
41 | """
42 | }
43 |
44 | def parsedSettings(idStr: String, i: Int) =
45 | SettingsProcessingFlow.fromRawMap(fallback) {
46 | Map(id(idStr) -> settings(i))
47 | }(id(idStr)).getOrElse(sys.error("should not happen"))
48 |
49 | "The settings processing flow" should {
50 | "process settings changes" in {
51 | val (pub, sub) = probes()
52 |
53 | pub.sendNext(Map(
54 | id("a") -> settings(1),
55 | id("b") -> settings(2)
56 | ))
57 |
58 | pub.sendNext(Map(
59 | id("a") -> settings(1),
60 | id("b") -> settings(3)
61 | ))
62 |
63 | pub.sendNext(Map(
64 | id("b") -> settings(3)
65 | ))
66 |
67 | pub.sendNext(Map(
68 | id("b") -> None
69 | ))
70 |
71 | sub.request(2)
72 | sub.expectNextUnordered(
73 | Start(id("a"), parsedSettings("a", 1)),
74 | Start(id("b"), parsedSettings("b", 2))
75 | )
76 |
77 | sub.request(1)
78 | sub.expectNext(
79 | Start(id("b"), parsedSettings("b", 3))
80 | )
81 |
82 | sub.request(1)
83 | sub.expectNext(
84 | Stop(id("a"))
85 | )
86 |
87 | sub.request(1)
88 | sub.expectNext(
89 | Stop(id("b"))
90 | )
91 | }
92 |
93 | "stop mirrors whose settings failed to parse" in {
94 | val (pub, sub) = probes()
95 |
96 | pub.sendNext(Map(
97 | id("a") -> settings(1)
98 | ))
99 |
100 | sub.request(1)
101 | sub.expectNext(
102 | Start(id("a"), parsedSettings("a", 1))
103 | )
104 |
105 | pub.sendNext(Map(
106 | id("b") -> Some("bla =")
107 | ))
108 |
109 | sub.request(1)
110 | sub.expectNext(
111 | Stop(id("a"))
112 | )
113 | }
114 | }
115 | }
116 |
--------------------------------------------------------------------------------
/core/src/test/scala/com/supersonic/kafka_mirror/MirrorManagerFlowTest.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.kafka_mirror
2 |
3 | import akka.actor.ActorSystem
4 | import akka.event.Logging
5 | import akka.stream.ActorMaterializer
6 | import akka.testkit.TestKit
7 | import com.supersonic.kafka_mirror.MirrorCommand._
8 | import com.supersonic.kafka_mirror.TestUtil._
9 | import org.scalatest.{AsyncWordSpecLike, Matchers}
10 |
11 | class MirrorManagerFlowTest extends TestKit(ActorSystem("MirrorManagerFlowTest"))
12 | with AsyncWordSpecLike
13 | with Matchers {
14 |
15 | implicit val ec = system.dispatcher
16 | implicit val materializer = ActorMaterializer()
17 | implicit val logger = Logging(system.eventStream, "SettingsProcessingFlowTest")
18 |
19 | val mirrorMaker = new MockMirrorMaker(new MockKafkaMirror)
20 |
21 | def probes() = {
22 | val flow = new MirrorManager(mirrorMaker).flow
23 |
24 | TestUtil.probes(flow)
25 | }
26 |
27 | "The mirror manager flow" should {
28 | "start and stop mirrors" in {
29 | val (pub, sub) = probes()
30 |
31 | pub.sendNext(Start(id("a"), runnableSettings("a")))
32 |
33 | sub.request(2)
34 |
35 | sub.expectNext() shouldBe empty
36 |
37 | val s1 = sub.expectNext()
38 |
39 | s1 should contain key id("a")
40 |
41 | val control = s1(id("a")).control
42 |
43 | pub.sendNext(Stop(id("a")))
44 |
45 | sub.request(1)
46 | val s2 = sub.expectNext()
47 |
48 | s2 shouldBe empty
49 |
50 | control.isShutdown.map { _ =>
51 | succeed
52 | }
53 | }
54 |
55 | "restart failed mirrors" in {
56 | val (pub, sub) = probes()
57 |
58 | pub.sendNext(Start(id("a"), runnableSettings("a")))
59 |
60 | sub.request(2)
61 |
62 | sub.expectNext()
63 |
64 | val s1 = sub.expectNext()
65 | val control = s1(id("a")).control
66 |
67 | control.shutdown().map { _ =>
68 | MirrorManager.isShutDownNow(control) shouldBe true
69 | pub.sendNext(VerifyState)
70 |
71 | sub.request(1)
72 | val s2 = sub.expectNext()
73 |
74 | s2.toList should have size 1
75 |
76 | MirrorManager.isShutDownNow(s2(id("a")).control) shouldBe false
77 | }
78 | }
79 |
80 | "shutdown all mirrors and complete the stream" in {
81 | val (pub, sub) = probes()
82 |
83 | pub.sendNext(Start(id("a"), runnableSettings("a")))
84 | pub.sendNext(Start(id("b"), runnableSettings("b")))
85 | pub.sendNext(Start(id("c"), runnableSettings("c")))
86 |
87 | sub.request(4)
88 |
89 | sub.expectNext() shouldBe empty
90 | sub.expectNext()
91 | sub.expectNext()
92 |
93 | val state = sub.expectNext()
94 |
95 | state should contain key id("a")
96 | state should contain key id("b")
97 | state should contain key id("c")
98 |
99 | val c1 = state(id("a")).control
100 | val c2 = state(id("b")).control
101 | val c3 = state(id("c")).control
102 |
103 | pub.sendNext(Shutdown)
104 |
105 | sub.request(1)
106 | sub.expectComplete()
107 |
108 | for {
109 | _ <- c1.isShutdown
110 | _ <- c2.isShutdown
111 | _ <- c3.isShutdown
112 | } yield succeed
113 | }
114 | }
115 | }
116 |
--------------------------------------------------------------------------------
/core/src/test/scala/com/supersonic/kafka_mirror/TestUtil.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.kafka_mirror
2 |
3 | import akka.Done
4 | import akka.actor.ActorSystem
5 | import akka.event.LoggingAdapter
6 | import akka.kafka.scaladsl.Consumer.Control
7 | import akka.stream.Materializer
8 | import akka.stream.scaladsl.{Flow, Keep, Source}
9 | import akka.stream.testkit.scaladsl.{TestSink, TestSource}
10 | import akka.stream.testkit.{TestPublisher, TestSubscriber}
11 | import com.typesafe.config.ConfigFactory
12 | import org.apache.kafka.clients.producer.Producer
13 | import org.apache.kafka.common.{Metric, MetricName}
14 | import scala.concurrent.{ExecutionContext, Future, Promise}
15 |
16 | object TestUtil {
17 |
18 | def id(str: String) = MirrorID(str)
19 |
20 | def settings(str: String) = {
21 | def conf(str: String, i: Int) = ConfigFactory.parseString(s"$str = $i")
22 |
23 | ExternalKafkaMirrorSettings(ExternalKafkaSettings(conf(str, 1), conf(str, 2)),
24 | MirrorSettings(Set(s"$str-topic"), 1, 2))
25 | }
26 |
27 | def runnableSettings(str: String)
28 | (implicit system: ActorSystem) = {
29 | val systemConfig = system.settings.config
30 | def conf(str: String, i: Int) = ConfigFactory.parseString(s"""
31 | $str = $i
32 | kafka-clients = {
33 | bootstrap.servers = "localhost:9092"
34 | }
35 | """)
36 | .withFallback(systemConfig.getConfig("akka.kafka.consumer"))
37 | .withFallback(systemConfig.getConfig("akka.kafka.producer"))
38 |
39 | ExternalKafkaMirrorSettings(ExternalKafkaSettings(conf(str, 1), conf(str, 2)),
40 | MirrorSettings(Set(s"$str-topic"), 1, 2))
41 | }
42 |
43 | def probes[A, B](flow: Flow[A, B, _])
44 | (implicit system: ActorSystem,
45 | materializer: Materializer): (TestPublisher.Probe[A], TestSubscriber.Probe[B]) =
46 | TestSource.probe[A]
47 | .via(flow)
48 | .toMat(TestSink.probe[B])(Keep.both)
49 | .run()
50 |
51 | class MockMirrorMaker(create: => KafkaMirror) extends MirrorMaker {
52 | type Mirror = KafkaMirror
53 |
54 | def apply(mirrorID: MirrorID, producer: Producer[_, _])
55 | (source: LoggingAdapter => Source[Done, Control]) = {
56 | // since we don't want it to actually run in the test and it will
57 | // not be shutdown otherwise (since we are using a mock mirror)
58 | producer.close()
59 |
60 | create
61 | }
62 | }
63 |
64 | class MockKafkaMirror(implicit executionContext: ExecutionContext) extends KafkaMirror {
65 | @volatile var control = Option.empty[MockControl]
66 |
67 | def start() = {
68 | val c = new MockControl
69 | control = Some(c)
70 |
71 | c
72 | }
73 |
74 | def isStarted() = control.isDefined
75 | }
76 |
77 | object FailingKafkaMirror extends KafkaMirror {
78 | def start() = sys.error("mirror failed")
79 | }
80 |
81 | class MockControl(implicit executionContext: ExecutionContext) extends Control {
82 | private val shutDownState = Promise[Done]()
83 |
84 | def shutdownNow(): Unit = {
85 | val _ = shutDownState.trySuccess(Done)
86 | }
87 |
88 | def isShutdownNow(): Boolean = shutDownState.isCompleted
89 |
90 | override def shutdown() = Future {
91 | shutdownNow()
92 | Done
93 | }
94 |
95 | override def stop() = shutdown()
96 |
97 | override val isShutdown = shutDownState.future
98 |
99 | override def metrics: Future[Map[MetricName, Metric]] = Future.successful(Map.empty)
100 | }
101 |
102 | object FailingControl extends Control {
103 | override def shutdown() = Future.failed(new Exception("failed to shutdown"))
104 |
105 | override def stop() = shutdown()
106 |
107 | override def isShutdown = stop()
108 |
109 | override def metrics: Future[Map[MetricName, Metric]] = Future.successful(Map.empty)
110 | }
111 | }
112 |
--------------------------------------------------------------------------------
/core/src/it/scala/com/supersonic/kafka_mirror/KafkaMirrorIntegrationTest.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.kafka_mirror
2 |
3 | import akka.actor.ActorSystem
4 | import akka.event.Logging
5 | import akka.stream.scaladsl.Keep
6 | import akka.stream.testkit.scaladsl.TestSink
7 | import akka.testkit.TestKit
8 | import org.scalatest.{Matchers, WordSpecLike}
9 | import scala.concurrent.Await
10 | import concurrent.duration._
11 |
12 | class KafkaMirrorIntegrationTest extends TestKit(ActorSystem("KafkaMirrorIntegrationTest"))
13 | with AkkaStreamsKafkaIntegrationSpec
14 | with WordSpecLike
15 | with Matchers {
16 |
17 | val logger = Logging(system.eventStream, "KafkaMirrorIntegrationTest")
18 |
19 | val kafkaMirror = KafkaMirror[String, String](new SourceBackedMirrorMaker(materializer, logger)) _
20 |
21 | "The Kafka mirror" should {
22 | val group1 = createGroup(1)
23 | val group2 = createGroup(2)
24 |
25 | def makeMirror(whitelist: String*) =
26 | kafkaMirror(MirrorID("test"), mirrorSettings(group1)(whitelist: _*))
27 |
28 | val messages1 = 1 to 100
29 | val messages2 = 101 to 200
30 |
31 | "move messages between topics in different Kafka servers using the mirror" in {
32 | val topic1 = createTopicOnServers(1)
33 | val topic2 = createTopicOnServers(2)
34 |
35 | produceMessages(topic1, messages1)
36 | produceMessages(topic2, messages2)
37 |
38 | val mirror = makeMirror(topic1, topic2)
39 |
40 | val pullingProbe = mirror.source.runWith(TestSink.probe)
41 |
42 | pullingProbe
43 | .request(messages1.size + messages2.size) // should request all messages, even if batching is set to 1
44 | .expectNext(10.seconds) // we can't expect as many messages, since batching can compact many messages into one
45 |
46 | verifyTopic(topic1, group2, messages1)
47 | verifyTopic(topic2, group2, messages2)
48 |
49 | pullingProbe.cancel()
50 | mirror.producer.close()
51 | }
52 |
53 | "commit messages in the source topic upon mirroring" in {
54 | val topic = createTopicOnServers(1)
55 |
56 | produceMessages(topic, messages1)
57 |
58 | val mirror = makeMirror(topic)
59 |
60 | val (control, pullingProbe) = mirror.source.toMat(TestSink.probe)(Keep.both).run()
61 |
62 | pullingProbe // requesting the first batch
63 | .request(1)
64 | .expectNext(10.seconds)
65 |
66 | pullingProbe.cancel()
67 | Await.result(control.isShutdown, 1.minute)
68 |
69 | // continuing consumption within the same group,
70 | // should skip messages from the first batch since it was already committed
71 | val sourceTopicProbe = createProbe(kafkaHelper1.createConsumerSettings(group1), topic)
72 | val element = sourceTopicProbe.request(1).expectNext(10.seconds)
73 |
74 | element.toInt should be > messages1.head
75 |
76 | sourceTopicProbe.cancel()
77 | mirror.producer.close()
78 | }
79 |
80 | "support message bucketing" in {
81 | val bucketing = BucketSettings(
82 | mirrorBuckets = 3,
83 | totalBuckets = 5
84 | )
85 |
86 | val topic = createTopicOnServers(1)
87 |
88 | produceMessages(topic, messages1)
89 |
90 | val mirror = kafkaMirror(MirrorID("test"), mirrorSettings(group1, Some(bucketing))(topic))
91 |
92 | val pullingProbe = mirror.source.runWith(TestSink.probe)
93 |
94 | pullingProbe
95 | .request(messages1.size) // should request all messages, even if batching is set to 1
96 | .expectNext(10.seconds) // we can't expect as many messages, since batching can compact many messages into one
97 |
98 | val mirroredMessages = messages1.filter(_ % 5 < 3)
99 |
100 | verifyTopic(topic, group1, mirroredMessages)
101 |
102 | pullingProbe.cancel()
103 | mirror.producer.close()
104 | }
105 | }
106 | }
107 |
--------------------------------------------------------------------------------
/.github/workflows/ci.yml:
--------------------------------------------------------------------------------
1 | # This file was automatically generated by sbt-github-actions using the
2 | # githubWorkflowGenerate task. You should add and commit this file to
3 | # your git repository. It goes without saying that you shouldn't edit
4 | # this file by hand! Instead, if you wish to make changes, you should
5 | # change your sbt build configuration to revise the workflow description
6 | # to meet your needs, then regenerate this file.
7 |
8 | name: Continuous Integration
9 |
10 | on:
11 | pull_request:
12 | branches: ['*']
13 | push:
14 | branches: ['*']
15 | tags: [v*]
16 |
17 | env:
18 | GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }}
19 |
20 | jobs:
21 | build:
22 | name: Build and Test
23 | strategy:
24 | matrix:
25 | os: [ubuntu-latest]
26 | scala: [2.12.13]
27 | java: [adopt@1.8]
28 | runs-on: ${{ matrix.os }}
29 | steps:
30 | - name: Checkout current branch (full)
31 | uses: actions/checkout@v2
32 | with:
33 | fetch-depth: 0
34 |
35 | - name: Setup Java and Scala
36 | uses: olafurpg/setup-scala@v10
37 | with:
38 | java-version: ${{ matrix.java }}
39 |
40 | - name: Cache sbt
41 | uses: actions/cache@v2
42 | with:
43 | path: |
44 | ~/.sbt
45 | ~/.ivy2/cache
46 | ~/.coursier/cache/v1
47 | ~/.cache/coursier/v1
48 | ~/AppData/Local/Coursier/Cache/v1
49 | ~/Library/Caches/Coursier/v1
50 | key: ${{ runner.os }}-sbt-cache-v2-${{ hashFiles('**/*.sbt') }}-${{ hashFiles('project/build.properties') }}
51 |
52 | - name: Check that workflows are up to date
53 | run: sbt ++${{ matrix.scala }} githubWorkflowCheck
54 |
55 | - name: Build project
56 | run: sbt ++${{ matrix.scala }} test
57 |
58 | - name: Compress target directories
59 | run: tar cf targets.tar app-common/target consul-app/target target core/target file-watcher-app/target project/target
60 |
61 | - name: Upload target directories
62 | uses: actions/upload-artifact@v2
63 | with:
64 | name: target-${{ matrix.os }}-${{ matrix.scala }}-${{ matrix.java }}
65 | path: targets.tar
66 |
67 | publish:
68 | name: Publish Artifacts
69 | needs: [build]
70 | if: github.event_name != 'pull_request' && (startsWith(github.ref, 'refs/tags/v'))
71 | strategy:
72 | matrix:
73 | os: [ubuntu-latest]
74 | scala: [2.12.13]
75 | java: [adopt@1.8]
76 | runs-on: ${{ matrix.os }}
77 | steps:
78 | - name: Checkout current branch (full)
79 | uses: actions/checkout@v2
80 | with:
81 | fetch-depth: 0
82 |
83 | - name: Setup Java and Scala
84 | uses: olafurpg/setup-scala@v10
85 | with:
86 | java-version: ${{ matrix.java }}
87 |
88 | - name: Cache sbt
89 | uses: actions/cache@v2
90 | with:
91 | path: |
92 | ~/.sbt
93 | ~/.ivy2/cache
94 | ~/.coursier/cache/v1
95 | ~/.cache/coursier/v1
96 | ~/AppData/Local/Coursier/Cache/v1
97 | ~/Library/Caches/Coursier/v1
98 | key: ${{ runner.os }}-sbt-cache-v2-${{ hashFiles('**/*.sbt') }}-${{ hashFiles('project/build.properties') }}
99 |
100 | - name: Download target directories (2.12.13)
101 | uses: actions/download-artifact@v2
102 | with:
103 | name: target-${{ matrix.os }}-2.12.13-${{ matrix.java }}
104 |
105 | - name: Inflate target directories (2.12.13)
106 | run: |
107 | tar xf targets.tar
108 | rm targets.tar
109 |
110 | - env:
111 | PGP_PASSPHRASE: ${{ secrets.PGP_PASSPHRASE }}
112 | PGP_SECRET: ${{ secrets.PGP_SECRET }}
113 | SONATYPE_PASSWORD: ${{ secrets.SONATYPE_PASSWORD }}
114 | SONATYPE_USERNAME: ${{ secrets.SONATYPE_USERNAME }}
115 | run: sbt ++${{ matrix.scala }} ci-release
--------------------------------------------------------------------------------
/core/src/main/scala/com/supersonic/kafka_mirror/mirror_settings.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.kafka_mirror
2 |
3 | import java.util.concurrent.ThreadLocalRandom
4 | import akka.kafka.{ConsumerSettings, ProducerSettings}
5 | import com.typesafe.config.Config
6 | import org.apache.kafka.common.serialization.{ByteArrayDeserializer, ByteArraySerializer}
7 | import org.apache.kafka.common.utils.Utils
8 | import scala.util.Try
9 |
10 | /** Note that equality doesn't work on this class since the ***Settings classes don't override
11 | * the equals method.
12 | */
13 | case class KafkaSettings[K, V](consumer: ConsumerSettings[K, V],
14 | producer: ProducerSettings[K, V])
15 |
16 | /** Settings for a Kafka mirror.
17 | *
18 | * @param whitelist The topics that should be mirrored, i.e., the same topics on in the source
19 | * will be mirrored to the target.
20 | * @param commitBatchSize How many messages to batch before committing their offsets to Kafka.
21 | * @param commitParallelism The parallelism level used to commit the offsets.
22 | * @param bucketing Settings to enable bucketing of mirrored values.
23 | * @param enabled Whether the mirror should be enabled or not.
24 | * @param partitionFromKeys Whether the partition number when mirroring should be computed from the
25 | * key or not.
26 | * @param topicsToRename Map of src to dest topic to rename when mirroring the message to the producer.
27 | *
28 | */
29 | case class MirrorSettings(whitelist: Set[String],
30 | commitBatchSize: Int = 1000,
31 | commitParallelism: Int = 4,
32 | bucketing: Option[BucketSettings] = None,
33 | enabled: Boolean = true,
34 | partitionFromKeys: Boolean = false,
35 | topicsToRename: Map[String, String] = Map.empty)
36 |
37 | /**
38 | * Defines settings for mirroring buckets.
39 | * The (mirrorBuckets / totalBuckets) ratio is the percentage of traffic to be mirrored.
40 | *
41 | * @param mirrorBuckets The number of buckets that should be mirrored.
42 | * @param totalBuckets The total number of buckets, used to calculate the percentage of traffic
43 | * to mirror.
44 | */
45 | case class BucketSettings(mirrorBuckets: Int, totalBuckets: Int) {
46 | assert(mirrorBuckets <= totalBuckets, "The number of mirroring buckets cannot exceed the total number of buckets")
47 | }
48 |
49 | /** Note that equality doesn't work on this class since the [[KafkaSettings]] doesn't have
50 | * a good equality method.
51 | *
52 | * @param generatePartition Takes the total number of partitions and generates a random partition
53 | * number (will be random in actual usage).
54 | */
55 | case class KafkaMirrorSettings[K, V](kafka: KafkaSettings[K, V],
56 | mirror: MirrorSettings,
57 | hashKey: K => Int,
58 | generatePartition: Int => Int)
59 |
60 | /** Copying the [[KafkaMirrorSettings]] hierarchy, but instead of using the dedicated consumer/producer
61 | * settings objects using [[Config]]s, this enables logical equality on these classes (since
62 | * the dedicated classes are not defining a custom equality method).
63 | */
64 | case class ExternalKafkaSettings(consumer: Config, producer: Config)
65 | case class ExternalKafkaMirrorSettings(kafka: ExternalKafkaSettings, mirror: MirrorSettings) {
66 |
67 | def toKafkaMirrorSettings: Try[KafkaMirrorSettings[Array[Byte], Array[Byte]]] = Try {
68 | val consumerSettings = ConsumerSettings(kafka.consumer, new ByteArrayDeserializer, new ByteArrayDeserializer)
69 | val producerSettings = ProducerSettings(kafka.producer, new ByteArraySerializer, new ByteArraySerializer)
70 |
71 | def hashKey(k: Array[Byte]) =
72 | Utils.abs(Utils.murmur2(if (k == null) Array.empty else k))
73 |
74 | def generatePartition(totalPartitions: Int) =
75 | ThreadLocalRandom.current().nextInt(totalPartitions)
76 |
77 | KafkaMirrorSettings(
78 | KafkaSettings(consumerSettings, producerSettings),
79 | mirror,
80 | hashKey,
81 | generatePartition)
82 | }
83 | }
84 |
--------------------------------------------------------------------------------
/file-watcher-app/src/test/scala/com/supersonic/file_watch/DirectoryFilesSourceTest.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.file_watch
2 |
3 | import akka.actor.ActorSystem
4 | import akka.event.Logging
5 | import akka.stream.ActorMaterializer
6 | import akka.stream.scaladsl.Keep
7 | import akka.stream.testkit.scaladsl.TestSink
8 | import akka.testkit.TestKit
9 | import org.scalatest.{Matchers, WordSpecLike}
10 | import scala.concurrent.duration._
11 |
12 | class DirectoryFilesSourceTest extends TestKit(ActorSystem("DirectorySettingsProcessingSourceTest"))
13 | with DirectoryFilesSourceTestUtil
14 | with WordSpecLike
15 | with Matchers {
16 | implicit val materializer = ActorMaterializer()
17 | implicit val logger = Logging(system.eventStream, "DirectorySettingsProcessingSourceTest")
18 |
19 | "The directory-source" should {
20 | "listen to file creation" in withDirectoryProbe { (fileHelper, probe) =>
21 | import fileHelper._
22 |
23 | probe.requestNext() shouldBe empty
24 |
25 | createFile(name = "a", content = "a-content")
26 |
27 | probe.requestNext() shouldBe Map(path("a") -> Some("a-content"))
28 | }
29 |
30 | "listen to file modification" in withDirectoryProbe { (fileHelper, probe) =>
31 | import fileHelper._
32 |
33 | probe.requestNext()
34 |
35 | createFile(name = "a", content = "a-content")
36 | probe.requestNext() shouldBe Map(path("a") -> Some("a-content"))
37 |
38 | modifyFile(name = "a", content = "a-content-2")
39 | probe.requestNext() shouldBe Map(path("a") -> Some("a-content-2"))
40 | }
41 |
42 | "listen to file deletion" in withDirectoryProbe { (fileHelper, probe) =>
43 | import fileHelper._
44 |
45 | probe.requestNext()
46 |
47 | createFile(name = "a", content = "a-content")
48 | probe.requestNext() shouldBe Map(path("a") -> Some("a-content"))
49 |
50 | deleteFile(name = "a")
51 | probe.requestNext()
52 | probe.requestNext() shouldBe empty
53 | }
54 |
55 | "ignore directories" in withDirectoryProbe { (fileHelper, probe) =>
56 | import fileHelper._
57 |
58 | probe.requestNext() shouldBe empty
59 |
60 | createDir(name = "a")
61 |
62 | probe.requestNext() shouldBe Map(path("a") -> None)
63 | }
64 |
65 | "read all files that are present on startup" in {
66 | withDirectory { fileHelper =>
67 | import fileHelper._
68 |
69 | createFile(name = "a", content = "a-content")
70 | createFile(name = "b", content = "b-content")
71 | createDir(name = "c")
72 |
73 | val source = new DirectoryFilesSource()(logger)(fileHelper.dir, 5.seconds, 1000)
74 |
75 | val probe = source.toMat(TestSink.probe)(Keep.right).run()
76 |
77 | try {
78 | probe.requestNext() shouldBe Map(
79 | path("a") -> Some("a-content"),
80 | path("b") -> Some("b-content"),
81 | path("c") -> None)
82 |
83 | deleteFile("a")
84 |
85 | probe.requestNext() shouldBe Map(
86 | path("b") -> Some("b-content"),
87 | path("c") -> None)
88 | }
89 | finally {
90 | val _ = probe.cancel()
91 | }
92 | }
93 | }
94 |
95 | "mix and match operations" in withDirectoryProbe { (fileHelper, probe) =>
96 | import fileHelper._
97 |
98 | probe.requestNext() shouldBe empty
99 |
100 | createFile(name = "a", content = "a-content")
101 | probe.requestNext() shouldBe Map(path("a") -> Some("a-content"))
102 | probe.requestNext() // for some reason, each creation event is accompanied with a modification event
103 |
104 | createFile(name = "b", content = "b-content")
105 | probe.requestNext() shouldBe Map(
106 | path("a") -> Some("a-content"),
107 | path("b") -> Some("b-content"))
108 | probe.requestNext()
109 |
110 | createFile(name = "c", content = "c-content")
111 | probe.requestNext() shouldBe Map(
112 | path("a") -> Some("a-content"),
113 | path("b") -> Some("b-content"),
114 | path("c") -> Some("c-content"))
115 | probe.requestNext()
116 |
117 | createDir("d")
118 | probe.requestNext() shouldBe Map(
119 | path("a") -> Some("a-content"),
120 | path("b") -> Some("b-content"),
121 | path("c") -> Some("c-content"),
122 | path("d") -> None)
123 |
124 | deleteFile(name = "c")
125 | probe.requestNext() shouldBe Map(
126 | path("a") -> Some("a-content"),
127 | path("b") -> Some("b-content"),
128 | path("d") -> None)
129 |
130 | modifyFile(name = "b", content = "b-content-2")
131 | probe.requestNext() shouldBe Map(
132 | path("a") -> Some("a-content"),
133 | path("b") -> Some("b-content-2"),
134 | path("d") -> None)
135 |
136 | deleteFile("d")
137 | probe.requestNext() shouldBe Map(
138 | path("a") -> Some("a-content"),
139 | path("b") -> Some("b-content-2"))
140 | }
141 | }
142 | }
143 |
--------------------------------------------------------------------------------
/core/src/test/scala/com/supersonic/kafka_mirror/StaggeredEventsFilterGateTest.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.kafka_mirror
2 |
3 | import akka.actor.ActorSystem
4 | import akka.pattern
5 | import akka.stream.ActorMaterializer
6 | import akka.stream.testkit.TestPublisher
7 | import akka.testkit.TestKit
8 | import com.supersonic.kafka_mirror.TestUtil._
9 | import org.scalatest.concurrent.ScalaFutures
10 | import org.scalatest.{Matchers, WordSpecLike}
11 | import scala.concurrent.Future
12 | import scala.concurrent.duration._
13 |
14 | class StaggeredEventsFilterGateTest extends TestKit(ActorSystem("StaggeredEventsFilterGateTest"))
15 | with WordSpecLike
16 | with Matchers
17 | with ScalaFutures {
18 | override implicit val patienceConfig = PatienceConfig(30.second)
19 | implicit val ec = system.dispatcher
20 | implicit val materializer = ActorMaterializer()
21 |
22 | def gate(staggerTime: FiniteDuration) = StaggeredEventsFilterGate[Int](staggerTime)
23 |
24 | def sendAfter[A](probe: TestPublisher.Probe[A])(duration: FiniteDuration)(value: A) =
25 | pattern.after(duration, using = system.scheduler) {
26 | probe.sendNext(value)
27 | Future.successful(duration)
28 | }
29 |
30 | def sleep(duration: FiniteDuration) =
31 | pattern.after(duration, using = system.scheduler) {
32 | Future.successful(duration)
33 | }
34 |
35 | "The staggered events filter gate" should {
36 | "pass the first element if there aren't any other coming" in {
37 | val (pub, sub) = probes(gate(500.millis))
38 |
39 | sub.request(1)
40 | pub.sendNext(1)
41 |
42 | sub.expectNext(1)
43 | }
44 |
45 | "hold on to the first element for the stagger duration" in {
46 | val (pub, sub) = probes(gate(2.second))
47 |
48 | sub.request(1)
49 | pub.sendNext(1)
50 |
51 | an[AssertionError] should be thrownBy {
52 | sub.expectNext(1.second)
53 | }
54 | }
55 |
56 | "pass on the last element when multiple events appear together" in {
57 | val (pub, sub) = probes(gate(500.millis))
58 |
59 | sub.request(1)
60 |
61 | pub.sendNext(1)
62 | pub.sendNext(2)
63 | pub.sendNext(3)
64 |
65 | sub.expectNext(3)
66 | }
67 |
68 | "pass on the last element when multiple events appear within less of the stagger time of each other" in {
69 | val (pub, sub) = probes(gate(1.second))
70 | val send = sendAfter(pub)(100.millis) _
71 |
72 | sub.request(1)
73 |
74 | for {
75 | _ <- send(1)
76 | _ <- send(2)
77 | _ <- send(3)
78 | } yield ()
79 |
80 | sub.expectNext(3)
81 | }
82 |
83 | "wait between a groups of incoming events that that a separated by the stagger time" in {
84 | val (pub, sub) = probes(gate(1.second))
85 | val send = sendAfter(pub)(100.millis) _
86 |
87 | sub.request(2)
88 |
89 | for {
90 | _ <- send(1)
91 | _ <- send(2)
92 | _ <- send(3)
93 | _ <- sleep(2.second)
94 | _ <- send(4)
95 | _ <- send(5)
96 | _ <- send(6)
97 | } yield ()
98 |
99 | sub.expectNext(3, 6)
100 | }
101 |
102 | "reset the timer even if the total wait goes over the stagger time" in {
103 | val staggerTime = 500.millis
104 | val (pub, sub) = probes(gate(staggerTime))
105 | val send = sendAfter(pub)(200.millis) _
106 |
107 | sub.request(1)
108 |
109 | val totalTime = for {
110 | t1 <- send(1)
111 | t2 <- send(2)
112 | t3 <- send(3)
113 | t4 <- send(4)
114 | t5 <- send(5)
115 | t6 <- send(6)
116 | } yield List(t1, t2, t3, t4, t5, t6).reduce(_ + _)
117 |
118 | totalTime.futureValue should be > staggerTime
119 |
120 | sub.expectNext(6)
121 | }
122 |
123 | "not query the source when there isn't any demand" in {
124 | val (pub, sub) = probes(gate(200.millis))
125 | val send = sendAfter(pub)(250.millis) _
126 |
127 | sub.request(2)
128 |
129 | pub.sendNext(1)
130 | val sent = send(2).map { _ =>
131 | val _ = pub.sendError(new Exception("shouldn't be read"))
132 | }
133 |
134 | whenReady(sent) { _ =>
135 | sub.expectNext(1)
136 | }
137 | }
138 |
139 | "not push events when they weren't requested" in {
140 | val (pub, sub) = probes(gate(100.millis))
141 |
142 | sub.request(1)
143 | pub.sendNext(1)
144 |
145 | sub.expectNext(1)
146 |
147 | pub.sendNext(2)
148 |
149 | // note that we didn't request a message here
150 | noException should be thrownBy {
151 | sub.expectNoMessage(200.millis)
152 | }
153 | }
154 |
155 | "produce events that were received before they were requested" in {
156 | val (pub, sub) = probes(gate(100.millis))
157 |
158 | sub.request(1)
159 | pub.sendNext(1)
160 |
161 | sub.expectNext(1)
162 |
163 | pub.sendNext(2)
164 |
165 | // note that we didn't request a message here
166 | sub.expectNoMessage(200.millis)
167 |
168 | sub.request(1)
169 | sub.expectNext(2)
170 | }
171 | }
172 | }
173 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/supersonic/kafka_mirror/SettingsProcessingFlow.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.kafka_mirror
2 |
3 | import akka.NotUsed
4 | import akka.event.LoggingAdapter
5 | import akka.stream.scaladsl.Flow
6 | import cats.data.{NonEmptyList, Validated}
7 | import cats.syntax.apply._
8 | import com.supersonic.kafka_mirror.MirrorCommand._
9 | import com.supersonic.util.ValidatedUtil._
10 | import com.typesafe.config.{Config, ConfigFactory}
11 | import net.ceedubs.ficus.Ficus._
12 | import net.ceedubs.ficus.readers.ArbitraryTypeReader._
13 | import scala.util.control.NoStackTrace
14 | import scala.util.{Failure, Try}
15 |
16 | object SettingsProcessingFlow {
17 | /** Creates a flow that processes input maps that are keyed by the mirror IDs and the values
18 | * are Kafka settings, and converts them into a stream of [[MirrorCommand]]s that corresponds
19 | * to the changes in the incoming settings.
20 | */
21 | def apply(fallback: Config)
22 | (implicit logger: LoggingAdapter): Flow[Map[MirrorID, Option[String]], MirrorCommand, NotUsed] = {
23 | val init = (Map.empty[MirrorID, ExternalKafkaMirrorSettings], List.empty[MirrorCommand])
24 |
25 | def error(id: MirrorID)(e: SettingsErrors) = {
26 | logger.error(e,
27 | s"Invalid Kafka mirror configuration for [${id.value}], " +
28 | s"the mirror will not be started and any running mirrors with this ID will be stopped")
29 |
30 | None
31 | }
32 |
33 | Flow[Map[MirrorID, Option[String]]]
34 | .map(fromRawMap(fallback))
35 | .map { state =>
36 | state.flatMap {
37 | case (id, maybeSettings) =>
38 | maybeSettings.fold( //TODO do we actually want to stop running the mirror, or maybe just revert to the old settings?
39 | error(id),
40 | s => Some(id -> s)
41 | )
42 | }
43 | }
44 | .scan(init) { case ((prevState, _), curState) =>
45 | val commands = computeCommands(
46 | prevState = prevState,
47 | curState = curState)
48 |
49 | (curState, commands)
50 | }
51 | .mapConcat { case (_, commands) => commands }
52 | }
53 |
54 | /** Computes the [[MirrorCommand]]s that correspond to the change of state of the settings. */
55 | def computeCommands(prevState: Map[MirrorID, ExternalKafkaMirrorSettings],
56 | curState: Map[MirrorID, ExternalKafkaMirrorSettings]): List[MirrorCommand] = {
57 |
58 | val ids = (prevState.keys ++ curState.keys).toSet
59 |
60 | ids.flatMap { id =>
61 | (prevState.get(id), curState.get(id)) match {
62 | case (Some(prevSettings), Some(curSettings))
63 | if prevSettings == curSettings => List.empty
64 |
65 | case (Some(_), Some(curSettings)) if shouldStart(curSettings) =>
66 | List(Start(id, curSettings))
67 |
68 | case (Some(_), Some(curSettings)) if !shouldStart(curSettings) =>
69 | List(Stop(id))
70 |
71 | case (Some(_), None) =>
72 | List(Stop(id))
73 |
74 | case (None, Some(curSettings)) if shouldStart(curSettings) =>
75 | List(Start(id, curSettings))
76 |
77 | case (None, Some(curSettings)) if !shouldStart(curSettings) =>
78 | List.empty
79 |
80 | case (None, None) =>
81 | List.empty
82 | }
83 | }.toList
84 | }
85 |
86 | private def shouldStart(settings: ExternalKafkaMirrorSettings) =
87 | settings.mirror.enabled
88 |
89 | /** Converts a bunch of [[Config]] objects into [[ExternalKafkaMirrorSettings]].
90 | *
91 | * @param fallback Provides defaults for the consumer and producer [[Config]]s.
92 | */
93 | def fromConfig(fallback: Config)
94 | (consumer: Config,
95 | producer: Config,
96 | mirror: MirrorSettings): ValidOrErrors[ExternalKafkaMirrorSettings] = {
97 |
98 | val consumerFallback = fallback.as[ValidOrErrors[Config]]("akka.kafka.consumer")
99 | val producerFallback = fallback.as[ValidOrErrors[Config]]("akka.kafka.producer")
100 |
101 | val consumerFull = consumerFallback.map(consumer.withFallback)
102 | val producerFull = producerFallback.map(producer.withFallback)
103 |
104 | (consumerFull, producerFull).mapN { (c, p) =>
105 | ExternalKafkaMirrorSettings(
106 | ExternalKafkaSettings(
107 | consumer = c,
108 | producer = p),
109 | mirror)
110 | }
111 | }
112 |
113 | val (consumerKey, producerKey, mirrorKey) = ("consumer", "producer", "mirror")
114 |
115 | /** Reads settings objects from a map that is keyed by the mirror IDs. */
116 | def fromRawMap(fallback: Config)
117 | (data: Map[MirrorID, Option[String]]): Map[MirrorID, Validated[SettingsErrors, ExternalKafkaMirrorSettings]] = {
118 |
119 | def parseConfigData(id: String, rawData: Option[String]) = {
120 | def error = Failure {
121 | new NoSuchElementException(s"Data is missing for ID: [$id]")
122 | }
123 |
124 | Validated.fromTry {
125 | rawData.map(Try(_)).getOrElse(error).flatMap(str => Try(ConfigFactory.parseString(str)))
126 | }.toValidatedNel
127 | }
128 |
129 | val validatedMirrors = data.map { case (id, rawData) =>
130 | id -> {
131 | val validatedConfig = parseConfigData(id.value, rawData).andThen { config =>
132 | val consumer = config.as[ValidOrErrors[Config]](consumerKey)
133 | val producer = config.as[ValidOrErrors[Config]](producerKey)
134 | val mirror = config.as[ValidOrErrors[MirrorSettings]](mirrorKey)
135 |
136 | (consumer, producer, mirror).mapN(fromConfig(fallback)).flatten
137 | }
138 |
139 | validatedConfig.leftMap(new SettingsErrors(_))
140 | }
141 | }
142 |
143 | validatedMirrors
144 | }
145 |
146 | class SettingsErrors(errors: NonEmptyList[Throwable]) extends NoStackTrace { // no stack trace because it clutters the output
147 | private def errorsString(errors: NonEmptyList[Throwable]) =
148 | errors.toList.map("- " ++ _.getMessage).mkString("\n")
149 |
150 | override def getMessage = s"""
151 | |Failed to parse the settings.
152 | |The settings should be in the following form:
153 | |A single HOCON configuration entry under the root with the keys:
154 | |- '$consumerKey'
155 | |- '$producerKey'
156 | |- '$mirrorKey'
157 | |Each with its own valid configuration nested in it.
158 | |
159 | |The errors that were produced are:
160 | |${errorsString(errors)}""".stripMargin
161 | }
162 | }
163 |
--------------------------------------------------------------------------------
/core/src/it/scala/com/supersonic/kafka_mirror/AkkaStreamsKafkaIntegrationSpec.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.kafka_mirror
2 |
3 | import java.net.ServerSocket
4 | import java.util.UUID
5 | import akka.NotUsed
6 | import akka.kafka.ProducerMessage.Message
7 | import akka.kafka.scaladsl.{Consumer, Producer}
8 | import akka.kafka.{ConsumerSettings, ProducerSettings, Subscriptions}
9 | import akka.stream.ActorMaterializer
10 | import akka.stream.scaladsl.{Keep, Sink, Source}
11 | import akka.stream.testkit.TestSubscriber
12 | import akka.stream.testkit.scaladsl.TestSink
13 | import akka.testkit.TestKitBase
14 | import kafka.server.KafkaServer
15 | import net.manub.embeddedkafka.{EmbeddedKafka, EmbeddedKafkaConfig}
16 | import org.apache.kafka.clients.consumer.ConsumerConfig
17 | import org.apache.kafka.clients.producer.ProducerRecord
18 | import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializer}
19 | import org.apache.zookeeper.server.ServerCnxnFactory
20 | import org.scalatest.{BeforeAndAfterAll, TestSuite}
21 | import scala.collection.immutable
22 | import scala.concurrent.Await
23 | import scala.concurrent.duration._
24 | import scala.reflect.io.Directory
25 |
26 | /** A base trait for tests that need to use an embedded Kafka instance together with Akka streams. */
27 | trait AkkaStreamsKafkaIntegrationSpec extends TestKitBase
28 | with TestSuite
29 | with BeforeAndAfterAll {
30 | implicit val ec = system.dispatcher
31 | implicit val materializer = ActorMaterializer()(system)
32 |
33 | case class KafkaConnection(kafka: KafkaServer,
34 | zookeeper: ServerCnxnFactory,
35 | config: EmbeddedKafkaConfig)
36 |
37 | private var kafkaConnection1: KafkaConnection = _
38 | private var kafkaConnection2: KafkaConnection = _
39 |
40 | def kafkaHelper1 = new KafkaHelper(kafkaConnection1.config)
41 |
42 | def kafkaHelper2 = new KafkaHelper(kafkaConnection2.config)
43 |
44 | def uuid() = UUID.randomUUID().toString
45 |
46 | def createTopicOnServers(number: Int) = {
47 | val topic = s"topic$number-" + uuid()
48 |
49 | EmbeddedKafka.createCustomTopic(topic)(kafkaConnection1.config)
50 | EmbeddedKafka.createCustomTopic(topic)(kafkaConnection2.config)
51 |
52 | topic
53 | }
54 |
55 | def createGroup(number: Int) = s"group$number-" + uuid()
56 |
57 | def createProbe(consumerSettings: ConsumerSettings[String, String],
58 | topic: String): TestSubscriber.Probe[String] = {
59 | Consumer.committableSource(consumerSettings, Subscriptions.topics(topic))
60 | .mapAsync(5) { message =>
61 | message.committableOffset.commitScaladsl().map { _ =>
62 | message.record.value
63 | }
64 | }
65 | .runWith(TestSink.probe)
66 | }
67 |
68 | def produceMessages(topic: String, messages: Range) =
69 | Await.result(kafkaHelper1.produce(topic, messages), Duration.Inf)
70 |
71 | def mirrorSettings(group: String, bucketSettings: Option[BucketSettings] = None)(whitelist: String*) =
72 | KafkaMirrorSettings(
73 | KafkaSettings(kafkaHelper1.createConsumerSettings(group), kafkaHelper2.producerSettings),
74 | MirrorSettings(
75 | whitelist = whitelist.toSet,
76 | commitBatchSize = 20,
77 | commitParallelism = 3,
78 | bucketSettings),
79 | (_: String).toInt,
80 | // since partition numbers start from '0', we must subtract here
81 | (n: Int) => n - 1)
82 |
83 | def verifyTopic(topic: String, group: String, messages: immutable.Seq[Int]) = {
84 | val targetTopicProbe = createProbe(kafkaHelper2.createConsumerSettings(group), topic)
85 |
86 | targetTopicProbe
87 | .request(messages.size)
88 |
89 | messages.foreach { elem =>
90 | targetTopicProbe.expectNext(10.seconds, elem.toString)
91 | }
92 |
93 | targetTopicProbe.request(messages.size)
94 | targetTopicProbe.expectNoMessage(3.seconds)
95 |
96 | targetTopicProbe.cancel()
97 | }
98 |
99 | override protected def beforeAll(): Unit = {
100 | super.beforeAll()
101 | startKafka()
102 | }
103 |
104 | override def afterAll(): Unit = {
105 | shutdown(system, 30.seconds)
106 | shutdownKafka()
107 | super.afterAll()
108 | }
109 |
110 | private def startKafka() = {
111 | def connectToKafka() = {
112 |
113 | val zkLogsDir = Directory.makeTemp("zookeeper-logs")
114 | val kafkaLogsDir = Directory.makeTemp("kafka-logs")
115 |
116 | val zookeeper = EmbeddedKafka.startZooKeeper(0, zkLogsDir)
117 |
118 | val config = EmbeddedKafkaConfig(kafkaPort = getFreePort(), zooKeeperPort = zookeeper.getLocalPort)
119 | val kafka = EmbeddedKafka.startKafka(config, kafkaLogsDir)
120 |
121 | KafkaConnection(kafka, zookeeper, config)
122 | }
123 |
124 | kafkaConnection1 = connectToKafka()
125 | kafkaConnection2 = connectToKafka()
126 | }
127 |
128 | private def shutdownKafka() =
129 | List(kafkaConnection1, kafkaConnection2).foreach { connection =>
130 | connection.kafka.shutdown()
131 | connection.zookeeper.shutdown()
132 | }
133 |
134 | private def getFreePort() = {
135 | var socket: ServerSocket = null
136 | try {
137 | socket = new ServerSocket(0)
138 | socket.setReuseAddress(true)
139 | socket.getLocalPort
140 | } finally socket.close()
141 | }
142 |
143 | class KafkaHelper(embeddedKafkaConfig: EmbeddedKafkaConfig) {
144 | val bootstrapServers = s"localhost:${embeddedKafkaConfig.kafkaPort}"
145 |
146 | val producerSettings = ProducerSettings(system, new StringSerializer, new StringSerializer)
147 | .withBootstrapServers(bootstrapServers)
148 |
149 | val partition0 = 0
150 |
151 | /** Produce messages to topic using specified range and return
152 | * a Future so the caller can synchronize consumption.
153 | */
154 | def produce(topic: String, range: Range) = {
155 | val source = Source(range)
156 | .map(n => {
157 | val record = new ProducerRecord(topic, partition0, n.toString, n.toString)
158 |
159 | Message(record, NotUsed)
160 | })
161 | .viaMat(Producer.flexiFlow(producerSettings))(Keep.right)
162 |
163 | source.runWith(Sink.ignore)
164 | }
165 |
166 | def createConsumerSettings(group: String): ConsumerSettings[String, String] = {
167 | ConsumerSettings(system, new StringDeserializer, new StringDeserializer)
168 | .withBootstrapServers(bootstrapServers)
169 | .withGroupId(group)
170 | .withProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest")
171 | }
172 | }
173 | }
174 |
175 |
176 |
--------------------------------------------------------------------------------
/app-common/src/it/scala/com/supersonic/integration/KafkaMirrorIntegrationTemplate.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.integration
2 |
3 | import akka.actor.ActorSystem
4 | import akka.event.Logging
5 | import akka.stream.ActorMaterializer
6 | import akka.stream.scaladsl.Source
7 | import akka.stream.testkit.TestSubscriber.Probe
8 | import com.supersonic.integration.KafkaMirrorIntegrationTemplate.MirrorConfigManager
9 | import com.supersonic.kafka_mirror.MirrorCommand.VerifyState
10 | import com.supersonic.kafka_mirror._
11 | import org.scalatest.{AsyncWordSpecLike, Matchers}
12 | import scala.concurrent.duration._
13 | import scala.concurrent.{Await, Future}
14 |
15 | /** A template class for testing a Kafka mirroring flow. */
16 | trait KafkaMirrorIntegrationTemplate extends AkkaStreamsKafkaIntegrationSpec
17 | with AsyncWordSpecLike
18 | with Matchers {
19 | protected def configSourceName: String
20 |
21 | /** The type to which the configuration source is materialized. */
22 | protected type ConfigMat
23 |
24 | protected implicit lazy val logger = Logging(system.eventStream, "KafkaMirrorIntegration")
25 |
26 | private lazy val mirrorActorSystem = ActorSystem("MirrorActorSystem")
27 | private lazy val mirrorMaker = new SourceBackedMirrorMaker(ActorMaterializer()(mirrorActorSystem), logger)
28 | private lazy val mirrorManager = new MirrorManager(mirrorMaker)(mirrorActorSystem.dispatcher, logger)
29 |
30 | private def messages1 = 1 to 100
31 |
32 | private def messages2 = 101 to 150
33 |
34 | protected type MirrorConfigSource = Source[Map[MirrorID, Option[String]], ConfigMat]
35 | protected type RunningMirrorSource = Source[Map[MirrorID, RunningMirror], ConfigMat]
36 |
37 | /** Creates a probe that is used to test the whole flow.
38 | * The transform takes some input and converts into a Kafka-mirroring source.
39 | */
40 | protected def withProbe[B](transform: MirrorConfigSource => RunningMirrorSource)
41 | (f: (Probe[Map[MirrorID, RunningMirror]], MirrorConfigManager) => B): B
42 |
43 | private def makeMirrorSource(source: MirrorConfigSource): RunningMirrorSource =
44 | source.via(SettingsProcessingFlow(system.settings.config))
45 | .merge(Source.tick(1.second, 1.second, VerifyState))
46 | .via(mirrorManager.flow)
47 |
48 | protected def runKafkaMirrorIntegration(): Unit = {
49 | s"A Kafka mirror backed by $configSourceName" should {
50 |
51 | s"start and stop and clean mirrors according to changes in $configSourceName" in {
52 | val topic1 = createTopicOnServers(1)
53 | val topic2 = createTopicOnServers(2)
54 |
55 | val group1 = createGroup(1)
56 | val group2 = createGroup(2)
57 | val group3 = createGroup(3)
58 | val group4 = createGroup(4)
59 |
60 | withProbe(makeMirrorSource) { (mirrorsProbe, mirrorConfigManager) =>
61 | import mirrorConfigManager._
62 |
63 | val waitForID = Waiter(mirrorsProbe)
64 |
65 | produceMessages(topic1, messages1)
66 |
67 | info("starting the 'bar' mirror")
68 | addMirror("bar", mirrorSettings(topic1, group1))
69 |
70 | info("verifying that 'bar' is active")
71 | waitForID("bar")
72 | verifyMirroring(topic1, group2, messages1)
73 |
74 | info("starting the 'qux' mirror")
75 | addMirror("qux", mirrorSettings(topic2, group3))
76 |
77 | produceMessages(topic2, messages1)
78 |
79 | info("verifying that 'qux' is active")
80 | waitForID("qux")
81 | verifyMirroring(topic2, group4, messages1)
82 |
83 | info("stopping bar")
84 | deleteMirror("bar")
85 | waitForID("bar", checkPresent = false)
86 |
87 | info("verifying that 'bar' is no longer active")
88 | verifyNoMirroring(topic1, group2)
89 |
90 | info("shutting down the 'qux' mirror and waiting for it to be automatically restarted")
91 | waitForID.state(MirrorID("qux")).control.shutdown().map { _ =>
92 | produceMessages(topic2, messages2)
93 |
94 | info("verifying that 'qux' was restarted")
95 | verifyMirroring(topic2, group4, messages2)
96 |
97 | succeed
98 | }
99 | }
100 | }
101 | }
102 | }
103 |
104 | def verifyMirroring(topic: String, group: String, messages: Range) = Future {
105 | val topicProbe = createProbe(kafkaHelper2.createConsumerSettings(group), topic)
106 |
107 | topicProbe
108 | .request(messages.size)
109 | .expectNextN(messages.map(_.toString))
110 |
111 | topicProbe.cancel()
112 | }
113 |
114 | def verifyNoMirroring(topic: String, group: String) = {
115 | Await.result(kafkaHelper1.produce(topic, 200 to 250), remainingOrDefault)
116 |
117 | val targetProbe = createProbe(kafkaHelper2.createConsumerSettings(group), topic)
118 | targetProbe
119 | .request(50)
120 | .expectNoMessage(3.seconds)
121 |
122 | targetProbe.cancel()
123 | }
124 |
125 | def mirrorSettings(topic: String, group: String) = s"""
126 | consumer = {
127 | kafka-clients = {
128 | bootstrap.servers = "${kafkaHelper1.bootstrapServers}"
129 | group.id = "$group"
130 | auto.offset.reset = "earliest"
131 | }
132 | }
133 |
134 | producer = {
135 | kafka-clients = {
136 | bootstrap.servers = "${kafkaHelper2.bootstrapServers}"
137 | }
138 | }
139 |
140 | mirror = {
141 | whitelist = [$topic]
142 | commitBatchSize = 5
143 | commitParallelism = 4
144 | }
145 | """
146 |
147 | case class Waiter(mirrorsProbe: Probe[Map[MirrorID, RunningMirror]]) {
148 | @volatile var state: Map[MirrorID, RunningMirror] = Map.empty
149 |
150 | def apply(id: String, checkPresent: Boolean = true) = {
151 | var attempts = 50
152 |
153 | def check() = {
154 | val mirrorPresent = state.contains(MirrorID(id))
155 |
156 | if (checkPresent) mirrorPresent
157 | else !mirrorPresent
158 | }
159 |
160 | while (!check() && attempts > 0) {
161 | state = mirrorsProbe.requestNext(1.minute)
162 | attempts -= 1
163 | }
164 |
165 | if (attempts == 0) fail(s"Failed while waiting for the [$id] mirror ID")
166 | }
167 | }
168 | }
169 |
170 | object KafkaMirrorIntegrationTemplate {
171 | /** A trait that allows creating and removing mirror configurations. */
172 | trait MirrorConfigManager {
173 |
174 | /** Creates a new mirror with the given settings. */
175 | def addMirror(name: String, settings: String): Unit
176 |
177 | /** Deletes a mirror with the given name. */
178 | def deleteMirror(name: String): Unit
179 | }
180 | }
181 |
--------------------------------------------------------------------------------
/app-common/src/main/scala/com/supersonic/main/KafkaMirrorAppTemplate.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.main
2 |
3 | import akka.actor.ActorSystem
4 | import akka.event.{Logging, LoggingAdapter}
5 | import akka.http.scaladsl.Http
6 | import akka.http.scaladsl.marshallers.sprayjson.SprayJsonSupport._
7 | import akka.http.scaladsl.server.Directives._
8 | import akka.http.scaladsl.server.Route
9 | import akka.stream.FlowMonitorState.{Failed, Finished, Initialized, Received}
10 | import akka.stream.scaladsl.{Keep, Sink, Source}
11 | import akka.stream.{ActorMaterializer, FlowMonitor, Materializer}
12 | import com.supersonic.BuildInfo
13 | import com.supersonic.kafka_mirror.MirrorCommand.VerifyState
14 | import com.supersonic.kafka_mirror._
15 | import com.supersonic.main.KafkaMirrorAppTemplate._
16 | import com.supersonic.util.ActorSystemUtil._
17 | import com.supersonic.util.SignalingUtil
18 | import com.typesafe.config.ConfigFactory
19 | import net.ceedubs.ficus.Ficus._
20 | import net.ceedubs.ficus.readers.ValueReader
21 | import shapeless.Typeable
22 | import spray.json.JsonFormat
23 | import scala.concurrent.duration._
24 | import scala.concurrent.{Await, ExecutionContext}
25 |
26 | /** An abstract skeleton for apps that run a Kafka mirroring service.
27 | * This includes a healthcheck server with info about the app as well as the mirroring process itself.
28 | *
29 | * Implementors of this class need to provide a backend source for mirror configuration to run this
30 | * app.
31 | */
32 | trait KafkaMirrorAppTemplate {
33 | /** The value that is materialized by the source of mirror configuration for the app. */
34 | protected type MirrorConfigSourceMat
35 |
36 | /** The application setting to be used by the application. */
37 | protected type AppSettings <: BaseAppSettings
38 |
39 | protected implicit def appSettingsValueReader: ValueReader[AppSettings]
40 |
41 | protected implicit def appSettingsJSONFormat: JsonFormat[AppSettings]
42 |
43 | /** Needed for the derivation of the AppState JSON format */
44 | protected implicit def appSettingsTypeable: Typeable[AppSettings]
45 |
46 | /** Will be used to render the current state in an HTTP-route. */
47 | protected implicit val appStateFormat = AppState.format[AppSettings]
48 |
49 | /** A callback to be invoked when the application completes. */
50 | protected def onCompletion(matValue: MirrorConfigSourceMat): Unit
51 |
52 | /** An optional custom route to be served by the application instead of [[defaultRoute]].
53 | * The given callback give access to the current state of the application.
54 | */
55 | protected def customRoute(currentState: () => AppState[AppSettings]): Option[Route]
56 |
57 | /** The type of sources that provide mirror configuration for this app. */
58 | protected type MirrorConfigSource = Source[Map[MirrorID, Option[String]], MirrorConfigSourceMat]
59 |
60 | /** Given the settings for the application, creates a source of mirror configuration and invoke
61 | * the given continuation with it.
62 | *
63 | * @param continuation A continuation that must be invoked when by the implementors of this class.
64 | */
65 | protected def withConfigSource[A](appSettings: AppSettings)
66 | (continuation: MirrorConfigSource => A)
67 | (implicit logger: LoggingAdapter,
68 | executionContext: ExecutionContext): A
69 |
70 | def main(args: Array[String]): Unit = withActorSystem("KafkaMirrorApp") { implicit system =>
71 | implicit val logger: LoggingAdapter = Logging(system.eventStream, "KafkaMirrorApp")
72 | implicit val materializer: Materializer = ActorMaterializer()
73 | implicit val executionContext: ExecutionContext = system.dispatcher
74 |
75 | val appSettings = ConfigFactory.load().as[AppSettings]("kafka-mirror-settings")
76 |
77 | withConfigSource(appSettings) { mirrorConfigSource =>
78 | val mirrorMaker = new SourceBackedMirrorMaker(materializer, logger)
79 | val mirrorManager = new MirrorManager(mirrorMaker)
80 |
81 | val stateVerificationInterval = appSettings.stateVerificationInterval.getOrElse(stateVerificationIntervalDefault)
82 |
83 | val flow = mirrorConfigSource
84 | .via(SettingsProcessingFlow(system.settings.config))
85 | .merge(Source.tick(stateVerificationInterval, stateVerificationInterval, VerifyState))
86 | .mergeMat(Source.maybe[MirrorCommand.Shutdown.type])(Keep.both)
87 | .via(mirrorManager.flow)
88 | .monitor
89 |
90 | val (((mirrorConfigSourceMat, shutdownHandle), monitor), endOfWorld) = flow.toMat(Sink.ignore)(Keep.both).run()
91 |
92 | val serverBinding = initRoute(appSettings, monitor)
93 |
94 | def shutdown() = {
95 | onCompletion(mirrorConfigSourceMat)
96 | shutdownHandle.trySuccess(Some(MirrorCommand.Shutdown))
97 | val _ = Await.ready(serverBinding.unbind(), serverShutdownTimeout)
98 | }
99 |
100 | endOfWorld.onComplete { _ =>
101 | onCompletion(mirrorConfigSourceMat)
102 | }
103 |
104 | SignalingUtil.registerHandler(shutdown)
105 |
106 | val _ = Await.result(endOfWorld, Duration.Inf)
107 | }
108 | }
109 |
110 | private def initRoute(appSettings: AppSettings,
111 | monitor: FlowMonitor[Map[MirrorID, RunningMirror]])
112 | (implicit logger: LoggingAdapter,
113 | actorSystem: ActorSystem,
114 | materializer: Materializer,
115 | executionContext: ExecutionContext): Http.ServerBinding = {
116 | val currentState = () => { //TODO add mirrors that are present in the config but not properly configured
117 | val currentMirrors = monitor.state match {
118 | case Received(mirrors) => Right(mirrors.mapValues(_.mirrorSettings).toList)
119 | case Initialized => Left("Initialized Kafka mirrors stream")
120 | case Failed(t) => Left(s"Failed while running Kafka mirrors stream: ${t.getMessage}")
121 | case Finished => Left(s"Finished running Kafka mirrors stream")
122 | }
123 |
124 | AppState(
125 | appVersion = BuildInfo.gitDescribedVersion,
126 | commitHash = BuildInfo.gitCommit,
127 | settings = appSettings,
128 | mirrors = currentMirrors)
129 | }
130 |
131 | val route = customRoute(currentState).getOrElse(defaultRoute(currentState))
132 |
133 | val eventualServerBinding = Http().bindAndHandle(route, "0.0.0.0", appSettings.port)
134 |
135 | val serverBinding = Await.result(eventualServerBinding, Duration.Inf)
136 | logger.info(s"Server: ${serverBinding.localAddress} started successfully")
137 |
138 | serverBinding
139 | }
140 |
141 | private def defaultRoute(currentState: () => AppState[AppSettings]): Route =
142 | path("healthcheck") {
143 | get {
144 | complete(currentState())
145 | }
146 | }
147 | }
148 |
149 | object KafkaMirrorAppTemplate {
150 | trait BaseAppSettings {
151 | def port: Int
152 |
153 | def stateVerificationInterval: Option[FiniteDuration]
154 | }
155 |
156 | val stateVerificationIntervalDefault = 5.seconds
157 | val serverShutdownTimeout = 5.seconds
158 | }
159 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/supersonic/kafka_mirror/KafkaMirror.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.kafka_mirror
2 |
3 | import akka.Done
4 | import akka.event.LoggingAdapter
5 | import akka.kafka.ConsumerMessage.{CommittableMessage, CommittableOffset, CommittableOffsetBatch}
6 | import akka.kafka.ProducerMessage.Message
7 | import akka.kafka.scaladsl.Consumer.Control
8 | import akka.kafka.scaladsl.{Consumer, Producer}
9 | import akka.kafka.{ProducerMessage, Subscriptions}
10 | import akka.stream.scaladsl.{Keep, Sink, Source}
11 | import akka.stream.{ActorAttributes, Materializer, Supervision}
12 | import org.apache.kafka.clients.consumer.CommitFailedException
13 | import org.apache.kafka.clients.producer.{ProducerRecord, Producer => KafkaProducer}
14 | import org.apache.kafka.common.record.RecordBatch
15 |
16 | /** Represents a mirror between two Kafka topics.
17 | * The mirror can be started, an action that produces a [[Control]] through which the mirroring
18 | * can be shut down.
19 | */
20 | trait KafkaMirror {
21 | /** Starts the mirror described by this instance and returns a [[Control]] to the process. */
22 | def start(): Control
23 | }
24 |
25 | /** Encapsulates the logic for the creation of Kafka mirrors. */
26 | trait MirrorMaker {
27 | type Mirror <: KafkaMirror
28 |
29 | /** Builds a new mirror with the given ID.
30 | *
31 | * @param producer The producer that is being used by the mirror. Needed so that
32 | * so that it can be closed upon completion.
33 | */
34 | def apply(mirrorID: MirrorID, producer: KafkaProducer[_, _])
35 | (source: LoggingAdapter => Source[Done, Control]): Mirror
36 | }
37 |
38 | object KafkaMirror {
39 | /** A workaround for https://github.com/akka/alpakka-kafka/issues/755
40 | * Can be removed once Alpakka Kafka 1.0.2 is released.
41 | */
42 | private val recoverCommitErrors: Supervision.Decider = {
43 | case _: CommitFailedException => Supervision.Resume
44 | case _ => Supervision.Stop
45 | }
46 |
47 | /** Creates a mirror that streams data from the source topic to the target topic as
48 | * specified in the settings object.
49 | */
50 | def apply[K, V](makeMirror: MirrorMaker)
51 | (id: MirrorID,
52 | settings: KafkaMirrorSettings[K, V]): makeMirror.Mirror = {
53 | // This producer will not be managed by the library, so we have to close it upon stream
54 | // completion, that's why we pass it to [[makeMirror]].
55 | val producer = settings.kafka.producer.createKafkaProducer()
56 |
57 | makeMirror(id, producer) { implicit logger =>
58 | val mirrorName = s"kafka-mirror-${id.value}"
59 |
60 | Consumer.committableSource(
61 | settings.kafka.consumer,
62 | Subscriptions.topics(settings.mirror.whitelist))
63 | .mapConcat(makeMessage[K, V](
64 | settings.mirror,
65 | settings.hashKey,
66 | producer.partitionsFor(_).size,
67 | settings.generatePartition) _ andThen (_.toList))
68 | .via(Producer.flexiFlow(settings.kafka.producer, producer))
69 | .map(_.passThrough)
70 | .batch(
71 | max = settings.mirror.commitBatchSize,
72 | CommittableOffsetBatch.empty.updated)(_ updated _)
73 | // TODO or should it be producerSettings.parallelism?
74 | .mapAsync(settings.mirror.commitParallelism)(_.commitScaladsl())
75 | .withAttributes(ActorAttributes.supervisionStrategy(recoverCommitErrors))
76 | .named(mirrorName) // to help debugging
77 | .log(mirrorName)
78 | }
79 | }
80 |
81 | /** Converts a message from a specific topic in a consumer to the corresponding topic and message in the
82 | * producer.
83 | * This includes support for bucketing, as a result it is possible that there will be no messages.
84 | */
85 | private[kafka_mirror] def makeMessage[K, V](mirror: MirrorSettings,
86 | hashKey: K => Int,
87 | getNumOfPartitions: String => Int,
88 | generatePartition: Int => Int)
89 | (message: CommittableMessage[K, V]): Option[Message[K, V, CommittableOffset]] = {
90 | // can be null, leaving it here, since we need to pass it along to Kafka later on
91 | val rawKey = message.record.key
92 | val maybeKey = Option(rawKey)
93 |
94 | def shouldSend(bucketSettings: BucketSettings): Boolean =
95 | // 'null' keys are always mirrored, since bucketing is used for (deterministic) mirroring by key
96 | // there is no point to mirror 'null' as it will always land in the same bucket.
97 | // So we consider 'null' to be an indication that the user doesn't need bucketing
98 | maybeKey
99 | .forall(key => hashKey(key) % bucketSettings.totalBuckets < bucketSettings.mirrorBuckets)
100 |
101 | def getPartitionNumber(topic: String) = {
102 | val totalPartitions = getNumOfPartitions(topic)
103 |
104 | def partitionFromKey(key: K) = hashKey(key) % totalPartitions
105 |
106 | def randomPartition = generatePartition(totalPartitions)
107 |
108 | maybeKey
109 | .filter(_ => mirror.partitionFromKeys)
110 | .map(partitionFromKey)
111 | .getOrElse(randomPartition)
112 | }
113 |
114 | val send = mirror.bucketing.forall(shouldSend)
115 | if (send) {
116 | val timestamp: java.lang.Long =
117 | if (message.record.timestamp == RecordBatch.NO_TIMESTAMP) null else message.record.timestamp
118 |
119 | val topic = mirror.topicsToRename.getOrElse(message.record.topic, message.record.topic)
120 |
121 | val partition = getPartitionNumber(topic)
122 |
123 | Some {
124 | ProducerMessage.Message(new ProducerRecord[K, V](
125 | topic,
126 | partition,
127 | timestamp,
128 | rawKey,
129 | message.record.value),
130 | message.committableOffset)
131 | }
132 | }
133 | else Option.empty
134 | }
135 | }
136 |
137 | /** A Kafka mirror that is backed by a [[Source]].
138 | *
139 | * @param producer The producer that's being used by the mirror. Needed here for cleanup after the
140 | * mirror completes.
141 | */
142 | case class SourcedKafkaMirror(mirrorID: MirrorID,
143 | source: Source[Done, Control],
144 | producer: KafkaProducer[_, _],
145 | materializer: Materializer,
146 | logger: LoggingAdapter) extends KafkaMirror {
147 | def start(): Control = {
148 | logger.info(s"Starting Kafka mirror with ID: [${mirrorID.value}]")
149 |
150 | val closeProducer = Sink.onComplete { _ =>
151 | // note that some of the tests do not use the [[start]] method, which means that this 'close'
152 | // call will not be performed there.
153 | producer.close()
154 | }
155 |
156 | source.toMat(closeProducer)(Keep.left).run()(materializer)
157 | }
158 | }
159 |
160 | /** Creates Kafka mirrors that are backed by a [[Source]] using the provided materializer. */
161 | class SourceBackedMirrorMaker(materializer: Materializer,
162 | logger: LoggingAdapter) extends MirrorMaker {
163 | type Mirror = SourcedKafkaMirror
164 |
165 | def apply(mirrorID: MirrorID, producer: KafkaProducer[_, _])
166 | (source: LoggingAdapter => Source[Done, Control]) =
167 | SourcedKafkaMirror(mirrorID, source(logger), producer, materializer, logger)
168 | }
169 |
--------------------------------------------------------------------------------
/core/src/test/scala/com/supersonic/kafka_mirror/MirrorManagerHelpersTest.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.kafka_mirror
2 |
3 | import akka.actor.ActorSystem
4 | import akka.event.Logging
5 | import akka.stream.KillSwitches
6 | import com.supersonic.kafka_mirror.MirrorCommand._
7 | import com.supersonic.kafka_mirror.MirrorManager._
8 | import com.supersonic.kafka_mirror.TestUtil.{settings, _}
9 | import org.scalatest.{AsyncWordSpec, Inspectors, Matchers}
10 | import scala.concurrent.Future
11 |
12 | class MirrorManagerHelpersTest extends AsyncWordSpec with Matchers with Inspectors {
13 | implicit val system = ActorSystem("MirrorManagerHelpersTest")
14 | implicit val logger = Logging(system.eventStream, "MirrorManagerHelpersTest")
15 |
16 | "Executing the mirror commands" when {
17 | def execute(newMirror: => KafkaMirror = new MockKafkaMirror) =
18 | executeMirrors(new MockMirrorMaker(newMirror), KillSwitches.shared("kill-switch")) _
19 |
20 | "receiving the start command it" should {
21 | "restart the mirror if it's present" in {
22 | val newMirror = new MockKafkaMirror
23 | val command = Start(id("a"), runnableSettings("a"))
24 |
25 | val control = new MockControl
26 | val mirror = RunningMirror(control, settings("a"))
27 |
28 | val state = Map(id("a") -> mirror)
29 |
30 | execute(newMirror)(state, command).map { newState =>
31 | control.isShutdownNow() shouldBe true
32 | newState should contain key id("a")
33 | newMirror.isStarted() shouldBe true
34 | newState(id("a")).control shouldBe newMirror.control.get
35 | }
36 | }
37 |
38 | "start the mirror it isn't present" in {
39 | val newMirror = new MockKafkaMirror
40 | val command = Start(id("a"), runnableSettings("a"))
41 |
42 | execute(newMirror)(Map.empty, command).map { newState =>
43 | newState should contain key id("a")
44 | newMirror.isStarted() shouldBe true
45 | newState(id("a")).control shouldBe newMirror.control.get
46 | }
47 | }
48 |
49 | "fail to start when required Kafka settings are missing" in {
50 | val s = settings("a") // the settings should not be runnable
51 | an[Exception] should be thrownBy s.toKafkaMirrorSettings.get
52 |
53 | execute()(Map.empty, Start(id("a"), s)).map { newState =>
54 | newState shouldBe empty
55 | }
56 | }
57 |
58 | "fail to start when the mirror fails to start" in {
59 | val command = Start(id("a"), runnableSettings("a"))
60 |
61 | execute(FailingKafkaMirror)(Map.empty, command).map { newState =>
62 | newState shouldBe empty
63 | }
64 | }
65 | }
66 |
67 | "receiving the stop command it" should {
68 | "stop the mirror if it is present and remove it" in {
69 | val control = new MockControl
70 | val mirror = RunningMirror(control, settings("a"))
71 |
72 | val command = Stop(id("a"))
73 |
74 | val state = Map(id("a") -> mirror)
75 |
76 | execute()(state, command).map { newState =>
77 | control.isShutdownNow() shouldBe true
78 | newState shouldNot contain key id("a")
79 | }
80 | }
81 |
82 | "do nothing if it isn't" in {
83 | val command = Stop(id("a"))
84 |
85 | execute()(Map.empty, command).map { newState =>
86 | newState shouldBe empty
87 | }
88 | }
89 |
90 | "remove mirrors that failed to stop" in {
91 | val mirror = RunningMirror(FailingControl, settings("a"))
92 |
93 | val command = Stop(id("a"))
94 |
95 | val state = Map(id("a") -> mirror)
96 |
97 | execute()(state, command).map { newState =>
98 | newState shouldNot contain key id("a")
99 | }
100 | }
101 | }
102 |
103 | "receiving the shutdown command it" should {
104 | "stop all current mirrors and remove them" in {
105 | val c1, c2, c3, c4 = new MockControl
106 |
107 | val m1 = RunningMirror(c1, settings("a"))
108 | val m2 = RunningMirror(c2, settings("b"))
109 | val m3 = RunningMirror(c3, settings("c"))
110 | val m4 = RunningMirror(c4, settings("d"))
111 |
112 | val command = Shutdown
113 |
114 | val state = Map(
115 | id("a") -> m1,
116 | id("b") -> m2,
117 | id("c") -> m3,
118 | id("d") -> m4
119 | )
120 |
121 | execute()(state, command).map { newState =>
122 | forAll(List(c1, c2, c3, c4)) {
123 | _.isShutdownNow() shouldBe true
124 | }
125 | newState shouldBe empty
126 | }
127 | }
128 | }
129 |
130 | "receiving a verify command it" should {
131 | "restart all stopped mirrors" in {
132 | val command = VerifyState
133 |
134 | val c1, c2, c3, c4, c5, c6 = new MockControl
135 |
136 | val state = Map(
137 | id("a") -> RunningMirror(c1, runnableSettings("a")),
138 | id("b") -> RunningMirror(c2, runnableSettings("b")),
139 | id("c") -> RunningMirror(c3, runnableSettings("c")),
140 | id("d") -> RunningMirror(c4, runnableSettings("d")),
141 | id("e") -> RunningMirror(c5, runnableSettings("e")),
142 | id("f") -> RunningMirror(c6, runnableSettings("f"))
143 | )
144 |
145 | c2.shutdownNow()
146 | c4.shutdownNow()
147 | c6.shutdownNow()
148 |
149 | execute()(state, command).map { newState =>
150 | newState.keySet shouldBe Set(id("a"), id("b"), id("c"), id("d"), id("e"), id("f"))
151 |
152 | val controls = Set(id("b"), id("d"), id("f")).map(newState).map(_.control)
153 |
154 | forAll(controls) { c =>
155 | MirrorManager.isShutDownNow(c) shouldBe false
156 | }
157 |
158 | forAll(List(c1, c3, c5)) { c =>
159 | c.isShutdownNow() shouldBe false
160 | }
161 | }
162 | }
163 | }
164 | }
165 |
166 | "Future map updating" when {
167 | val map = Map(
168 | 1 -> "a",
169 | 2 -> "b",
170 | 3 -> "c")
171 |
172 | val mapOver = futureUpdateMap[Int, String](map) _
173 |
174 | val clear = Future.successful(None)
175 |
176 | "the value is present it" should {
177 | "apply the 'whenPresent' function and update the key if the result is present" in {
178 | mapOver(2,
179 | s => Future.successful(Some(s ++ "-after")),
180 | () => clear
181 | ).map { result =>
182 | result shouldBe Map(
183 | 1 -> "a",
184 | 2 -> "b-after",
185 | 3 -> "c"
186 | )
187 | }
188 | }
189 |
190 | "apply then 'whenPresent' function and remove the key if the result is missing" in {
191 | mapOver(2,
192 | _ => clear,
193 | () => clear
194 | ).map { result =>
195 | result shouldBe Map(
196 | 1 -> "a",
197 | 3 -> "c"
198 | )
199 | }
200 | }
201 | }
202 |
203 | "the value is missing it" should {
204 | "apply the 'whenMissing' function and set the key if the result is present" in {
205 | mapOver(4,
206 | _ => clear,
207 | () => Future.successful(Some("d"))
208 | ).map { result =>
209 | result shouldBe Map(
210 | 1 -> "a",
211 | 2 -> "b",
212 | 3 -> "c",
213 | 4 -> "d"
214 | )
215 | }
216 | }
217 |
218 | "apply the 'whenMissing' function and return the original map" in {
219 | mapOver(4,
220 | _ => clear,
221 | () => clear
222 | ).map { result =>
223 | result shouldBe Map(
224 | 1 -> "a",
225 | 2 -> "b",
226 | 3 -> "c"
227 | )
228 | }
229 | }
230 | }
231 | }
232 | }
233 |
--------------------------------------------------------------------------------
/core/src/test/scala/com/supersonic/kafka_mirror/SettingsProcessingFlowHelpersTest.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.kafka_mirror
2 |
3 | import cats.scalatest.{ValidatedMatchers, ValidatedValues}
4 | import com.supersonic.kafka_mirror.MirrorCommand._
5 | import com.supersonic.kafka_mirror.SettingsProcessingFlow._
6 | import com.supersonic.kafka_mirror.TestUtil._
7 | import com.typesafe.config.ConfigFactory
8 | import net.ceedubs.ficus.Ficus._
9 | import net.ceedubs.ficus.readers.ArbitraryTypeReader._
10 | import org.scalatest.{Matchers, WordSpec}
11 | import com.softwaremill.quicklens._
12 |
13 | class SettingsProcessingFlowHelpersTest extends WordSpec with
14 | Matchers with
15 | ValidatedMatchers with
16 | ValidatedValues {
17 |
18 | val fallback = ConfigFactory.parseString("""
19 | akka.kafka.consumer = { a = 1 }
20 | akka.kafka.producer = { b = 2 }
21 | """)
22 |
23 | val (consumerString, producerString) = ("c = 3", "d = 4")
24 | val consumer = ConfigFactory.parseString(consumerString)
25 | val producer = ConfigFactory.parseString(producerString)
26 |
27 | val mirrorString = """
28 | mirror = {
29 | whitelist = ["topic1", "topic2"]
30 | commitBatchSize = 1
31 | commitParallelism = 4
32 | enabled = true
33 | }
34 | """
35 |
36 | val mirror = ConfigFactory.parseString(mirrorString).as[MirrorSettings]("mirror")
37 |
38 | "Computing new mirror commands" should {
39 | "produce no commands when the state is unchanged" in {
40 | val state = Map(id("a") -> settings("a"))
41 | computeCommands(state, state) shouldBe empty
42 | }
43 |
44 | "if the state of a mirror changes, stop and then start it with the new settings" in {
45 | val state1 = Map(id("a") -> settings("a"))
46 | val state2 = Map(id("a") -> settings("b"))
47 |
48 | computeCommands(state1, state2) shouldBe List(Start(id("a"), settings("b")))
49 | }
50 |
51 | "if a mirror is not present anymore, stop it" in {
52 | val state = Map(id("a") -> settings("a"))
53 |
54 | computeCommands(state, Map.empty) shouldBe List(Stop(id("a")))
55 | }
56 |
57 | "if a new mirror appears, start it" in {
58 | val state = Map(id("a") -> settings("a"))
59 |
60 | computeCommands(Map.empty, state) shouldBe List(Start(id("a"), settings("a")))
61 | }
62 |
63 | "do nothing when no mirrors are present" in {
64 | computeCommands(Map.empty, Map.empty) shouldBe empty
65 | }
66 |
67 | "if a new mirror appears in a disabled state, ignore it" in {
68 | val sets = settings("a").modify(_.mirror.enabled).setTo(false)
69 | val state = Map(id("a") -> sets)
70 |
71 | computeCommands(Map.empty, state) shouldBe empty
72 | }
73 |
74 | "if a mirror's state changes to the disabled state, stop it" in {
75 | val sets1 = settings("a").modify(_.mirror.enabled).setTo(true)
76 | val sets2 = settings("a").modify(_.mirror.enabled).setTo(false)
77 | val state1 = Map(id("a") -> sets1)
78 | val state2 = Map(id("a") -> sets2)
79 |
80 | computeCommands(state1, state2) shouldBe List(Stop(id("a")))
81 | }
82 |
83 | "if a mirror's state changes from disabled to enabled, start it" in {
84 | val sets1 = settings("a").modify(_.mirror.enabled).setTo(false)
85 | val sets2 = settings("a").modify(_.mirror.enabled).setTo(true)
86 | val state1 = Map(id("a") -> sets1)
87 | val state2 = Map(id("a") -> sets2)
88 |
89 | computeCommands(state1, state2) shouldBe List(Start(id("a"), sets2))
90 | }
91 |
92 | "all together now" in {
93 | val state1 = Map(
94 | id("a") -> settings("a"),
95 | id("b") -> settings("b"),
96 | id("c") -> settings("c")
97 | )
98 |
99 | val state2 = Map(
100 | id("a") -> settings("a"),
101 | id("b") -> settings("b1"),
102 | id("d") -> settings("d")
103 | )
104 |
105 | val commands = computeCommands(state1, state2)
106 |
107 | commands should have size 3
108 |
109 | // we can't rely on the fact that the commands for different IDs appear in any particular order
110 | commands should contain(Start(id("b"), settings("b1")))
111 | commands should contain(Stop(id("c")))
112 | commands should contain(Start(id("d"), settings("d")))
113 | }
114 | }
115 |
116 | "Parsing settings from configs" should {
117 | val settings = fromConfig(fallback)(consumer, producer, mirror).value
118 |
119 | "use the correct fallback for consumers" in {
120 | val consumerSettings = settings.kafka.consumer
121 |
122 | consumerSettings.getInt("a") shouldBe 1
123 | consumerSettings.getInt("c") shouldBe 3
124 | }
125 |
126 | "use the correct fallback for producers" in {
127 | val producerSettings = settings.kafka.producer
128 |
129 | producerSettings.getInt("b") shouldBe 2
130 | producerSettings.getInt("d") shouldBe 4
131 | }
132 |
133 | "pass on the mirror settings" in {
134 | settings.mirror shouldBe mirror
135 | }
136 |
137 | "fail when the fallback does not contain the relevant akka/kafka keys" in {
138 | val fallback1 = ConfigFactory.parseString("akka.kafka.consumer = { a = 1 }")
139 | val settings1 = fromConfig(fallback1)(consumer, producer, mirror)
140 |
141 | settings1 shouldBe invalid
142 |
143 | val fallback2 = ConfigFactory.parseString("akka.kafka.producer = { a = 1 }")
144 | val settings2 = fromConfig(fallback2)(consumer, producer, mirror)
145 |
146 | settings2 shouldBe invalid
147 | }
148 | }
149 |
150 | "Parsing settings from raw data" when {
151 | val parse = fromRawMap(fallback) _
152 |
153 | def failParse(data: Map[MirrorID, Option[String]]) = {
154 | val parsed = parse(data)
155 | parsed should have size 1
156 | parsed should contain key MirrorID("bar")
157 | parsed(MirrorID("bar")) shouldBe invalid
158 | }
159 |
160 | "receiving a single key in HOCON form" should {
161 | val dataRoot = Map(
162 | MirrorID("bar") -> Some(s"""
163 | consumer = {$consumerString}
164 | producer = {$producerString}
165 | $mirrorString
166 | """)
167 | )
168 |
169 | def remove(str: String) =
170 | dataRoot.mapValues(_.map(_.replace(str, "")))
171 |
172 | "parse the settings" in {
173 | parse(dataRoot) shouldBe Map(
174 | MirrorID("bar") -> fromConfig(fallback)(consumer, producer, mirror))
175 | }
176 |
177 | "fail" when {
178 | "the consumer key is missing" in {
179 | failParse(remove(s"consumer = {$consumerString}"))
180 | }
181 |
182 | "the producer key is missing" in {
183 | failParse(remove(s"producer = {$producerString}"))
184 | }
185 |
186 | "the mirror key is missing" in {
187 | failParse(remove(mirrorString))
188 | }
189 |
190 | "configuration is malformed" in {
191 | failParse(dataRoot.updated(MirrorID("bar"), Some("bla = ")))
192 | }
193 |
194 | "one of the sub-keys is missing" in {
195 | failParse(dataRoot.updated(MirrorID("bar"), None))
196 | }
197 | }
198 | }
199 |
200 | "support parsing multiple mirrors at a time" in {
201 | val mirrorConfig = Some(s"""
202 | consumer = {$consumerString}
203 | producer = {$producerString}
204 | $mirrorString
205 | """)
206 |
207 | val data = Map(
208 | MirrorID("a") -> mirrorConfig,
209 | MirrorID("b") -> mirrorConfig)
210 |
211 | val settings = fromConfig(fallback)(consumer, producer, mirror)
212 | parse(data) shouldBe Map(
213 | MirrorID("a") -> settings,
214 | MirrorID("b") -> settings
215 | )
216 | }
217 | }
218 | }
219 |
--------------------------------------------------------------------------------
/core/src/test/scala/com/supersonic/kafka_mirror/KafkaMirrorTest.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.kafka_mirror
2 |
3 | import akka.kafka.ConsumerMessage._
4 | import org.apache.kafka.clients.consumer.ConsumerRecord
5 | import org.apache.kafka.common.record.TimestampType
6 | import org.scalatest.{Inspectors, Matchers, OptionValues, WordSpecLike}
7 |
8 | class KafkaMirrorTest extends WordSpecLike
9 | with Matchers
10 | with Inspectors
11 | with OptionValues {
12 |
13 | val bucketing = BucketSettings(
14 | mirrorBuckets = 3,
15 | totalBuckets = 5
16 | )
17 |
18 | val hashKey = (_: String).toInt
19 |
20 | val partitionNum = 5
21 |
22 | val getNumberOfPartitions = (_: String) => partitionNum
23 |
24 | val generatePartition = (n: Int) => n
25 |
26 | def mirrorSettings(topicsToRename: Map[String, String] = Map.empty,
27 | bucketSettings: Option[BucketSettings] = None)
28 | (whitelist: String*) =
29 | MirrorSettings(
30 | whitelist = whitelist.toSet,
31 | commitBatchSize = 20,
32 | commitParallelism = 3,
33 | bucketing = bucketSettings,
34 | partitionFromKeys = false,
35 | topicsToRename = topicsToRename)
36 |
37 | "The message producing function" should {
38 | val topic = "some-topic"
39 | val renamedTopic = "renamed-topic"
40 |
41 | val offset = new CommittableOffset {
42 | val partitionOffset = PartitionOffset(GroupTopicPartition("some-group", topic, 1), 13L)
43 |
44 | def commitJavadsl() = ???
45 |
46 | def commitScaladsl() = ???
47 |
48 | def batchSize: Long = ???
49 | }
50 |
51 | def makeConsumerMessage(key: String) = {
52 | val record = new ConsumerRecord(topic, 1, 13L, 15L, TimestampType.CREATE_TIME, 17L, 99, 31, key, "the-value")
53 |
54 | CommittableMessage(record, offset)
55 | }
56 |
57 | def makeRenamedConsumerMessage(key: String) = {
58 | val record = new ConsumerRecord(renamedTopic, 1, 13L, 15L, TimestampType.CREATE_TIME, 17L, 99, 31, key, "the-value")
59 |
60 | CommittableMessage(record, offset)
61 | }
62 |
63 | def makeMessage(mirror: MirrorSettings, message: CommittableMessage[String, String]) =
64 | KafkaMirror.makeMessage(mirror, hashKey, getNumberOfPartitions, generatePartition)(message)
65 |
66 | "create a message for the right topic in the producer" in {
67 | val settings = mirrorSettings()(topic)
68 | val consumerMessage = makeConsumerMessage("the-key")
69 |
70 | val maybeMessage = makeMessage(settings, consumerMessage)
71 |
72 | val message = maybeMessage.value
73 |
74 | message.passThrough shouldBe consumerMessage.committableOffset
75 | message.record.topic shouldBe consumerMessage.record.topic
76 | message.record.partition shouldBe partitionNum
77 | message.record.timestamp shouldBe consumerMessage.record.timestamp
78 | message.record.key shouldBe consumerMessage.record.key
79 | message.record.value shouldBe consumerMessage.record.value
80 | }
81 |
82 | "create a message for the right topic in the producer with topic rename" in {
83 | val settings = mirrorSettings(topicsToRename = Map(topic -> renamedTopic))(topic)
84 | val consumerMessage = makeConsumerMessage("the-key")
85 | val renamedConsumerMessage = makeRenamedConsumerMessage("the-key")
86 |
87 | val maybeMessage = makeMessage(settings, consumerMessage)
88 |
89 | val message = maybeMessage.value
90 |
91 | message.passThrough shouldBe renamedConsumerMessage.committableOffset
92 | message.record.topic shouldBe renamedConsumerMessage.record.topic
93 | message.record.partition shouldBe partitionNum
94 | message.record.timestamp shouldBe renamedConsumerMessage.record.timestamp
95 | message.record.key shouldBe renamedConsumerMessage.record.key
96 | message.record.value shouldBe renamedConsumerMessage.record.value
97 | }
98 |
99 | "use deterministic bucketing if bucketing settings are present" in {
100 | val settingsWithBucketing = mirrorSettings(bucketSettings = Some(bucketing))(List("some-topic"): _*) // avoiding varargs due to a bug in the compiler
101 | val settingsNoBucketing = mirrorSettings()("some-topic")
102 |
103 | val makeMessageWithBucketing =
104 | makeMessage(settingsWithBucketing, _: CommittableMessage[String, String])
105 | val makeMessageWithNoBucketing =
106 | makeMessage(settingsNoBucketing, _: CommittableMessage[String, String])
107 |
108 | def verifyNoMessage(message: CommittableMessage[String, String]) =
109 | makeMessageWithBucketing(message) shouldBe empty
110 |
111 | def verifyMessage(message: CommittableMessage[String, String]) =
112 | makeMessageWithBucketing(message) shouldBe makeMessageWithNoBucketing(message)
113 |
114 | def messages(nums: Int*) = nums.map(_.toString).map(makeConsumerMessage)
115 |
116 | forEvery(messages(1, 2, 5, 6, 7, 10, 11, 12, 15))(verifyMessage)
117 |
118 | forEvery(messages(3, 4, 8, 9, 13, 14))(verifyNoMessage)
119 | }
120 |
121 | "not use bucketing if the message key is null" in {
122 | def hashKey(str: String) = bucketing.totalBuckets - 1 // we want to always be not-mirrored
123 |
124 | val settingsWithBucketing = mirrorSettings(bucketSettings = Some(bucketing))(List("some-topic"): _*) // avoiding varargs due to a bug in the compiler
125 | val settingsNoBucketing = mirrorSettings()("some-topic")
126 |
127 | val makeMessageWithBucketing =
128 | KafkaMirror.makeMessage[String, String](settingsWithBucketing, hashKey, getNumberOfPartitions, generatePartition) _
129 | val makeMessageWithNoBucketing =
130 | KafkaMirror.makeMessage[String, String](settingsNoBucketing, hashKey, getNumberOfPartitions, generatePartition) _
131 |
132 | val message = makeConsumerMessage(null)
133 | makeMessageWithBucketing(message) shouldBe makeMessageWithNoBucketing(message)
134 | }
135 |
136 | "handle a missing timestamp in the incoming message" in {
137 | val settings = mirrorSettings()("some-topic")
138 |
139 | val timestamp = -1L
140 | val record = new ConsumerRecord(topic, 1, 13L, timestamp, TimestampType.NO_TIMESTAMP_TYPE, 17L, 99, 31, "some-key", "the-value")
141 |
142 | val consumerMessage = CommittableMessage(record, offset)
143 |
144 | val maybeMessage = KafkaMirror.makeMessage(settings, hashKey, getNumberOfPartitions, generatePartition)(consumerMessage)
145 |
146 | val message = maybeMessage.value
147 |
148 | message.record.timestamp shouldBe null
149 | }
150 |
151 | "generate the partition number from the key when the flag is on and the key is present" in {
152 | val settings = mirrorSettings()(topic)
153 | .copy(partitionFromKeys = true)
154 |
155 | val consumerMessage = makeConsumerMessage("3")
156 |
157 | val maybeMessage = makeMessage(settings, consumerMessage)
158 |
159 | val partition = maybeMessage.value.record.partition
160 |
161 | partition should not be partitionNum
162 | partition shouldBe 3
163 | }
164 |
165 | "generate the partition number with the generating function" when {
166 | "the flag is off" in {
167 | val settings = mirrorSettings()(topic)
168 | .copy(partitionFromKeys = false)
169 |
170 | val consumerMessage = makeConsumerMessage("3")
171 |
172 | val maybeMessage = makeMessage(settings, consumerMessage)
173 |
174 | val partition = maybeMessage.value.record.partition
175 |
176 | partition shouldBe partitionNum
177 | }
178 |
179 | "the flag is on but the key is 'null'" in {
180 | val settings = mirrorSettings()(topic)
181 | .copy(partitionFromKeys = true)
182 |
183 | val consumerMessage = makeConsumerMessage(key = null)
184 |
185 | val maybeMessage = makeMessage(settings, consumerMessage)
186 |
187 | val partition = maybeMessage.value.record.partition
188 |
189 | partition shouldBe partitionNum
190 | }
191 | }
192 | }
193 | }
194 |
--------------------------------------------------------------------------------
/core/src/main/scala/com/supersonic/kafka_mirror/MirrorManager.scala:
--------------------------------------------------------------------------------
1 | package com.supersonic.kafka_mirror
2 |
3 | import akka.event.LoggingAdapter
4 | import akka.kafka.scaladsl.Consumer.Control
5 | import akka.stream.scaladsl.Flow
6 | import akka.stream.{KillSwitch, KillSwitches}
7 | import akka.{Done, NotUsed}
8 | import com.supersonic.kafka_mirror.MirrorCommand._
9 | import com.supersonic.kafka_mirror.MirrorManager._
10 | import scala.concurrent.{ExecutionContext, Future}
11 | import scala.util.control.NonFatal
12 |
13 | /** Provides a flow from [[MirrorCommand]]s to a map of [[RunningMirror]]s.
14 | *
15 | * For every incoming [[MirrorCommand]] the state is updated and the necessary side effects are
16 | * performed.
17 | */
18 | class MirrorManager(mirrorMaker: MirrorMaker)
19 | (implicit executionContext: ExecutionContext,
20 | logger: LoggingAdapter) {
21 | private val killSwitch = KillSwitches.shared("kill-switch")
22 |
23 | val flow: Flow[MirrorCommand, Map[MirrorID, RunningMirror], NotUsed] = {
24 | val init = Map.empty[MirrorID, RunningMirror]
25 |
26 | Flow[MirrorCommand].scanAsync(init)(executeMirrors(mirrorMaker, killSwitch))
27 | .via(killSwitch.flow)
28 | }
29 | }
30 |
31 | private[kafka_mirror] object MirrorManager {
32 | /** The next step that should executed on the Kafka mirrors state. */
33 | private sealed trait ExecutionStep extends Product with Serializable
34 |
35 | private object ExecutionStep {
36 | type NewState = Future[Option[RunningMirror]]
37 |
38 | /** Concrete functions that should be executed on the value associated with the given [[MirrorID]]. */
39 | case class ExecutionCommand(id: MirrorID,
40 | whenPresent: RunningMirror => NewState,
41 | whenMissing: () => NewState) extends ExecutionStep
42 |
43 | /** Indicates that all mirrors currently running should be stopped. */
44 | case class StopAll(stop: MirrorID => RunningMirror => Future[None.type]) extends ExecutionStep
45 |
46 | /** That all stopped mirrors should be restarted. */
47 | case class RestartStopped(start: (MirrorID, ExternalKafkaMirrorSettings) => NewState) extends ExecutionStep
48 | }
49 |
50 | import ExecutionStep._
51 |
52 | /** Given the state of the Kafka mirrors, executes the given command, this includes side-effects.
53 | * Produces a new state of mirrors that corresponds to the state of affairs after execution.
54 | */
55 | def executeMirrors(makeMirror: MirrorMaker, killSwitch: KillSwitch)
56 | (mirrors: Map[MirrorID, RunningMirror],
57 | command: MirrorCommand)
58 | (implicit executionContext: ExecutionContext,
59 | logger: LoggingAdapter): Future[Map[MirrorID, RunningMirror]] = {
60 |
61 | val nextExecutionStep = computeNextExecutionStep(makeMirror)(command)
62 |
63 | nextExecutionStep match {
64 | case ExecutionCommand(id, whenPresent, whenMissing) =>
65 | futureUpdateMap(mirrors)(id, whenPresent, whenMissing)
66 |
67 | case StopAll(stop) => stopAll(mirrors, stop, killSwitch)
68 |
69 | case RestartStopped(start) => restartStopped(mirrors, start)
70 | }
71 | }
72 |
73 | /** Computing the next execution step given the [[MirrorCommand]].
74 | * Not actually performing anything at this stage.
75 | */
76 | private def computeNextExecutionStep(mirrorMaker: MirrorMaker)
77 | (command: MirrorCommand)
78 | (implicit executionContext: ExecutionContext,
79 | logger: LoggingAdapter): ExecutionStep = {
80 |
81 | val mirrorLifeCycleHandler = new MirrorLifeCycleHandler(mirrorMaker)
82 | import mirrorLifeCycleHandler._
83 |
84 | command match {
85 | case Start(id, mirrorSettings) =>
86 | ExecutionCommand(
87 | id,
88 | whenPresent = restart(id, mirrorSettings),
89 | whenMissing = start(id, mirrorSettings) _
90 | )
91 |
92 | case Stop(id) =>
93 | ExecutionCommand(
94 | id,
95 | whenPresent = stop(id),
96 | whenMissing = () => clear
97 | )
98 |
99 | case Shutdown => StopAll(stop)
100 |
101 | case VerifyState => RestartStopped(start(_, _)())
102 | }
103 | }
104 |
105 | /** Wraps around functions needed to start/stop mirrors. */
106 | private class MirrorLifeCycleHandler(makeMirror: MirrorMaker)
107 | (implicit executionContext: ExecutionContext,
108 | logger: LoggingAdapter) {
109 | val clear = Future.successful(None)
110 |
111 | def stop(id: MirrorID)
112 | (mirror: RunningMirror): Future[None.type] = {
113 | def errorShuttingDown(e: Throwable) = {
114 | logger.error(e, s"Error while shutting down Kafka mirror: [${id.value}]")
115 | Done
116 | }
117 |
118 | logger.info(s"Trying to shutdown Kafka mirror: [${id.value}]")
119 | mirror.control.shutdown()
120 | .recover {
121 | case NonFatal(e) => errorShuttingDown(e)
122 | }
123 | .map { _ =>
124 | logger.info(s"Shutdown Kafka mirror: [${id.value}]")
125 | None
126 | }
127 | }
128 |
129 | def start(id: MirrorID, mirrorSettings: ExternalKafkaMirrorSettings)
130 | (): Future[Option[RunningMirror]] = {
131 | def errorStarting(e: Throwable) = {
132 | logger.error(e, s"Failed to start Kafka mirror: [${id.value}]")
133 | clear
134 | }
135 |
136 | mirrorSettings.toKafkaMirrorSettings.map { kafkaSettings =>
137 | val mirror = KafkaMirror(makeMirror)(id, kafkaSettings)
138 | mirror.start()
139 | }.fold(
140 | errorStarting,
141 | control => Future.successful(Some(RunningMirror(control, mirrorSettings)))
142 | )
143 | }
144 |
145 | def restart(id: MirrorID, mirrorSettings: ExternalKafkaMirrorSettings)
146 | (mirror: RunningMirror): Future[Option[RunningMirror]] =
147 | stop(id)(mirror).flatMap { _ =>
148 | start(id, mirrorSettings)
149 | }
150 | }
151 |
152 | /** Restarts all mirrors that are currently stopped. */
153 | private def restartStopped(mirrors: Map[MirrorID, RunningMirror],
154 | start: (MirrorID, ExternalKafkaMirrorSettings) => NewState)
155 | (implicit logger: LoggingAdapter,
156 | executionContext: ExecutionContext): Future[Map[MirrorID, RunningMirror]] = {
157 | Future.sequence {
158 | mirrors.map { case (id, mirror) =>
159 | if (isShutDownNow(mirror.control)) {
160 | logger.error(s"Kafka mirror with ID: [${id.value}], terminated unexpectedly, attempting to restart")
161 |
162 | start(id, mirror.mirrorSettings).map(_.map((id, _)))
163 | }
164 | else Future.successful(Some((id, mirror)))
165 | }
166 | }.map(_.flatten.toMap)
167 | }
168 |
169 | /** Stops all mirrors and terminates the stream. */
170 | private def stopAll(mirrors: Map[MirrorID, RunningMirror],
171 | stop: MirrorID => RunningMirror => Future[None.type],
172 | killSwitch: KillSwitch)
173 | (implicit executionContext: ExecutionContext): Future[Map[MirrorID, RunningMirror]] = {
174 | Future.sequence {
175 | mirrors.map { case (id, mirror) =>
176 | stop(id)(mirror)
177 | }
178 | }.map { _ =>
179 | killSwitch.shutdown()
180 | Map.empty
181 | }
182 | }
183 |
184 | def isShutDownNow(control: Control) = control.isShutdown
185 | .value // we care only whether the system was shutdown now, hence no waiting on the future value
186 | .map(_.isSuccess)
187 | .exists(identity)
188 |
189 | /** Tries to fetch a value from the map and apply a function to it. If the value is missing
190 | * then applying the fallback function.
191 | * Given the resulting optional value, removing the key from the map if it's [[None]], if not
192 | * replacing the key's value with the new value.
193 | */
194 | def futureUpdateMap[A, B](valuesMap: Map[A, B])
195 | (key: A,
196 | whenPresent: B => Future[Option[B]],
197 | whenMissing: () => Future[Option[B]])
198 | (implicit executionContext: ExecutionContext): Future[Map[A, B]] =
199 | valuesMap.get(key)
200 | .map(whenPresent)
201 | .getOrElse(whenMissing())
202 | .map { maybeNewValue =>
203 | maybeNewValue.map { newValue =>
204 | valuesMap.updated(key, newValue)
205 | }.getOrElse(valuesMap - key)
206 | }
207 | }
208 |
--------------------------------------------------------------------------------
/README.md:
--------------------------------------------------------------------------------
1 | # Kafka Snow White
2 |
3 | [](https://github.com/SupersonicAds/kafka-snow-white/actions?query=branch%3Amaster)
4 | [](https://maven-badges.herokuapp.com/maven-central/com.supersonic/kafka-snow-white_2.12)
5 |
6 | > Mirror, mirror on the wall, who is the fairest of them all?
7 |
8 | A Kafka mirroring service based on Akka Streams Kafka. The service can be used to move messages between topics on different Kafka clusters.
9 |
10 | ## Getting Kafka Snow White
11 |
12 | Kafka Snow White is available as executable JAR files in multiple variations (see below), these can be downloaded from Maven Central:
13 | - [kafka-snow-white-file-watcher-app](https://mvnrepository.com/artifact/com.supersonic/kafka-snow-white-file-watcher-app)
14 | - [kafka-snow-white-consul-app](https://mvnrepository.com/artifact/com.supersonic/kafka-snow-white-consul-app)
15 |
16 | Pick the `xxx-assembly.jar` files where `xxx` stands for the app name and version.
17 |
18 | The code is also available as a library and can be used via SBT with:
19 | ```scala
20 | // If you want to run the service from code
21 | libraryDependencies += "com.supersonic" %% "kafka-snow-white-consul-app" % "1.5.2"
22 | libraryDependencies += "com.supersonic" %% "kafka-snow-white-file-watcher" % "1.5.2"
23 |
24 | // If you want only the basic mirroring functionality with no backing mechanism
25 | libraryDependencies += "com.supersonic" %% "kafka-snow-white-core" % "1.5.2"
26 |
27 | // If you want to create a new backend for mirroring
28 | libraryDependencies += "com.supersonic" %% "kafka-snow-white-app-common" % "1.5.2"
29 | ```
30 |
31 | ## Motivation
32 | Given two different Kafka clusters (`cluster1` and `cluster2`) we want to move messages from the topic `some-topic` on `cluster1` to `some-topic` on `cluster2`.
33 | The Kafka Snow White service lets one define a mirror, that does exactly this (see setup details below). Given the appropriate mirror, if we start the Kafka Snow White service with it, it will connect to `cluster1` consume all messages from `some-topic` and then produce the messages to `some-topic` on `cluster2`.
34 |
35 | ## Setup
36 |
37 | The Kafka Snow White service is available in two flavors which differ in the way that they are configured.
38 | The first takes configuration from files on the file-system and each mirror corresponds to a single file. The second, takes configuration from Consul keys in Consul's KV-store, and a mirror corresponds to a single key.
39 | For setup instructions for each flavor, see the README files under [file-watcher-app](file-watcher-app/README.md) and [consul-app](consul-app/README.md).
40 |
41 | ### Mirror Configuration Format
42 |
43 | The configuration schema (using the [HOCON format](https://github.com/lightbend/config/blob/master/HOCON.md)) for both flavors of the Kafka Snow White service is the same. The example mirror that was described in the motivation section above can be set up as follows:
44 | ```
45 | consumer {
46 | kafka-clients {
47 | bootstrap.servers = "cluster1:9092"
48 | group.id = "some-group"
49 | auto.offset.reset = "earliest"
50 | }
51 | }
52 |
53 | producer {
54 | kafka-clients = {
55 | bootstrap.servers = "cluster2:9092"
56 | }
57 | }
58 |
59 | mirror {
60 | whitelist = ["some-topic"]
61 | commitBatchSize = 1000
62 | commitParallelism = 4
63 | }
64 | ```
65 |
66 | A mirror specification has three sections:
67 | - `consumer` - setting up the consumer that will be used by the mirror (this corresponds to the source cluster)
68 | - `producer` - setting up the producer that will be used by the mirror (this corresponds to the target cluster)
69 | - `mirror` - settings for the mirror itself
70 |
71 | Since the mirror is backed by the Akka Streams Kafka library, the `consumer` and `producer` settings are passed directly to it. The full specification of the available settings can be found in the Akka Streams Kafka documentation for the [consumer](https://doc.akka.io/docs/akka-stream-kafka/current/consumer.html#settings) and the [producer](https://doc.akka.io/docs/akka-stream-kafka/current/producer.html#settings).
72 |
73 | The mirror settings are defined as follows:
74 | ```
75 | mirror {
76 | # The list of topics the mirror listens to (mandatory).
77 | # Unless overridden in the `topicsToRename` field each topic in this list
78 | # will be mirrored in the target cluster.
79 | whitelist = ['some-topic-1', 'some-topic-2']
80 |
81 | # How many messages to batch before committing their offsets to Kafka (optional, default 1000).
82 | commitBatchSize = 1000
83 |
84 | # The parallelism level used to commit the offsets (optional, default 4).
85 | commitParallelism = 4
86 |
87 | # Settings to enable bucketing of mirrored values (optional).
88 | # The (mirrorBuckets / totalBuckets) ratio is the percentage of traffic to be mirrored.
89 | bucketing {
90 | # The number of buckets that should be mirrored (mandatory, no default).
91 | mirrorBuckets = 4
92 |
93 | # The total number of buckets, used to calculate the percentage of traffic
94 | # to mirror (mandatory, no default).
95 | totalBuckets = 16
96 | }
97 |
98 | # Whether the mirror should be enabled or not (optional, default 'true').
99 | enabled = true
100 |
101 | # Whether the partition number for the mirrored messages should be generated by hashing
102 | # the message key, excluding 'null's (optional, default 'false'). When set to false, the partition number
103 | # will be randomly generated.
104 | partitionFromKeys = false
105 |
106 | # Map of source to target topics to rename when mirroring messages to the
107 | # target cluster (optional, default empty).
108 | topicsToRename {
109 | some-topic-2 = "renamed-topic"
110 | }
111 | }
112 | ```
113 |
114 | ### Topic Renaming
115 | By default the topic `whitelist` defines topics that should be mirrored from the source cluster in the target cluster. This means that each topic in the list will be recreated with the same name in the target cluster. It is possible to override this behavior by providing the `topicsToRename` map. In the example above, the `whitelist` contains two topics (`some-topic-1` and `some-topic-2`) and the `topicsToRename` map contains a single entry that maps `some-topic-2` to `renamed-topic`. When the mirror will be run, the contents of `some-topic-2` in the source cluster will be mirrored in the target cluster in a new topic called `renamed-topic`. While the `some-topic-1` topic will be mirrored in the target cluster under the same name (since it does not have an entry in the map).
116 |
117 | ### Bucketing
118 |
119 | It is possible to (probabilistically) mirror only some percentage of the messages in a topic, to do this we need to specify the `bucketing` settings. As in the example above, we set `mirrorBuckets` to `4` and `totalBuckets` to `16`, this means that only `4 / 16 = 25%` of the messages will be mirrored by the mirror.
120 |
121 | The selection of which messages to mirror is based on the key of the incoming message, meaning that if a key was once mirrored it will always be mirrored when the same key is encountered again. This can be useful when mirroring traffic between production and staging/testing environments (assuming that the key is not `null` and has some logical info in it).
122 |
123 | Note: this implies that if the keys are all the same then all messages will be either mirrored or not depending on the key's value.
124 |
125 | Bucketing is not supported for `null` keys, all such messages will be mirrored independent of the bucketing settings.
126 |
127 | ## The Healthcheck Server
128 |
129 | Upon startup, the Kafka Snow White service starts a healthcheck server (by default on port `8080`). The healthcheck data can be accessed via:
130 | ```
131 | localhost:8080/healthcheck
132 | ```
133 |
134 | The route contains information about the running service including data about the currently running mirrors. This can be useful to troubleshoot the settings of a mirror when trying to set it up for the first time. Note, mirrors that failed to start or are disabled are currently not displayed.
135 |
136 | The default port can be overridden by setting the `KAFKA_SNOW_WHITE_HEALTHCHECK_PORT` environment variable. For example:
137 | ```
138 | export KAFKA_SNOW_WHITE_HEALTHCHECK_PORT=8081
139 |
140 | # ... start the Kafka Snow White service
141 | ```
142 | Or, in a configuration file:
143 | ```
144 | kafka-mirror-settings {
145 | port = 8081
146 | }
147 | ```
148 |
149 | ## Compared to Other Solutions
150 |
151 | There are other mirroring solutions out there, notably Apache Kafka's own [MirrorMaker](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=27846330) and Uber's [uReplicator](https://github.com/uber/uReplicator).
152 |
153 | Compared to MirrorMaker, Kafka Snow White should be easier to manage:
154 | - The configuration can be reloaded dynamically via files or Consul
155 | - Each mirror is defined independently with no need to restart or affect other mirrors when a given mirror is changed
156 |
157 | uReplicator is a more featureful solution but also requires a more complicated setup (e.g., a [Helix controller](http://helix.apache.org/)). Compared to it Kafka Snow White can be deployed as a single self-contained JAR file.
158 |
--------------------------------------------------------------------------------