apoorvmittal10 commented on code in PR #16860: URL: https://github.com/apache/kafka/pull/16860#discussion_r1713881944
########## tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleShareConsumer.java: ########## @@ -0,0 +1,185 @@ +/* + * 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; + +import org.apache.kafka.clients.consumer.AcknowledgeType; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.KafkaShareConsumer; +import org.apache.kafka.clients.consumer.ShareConsumer; +import org.apache.kafka.common.MessageFormatter; +import org.apache.kafka.common.errors.AuthenticationException; +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.errors.WakeupException; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.common.utils.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.PrintStream; +import java.time.Duration; +import java.util.Collections; +import java.util.Iterator; +import java.util.Optional; +import java.util.concurrent.CountDownLatch; + + +/** + * Share Consumer that dumps messages to standard out. + */ +public class ConsoleShareConsumer { + + private static final Logger LOG = LoggerFactory.getLogger(ConsoleShareConsumer.class); + private static final CountDownLatch SHUTDOWN_LATCH = new CountDownLatch(1); + + static int messageCount = 0; + + public static void main(String[] args) throws Exception { + ConsoleShareConsumerOptions opts = new ConsoleShareConsumerOptions(args); + try { + run(opts); + } catch (AuthenticationException ae) { + LOG.error("Authentication failed: terminating consumer process", ae); + Exit.exit(1); + } catch (Throwable t) { + LOG.error("Unknown error when running consumer: ", t); + Exit.exit(1); + } + } + + public static void run(ConsoleShareConsumerOptions opts) { + messageCount = 0; + long timeoutMs = opts.timeoutMs() >= 0 ? opts.timeoutMs() : Long.MAX_VALUE; + + ShareConsumer<byte[], byte[]> consumer = new KafkaShareConsumer<>(opts.consumerProps(), new ByteArrayDeserializer(), new ByteArrayDeserializer()); + ConsumerWrapper consumerWrapper = new ConsumerWrapper(opts.topicArg(), consumer, timeoutMs); + + addShutdownHook(consumerWrapper); + + try { + process(opts.maxMessages(), opts.formatter(), consumerWrapper, System.out, opts.rejectMessageOnError(), opts.acknowledgeType()); + } finally { + consumerWrapper.cleanup(); + opts.formatter().close(); + reportRecordCount(); + + SHUTDOWN_LATCH.countDown(); + } + } + + static void addShutdownHook(ConsumerWrapper consumer) { + Exit.addShutdownHook("consumer-shutdown-hook", () -> { + try { + consumer.wakeup(); + SHUTDOWN_LATCH.await(); + } catch (Throwable t) { + LOG.error("Exception while running shutdown hook: ", t); + } + }); + } + + static void process(int maxMessages, MessageFormatter formatter, ConsumerWrapper consumer, PrintStream output, + boolean rejectMessageOnError, AcknowledgeType acknowledgeType) { + while (messageCount < maxMessages || maxMessages == -1) { + ConsumerRecord<byte[], byte[]> msg; + try { + msg = consumer.receive(); + } catch (WakeupException we) { + LOG.trace("Caught WakeupException because consumer is shutdown, ignore and terminate."); + // Consumer will be closed + return; + } catch (Throwable t) { + LOG.error("Error processing message, terminating consumer process: ", t); + // Consumer will be closed + return; + } + messageCount += 1; + try { + formatter.writeTo(new ConsumerRecord<>(msg.topic(), msg.partition(), msg.offset(), msg.timestamp(), msg.timestampType(), + 0, 0, msg.key(), msg.value(), msg.headers(), Optional.empty()), output); + consumer.acknowledge(msg, acknowledgeType); + } catch (Throwable t) { + if (rejectMessageOnError) { + LOG.error("Error processing message, skipping this message: ", t); + consumer.acknowledge(msg, AcknowledgeType.REJECT); + } else { + // Consumer will be closed + throw t; + } + } + if (checkErr(output)) { + // Consumer will be closed + return; + } + } + } + + static void reportRecordCount() { Review Comment: Can it be private, seems we never us it in tests? ########## tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleShareConsumer.java: ########## @@ -0,0 +1,185 @@ +/* + * 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; + +import org.apache.kafka.clients.consumer.AcknowledgeType; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.KafkaShareConsumer; +import org.apache.kafka.clients.consumer.ShareConsumer; +import org.apache.kafka.common.MessageFormatter; +import org.apache.kafka.common.errors.AuthenticationException; +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.errors.WakeupException; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.common.utils.Time; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.PrintStream; +import java.time.Duration; +import java.util.Collections; +import java.util.Iterator; +import java.util.Optional; +import java.util.concurrent.CountDownLatch; + + +/** + * Share Consumer that dumps messages to standard out. + */ +public class ConsoleShareConsumer { + + private static final Logger LOG = LoggerFactory.getLogger(ConsoleShareConsumer.class); + private static final CountDownLatch SHUTDOWN_LATCH = new CountDownLatch(1); + + static int messageCount = 0; + + public static void main(String[] args) throws Exception { + ConsoleShareConsumerOptions opts = new ConsoleShareConsumerOptions(args); + try { + run(opts); + } catch (AuthenticationException ae) { + LOG.error("Authentication failed: terminating consumer process", ae); + Exit.exit(1); + } catch (Throwable t) { + LOG.error("Unknown error when running consumer: ", t); + Exit.exit(1); + } + } + + public static void run(ConsoleShareConsumerOptions opts) { + messageCount = 0; + long timeoutMs = opts.timeoutMs() >= 0 ? opts.timeoutMs() : Long.MAX_VALUE; + + ShareConsumer<byte[], byte[]> consumer = new KafkaShareConsumer<>(opts.consumerProps(), new ByteArrayDeserializer(), new ByteArrayDeserializer()); + ConsumerWrapper consumerWrapper = new ConsumerWrapper(opts.topicArg(), consumer, timeoutMs); + + addShutdownHook(consumerWrapper); + + try { + process(opts.maxMessages(), opts.formatter(), consumerWrapper, System.out, opts.rejectMessageOnError(), opts.acknowledgeType()); + } finally { + consumerWrapper.cleanup(); + opts.formatter().close(); + reportRecordCount(); + + SHUTDOWN_LATCH.countDown(); + } + } + + static void addShutdownHook(ConsumerWrapper consumer) { + Exit.addShutdownHook("consumer-shutdown-hook", () -> { + try { + consumer.wakeup(); + SHUTDOWN_LATCH.await(); + } catch (Throwable t) { + LOG.error("Exception while running shutdown hook: ", t); + } + }); + } + + static void process(int maxMessages, MessageFormatter formatter, ConsumerWrapper consumer, PrintStream output, + boolean rejectMessageOnError, AcknowledgeType acknowledgeType) { + while (messageCount < maxMessages || maxMessages == -1) { + ConsumerRecord<byte[], byte[]> msg; + try { + msg = consumer.receive(); + } catch (WakeupException we) { + LOG.trace("Caught WakeupException because consumer is shutdown, ignore and terminate."); + // Consumer will be closed + return; + } catch (Throwable t) { + LOG.error("Error processing message, terminating consumer process: ", t); + // Consumer will be closed + return; + } + messageCount += 1; + try { + formatter.writeTo(new ConsumerRecord<>(msg.topic(), msg.partition(), msg.offset(), msg.timestamp(), msg.timestampType(), + 0, 0, msg.key(), msg.value(), msg.headers(), Optional.empty()), output); + consumer.acknowledge(msg, acknowledgeType); + } catch (Throwable t) { + if (rejectMessageOnError) { Review Comment: Can we have unit test for reject functionality? ########## tools/src/main/java/org/apache/kafka/tools/consumer/ConsoleShareConsumerOptions.java: ########## @@ -0,0 +1,268 @@ +/* + * 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; + +import joptsimple.OptionException; +import joptsimple.OptionSpec; +import org.apache.kafka.clients.consumer.AcknowledgeType; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.common.MessageFormatter; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.server.util.CommandDefaultOptions; +import org.apache.kafka.server.util.CommandLineUtils; + +import java.io.IOException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; + +public final class ConsoleShareConsumerOptions extends CommandDefaultOptions { + private final OptionSpec<String> messageFormatterOpt; + private final OptionSpec<String> messageFormatterConfigOpt; + private final OptionSpec<String> messageFormatterArgOpt; + private final OptionSpec<String> keyDeserializerOpt; + private final OptionSpec<String> valueDeserializerOpt; + private final OptionSpec<Integer> maxMessagesOpt; + private final OptionSpec<?> rejectMessageOnErrorOpt; + private final OptionSpec<?> rejectOpt; + private final OptionSpec<?> releaseOpt; + private final OptionSpec<String> topicOpt; + private final OptionSpec<Integer> timeoutMsOpt; + private final OptionSpec<String> bootstrapServerOpt; + private final OptionSpec<String> groupIdOpt; + private final Properties consumerProps; + private final MessageFormatter formatter; + + public ConsoleShareConsumerOptions(String[] args) throws IOException { + super(args); + topicOpt = parser.accepts("topic", "The topic to consume on.") + .withRequiredArg() + .describedAs("topic") + .ofType(String.class); + OptionSpec<String> consumerPropertyOpt = parser.accepts("consumer-property", "A mechanism to pass user-defined properties in the form key=value to the consumer.") + .withRequiredArg() + .describedAs("consumer_prop") + .ofType(String.class); + OptionSpec<String> consumerConfigOpt = parser.accepts("consumer.config", "Consumer config properties file. Note that " + consumerPropertyOpt + " takes precedence over this config.") + .withRequiredArg() + .describedAs("config file") + .ofType(String.class); + messageFormatterOpt = parser.accepts("formatter", "The name of a class to use for formatting Kafka messages for display.") + .withRequiredArg() + .describedAs("class") + .ofType(String.class) + .defaultsTo(DefaultMessageFormatter.class.getName()); + messageFormatterArgOpt = parser.accepts("property", + "The properties to initialize the message formatter. Default properties include: \n" + + " print.timestamp=true|false\n" + + " print.key=true|false\n" + + " print.offset=true|false\n" + + " print.delivery=true|false\n" + + " print.partition=true|false\n" + + " print.headers=true|false\n" + + " print.value=true|false\n" + + " key.separator=<key.separator>\n" + + " line.separator=<line.separator>\n" + + " headers.separator=<line.separator>\n" + + " null.literal=<null.literal>\n" + + " key.deserializer=<key.deserializer>\n" + + " value.deserializer=<value.deserializer>\n" + + " header.deserializer=<header.deserializer>\n" + + "\nUsers can also pass in customized properties for their formatter; more specifically, users can pass in properties keyed with 'key.deserializer.', 'value.deserializer.' and 'headers.deserializer.' prefixes to configure their deserializers.") + .withRequiredArg() + .describedAs("prop") + .ofType(String.class); + messageFormatterConfigOpt = parser.accepts("formatter-config", "Config properties file to initialize the message formatter. Note that " + messageFormatterArgOpt + " takes precedence over this config.") + .withRequiredArg() + .describedAs("config file") + .ofType(String.class); + maxMessagesOpt = parser.accepts("max-messages", "The maximum number of messages to consume before exiting. If not set, consumption is continual.") + .withRequiredArg() + .describedAs("num_messages") + .ofType(Integer.class); + timeoutMsOpt = parser.accepts("timeout-ms", "If specified, exit if no message is available for consumption for the specified interval.") + .withRequiredArg() + .describedAs("timeout_ms") + .ofType(Integer.class); + rejectOpt = parser.accepts("reject", "If specified, messages are rejected as they are consumed."); + releaseOpt = parser.accepts("release", "If specified, messages are released as they are consumed."); + rejectMessageOnErrorOpt = parser.accepts("reject-message-on-error", "If there is an error when processing a message, " + + "reject it instead of halt."); + bootstrapServerOpt = parser.accepts("bootstrap-server", "REQUIRED: The server(s) to connect to.") + .withRequiredArg() + .describedAs("server to connect to") + .ofType(String.class); + keyDeserializerOpt = parser.accepts("key-deserializer") + .withRequiredArg() + .describedAs("deserializer for key") + .ofType(String.class); + valueDeserializerOpt = parser.accepts("value-deserializer") + .withRequiredArg() + .describedAs("deserializer for values") + .ofType(String.class); + groupIdOpt = parser.accepts("group", "The share group id of the consumer.") + .withRequiredArg() + .describedAs("share group id") + .ofType(String.class); + + try { + options = parser.parse(args); + } catch (OptionException oe) { + CommandLineUtils.printUsageAndExit(parser, oe.getMessage()); + } + + CommandLineUtils.maybePrintHelpOrVersion(this, "This tool helps to read data from Kafka topics using share groups and outputs it to standard output."); + + checkRequiredArgs(); + + if (options.has(rejectOpt) && options.has(releaseOpt)) { + CommandLineUtils.printUsageAndExit(parser, "At most one of --reject and --release may be specified."); + } Review Comment: I don't find a test for this, can we please add. -- 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