├── .gitignore
├── .travis.yml
├── CHANGES.md
├── LICENSE
├── README.md
├── build.sbt
├── project
├── build.properties
└── plugins.sbt
├── runtime
├── start-kafka.sh
└── start-zk.sh
├── src
├── main
│ └── scala
│ │ └── spinoco
│ │ └── fs2
│ │ └── kafka
│ │ ├── KafkaClient.scala
│ │ ├── failure
│ │ └── failures.scala
│ │ ├── kafka.scala
│ │ └── network
│ │ ├── BrokerAddress.scala
│ │ └── BrokerConnection.scala
└── test
│ └── scala
│ └── spinoco
│ └── fs2
│ └── kafka
│ ├── DockerSupport.scala
│ ├── Fs2KafkaClientSpec.scala
│ ├── Fs2KafkaRuntimeSpec.scala
│ ├── KafkaClientLastOffsetSpec.scala
│ ├── KafkaClientPublishSpec.scala
│ ├── KafkaClientSubscribeSpec.scala
│ ├── KafkaClusterPublish.scala
│ ├── KafkaClusterSubscribeSpec.scala
│ └── network
│ ├── BrokerConnection08SPec.scala
│ ├── BrokerConnection09Spec.scala
│ ├── BrokerConnection10Spec.scala
│ ├── BrokerConnectionApp.scala
│ ├── BrokerConnectionKafkaSpecBase.scala
│ └── BrokerConnectionSpec.scala
└── version.sbt
/.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 | .idea
19 |
--------------------------------------------------------------------------------
/.travis.yml:
--------------------------------------------------------------------------------
1 | language : scala
2 | sudo: required
3 |
4 | services:
5 | - docker
6 |
7 | addons:
8 | hosts:
9 | - broker1
10 | - broker2
11 | - broker3
12 |
13 | scala:
14 | - 2.11.12
15 | - 2.12.6
16 |
17 | cache:
18 | directories:
19 | - $HOME/.ivy2
20 | - $HOME/.sbt
21 |
22 | env:
23 | matrix:
24 | - KAFKA_TEST_RUNTIME=V_8_2_0 KAFKA_TEST_PROTOCOL=Kafka_0_8
25 | - KAFKA_TEST_RUNTIME=V_0_9_0_1 KAFKA_TEST_PROTOCOL=Kafka_0_8
26 | - KAFKA_TEST_RUNTIME=V_0_9_0_1 KAFKA_TEST_PROTOCOL=Kafka_0_9
27 | - KAFKA_TEST_RUNTIME=V_0_10_0 KAFKA_TEST_PROTOCOL=Kafka_0_8
28 | - KAFKA_TEST_RUNTIME=V_0_10_0 KAFKA_TEST_PROTOCOL=Kafka_0_9
29 | - KAFKA_TEST_RUNTIME=V_0_10_0 KAFKA_TEST_PROTOCOL=Kafka_0_10
30 | - KAFKA_TEST_RUNTIME=V_0_10_1 KAFKA_TEST_PROTOCOL=Kafka_0_8
31 | - KAFKA_TEST_RUNTIME=V_0_10_1 KAFKA_TEST_PROTOCOL=Kafka_0_9
32 | - KAFKA_TEST_RUNTIME=V_0_10_1 KAFKA_TEST_PROTOCOL=Kafka_0_10
33 | - KAFKA_TEST_RUNTIME=V_0_10_1 KAFKA_TEST_PROTOCOL=Kafka_0_10_1
34 | - KAFKA_TEST_RUNTIME=V_0_10_2 KAFKA_TEST_PROTOCOL=Kafka_0_8
35 | - KAFKA_TEST_RUNTIME=V_0_10_2 KAFKA_TEST_PROTOCOL=Kafka_0_9
36 | - KAFKA_TEST_RUNTIME=V_0_10_2 KAFKA_TEST_PROTOCOL=Kafka_0_10
37 | - KAFKA_TEST_RUNTIME=V_0_10_2 KAFKA_TEST_PROTOCOL=Kafka_0_10_1
38 | - KAFKA_TEST_RUNTIME=V_0_10_2 KAFKA_TEST_PROTOCOL=Kafka_0_10_2
39 | - KAFKA_TEST_RUNTIME=V_0_11_0 KAFKA_TEST_PROTOCOL=Kafka_0_10_2
40 | - KAFKA_TEST_RUNTIME=V_0_11_0_1 KAFKA_TEST_PROTOCOL=Kafka_0_10_2
41 | - KAFKA_TEST_RUNTIME=V_1_0_0 KAFKA_TEST_PROTOCOL=Kafka_0_10_2
42 |
43 |
44 | jdk:
45 | - oraclejdk8
46 |
47 |
48 | script:
49 | - sbt ++$TRAVIS_SCALA_VERSION -Dfile.encoding=UTF8 test
50 |
51 |
52 |
--------------------------------------------------------------------------------
/CHANGES.md:
--------------------------------------------------------------------------------
1 | # 0.1.2
2 |
3 | - Fixed processing of incomplete fetch request
--------------------------------------------------------------------------------
/LICENSE:
--------------------------------------------------------------------------------
1 | The MIT License (MIT)
2 |
3 | Copyright (c) 2016-2017 Spinoco Czech Republic, a.s.
4 |
5 | Permission is hereby granted, free of charge, to any person obtaining a copy
6 | of this software and associated documentation files (the "Software"), to deal
7 | in the Software without restriction, including without limitation the rights
8 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
9 | copies of the Software, and to permit persons to whom the Software is
10 | furnished to do so, subject to the following conditions:
11 |
12 | The above copyright notice and this permission notice shall be included in all
13 | copies or substantial portions of the Software.
14 |
15 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
16 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
17 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
18 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
19 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
20 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
21 | SOFTWARE.
22 |
23 |
--------------------------------------------------------------------------------
/README.md:
--------------------------------------------------------------------------------
1 | # fs2-kafka
2 | Kafka client implemented with [fs2](https://github.com/functional-streams-for-scala/fs2) library
3 |
4 | [](https://travis-ci.org/Spinoco/fs2-kafka)
5 | [](https://gitter.im/fs2-kafka/Lobby)
6 |
7 | ## Overview
8 |
9 | fs2-kafka is a simple client for consuming and publishing messages from / to Apache Kafka. It has minimalistic functionality and is fully implemented without any dependency on kafka native driver. It is fully asynchronous and non-blocking.
10 |
11 | Thanks to fs2, it comes with full backpressure and streaming support out of the box, and plays well with other fs2 libraries such as [fs2-http](https://github.com/Spinoco/fs2-http).
12 |
13 | fs2-kafka was built with minimal dependencies (apart from fs2, only scodec and shapeless is used).
14 |
15 | fs2-kafka only supports a subset of features compared to the native kafka client. There is for example no support for creating/administering topics and partitions, or for Kafka Connect / Kafka Streams. The reason for that is that we wanted the client to be as simple as possible, thus leaving all non-essential functionality to other solutions. Please note there is also no support for SASL Kafka protocol now.
16 |
17 |
18 |
19 | ## Features
20 |
21 | - Subscribe to topic / partition, with configurable pre-fetch behaviour.
22 | - Publish to topic / partition
23 | - Query metadata from kafka cluster
24 | - Query offset range for topic / partition
25 |
26 | For the compression of messages fs2-kafka supports GZIP and Snappy compression.
27 |
28 | fs2-kafka supports versions between 0.8.2 and 0.10.2 of kafka cluster with respective protocols. The protocols are cross-tested against the different versions of Kafka (i.e. 0.10.2 is tested to wrok ok with 0.8.2, 0.9.1, 0.10, 0.10.1 and 0.10.2 protocols).
29 |
30 | ## SBT
31 |
32 | Add this to your build file
33 |
34 | ```
35 | libraryDependencies += "com.spinoco" %% "fs2-kafka" % "0.2.0"
36 | ```
37 |
38 | ## Dependencies
39 |
40 | version | scala | fs2 | scodec | shapeless
41 | ---------|-----------|-----------|---------|-----------
42 | 0.4.0-M2 | 2.11, 2.12| 1.0.0-M2| 1.10.3 | 2.3.2
43 | 0.2.0 | 2.11, 2.12| 0.10.0| 1.10.3 | 2.3.2
44 | 0.1.2 | 2.11, 2.12| 0.9.7| 1.10.3 | 2.3.2
45 |
46 |
47 | ## Usage
48 |
49 | Throughout this simple usage guide, please consider having the following imports on your classpath:
50 |
51 | ```scala
52 | import spinoco.fs2.kafka
53 | import spinoco.fs2.kafka._
54 | import spinoco.protocol.kafka._
55 | import scala.concurrent.duration._
56 | ```
57 | If you type console from fs2-kafka's project sbt, these imports are added for you automatically.
58 |
59 | ### Basics
60 |
61 | The client can be used with any `F` (effectfull type) that conforms to `Async[F]`. As a reference throughout the guide we will use fs2.Task.
62 |
63 | To obtain a Kafka client the following code can be used :
64 |
65 | ```scala
66 |
67 | kafka.client(
68 | ensemble = Set(broker("kafka-broker1-dns-name", port = 9092))
69 | , protocol = ProtocolVersion.Kafka_0_10_2
70 | , clientName = "my-client-name"
71 | ) flatMap { kafkaClient =>
72 | /* your code using Kafka Client **/
73 | ???.asInstanceOf[Stream[F, A]]
74 | }
75 |
76 | ```
77 |
78 | Note that the resulting type of this program is `Stream[F, A]`, which means that stream needs to be run, and once it finishes the kafka client will terminate.
79 |
80 | The `protocol` parameter allows to explicitly specify the protocol to be used with Kafka ensemble, to make production migrations and upgrades easy.
81 |
82 |
83 | ### Publishing to topics
84 |
85 | fs2-kafka has 4 possible ways to publish to a topic.
86 |
87 | - `publish1` - publishes one message to topic and partition and awaits confirmation from the leader
88 | - `publishN` - publishes a chunk of messages to topic / partition and awaits confirmation from the leader
89 |
90 | - `publish1Unsafe` - like publish1, except this won't confirm that publish was successful
91 | - `publishNUnsafe` - like publishN, except this won't confirm that publish was successful
92 |
93 | The first two methods (`publish1` and `publishN`) allow to publish messages safely, meaning that once the message is published, the result contains the offset of the first successfully published message. Each of these two methods has two important parameters that have to be specified:
94 |
95 | - `requireQuorum` : When set to true, this indicates that quorum (majority) of ISR (In sync replicas) must aknowledge the publish operation in order for it to be successful. When set to `false`, only the leader must acknowledge the publish.
96 | - `serverAckTimeout` : A timeout at server (broker) to confirm the message publish operation. Note that this timeout is only relevant when publish is sent to kafka broker. There is no timeout locally within the client, so for example if there are no brokers available, the message will be queued up until the leader will become avaialble. That local timeout is subject to be implemented with `F` (see examples below).
97 |
98 | The second two methods to publish to topic (`publish1Unsafe` and `publishNUnsafe`) allow to publish without confirmation from the server. Locally, there is only minimal verification that publish was successful, essentially once the leader is avaiailble and connection with the leader has been established, the publish is considered to be successful.
99 |
100 | Please note there is a difference from how the kafka native client behaves when publishing. Kafka client retries (3x) when there was an unsuccesful publish. fs2-kafka instead fails as soon as possible and leaves it up to the user to choose from any retry mechanics.
101 |
102 | ### Examples of publishing to topic:
103 |
104 | ```scala
105 |
106 | /// assuming val kafkaClient: KafkaClient = ...
107 |
108 | // publishes safely 1 message to topic-A partition 0, with Key 1, and value [1, 2, 3]
109 | // returns when the publish was accepted by a majority (quorum) of the servers. Fails, when server doesn't get acknowledgements from ISRs in 10s or less.
110 | kafkaClient.publish1(topic("topic-A"), partition(0), ByteVector(1), ByteVector(1,2,3), requireQuorum = true, serverAckTimeout = 10 seconds)
111 |
112 | // consumes a stream of key value pairs and publishes them in chunks to a broker. Note that this publishes the stream of messages in chunks utilitizing the kafka batch processing.
113 | // returns when the publish was accepted by a majority (quorum) of the servers. Fails, when server doesn't get acknowledgements from ISRs in 10s or less.
114 | // Additionally the chunks are not compressed, but GZIP or Snappy can be plugged in by specifying the compress attribute.
115 | // The resulting stream will contain the starting offset of the very first message in chunk.
116 | val streamOfValues: Stream[Task, (ByteVector, ByteVector)] = ???
117 | streamOfValues.chunk.evalMap {
118 | kafkaClient.publishN(topic("topic-A"), partition(0), requireQuorum = true, serverAckTimeout = 10 seconds, compress = None)
119 | }
120 |
121 | // publishes an s1 message to topic-A partition 0, with Key 1, and value [1, 2, 3]
122 | // returns immediately, when there is a leader known for topic/ partition
123 | kafkaClient.publish1Unsafe(topic("topic-A"), partition(0), ByteVector(1), ByteVector(1,2,3))
124 |
125 | // consumes a stream of key value pairs and publishes them in chunks to a broker. Note that this publishes the stream of messages in chunks utilitizing the kafka batch processing.
126 | // returns immediately.
127 | // Additionally the chunks are not compressed, but GZIP or Snappy can be plugged in by specifying the compress attribute.
128 | val streamOfValues: Stream[Task, (ByteVector, ByteVector)] = ???
129 | streamOfValues.chunk.evalMap {
130 | kafkaClient.publishNUnsafe(topic("topic-A"), partition(0),compress = None)
131 | }
132 |
133 | ```
134 |
135 | ### Querying kafka for available offsets
136 |
137 | In some scenarios, it is useful to know kafka first and last offset at any given time. As you will see below, the fs2-kafka makes the "tail" of the topic with every message (tail - last message in topic) available. However, you may need to know the available range before you start consuming the topic.
138 |
139 | fs2-kafka has `offsetRangeFor` API for that purpose. When evaluated, this will return the offset of the first and the next message to be published in every topic/partition. When the offsets are equal, the topic/partition is empty.
140 |
141 | exmaple:
142 |
143 | ```scala
144 |
145 | // Queries the first and the next-to-be-published offset of messages in topic `topic-A` partition 0
146 | kafkaClient.offsetRangeFor(topic("topic-A"), partition(0)) flatMap { case (first, next) =>
147 | ???
148 | }
149 |
150 | ```
151 |
152 | ### Subscription
153 |
154 | fs2-kafka client only has one method of subscriptions for the topic. Subscription to topic is utilizing the `subscribe` method with several parameters that can finetune its exact behaviour.
155 |
156 | Subscription to kafka topic always subscribes with the first message specified in the `offset` parameter and always awaits the next messages to be published to the topic (won't terminate when there are no more messages in topic available). That behaviour, in combination with `offsetRangeFor`, shall give the user the posibility to express any subscription pattern necessary.
157 |
158 | ### Subscribe at invalid offset range
159 |
160 | fs2-kafka client is able to recover itself when there is an invalid offset specified for the `offset` parameter. The rules to recover from an invalid offset are:
161 |
162 | - When the offset is lower than the offset of the first message available in the topic, the subscription will start at the very first message available
163 | - When the offset is greater than the offset of the last message availble in the topic, the subscription will start at the very next message arriving to topic since the stream was evaluated.
164 |
165 |
166 | ### Error Recovery
167 |
168 | fs2-kafka client may recover from broker failures. It also allows the recovery to be handled externally by user logic. However, simple machanics have been built in.
169 |
170 | Client's recovery of the `subscribe` method is controlled by `leaderFailureTimeout` and `leaderFailureMaxAttempts` parameters. Default behaviour is to allow for three consecutive leader failures and sleep for 5s between these failures before giving up and failing the subscription. When the user wants to control failure recovery manually, `0 millis` should be used for `leaderFailureTimeout` and `0` for `leaderFailureMaxAttempts` parameters, causing the subscription to fail at any error (except for invalid offsets, which are always handled).
171 |
172 | fs2-kafka client `subscribe` will recover from any failures, and will start the next subscription from the next offset to the last received message.
173 |
174 | ### Further subscription fine-tuning
175 |
176 | fs2-kafka subscription allows to further finetune the behaviour of the subscriptions. When subscribing, the fs2-kafka client may receive more messages in a single chunk, and in that case fs2-kafka fully utiliizes the "chunk" optimization of the fs2 library. Furthermore, the following parameters may control the behaviour of the subscription :
177 |
178 | - `minChunkByteSize` - Controls the minimum number of bytes that must be ready at kafka leader in order to complete a single fetch chunk. If there is less than that amout of bytes, the susbcriber will wait up to that amount of bytes before emitting any messages.
179 | - `maxChunkByteSize` - Controls the maximum number of bytes that is allowed in a single chunk. If there is more bytes than this number available at leader, the client will split the results to more chunks // requests.
180 | - `maxWaitTime` - Controls how much time we wait before the next `fetch` attempt is performed at the leader.
181 |
182 | ### Configurable pre-fetching
183 |
184 | To improve performance, fs2-kafka allows to pre-fetch data from the kafka topic by setting the `prefetch` parameter to `true` (default). This allows to pre-fetch the next chunk of messages from the topic while the current chunk of messages is processed at client. This may significantly increase performance, specifically when dealing with large chunks of messages.
185 |
186 |
187 | ### Subscribe from head
188 |
189 | In fs2-kafka `Head` is considered to be the first message ever published to a topic. Typically, the user may want to either consume all messages in the topic and process any new messages arriving to the topic, or finish subscription on the very last message available in topic.
190 |
191 | To subscribe for all messages and await the next messages to be published in a topic use:
192 |
193 | ```scala
194 |
195 | kafkaClient.subscribe(topic("topic-A"), partition(0), HeadOffset) : Stream[F, TopicMessage]
196 |
197 | ```
198 |
199 | A more complex example occurs when you want to subscribe for messages and stop the subscription at the very last message published to a topic:
200 |
201 | ```scala
202 |
203 | kafkaClient.offsetRangeFor(topic("topic-A"), partition(0)) flatMap { case (first, next) =>
204 | kafkaClient.subscribe(topic("topic-A"), partition(0), first) takeWhile { _.offset <= next }
205 | }: Stream[F, TopicMessage]
206 |
207 | ```
208 |
209 | ### Subscribe from an exact offset
210 |
211 | Very similar to `Head`, the subscription offset can be specified when subscribing from an exact position:
212 |
213 | ```scala
214 | val desiredOffsetToSubscribeFrom: Long = ???
215 | kafkaClient.subscribe(topic("topic-A"), partition(0), offset(desiredOffsetToSubscribeFrom)) : Stream[F, TopicMessage]
216 |
217 | ```
218 |
219 | ### Subscribe at tail
220 |
221 | When subscribing at `Tail`, the subscriber will receive any messages that have not been published to the topic yet:
222 |
223 | ```scala
224 |
225 | kafkaClient.subscribe(topic("topic-A"), partition(0), TailOffset) : Stream[F, TopicMessage]
226 |
227 | ```
228 |
229 |
230 | ## Design
231 |
232 | fs2-kafka is very simple when it comes to internal architecture. Even though kafka supports a variety of patterns, we chose a simple, minimalistic architecture that in our opinion will fit most real-world use-cases.
233 |
234 | ### Subscribers
235 |
236 | Every subscriber in fs2-kafka has their own dedicated TCP connection with the leader. We don't do any connection pooling. The reason for this is that we want to have a predictable performance for every topic / partition combination. Kafka inherently blocks fetch requests serving them in FIFO order. That effectively disallows for reusing TCP connections for topics with mixed message types (i.e. short and long messages). Reusing TCP connections will lead to unpredictable performance.
237 |
238 |
239 | ### Publishers
240 |
241 | Unlike subscribers, publishers do share TCP connections. Each topic/partition combination has only one dedicated TCP connection that is opened with the very first publish attempt to topic/partition and is kept open to be reused for successive publish attempts. fs2-kafka does not reuse connections between topic/partition for the same reasons as the subscribers: FIFO ordering and predictable performance.
242 |
--------------------------------------------------------------------------------
/build.sbt:
--------------------------------------------------------------------------------
1 | import com.typesafe.sbt.pgp.PgpKeys.publishSigned
2 |
3 | val ReleaseTag = """^release/([\d\.]+a?)$""".r
4 |
5 | lazy val contributors = Seq(
6 | "pchlupacek" -> "Pavel Chlupáček"
7 | )
8 |
9 | lazy val commonSettings = Seq(
10 | organization := "com.spinoco",
11 | scalaVersion := "2.12.6",
12 | crossScalaVersions := Seq("2.11.12", "2.12.6"),
13 | scalacOptions ++= Seq(
14 | "-feature",
15 | "-deprecation",
16 | "-language:implicitConversions",
17 | "-language:higherKinds",
18 | "-language:existentials",
19 | "-language:postfixOps",
20 | "-Xfatal-warnings",
21 | "-Yno-adapted-args",
22 | "-Ywarn-value-discard",
23 | "-Ywarn-unused-import"
24 | ),
25 | javaOptions += "-Djava.net.preferIPv4Stack=true",
26 | scalacOptions in (Compile, console) ~= {_.filterNot("-Ywarn-unused-import" == _)},
27 | scalacOptions in (Test, console) := (scalacOptions in (Compile, console)).value,
28 | libraryDependencies ++= Seq(
29 | compilerPlugin("org.scalamacros" % "paradise" % "2.1.0" cross CrossVersion.full)
30 | , "org.scalatest" %% "scalatest" % "3.0.0" % "test"
31 | , "org.scalacheck" %% "scalacheck" % "1.13.4" % "test"
32 | , "co.fs2" %% "fs2-core" % "1.0.2"
33 | , "co.fs2" %% "fs2-io" % "1.0.2"
34 | , "com.spinoco" %% "protocol-kafka" % "0.3.15"
35 | , "com.spinoco" %% "fs2-log-core" % "0.1.0"
36 | ),
37 | scmInfo := Some(ScmInfo(url("https://github.com/Spinoco/fs2-kafka"), "git@github.com:Spinoco/fs2-kafka.git")),
38 | homepage := None,
39 | licenses += ("MIT", url("http://opensource.org/licenses/MIT")),
40 | initialCommands := s"""
41 | import fs2._
42 | import fs2.util._
43 | import spinoco.fs2.kafka
44 | import spinoco.fs2.kafka._
45 | import spinoco.protocol.kafka._
46 | import scala.concurrent.duration._
47 | """
48 | ) ++ testSettings ++ scaladocSettings ++ publishingSettings ++ releaseSettings
49 |
50 | lazy val testSettings = Seq(
51 | parallelExecution in Test := false,
52 | fork in Test := true,
53 | testOptions in Test += Tests.Argument(TestFrameworks.ScalaTest, "-oDF"),
54 | publishArtifact in Test := true
55 | )
56 |
57 | lazy val scaladocSettings = Seq(
58 | scalacOptions in (Compile, doc) ++= Seq(
59 | "-doc-source-url", scmInfo.value.get.browseUrl + "/tree/master€{FILE_PATH}.scala",
60 | "-sourcepath", baseDirectory.in(LocalRootProject).value.getAbsolutePath,
61 | "-implicits",
62 | "-implicits-show-all"
63 | ),
64 | scalacOptions in (Compile, doc) ~= { _ filterNot { _ == "-Xfatal-warnings" } },
65 | autoAPIMappings := true
66 | )
67 |
68 | lazy val publishingSettings = Seq(
69 | publishTo := {
70 | val nexus = "https://oss.sonatype.org/"
71 | if (version.value.trim.endsWith("SNAPSHOT"))
72 | Some("snapshots" at nexus + "content/repositories/snapshots")
73 | else
74 | Some("releases" at nexus + "service/local/staging/deploy/maven2")
75 | },
76 | credentials ++= (for {
77 | username <- Option(System.getenv().get("SONATYPE_USERNAME"))
78 | password <- Option(System.getenv().get("SONATYPE_PASSWORD"))
79 | } yield Credentials("Sonatype Nexus Repository Manager", "oss.sonatype.org", username, password)).toSeq,
80 | publishMavenStyle := true,
81 | pomIncludeRepository := { _ => false },
82 | pomExtra := {
83 | https://github.com/Spinoco/fs2-kafka
84 |
85 | {for ((username, name) <- contributors) yield
86 |
87 | {username}
88 | {name}
89 | http://github.com/{username}
90 |
91 | }
92 |
93 | },
94 | pomPostProcess := { node =>
95 | import scala.xml._
96 | import scala.xml.transform._
97 | def stripIf(f: Node => Boolean) = new RewriteRule {
98 | override def transform(n: Node) =
99 | if (f(n)) NodeSeq.Empty else n
100 | }
101 | val stripTestScope = stripIf { n => n.label == "dependency" && (n \ "scope").text == "test" }
102 | new RuleTransformer(stripTestScope).transform(node)(0)
103 | }
104 | )
105 |
106 | lazy val releaseSettings = Seq(
107 | releaseCrossBuild := true,
108 | releasePublishArtifactsAction := PgpKeys.publishSigned.value
109 | )
110 |
111 | lazy val `f2-kafka` =
112 | project.in(file("."))
113 | .settings(commonSettings)
114 | .settings(
115 | name := "fs2-kafka"
116 | )
117 |
118 |
119 |
120 |
--------------------------------------------------------------------------------
/project/build.properties:
--------------------------------------------------------------------------------
1 | sbt.version=1.2.7
2 |
--------------------------------------------------------------------------------
/project/plugins.sbt:
--------------------------------------------------------------------------------
1 | addSbtPlugin("com.github.tkawachi" % "sbt-doctest" % "0.9.2")
2 | addSbtPlugin("com.github.gseitz" % "sbt-release" % "1.0.10")
3 | addSbtPlugin("com.jsuereth" % "sbt-pgp" % "1.1.0")
4 | addSbtPlugin("org.xerial.sbt" % "sbt-sonatype" % "2.3")
5 | addSbtPlugin("io.get-coursier" % "sbt-coursier" % "1.0.3")
--------------------------------------------------------------------------------
/runtime/start-kafka.sh:
--------------------------------------------------------------------------------
1 | #!/usr/bin/env bash
2 |
3 |
4 | DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
5 |
6 | docker run -d --name="spinoco-fs2-kafka" --restart=no --link spinoco-zk:spinoco-zk \
7 | -e "KAFKA_ADVERTISED_HOST_NAME=127.0.0.1" \
8 | -e "KAFKA_ADVERTISED_PORT=9092" \
9 | -e "KAFKA_ZOOKEEPER_CONNECT=spinoco-zk:2181" \
10 | -e "KAFKA_CREATE_TOPICS=test:1:1" \
11 | -p 9092:9092/tcp \
12 | wurstmeister/kafka:0.10.0.0
--------------------------------------------------------------------------------
/runtime/start-zk.sh:
--------------------------------------------------------------------------------
1 | #!/usr/bin/env bash
2 |
3 | DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
4 |
5 | mkdir -p $DIR/data/zookeeper
6 |
7 | docker run -d --name="spinoco-zk" --restart=no \
8 | -p 2181:2181/tcp \
9 | jplock/zookeeper:3.4.8
--------------------------------------------------------------------------------
/src/main/scala/spinoco/fs2/kafka/KafkaClient.scala:
--------------------------------------------------------------------------------
1 | package spinoco.fs2.kafka
2 |
3 | import java.nio.channels.AsynchronousChannelGroup
4 | import java.time.LocalDateTime
5 | import java.util.Date
6 |
7 | import cats.Applicative
8 | import cats.effect.concurrent.{Deferred, Ref}
9 | import cats.effect._
10 | import cats.kernel.Eq
11 | import cats.implicits._
12 | import fs2._
13 | import fs2.concurrent.{Queue, SignallingRef}
14 | import scodec.bits.ByteVector
15 | import shapeless.{Typeable, tag}
16 | import shapeless.tag._
17 | import spinoco.fs2.kafka.KafkaClient.impl.PartitionPublishConnection
18 | import spinoco.fs2.kafka.failure._
19 | import spinoco.fs2.kafka.network.{BrokerAddress, BrokerConnection}
20 | import spinoco.fs2.log._
21 | import spinoco.protocol.kafka.Message.SingleMessage
22 | import spinoco.protocol.kafka.Request._
23 | import spinoco.protocol.kafka.{ProtocolVersion, Request, _}
24 | import spinoco.protocol.kafka.Response._
25 |
26 | import scala.concurrent.duration._
27 | /**
28 | * Client that binds to kafka broker. Usually application need only one client.
29 | *
30 | * Client lives until the emitted process is interrupted, or fails.
31 | *
32 | */
33 | sealed trait KafkaClient[F[_]] {
34 |
35 |
36 | /**
37 | * Subscribes to specified topic to receive messages published to that topic.
38 | *
39 | * Essentially this acts sort of unix `tail` command.
40 | *
41 | *
42 | * Note that user can fine-tune reads from topic by specifying `minChunkByteSize`, `maxChunkByteSize` and `maxWaitTime` parameters
43 | * to optimize chunking and flow control of reads from Kafka. Default values provide polling each 1 minute whenever at least one message is available.
44 | *
45 | * User can by fine-tuning the maxWaitTime and `leaderFailureMaxAttempts` recovery in case of leadership changes in kafka cluster.
46 | *
47 | * For example, when leader fails, the stream will stop for about `leaderFailureTimeout` and then tries to continue where the last fetch ended.
48 | * However wehn there are leaderFailureMaxAttempts successive failures, then the stream will fail.
49 | *
50 | * Setting `leaderFailureTimeout` to 0 and `leaderFailureMaxAttempts` to 0 will cause resulting stream to fail immediatelly when any failure occurs.
51 | *
52 | *
53 | * @param topicId Name of the topic to subscribe to
54 | * @param partition Partition to subscribe to
55 | * @param offset Offset of the topic to start to read from. First received message may have offset larger
56 | * than supplied offset only if the oldest message has offset higher than supplied offset.
57 | * Otherwise this will always return first message with this offset. -1 specified start from tail (new message arriving to topic)
58 | * @param prefetch When true, the implementation will prefetch next chunk of messages from kafka while processing last chunk of messages.
59 | * @param minChunkByteSize Min size of bytes to read from kafka in single attempt. That number of bytes must be available, in order for read to succeed.
60 | * @param maxChunkByteSize Max number of bytes to include in reply. Should be always > than max siz of single message including key.
61 | * @param maxWaitTime Maximum time to wait before reply, even when `minChunkByteSize` is not satisfied.
62 | * @param leaderFailureTimeout When fetch from Kafka leader fails, this will try to recover connection every this period up to `leaderFailureMaxAttempts` attempt count is exhausted
63 | * @param leaderFailureMaxAttempts Maximum attempts to recover from leader failure, then this will fail.
64 | * @return
65 | */
66 | def subscribe(
67 | topicId: String @@ TopicName
68 | , partition: Int @@ PartitionId
69 | , offset: Long @@ Offset
70 | , prefetch: Boolean = true
71 | , minChunkByteSize: Int = 1
72 | , maxChunkByteSize: Int = 1024 * 1024
73 | , maxWaitTime: FiniteDuration = 1.minute
74 | , leaderFailureTimeout: FiniteDuration = 3.seconds
75 | , leaderFailureMaxAttempts: Int = 20
76 | ): Stream[F, TopicMessage]
77 |
78 |
79 | /**
80 | * Queries offsets for given topic and partition.
81 | * Returns offset of first message kept (head) and offset of next message that will arrive to topic.
82 | * When numbers are equal, then the topic does not include any messages at all.
83 | *
84 | * @param topicId Id of the topic
85 | * @param partition Id of the partition
86 | * @return
87 | */
88 | def offsetRangeFor(
89 | topicId: String @@ TopicName
90 | , partition: Int @@ PartitionId
91 | ): F[(Long @@ Offset, Long @@ Offset)]
92 |
93 | /**
94 | * Publishes single message to the supplied topic.
95 | * Returns None, if the message was not published due topic/partition not existent or
96 | * Some(offset) of published message.
97 | *
98 | * When `F` finishes its evaluation, message is guaranteed to be seen by the ensemble.
99 | *
100 | * @param topicId Topic to publish to
101 | * @param partition Partition to publish to
102 | * @param key Key of the message
103 | * @param message Message itself
104 | * @param requireQuorum If true, this requires quorum of ISR to commit message before leader will reply.
105 | * If false, only leader is required to confirm this publish request.
106 | * @param serverAckTimeout Timeout server waits for replicas to ack the request. If the publish request won't be acked by
107 | * server in this time, then the request fails to be published.
108 | * @return
109 | */
110 | def publish1(
111 | topicId : String @@ TopicName
112 | , partition : Int @@ PartitionId
113 | , key : ByteVector
114 | , message : ByteVector
115 | , requireQuorum : Boolean
116 | , serverAckTimeout: FiniteDuration
117 | ): F[Long] = publishN(topicId, partition, requireQuorum, serverAckTimeout, None)(Chunk.singleton((key, message)))
118 |
119 | /**
120 | * Like `publish` except this won't wait for the confirmation that message was published (fire'n forget).
121 | *
122 | * Note that this does not guarantee that message was even sent to server. It will get queued and will
123 | * be delivered to server within earliest opportunity (once server will be ready to accept it).
124 | *
125 | */
126 | def publishUnsafe1(
127 | topicId: String @@ TopicName
128 | , partition: Int @@ PartitionId
129 | , key: ByteVector
130 | , message: ByteVector
131 | ): F[Unit] = publishUnsafeN(topicId, partition, None)(Chunk.singleton((key, message)))
132 |
133 | /**
134 | * Publishes Chunk of messages to the ensemble. The messages are published as a whole batch, so when this
135 | * terminates, all messages are guaranteed to be processed by kafka server.
136 | *
137 | * Returns offset of very first message published.
138 | *
139 | * @param topicId Id of the topic to publish to.
140 | * @param partition Partition to publish to.
141 | * @param compress When defined, messages will be compressed by supplied algorithm.
142 | * @param serverAckTimeout Defines how long to wait for server to confirm that messages were published.
143 | * Note that this will fail the resulting task if timeout is exceeded, but that won't guarantee that
144 | * messages were not published.
145 | * @param messages Chunk of messages to publish. First is id of the topic, second is partition, then key and message itself.
146 | * Additionally `A` may be passed to pair the offset of the message in resulting chunk.
147 | * @param requireQuorum If true, this requires quorum of ISR to commit message before leader will reply.
148 | * If false, only leader is required to confirm this publish request.
149 | *
150 | * @return
151 | */
152 | def publishN(
153 | topicId: String @@ TopicName
154 | , partition: Int @@ PartitionId
155 | , requireQuorum: Boolean
156 | , serverAckTimeout: FiniteDuration
157 | , compress: Option[Compression.Value]
158 | )(messages: Chunk[(ByteVector, ByteVector)]): F[Long]
159 |
160 | /**
161 | * Like `publishN` except this won't await for messages to be confirmed to be published successfully.
162 | *
163 | * Note that this does not guarantee that message was even sent to server. It will get queued and will
164 | * be delivered to server within earliest opportunity (once server will be ready to accept it).
165 | *
166 | */
167 | def publishUnsafeN(
168 | topic: String @@ TopicName
169 | , partition: Int @@ PartitionId
170 | , compress: Option[Compression.Value]
171 | )(messages: Chunk[(ByteVector, ByteVector)]): F[Unit]
172 |
173 |
174 | /**
175 | * Creates discrete signal of leaders that is queried from periodical query of metadata from brokers.
176 | *
177 | * While this stream is consumed, this will keep connection with very first broker that have answered request for metadata succesfully.
178 | *
179 | * If there is no broker available to server metadata request (all brokers was failing recently w/o providing valid response), this will fail as NoBrokerAvailable.
180 | *
181 | * If the broker from which metadata are queried will fail, this will try next broker in supplied seed.
182 | *
183 | * @param delay Delay to refresh new metadata from last known good broker
184 | */
185 | def leaders(delay: FiniteDuration): Stream[F, Map[(String @@ TopicName, Int @@ PartitionId), BrokerAddress]]
186 |
187 | /**
188 | * Like `leaders` but queries only for supplied topics
189 | */
190 | def leaderFor(delay: FiniteDuration)(topic: (String @@ TopicName), topics: (String @@ TopicName) *): Stream[F, Map[(String @@ TopicName, Int @@ PartitionId), BrokerAddress]]
191 |
192 |
193 | }
194 |
195 |
196 | object KafkaClient {
197 |
198 | /**
199 | *
200 | * @param ensemble Ensemble to connect to. Must not be empty.
201 | * @param protocol Protocol that will be used for requests. This shall be lowest common protocol supported by all brokers.
202 | * @param clientName Name of the client. Name is suffixed for different type of connections to broker:
203 | * - initial-meta-rq : Initial connection to query all available brokers
204 | * - control : Control connection where publish requests and metadata requests are sent to
205 | * - fetch: Connection where fetch requests are sent to.
206 | * @param brokerWriteTimeout Timeout to complete single write (tcp) operation to broker before failing it.
207 | * @param queryOffsetTimeout Timeout to query any partition offset.
208 | * @param brokerReadMaxChunkSize Max size of chunk that is read in single tcp operation from broker
209 | * @param getLeaderDelay How often re-query for leader if the leader is not known. Applies only for publish conmections.
210 | * @param brokerControlQueueBound Max number of unprocessed messages to keep for broker, before stopping accepting new messages for broker.
211 | *
212 | * @see [[spinoco.fs2.kafka.client]]
213 | */
214 | def apply[F[_] : ConcurrentEffect : Timer : Log: ContextShift](
215 | ensemble: Set[BrokerAddress]
216 | , protocol: ProtocolVersion.Value
217 | , clientName: String
218 | , getNow: => LocalDateTime = LocalDateTime.now()
219 | , brokerWriteTimeout: Option[FiniteDuration] = Some(10.seconds)
220 | , queryOffsetTimeout: FiniteDuration = 10.seconds
221 | , brokerReadMaxChunkSize: Int = 256 * 1024
222 | , getLeaderDelay: FiniteDuration = 3.seconds
223 | , brokerControlQueueBound: Int = 10 * 1000
224 | )(implicit AG: AsynchronousChannelGroup): Stream[F,KafkaClient[F]] = {
225 |
226 | def brokerConnection(addr: BrokerAddress):Pipe[F,RequestMessage,ResponseMessage] = s =>
227 | Stream.eval(addr.toInetSocketAddress[F]).flatMap { inetSocketAddress =>
228 | s through BrokerConnection[F](inetSocketAddress, brokerWriteTimeout, brokerReadMaxChunkSize)
229 | }
230 |
231 | val fetchMeta = impl.requestReplyBroker[F, Request.MetadataRequest, Response.MetadataResponse](brokerConnection, protocol, s"$clientName-meta-rq") _
232 |
233 | def publishConnection(topicId: String @@ TopicName, partitionId: Int @@ PartitionId): F[PartitionPublishConnection[F]] = {
234 | impl.publishLeaderConnection(
235 | connection = brokerConnection
236 | , protocol = protocol
237 | , clientId = s"$clientName-produce"
238 | , getLeaderFor = impl.leaderFor(fetchMeta, ensemble.toSeq)
239 | , getLeaderDelay = getLeaderDelay
240 | , topicId = topicId
241 | , partition = partitionId
242 | )
243 | }
244 |
245 |
246 | Stream.bracket(impl.mkClient(
247 | ensemble = ensemble
248 | , publishConnection = publishConnection
249 | , fetchMetadata = fetchMeta
250 | , fetchConnection = impl.fetchBrokerConnection(brokerConnection, protocol, s"$clientName-fetch")
251 | , offsetConnection = impl.offsetConnection(brokerConnection, protocol, s"$clientName-offset")
252 | , metaRequestConnection = impl.metadataConnection(brokerConnection, protocol, s"$clientName-meta")
253 | , queryOffsetTimeout = queryOffsetTimeout
254 | , protocol = protocol
255 | ))({ case (_, shutdown) => shutdown }).map(_._1)
256 | }
257 |
258 |
259 |
260 | protected[kafka] object impl {
261 |
262 | sealed trait PartitionPublishConnection[F[_]] {
263 | def run: F[Unit]
264 | def shutdown: F[Unit]
265 | def publish(data: Vector[Message], timeout: FiniteDuration, acks: RequiredAcks.Value): F[Option[(Long @@ Offset, Option[Date])]]
266 | }
267 |
268 | sealed trait Publisher[F[_]] {
269 | def shutdown: F[Unit]
270 | def publish(topic: String @@ TopicName, partition: Int @@ PartitionId, data: Vector[Message], timeout: FiniteDuration, acks: RequiredAcks.Value): F[Option[(Long @@ Offset, Option[Date])]]
271 | }
272 |
273 |
274 | /**
275 | * Creates a client and F that cleans up lients resources.
276 | * @param ensemble Initial kafka clients to connect to
277 | * @param fetchMetadata A function fo fetch metadata from client specified provided address and signal of state.
278 | * @return
279 | */
280 | def mkClient[F[_] : Log : Concurrent : Timer](
281 | ensemble: Set[BrokerAddress]
282 | , publishConnection: (String @@ TopicName, Int @@ PartitionId) => F[PartitionPublishConnection[F]]
283 | , fetchMetadata: (BrokerAddress, MetadataRequest) => F[MetadataResponse]
284 | , fetchConnection : BrokerAddress => Pipe[F, FetchRequest, (FetchRequest, FetchResponse)]
285 | , offsetConnection : BrokerAddress => Pipe[F, OffsetsRequest, OffsetResponse]
286 | , metaRequestConnection: BrokerAddress => Pipe[F, MetadataRequest, MetadataResponse]
287 | , queryOffsetTimeout: FiniteDuration
288 | , protocol: ProtocolVersion.Value
289 | ): F[(KafkaClient[F], F[Unit])] = {
290 | mkPublishers(publishConnection) map { publisher =>
291 |
292 | val queryOffsetRange = impl.queryOffsetRange(impl.leaderFor(fetchMetadata, ensemble.toSeq), offsetConnection, queryOffsetTimeout) _
293 |
294 | def preparePublishMessages(messages: Chunk[(ByteVector, ByteVector)], compress: Option[Compression.Value]) = {
295 | val singleMessages = messages.map { case (k, v) => Message.SingleMessage(0, MessageVersion.V0, None, k , v) }
296 | compress match {
297 | case None => singleMessages.toVector
298 | case Some(compression) => Vector(Message.CompressedMessages(0, MessageVersion.V0, compression, None, singleMessages.toVector))
299 | }
300 | }
301 |
302 | val NoResponseTimeout = 10.seconds
303 |
304 | val client = new KafkaClient[F] {
305 |
306 | def subscribe(
307 | topicId: @@[String, TopicName]
308 | , partition: @@[Int, PartitionId]
309 | , offset: @@[Long, Offset]
310 | , prefetch: Boolean
311 | , minChunkByteSize: Int
312 | , maxChunkByteSize: Int
313 | , maxWaitTime: FiniteDuration
314 | , leaderFailureTimeout: FiniteDuration
315 | , leaderFailureMaxAttempts: Int
316 | ): Stream[F, TopicMessage] =
317 | subscribePartition[F](
318 | topicId = topicId
319 | , partition = partition
320 | , firstOffset = offset
321 | , prefetch = prefetch
322 | , minChunkByteSize = minChunkByteSize
323 | , maxChunkByteSize = maxChunkByteSize
324 | , maxWaitTime = maxWaitTime
325 | , protocol = protocol
326 | , fetchConnection = fetchConnection
327 | , getLeader = impl.leaderFor(fetchMetadata, ensemble.toSeq)
328 | , queryOffsetRange = queryOffsetRange
329 | , leaderFailureTimeout = leaderFailureTimeout
330 | , leaderFailureMaxAttempts = leaderFailureMaxAttempts
331 | )
332 |
333 | def offsetRangeFor(
334 | topicId: @@[String, TopicName]
335 | , partition: @@[Int, PartitionId]
336 | ): F[(Long @@ Offset, Long @@ Offset)] =
337 | queryOffsetRange(topicId, partition)
338 |
339 | def publishN(
340 | topicId: String @@ TopicName
341 | , partition: Int @@ PartitionId
342 | , requireQuorum: Boolean
343 | , serverAckTimeout: FiniteDuration
344 | , compress: Option[Compression.Value]
345 | )(messages: Chunk[(ByteVector, ByteVector)]): F[Long] = {
346 |
347 | val toPublish = preparePublishMessages(messages, compress)
348 | val requiredAcks = if (requireQuorum) RequiredAcks.Quorum else RequiredAcks.LocalOnly
349 | publisher.publish(topicId, partition, toPublish, serverAckTimeout, requiredAcks) flatMap {
350 | case None => Sync[F].raiseError(new Throwable(s"Successfully published to $topicId, $partition, but no result available?"))
351 | case Some((o, _)) => Applicative[F].pure(o)
352 | }
353 | }
354 |
355 | def publishUnsafeN(
356 | topicId: @@[String, TopicName]
357 | , partition: @@[Int, PartitionId]
358 | , compress: Option[Compression.Value]
359 | )(messages: Chunk[(ByteVector, ByteVector)]): F[Unit] = {
360 | val toPublish = preparePublishMessages(messages, compress)
361 | publisher.publish(topicId, partition, toPublish, NoResponseTimeout, RequiredAcks.NoResponse) void
362 | }
363 |
364 | def leaders(delay: FiniteDuration): Stream[F, Map[(@@[String, TopicName], @@[Int, PartitionId]), BrokerAddress]] =
365 | impl.leadersDiscrete(
366 | metaRequestConnection = metaRequestConnection
367 | , seed = ensemble.toSeq
368 | , delay = delay
369 | , topics = Vector.empty
370 | )
371 |
372 |
373 | def leaderFor(delay: FiniteDuration)(topic: @@[String, TopicName], topics: @@[String, TopicName]*): Stream[F, Map[(@@[String, TopicName], @@[Int, PartitionId]), BrokerAddress]] =
374 | impl.leadersDiscrete(
375 | metaRequestConnection = metaRequestConnection
376 | , seed = ensemble.toSeq
377 | , delay = delay
378 | , topics = Vector(topic) ++ topics
379 | )
380 |
381 | }
382 |
383 | client -> publisher.shutdown
384 | }
385 | }
386 |
387 |
388 |
389 |
390 | /**
391 | * Queries all supplied seeds for first leader and then returns that leader. Returns None if no seed replied with leader for that partition
392 | * @param requestMeta A function that requests signle metadata
393 | * @param seed A seed of brokers
394 | * @param topicId Id of topic
395 | * @param partition Id of partition
396 | * @tparam F
397 | * @return
398 | */
399 | def leaderFor[F[_] : Sync](
400 | requestMeta: (BrokerAddress, MetadataRequest) => F[MetadataResponse]
401 | , seed: Seq[BrokerAddress]
402 | )(topicId: String @@ TopicName, partition: Int @@ PartitionId) :F[Option[BrokerAddress]] = {
403 | Stream.emits(seed)
404 | .evalMap { address => requestMeta(address, MetadataRequest(Vector(topicId))).attempt }
405 | .collect { case Right(response) => response }
406 | .map { resp =>
407 | resp.topics.find(_.name == topicId) flatMap { _.partitions.find( _.id == partition)} flatMap {
408 | _.leader flatMap { leaderId => resp.brokers.find { _.nodeId == leaderId } map { b => BrokerAddress(b.host, b.port) } }
409 | }
410 | }
411 | .collectFirst { case Some(broker) => broker }
412 | .compile
413 | .last
414 | }
415 |
416 |
417 |
418 |
419 |
420 |
421 | val consumerBrokerId = tag[Broker](-1)
422 |
423 |
424 | /**
425 | * Augments connection to broker to FetchRequest/FetchResponse pattern.
426 | *
427 | * Apart of supplying fetch fith proper details, this echoes original request with every fetch
428 | *
429 | * @param brokerConnection Connection to broker
430 | * @param version protocol version
431 | * @param clientId Id of client
432 | * @param address Address of broker.
433 | */
434 | def fetchBrokerConnection[F[_] : Concurrent](
435 | brokerConnection : BrokerAddress => Pipe[F, RequestMessage, ResponseMessage]
436 | , version: ProtocolVersion.Value
437 | , clientId: String
438 | )(address: BrokerAddress): Pipe[F, FetchRequest, (FetchRequest, FetchResponse)] = { s =>
439 | Stream.eval(SignallingRef[F, Map[Int, FetchRequest]](Map.empty)).flatMap { openRequestSignal =>
440 | s.zip(indexer[F]).evalMap { case (request, idx) =>
441 | openRequestSignal.update(_ + (idx -> request)) as RequestMessage(version, idx, clientId, request)
442 | }
443 | .through(brokerConnection(address)).evalMap { resp => resp.response match {
444 | case fetch: FetchResponse =>
445 | openRequestSignal.get.map { _.get(resp.correlationId) }.flatMap {
446 | case Some(req) => openRequestSignal.update(_ - resp.correlationId) as ((req, fetch))
447 | case None => Sync[F].raiseError[(FetchRequest, FetchResponse)](new Throwable(s"Invalid response to fetch request, request not available: $resp"))
448 | }
449 | case _ =>
450 | Sync[F].raiseError[(FetchRequest, FetchResponse)](new Throwable(s"Invalid response to fetch request: $resp"))
451 | }}
452 | }
453 | }
454 |
455 | private def indexer[F[_]]: Stream[F, Int] = Stream.range(0, Int.MaxValue).covary[F].repeat
456 |
457 |
458 | /**
459 | * Creates connection that allows to submit offset Requests.
460 | */
461 | def offsetConnection[F[_] : RaiseThrowable](
462 | brokerConnection : BrokerAddress => Pipe[F, RequestMessage, ResponseMessage]
463 | , version: ProtocolVersion.Value
464 | , clientId: String
465 | )(address: BrokerAddress): Pipe[F, OffsetsRequest, OffsetResponse] = { s =>
466 | (s.zip(indexer) map { case (request, idx) =>
467 | RequestMessage(version, idx, clientId, request)
468 | } through brokerConnection(address)) flatMap { resp => resp.response match {
469 | case offset: OffsetResponse => Stream.emit(offset)
470 | case _ => Stream.raiseError(UnexpectedResponse(address, resp))
471 | }}
472 | }
473 |
474 |
475 | /**
476 | * Creates connection that allows to submit offset Requests.
477 | */
478 | def metadataConnection[F[_] : Sync](
479 | brokerConnection : BrokerAddress => Pipe[F, RequestMessage, ResponseMessage]
480 | , version: ProtocolVersion.Value
481 | , clientId: String
482 | )(address: BrokerAddress): Pipe[F, MetadataRequest, MetadataResponse] = { s =>
483 | Stream.eval(Ref.of[F, Option[MetadataRequest]](None)) flatMap { requestRef =>
484 | (s.evalMap(mrq => requestRef.update(_ => Some(mrq)) as mrq).zip(indexer) map { case (request, idx) =>
485 | RequestMessage(version, idx, clientId, request)
486 | } through brokerConnection(address)) flatMap { resp => resp.response match {
487 | case meta: MetadataResponse => Stream.emit(meta)
488 | case other =>
489 | Stream.eval(requestRef.get) flatMap {
490 | case Some(request) => Stream.raiseError[F](InvalidBrokerResponse(address, "MetadataResponse", request, Some(other)))
491 | case None => Stream.raiseError[F](UnexpectedResponse(address, resp))
492 | }
493 | }}}
494 | }
495 |
496 |
497 | /**
498 | * Subscribes to given partition and topic starting offet supplied.
499 | * Each subscription creates single connection to isr.
500 | *
501 | *
502 | * @param topicId Id of the topic
503 | * @param partition Partition id
504 | * @param firstOffset Offset from where to start (including this one). -1 designated start with very first message published (tail)
505 | * @param getLeader Function to query for available leader
506 | * @param queryOffsetRange Queries range of offset kept for given topic. First is head (oldest message offset) second is tail (offset of the message not yet in topic)
507 | * @return
508 | */
509 | def subscribePartition[F[_] : Concurrent : Timer : Log](
510 | topicId : String @@ TopicName
511 | , partition : Int @@ PartitionId
512 | , firstOffset : Long @@ Offset
513 | , prefetch : Boolean
514 | , minChunkByteSize: Int
515 | , maxChunkByteSize: Int
516 | , maxWaitTime : FiniteDuration
517 | , protocol : ProtocolVersion.Value
518 | , fetchConnection : BrokerAddress => Pipe[F, FetchRequest, (FetchRequest, FetchResponse)]
519 | , getLeader : (String @@ TopicName, Int @@ PartitionId) => F[Option[BrokerAddress]]
520 | , queryOffsetRange : (String @@ TopicName, Int @@ PartitionId) => F[(Long @@ Offset, Long @@ Offset)]
521 | , leaderFailureTimeout: FiniteDuration
522 | , leaderFailureMaxAttempts: Int
523 | ): Stream[F, TopicMessage] = {
524 |
525 | Stream.eval(Ref.of((firstOffset, 0))) flatMap { startFromRef =>
526 | def fetchFromBroker(broker: BrokerAddress): Stream[F, TopicMessage] = {
527 | def tryRecover(rsn: Throwable): Stream[F, TopicMessage] = {
528 | Stream.eval(Log[F].error(s"Leader failed fetch ", Detail(broker) and topicId and partition, Some(rsn))) >>
529 | Stream.eval(startFromRef.get map { _._2 }) flatMap { failures =>
530 | if (failures >= leaderFailureMaxAttempts) Stream.raiseError[F](rsn)
531 | else {
532 | Stream.eval(startFromRef.update { case (start, failures) => (start, failures + 1) }) >>
533 | Stream.sleep(leaderFailureTimeout) >>
534 | Stream.eval(getLeader(topicId, partition)) flatMap {
535 | case None => tryRecover(LeaderNotAvailable(topicId, partition))
536 | case Some(leader) => fetchFromBroker(leader)
537 | }
538 |
539 | }
540 | }
541 | }
542 |
543 | Stream.eval(Queue.unbounded[F, FetchRequest]) flatMap { requestQueue =>
544 | def requestNextChunk: F[Long @@ Offset] = {
545 | startFromRef.get map { _._1 } flatMap { startFrom =>
546 | requestQueue.enqueue1(
547 | FetchRequest(consumerBrokerId, maxWaitTime, minChunkByteSize, None, Vector((topicId, Vector((partition, startFrom, maxChunkByteSize)))))
548 | ) as startFrom
549 | }
550 | }
551 |
552 | (Stream.eval(requestNextChunk) flatMap { thisChunkStart =>
553 | (requestQueue.dequeue through fetchConnection (broker)) flatMap { case (request, fetch) =>
554 | fetch.data.find(_._1 == topicId).flatMap(_._2.find(_.partitionId == partition)) match {
555 | case None =>
556 | Stream.raiseError[F](InvalidBrokerResponse(broker, "FetchResponse", request, Some(fetch)))
557 |
558 | case Some(result) =>
559 | result.error match {
560 | case Some(error) =>
561 | Stream.raiseError[F](BrokerReportedFailure(broker, request, error))
562 |
563 | case None =>
564 | val messages = messagesFromResult(protocol, result)
565 |
566 | val updateLastKnown = messages.lastOption.map(m => m.offset) match {
567 | case None => Stream.empty.covary[F] // No messages emitted, just go on
568 | case Some(lastOffset) => Stream.eval_(startFromRef.set ((offset(lastOffset + 1), 0)))
569 | }
570 |
571 | val removeHead = messages.dropWhile(_.offset < thisChunkStart)
572 |
573 | updateLastKnown ++ {
574 | if (prefetch) Stream.eval_(requestNextChunk) ++ Stream.emits(removeHead)
575 | else Stream.emits(removeHead) ++ Stream.eval_(requestNextChunk)
576 | }
577 | }
578 | }
579 | }}) ++ {
580 | // in normal situations this append shall never be consulted. But the broker may close connection from its side
581 | // and in that case we need to start querying from the last unfinished request or eventually continue from the
582 | // as such we fail there and OnError shall handle failure of early termination from broker
583 | Stream.raiseError[F](new Throwable(s"Leader closed connection early: $broker ($topicId, $partition)"))
584 | }
585 |
586 | } handleErrorWith {
587 | case err: LeaderNotAvailable => tryRecover(err)
588 |
589 | case err: BrokerReportedFailure => err.failure match {
590 | case ErrorType.OFFSET_OUT_OF_RANGE =>
591 | Stream.eval(queryOffsetRange(topicId, partition)).attempt flatMap {
592 | case Right((min, max)) =>
593 | Stream.eval(startFromRef.get) flatMap { case (startFrom, _) =>
594 | if (startFrom < min) Stream.eval(startFromRef.set((min, 0))) >> fetchFromBroker(broker)
595 | else if (startFrom > max) Stream.eval(startFromRef.set((max, 0))) >> fetchFromBroker(broker)
596 | else Stream.raiseError[F](new Throwable(s"Offset supplied is in acceptable range, but still not valid: $startFrom ($min, $max)", err))
597 | }
598 |
599 | case Left(err) => tryRecover(err)
600 | }
601 |
602 | case other => tryRecover(err)
603 | }
604 |
605 | case other => tryRecover(other)
606 | }
607 | }
608 |
609 | def start: Stream[F, TopicMessage] =
610 | Stream.eval(getLeader(topicId, partition)) flatMap {
611 | case Some(broker) => fetchFromBroker(broker)
612 | case None =>
613 | // leader unavailable
614 | Stream.eval(startFromRef.modify { case (o, fail) => ((o, fail + 1 ), fail) }) flatMap { failures =>
615 | if (failures > leaderFailureMaxAttempts) Stream.raiseError[F](NoBrokerAvailable)
616 | else Stream.sleep(leaderFailureTimeout) >> start
617 | }
618 | }
619 |
620 | start
621 | }
622 |
623 | }
624 |
625 |
626 | /**
627 | * Because result of fetch can retrieve messages in compressed and nested forms,
628 | * This decomposes result to simple vector by traversing through the nested message results.
629 | *
630 | * @param result Result from teh fetch
631 | * @return
632 | */
633 | def messagesFromResult(protocol: ProtocolVersion.Value, result: Response.PartitionFetchResult): Vector[TopicMessage] = {
634 |
635 | // Extract compressed messages. No nested compressed messages support
636 | def extractCompressed(m: Vector[Message], lastOffset: Long): Vector[SingleMessage] = {
637 | protocol match {
638 | case ProtocolVersion.Kafka_0_8 |
639 | ProtocolVersion.Kafka_0_9 =>
640 | m.collect { case sm: SingleMessage => sm }
641 |
642 | case ProtocolVersion.Kafka_0_10 |
643 | ProtocolVersion.Kafka_0_10_1 |
644 | ProtocolVersion.Kafka_0_10_2 =>
645 | val first = lastOffset - m.size + 1
646 | m.collect { case sm: SingleMessage => sm.copy(offset = offset(sm.offset + first)) }
647 | }
648 |
649 | }
650 |
651 | def toTopicMessage(message: SingleMessage): TopicMessage =
652 | TopicMessage(offset(message.offset), message.key, message.value, result.highWMOffset)
653 |
654 | result.messages flatMap {
655 | case message: Message.SingleMessage => Vector(toTopicMessage(message))
656 | case messages: Message.CompressedMessages => extractCompressed(messages.messages, messages.offset) map toTopicMessage
657 | }
658 |
659 | }
660 |
661 |
662 |
663 |
664 | /**
665 | * Queries offsets for given topic and partition.
666 | * Returns offset of first message kept (head) and offset of next message that will arrive to topic.
667 | * When numbers are equal, then the topic does not include any messages at all.
668 | * @param topicId Id of the topic
669 | * @param partition Id of the partition
670 | * @param getLeader Queries leader for the partition supplied
671 | * @param brokerOffsetConnection A function to create connection to broker to send // receive OffsetRequests
672 | * @tparam F
673 | */
674 | def queryOffsetRange[F[_] : Sync : Timer](
675 | getLeader: (String @@ TopicName, Int @@ PartitionId) => F[Option[BrokerAddress]]
676 | , brokerOffsetConnection : BrokerAddress => Pipe[F, OffsetsRequest, OffsetResponse]
677 | , maxTimeForQuery: FiniteDuration
678 | )(
679 | topicId: String @@ TopicName
680 | , partition: Int @@ PartitionId
681 | ): F[(Long @@ Offset, Long @@ Offset)] = {
682 | getLeader(topicId, partition) flatMap {
683 | case None => Sync[F].raiseError(LeaderNotAvailable(topicId, partition))
684 | case Some(broker) =>
685 | val requestOffsetDataMin = OffsetsRequest(consumerBrokerId, Vector((topicId, Vector((partition, new Date(-1), Some(Int.MaxValue))))))
686 | val requestOffsetDataMax = OffsetsRequest(consumerBrokerId, Vector((topicId, Vector((partition, new Date(-2), Some(Int.MaxValue))))))
687 | (((Stream(requestOffsetDataMin, requestOffsetDataMax) ++ Stream.sleep_(maxTimeForQuery)) through brokerOffsetConnection(broker)).take(2).compile.toVector) flatMap { responses =>
688 | val results = responses.flatMap(_.data.filter(_._1 == topicId).flatMap(_._2.find(_.partitionId == partition)))
689 | results.collectFirst(Function.unlift(_.error)) match {
690 | case Some(err) => Sync[F].raiseError(BrokerReportedFailure(broker, requestOffsetDataMin, err))
691 | case None =>
692 | val offsets = results.flatMap { _.offsets } map { o => (o: Long) }
693 | if (offsets.isEmpty) Sync[F].raiseError(new Throwable(s"Invalid response. No offsets available: $responses, min: $requestOffsetDataMin, max: $requestOffsetDataMax"))
694 | else Applicative[F].pure ((offset(offsets.min), offset(offsets.max)))
695 | }
696 | }
697 | }
698 | }
699 |
700 |
701 | /**
702 | * Request // reply communication to broker. This sends one message `I` and expect one result `O`
703 | */
704 | def requestReplyBroker[F[_] : Concurrent, I <: Request, O <: Response : Typeable](
705 | f: BrokerAddress => Pipe[F, RequestMessage, ResponseMessage]
706 | , protocol: ProtocolVersion.Value
707 | , clientId: String
708 | )(address: BrokerAddress, input: I): F[O] = {
709 | Deferred[F, Either[Throwable, Option[ResponseMessage]]] flatMap { promise =>
710 | Concurrent[F].start(((Stream.emit(RequestMessage(protocol, 1, clientId, input)) ++ Stream.eval(promise.get).drain) through f(address) take 1).compile.last.attempt.flatMap { r => promise.complete(r) }) >>
711 | promise.get flatMap {
712 | case Right(Some(response)) => Typeable[O].cast(response.response) match {
713 | case Some(o) => Applicative[F].pure(o)
714 | case None => Sync[F].raiseError(InvalidBrokerResponse(address, Typeable[O].describe, input, Some(response.response)))
715 | }
716 | case Right(None) => Sync[F].raiseError(InvalidBrokerResponse(address, Typeable[O].describe, input, None))
717 | case Left(err) => Sync[F].raiseError(BrokerRequestFailure(address, input, err))
718 | }
719 | }
720 | }
721 |
722 |
723 | /**
724 | * With every leader for each topic and partition active this keeps connection open.
725 | * Connection is open once the topic and partition will get first produce request to serve.
726 | * @param connection Function handling connection to Kafka Broker
727 | * @param topicId Id of the topic
728 | * @param partition Id of the partition
729 | * @param protocol Protocol
730 | * @param clientId Id of the client
731 | * @param getLeaderFor Returns a leader for supplied topic and partition
732 | * @param getLeaderDelay Wait that much time to retry for new leader if leader is not known
733 | */
734 | def publishLeaderConnection[F[_] : Log : Concurrent : Timer](
735 | connection: BrokerAddress => Pipe[F, RequestMessage, ResponseMessage]
736 | , protocol: ProtocolVersion.Value
737 | , clientId: String
738 | , getLeaderFor: (String @@ TopicName, Int @@ PartitionId) => F[Option[BrokerAddress]]
739 | , getLeaderDelay: FiniteDuration
740 | , topicId: String @@ TopicName
741 | , partition: Int @@ PartitionId
742 | ) : F[PartitionPublishConnection[F]] = {
743 | type Response = Option[(Long @@ Offset, Option[Date])]
744 | SignallingRef[F, Boolean](false) flatMap { termSignal =>
745 | Queue.bounded[F, (ProduceRequest, Either[Throwable, Response] => F[Unit])](1) flatMap { queue =>
746 | Ref.of[F, Map[Int, (ProduceRequest, Either[Throwable, Response] => F[Unit])]](Map.empty) map { ref =>
747 |
748 | def registerMessage(in: (ProduceRequest, Either[Throwable, Response] => F[Unit]), idx: Int): F[RequestMessage] = {
749 | val (produce, cb) = in
750 |
751 | val msg = RequestMessage(
752 | version = protocol
753 | , correlationId = idx
754 | , clientId = clientId
755 | , request = produce
756 | )
757 |
758 | produce.requiredAcks match {
759 | case RequiredAcks.NoResponse => cb(Right(None)) as msg
760 | case _ => ref.update { _ + (idx -> ((produce, cb))) } as msg
761 | }
762 | }
763 |
764 | def getRequest(response: ResponseMessage): F[Option[(ProduceRequest, Either[Throwable, Response] => F[Unit])]] = {
765 | ref.modify { m => (m - response.correlationId, m.get(response.correlationId)) }
766 | }
767 |
768 |
769 | def completeNotProcessed(failure: Throwable): F[Unit] = {
770 | import cats.instances.list._
771 | ref.modify(prev => (Map.empty, prev)) map { _.values } flatMap { toCancel =>
772 | val fail = Left(failure)
773 | toCancel.toList.traverse(_._2 (fail)).void
774 | }
775 | }
776 |
777 |
778 | // When leader is available this is run to publish any incoming messages to server for processing
779 | // Message is processed from queue, then added to map of open messages and then send to server
780 | // this may only finish when either broker closes connection or fails.
781 | def leaderAvailable(leader: BrokerAddress): Stream[F, Unit] = {
782 | Stream.eval(Log[F].info(s"Leader available for publishing to topic", Detail(topicId) and partition and leader)) >>
783 | (((queue.dequeue.zip(indexer) evalMap (registerMessage _ tupled)) through connection(leader)) flatMap { response =>
784 | Stream.eval(getRequest(response)) flatMap {
785 | case Some((req, cb)) =>
786 | response match {
787 | case ResponseMessage(_, produceResp: ProduceResponse) =>
788 | produceResp.data.find(_._1 == topicId).flatMap(_._2.find(_._1 == partition)) match {
789 | case None => Stream.raiseError[F](UnexpectedResponse(leader, response))
790 |
791 | case Some((_, result)) => result.error match {
792 | case None => Stream.eval_(cb(Right(Some((result.offset, result.time)))))
793 | case Some(err) => Stream.eval_(cb(Left(BrokerReportedFailure(leader, req, err))))
794 | }
795 | }
796 |
797 | case _ => Stream.raiseError[F](UnexpectedResponse(leader, response))
798 | }
799 |
800 | case None =>
801 | Stream.raiseError[F](UnexpectedResponse(leader, response))
802 | }
803 | }) ++ Stream.raiseError[F](new Throwable("Broker terminated connection")) // the first part of the stream shall never terminate unless broker terminates connection, which we convert to failure
804 | }
805 |
806 | val getLeader: Stream[F, Option[BrokerAddress]] =
807 | Stream.eval { getLeaderFor(topicId, partition) }
808 |
809 | // when leader is not available this rejects all requests.
810 | // each `getLeaderDelay` this refreshes new known metadata and once leader is knwon for given topic/partition
811 | // this will terminate with leader address
812 | def leaderUnavailable: Stream[F, BrokerAddress] = {
813 | Stream.eval(Log[F].error(s"Leader unavailable for publishing to topic", Detail(topicId) and partition)) >> {
814 | Stream.eval(SignallingRef[F, Option[BrokerAddress]](None)) flatMap { leaderSignal =>
815 | val cancelIncoming = queue.dequeue.evalMap { case (_, cb) => cb(Left(LeaderNotAvailable(topicId, partition))) } drain
816 | val queryLeader = ((Stream.awakeEvery[F](getLeaderDelay) >> getLeader) evalMap { r => leaderSignal.set (r) }) drain
817 |
818 | (cancelIncoming mergeHaltBoth queryLeader).interruptWhen(leaderSignal.map {
819 | _.nonEmpty
820 | }) ++
821 | (leaderSignal.discrete.take(1) flatMap {
822 | case None => leaderUnavailable // impossible
823 | case Some(leader) =>
824 | Stream.eval(Log[F].debug(s"Publisher got leader for topic", Detail(leader) and topicId and partition)) >> Stream.emit(leader)
825 | })
826 | }
827 | }
828 | }
829 |
830 |
831 |
832 | // main runner
833 | // this never terminates
834 | def runner(lastFailed: Option[BrokerAddress], knownLeader: Stream[F, Option[BrokerAddress]]): Stream[F, Unit] = {
835 | knownLeader flatMap {
836 | case None =>
837 | leaderUnavailable flatMap { leader => runner(None, Stream(Some(leader))) }
838 |
839 | case Some(leader) =>
840 | lastFailed match {
841 | case Some(failedBrokerAddress) if leader == failedBrokerAddress =>
842 | // this indicates that cluster sill thinks the leader is same as the one that failed us, for that reason
843 | // we have to suspend execution for while and retry in FiniteDuration
844 | Stream.eval(Log[F].warn(s"New elected leader is same like the old one, awaiting next leader", Detail(leader) and topicId and partition)) >>
845 | leaderUnavailable flatMap { leader => runner(None, Stream(Some(leader))) }
846 |
847 | case _ =>
848 | // connection with leader will always fail with error.
849 | // so when that happens, all open requests are completed and runner is rerun to switch likely to leaderUnavailable.
850 | // as the last action runner is restarted
851 | leaderAvailable(leader) handleErrorWith { failure =>
852 | Stream.eval(Log[F].error(s"Failure of publishing connection to broker", Detail(leader) and topicId and partition, Some(failure))) >>
853 | Stream.eval(completeNotProcessed(failure)) >>
854 | runner(Some(leader), getLeader)
855 | }
856 | }
857 |
858 | }
859 | }
860 |
861 |
862 | new PartitionPublishConnection[F] {
863 |
864 | def run: F[Unit] =
865 | Log[F].info(s"Starting publish connection", Detail(topicId) and partition) *>
866 | (runner(None, getLeader) interruptWhen termSignal).compile.drain.attempt flatMap { r =>
867 | completeNotProcessed(r.left.toOption.getOrElse(ClientTerminated)) *>
868 | Log[F].info(s"Publish connection terminated", Detail(topicId) and partition, r.left.toOption)
869 | }
870 |
871 | def shutdown: F[Unit] =
872 | Log[F].info(s"Shutting-down publish connection", Detail(topicId) and partition) *> termSignal.set(true)
873 |
874 | def publish(messages: Vector[Message], timeout: FiniteDuration, acks: RequiredAcks.Value): F[Option[(Long @@ Offset, Option[Date])]] = {
875 | Deferred[F, Either[Throwable, Response]] flatMap { promise =>
876 | val request = ProduceRequest(
877 | requiredAcks = acks
878 | , timeout = timeout
879 | , messages = Vector((topicId, Vector((partition, messages))))
880 | )
881 |
882 | queue.enqueue1((request, promise.complete)) >> promise.get flatMap {
883 | case Left(err) => Sync[F].raiseError(err)
884 | case Right(r) => Applicative[F].pure(r)
885 | }
886 |
887 | }
888 |
889 | }
890 | }
891 |
892 | }}}
893 | }
894 |
895 |
896 | /**
897 | * Produces a publisher that for every publishes partition-topic will spawn `PartitionPublishConnection`.
898 | * That connection is handling then all publish requests for given partition.
899 | * Connections are cached are re-used on next publish.
900 | *
901 | * @param createPublisher Function to create single publish connection to given partition.
902 | *
903 | */
904 | def mkPublishers[F[_] : Concurrent](
905 | createPublisher: (String @@ TopicName, Int @@ PartitionId) => F[PartitionPublishConnection[F]]
906 | ): F[Publisher[F]] = {
907 | case class PublisherState(shutdown: Boolean, connections: Map[TopicAndPartition, PartitionPublishConnection[F]])
908 | implicit val stateEq : Eq[PublisherState] = Eq.fromUniversalEquals
909 |
910 | Ref.of(PublisherState(false, Map.empty)) map { stateRef =>
911 |
912 | new Publisher[F] {
913 | import cats.instances.list._
914 |
915 | def shutdown: F[Unit] = {
916 | stateRef.modify { prev => (prev.copy(shutdown = true), prev) } flatMap { previous =>
917 | previous.connections.values.toList.traverse(_.shutdown).void
918 | }
919 | }
920 |
921 | def publish(topic: String @@ TopicName, partition: Int @@ PartitionId, data: Vector[Message], timeout: FiniteDuration, acks: RequiredAcks.Value): F[Option[(Long @@ Offset, Option[Date])]] = {
922 | stateRef.get map { _.connections.get((topic, partition)) } flatMap {
923 | case Some(ppc) => ppc.publish(data, timeout, acks)
924 | case None =>
925 | // lets create a new connection and try to swap it in
926 | createPublisher(topic, partition) flatMap { ppc =>
927 | stateRef.modify { s =>
928 | if (s.shutdown) (s, (s, false))
929 | else {
930 | // add to connections only if there is no current connection yet
931 | if (s.connections.isDefinedAt((topic, partition))) (s, (s, false))
932 | else {
933 | val now = s.copy(connections = s.connections + ((topic, partition) -> ppc))
934 | (now, (s, s != now))
935 | }
936 | }
937 | } flatMap { case (previous, changed) =>
938 | if (previous.shutdown) {
939 | Sync[F].raiseError(ClientTerminated)
940 | } else if (changed) {
941 | // we have won the race, so we shall start the publisher and then publish
942 | Concurrent[F].start(ppc.run) >> publish(topic, partition, data, timeout, acks)
943 | } else {
944 | // someone else won the ppc, we shall publish to new publisher.
945 | publish(topic, partition, data, timeout, acks)
946 | }
947 | }}
948 |
949 | }
950 | }
951 |
952 | }
953 |
954 | }
955 | }
956 |
957 |
958 | /**
959 | * Creates discrete signal of leaders that is queried from periodical query of metadata from brokers.
960 | * This will query supplied seeds in order given and then with first seed that succeeds this will compile
961 | * map of metadata that is emitted.
962 | *
963 | * While this stream is consumed, this will keep connection with very first broker that have answered this.
964 | *
965 | * If there is no broker available to server metadata request, this will fail as NoBrokerAvailable
966 | *
967 | * If the broker from which metadata are queried will fail, this will try next broker in supplied seed.
968 | *
969 | * @param metaRequestConnection connection to create against the given broker
970 | * @param seed Seed of ensemble to use to query metadata from
971 | * @param delay Delay to refresh new metadata from last known good broker
972 | * @param topics If nonempty, filters topic for which the metadata are queried
973 | * @tparam F
974 | * @return
975 | */
976 | def leadersDiscrete[F[_] : Concurrent : Timer : Log](
977 | metaRequestConnection: BrokerAddress => Pipe[F, MetadataRequest, MetadataResponse]
978 | , seed: Seq[BrokerAddress]
979 | , delay: FiniteDuration
980 | , topics: Vector[String @@ TopicName]
981 | ): Stream[F, Map[(String @@ TopicName, Int @@ PartitionId), BrokerAddress]] = {
982 | val metaRq = MetadataRequest(topics)
983 |
984 | // build map of available leaders from response received.
985 | def buildMap(resp: MetadataResponse): Map[(String @@ TopicName, Int @@ PartitionId), BrokerAddress] = {
986 | val brokersById = (resp.brokers map { b => (b.nodeId, BrokerAddress(b.host, b.port)) }).toMap
987 | (resp.topics flatMap { tp => tp.partitions flatMap { p => p.leader flatMap { brokersById.get } map { ((tp.name, p.id), _) } } }).toMap
988 | }
989 |
990 | def go(remains: Seq[BrokerAddress], success: Boolean): Stream[F, Map[(String @@ TopicName, Int @@ PartitionId), BrokerAddress]] = {
991 | remains.headOption match {
992 | case None =>
993 | if (success) go(seed, success = false)
994 | else Stream.raiseError[F](NoBrokerAvailable)
995 |
996 | case Some(broker) =>
997 | Stream.eval(Ref.of[F, Boolean](success)) flatMap { successRef =>
998 | ((Stream.eval(Queue.bounded[F, MetadataRequest](1)) flatMap { requestQ =>
999 | Stream.eval(requestQ.enqueue1(metaRq)) >>
1000 | (requestQ.dequeue through metaRequestConnection(broker)) flatMap { response =>
1001 | // we will here process the brokers and topics, and schedule next request after a given timeout
1002 | Stream.eval_(successRef.set(true)) ++
1003 | Stream.emit(buildMap(response)) ++
1004 | Stream.sleep_(delay) ++
1005 | Stream.eval_(requestQ.enqueue1(metaRq))
1006 | }
1007 | }) ++ Stream.raiseError[F](new Throwable(s"Broker Terminated connection early while monitoring for leader: $broker"))) handleErrorWith { failure =>
1008 | Stream.eval(successRef.get) flatMap { onceOk =>
1009 | Stream.eval(Log[F].error(s"Broker terminated early while fetching metadata update" , Detail(onceOk), Some(failure))) >>
1010 | go(remains.tail, onceOk)
1011 | }
1012 | }
1013 | }
1014 |
1015 | }
1016 | }
1017 | go(seed, success = false)
1018 | }
1019 |
1020 |
1021 |
1022 |
1023 | }
1024 |
1025 |
1026 | }
1027 |
1028 |
1029 |
1030 |
1031 |
1032 |
--------------------------------------------------------------------------------
/src/main/scala/spinoco/fs2/kafka/failure/failures.scala:
--------------------------------------------------------------------------------
1 | package spinoco.fs2.kafka.failure
2 |
3 |
4 | import shapeless.tag.@@
5 | import spinoco.fs2.kafka.network.BrokerAddress
6 | import spinoco.protocol.kafka._
7 |
8 | /**
9 | * Various errors that are eventually to be processed to user code.
10 | */
11 |
12 |
13 | /** Broker responded with invalid response **/
14 | case class InvalidBrokerResponse(address: BrokerAddress, expected: String, request: Request, got: Option[Response]) extends Throwable(s"Broker[$address]: Invalid response to $request, expected: $expected, got: $got")
15 |
16 | /** Given topic // partition has no leader available **/
17 | case class LeaderNotAvailable(topicName: String @@ TopicName, partition: Int @@ PartitionId) extends Throwable(s"Topic: $topicName, partition: $partition has no leader available")
18 |
19 | /** There are no brokers available **/
20 | case object NoBrokerAvailable extends Throwable("No Broker available")
21 |
22 | /** Connection with broker has been terminated with given exception **/
23 | case class BrokerRequestFailure(address: BrokerAddress,request: Request, thrown: Throwable) extends Throwable(s"Broker[$address]: Request failed", thrown)
24 |
25 | /** for supplied request broker has reported failure when respoding **/
26 | case class BrokerReportedFailure(address: BrokerAddress, request: Request, failure: ErrorType.Value) extends Throwable(s"Broker[$address]: Request reported failure: $failure")
27 |
28 | /** for supplied request broker responded with unexpected response **/
29 | case class UnexpectedResponse(address: BrokerAddress, response: ResponseMessage) extends Throwable(s"Unexpected response received from $address: $response")
30 |
31 | /** Publish to client that has been terminate already **/
32 | case object ClientTerminated extends Throwable("Kafka client was terminated")
33 |
--------------------------------------------------------------------------------
/src/main/scala/spinoco/fs2/kafka/kafka.scala:
--------------------------------------------------------------------------------
1 | package spinoco.fs2
2 |
3 |
4 | import java.nio.channels.AsynchronousChannelGroup
5 |
6 | import cats.Show
7 | import cats.effect.{ConcurrentEffect, ContextShift, Timer}
8 | import fs2._
9 | import scodec.bits.ByteVector
10 | import shapeless.tag
11 | import shapeless.tag._
12 | import spinoco.fs2.kafka.network.BrokerAddress
13 | import spinoco.fs2.log.Log
14 | import spinoco.protocol.kafka.{Offset, PartitionId, ProtocolVersion, TopicName}
15 |
16 |
17 | package object kafka {
18 |
19 | /**
20 | * Message read from the topic.
21 | * @param offset Offset of the message
22 | * @param key Key of the message
23 | * @param message Message content
24 | * @param tail Offset of last message in the topic
25 | */
26 | case class TopicMessage(offset: Long @@ Offset, key: ByteVector, message: ByteVector, tail: Long @@ Offset)
27 |
28 | type TopicAndPartition = (String @@ TopicName, Int @@ PartitionId)
29 |
30 | /**
31 | * Build a stream, that when run will produce single kafka client.
32 | *
33 | * Initially client spawns connections to nodes specified in ensemble and queries them for the topology.
34 | * After topology is known, it then initiates connection to each Kafka Broker listed in topology.
35 | * That connection is then used to publish messages to topic/partition that given broker is leader of.
36 | *
37 | * For the subscription client always initiate separate connections to 'followers'. Only in such case there is
38 | * no ISR (follower) available client initiate subscribe connection to 'leader'.
39 | *
40 | *
41 | * Client automatically reacts and recovers from any topology changes that may occur in ensemble:
42 | * - When the leader is changed, the publish requests goes to newly designated leader.
43 | * - When follower dies, or changes its role as leader, then subsequent reads are sent to another follower, if available.
44 | *
45 | *
46 | * @param ensemble Ensemble to connect to. Must not be empty.
47 | * @param protocol Protocol that will be used for requests. This shall be lowest common protocol supported by all brokers.
48 | * @param clientName Name of the client. Name is suffixed for different type of connections to broker:
49 | * - initial-meta-rq : Initial connection to query all available brokers
50 | * - control : Control connection where publish requests and maetadat requests are sent to
51 | * - fetch: Connection where fetch requests are sent to.
52 | */
53 | def client[F[_] : Log : ConcurrentEffect : Timer: ContextShift](
54 | ensemble: Set[BrokerAddress]
55 | , protocol: ProtocolVersion.Value
56 | , clientName: String
57 | )(implicit AG: AsynchronousChannelGroup):Stream[F,KafkaClient[F]] =
58 | KafkaClient(ensemble, protocol, clientName)
59 |
60 |
61 | /** types correctly name of the topic **/
62 | def topic(name: String): String @@ TopicName = tag[TopicName](name)
63 |
64 | /** types correctly id of the partition**/
65 | def partition(id: Int): Int @@ PartitionId = tag[PartitionId](id)
66 |
67 | /** types the offset in the topic**/
68 | def offset(offset: Long): Long @@ Offset = tag[Offset](offset)
69 |
70 | /** Starting from this offset will assure that we will read always from very oldest message (head) kept in topic **/
71 | val HeadOffset = offset(0)
72 |
73 | /** Starting from this offset will assure we starting with most recent messages written to topic (tail) **/
74 | val TailOffset = offset(Long.MaxValue)
75 |
76 | /** syntax helper to construct broker address **/
77 | def broker(host: String, port: Int): BrokerAddress = BrokerAddress(host, port)
78 |
79 | private[kafka] implicit val topicNameShowInstance: Show[String @@ TopicName] =
80 | Show.show(name => name: String)
81 |
82 | private[kafka] implicit val partitionIdShowInstance: Show[Int @@ PartitionId] =
83 | Show.fromToString
84 |
85 | }
86 |
--------------------------------------------------------------------------------
/src/main/scala/spinoco/fs2/kafka/network/BrokerAddress.scala:
--------------------------------------------------------------------------------
1 | package spinoco.fs2.kafka.network
2 |
3 | import java.net.InetSocketAddress
4 |
5 | import cats.Show
6 | import cats.effect.Sync
7 |
8 |
9 | /**
10 | * Address and generation of the broker
11 | * @param host Name of the broker
12 | * @param port Port of the broker
13 | */
14 | case class BrokerAddress(
15 | host: String
16 | , port: Int
17 | ) { self =>
18 |
19 | def toInetSocketAddress[F[_] : Sync]: F[InetSocketAddress] =
20 | Sync[F].catchNonFatal { new InetSocketAddress(self.host, self.port) }
21 |
22 | }
23 |
24 |
25 | object BrokerAddress {
26 |
27 | implicit val showInstance: Show[BrokerAddress] =
28 | Show.show { addr => s"${addr.host}:${addr.port}" }
29 |
30 | }
31 |
32 |
33 |
34 |
--------------------------------------------------------------------------------
/src/main/scala/spinoco/fs2/kafka/network/BrokerConnection.scala:
--------------------------------------------------------------------------------
1 | package spinoco.fs2.kafka.network
2 |
3 | import java.net.InetSocketAddress
4 | import java.nio.channels.AsynchronousChannelGroup
5 |
6 | import cats.syntax.all._
7 | import cats.{Applicative, Monad}
8 | import cats.effect.{ConcurrentEffect, ContextShift, Timer}
9 | import cats.effect.concurrent.Ref
10 | import fs2._
11 | import fs2.Stream._
12 | import scodec.bits.ByteVector
13 | import spinoco.protocol.kafka.Request.{ProduceRequest, RequiredAcks}
14 | import spinoco.protocol.kafka.codec.MessageCodec
15 | import spinoco.protocol.kafka.{ApiKey, RequestMessage, ResponseMessage}
16 |
17 | import scala.annotation.tailrec
18 | import scala.concurrent.duration._
19 |
20 |
21 |
22 | object BrokerConnection {
23 |
24 | /**
25 | *
26 | * Pipe to send/receive messages to/from Broker.
27 | *
28 | * When this resulting Stream is run, it will connect to destination broker,
29 | * then it will start to consume messages from `source` and send them
30 | * to given broker with kafka protocol.
31 | *
32 | * In parallel this receives any messages from broker, de-serializes them
33 | * and emits them.
34 | *
35 | * At any time, when the connection fails, or messages cannot be encoded/decoded
36 | * this process will fail resulting in termination of the connection with Broker.
37 | *
38 | * @param address Address of the kafka Broker
39 | * @param writeTimeout Timeout for performing the write operations
40 | * @param AG
41 | * @tparam F
42 | * @return
43 | */
44 | def apply[F[_] : ConcurrentEffect : Timer: ContextShift](
45 | address: InetSocketAddress
46 | , writeTimeout: Option[FiniteDuration] = None
47 | , readMaxChunkSize: Int = 256 * 1024 // 256 Kilobytes
48 | )(implicit AG:AsynchronousChannelGroup): Pipe[F, RequestMessage, ResponseMessage] = {
49 | (source: Stream[F,RequestMessage]) =>
50 | Stream.resource(fs2.io.tcp.Socket.client[F](address)).flatMap { socket =>
51 | eval(Ref.of(Map.empty[Int,RequestMessage])).flatMap { openRequests =>
52 | val send = source.through(impl.sendMessages(
53 | openRequests = openRequests
54 | , sendOne = (x) => socket.write(x, writeTimeout)
55 | ))
56 |
57 | val receive =
58 | socket.reads(readMaxChunkSize, timeout = None)
59 | .through(impl.receiveMessages(
60 | openRequests = openRequests
61 | ))
62 |
63 | (send.drain.onFinalize(socket.endOfInput) mergeHaltBoth receive)
64 | }
65 | }
66 | }
67 |
68 |
69 | object impl {
70 |
71 | /**
72 | * Pipe that will send one message with `sendOne` shile updating the `openRequests`.
73 | * The `openRequests` or not updated for ProduceRequests that do not expect confirmation from kafka
74 | * @param openRequests
75 | * @param sendOne
76 | * @param F
77 | * @tparam F
78 | * @return
79 | */
80 | def sendMessages[F[_] : Applicative : RaiseThrowable](
81 | openRequests: Ref[F,Map[Int,RequestMessage]]
82 | , sendOne: Chunk[Byte] => F[Unit]
83 | )(implicit F: Monad[F]):Sink[F,RequestMessage] = {
84 | _.evalMap { rm =>
85 | rm.request match {
86 | case produce: ProduceRequest if produce.requiredAcks == RequiredAcks.NoResponse =>
87 | Applicative[F].pure(rm)
88 | case _ =>
89 | openRequests.update(_ + (rm.correlationId -> rm)) as rm
90 | }
91 | }
92 | .flatMap { rm =>
93 | MessageCodec.requestCodec.encode(rm).fold(
94 | err => raiseError(new Throwable(s"Failed to serialize message: $err : $rm"))
95 | , data => eval(sendOne(Chunk.bytes(data.toByteArray)))
96 | )
97 | }
98 | }
99 |
100 |
101 | def receiveMessages[F[_] : RaiseThrowable](
102 | openRequests: Ref[F,Map[Int,RequestMessage]]
103 | ):Pipe[F,Byte,ResponseMessage] = {
104 | _.through(receiveChunks)
105 | .through(decodeReceived(openRequests))
106 | }
107 |
108 | /**
109 | * Collects bytes as they arrive producing chunks of ByteVector
110 | * This reads 32 bits size first, then it reads up to that size of message data
111 | * emitting single ByteVector.
112 | *
113 | * This combinator respects chunks. So if there was more chunks collected in single
114 | * go, they all are emitted in chunk.
115 | *
116 | *
117 | * @return
118 | */
119 | def receiveChunks[F[_]: RaiseThrowable]: Pipe[F,Byte,ByteVector] = {
120 |
121 | def go(acc: ByteVector, msgSz: Option[Int], s: Stream[F, Byte]): Pull[F, ByteVector, Unit] = {
122 | s.pull.uncons flatMap {
123 | case Some((ch, tail)) =>
124 | val bs = ch.toBytes
125 | val buff = acc ++ ByteVector.view(bs.values, bs.offset, bs.size)
126 | val (rem, sz, out) = collectChunks(buff, msgSz)
127 |
128 | Pull.output(Chunk.vector(out)) >> go(rem, sz, tail)
129 |
130 | case None =>
131 | if (acc.nonEmpty) Pull.raiseError(new Throwable(s"Input terminated before all data were consumed. Buff: $acc"))
132 | else Pull.done
133 | }
134 | }
135 |
136 | s => go(ByteVector.empty, None, s).stream
137 | }
138 |
139 |
140 | /**
141 | * Collects chunks of messages received.
142 | * Each chunk is forming whole message, that means this looks for the first 4 bytes, that indicates message size,
143 | * then this take up to that size to produce single ByteVector of message content, and emits that
144 | * content it term of Chunk. Note that Segment may be empty or may contain multiple characters
145 | */
146 |
147 | def collectChunks(
148 | in: ByteVector
149 | , msgSz:Option[Int]
150 | ):(ByteVector, Option[Int], Vector[ByteVector]) = {
151 | @tailrec
152 | def go(buff: ByteVector, currSz: Option[Int], acc: Vector[ByteVector]): (ByteVector, Option[Int], Vector[ByteVector]) = {
153 | currSz match {
154 | case None =>
155 | if (buff.size < 4) (buff, None, acc)
156 | else {
157 | val (sz, rem) = buff.splitAt(4)
158 | go(rem, Some(sz.toInt()), acc)
159 | }
160 |
161 | case Some(sz) =>
162 | if (buff.size < sz) (buff, Some(sz), acc)
163 | else {
164 | val (h,t) = buff.splitAt(sz)
165 | go(t, None, acc :+ h)
166 | }
167 | }
168 | }
169 | go(in, msgSz, Vector.empty)
170 | }
171 |
172 |
173 | /**
174 | * Decodes message received. Due to kafka protocol not having response type encoded
175 | * in protocol itself, we need to consult correlation id, that is read first from the
176 | * message to identify response type.
177 | *
178 | * If request is found for given message, this will remove that request from supplied
179 | * map and will deserialize message according the request type.
180 | *
181 | * If request cannot be found, this fails, as well as when message cannot be decoded.
182 | *
183 | * @param openRequests Ref of current open requests.
184 | * @tparam F
185 | * @return
186 | */
187 | def decodeReceived[F[_] : RaiseThrowable](
188 | openRequests: Ref[F,Map[Int,RequestMessage]]
189 | ):Pipe[F,ByteVector,ResponseMessage] = {
190 | _.flatMap { bs =>
191 | if (bs.size < 4) Stream.raiseError(new Throwable(s"Message chunk does not have correlation id included: $bs"))
192 | else {
193 | val correlationId = bs.take(4).toInt()
194 | eval(openRequests.modify { m => (m - correlationId, m) }).flatMap { m =>
195 | m.get(correlationId) match {
196 | case None => Stream.raiseError(new Throwable(s"Received message correlationId for message that does not exists: $correlationId : $bs : $m"))
197 | case Some(req) =>
198 | MessageCodec.responseCodecFor(req.version, ApiKey.forRequest(req.request)).decode(bs.drop(4).bits)
199 | .fold(
200 | err => Stream.raiseError(new Throwable(s"Failed to decode response to request: $err : $req : $bs"))
201 | , result => Stream.emit(ResponseMessage(correlationId,result.value))
202 | )
203 | }
204 | }
205 | }
206 | }
207 | }
208 |
209 | }
210 |
211 | }
212 |
--------------------------------------------------------------------------------
/src/test/scala/spinoco/fs2/kafka/DockerSupport.scala:
--------------------------------------------------------------------------------
1 | package spinoco.fs2.kafka
2 |
3 | import cats.effect.concurrent.{Ref, Semaphore}
4 | import cats.effect.IO
5 | import cats.syntax.all._
6 | import fs2._
7 | import fs2.concurrent.Queue
8 | import shapeless.tag
9 | import shapeless.tag.@@
10 |
11 | import scala.sys.process.{Process, ProcessLogger}
12 |
13 | /**
14 | *Various helpers for interacting with docker instance
15 | */
16 | object DockerSupport {
17 | val ExtractVersion = """Docker version ([0-9\.\-a-z]+), build ([0-9a-fA-F]+).*""".r("version", "build")
18 | sealed trait DockerId
19 | import Fs2KafkaClientResources._
20 |
21 | /** Returns version of docker, if that docker is available. **/
22 | def dockerVersion:IO[Option[String]] = IO {
23 | val output = Process("docker -v").!!
24 | for {
25 | m <- ExtractVersion.findAllMatchIn(output).toList.headOption
26 | version <- Option(m.group("version"))
27 | build <- Option(m.group("build"))
28 | } yield version + " " + build
29 | }
30 |
31 | /**
32 | * installs given image, if needed, that means when it is not yet installed locally.
33 | * Returns true if image was pulled, false if it was already present
34 | * @param imageName name of the image inclusive the tag
35 | * @return
36 | */
37 | def installImageWhenNeeded(imageName:String):IO[Boolean] = IO {
38 | val current:String= Process(s"docker images $imageName -q").!!
39 | if (current.lines.isEmpty) {
40 | Process(s"docker pull $imageName").!!
41 | true
42 | } else false
43 | }
44 |
45 | /**
46 | * Runs the given image (in background), returning the `id` of the image as an result.
47 | * @param imageName Name (uri) of the image
48 | * @param name Name of the image run (must be unique or not supplied)
49 | * @param props Parameters, props to docker `run` command
50 | * @return
51 | */
52 | def runImage(imageName: String, name: Option[String])(props: String*):IO[String @@ DockerId] = IO {
53 | val cmd = s"docker run -d ${ name.map(n => s"--name=$n").mkString } ${props.mkString(" ")} $imageName"
54 | tag[DockerId](Process(cmd).!!.trim)
55 | }
56 |
57 | /**
58 | * Follows log of executed image. Adequate to docker logs -f
59 | * @param imageId Id of image to follow
60 | * @return
61 | */
62 | def followImageLog(imageId:String @@ DockerId): Stream[IO,String] = {
63 | Stream.eval(Semaphore[IO](1)) flatMap { semaphore =>
64 | Stream.eval(Ref.of(false)) flatMap { isDone =>
65 | Stream.eval(Queue.unbounded[IO,String]).flatMap { q =>
66 |
67 | def enqueue(s: String): Unit = {
68 | semaphore.release >>
69 | isDone.get.flatMap { done => if (!done) q.enqueue1(s) else IO.unit } >>
70 | semaphore.acquire
71 | } unsafeRunSync
72 |
73 | val logger = new ProcessLogger {
74 | def buffer[T](f: => T): T = f
75 | def out(s: => String): Unit = enqueue(s)
76 |
77 | def err(s: => String): Unit = enqueue(s)
78 | }
79 |
80 | Stream.bracket(IO(Process(s"docker logs -f $imageId").run(logger)))(
81 | p => semaphore.release >> isDone.set(true) >> IO(p.destroy()) >> semaphore.acquire
82 | ).flatMap { _ => q.dequeue }
83 | }}}
84 | }
85 |
86 | def runningImages: IO[Set[String @@ DockerId]] = IO {
87 | Process(s"docker ps -q").!!.lines.filter(_.trim.nonEmpty).map(tag[DockerId](_)).toSet
88 | }
89 |
90 | def availableImages: IO[Set[String @@ DockerId]] = IO {
91 | Process(s"docker ps -aq").!!.lines.filter(_.trim.nonEmpty).map(tag[DockerId](_)).toSet
92 | }
93 |
94 |
95 | /**
96 | * Issues a kill to image with given id
97 | */
98 | def killImage(imageId: String @@ DockerId):IO[Unit] = {
99 | IO { Process(s"docker kill $imageId").!! } >>
100 | runningImages.flatMap { allRun =>
101 | if (allRun.exists(imageId.startsWith)) killImage(imageId)
102 | else IO.pure(())
103 | }
104 | }
105 |
106 | /**
107 | * Cleans supplied image from the docker
108 | */
109 | def cleanImage(imageId: String @@ DockerId):IO[Unit] = {
110 | IO { Process(s"docker rm $imageId").!! } >>
111 | availableImages.flatMap { allAvail =>
112 | if (allAvail.exists(imageId.startsWith)) cleanImage(imageId)
113 | else IO.pure(())
114 | }
115 | }
116 |
117 |
118 | def createNetwork(name: String, ipSubnet:String = "172.30.0.0/16 "): IO[Unit] = IO {
119 | Process(s"""docker network create --subnet $ipSubnet $name""").!!
120 | ()
121 | }
122 |
123 |
124 | def removeNetwork(name: String): IO[Unit] = IO {
125 | Process(s"""docker network rm $name""").!!
126 | ()
127 | }
128 |
129 |
130 |
131 | }
132 |
--------------------------------------------------------------------------------
/src/test/scala/spinoco/fs2/kafka/Fs2KafkaClientSpec.scala:
--------------------------------------------------------------------------------
1 | package spinoco.fs2.kafka
2 |
3 | import java.nio.channels.AsynchronousChannelGroup
4 | import java.util.concurrent.Executors
5 |
6 | import cats.effect.{Concurrent, ContextShift, IO, Timer}
7 | import org.scalatest.concurrent.{Eventually, TimeLimitedTests}
8 | import org.scalatest.{FreeSpec, Matchers}
9 | import org.scalatest.prop.GeneratorDrivenPropertyChecks
10 | import org.scalatest.time.SpanSugar._
11 |
12 | import scala.concurrent.ExecutionContext
13 |
14 | object Fs2KafkaClientResources {
15 | implicit val _cxs: ContextShift[IO] = IO.contextShift(ExecutionContext.global)
16 | implicit val _timer: Timer[IO] = IO.timer(ExecutionContext.global)
17 | implicit val _concurrent: Concurrent[IO] = IO.ioConcurrentEffect(_cxs)
18 | implicit val AG: AsynchronousChannelGroup = AsynchronousChannelGroup.withThreadPool(Executors.newFixedThreadPool(8))
19 |
20 |
21 | }
22 |
23 | class Fs2KafkaClientSpec extends FreeSpec
24 | with GeneratorDrivenPropertyChecks
25 | with Matchers
26 | with TimeLimitedTests
27 | with Eventually {
28 |
29 | val timeLimit = 90.seconds
30 |
31 | override implicit val patienceConfig: PatienceConfig = PatienceConfig(timeout = timeLimit)
32 |
33 | implicit override val generatorDrivenConfig: PropertyCheckConfiguration =
34 | PropertyCheckConfiguration(minSuccessful = 25, workers = 1)
35 |
36 |
37 | implicit val _cxs: ContextShift[IO] = Fs2KafkaClientResources._cxs
38 | implicit val _timer: Timer[IO] = Fs2KafkaClientResources._timer
39 | implicit val _concurrent: Concurrent[IO] = Fs2KafkaClientResources._concurrent
40 | implicit val AG: AsynchronousChannelGroup = Fs2KafkaClientResources.AG
41 |
42 | val TestTopic:String = "test-topic"
43 |
44 |
45 |
46 |
47 | }
--------------------------------------------------------------------------------
/src/test/scala/spinoco/fs2/kafka/Fs2KafkaRuntimeSpec.scala:
--------------------------------------------------------------------------------
1 | package spinoco.fs2.kafka
2 |
3 | import java.net.InetAddress
4 |
5 | import cats.effect.IO
6 | import cats.syntax.all._
7 | import fs2._
8 | import org.scalatest.{Args, Status}
9 | import scodec.bits.ByteVector
10 | import shapeless.tag
11 | import shapeless.tag.@@
12 | import spinoco.fs2.kafka.network.BrokerAddress
13 | import spinoco.fs2.log.{Log, StandardProviders}
14 | import spinoco.protocol.kafka.{Broker, PartitionId, ProtocolVersion, TopicName}
15 |
16 | import scala.sys.process.{Process, ProcessLogger}
17 | import scala.concurrent.duration._
18 | import scala.util.Try
19 |
20 |
21 | object Fs2KafkaRuntimeSpec {
22 | val ZookeeperImage = "jplock/zookeeper:3.4.8"
23 | val DefaultZkPort:Int = 2181
24 |
25 | val Kafka8Image = "wurstmeister/kafka:0.8.2.0"
26 | val Kafka9Image = "wurstmeister/kafka:0.9.0.1"
27 | val Kafka10Image = "wurstmeister/kafka:0.10.0.0"
28 | val Kafka101Image = "wurstmeister/kafka:0.10.1.0"
29 | val Kafka102Image = "wurstmeister/kafka:0.10.2.0"
30 | val Kafka11Image = "wurstmeister/kafka:0.11.0.0"
31 | val Kafka1101Image = "wurstmeister/kafka:0.11.0.1"
32 | val Kafka1Image = "wurstmeister/kafka:1.0.0"
33 | }
34 |
35 | object KafkaRuntimeRelease extends Enumeration {
36 | val V_8_2_0 = Value
37 | val V_0_9_0_1 = Value
38 | val V_0_10_0 = Value
39 | val V_0_10_1 = Value
40 | val V_0_10_2 = Value
41 | val V_0_11_0 = Value
42 | val V_0_11_0_1 = Value
43 | val V_1_0_0 = Value
44 | }
45 |
46 |
47 | /**
48 | * Specification that will start kafka runtime before tests are performed.
49 | * Note that data are contained withing docker images, so once the image stops, the data needs to be recreated.
50 | */
51 | class Fs2KafkaRuntimeSpec extends Fs2KafkaClientSpec {
52 | import DockerSupport._
53 | import Fs2KafkaRuntimeSpec._
54 |
55 | val runtime: KafkaRuntimeRelease.Value = Option(System.getenv().get("KAFKA_TEST_RUNTIME")).map(KafkaRuntimeRelease.withName).getOrElse(KafkaRuntimeRelease.V_1_0_0)
56 | val protocol: ProtocolVersion.Value = Option(System.getenv().get("KAFKA_TEST_PROTOCOL")).map(ProtocolVersion.withName).getOrElse(ProtocolVersion.Kafka_0_10_2)
57 |
58 | def skipFor(versions: (KafkaRuntimeRelease.Value, ProtocolVersion.Value)*)(test: => Any): Any = {
59 | if (! versions.contains((runtime, protocol))) test
60 | }
61 |
62 | lazy val thisLocalHost: InetAddress = {
63 | val addr = InetAddress.getLocalHost
64 | if (addr == null) throw new Exception("Localhost cannot be identified")
65 | addr
66 | }
67 |
68 | val testTopicA = topic("test-topic-A")
69 | val part0 = partition(0)
70 |
71 | val localBroker1_9092 = BrokerAddress(thisLocalHost.getHostAddress, 9092)
72 | val localBroker2_9192 = BrokerAddress(thisLocalHost.getHostAddress, 9192)
73 | val localBroker3_9292 = BrokerAddress(thisLocalHost.getHostAddress, 9292)
74 |
75 | val localCluster = Set(localBroker1_9092, localBroker2_9192, localBroker3_9292)
76 |
77 | implicit lazy val log: Log[IO] =
78 | StandardProviders.console[IO](Log.Level.Trace).flatMap { implicit provider =>
79 | Log.sync[IO]
80 | }.use(log => IO(log)).unsafeRunSync() // we cheat here hence we know the console and sync log are pure, os no cleanups called.
81 |
82 |
83 |
84 | /**
85 | * Starts zookeeper listening on given port. ZK runs on host network.
86 | * @return
87 | */
88 | def startZk(port:Int = DefaultZkPort):IO[String @@ DockerId] = {
89 | for {
90 | _ <- dockerVersion.flatMap(_.fold[IO[String]](IO.raiseError(new Throwable("Docker is not available")))(IO.pure))
91 | _ <- installImageWhenNeeded(ZookeeperImage)
92 | _ <- IO { println(s"STARTING ZK @$port") }
93 | runId <- runImage(ZookeeperImage,None)(
94 | "--restart=no"
95 | , "--net=fs2-kafka-network"
96 | , "--name=zookeeper"
97 | , s"-p $port:$port/tcp"
98 | )
99 | } yield runId
100 | }
101 |
102 |
103 | /** stops and cleans the given image **/
104 | def stopImage(zkImageId: String @@ DockerId):IO[Unit] = {
105 | runningImages flatMap { allRunning =>
106 | if (allRunning.exists(zkImageId.startsWith)) killImage(zkImageId) >> cleanImage(zkImageId)
107 | else availableImages flatMap { allAvailable =>
108 | if (allAvailable.exists(zkImageId.startsWith)) cleanImage(zkImageId)
109 | else IO.pure(())
110 | }
111 | }
112 |
113 | }
114 |
115 | /** starts kafka. Kafka runs in host network **/
116 | def startKafka(image: String, port: Int, zkPort: Int = DefaultZkPort, brokerId: Int = 1): IO[String @@ DockerId] = {
117 | for {
118 | _ <- dockerVersion.flatMap(_.fold[IO[String]](IO.raiseError(new Throwable("Docker is not available")))(IO.pure))
119 | _ <- installImageWhenNeeded(image)
120 | params = Seq(
121 | "--restart=no"
122 | , "--net=fs2-kafka-network"
123 | , s"--name=broker$brokerId"
124 | , s"""--env KAFKA_PORT=$port"""
125 | , s"""--env KAFKA_BROKER_ID=$brokerId"""
126 | , s"""--env KAFKA_ADVERTISED_HOST_NAME=broker$brokerId"""
127 | , s"""--env KAFKA_ADVERTISED_PORT=$port"""
128 | , s"""--env KAFKA_ZOOKEEPER_CONNECT=zookeeper:$zkPort"""
129 | , s"-p $port:$port/tcp"
130 |
131 | )
132 | - <- IO { println(s"STARTING BROKER[$brokerId] @$port") }
133 | runId <- runImage(image,None)(params :_*)
134 | } yield runId
135 | }
136 |
137 |
138 | /** creates supplied kafka topic with number of partitions, starting at index 0 **/
139 | def createKafkaTopic (
140 | kafkaDockerId: String @@ DockerId
141 | , name: String @@ TopicName
142 | , partitionCount: Int = 1
143 | , replicas: Int = 1
144 | ):IO[Unit] = IO {
145 | Process("docker", Seq(
146 | "exec", "-i"
147 | , kafkaDockerId
148 | , "bash", "-c", s"$$KAFKA_HOME/bin/kafka-topics.sh --zookeeper zookeeper --create --topic $name --partitions $partitionCount --replication-factor $replicas"
149 | )).!!
150 | ()
151 | }
152 |
153 | def cleanAll: IO[Unit] = IO {
154 | val images = Process("docker", Seq("ps", "-qa")).lineStream
155 | Try(Process("docker", Seq("kill") ++ images).!!(ProcessLogger(_ => ())))
156 | Try(Process("docker", Seq("rm") ++ images).!!(ProcessLogger(_ => ())))
157 | Try(Process("docker", Seq("network", "rm", "fs2-kafka-network") ++ images).!!(ProcessLogger(_ => ())))
158 | ()
159 | }
160 |
161 |
162 | /** process emitting once docker id of zk and kafka in singleton (one node) **/
163 | def withKafkaSingleton[A](version: KafkaRuntimeRelease.Value)(f: (String @@ DockerId, String @@ DockerId) => Stream[IO, A]):Stream[IO,A] = {
164 | Stream.eval(cleanAll) >>
165 | Stream.eval(createNetwork("fs2-kafka-network")) >>
166 | Stream.eval(startZk()).flatMap { zkId =>
167 | awaitZKStarted(zkId) ++ Stream.sleep_[IO](2.seconds) ++
168 | Stream.eval(startK(version, 1)).flatMap { kafkaId =>
169 | (awaitKStarted(version, kafkaId) ++ f(zkId, kafkaId))
170 | .onFinalize {
171 | stopImage(kafkaId) >>
172 | stopImage(zkId) >>
173 | removeNetwork("fs2-kafka-network")
174 | }
175 | }}
176 |
177 | }
178 |
179 | def withKafkaClient[A](version: KafkaRuntimeRelease.Value, protocol: ProtocolVersion.Value)(f: KafkaClient[IO] => Stream[IO, A]): Stream[IO, A] = {
180 | withKafkaSingleton(version) { (_, kafkaDockerId) =>
181 | Stream.sleep[IO](1.second) >>
182 | Stream.eval(createKafkaTopic(kafkaDockerId, testTopicA)) >>
183 | KafkaClient[IO](Set(localBroker1_9092), protocol, "test-client") flatMap { kc =>
184 | awaitLeaderAvailable(kc, testTopicA, part0).drain ++ f(kc)
185 | }
186 | }
187 | }
188 |
189 | def startK(version: KafkaRuntimeRelease.Value, brokerId: Int):IO[String @@ DockerId] = {
190 | val port = 9092+ 100*(brokerId -1)
191 | version match {
192 | case KafkaRuntimeRelease.V_8_2_0 => startKafka(Kafka8Image, port = port, brokerId = brokerId)
193 | case KafkaRuntimeRelease.V_0_9_0_1 => startKafka(Kafka9Image, port = port, brokerId = brokerId)
194 | case KafkaRuntimeRelease.V_0_10_0 => startKafka(Kafka10Image, port = port, brokerId = brokerId)
195 | case KafkaRuntimeRelease.V_0_10_1 => startKafka(Kafka101Image, port = port, brokerId = brokerId)
196 | case KafkaRuntimeRelease.V_0_10_2 => startKafka(Kafka102Image, port = port, brokerId = brokerId)
197 | case KafkaRuntimeRelease.V_0_11_0 => startKafka(Kafka11Image, port = port, brokerId = brokerId)
198 | case KafkaRuntimeRelease.V_0_11_0_1 => startKafka(Kafka1101Image, port = port, brokerId = brokerId)
199 | case KafkaRuntimeRelease.V_1_0_0 => startKafka(Kafka1Image, port = port, brokerId = brokerId)
200 | }
201 | }
202 |
203 | def awaitZKStarted(zkId: String @@ DockerId):Stream[IO,Nothing] = {
204 | followImageLog(zkId).takeWhile(! _.contains("binding to port")).drain ++
205 | Stream.eval_(IO(println(s"Zookeeper started at $zkId")))
206 | }
207 |
208 | def awaitKStarted(version: KafkaRuntimeRelease.Value, kafkaId: String @@ DockerId): Stream[IO, Nothing] = {
209 | val output = Stream.eval_(IO(println(s"Broker $version started at $kafkaId")))
210 | version match {
211 | case KafkaRuntimeRelease.V_8_2_0 =>
212 | followImageLog(kafkaId).takeWhile(! _.contains("New leader is ")).drain ++ output
213 |
214 | case KafkaRuntimeRelease.V_0_9_0_1 =>
215 | followImageLog(kafkaId).takeWhile(! _.contains("New leader is ")).drain ++ output
216 |
217 | case KafkaRuntimeRelease.V_0_10_0 =>
218 | followImageLog(kafkaId).takeWhile(! _.contains("New leader is ")).drain ++ output
219 |
220 | case KafkaRuntimeRelease.V_0_10_1 =>
221 | followImageLog(kafkaId).takeWhile(! _.contains("New leader is ")).drain ++ output
222 |
223 | case KafkaRuntimeRelease.V_0_10_2 =>
224 | followImageLog(kafkaId).takeWhile(! _.contains("New leader is ")).drain ++ output
225 |
226 | case KafkaRuntimeRelease.V_0_11_0 =>
227 | followImageLog(kafkaId).takeWhile(! _.contains(s"[Kafka Server 1], started")).drain ++ output
228 |
229 | case KafkaRuntimeRelease.V_0_11_0_1 =>
230 | followImageLog(kafkaId).takeWhile(! _.contains(s"[Kafka Server 1], started")).drain ++ output
231 |
232 | case KafkaRuntimeRelease.V_1_0_0 =>
233 | followImageLog(kafkaId).takeWhile(! _.contains(s"[KafkaServer id=1] started")).drain ++ output
234 | }
235 | }
236 |
237 | def awaitKFollowerReady(version: KafkaRuntimeRelease.Value, kafkaId: String @@ DockerId, brokerId: Int): Stream[IO, Nothing] = {
238 | val output = Stream.eval_(IO(println(s"Broker $brokerId (follower) $version started at $kafkaId")))
239 | version match {
240 | case KafkaRuntimeRelease.V_8_2_0 =>
241 | followImageLog(kafkaId).takeWhile(! _.contains(s"[Kafka Server $brokerId], started")).drain ++ output
242 |
243 | case KafkaRuntimeRelease.V_0_9_0_1 =>
244 | followImageLog(kafkaId).takeWhile(! _.contains(s"[Kafka Server $brokerId], started")).drain ++ output
245 |
246 | case KafkaRuntimeRelease.V_0_10_0 =>
247 | followImageLog(kafkaId).takeWhile(! _.contains(s"[Kafka Server $brokerId], started")).drain ++ output
248 |
249 | case KafkaRuntimeRelease.V_0_10_1 =>
250 | followImageLog(kafkaId).takeWhile(! _.contains(s"[Kafka Server $brokerId], started")).drain ++ output
251 |
252 | case KafkaRuntimeRelease.V_0_10_2 =>
253 | followImageLog(kafkaId).takeWhile(! _.contains(s"[Kafka Server $brokerId], started")).drain ++ output
254 |
255 | case KafkaRuntimeRelease.V_0_11_0 =>
256 | followImageLog(kafkaId).takeWhile(! _.contains(s"[Kafka Server $brokerId], started")).drain ++ output
257 |
258 | case KafkaRuntimeRelease.V_0_11_0_1 =>
259 | followImageLog(kafkaId).takeWhile(! _.contains(s"[Kafka Server $brokerId], started")).drain ++ output
260 |
261 | case KafkaRuntimeRelease.V_1_0_0 =>
262 | followImageLog(kafkaId).takeWhile(! _.contains(s"[KafkaServer id=$brokerId] started")).drain ++ output
263 | }
264 | }
265 |
266 |
267 |
268 | case class KafkaNodes(
269 | zk: String @@ DockerId
270 | , nodes: Map[Int @@ Broker, String @@ DockerId]
271 | ) { self =>
272 |
273 | def broker1DockerId : String @@ DockerId = nodes(tag[Broker](1))
274 | def broker2DockerId : String @@ DockerId = nodes(tag[Broker](2))
275 | def broker3DockerId : String @@ DockerId = nodes(tag[Broker](3))
276 |
277 |
278 | }
279 |
280 | /** start 3 node kafka cluster with zookeeper **/
281 | def withKafkaCluster(version: KafkaRuntimeRelease.Value): Stream[IO, KafkaNodes] = {
282 | Stream.eval_(createNetwork("fs2-kafka-network")) ++
283 | Stream.bracket(startZk())(stopImage).flatMap { zkId => {
284 | awaitZKStarted(zkId) ++ Stream.sleep_[IO](2.seconds) ++
285 | Stream.bracket(startK(version, 1))(stopImage).flatMap { broker1 =>
286 | awaitKStarted(version, broker1) ++ Stream.sleep_[IO](2.seconds) ++
287 | Stream.bracket(startK(version, 2))(stopImage).flatMap { broker2 =>
288 | awaitKFollowerReady(version, broker2, 2) ++ Stream.sleep_[IO](2.seconds) ++
289 | Stream.bracket(startK(version, 3))(stopImage).flatMap { broker3 =>
290 | awaitKFollowerReady(version, broker3, 3) ++ Stream.sleep_[IO](2.seconds) ++
291 | Stream.emit(KafkaNodes(zkId, Map(tag[Broker](1) -> broker1, tag[Broker](2) -> broker2, tag[Broker](3) -> broker3)))
292 | }
293 | }
294 | }
295 | }
296 | }
297 | .onFinalize(removeNetwork("fs2-kafka-network"))
298 | }
299 |
300 |
301 | def publishNMessages(client: KafkaClient[IO],from: Int, to: Int, quorum: Boolean = false): IO[Unit] = {
302 |
303 | Stream.range(from, to).evalMap { idx =>
304 | client.publish1(testTopicA, part0, ByteVector(1), ByteVector(idx), quorum, 10.seconds)
305 | }
306 | .compile.drain
307 |
308 | }
309 |
310 | def generateTopicMessages(from: Int, to: Int, tail: Long): Vector[TopicMessage] = {
311 | ((from until to) map { idx =>
312 | TopicMessage(offset(idx.toLong), ByteVector(1), ByteVector(idx), offset(tail) )
313 | }) toVector
314 | }
315 |
316 |
317 | def killLeader(client: KafkaClient[IO], nodes: KafkaNodes, topic: String @@ TopicName, partition: Int @@ PartitionId): Stream[IO, Nothing] = {
318 | client.leaderFor(500.millis)(topic).take(1) map { _((topic, partition)) } flatMap { leader =>
319 | leader match {
320 | case BrokerAddress(_, 9092) => Stream.eval_(killImage(nodes.nodes(tag[Broker](1))))
321 | case BrokerAddress(_, 9192) => Stream.eval_(killImage(nodes.nodes(tag[Broker](2))))
322 | case BrokerAddress(_, 9292) => Stream.eval_(killImage(nodes.nodes(tag[Broker](3))))
323 | case other => Stream.raiseError(new Throwable(s"Unexpected broker: $other"))
324 | }
325 | }
326 | }
327 |
328 |
329 |
330 | def awaitLeaderAvailable(client: KafkaClient[IO], topic: String @@ TopicName, partition: Int @@ PartitionId): Stream[IO, BrokerAddress] = {
331 | client.leaderFor(500.millis)(topic).map(_.get((topic, partition))).unNone.take(1)
332 | }
333 |
334 | def awaitNewLeaderAvailable(client: KafkaClient[IO], topic: String @@ TopicName, partition: Int @@ PartitionId, previous: BrokerAddress): Stream[IO, BrokerAddress] = {
335 | client.leaderFor(500.millis)(topic).map(_.get((topic, partition)).filterNot(_ == previous)).unNone.take(1)
336 | }
337 |
338 | override def runTest(testName: String, args: Args): Status = {
339 | println(s"Starting: $testName")
340 | try super.runTest(testName, args)
341 | finally println(s"Stopping: $testName")
342 | }
343 | }
344 |
--------------------------------------------------------------------------------
/src/test/scala/spinoco/fs2/kafka/KafkaClientLastOffsetSpec.scala:
--------------------------------------------------------------------------------
1 | package spinoco.fs2.kafka
2 |
3 |
4 | import fs2._
5 | import scodec.bits.ByteVector
6 | import shapeless.tag
7 | import spinoco.protocol.kafka._
8 |
9 | import scala.concurrent.duration._
10 |
11 |
12 |
13 | class KafkaClientLastOffsetSpec extends Fs2KafkaRuntimeSpec {
14 |
15 | s"Last Offset (single broker)" - {
16 |
17 | "queries when topic is empty" in {
18 | withKafkaClient(runtime, protocol) { kc =>
19 | Stream.eval(kc.offsetRangeFor(testTopicA, tag[PartitionId](0)))
20 | }.compile.toVector.unsafeRunTimed(30.seconds) shouldBe Some(Vector((offset(0), offset(0))))
21 | }
22 |
23 |
24 | "queries when topic is non-empty" in {
25 | withKafkaClient(runtime, protocol) { kc =>
26 | Stream.eval(kc.publish1(testTopicA, part0, ByteVector(1, 2, 3), ByteVector(5, 6, 7), false, 10.seconds)) >>
27 | Stream.eval(kc.offsetRangeFor(testTopicA, tag[PartitionId](0)))
28 | }.compile.toVector.unsafeRunTimed(30.seconds) shouldBe Some(Vector((offset(0), offset(1))))
29 | }
30 |
31 |
32 | }
33 |
34 |
35 | }
36 |
--------------------------------------------------------------------------------
/src/test/scala/spinoco/fs2/kafka/KafkaClientPublishSpec.scala:
--------------------------------------------------------------------------------
1 | package spinoco.fs2.kafka
2 |
3 | import cats.effect.IO
4 |
5 | import fs2._
6 | import scodec.bits.ByteVector
7 | import spinoco.protocol.kafka.Compression
8 |
9 | import scala.concurrent.duration._
10 |
11 |
12 |
13 |
14 | class KafkaClientPublishSpec extends Fs2KafkaRuntimeSpec {
15 |
16 |
17 | s"single-broker" - {
18 |
19 | "publish-unsafe" in {
20 |
21 | def publish(kc: KafkaClient[IO]) = {
22 | Stream.range(0, 10) evalMap { idx =>
23 | kc.publishUnsafe1(testTopicA, part0, ByteVector(1), ByteVector(idx))
24 | } drain
25 | }
26 |
27 | withKafkaClient(runtime, protocol) { kc =>
28 | publish(kc) ++
29 | Stream.sleep[IO](2.second) >> // wait for message to be accepted
30 | kc.subscribe(testTopicA, part0, offset(0l)).take(10)
31 | }.compile.toVector.unsafeRunTimed(30.seconds).map(_.size) shouldBe Some(10)
32 |
33 | }
34 |
35 | "publish-response" in {
36 | def publish(kc: KafkaClient[IO]) = {
37 | Stream.range(0, 10) evalMap { idx =>
38 | kc.publish1(testTopicA, part0, ByteVector(1), ByteVector(idx), requireQuorum = false, serverAckTimeout = 3.seconds)
39 | } map (Left(_))
40 | }
41 |
42 | withKafkaClient(runtime, protocol) { kc =>
43 | publish(kc) ++
44 | (kc.subscribe(testTopicA, part0, offset(0l)) map (Right(_)))
45 | }.take(20).compile.toVector.unsafeRunTimed(30.seconds) shouldBe Some {
46 | (for { idx <- 0 until 10} yield Left(offset(idx))).toVector ++
47 | (for { idx <- 0 until 10} yield Right(TopicMessage(offset(idx), ByteVector(1), ByteVector(idx), offset(10)))).toVector
48 | }
49 | }
50 |
51 |
52 | "publishN-unsafe" in {
53 | def publish(kc: KafkaClient[IO]) = {
54 | Stream.range(0, 10) evalMap { idx =>
55 | kc.publishUnsafeN(testTopicA, part0, compress = None)(Chunk.seq(for { i <- 0 until 10} yield (ByteVector(i), ByteVector(i*idx))))
56 | } drain
57 | }
58 |
59 | withKafkaClient(runtime, protocol) { kc =>
60 | publish(kc) ++
61 | Stream.sleep[IO](3.second) >> // wait for message to be accepted
62 | kc.subscribe(testTopicA, part0, offset(0l)).take(100)
63 | }.compile.toVector.unsafeRunTimed(30.seconds).map(_.size) shouldBe Some(100)
64 |
65 | }
66 |
67 |
68 | "publishN-response" in {
69 | def publish(kc: KafkaClient[IO]) = {
70 | Stream.range(0, 10) evalMap { idx =>
71 | kc.publishN(testTopicA, part0, requireQuorum = false, serverAckTimeout = 3.seconds, compress = None)(Chunk.seq(for { i <- 0 until 10} yield (ByteVector(i), ByteVector(idx))))
72 | } map (Left(_))
73 | }
74 |
75 | withKafkaClient(runtime, protocol) { kc =>
76 | publish(kc) ++
77 | (kc.subscribe(testTopicA, part0, offset(0l)) map (Right(_)))
78 | }.take(110).compile.toVector.unsafeRunTimed(30.seconds) shouldBe Some {
79 | (for { idx <- 0 until 10} yield Left(offset(idx*10))).toVector ++
80 | (for { idx <- 0 until 100} yield Right(TopicMessage(offset(idx), ByteVector(idx % 10), ByteVector(idx / 10), offset(100)))).toVector
81 | }
82 | }
83 |
84 |
85 | "publishN-unsafe-compressed-gzip" in {
86 | def publish(kc: KafkaClient[IO]) = {
87 | Stream.range(0, 10) evalMap { idx =>
88 | kc.publishUnsafeN(testTopicA, part0, compress = Some(Compression.GZIP))(Chunk.seq(for {i <- 0 until 10} yield (ByteVector(i), ByteVector(i*idx))))
89 | } drain
90 | }
91 |
92 | withKafkaClient(runtime, protocol) { kc =>
93 | publish(kc) ++
94 | Stream.sleep[IO](3.second) >> // wait for message to be accepted
95 | kc.subscribe(testTopicA, part0, offset(0l)).take(100)
96 | }.compile.toVector.unsafeRunTimed(30.seconds).map(_.size) shouldBe Some(100)
97 |
98 | }
99 |
100 |
101 | "publishN-response-compressed-gzip" in {
102 | def publish(kc: KafkaClient[IO]) = {
103 | Stream.range(0, 10) evalMap { idx =>
104 | kc.publishN(testTopicA, part0, requireQuorum = false, serverAckTimeout = 3.seconds, compress = Some(Compression.GZIP))(Chunk.seq(for { i <- 0 until 10 } yield (ByteVector(i), ByteVector(idx))))
105 | } map (Left(_))
106 | }
107 |
108 | withKafkaClient(runtime, protocol) { kc =>
109 | publish(kc) ++
110 | ((kc.subscribe(testTopicA, part0, offset(0l)) map (Right(_))) take 100)
111 | }.compile.toVector.unsafeRunTimed(30.seconds) shouldBe Some {
112 | (for { idx <- 0 until 10} yield Left(offset(idx*10))).toVector ++
113 | (for { idx <- 0 until 100} yield Right(TopicMessage(offset(idx), ByteVector(idx % 10), ByteVector(idx / 10), offset(100)))).toVector
114 | }
115 | }
116 |
117 |
118 | "publishN-response-compressed-gzip-not-aligned" in {
119 | def publish(kc: KafkaClient[IO]) = {
120 | Stream.range(0, 10) evalMap { idx =>
121 | kc.publishN(testTopicA, part0, requireQuorum = false, serverAckTimeout = 3.seconds, compress = Some(Compression.GZIP))(Chunk.seq(for { i <- 0 until 10 } yield (ByteVector(i), ByteVector(idx))))
122 | } map (Left(_))
123 | }
124 |
125 | withKafkaClient(runtime, protocol) { kc =>
126 | publish(kc) ++
127 | ((kc.subscribe(testTopicA, part0, offset(5l)) map (Right(_))) take 95)
128 | }.compile.toVector.unsafeRunTimed(30.seconds) shouldBe Some {
129 | (for { idx <- 0 until 10} yield Left(offset(idx*10))).toVector ++
130 | (for { idx <- 0 until 100} yield Right(TopicMessage(offset(idx), ByteVector(idx % 10), ByteVector(idx / 10), offset(100)))).drop(5).toVector
131 | }
132 | }
133 |
134 |
135 | "publishN-unsafe-compressed-snappy" in {
136 | def publish(kc: KafkaClient[IO]) = {
137 | Stream.range(0, 10) evalMap { idx =>
138 | kc.publishUnsafeN(testTopicA, part0, compress = Some(Compression.Snappy))(Chunk.seq(for {i <- 0 until 10} yield (ByteVector(i), ByteVector(i*idx))))
139 | } drain
140 | }
141 |
142 | withKafkaClient(runtime, protocol) { kc =>
143 | publish(kc) ++
144 | Stream.sleep[IO](3.second) >> // wait for message to be accepted
145 | kc.subscribe(testTopicA, part0, offset(0l)).take(100)
146 | }.compile.toVector.unsafeRunTimed(30.seconds).map(_.size) shouldBe Some(100)
147 |
148 | }
149 |
150 |
151 | "publishN-response-compressed-snappy" in {
152 | def publish(kc: KafkaClient[IO]) = {
153 | Stream.range(0, 10) evalMap { idx =>
154 | kc.publishN(testTopicA, part0, requireQuorum = false, serverAckTimeout = 3.seconds, compress = Some(Compression.Snappy))(Chunk.seq(for { i <- 0 until 10 } yield (ByteVector(i), ByteVector(idx))))
155 | } map (Left(_))
156 | }
157 |
158 | withKafkaClient(runtime, protocol) { kc =>
159 | publish(kc) ++
160 | ((kc.subscribe(testTopicA, part0, offset(0l)) map (Right(_))) take 100)
161 | }.compile.toVector.unsafeRunTimed(30.seconds) shouldBe Some {
162 | (for { idx <- 0 until 10} yield Left(offset(idx*10))).toVector ++
163 | (for { idx <- 0 until 100} yield Right(TopicMessage(offset(idx), ByteVector(idx % 10), ByteVector(idx / 10), offset(100)))).toVector
164 | }
165 | }
166 |
167 | }
168 |
169 |
170 |
171 | }
172 |
--------------------------------------------------------------------------------
/src/test/scala/spinoco/fs2/kafka/KafkaClientSubscribeSpec.scala:
--------------------------------------------------------------------------------
1 | package spinoco.fs2.kafka
2 |
3 | import cats.effect.IO
4 | import fs2._
5 |
6 | import scala.concurrent.duration._
7 |
8 |
9 |
10 | /**
11 | * Created by pach on 31/05/17.
12 | */
13 | class KafkaClientSubscribeSpec extends Fs2KafkaRuntimeSpec {
14 |
15 |
16 | s"single-broker" - {
17 |
18 | "subscribe-at-zero" in {
19 |
20 | withKafkaClient(runtime, protocol) { kc =>
21 | Stream.eval(publishNMessages(kc, 0, 20)) >>
22 | kc.subscribe(testTopicA, part0, offset(0l)).take(10)
23 | }.compile.toVector.unsafeRunTimed(30.seconds) shouldBe Some(generateTopicMessages(0, 10, 20))
24 | }
25 |
26 |
27 | "subscribe-at-zero-empty" in {
28 | withKafkaClient(runtime, protocol) { kc =>
29 | Stream[IO, Stream[IO, TopicMessage]](
30 | kc.subscribe(testTopicA, part0, offset(0l))
31 | , Stream.sleep_[IO](1.second) ++ Stream.eval_(publishNMessages(kc, 0, 20))
32 | ).parJoinUnbounded.take(10)
33 | }.compile.toVector.unsafeRunTimed(30.seconds).map { _.map { _.copy(tail = offset(0)) } } shouldBe Some(generateTopicMessages(0, 10, 0))
34 |
35 | }
36 |
37 | "subscriber before head" in {
38 | withKafkaClient(runtime, protocol) { kc =>
39 | Stream[IO, Stream[IO, TopicMessage]](
40 | kc.subscribe(testTopicA, part0, offset(-1l))
41 | , Stream.sleep_[IO](1.second) ++ Stream.eval_(publishNMessages(kc, 0, 20))
42 | ).parJoinUnbounded.take(10)
43 | }.compile.toVector.unsafeRunTimed(30.seconds).map { _.map { _.copy(tail = offset(0)) } } shouldBe Some(generateTopicMessages(0, 10, 0))
44 | }
45 |
46 | "subscriber after head" in {
47 | withKafkaClient(runtime, protocol) { kc =>
48 | Stream[IO, Stream[IO, TopicMessage]](
49 | Stream.eval_(publishNMessages(kc, 0, 20)) ++ kc.subscribe(testTopicA, part0, TailOffset)
50 | , Stream.sleep_[IO](1.second) ++ Stream.eval_(publishNMessages(kc, 20, 40))
51 | ).parJoinUnbounded.take(10)
52 | }.compile.toVector.unsafeRunTimed(30.seconds).map { _.map { _.copy(tail = offset(0)) }} shouldBe Some(generateTopicMessages(20, 30, 0))
53 |
54 | }
55 |
56 | }
57 |
58 |
59 |
60 |
61 | }
62 |
--------------------------------------------------------------------------------
/src/test/scala/spinoco/fs2/kafka/KafkaClusterPublish.scala:
--------------------------------------------------------------------------------
1 | package spinoco.fs2.kafka
2 |
3 | import cats.effect.IO
4 |
5 |
6 |
7 | import fs2._
8 | import scodec.bits.ByteVector
9 | import spinoco.protocol.kafka.ProtocolVersion
10 |
11 | import scala.concurrent.duration._
12 |
13 |
14 | /**
15 | * Created by pach on 06/06/17.
16 | */
17 | class KafkaClusterPublish extends Fs2KafkaRuntimeSpec {
18 |
19 |
20 |
21 | s"cluster" - {
22 |
23 |
24 | "publish-response" in skipFor(
25 | KafkaRuntimeRelease.V_0_9_0_1 -> ProtocolVersion.Kafka_0_8
26 | , KafkaRuntimeRelease.V_0_9_0_1 -> ProtocolVersion.Kafka_0_9
27 | ) {
28 | def publish(kc: KafkaClient[IO]) = {
29 | Stream.range(0, 10) evalMap { idx =>
30 | kc.publish1(testTopicA, part0, ByteVector(1), ByteVector(idx), requireQuorum = true, serverAckTimeout = 3.seconds)
31 | } map (Left(_))
32 | }
33 |
34 | withKafkaCluster(runtime).flatMap { nodes =>
35 | Stream.sleep[IO](3.second) >>
36 | Stream.eval(createKafkaTopic(nodes.broker1DockerId, testTopicA)) >> {
37 | KafkaClient[IO](Set(localBroker1_9092), protocol, "test-client") flatMap { kc =>
38 | awaitLeaderAvailable(kc, testTopicA, part0) >>
39 | publish(kc) ++
40 | (kc.subscribe(testTopicA, part0, offset(0l)) map (Right(_)))
41 | } take 20
42 | }
43 | }.compile.toVector.unsafeRunTimed(100.seconds) shouldBe Some(
44 | (for { idx <- 0 until 10} yield Left(offset(idx))).toVector ++
45 | (for { idx <- 0 until 10} yield Right(TopicMessage(offset(idx), ByteVector(1), ByteVector(idx), offset(10)))).toVector
46 | )
47 | }
48 |
49 |
50 | }
51 |
52 | }
53 |
--------------------------------------------------------------------------------
/src/test/scala/spinoco/fs2/kafka/KafkaClusterSubscribeSpec.scala:
--------------------------------------------------------------------------------
1 | package spinoco.fs2.kafka
2 |
3 | import cats.effect.IO
4 | import fs2._
5 | import spinoco.protocol.kafka.ProtocolVersion
6 |
7 | import scala.concurrent.duration._
8 |
9 |
10 |
11 | /**
12 | * Created by pach on 06/06/17.
13 | */
14 | class KafkaClusterSubscribeSpec extends Fs2KafkaRuntimeSpec {
15 |
16 | s"cluster" - {
17 |
18 | "subscribe-at-zero" in skipFor(
19 | KafkaRuntimeRelease.V_0_9_0_1 -> ProtocolVersion.Kafka_0_8
20 | , KafkaRuntimeRelease.V_0_9_0_1 -> ProtocolVersion.Kafka_0_9
21 | ) {
22 | withKafkaCluster(runtime).flatMap { nodes =>
23 |
24 | Stream.eval(createKafkaTopic(nodes.broker1DockerId, testTopicA, replicas = 3)) >>
25 | KafkaClient[IO](localCluster, protocol, "test-client").flatMap { kc =>
26 | awaitLeaderAvailable(kc, testTopicA, part0) >>
27 | Stream.eval(publishNMessages(kc, 0, 20, quorum = true)) >>
28 | kc.subscribe(testTopicA, part0, HeadOffset)
29 | }.take(10)
30 | }.compile.toVector.unsafeRunTimed(180.seconds) shouldBe Some(generateTopicMessages(0, 10, 20))
31 |
32 | }
33 |
34 | "subscribe-at-tail" in skipFor(
35 | KafkaRuntimeRelease.V_0_9_0_1 -> ProtocolVersion.Kafka_0_8
36 | , KafkaRuntimeRelease.V_0_9_0_1 -> ProtocolVersion.Kafka_0_9
37 | ) {
38 |
39 | withKafkaCluster(runtime).flatMap { nodes =>
40 |
41 | Stream.eval(createKafkaTopic(nodes.broker1DockerId, testTopicA, replicas = 3)) >>
42 | KafkaClient[IO](localCluster, protocol, "test-client").flatMap { kc =>
43 | awaitLeaderAvailable(kc, testTopicA, part0) >>
44 | Stream.eval(publishNMessages(kc, 0, 20, quorum = true)) >>
45 | Stream(
46 | kc.subscribe(testTopicA, part0, TailOffset)
47 | , Stream.sleep_[IO](3.second) ++ Stream.eval_(publishNMessages(kc, 20, 30, quorum = true))
48 | ).parJoinUnbounded
49 | }.take(10)
50 | }.compile.toVector.unsafeRunTimed(180.seconds).map { _.map { _.copy(tail = offset(30)) } } shouldBe Some(generateTopicMessages(20, 30, 30))
51 | }
52 |
53 | "recovers from leader-failure" in skipFor(
54 | KafkaRuntimeRelease.V_0_9_0_1 -> ProtocolVersion.Kafka_0_8
55 | , KafkaRuntimeRelease.V_0_9_0_1 -> ProtocolVersion.Kafka_0_9
56 | ) {
57 |
58 | withKafkaCluster(runtime).flatMap { nodes =>
59 | Stream.eval(createKafkaTopic(nodes.broker1DockerId, testTopicA, replicas = 3)) >>
60 | KafkaClient[IO](localCluster, protocol, "test-client").flatMap { kc =>
61 | awaitLeaderAvailable(kc, testTopicA, part0) flatMap { leader =>
62 | Stream.eval(publishNMessages(kc, 0, 20, quorum = true)) >>
63 | Stream(
64 | kc.subscribe(testTopicA, part0, HeadOffset)
65 | , Stream.sleep[IO](5.seconds) >>
66 | killLeader(kc, nodes, testTopicA, part0)
67 |
68 | , Stream.sleep[IO](10.seconds) >>
69 | awaitNewLeaderAvailable(kc, testTopicA, part0, leader) >>
70 | Stream.sleep[IO](3.seconds) >>
71 | Stream.eval_(publishNMessages(kc, 20, 30, quorum = true))
72 | ).parJoinUnbounded
73 | }}.take(30)
74 | }.compile.toVector.unsafeRunTimed(240.seconds).map { _.map { _.copy(tail = offset(30)) } } shouldBe Some(generateTopicMessages(0, 30, 30))
75 |
76 | }
77 | }
78 | }
79 |
--------------------------------------------------------------------------------
/src/test/scala/spinoco/fs2/kafka/network/BrokerConnection08SPec.scala:
--------------------------------------------------------------------------------
1 | package spinoco.fs2.kafka.network
2 |
3 | import java.net.InetSocketAddress
4 | import java.util.Date
5 |
6 | import cats.effect.IO
7 | import fs2._
8 | import scodec.bits.ByteVector
9 | import shapeless.tag
10 | import spinoco.protocol.kafka.Message.SingleMessage
11 | import spinoco.protocol.kafka.Request._
12 | import spinoco.protocol.kafka.Response._
13 | import spinoco.protocol.kafka._
14 | import spinoco.fs2.kafka.{KafkaRuntimeRelease, partition}
15 |
16 | import scala.concurrent.duration._
17 |
18 | /**
19 | * Specification to verify behaviour with Kafka 0.8
20 | * Note that this required local docker instance to work properly
21 | */
22 | class BrokerConnection08SPec extends BrokerConnectionKafkaSpecBase {
23 | "Kafka 0.8.2" - {
24 | "Publish and subscribe message" in {
25 | val result =
26 | withKafkaSingleton(KafkaRuntimeRelease.V_8_2_0) { case (zkId, kafkaId) =>
27 | val createTopic = Stream.eval_(createKafkaTopic(kafkaId,testTopic1))
28 | val publishOne = (Stream(
29 | RequestMessage(
30 | version = ProtocolVersion.Kafka_0_8
31 | , correlationId = 1
32 | , clientId = "test-publisher"
33 | , request = ProduceRequest(
34 | requiredAcks = RequiredAcks.LocalOnly
35 | , timeout = 10.seconds
36 | , messages = Vector((testTopic1, Vector((part0, Vector(SingleMessage(0l, MessageVersion.V0, None, ByteVector(1,2,3), ByteVector(5,6,7)))))))
37 | )
38 | )
39 | ) ++ Stream.sleep_[IO](1.minute))
40 | .through(BrokerConnection(new InetSocketAddress("127.0.0.1",9092)))
41 | .take(1).map(Left(_))
42 |
43 | val fetchOne =
44 | (Stream(RequestMessage(
45 | version = ProtocolVersion.Kafka_0_8
46 | , correlationId = 2
47 | , clientId = "test-subscriber"
48 | , request = FetchRequest(
49 | replica = tag[Broker](-1)
50 | , maxWaitTime = 1.second
51 | , minBytes = 1
52 | , maxBytes = None
53 | , topics = Vector((testTopic1, Vector((part0, tag[Offset](0), 10240))))
54 | )
55 | )) ++ Stream.sleep_[IO](1.minute))
56 | .through(BrokerConnection(new InetSocketAddress("127.0.0.1",9092)))
57 | .take(1).map(Right(_))
58 |
59 |
60 | createTopic ++ publishOne ++ fetchOne
61 | }.compile.toVector.unsafeRunSync()
62 |
63 |
64 | result shouldBe Vector(
65 | Left(ResponseMessage(1, ProduceResponse(Vector((testTopic1,Vector((part0,PartitionProduceResult(None,tag[Offset](0),None))))), throttleTime = None)))
66 | , Right(ResponseMessage(2,FetchResponse(Vector((testTopic1, Vector(PartitionFetchResult(part0, None, tag[Offset](1), Vector(SingleMessage(0,MessageVersion.V0, None, ByteVector(1,2,3), ByteVector(5,6,7))))))), throttleTime = None)))
67 | )
68 | }
69 |
70 | "Fetch metadata for topics" in {
71 | val result =
72 | withKafkaSingleton(KafkaRuntimeRelease.V_8_2_0) { case (zkId, kafkaId) =>
73 | val createTopic1 = Stream.eval_(createKafkaTopic(kafkaId, testTopic1))
74 | val createTopic2 = Stream.eval_(createKafkaTopic(kafkaId, testTopic2))
75 |
76 | val fetchMeta =
77 | (Stream(RequestMessage(
78 | version = ProtocolVersion.Kafka_0_8
79 | , correlationId = 1
80 | , clientId = "test-subscriber"
81 | , request = MetadataRequest(Vector())
82 | )) ++ Stream.sleep_[IO](1.minute))
83 | .through(BrokerConnection(new InetSocketAddress("127.0.0.1",9092)))
84 | .take(1)
85 |
86 | createTopic1 ++ createTopic2 ++ fetchMeta
87 |
88 | }.compile.toVector.unsafeRunSync()
89 |
90 | val metaResponse = result.collect { case ResponseMessage(1, meta:MetadataResponse) => meta }
91 |
92 | metaResponse.size shouldBe 1
93 | metaResponse.flatMap(_.brokers).size shouldBe 1
94 | metaResponse.flatMap(_.topics).size shouldBe 2
95 | }
96 |
97 |
98 | "Fetch offsets topics" in {
99 | val result =
100 | withKafkaSingleton(KafkaRuntimeRelease.V_8_2_0) { case (zkId, kafkaId) =>
101 | val createTopic1 = Stream.eval_(createKafkaTopic(kafkaId, testTopic1))
102 |
103 | val fetchOffsets=
104 | (Stream(RequestMessage(
105 | version = ProtocolVersion.Kafka_0_8
106 | , correlationId = 1
107 | , clientId = "test-subscriber"
108 | , request = OffsetsRequest(tag[Broker](-1), Vector((testTopic1, Vector((partition(0), new Date(-1), Some(10))))))
109 | )) ++ Stream.sleep_[IO](1.minute))
110 | .through(BrokerConnection(new InetSocketAddress("127.0.0.1",9092)))
111 | .take(1)
112 |
113 | createTopic1 ++ fetchOffsets
114 |
115 | }.compile.toVector.unsafeRunSync()
116 |
117 | val offsetResponse = result.collect { case ResponseMessage(1, offset:OffsetResponse) => offset }
118 |
119 | offsetResponse.size shouldBe 1
120 | offsetResponse.flatMap(_.data) shouldBe Vector(
121 | (testTopic1, Vector(PartitionOffsetResponse(partition(0), None, new Date(0), Vector(tag[Offset](0)))))
122 | )
123 | }
124 |
125 |
126 |
127 | }
128 |
129 |
130 |
131 |
132 | }
133 |
--------------------------------------------------------------------------------
/src/test/scala/spinoco/fs2/kafka/network/BrokerConnection09Spec.scala:
--------------------------------------------------------------------------------
1 | package spinoco.fs2.kafka.network
2 |
3 | import java.net.InetSocketAddress
4 | import java.util.Date
5 |
6 | import cats.effect.IO
7 | import fs2._
8 | import scodec.bits.ByteVector
9 | import shapeless.tag
10 | import spinoco.fs2.kafka.{KafkaRuntimeRelease, partition}
11 | import spinoco.protocol.kafka.Message.SingleMessage
12 | import spinoco.protocol.kafka.Request._
13 | import spinoco.protocol.kafka.Response._
14 | import spinoco.protocol.kafka._
15 |
16 | import scala.concurrent.duration._
17 |
18 | /**
19 | * Created by pach on 11/09/16.
20 | */
21 | class BrokerConnection09Spec extends BrokerConnectionKafkaSpecBase {
22 | "Kafka 0.9.0" - {
23 | "Publish and subscribe message" in {
24 | val result =
25 | withKafkaSingleton(KafkaRuntimeRelease.V_0_9_0_1) { case (zkId, kafkaId) =>
26 | val createTopic = Stream.eval_(createKafkaTopic(kafkaId, testTopic1))
27 | val publishOne = (Stream(
28 | RequestMessage(
29 | version = ProtocolVersion.Kafka_0_9
30 | , correlationId = 1
31 | , clientId = "test-publisher"
32 | , request = ProduceRequest(
33 | requiredAcks = RequiredAcks.LocalOnly
34 | , timeout = 10.seconds
35 | , messages = Vector((testTopic1, Vector((part0, Vector(SingleMessage(0l, MessageVersion.V0, None, ByteVector(1, 2, 3), ByteVector(5, 6, 7)))))))
36 | )
37 | )
38 | ) ++ Stream.sleep_[IO](1.minute))
39 | .through(BrokerConnection(new InetSocketAddress("127.0.0.1", 9092)))
40 | .take(1).map(Left(_))
41 |
42 | val fetchOne =
43 | (Stream(RequestMessage(
44 | version = ProtocolVersion.Kafka_0_9
45 | , correlationId = 2
46 | , clientId = "test-subscriber"
47 | , request = FetchRequest(
48 | replica = tag[Broker](-1)
49 | , maxWaitTime = 1.second
50 | , minBytes = 1
51 | , maxBytes = None
52 | , topics = Vector((testTopic1, Vector((part0, tag[Offset](0), 10240))))
53 | )
54 | )) ++ Stream.sleep_[IO](1.minute))
55 | .through(BrokerConnection(new InetSocketAddress("127.0.0.1", 9092)))
56 | .take(1).map(Right(_))
57 |
58 |
59 | createTopic ++ publishOne ++ fetchOne
60 | }.compile.toVector.unsafeRunSync()
61 |
62 | result shouldBe Vector(
63 | Left(ResponseMessage(1, ProduceResponse(Vector((testTopic1, Vector((part0, PartitionProduceResult(None, tag[Offset](0), None))))), throttleTime = Some(0.millis))))
64 | , Right(ResponseMessage(2, FetchResponse(Vector((testTopic1, Vector(PartitionFetchResult(part0, None, tag[Offset](1), Vector(SingleMessage(0, MessageVersion.V0, None, ByteVector(1, 2, 3), ByteVector(5, 6, 7))))))), throttleTime = Some(0.millis))))
65 | )
66 |
67 | }
68 |
69 |
70 | "Fetch metadata for topics" in {
71 | val result =
72 | withKafkaSingleton(KafkaRuntimeRelease.V_0_9_0_1) { case (zkId, kafkaId) =>
73 | val createTopic1 = Stream.eval_(createKafkaTopic(kafkaId, testTopic1))
74 | val createTopic2 = Stream.eval_(createKafkaTopic(kafkaId, testTopic2))
75 |
76 | val fetchMeta =
77 | (Stream(RequestMessage(
78 | version = ProtocolVersion.Kafka_0_8
79 | , correlationId = 1
80 | , clientId = "test-subscriber"
81 | , request = MetadataRequest(Vector())
82 | )) ++ Stream.sleep_[IO](1.minute))
83 | .through(BrokerConnection(new InetSocketAddress("127.0.0.1",9092)))
84 | .take(1)
85 |
86 | createTopic1 ++ createTopic2 ++ fetchMeta
87 |
88 | }.compile.toVector.unsafeRunSync()
89 |
90 | val metaResponse = result.collect { case ResponseMessage(1, meta:MetadataResponse) => meta }
91 |
92 | metaResponse.size shouldBe 1
93 | metaResponse.flatMap(_.brokers).size shouldBe 1
94 | metaResponse.flatMap(_.topics).size shouldBe 2
95 | }
96 |
97 |
98 | "Fetch offsets topics" in {
99 | val result =
100 | withKafkaSingleton(KafkaRuntimeRelease.V_0_9_0_1) { case (zkId, kafkaId) =>
101 | val createTopic1 = Stream.eval_(createKafkaTopic(kafkaId, testTopic1))
102 |
103 | val fetchOffsets=
104 | (Stream(RequestMessage(
105 | version = ProtocolVersion.Kafka_0_8
106 | , correlationId = 1
107 | , clientId = "test-subscriber"
108 | , request = OffsetsRequest(tag[Broker](-1), Vector((testTopic1, Vector((partition(0), new Date(-1), Some(10))))))
109 | )) ++ Stream.sleep_[IO](1.minute))
110 | .through(BrokerConnection(new InetSocketAddress("127.0.0.1",9092)))
111 | .take(1)
112 |
113 | createTopic1 ++ fetchOffsets
114 |
115 | }.compile.toVector.unsafeRunSync()
116 |
117 | val offsetResponse = result.collect { case ResponseMessage(1, offset:OffsetResponse) => offset }
118 |
119 | offsetResponse.size shouldBe 1
120 | offsetResponse.flatMap(_.data) shouldBe Vector(
121 | (testTopic1, Vector(PartitionOffsetResponse(partition(0), None, new Date(0), Vector(tag[Offset](0)))))
122 | )
123 | }
124 |
125 |
126 | }
127 | }
128 |
--------------------------------------------------------------------------------
/src/test/scala/spinoco/fs2/kafka/network/BrokerConnection10Spec.scala:
--------------------------------------------------------------------------------
1 | package spinoco.fs2.kafka.network
2 |
3 | import java.net.InetSocketAddress
4 | import java.util.Date
5 |
6 | import cats.effect.IO
7 | import fs2._
8 | import scodec.bits.ByteVector
9 | import shapeless.tag
10 | import spinoco.fs2.kafka.{KafkaRuntimeRelease, partition}
11 | import spinoco.protocol.kafka.Message.SingleMessage
12 | import spinoco.protocol.kafka.Request._
13 | import spinoco.protocol.kafka.Response._
14 | import spinoco.protocol.kafka._
15 |
16 | import scala.concurrent.duration._
17 |
18 | /**
19 | * Created by pach on 11/09/16.
20 | */
21 | class BrokerConnection10Spec extends BrokerConnectionKafkaSpecBase {
22 | "Kafka 0.10.0" - {
23 | "Publish and subscribe message" in {
24 | val result =
25 | withKafkaSingleton(KafkaRuntimeRelease.V_0_10_0) { case (zkId, kafkaId) =>
26 | val createTopic = Stream.eval_(createKafkaTopic(kafkaId, testTopic1))
27 | val publishOne = (Stream(
28 | RequestMessage(
29 | version = ProtocolVersion.Kafka_0_10
30 | , correlationId = 1
31 | , clientId = "test-publisher"
32 | , request = ProduceRequest(
33 | requiredAcks = RequiredAcks.LocalOnly
34 | , timeout = 10.seconds
35 | , messages = Vector((testTopic1, Vector((part0, Vector(SingleMessage(0l, MessageVersion.V0, None, ByteVector(1, 2, 3), ByteVector(5, 6, 7)))))))
36 | )
37 | )
38 | ) ++ Stream.sleep_[IO](1.minute))
39 | .through(BrokerConnection(new InetSocketAddress("127.0.0.1", 9092)))
40 | .take(1).map(Left(_))
41 |
42 | val fetchOne =
43 | (Stream(RequestMessage(
44 | version = ProtocolVersion.Kafka_0_10
45 | , correlationId = 2
46 | , clientId = "test-subscriber"
47 | , request = FetchRequest(
48 | replica = tag[Broker](-1)
49 | , maxWaitTime = 1.second
50 | , minBytes = 1
51 | , maxBytes = None
52 | , topics = Vector((testTopic1, Vector((part0, tag[Offset](0), 10240))))
53 | )
54 | )) ++ Stream.sleep_[IO](1.minute))
55 | .through(BrokerConnection(new InetSocketAddress("127.0.0.1", 9092)))
56 | .take(1).map(Right(_))
57 |
58 |
59 | createTopic ++ publishOne ++ fetchOne
60 | }.compile.toVector.unsafeRunSync()
61 |
62 | result shouldBe Vector(
63 | Left(ResponseMessage(1, ProduceResponse(Vector((testTopic1, Vector((part0, PartitionProduceResult(None, tag[Offset](0), None))))), throttleTime = Some(0.millis))))
64 | , Right(ResponseMessage(2, FetchResponse(Vector((testTopic1, Vector(PartitionFetchResult(part0, None, tag[Offset](1), Vector(SingleMessage(0, MessageVersion.V1, None, ByteVector(1, 2, 3), ByteVector(5, 6, 7))))))), throttleTime = Some(0.millis))))
65 | )
66 |
67 | }
68 |
69 |
70 | "Fetch metadata for topics" in {
71 | val result =
72 | withKafkaSingleton(KafkaRuntimeRelease.V_0_10_0) { case (zkId, kafkaId) =>
73 | val createTopic1 = Stream.eval_(createKafkaTopic(kafkaId, testTopic1))
74 | val createTopic2 = Stream.eval_(createKafkaTopic(kafkaId, testTopic2))
75 |
76 | val fetchMeta =
77 | (Stream(RequestMessage(
78 | version = ProtocolVersion.Kafka_0_8
79 | , correlationId = 1
80 | , clientId = "test-subscriber"
81 | , request = MetadataRequest(Vector())
82 | )) ++ Stream.sleep_[IO](1.minute))
83 | .through(BrokerConnection(new InetSocketAddress("127.0.0.1",9092)))
84 | .take(1)
85 |
86 | createTopic1 ++ createTopic2 ++ fetchMeta
87 |
88 | }.compile.toVector.unsafeRunSync()
89 |
90 | val metaResponse = result.collect { case ResponseMessage(1, meta:MetadataResponse) => meta }
91 |
92 | metaResponse.size shouldBe 1
93 | metaResponse.flatMap(_.brokers).size shouldBe 1
94 | metaResponse.flatMap(_.topics).size shouldBe 2
95 | }
96 |
97 |
98 | "Fetch offsets topics" in {
99 | val result =
100 | withKafkaSingleton(KafkaRuntimeRelease.V_0_10_0) { case (zkId, kafkaId) =>
101 | val createTopic1 = Stream.eval_(createKafkaTopic(kafkaId, testTopic1))
102 |
103 | val fetchOffsets=
104 | (Stream(RequestMessage(
105 | version = ProtocolVersion.Kafka_0_8
106 | , correlationId = 1
107 | , clientId = "test-subscriber"
108 | , request = OffsetsRequest(tag[Broker](-1), Vector((testTopic1, Vector((partition(0), new Date(-1), Some(10))))))
109 | )) ++ Stream.sleep_[IO](1.minute))
110 | .through(BrokerConnection(new InetSocketAddress("127.0.0.1",9092)))
111 | .take(1)
112 |
113 | createTopic1 ++ fetchOffsets
114 |
115 | }.compile.toVector.unsafeRunSync()
116 |
117 | val offsetResponse = result.collect { case ResponseMessage(1, offset:OffsetResponse) => offset }
118 |
119 | offsetResponse.size shouldBe 1
120 | offsetResponse.flatMap(_.data) shouldBe Vector(
121 | (testTopic1, Vector(PartitionOffsetResponse(partition(0), None, new Date(0), Vector(tag[Offset](0)))))
122 | )
123 | }
124 |
125 |
126 | }
127 | }
128 |
--------------------------------------------------------------------------------
/src/test/scala/spinoco/fs2/kafka/network/BrokerConnectionApp.scala:
--------------------------------------------------------------------------------
1 | package spinoco.fs2.kafka.network
2 |
3 | import java.net.InetSocketAddress
4 |
5 | import fs2._
6 | import cats.effect.IO
7 | import scodec.bits.ByteVector
8 | import shapeless.tag
9 | import spinoco.protocol.kafka.Request.RequiredAcks
10 | import spinoco.protocol.kafka._
11 |
12 | import scala.concurrent.duration._
13 |
14 | /**
15 | * Created by pach on 28/08/16.
16 | */
17 | object BrokerConnectionApp extends App {
18 |
19 | import spinoco.fs2.kafka.Fs2KafkaClientResources._
20 |
21 | def metadata = {
22 | val source =
23 | Stream[IO, RequestMessage](
24 | RequestMessage(
25 | version = ProtocolVersion.Kafka_0_8
26 | , correlationId = 1
27 | , clientId = "manager"
28 | , request = Request.MetadataRequest(Vector.empty)
29 | )
30 | ) ++ Stream.sleep_[IO](10.seconds)
31 |
32 | source.through(BrokerConnection(
33 | address = new InetSocketAddress("127.0.0.1", 9092)
34 | ))
35 | .evalMap(rcv => IO {
36 | println(rcv)
37 | })
38 | .compile.drain.unsafeRunSync()
39 | }
40 |
41 | def publish = {
42 | val source =
43 | Stream[IO, RequestMessage](
44 | RequestMessage(
45 | version = ProtocolVersion.Kafka_0_8
46 | , correlationId = 2
47 | , clientId = "publisher"
48 | , request = Request.ProduceRequest(
49 | requiredAcks = RequiredAcks.LocalOnly
50 | , timeout = 10.seconds
51 | , messages = Vector(
52 | (tag[TopicName]("test"), Vector(
53 | (tag[PartitionId](0), Vector(
54 | Message.SingleMessage(0l,MessageVersion.V0,None,ByteVector(1,2,3), ByteVector(5,6,7,8))
55 | ))
56 | ))
57 | )
58 | )
59 | )
60 | ) ++ Stream.sleep_[IO](10.seconds)
61 |
62 | source.through(BrokerConnection(
63 | address = new InetSocketAddress("127.0.0.1", 9092)
64 | ))
65 | .evalMap(rcv => IO {
66 | println(rcv)
67 | })
68 | .compile.drain.unsafeRunSync()
69 |
70 | }
71 |
72 | def fetch = {
73 | val source =
74 | Stream[IO, RequestMessage](
75 | RequestMessage(
76 | version = ProtocolVersion.Kafka_0_8
77 | , correlationId = 2
78 | , clientId = "fetcher"
79 | , request = Request.FetchRequest(
80 | replica = tag[Broker](-1)
81 | , maxWaitTime = 10.seconds
82 | , minBytes = 0
83 | , maxBytes = None
84 | , topics = Vector((tag[TopicName]("test"), Vector(
85 | (tag[PartitionId](0), tag[Offset](0), 1024*1024)
86 | )))
87 | )
88 | )
89 | ) ++ Stream.sleep_[IO](10.seconds)
90 |
91 | source.through(BrokerConnection(
92 | address = new InetSocketAddress("127.0.0.1", 9092)
93 | ))
94 | .evalMap(rcv => IO {
95 | println(rcv)
96 | })
97 | .compile.drain.unsafeRunSync()
98 | }
99 |
100 | //metadata
101 | //publish
102 | fetch
103 | }
104 |
--------------------------------------------------------------------------------
/src/test/scala/spinoco/fs2/kafka/network/BrokerConnectionKafkaSpecBase.scala:
--------------------------------------------------------------------------------
1 | package spinoco.fs2.kafka.network
2 |
3 | import spinoco.fs2.kafka.Fs2KafkaRuntimeSpec
4 | import shapeless.tag
5 | import spinoco.protocol.kafka.TopicName
6 |
7 | /**
8 | * Created by pach on 03/09/16.
9 | */
10 | class BrokerConnectionKafkaSpecBase extends Fs2KafkaRuntimeSpec {
11 |
12 | val testTopic1 = tag[TopicName]("test-topic-1")
13 | val testTopic2 = tag[TopicName]("test-topic-2")
14 |
15 | }
16 |
--------------------------------------------------------------------------------
/src/test/scala/spinoco/fs2/kafka/network/BrokerConnectionSpec.scala:
--------------------------------------------------------------------------------
1 | package spinoco.fs2.kafka.network
2 |
3 | import cats.effect.IO
4 | import cats.effect.concurrent.Ref
5 |
6 | import spinoco.fs2.kafka.Fs2KafkaClientSpec
7 | import fs2._
8 | import scodec.bits.ByteVector
9 | import shapeless.tag
10 |
11 | import spinoco.protocol.kafka.Request.RequiredAcks
12 | import spinoco.protocol.kafka.Response.MetadataResponse
13 | import spinoco.protocol.kafka._
14 | import spinoco.protocol.kafka.codec.MessageCodec
15 | import scala.concurrent.duration._
16 |
17 | class BrokerConnectionSpec extends Fs2KafkaClientSpec {
18 | import BrokerConnection._
19 |
20 |
21 |
22 | val metaRequestMessage = RequestMessage(
23 | version = ProtocolVersion.Kafka_0_8
24 | , correlationId = 1
25 | , clientId = "client-1"
26 | , request = Request.MetadataRequest(Vector.empty)
27 | )
28 |
29 | val metaResponse = ResponseMessage(
30 | correlationId = 1
31 | , response = MetadataResponse(Vector.empty,Vector.empty)
32 | )
33 |
34 | val produceRequest = Request.ProduceRequest(
35 | requiredAcks = RequiredAcks.LocalOnly
36 | , timeout = 10.seconds
37 | , messages = Vector(
38 | (tag[TopicName]("test"), Vector(
39 | (tag[PartitionId](0), Vector(
40 | Message.SingleMessage(0l,MessageVersion.V0,None,ByteVector(1,2,3), ByteVector(5,6,7,8))
41 | ))
42 | ))
43 | )
44 | )
45 |
46 | val produceRequestMessage = RequestMessage(
47 | version = ProtocolVersion.Kafka_0_8
48 | , correlationId = 1
49 | , clientId = "client-1"
50 | , request = produceRequest
51 | )
52 |
53 |
54 |
55 | "Sending of messages" - {
56 |
57 | "will send and register MetadataRequest" in {
58 | var send:Vector[ByteVector] = Vector.empty
59 | val ref = Ref.of[IO, Map[Int, RequestMessage]](Map.empty).unsafeRunSync()
60 | Stream(
61 | metaRequestMessage
62 | ).covary[IO].through(impl.sendMessages[IO](
63 | openRequests = ref
64 | , sendOne = { (chunk:Chunk[Byte]) => IO{ val bs = chunk.toBytes; send = send :+ ByteVector.view(bs.values).drop(bs.offset).take(bs.size) }}
65 | )).compile.drain.unsafeRunSync()
66 |
67 | send.size shouldBe 1
68 | ref.get.unsafeRunSync().get(metaRequestMessage.correlationId) shouldBe Some(metaRequestMessage)
69 | }
70 |
71 | "will send and register Produce Request " in {
72 | var send:Vector[ByteVector] = Vector.empty
73 | val ref = Ref.of[IO, Map[Int,RequestMessage]](Map.empty).unsafeRunSync()
74 | Stream(
75 | produceRequestMessage
76 | ).covary[IO].through(impl.sendMessages[IO](
77 | openRequests = ref
78 | , sendOne = { (chunk:Chunk[Byte]) => IO{ val bs = chunk.toBytes; send = send :+ ByteVector.view(bs.values).drop(bs.offset).take(bs.size) }}
79 | )).compile.drain.unsafeRunSync()
80 |
81 | send.size shouldBe 1
82 | ref.get.unsafeRunSync().get(produceRequestMessage.correlationId) shouldBe Some(produceRequestMessage)
83 | }
84 |
85 | "will send Produce Request bot won't register if reply is not expected" in {
86 | var send:Vector[ByteVector] = Vector.empty
87 | val ref = Ref.of[IO, Map[Int,RequestMessage]](Map.empty).unsafeRunSync()
88 | Stream(
89 | produceRequestMessage.copy(request = produceRequest.copy(requiredAcks = RequiredAcks.NoResponse))
90 | ).covary[IO].through(impl.sendMessages[IO](
91 | openRequests = ref
92 | , sendOne = { (chunk:Chunk[Byte]) => IO{ val bs = chunk.toBytes; send = send :+ ByteVector.view(bs.values).drop(bs.offset).take(bs.size) }}
93 | )).compile.drain.unsafeRunSync()
94 |
95 | send.size shouldBe 1
96 | ref.get.unsafeRunSync().get(produceRequestMessage.correlationId) shouldBe None
97 | }
98 |
99 | }
100 |
101 |
102 | "Receiving of messages" - {
103 |
104 |
105 | "correctly chunks based on size of message" in forAll { (messages:Seq[Seq[Seq[Byte]]]) =>
106 | val source = messages.map { oneMsg =>
107 | val sizeOfMsg = oneMsg.map(_.size).sum
108 | val chunks = oneMsg.map(sb => Chunk.bytes(sb.toArray))
109 | chunks.foldLeft(Stream.chunk[IO, Byte](Chunk.bytes(ByteVector.fromInt(sizeOfMsg).toArray))) { case (s,next) => s ++ Stream.chunk(next) }
110 | }.foldLeft(Stream.empty:Stream[IO, Byte])(_ ++ _)
111 |
112 | val resultMsg = source.through(impl.receiveChunks).compile.toVector.unsafeRunSync()
113 | val expectedMsg = messages.map { oneMsg =>
114 | oneMsg.foldLeft(ByteVector.empty){ case (bv, n) => bv ++ ByteVector.view(n.toArray) }
115 | }
116 |
117 | resultMsg shouldBe expectedMsg.toVector
118 |
119 | }
120 |
121 | "correctly decodes received message (MetadataResponse)" in {
122 |
123 | val ref = Ref.of[IO, Map[Int,RequestMessage]](Map.empty).unsafeRunSync()
124 | ref.set(Map(1 -> metaRequestMessage)).unsafeRunSync()
125 |
126 | val bytes =
127 | MessageCodec.responseCodecFor(ProtocolVersion.Kafka_0_10,ApiKey.MetadataRequest).encode(metaResponse.response)
128 | .flatMap(rbv => MessageCodec.responseCorrelationCodec.encode(metaResponse.correlationId -> rbv))
129 | .getOrElse(fail("Encoding of response failed"))
130 |
131 | val result = Stream(bytes.bytes.drop(4)).covary[IO].through(impl.decodeReceived[IO](ref)).compile.toVector.unsafeRunSync()
132 |
133 | result shouldBe Vector(metaResponse)
134 | ref.get.unsafeRunSync() shouldBe Map()
135 | }
136 |
137 |
138 | }
139 |
140 | }
141 |
--------------------------------------------------------------------------------
/version.sbt:
--------------------------------------------------------------------------------
1 | version in ThisBuild := "0.4.1-SNAPSHOT"
2 |
--------------------------------------------------------------------------------