├── .gitignore ├── README.md ├── build.sbt ├── project ├── build.properties └── plugins.sbt └── src └── main └── scala └── au └── com └── simplesteph └── kafka └── kafka0_11 └── demo ├── ExactlyOnceLowLevel.scala ├── HeadersProducerRecord.scala ├── IdempotentProducer.scala ├── KafkaAdminClientDemo.scala ├── TransactionalConsumer.scala └── TransactionalProducer.scala /.gitignore: -------------------------------------------------------------------------------- 1 | .idea/ 2 | target/ -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Kafka 0.11 Demo 2 | 3 | This repository contains code snippets in Scala, using the Java clients, that demonstrate how to use the Kafka 0.11 new APIs. 4 | The code is incomplete, and that's intended: the goal is to bring the focus on the new APIs only. 5 | 6 | 7 | For more information, you can find the Kafka 0.11 release notes here: https://archive.apache.org/dist/kafka/0.11.0.0/RELEASE_NOTES.html 8 | 9 | # In this Repository 10 | 11 | - Idempotent Producer 12 | - Transactional Producer 13 | - Transactional Consumer 14 | - Exactly Once Low Level 15 | - Kafka Admin Client 16 | - Headers on Producer Record (new message type) 17 | 18 | # And Kafka Streams? 19 | 20 | Kafka 0.11 has exactly once semantics support for Kafka Streams. To learn about it, I'd recommend my other repo: 21 | 22 | https://github.com/simplesteph/kafka-streams-course 23 | 24 | # Contributions 25 | 26 | Feel free to do PR if you see something missing, or if there's a new feature I've missed, or you feel like re-writing things in Java. It's always appreciated -------------------------------------------------------------------------------- /build.sbt: -------------------------------------------------------------------------------- 1 | name := "kafka-0.11-demo" 2 | 3 | version := "0.1" 4 | 5 | scalaVersion := "2.12.3" 6 | 7 | libraryDependencies ++= Seq( 8 | "org.apache.kafka" % "kafka-clients" % "0.11.0.0" 9 | ) -------------------------------------------------------------------------------- /project/build.properties: -------------------------------------------------------------------------------- 1 | sbt.version = 0.13.16 -------------------------------------------------------------------------------- /project/plugins.sbt: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/simplesteph/kafka-0.11-examples/2dc78a80b131b4a37f53814ffea963f688eeaa41/project/plugins.sbt -------------------------------------------------------------------------------- /src/main/scala/au/com/simplesteph/kafka/kafka0_11/demo/ExactlyOnceLowLevel.scala: -------------------------------------------------------------------------------- 1 | package au.com.simplesteph.kafka.kafka0_11.demo 2 | 3 | import java.util 4 | import java.util.Properties 5 | 6 | import org.apache.kafka.clients.consumer._ 7 | import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} 8 | import org.apache.kafka.common.{KafkaException, TopicPartition} 9 | import org.apache.kafka.common.errors.{AuthorizationException, OutOfOrderSequenceException, ProducerFencedException} 10 | import org.apache.kafka.common.requests.IsolationLevel 11 | import org.apache.kafka.common.serialization.StringSerializer 12 | 13 | 14 | object ExactlyOnceLowLevel { 15 | 16 | 17 | 18 | def main(args: Array[String]): Unit = { 19 | 20 | val producer = createProducer() 21 | val consumer = createConsumer() 22 | producer.initTransactions() 23 | 24 | consumer.subscribe(util.Arrays.asList("foo", "bar")) 25 | 26 | try { 27 | 28 | while (true) { 29 | 30 | // read 31 | val records = consumer.poll(100) 32 | val producerRecords: List[ProducerRecord[String, String]] = doComplicatedStuff(records) 33 | producer.beginTransaction() 34 | producerRecords.foreach(producer.send) 35 | producer.sendOffsetsToTransaction(findOutOffsets(records), "my-transactional-consumer-group") // a bit annoying here to reference group id rwice 36 | producer.commitTransaction() 37 | // EXACTLY ONCE! 38 | } 39 | 40 | } catch { 41 | case e@(_: ProducerFencedException | _: OutOfOrderSequenceException | _: AuthorizationException) => 42 | // We can't recover from these exceptions, so our only option is to close the producer and exit. 43 | producer.close() 44 | case e: KafkaException => 45 | // For all other exceptions, just abort the transaction and try again. 46 | producer.abortTransaction() 47 | } 48 | 49 | 50 | } 51 | 52 | def createProducer(): KafkaProducer[String, String] = { 53 | val producerProps = new Properties() 54 | producerProps.put("bootstrap.servers", "localhost:9092") 55 | producerProps.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "my-transactional-id") // this has to be set!!! (unique for each producer you're having) 56 | producerProps.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true") // has to be idempotent 57 | new KafkaProducer[String, String](producerProps, new StringSerializer, new StringSerializer) 58 | } 59 | 60 | def createConsumer(): KafkaConsumer[String, String] = { 61 | val consumerProps = new Properties() 62 | consumerProps.put("bootstrap.servers", "localhost:9092") 63 | consumerProps.put("group.id", "my-transactional-consumer-group") 64 | consumerProps.put("enable.auto.commit", "false") 65 | consumerProps.put("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer") 66 | consumerProps.put("value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer") 67 | 68 | consumerProps.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, IsolationLevel.READ_COMMITTED) // this has to be set!!! 69 | new KafkaConsumer[String, String](consumerProps) 70 | } 71 | 72 | // map, filter, reduce, aggregate, have fun 73 | def doComplicatedStuff(records: ConsumerRecords[String, String]): List[ProducerRecord[String, String]] = ??? 74 | 75 | // should be a helper method within kafka, but so far we have to implement it 76 | def findOutOffsets(records: ConsumerRecords[String, String]): util.Map[TopicPartition, OffsetAndMetadata] = ??? 77 | 78 | 79 | } 80 | -------------------------------------------------------------------------------- /src/main/scala/au/com/simplesteph/kafka/kafka0_11/demo/HeadersProducerRecord.scala: -------------------------------------------------------------------------------- 1 | package au.com.simplesteph.kafka.kafka0_11.demo 2 | 3 | import java.util 4 | 5 | import org.apache.kafka.clients.producer.ProducerRecord 6 | import org.apache.kafka.common.header.Header 7 | import org.apache.kafka.common.header.internals.RecordHeader 8 | 9 | import collection.JavaConverters._ 10 | 11 | object HeadersProducerRecord { 12 | def main(args: Array[String]): Unit = { 13 | 14 | // partition is null so we let kafka decide it 15 | // unfortunately no API without partition parameter (downside of Java overloads) 16 | val partition: Int = null 17 | 18 | // header can contain lots of good stuff! 19 | // origin / destination 20 | // timestamps along the way 21 | // unique id for auditability 22 | // routing information! 23 | // go crazy! 24 | val header: Header = new RecordHeader("key", "valueAsBytes".getBytes()) 25 | val headers: util.List[Header] = List(header).asJava 26 | val producerRecord = new ProducerRecord[String, String]("topic", partition, "key", "value", headers) 27 | 28 | // better to set interceptors to set headers (set headers post .send() call) 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /src/main/scala/au/com/simplesteph/kafka/kafka0_11/demo/IdempotentProducer.scala: -------------------------------------------------------------------------------- 1 | package au.com.simplesteph.kafka.kafka0_11.demo 2 | 3 | import java.util.Properties 4 | 5 | import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} 6 | 7 | object IdempotentProducer { 8 | def main(args: Array[String]): Unit = { 9 | val props = new Properties() 10 | props.put("bootstrap.servers", "localhost:9092") 11 | props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer") 12 | props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer") 13 | 14 | props.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true") 15 | props.put(ProducerConfig.RETRIES_CONFIG, 3) // this is now safe !!!! 16 | props.put(ProducerConfig.ACKS_CONFIG, "all") // this has to be all 17 | props.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, 1) // this has to be 1 18 | 19 | val kafkaProducer = new KafkaProducer[String, String](props) 20 | 21 | // will send a producer record without duplicates as introduced by RETRIES 22 | kafkaProducer.send(new ProducerRecord[String, String]("my-topic-without-duplicates", "my-key", "my-value")) 23 | // this will still succeed, and a duplicate is in Kafka, because we wanted it! 24 | kafkaProducer.send(new ProducerRecord[String, String]("my-topic-without-duplicates", "my-key", "my-value")) 25 | 26 | kafkaProducer.close() 27 | 28 | } 29 | 30 | } 31 | -------------------------------------------------------------------------------- /src/main/scala/au/com/simplesteph/kafka/kafka0_11/demo/KafkaAdminClientDemo.scala: -------------------------------------------------------------------------------- 1 | package au.com.simplesteph.kafka.kafka0_11.demo 2 | 3 | import java.util 4 | import java.util.Properties 5 | 6 | import org.apache.kafka.clients.admin._ 7 | import org.apache.kafka.common.{KafkaFuture, Node} 8 | import org.apache.kafka.common.acl.{AccessControlEntry, AclBinding, AclOperation, AclPermissionType} 9 | import org.apache.kafka.common.config.{ConfigResource, TopicConfig} 10 | import org.apache.kafka.common.resource.{Resource, ResourceType} 11 | 12 | import collection.JavaConverters._ 13 | 14 | 15 | object KafkaAdminClientDemo { 16 | def main(args: Array[String]): Unit = { 17 | 18 | val props = new Properties() 19 | props.setProperty(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092") 20 | 21 | val adminClient = AdminClient.create(props) 22 | 23 | // ACLs 24 | val newAcl = new AclBinding(new Resource(ResourceType.TOPIC, "my-secure-topic"), new AccessControlEntry("my-user", "*", AclOperation.WRITE, AclPermissionType.ALLOW)) 25 | adminClient.createAcls(List(newAcl).asJavaCollection) 26 | // similarly 27 | adminClient.deleteAcls(???) 28 | adminClient.describeAcls(???) 29 | 30 | // TOPICS 31 | val numPartitions = 6 32 | val replicationFactor = 3.toShort 33 | val newTopic = new NewTopic("new-topic-name", numPartitions, replicationFactor) 34 | val configs = Map(TopicConfig.CLEANUP_POLICY_CONFIG -> TopicConfig.CLEANUP_POLICY_COMPACT, 35 | TopicConfig.COMPRESSION_TYPE_CONFIG -> "gzip") 36 | // settings some configs 37 | newTopic.configs(configs.asJava) 38 | adminClient.createTopics(List(newTopic).asJavaCollection) 39 | // similarly 40 | adminClient.deleteTopics(topicNames, options) 41 | adminClient.describeTopics(topicNames, options) 42 | adminClient.listTopics(new ListTopicsOptions().timeoutMs(500).listInternal(true)) 43 | // describe topic configs 44 | adminClient.describeConfigs(List(new ConfigResource(ConfigResource.Type.TOPIC, TopicConfig.CLEANUP_POLICY_CONFIG)).asJavaCollection) 45 | adminClient.alterConfigs(???) 46 | 47 | 48 | // get Kafka configs -> make your topic respect your cluster defaults 49 | // get Kafka configs -> be crazy 50 | adminClient.describeConfigs(List(new ConfigResource(ConfigResource.Type.BROKER, "default.replication.factor")).asJavaCollection) 51 | adminClient.alterConfigs() 52 | 53 | 54 | // CLUSTER stuff 55 | val cluster = adminClient.describeCluster() 56 | val clusterId: KafkaFuture[String] = cluster.clusterId() 57 | val controller: KafkaFuture[Node] = cluster.controller() 58 | val nodes: KafkaFuture[util.Collection[Node]] = cluster.nodes() 59 | // nodes info 60 | Node.noNode().host() 61 | Node.noNode().id() 62 | Node.noNode().port() 63 | Node.noNode().rack() 64 | 65 | 66 | 67 | } 68 | } 69 | -------------------------------------------------------------------------------- /src/main/scala/au/com/simplesteph/kafka/kafka0_11/demo/TransactionalConsumer.scala: -------------------------------------------------------------------------------- 1 | package au.com.simplesteph.kafka.kafka0_11.demo 2 | 3 | import java.util.Properties 4 | 5 | import org.apache.kafka.clients.consumer.ConsumerConfig 6 | import org.apache.kafka.common.requests.IsolationLevel 7 | import org.apache.kafka.clients.consumer.KafkaConsumer 8 | import java.util 9 | 10 | object TransactionalConsumer { 11 | def main(args: Array[String]): Unit = { 12 | val props = new Properties() 13 | props.put("bootstrap.servers", "localhost:9092") 14 | props.put("group.id", "my-transactional-consumer-group") 15 | props.put("enable.auto.commit", "false") 16 | props.put("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer") 17 | props.put("value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer") 18 | 19 | 20 | props.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, IsolationLevel.READ_COMMITTED) // this has to be set!!! 21 | 22 | val consumer = new KafkaConsumer[_, _](props) 23 | consumer.subscribe(util.Arrays.asList("foo", "bar")) 24 | while (true) { 25 | val records = consumer.poll(100) 26 | for (record <- records) { 27 | ??? 28 | } 29 | if (???) { 30 | // commit offsets once in a while 31 | consumer.commitSync() 32 | } 33 | } 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /src/main/scala/au/com/simplesteph/kafka/kafka0_11/demo/TransactionalProducer.scala: -------------------------------------------------------------------------------- 1 | package au.com.simplesteph.kafka.kafka0_11.demo 2 | 3 | import java.util.Properties 4 | 5 | import org.apache.kafka.clients.producer.{KafkaProducer, Producer, ProducerConfig, ProducerRecord} 6 | import org.apache.kafka.common.KafkaException 7 | import org.apache.kafka.common.errors.AuthorizationException 8 | import org.apache.kafka.common.errors.OutOfOrderSequenceException 9 | import org.apache.kafka.common.errors.ProducerFencedException 10 | import org.apache.kafka.common.serialization.StringSerializer 11 | 12 | object TransactionalProducer { 13 | def main(args: Array[String]): Unit = { 14 | val props = new Properties() 15 | props.put("bootstrap.servers", "localhost:9092") 16 | props.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "my-transactional-id") // this has to be set!!! (unique for each producer you're having) 17 | props.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true") // has to be idempotent 18 | 19 | 20 | val producer = new KafkaProducer[String ,String](props, new StringSerializer, new StringSerializer) 21 | producer.initTransactions() 22 | 23 | try { 24 | producer.beginTransaction() 25 | for (i <- Range(0, 100)) { 26 | producer.send(new ProducerRecord[String, String]("my-transactional-topic", Integer.toString(i), Integer.toString(i))) 27 | producer.send(new ProducerRecord[String, String]("my-other-topic", Integer.toString(i), Integer.toString(i))) 28 | } 29 | producer.commitTransaction() 30 | } catch { 31 | case e@(_: ProducerFencedException | _: OutOfOrderSequenceException | _: AuthorizationException) => 32 | // We can't recover from these exceptions, so our only option is to close the producer and exit. 33 | producer.close() 34 | case e: KafkaException => 35 | // For all other exceptions, just abort the transaction and try again. 36 | producer.abortTransaction() 37 | } 38 | } 39 | } 40 | --------------------------------------------------------------------------------