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


##########
tools/src/main/java/org/apache/kafka/tools/consumer/TransactionLogMessageFormatter.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.ByteBufferAccessor;
+import org.apache.kafka.coordinator.transaction.generated.TransactionLogKey;
+import 
org.apache.kafka.coordinator.transaction.generated.TransactionLogKeyJsonConverter;
+import org.apache.kafka.coordinator.transaction.generated.TransactionLogValue;
+import 
org.apache.kafka.coordinator.transaction.generated.TransactionLogValueJsonConverter;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.JsonNodeFactory;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+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;
+
+public class TransactionLogMessageFormatter 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";
+    
+    @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();
+            Optional<TransactionLogKey> transactionLogKey = 
readToTransactionLogKey(ByteBuffer.wrap(key));
+            settingKeyNode(json, transactionLogKey, keyVersion);
+        } else {
+            json.put(KEY, "NULL");
+        }
+
+        byte[] value = consumerRecord.value();
+        if (Objects.nonNull(value)) {
+            short valueVersion = ByteBuffer.wrap(value).getShort();
+            Optional<TransactionLogValue> transactionLogValue = 
readToTransactionLogValue(ByteBuffer.wrap(value));

Review Comment:
   Maybe we can inline those methods. for example:
   ```java
               JsonNode dataNode = 
readToTransactionLogValue(ByteBuffer.wrap(value))
                       .map(s -> TransactionLogValueJsonConverter.write(s, 
valueVersion))
                       .orElseGet(() -> new TextNode("unknown"));
               json.putObject(VALUE)
                   .put(VERSION, valueVersion)
                   .set(DATA, dataNode);
   ```



##########
tools/src/main/java/org/apache/kafka/tools/consumer/TransactionLogMessageFormatter.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.ByteBufferAccessor;
+import org.apache.kafka.coordinator.transaction.generated.TransactionLogKey;
+import 
org.apache.kafka.coordinator.transaction.generated.TransactionLogKeyJsonConverter;
+import org.apache.kafka.coordinator.transaction.generated.TransactionLogValue;
+import 
org.apache.kafka.coordinator.transaction.generated.TransactionLogValueJsonConverter;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.JsonNodeFactory;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+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;
+
+public class TransactionLogMessageFormatter 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";
+    
+    @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();

Review Comment:
   ditto:
   
   ```java
               JsonNode dataNode = readToTransactionLogKey(ByteBuffer.wrap(key))
                       .map(s -> TransactionLogKeyJsonConverter.write(s, 
keyVersion))
                       .orElseGet(() -> new TextNode("unknown"));
               json.putObject(VALUE)
                       .put(VERSION, keyVersion)
                       .set(DATA, dataNode);
   ```



##########
build.gradle:
##########
@@ -2106,6 +2106,8 @@ project(':tools') {
     implementation project(':server-common')
     implementation project(':connect:runtime')
     implementation project(':tools:tools-api')
+    implementation project(':transaction-coordinator')
+    implementation project(':group-coordinator')

Review Comment:
   this is unnecessary, right?



##########
tools/src/main/java/org/apache/kafka/tools/consumer/TransactionLogMessageFormatter.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.ByteBufferAccessor;
+import org.apache.kafka.coordinator.transaction.generated.TransactionLogKey;
+import 
org.apache.kafka.coordinator.transaction.generated.TransactionLogKeyJsonConverter;
+import org.apache.kafka.coordinator.transaction.generated.TransactionLogValue;
+import 
org.apache.kafka.coordinator.transaction.generated.TransactionLogValueJsonConverter;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.JsonNodeFactory;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+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;
+
+public class TransactionLogMessageFormatter 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";
+    
+    @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();
+            Optional<TransactionLogKey> transactionLogKey = 
readToTransactionLogKey(ByteBuffer.wrap(key));
+            settingKeyNode(json, transactionLogKey, keyVersion);
+        } else {
+            json.put(KEY, "NULL");

Review Comment:
   `json.set(KEY, NullNode.getInstance());`



##########
tools/src/main/java/org/apache/kafka/tools/consumer/TransactionLogMessageFormatter.java:
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.ByteBufferAccessor;
+import org.apache.kafka.coordinator.transaction.generated.TransactionLogKey;
+import 
org.apache.kafka.coordinator.transaction.generated.TransactionLogKeyJsonConverter;
+import org.apache.kafka.coordinator.transaction.generated.TransactionLogValue;
+import 
org.apache.kafka.coordinator.transaction.generated.TransactionLogValueJsonConverter;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.JsonNodeFactory;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+
+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;
+
+public class TransactionLogMessageFormatter 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";
+    
+    @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();
+            Optional<TransactionLogKey> transactionLogKey = 
readToTransactionLogKey(ByteBuffer.wrap(key));
+            settingKeyNode(json, transactionLogKey, keyVersion);
+        } else {
+            json.put(KEY, "NULL");
+        }
+
+        byte[] value = consumerRecord.value();
+        if (Objects.nonNull(value)) {
+            short valueVersion = ByteBuffer.wrap(value).getShort();
+            Optional<TransactionLogValue> transactionLogValue = 
readToTransactionLogValue(ByteBuffer.wrap(value));
+            settingValueNode(json, transactionLogValue, valueVersion);
+        } else {
+            json.put(VALUE, "NULL");

Review Comment:
   `json.set(VALUE, NullNode.getInstance());`



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