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


##########
tools/src/main/java/org/apache/kafka/tools/consumer/OffsetsMessageFormatter.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.ConsumerRecord;
+import org.apache.kafka.common.MessageFormatter;
+import org.apache.kafka.common.protocol.ApiMessage;
+import org.apache.kafka.common.protocol.ByteBufferAccessor;
+import org.apache.kafka.coordinator.group.generated.GroupMetadataKey;
+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.JsonNodeFactory;
+import com.fasterxml.jackson.databind.node.NullNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.fasterxml.jackson.databind.node.TextNode;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+import java.util.Objects;
+import java.util.Optional;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static 
org.apache.kafka.common.record.RecordBatch.NO_PARTITION_LEADER_EPOCH;
+import static 
org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP;
+
+/**
+ * Formatter for use with tools such as console consumer: Consumer should also 
set exclude.internal.topics to false.
+ */
+public class OffsetsMessageFormatter implements MessageFormatter {
+
+    private static final String VERSION = "version";
+    private static final String DATA = "data";
+    private static final String KEY = "key";
+    private static final String VALUE = "value";
+    private static final String UNKNOWN = "unknown";
+
+    @Override
+    public void writeTo(ConsumerRecord<byte[], byte[]> consumerRecord, 
PrintStream output) {
+        ObjectNode json = new ObjectNode(JsonNodeFactory.instance);
+
+        byte[] key = consumerRecord.key();
+        if (Objects.nonNull(key)) {
+            short keyVersion = ByteBuffer.wrap(key).getShort();
+            JsonNode dataNode = readToGroupMetadataKey(ByteBuffer.wrap(key))
+                    .map(logKey -> transferMetadataToJsonNode(logKey, 
keyVersion))
+                    .orElseGet(() -> new TextNode(UNKNOWN));
+            // Only print if the message is an offset record.
+            if (dataNode instanceof NullNode) {
+                return;
+            }
+            json.putObject(KEY)
+                    .put(VERSION, keyVersion)
+                    .set(DATA, dataNode);
+        } else {
+            json.set(KEY, NullNode.getInstance());
+        }
+
+        byte[] value = consumerRecord.value();
+        if (Objects.nonNull(value)) {
+            short valueVersion = ByteBuffer.wrap(value).getShort();
+            JsonNode dataNode = readToOffsetCommitValue(ByteBuffer.wrap(value))
+                    .map(logValue -> 
OffsetCommitValueJsonConverter.write(logValue, valueVersion))
+                    .orElseGet(() -> new TextNode(UNKNOWN));
+            json.putObject(VALUE)
+                    .put(VERSION, valueVersion)
+                    .set(DATA, dataNode);
+        } else {
+            json.set(VALUE, NullNode.getInstance());
+        }
+
+        try {
+            output.write(json.toString().getBytes(UTF_8));
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    private Optional<ApiMessage> readToGroupMetadataKey(ByteBuffer byteBuffer) 
{
+        short version = byteBuffer.getShort();
+        if (version >= OffsetCommitKey.LOWEST_SUPPORTED_VERSION
+                && version <= OffsetCommitKey.HIGHEST_SUPPORTED_VERSION) {
+            return Optional.of(new OffsetCommitKey(new 
ByteBufferAccessor(byteBuffer), version));
+        } else if (version >= GroupMetadataKey.LOWEST_SUPPORTED_VERSION && 
version <= GroupMetadataKey.HIGHEST_SUPPORTED_VERSION) {
+            return Optional.of(new GroupMetadataKey(new 
ByteBufferAccessor(byteBuffer), version));
+        } else {
+            return Optional.empty();
+        }
+    }
+
+    private static JsonNode transferMetadataToJsonNode(ApiMessage logKey, 
short keyVersion) {
+        if (logKey instanceof OffsetCommitKey) {
+            return OffsetCommitKeyJsonConverter.write((OffsetCommitKey) 
logKey, keyVersion);
+        } else if (logKey instanceof GroupMetadataKey) {
+            return NullNode.getInstance();
+        } else {
+            return new TextNode(UNKNOWN);
+        }
+    }
+
+    /**
+     * We ignore the timestamp of the message because GroupMetadataMessage has 
its own timestamp.
+     */
+    private Optional<OffsetCommitValue> readToOffsetCommitValue(ByteBuffer 
byteBuffer) {
+        short version = byteBuffer.getShort();
+        if (version >= OffsetCommitValue.LOWEST_SUPPORTED_VERSION
+                && version <= OffsetCommitValue.HIGHEST_SUPPORTED_VERSION) {
+            OffsetCommitValue value = new OffsetCommitValue(new 
ByteBufferAccessor(byteBuffer), version);
+            value.setLeaderEpoch(value.leaderEpoch() == 
NO_PARTITION_LEADER_EPOCH ? 0 : value.leaderEpoch());

Review Comment:
   IMHO, we should display the "raw" data. That is good hint in debugging.



##########
tools/src/test/java/org/apache/kafka/tools/consumer/OffsetMessageFormatterTest.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.ConsumerRecord;
+import org.apache.kafka.common.MessageFormatter;
+import org.apache.kafka.common.header.internals.RecordHeaders;
+import org.apache.kafka.common.protocol.MessageUtil;
+import org.apache.kafka.common.record.TimestampType;
+import org.apache.kafka.coordinator.group.generated.OffsetCommitKey;
+import org.apache.kafka.coordinator.group.generated.OffsetCommitValue;
+
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
+import java.util.Optional;
+import java.util.stream.Stream;
+
+import static java.util.Collections.emptyMap;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class OffsetMessageFormatterTest {
+
+    private static final OffsetCommitKey OFFSET_COMMIT_KEY = new 
OffsetCommitKey()

Review Comment:
   Could you please add test to make sure `OffsetMessageFormatter` can skip 
`GroupMetadataKey`?



-- 
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