chia7712 commented on code in PR #18695:
URL: https://github.com/apache/kafka/pull/18695#discussion_r1929570741
##########
tools/src/main/java/org/apache/kafka/tools/consumer/OffsetsMessageFormatter.java:
##########
@@ -16,50 +16,41 @@
*/
package org.apache.kafka.tools.consumer;
-import org.apache.kafka.common.errors.UnsupportedVersionException;
-import org.apache.kafka.common.protocol.ByteBufferAccessor;
+import org.apache.kafka.common.protocol.ApiMessage;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorRecordSerde;
+import org.apache.kafka.coordinator.group.GroupCoordinatorRecordSerde;
+import
org.apache.kafka.coordinator.group.generated.CoordinatorRecordJsonConverters;
import org.apache.kafka.coordinator.group.generated.CoordinatorRecordType;
-import org.apache.kafka.coordinator.group.generated.OffsetCommitKey;
-import
org.apache.kafka.coordinator.group.generated.OffsetCommitKeyJsonConverter;
-import org.apache.kafka.coordinator.group.generated.OffsetCommitValue;
-import
org.apache.kafka.coordinator.group.generated.OffsetCommitValueJsonConverter;
import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.node.NullNode;
-import com.fasterxml.jackson.databind.node.TextNode;
import java.nio.ByteBuffer;
/**
* Formatter for use with tools such as console consumer: Consumer should also
set exclude.internal.topics to false.
*/
-public class OffsetsMessageFormatter extends ApiMessageFormatter {
+public class OffsetsMessageFormatter extends CoordinatorRecordMessageFormatter
{
+ private CoordinatorRecordSerde serde = new GroupCoordinatorRecordSerde();
Review Comment:
Maybe we can move `serde` to `CoordinatorRecordMessageFormatter` like
`CoordinatorRecordMessageParser`?
##########
tools/src/main/java/org/apache/kafka/tools/consumer/CoordinatorRecordMessageFormatter.java:
##########
@@ -31,44 +34,44 @@
import static java.nio.charset.StandardCharsets.UTF_8;
-public abstract class ApiMessageFormatter implements MessageFormatter {
+public abstract class CoordinatorRecordMessageFormatter implements
MessageFormatter {
private static final String TYPE = "type";
private static final String VERSION = "version";
private static final String DATA = "data";
private static final String KEY = "key";
private static final String VALUE = "value";
- static final String UNKNOWN = "unknown";
@Override
public void writeTo(ConsumerRecord<byte[], byte[]> consumerRecord,
PrintStream output) {
+ if (Objects.isNull(consumerRecord.key())) return;
+
ObjectNode json = new ObjectNode(JsonNodeFactory.instance);
+ try {
+ CoordinatorRecord record = deserialize(
+ consumerRecord.key() != null ?
ByteBuffer.wrap(consumerRecord.key()) : null,
Review Comment:
line#47 has checked the null, so this check is redundant.
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]