]> juplo.de Git - demos/kafka/training/commitdiff
`simple-producer` in `simple-consumer` verwandelt -- ALIGN
authorKai Moritz <kai@juplo.de>
Sat, 14 Mar 2026 16:37:00 +0000 (17:37 +0100)
committerKai Moritz <kai@juplo.de>
Sun, 22 Mar 2026 20:09:32 +0000 (21:09 +0100)
Dockerfile
README.sh
docker/docker-compose.yml
pom.xml
settings.gradle
src/main/java/de/juplo/kafka/ExampleConsumer.java

index 74e66edfeced9a85a5719ba6f0ddc99ae7f87b50..22819afe542418593519a2e2b061286756263def 100644 (file)
@@ -3,4 +3,4 @@ VOLUME /tmp
 COPY target/*.jar /opt/app.jar
 COPY target/libs /opt/libs
 ENTRYPOINT [ "java", "-jar", "/opt/app.jar" ]
-CMD [ "kafka:9092", "test", "DCKR" ]
+CMD [ "kafka:9092", "test", "my-group", "DCKR" ]
index 3d98ace799653a8242fa2a3681e8936d7e3a3a78..85b8f960a0d7f927ffdc85ad0302ffcea271a54a 100755 (executable)
--- a/README.sh
+++ b/README.sh
@@ -1,6 +1,6 @@
 #!/bin/bash
 
-IMAGE=juplo/simple-producer:1.0-SNAPSHOT
+IMAGE=juplo/simple-consumer:1.0-SNAPSHOT
 
 if [ "$1" = "cleanup" ]
 then
@@ -10,7 +10,7 @@ then
 fi
 
 docker compose -f docker/docker-compose.yml up -d --remove-orphans kafka-1 kafka-2 kafka-3
-docker compose -f docker/docker-compose.yml rm -svf producer
+docker compose -f docker/docker-compose.yml rm -svf consumer
 
 if [[
   $(docker image ls -q $IMAGE) == "" ||
@@ -27,10 +27,13 @@ docker compose -f docker/docker-compose.yml up --remove-orphans setup || exit 1
 
 
 docker compose -f docker/docker-compose.yml up -d producer
+docker compose -f docker/docker-compose.yml up -d consumer
+
 sleep 5
+docker compose -f docker/docker-compose.yml stop consumer
 
-docker compose -f docker/docker-compose.yml exec cli kafkacat -b kafka:9092 -t test -c 20 -f'topic=%t\tpartition=%p\toffset=%o\tkey=%k\tvalue=%s\n'
+docker compose -f docker/docker-compose.yml start consumer
+sleep 5
 
-docker compose -f docker/docker-compose.yml stop producer
-docker compose -f docker/docker-compose.yml exec cli kafkacat -b kafka:9092 -t test -e -f'topic=%t\tpartition=%p\toffset=%o\tkey=%k\tvalue=%s\n'
-docker compose -f docker/docker-compose.yml logs producer
+docker compose -f docker/docker-compose.yml stop producer consumer
+docker compose -f docker/docker-compose.yml logs consumer
index 2ab5b6cdb6160be39004ce5504684bb2ca346b20..b1fbf6f973d3aa4fa1e2bcc91f681bfaba998b2f 100644 (file)
@@ -179,6 +179,10 @@ services:
     cpu_quota:  50000
     mem_limit:  100m
 
+  consumer:
+    image: juplo/simple-consumer:1.0-SNAPSHOT
+    command: kafka:9092 test my-group consumer
+
 volumes:
   controller-data:
   kafka-1-data:
diff --git a/pom.xml b/pom.xml
index 2a04e375eee86da46f03a64110650152d95fc1fa..9827f6883a00f50db43583672d6bd2bdf4f59cde 100644 (file)
--- a/pom.xml
+++ b/pom.xml
@@ -12,9 +12,9 @@
   </parent>
 
   <groupId>de.juplo.kafka</groupId>
-  <artifactId>simple-producer</artifactId>
-  <name>Super Simple Producer</name>
-  <description>A Simple Producer, programmed with pure Java, that sends messages via Kafka</description>
+  <artifactId>simple-consumer</artifactId>
+  <name>Simple Consumer-Group</name>
+  <description>Super Simple Consumer-Group, that is implemented as a plain Java-program</description>
   <version>1.0-SNAPSHOT</version>
 
   <properties>
@@ -63,7 +63,7 @@
             <manifest>
               <addClasspath>true</addClasspath>
               <classpathPrefix>libs/</classpathPrefix>
-              <mainClass>de.juplo.kafka.ExampleProducer</mainClass>
+              <mainClass>de.juplo.kafka.ExampleConsumer</mainClass>
             </manifest>
           </archive>
         </configuration>
index 5f64b227c60366b91f11f76cfebab3963f2098ea..71fb57379f24b760ecd98a0f15c68e27026e3829 100644 (file)
@@ -1 +1 @@
-rootProject.name = 'simple-producer'
+rootProject.name = 'simple-consumer'
index a76458890f62b9c25ccc571a77a4f4821fb6fa9f..534fb052819a44e94b79d4840e2b76e5f36a5a3d 100644 (file)
 package de.juplo.kafka;
 
 import lombok.extern.slf4j.Slf4j;
-import org.apache.kafka.clients.producer.Producer;
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.common.serialization.StringSerializer;
-
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.common.errors.WakeupException;
+import org.apache.kafka.common.serialization.StringDeserializer;
+
+import java.time.Duration;
+import java.util.Arrays;
 import java.util.Properties;
 
 
 @Slf4j
-public class ExampleProducer
+public class ExampleConsumer
 {
   private final String id;
   private final String topic;
-  private final Producer<String, String> producer;
+  private final Consumer<String, String> consumer;
 
-  private volatile boolean running = true;
-  private volatile boolean done = false;
-  private long produced = 0;
+  private volatile boolean running = false;
+  private long consumed = 0;
 
-  public ExampleProducer(
+  public ExampleConsumer(
     String broker,
     String topic,
+    String groupId,
     String clientId)
   {
     Properties props = new Properties();
     props.put("bootstrap.servers", broker);
+    props.put("group.id", groupId); // ID für die Offset-Commits
     props.put("client.id", clientId); // Nur zur Wiedererkennung
-    props.put("key.serializer", StringSerializer.class.getName());
-    props.put("value.serializer", StringSerializer.class.getName());
+    props.put("auto.offset.reset", "earliest"); // Von Beginn an lesen
+    props.put("partition.assignment.strategy", "org.apache.kafka.clients.consumer.CooperativeStickyAssignor");
+    props.put("key.deserializer", StringDeserializer.class.getName());
+    props.put("value.deserializer", StringDeserializer.class.getName());
 
     this.id = clientId;
     this.topic = topic;
-    producer = new KafkaProducer<>(props);
+    consumer = new KafkaConsumer<>(props);
   }
 
+
   public void run()
   {
-    long i = 0;
-
     try
     {
-      for (; running; i++)
+      log.info("{} - Subscribing to topic {}", id, topic);
+      consumer.subscribe(Arrays.asList(topic));
+      running = true;
+
+      while (true)
       {
-        send(Long.toString(i%10), Long.toString(i));
-        Thread.sleep(500);
+        ConsumerRecords<String, String> records = consumer.poll(Duration.ofSeconds(1));
+
+        log.info("{} - Received {} messages", id, records.count());
+        for (ConsumerRecord<String, String> record : records)
+        {
+          consumed++;
+          log.info(
+            "{} - {}: {}/{} - {}={}",
+            id,
+            record.offset(),
+            record.topic(),
+            record.partition(),
+            record.key(),
+            record.value()
+          );
+        }
       }
     }
-    catch (Exception e)
+    catch(WakeupException e)
     {
-      log.error("{} - Unexpected error!", id, e);
+      log.info("{} - Consumer was signaled to finish its work", id);
     }
-    finally
+    catch(Exception e)
     {
-      log.info("{}: Closing the KafkaProducer", id);
-      producer.close();
-      log.info("{}: Produced {} messages in total, exiting!", id, produced);
-      done = true;
+      log.error("{} - Unexpected error, unsubscribing!", id, e);
+      consumer.unsubscribe();
     }
-  }
-
-  void send(String key, String value)
-  {
-    final long sendRequested = System.currentTimeMillis();
-
-    final ProducerRecord<String, String> record = new ProducerRecord<>(
-      topic,  // Topic
-      key,    // Key
-      value   // Value
-    );
-
-    producer.send(record, (metadata, e) ->
+    finally
     {
-      long sendRequestProcessed = System.currentTimeMillis();
-      if (e == null)
-      {
-        // HANDLE SUCCESS
-        log.debug(
-          "{} - Sent message {}={}, partition={}, offset={}, timestamp={}, latency={}ms",
-          id,
-          key,
-          value,
-          metadata.partition(),
-          metadata.offset(),
-          metadata.timestamp(),
-          sendRequestProcessed - sendRequested
-        );
-      }
-      else
-      {
-        // HANDLE ERROR
-        log.error(
-          "{} - ERROR for message {}={}, latency={}ms: {}",
-          id,
-          key,
-          value,
-          sendRequestProcessed - sendRequested,
-          e.toString()
-        );
-      }
-    });
-
-    long sendRequestQueued = System.currentTimeMillis();
-    produced++;
-    log.trace(
-      "{} - Queued message {}={}, latency={}ms",
-      id,
-      key,
-      value,
-      sendRequestQueued - sendRequested
-    );
+      running = false;
+      log.info("{} - Closing the KafkaConsumer", id);
+      consumer.close();
+      log.info("{}: Consumed {} messages in total, exiting!", id, consumed);
+    }
   }
 
 
   public static void main(String[] args) throws Exception
   {
-    if (args.length != 3)
+    String broker = ":9092";
+    String topic = "test";
+    String groupId = "my-group";
+    String clientId = "DEV";
+
+    switch (args.length)
     {
-      log.error("Three arguments required!");
-      log.error("arg[0]: Broker-Address");
-      log.error("arg[1]: Topic");
-      log.error("arg[2]: Unique Client-ID");
-      System.exit(1);
-      return;
+      case 4:
+        clientId = args[3];
+      case 3:
+        groupId = args[2];
+      case 2:
+        topic = args[1];
+      case 1:
+        broker = args[0];
     }
 
-    log.info(
-      "Running ExampleProducer: broker={}, topic={}, client-id={}",
-      args[0],
-      args[1],
-      args[2]);
 
-    ExampleProducer instance = new ExampleProducer(args[0], args[1], args[2]);
+    ExampleConsumer instance = new ExampleConsumer(broker, topic, groupId, clientId);
 
     Runtime.getRuntime().addShutdownHook(new Thread(() ->
     {
-      instance.running = false;
-      while (!instance.done)
+      instance.consumer.wakeup();
+
+      while (instance.running)
       {
         log.info("{} - Waiting for main-thread...", instance.id);
         try
@@ -149,6 +129,12 @@ public class ExampleProducer
       log.info("{} - Shutdown completed.", instance.id);
     }));
 
+    log.info(
+      "Running ExampleConsumer: broker={}, topic={}, group-id={}, client-id={}",
+      broker,
+      topic,
+      groupId,
+      clientId);
     instance.run();
   }
 }