You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by da...@apache.org on 2019/03/24 09:09:48 UTC

[camel] branch master updated: CAMEL-13338 - Fix to use Kafka client KafkaConsumer.subscribe API with ConsumerRebalanceListener registration when topicIsPattern is set to false

This is an automated email from the ASF dual-hosted git repository.

davsclaus pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/camel.git


The following commit(s) were added to refs/heads/master by this push:
     new 54d6e4c  CAMEL-13338 - Fix to use Kafka client KafkaConsumer.subscribe API with ConsumerRebalanceListener registration when topicIsPattern is set to false
54d6e4c is described below

commit 54d6e4c7e464ddc19c2250fc7576934fadbf32ac
Author: Viswa Ramamoorthy <vi...@manh.com>
AuthorDate: Wed Mar 20 10:47:02 2019 -0400

    CAMEL-13338 - Fix to use Kafka client KafkaConsumer.subscribe API with ConsumerRebalanceListener registration when topicIsPattern is set to false
---
 .../camel/component/kafka/KafkaConsumer.java       |   2 +-
 .../kafka/KafkaConsumerRebalanceTest.java          | 102 +++++++++++++++++++++
 2 files changed, 103 insertions(+), 1 deletion(-)

diff --git a/components/camel-kafka/src/main/java/org/apache/camel/component/kafka/KafkaConsumer.java b/components/camel-kafka/src/main/java/org/apache/camel/component/kafka/KafkaConsumer.java
index 86872f5..ebb987c 100644
--- a/components/camel-kafka/src/main/java/org/apache/camel/component/kafka/KafkaConsumer.java
+++ b/components/camel-kafka/src/main/java/org/apache/camel/component/kafka/KafkaConsumer.java
@@ -242,7 +242,7 @@ public class KafkaConsumer extends DefaultConsumer {
                     consumer.subscribe(topicPattern, this);
                 } else {
                     log.info("Subscribing {} to topic {}", threadId, topicName);
-                    consumer.subscribe(Arrays.asList(topicName.split(",")));
+                    consumer.subscribe(Arrays.asList(topicName.split(",")), this);
                 }
 
                 StateRepository<String, String> offsetRepository = endpoint.getConfiguration().getOffsetRepository();
diff --git a/components/camel-kafka/src/test/java/org/apache/camel/component/kafka/KafkaConsumerRebalanceTest.java b/components/camel-kafka/src/test/java/org/apache/camel/component/kafka/KafkaConsumerRebalanceTest.java
new file mode 100644
index 0000000..b36b7c2
--- /dev/null
+++ b/components/camel-kafka/src/test/java/org/apache/camel/component/kafka/KafkaConsumerRebalanceTest.java
@@ -0,0 +1,102 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.component.kafka;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.camel.EndpointInject;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.mock.MockEndpoint;
+import org.apache.camel.impl.JndiRegistry;
+import org.apache.camel.spi.StateRepository;
+import org.junit.Test;
+
+public class KafkaConsumerRebalanceTest extends BaseEmbeddedKafkaTest {
+    private static final String TOPIC = "offset-rebalance";
+
+    @EndpointInject(uri = "mock:result")
+    private MockEndpoint result;
+
+    private OffsetStateRepository stateRepository;
+    private CountDownLatch messagesLatch;
+
+    @Override
+    protected void doPreSetup() throws Exception {
+        messagesLatch = new CountDownLatch(2);
+        stateRepository = new OffsetStateRepository(messagesLatch);
+    }
+
+    @Test
+    public void offsetGetStateMustHaveBeenCalledTwice() throws Exception {
+        boolean offsetGetStateCalled = messagesLatch.await(30000, TimeUnit.MILLISECONDS);
+        assertTrue("StateRepository.getState should have been called twice for topic " + TOPIC + 
+                ". Remaining count : " + messagesLatch.getCount(), offsetGetStateCalled);
+    }
+
+    @Override
+    protected JndiRegistry createRegistry() throws Exception {
+        JndiRegistry registry = super.createRegistry();
+        registry.bind("offset", stateRepository);
+        return registry;
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("kafka:" + TOPIC
+                             + "?groupId=" + TOPIC + "_GROUP"
+                             + "&autoCommitIntervalMs=1000"
+                             + "&autoOffsetReset=latest"
+                             + "&consumersCount=1"
+                             + "&offsetRepository=#offset")
+                        .routeId("consumer-rebalance-route")
+                        .to("mock:result");
+            }
+        };
+    }
+
+    public class OffsetStateRepository implements StateRepository<String, String> {
+        CountDownLatch messagesLatch = null;
+        
+        public OffsetStateRepository(CountDownLatch messagesLatch) {
+            this.messagesLatch = messagesLatch;
+        }
+
+        @Override
+        public void start() throws Exception {
+        }
+
+        @Override
+        public void stop() throws Exception {
+        }
+
+        @Override
+        public String getState(String key) {
+            if (key.contains(TOPIC)) {
+                messagesLatch.countDown();
+            }
+            return "-1";
+        }
+
+        @Override
+        public void setState(String key, String value) {
+        }
+    }
+}