chia7712 commented on code in PR #15766:
URL: https://github.com/apache/kafka/pull/15766#discussion_r1577874954


##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerRunnable.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.kafka.tools.consumer.group;
+
+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.Objects;
+import java.util.Optional;
+import java.util.Properties;
+
+import static java.util.Collections.singleton;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_PROTOCOL_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG;
+import static org.apache.kafka.common.GroupType.CONSUMER;
+
+public class ConsumerRunnable implements Runnable {
+    final String broker;
+    final String groupId;
+    final Optional<Properties> customPropsOpt;
+    final boolean syncCommit;
+    final String topic;
+    final String groupProtocol;
+    final String assignmentStrategy;
+    final Optional<String> remoteAssignor;
+    final Properties props = new Properties();
+    volatile boolean isShutdown = false;
+    KafkaConsumer<String, String> consumer;
+
+    boolean configured = false;
+
+    public ConsumerRunnable(String broker,
+                            String groupId,
+                            String groupProtocol,
+                            String topic,
+                            String assignmentStrategy,
+                            Optional<String> remoteAssignor,
+                            Optional<Properties> customPropsOpt,
+                            boolean syncCommit) {
+        this.broker = broker;
+        this.groupId = groupId;
+        this.customPropsOpt = customPropsOpt;
+        this.syncCommit = syncCommit;
+
+        this.topic = topic;
+        this.groupProtocol = groupProtocol;
+        this.assignmentStrategy = assignmentStrategy;
+        this.remoteAssignor = remoteAssignor;
+    }
+
+    void configure() {
+        configured = true;
+        configure(props);
+        customPropsOpt.ifPresent(props::putAll);
+        consumer = new KafkaConsumer<>(props);
+    }
+
+    void configure(Properties props) {
+        props.put("bootstrap.servers", broker);
+        props.put("group.id", groupId);
+        props.put("key.deserializer", StringDeserializer.class.getName());
+        props.put("value.deserializer", StringDeserializer.class.getName());
+        props.put(GROUP_PROTOCOL_CONFIG, groupProtocol);
+        if (Objects.equals(groupProtocol, CONSUMER.toString())) {
+            remoteAssignor.ifPresent(assignor -> 
props.put(GROUP_REMOTE_ASSIGNOR_CONFIG, assignor));
+        } else {
+            props.put("partition.assignment.strategy", assignmentStrategy);
+        }
+    }
+
+    void subscribe() {

Review Comment:
   We don't need this method as it is used by `run` only



##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerRunnable.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.kafka.tools.consumer.group;
+
+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.Objects;
+import java.util.Optional;
+import java.util.Properties;
+
+import static java.util.Collections.singleton;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_PROTOCOL_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG;
+import static org.apache.kafka.common.GroupType.CONSUMER;
+
+public class ConsumerRunnable implements Runnable {

Review Comment:
   This class is used by `ConsumerGroupExecutor` so we can move it to be a 
static class within `ConsumerGroupExecutor`



##########
tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerRunnable.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.kafka.tools.consumer.group;
+
+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.Objects;
+import java.util.Optional;
+import java.util.Properties;
+
+import static java.util.Collections.singleton;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_PROTOCOL_CONFIG;
+import static 
org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG;
+import static org.apache.kafka.common.GroupType.CONSUMER;
+
+public class ConsumerRunnable implements Runnable {
+    final String broker;
+    final String groupId;
+    final Optional<Properties> customPropsOpt;
+    final boolean syncCommit;
+    final String topic;
+    final String groupProtocol;
+    final String assignmentStrategy;
+    final Optional<String> remoteAssignor;
+    final Properties props = new Properties();
+    volatile boolean isShutdown = false;
+    KafkaConsumer<String, String> consumer;
+
+    boolean configured = false;
+
+    public ConsumerRunnable(String broker,
+                            String groupId,
+                            String groupProtocol,
+                            String topic,
+                            String assignmentStrategy,
+                            Optional<String> remoteAssignor,
+                            Optional<Properties> customPropsOpt,
+                            boolean syncCommit) {
+        this.broker = broker;
+        this.groupId = groupId;
+        this.customPropsOpt = customPropsOpt;
+        this.syncCommit = syncCommit;
+
+        this.topic = topic;
+        this.groupProtocol = groupProtocol;
+        this.assignmentStrategy = assignmentStrategy;
+        this.remoteAssignor = remoteAssignor;
+    }
+
+    void configure() {
+        configured = true;
+        configure(props);
+        customPropsOpt.ifPresent(props::putAll);
+        consumer = new KafkaConsumer<>(props);
+    }
+
+    void configure(Properties props) {
+        props.put("bootstrap.servers", broker);
+        props.put("group.id", groupId);
+        props.put("key.deserializer", StringDeserializer.class.getName());
+        props.put("value.deserializer", StringDeserializer.class.getName());
+        props.put(GROUP_PROTOCOL_CONFIG, groupProtocol);
+        if (Objects.equals(groupProtocol, CONSUMER.toString())) {
+            remoteAssignor.ifPresent(assignor -> 
props.put(GROUP_REMOTE_ASSIGNOR_CONFIG, assignor));
+        } else {
+            props.put("partition.assignment.strategy", assignmentStrategy);
+        }
+    }
+
+    void subscribe() {
+        consumer.subscribe(singleton(topic));
+    }
+
+    @Override
+    public void run() {
+        assert configured : "Must call configure before use";

Review Comment:
   We expect `configure` must be called so it would be nice to move it to the 
constructor. It means we should create the consumer in constructor.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to