KAFKA-14576: Move ConsoleConsumer to tools#15274
Conversation
OmniaGM
left a comment
There was a problem hiding this comment.
Nice work, overall looks good. I just left few comments
| } | ||
| } | ||
|
|
||
| static void run(ConsoleConsumerOptions conf) { |
There was a problem hiding this comment.
Just for consistency can we rename conf to optslike L.57?
|
|
||
| Iterator<ConsumerRecord<byte[], byte[]>> recordIter = Collections.emptyIterator(); | ||
|
|
||
| public ConsumerWrapper(Optional<String> topic, |
There was a problem hiding this comment.
Do we really need to wrap these parameters in Optional? Also, topic, partitionId, offset, includedTopics values all come from ConsoleConsumerOptions so maybe we can just send ConsoleConsumerOptions here and get ripped of the extra if/else in line 72 and the optional wrappers
There was a problem hiding this comment.
I opted to copy the previous logic to make it easier to review and avoid changing any behavior by mistake. I agree this could be refactored but I'd prefer doing that in a follow up PR.
| } else if (!topic.isPresent() && !partitionId.isPresent() && !offset.isPresent() && includedTopics.isPresent()) { | ||
| consumer.subscribe(Pattern.compile(includedTopics.get())); | ||
| } else { | ||
| throw new IllegalArgumentException("An invalid combination of arguments is provided. " + |
There was a problem hiding this comment.
this line feel more fit for ConsoleConsumerOptions.checkRequiredArgs if the combination is invalid we should exit earlier.
There was a problem hiding this comment.
I kept the logic this way to make it easier to review and compare against the previous Scala code. There are tons of options and even with the many tests we have and the lot of manual testing I did, I don't want to change the behavior of this tool.
What you propose sounds like a nice improvement but I'd prefer keeping this PR as a "translation" and do improvements in follow up PRs.
| this.timeoutMs = timeoutMs; | ||
|
|
||
| if (topic.isPresent() && partitionId.isPresent() && offset.isPresent() && !includedTopics.isPresent()) { | ||
| seek(topic.get(), partitionId.getAsInt(), offset.getAsLong()); |
There was a problem hiding this comment.
If we opt in for my suggestion above we can also refactor seek to decide what to do if offset is provided or not instead of the first 2 ifs we have here.
| seek(topic.get(), partitionId.getAsInt(), offset.getAsLong()); | ||
| } else if (topic.isPresent() && partitionId.isPresent() && !offset.isPresent() && !includedTopics.isPresent()) { | ||
| // default to latest if no offset is provided | ||
| seek(topic.get(), partitionId.getAsInt(), ListOffsetsRequest.LATEST_TIMESTAMP); |
There was a problem hiding this comment.
can't we use ConsoleConsumerOptions.parseOffset to either grab the given offset or the default instead of these 2 ifs?
| } | ||
|
|
||
| private MessageFormatter buildFormatter() { | ||
| MessageFormatter formatter = null; |
There was a problem hiding this comment.
Yes it's needed otherwise the compiler complains formatter may not have been initialized as it does not understand the catch clause causes the tool to exit.
| private byte[] headersSeparator = utfBytes(","); | ||
| private byte[] nullLiteral = utfBytes("null"); | ||
|
|
||
| private Optional<Deserializer<?>> keyDeserializer; |
There was a problem hiding this comment.
Shouldn't we specify the default value to be Optional.empty here
| private byte[] nullLiteral = utfBytes("null"); | ||
|
|
||
| private Optional<Deserializer<?>> keyDeserializer; | ||
| private Optional<Deserializer<?>> valueDeserializer; |
|
|
||
| private Optional<Deserializer<?>> keyDeserializer; | ||
| private Optional<Deserializer<?>> valueDeserializer; | ||
| private Optional<Deserializer<?>> headersDeserializer; |
| from kafkatest.directory_layout.kafka_path import KafkaPathResolverMixin | ||
| from kafkatest.services.monitor.jmx import JmxMixin, JmxTool | ||
| from kafkatest.version import DEV_BRANCH, LATEST_0_8_2, LATEST_0_9, LATEST_0_10_0, V_0_10_0_0, V_0_11_0_0, V_2_0_0 | ||
| from kafkatest.version import DEV_BRANCH, LATEST_0_8_2, LATEST_0_9, LATEST_0_10_0, V_0_10_0_0, V_0_11_0_0, V_2_0_0, LATEST_3_7 |
There was a problem hiding this comment.
LATEST_3_7 does not exist yet, we need to wait for 3.7.0 to be out before adding it.
There was a problem hiding this comment.
Do I understand it right that you want to merge this only after 3.7.0 is released?
There was a problem hiding this comment.
Actually since DEV_VERSION already points to 3.8, I should be able to introduce LATEST_3_7 even if it's not released yet. I'll try that.
There was a problem hiding this comment.
That seems to work. However while running the system tests I found an issue loading configs via --consumer.config. I pushed c6355fe to fix it.
I've kicked another run on our system tests CI, hopefully it will be clean now.
| if node.version > LATEST_3_7: | ||
| cmd += " --formatter org.apache.kafka.tools.consumer.LoggingMessageFormatter" | ||
| else: | ||
| cmd += " --formatter kafka.tools.LoggingMessageFormatter" |
There was a problem hiding this comment.
LoggingMessageFormatter, DefaultMessageFormatter, NoOpMessageFormatter are not part of the public API and as far as I can tell the class names are not visible anywhere so this shouldn't be considered an API change.
ff18f43 to
bd32996
Compare
|
I rebased on trunk to resolve conflicts. It seems my last fix has resolved the issues with the system tests so it is ready to review again. |
| Properties consumerProps = new Properties(); | ||
| consumerProps.putAll(consumerPropsFromFile); |
There was a problem hiding this comment.
Strange that this change is not doing the exact same thing. But reading the API, constructor with Property says that it creates an empty list with the passed properties as defaults. Tricky API design from Java's part...
There was a problem hiding this comment.
Yep! Default values are only retrieved when using getProperty() and not when using get(). This is why the unit test was passing.
|
@OmniaGM do you want to take another look? |
|
Thanks @OmniaGM ! |
|
None of the test failures seem related, merging to trunk |
Reviewers: Josep Prat <josep.prat@aiven.io>, Omnia Ibrahim <o.g.h.ibrahim@gmail.com>
Reviewers: Josep Prat <josep.prat@aiven.io>, Omnia Ibrahim <o.g.h.ibrahim@gmail.com>
Reviewers: Josep Prat <josep.prat@aiven.io>, Omnia Ibrahim <o.g.h.ibrahim@gmail.com>
Committer Checklist (excluded from commit message)