mirror of https://github.com/apache/kafka.git
KAFKA-18616; Refactor Tools's ApiMessageFormatter (#18695)
This patch refactors the `ApiMessageFormatter` to follow what we have done in https://github.com/apache/kafka/pull/18688. Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
parent
91758cc99d
commit
b6adec48c5
|
@ -338,6 +338,7 @@
|
||||||
|
|
||||||
<subpackage name="consumer">
|
<subpackage name="consumer">
|
||||||
<allow pkg="org.apache.kafka.tools"/>
|
<allow pkg="org.apache.kafka.tools"/>
|
||||||
|
<allow pkg="org.apache.kafka.coordinator.common.runtime" />
|
||||||
<subpackage name="group">
|
<subpackage name="group">
|
||||||
<allow pkg="org.apache.kafka.coordinator.group"/>
|
<allow pkg="org.apache.kafka.coordinator.group"/>
|
||||||
<allow pkg="kafka.api"/>
|
<allow pkg="kafka.api"/>
|
||||||
|
|
|
@ -18,6 +18,9 @@ package org.apache.kafka.tools.consumer;
|
||||||
|
|
||||||
import org.apache.kafka.clients.consumer.ConsumerRecord;
|
import org.apache.kafka.clients.consumer.ConsumerRecord;
|
||||||
import org.apache.kafka.common.MessageFormatter;
|
import org.apache.kafka.common.MessageFormatter;
|
||||||
|
import org.apache.kafka.common.protocol.ApiMessage;
|
||||||
|
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
|
||||||
|
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecordSerde;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.JsonNode;
|
import com.fasterxml.jackson.databind.JsonNode;
|
||||||
import com.fasterxml.jackson.databind.node.JsonNodeFactory;
|
import com.fasterxml.jackson.databind.node.JsonNodeFactory;
|
||||||
|
@ -31,44 +34,50 @@ import java.util.Objects;
|
||||||
|
|
||||||
import static java.nio.charset.StandardCharsets.UTF_8;
|
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 TYPE = "type";
|
||||||
private static final String VERSION = "version";
|
private static final String VERSION = "version";
|
||||||
private static final String DATA = "data";
|
private static final String DATA = "data";
|
||||||
private static final String KEY = "key";
|
private static final String KEY = "key";
|
||||||
private static final String VALUE = "value";
|
private static final String VALUE = "value";
|
||||||
static final String UNKNOWN = "unknown";
|
|
||||||
|
private final CoordinatorRecordSerde serde;
|
||||||
|
|
||||||
|
public CoordinatorRecordMessageFormatter(CoordinatorRecordSerde serde) {
|
||||||
|
this.serde = serde;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void writeTo(ConsumerRecord<byte[], byte[]> consumerRecord, PrintStream output) {
|
public void writeTo(ConsumerRecord<byte[], byte[]> consumerRecord, PrintStream output) {
|
||||||
|
if (Objects.isNull(consumerRecord.key())) return;
|
||||||
|
|
||||||
ObjectNode json = new ObjectNode(JsonNodeFactory.instance);
|
ObjectNode json = new ObjectNode(JsonNodeFactory.instance);
|
||||||
|
try {
|
||||||
|
CoordinatorRecord record = serde.deserialize(
|
||||||
|
ByteBuffer.wrap(consumerRecord.key()),
|
||||||
|
consumerRecord.value() != null ? ByteBuffer.wrap(consumerRecord.value()) : null
|
||||||
|
);
|
||||||
|
|
||||||
byte[] key = consumerRecord.key();
|
if (!isRecordTypeAllowed(record.key().apiKey())) return;
|
||||||
if (Objects.nonNull(key)) {
|
|
||||||
short keyVersion = ByteBuffer.wrap(key).getShort();
|
|
||||||
JsonNode dataNode = readToKeyJson(ByteBuffer.wrap(key));
|
|
||||||
|
|
||||||
if (dataNode instanceof NullNode) {
|
json
|
||||||
return;
|
.putObject(KEY)
|
||||||
|
.put(TYPE, record.key().apiKey())
|
||||||
|
.set(DATA, keyAsJson(record.key()));
|
||||||
|
|
||||||
|
if (Objects.nonNull(record.value())) {
|
||||||
|
json
|
||||||
|
.putObject(VALUE)
|
||||||
|
.put(VERSION, record.value().version())
|
||||||
|
.set(DATA, valueAsJson(record.value().message(), record.value().version()));
|
||||||
|
} else {
|
||||||
|
json.set(VALUE, NullNode.getInstance());
|
||||||
}
|
}
|
||||||
json.putObject(KEY)
|
} catch (CoordinatorRecordSerde.UnknownRecordTypeException ex) {
|
||||||
.put(TYPE, keyVersion)
|
|
||||||
.set(DATA, dataNode);
|
|
||||||
} else {
|
|
||||||
return;
|
return;
|
||||||
}
|
} catch (RuntimeException ex) {
|
||||||
|
throw new RuntimeException("Could not read record at offset " + consumerRecord.offset() +
|
||||||
byte[] value = consumerRecord.value();
|
" due to: " + ex.getMessage(), ex);
|
||||||
if (Objects.nonNull(value)) {
|
|
||||||
short valueVersion = ByteBuffer.wrap(value).getShort();
|
|
||||||
JsonNode dataNode = readToValueJson(ByteBuffer.wrap(value));
|
|
||||||
|
|
||||||
json.putObject(VALUE)
|
|
||||||
.put(VERSION, valueVersion)
|
|
||||||
.set(DATA, dataNode);
|
|
||||||
} else {
|
|
||||||
json.set(VALUE, NullNode.getInstance());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
|
@ -78,6 +87,7 @@ public abstract class ApiMessageFormatter implements MessageFormatter {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
protected abstract JsonNode readToKeyJson(ByteBuffer byteBuffer);
|
protected abstract boolean isRecordTypeAllowed(short recordType);
|
||||||
protected abstract JsonNode readToValueJson(ByteBuffer byteBuffer);
|
protected abstract JsonNode keyAsJson(ApiMessage message);
|
||||||
|
protected abstract JsonNode valueAsJson(ApiMessage message, short version);
|
||||||
}
|
}
|
|
@ -16,45 +16,36 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.tools.consumer;
|
package org.apache.kafka.tools.consumer;
|
||||||
|
|
||||||
import org.apache.kafka.common.errors.UnsupportedVersionException;
|
import org.apache.kafka.common.protocol.ApiMessage;
|
||||||
import org.apache.kafka.common.protocol.ByteBufferAccessor;
|
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.CoordinatorRecordType;
|
||||||
import org.apache.kafka.coordinator.group.generated.GroupMetadataKey;
|
|
||||||
import org.apache.kafka.coordinator.group.generated.GroupMetadataKeyJsonConverter;
|
|
||||||
import org.apache.kafka.coordinator.group.generated.GroupMetadataValue;
|
|
||||||
import org.apache.kafka.coordinator.group.generated.GroupMetadataValueJsonConverter;
|
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.JsonNode;
|
import com.fasterxml.jackson.databind.JsonNode;
|
||||||
import com.fasterxml.jackson.databind.node.NullNode;
|
|
||||||
import com.fasterxml.jackson.databind.node.TextNode;
|
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.util.Set;
|
||||||
|
|
||||||
public class GroupMetadataMessageFormatter extends ApiMessageFormatter {
|
public class GroupMetadataMessageFormatter extends CoordinatorRecordMessageFormatter {
|
||||||
@Override
|
private static final Set<Short> ALLOWED_RECORDS = Set.of(
|
||||||
protected JsonNode readToKeyJson(ByteBuffer byteBuffer) {
|
CoordinatorRecordType.GROUP_METADATA.id()
|
||||||
try {
|
);
|
||||||
switch (CoordinatorRecordType.fromId(byteBuffer.getShort())) {
|
|
||||||
case GROUP_METADATA:
|
|
||||||
return GroupMetadataKeyJsonConverter.write(
|
|
||||||
new GroupMetadataKey(new ByteBufferAccessor(byteBuffer), (short) 0),
|
|
||||||
(short) 0
|
|
||||||
);
|
|
||||||
|
|
||||||
default:
|
public GroupMetadataMessageFormatter() {
|
||||||
return NullNode.getInstance();
|
super(new GroupCoordinatorRecordSerde());
|
||||||
}
|
|
||||||
} catch (UnsupportedVersionException ex) {
|
|
||||||
return NullNode.getInstance();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected JsonNode readToValueJson(ByteBuffer byteBuffer) {
|
protected boolean isRecordTypeAllowed(short recordType) {
|
||||||
short version = byteBuffer.getShort();
|
return ALLOWED_RECORDS.contains(recordType);
|
||||||
if (version >= GroupMetadataValue.LOWEST_SUPPORTED_VERSION && version <= GroupMetadataValue.HIGHEST_SUPPORTED_VERSION) {
|
}
|
||||||
return GroupMetadataValueJsonConverter.write(new GroupMetadataValue(new ByteBufferAccessor(byteBuffer), version), version);
|
|
||||||
}
|
@Override
|
||||||
return new TextNode(UNKNOWN);
|
protected JsonNode keyAsJson(ApiMessage message) {
|
||||||
|
return CoordinatorRecordJsonConverters.writeRecordKeyAsJson(message);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected JsonNode valueAsJson(ApiMessage message, short version) {
|
||||||
|
return CoordinatorRecordJsonConverters.writeRecordValueAsJson(message, version);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -16,50 +16,40 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.tools.consumer;
|
package org.apache.kafka.tools.consumer;
|
||||||
|
|
||||||
import org.apache.kafka.common.errors.UnsupportedVersionException;
|
import org.apache.kafka.common.protocol.ApiMessage;
|
||||||
import org.apache.kafka.common.protocol.ByteBufferAccessor;
|
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.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.JsonNode;
|
||||||
import com.fasterxml.jackson.databind.node.NullNode;
|
|
||||||
import com.fasterxml.jackson.databind.node.TextNode;
|
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.util.Set;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Formatter for use with tools such as console consumer: Consumer should also set exclude.internal.topics to false.
|
* 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 {
|
||||||
@Override
|
private static final Set<Short> ALLOWED_RECORDS = Set.of(
|
||||||
protected JsonNode readToKeyJson(ByteBuffer byteBuffer) {
|
CoordinatorRecordType.LEGACY_OFFSET_COMMIT.id(),
|
||||||
try {
|
CoordinatorRecordType.OFFSET_COMMIT.id()
|
||||||
switch (CoordinatorRecordType.fromId(byteBuffer.getShort())) {
|
);
|
||||||
// We can read both record types with the offset commit one.
|
|
||||||
case LEGACY_OFFSET_COMMIT:
|
|
||||||
case OFFSET_COMMIT:
|
|
||||||
return OffsetCommitKeyJsonConverter.write(
|
|
||||||
new OffsetCommitKey(new ByteBufferAccessor(byteBuffer), (short) 0),
|
|
||||||
(short) 0
|
|
||||||
);
|
|
||||||
|
|
||||||
default:
|
public OffsetsMessageFormatter() {
|
||||||
return NullNode.getInstance();
|
super(new GroupCoordinatorRecordSerde());
|
||||||
}
|
|
||||||
} catch (UnsupportedVersionException ex) {
|
|
||||||
return NullNode.getInstance();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected JsonNode readToValueJson(ByteBuffer byteBuffer) {
|
protected boolean isRecordTypeAllowed(short recordType) {
|
||||||
short version = byteBuffer.getShort();
|
return ALLOWED_RECORDS.contains(recordType);
|
||||||
if (version >= OffsetCommitValue.LOWEST_SUPPORTED_VERSION && version <= OffsetCommitValue.HIGHEST_SUPPORTED_VERSION) {
|
}
|
||||||
return OffsetCommitValueJsonConverter.write(new OffsetCommitValue(new ByteBufferAccessor(byteBuffer), version), version);
|
|
||||||
}
|
@Override
|
||||||
return new TextNode(UNKNOWN);
|
protected JsonNode keyAsJson(ApiMessage message) {
|
||||||
|
return CoordinatorRecordJsonConverters.writeRecordKeyAsJson(message);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected JsonNode valueAsJson(ApiMessage message, short version) {
|
||||||
|
return CoordinatorRecordJsonConverters.writeRecordValueAsJson(message, version);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -16,45 +16,29 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.tools.consumer;
|
package org.apache.kafka.tools.consumer;
|
||||||
|
|
||||||
import org.apache.kafka.common.errors.UnsupportedVersionException;
|
import org.apache.kafka.common.protocol.ApiMessage;
|
||||||
import org.apache.kafka.common.protocol.ByteBufferAccessor;
|
import org.apache.kafka.coordinator.transaction.TransactionCoordinatorRecordSerde;
|
||||||
import org.apache.kafka.coordinator.transaction.generated.CoordinatorRecordType;
|
import org.apache.kafka.coordinator.transaction.generated.CoordinatorRecordJsonConverters;
|
||||||
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.JsonNode;
|
||||||
import com.fasterxml.jackson.databind.node.NullNode;
|
|
||||||
import com.fasterxml.jackson.databind.node.TextNode;
|
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
public class TransactionLogMessageFormatter extends CoordinatorRecordMessageFormatter {
|
||||||
|
public TransactionLogMessageFormatter() {
|
||||||
public class TransactionLogMessageFormatter extends ApiMessageFormatter {
|
super(new TransactionCoordinatorRecordSerde());
|
||||||
@Override
|
|
||||||
protected JsonNode readToKeyJson(ByteBuffer byteBuffer) {
|
|
||||||
try {
|
|
||||||
switch (CoordinatorRecordType.fromId(byteBuffer.getShort())) {
|
|
||||||
case TRANSACTION_LOG:
|
|
||||||
return TransactionLogKeyJsonConverter.write(
|
|
||||||
new TransactionLogKey(new ByteBufferAccessor(byteBuffer), (short) 0),
|
|
||||||
(short) 0
|
|
||||||
);
|
|
||||||
|
|
||||||
default:
|
|
||||||
return NullNode.getInstance();
|
|
||||||
}
|
|
||||||
} catch (UnsupportedVersionException ex) {
|
|
||||||
return NullNode.getInstance();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected JsonNode readToValueJson(ByteBuffer byteBuffer) {
|
protected boolean isRecordTypeAllowed(short recordType) {
|
||||||
short version = byteBuffer.getShort();
|
return true;
|
||||||
if (version >= TransactionLogValue.LOWEST_SUPPORTED_VERSION && version <= TransactionLogValue.HIGHEST_SUPPORTED_VERSION) {
|
}
|
||||||
return TransactionLogValueJsonConverter.write(new TransactionLogValue(new ByteBufferAccessor(byteBuffer), version), version);
|
|
||||||
}
|
@Override
|
||||||
return new TextNode(UNKNOWN);
|
protected JsonNode keyAsJson(ApiMessage message) {
|
||||||
|
return CoordinatorRecordJsonConverters.writeRecordKeyAsJson(message);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected JsonNode valueAsJson(ApiMessage message, short version) {
|
||||||
|
return CoordinatorRecordJsonConverters.writeRecordValueAsJson(message, version);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.kafka.coordinator.share.generated.ShareUpdateKey;
|
||||||
import org.apache.kafka.coordinator.share.generated.ShareUpdateKeyJsonConverter;
|
import org.apache.kafka.coordinator.share.generated.ShareUpdateKeyJsonConverter;
|
||||||
import org.apache.kafka.coordinator.share.generated.ShareUpdateValue;
|
import org.apache.kafka.coordinator.share.generated.ShareUpdateValue;
|
||||||
import org.apache.kafka.coordinator.share.generated.ShareUpdateValueJsonConverter;
|
import org.apache.kafka.coordinator.share.generated.ShareUpdateValueJsonConverter;
|
||||||
|
import org.apache.kafka.tools.consumer.CoordinatorRecordMessageFormatter;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.JsonNode;
|
import com.fasterxml.jackson.databind.JsonNode;
|
||||||
import com.fasterxml.jackson.databind.node.JsonNodeFactory;
|
import com.fasterxml.jackson.databind.node.JsonNodeFactory;
|
||||||
|
@ -132,7 +133,7 @@ public class ShareGroupStateMessageFormatter implements MessageFormatter {
|
||||||
* as per RPC spec.
|
* as per RPC spec.
|
||||||
* To differentiate, we need to use the corresponding key versions. This is acceptable as
|
* To differentiate, we need to use the corresponding key versions. This is acceptable as
|
||||||
* the records will always appear in pairs (key, value). However, this means that we cannot
|
* the records will always appear in pairs (key, value). However, this means that we cannot
|
||||||
* extend {@link org.apache.kafka.tools.consumer.ApiMessageFormatter} as it requires overriding
|
* extend {@link CoordinatorRecordMessageFormatter} as it requires overriding
|
||||||
* readToValueJson whose signature does not allow for passing keyversion.
|
* readToValueJson whose signature does not allow for passing keyversion.
|
||||||
*
|
*
|
||||||
* @param byteBuffer - Represents the raw data read from the topic
|
* @param byteBuffer - Represents the raw data read from the topic
|
||||||
|
|
|
@ -0,0 +1,68 @@
|
||||||
|
/*
|
||||||
|
* 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.record.TimestampType;
|
||||||
|
|
||||||
|
import org.junit.jupiter.api.TestInstance;
|
||||||
|
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;
|
||||||
|
|
||||||
|
@TestInstance(TestInstance.Lifecycle.PER_CLASS)
|
||||||
|
public abstract class CoordinatorRecordMessageFormatterTest {
|
||||||
|
private static final String TOPIC = "TOPIC";
|
||||||
|
|
||||||
|
protected abstract CoordinatorRecordMessageFormatter formatter();
|
||||||
|
protected abstract Stream<Arguments> parameters();
|
||||||
|
|
||||||
|
@ParameterizedTest
|
||||||
|
@MethodSource("parameters")
|
||||||
|
public void testMessageFormatter(byte[] keyBuffer, byte[] valueBuffer, String expectedOutput) {
|
||||||
|
ConsumerRecord<byte[], byte[]> record = new ConsumerRecord<>(
|
||||||
|
TOPIC,
|
||||||
|
0,
|
||||||
|
0,
|
||||||
|
0L,
|
||||||
|
TimestampType.CREATE_TIME,
|
||||||
|
0,
|
||||||
|
0,
|
||||||
|
keyBuffer,
|
||||||
|
valueBuffer,
|
||||||
|
new RecordHeaders(),
|
||||||
|
Optional.empty()
|
||||||
|
);
|
||||||
|
|
||||||
|
try (MessageFormatter formatter = formatter()) {
|
||||||
|
formatter.configure(emptyMap());
|
||||||
|
ByteArrayOutputStream out = new ByteArrayOutputStream();
|
||||||
|
formatter.writeTo(record, new PrintStream(out));
|
||||||
|
assertEquals(expectedOutput.replaceAll("\\s+", ""), out.toString());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -16,144 +16,179 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.tools.consumer;
|
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.protocol.MessageUtil;
|
||||||
import org.apache.kafka.common.record.TimestampType;
|
|
||||||
import org.apache.kafka.coordinator.group.generated.GroupMetadataKey;
|
import org.apache.kafka.coordinator.group.generated.GroupMetadataKey;
|
||||||
import org.apache.kafka.coordinator.group.generated.GroupMetadataValue;
|
import org.apache.kafka.coordinator.group.generated.GroupMetadataValue;
|
||||||
import org.apache.kafka.coordinator.group.generated.OffsetCommitKey;
|
import org.apache.kafka.coordinator.group.generated.OffsetCommitKey;
|
||||||
import org.apache.kafka.coordinator.group.generated.OffsetCommitValue;
|
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.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 java.util.stream.Stream;
|
||||||
|
|
||||||
import static java.util.Collections.emptyMap;
|
|
||||||
import static java.util.Collections.singletonList;
|
import static java.util.Collections.singletonList;
|
||||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
|
||||||
|
|
||||||
public class GroupMetadataMessageFormatterTest {
|
public class GroupMetadataMessageFormatterTest extends CoordinatorRecordMessageFormatterTest {
|
||||||
|
|
||||||
private static final OffsetCommitKey OFFSET_COMMIT_KEY = new OffsetCommitKey()
|
private static final OffsetCommitKey OFFSET_COMMIT_KEY = new OffsetCommitKey()
|
||||||
.setGroup("group-id")
|
.setGroup("group-id")
|
||||||
.setTopic("foo")
|
.setTopic("foo")
|
||||||
.setPartition(1);
|
.setPartition(1);
|
||||||
private static final OffsetCommitValue OFFSET_COMMIT_VALUE = new OffsetCommitValue()
|
private static final OffsetCommitValue OFFSET_COMMIT_VALUE = new OffsetCommitValue()
|
||||||
.setOffset(100L)
|
.setOffset(100L)
|
||||||
.setLeaderEpoch(10)
|
.setLeaderEpoch(10)
|
||||||
.setMetadata("metadata")
|
.setMetadata("metadata")
|
||||||
.setCommitTimestamp(1234L)
|
.setCommitTimestamp(1234L)
|
||||||
.setExpireTimestamp(-1L);
|
.setExpireTimestamp(-1L);
|
||||||
private static final GroupMetadataValue.MemberMetadata MEMBER_METADATA = new GroupMetadataValue.MemberMetadata()
|
private static final GroupMetadataValue.MemberMetadata MEMBER_METADATA = new GroupMetadataValue.MemberMetadata()
|
||||||
.setMemberId("member-1")
|
.setMemberId("member-1")
|
||||||
.setClientId("client-1")
|
.setClientId("client-1")
|
||||||
.setClientHost("host-1")
|
.setClientHost("host-1")
|
||||||
.setRebalanceTimeout(1000)
|
.setRebalanceTimeout(1000)
|
||||||
.setSessionTimeout(1500)
|
.setSessionTimeout(1500)
|
||||||
.setGroupInstanceId("group-instance-1")
|
.setGroupInstanceId("group-instance-1")
|
||||||
.setSubscription(new byte[]{0, 1})
|
.setSubscription(new byte[]{0, 1})
|
||||||
.setAssignment(new byte[]{1, 2});
|
.setAssignment(new byte[]{1, 2});
|
||||||
private static final GroupMetadataKey GROUP_METADATA_KEY = new GroupMetadataKey()
|
private static final GroupMetadataKey GROUP_METADATA_KEY = new GroupMetadataKey()
|
||||||
.setGroup("group-id");
|
.setGroup("group-id");
|
||||||
private static final GroupMetadataValue GROUP_METADATA_VALUE = new GroupMetadataValue()
|
private static final GroupMetadataValue GROUP_METADATA_VALUE = new GroupMetadataValue()
|
||||||
.setProtocolType("consumer")
|
.setProtocolType("consumer")
|
||||||
.setGeneration(1)
|
.setGeneration(1)
|
||||||
.setProtocol("range")
|
.setProtocol("range")
|
||||||
.setLeader("leader")
|
.setLeader("leader")
|
||||||
.setMembers(singletonList(MEMBER_METADATA))
|
.setMembers(singletonList(MEMBER_METADATA))
|
||||||
.setCurrentStateTimestamp(1234L);
|
.setCurrentStateTimestamp(1234L);
|
||||||
private static final String TOPIC = "TOPIC";
|
|
||||||
|
|
||||||
private static Stream<Arguments> parameters() {
|
@Override
|
||||||
|
protected CoordinatorRecordMessageFormatter formatter() {
|
||||||
|
return new GroupMetadataMessageFormatter();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Stream<Arguments> parameters() {
|
||||||
return Stream.of(
|
return Stream.of(
|
||||||
Arguments.of(
|
Arguments.of(
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 10, GROUP_METADATA_KEY).array(),
|
MessageUtil.toVersionPrefixedByteBuffer((short) 2, GROUP_METADATA_KEY).array(),
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 10, GROUP_METADATA_VALUE).array(),
|
MessageUtil.toVersionPrefixedByteBuffer((short) 0, GROUP_METADATA_VALUE).array(),
|
||||||
""
|
"""
|
||||||
),
|
{"key":{"type":2,"data":{"group":"group-id"}},
|
||||||
Arguments.of(
|
"value":{"version":0,
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 2, GROUP_METADATA_KEY).array(),
|
"data":{"protocolType":"consumer",
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 0, GROUP_METADATA_VALUE).array(),
|
"generation":1,
|
||||||
"{\"key\":{\"type\":2,\"data\":{\"group\":\"group-id\"}},\"value\":{\"version\":0," +
|
"protocol":"range",
|
||||||
"\"data\":{\"protocolType\":\"consumer\",\"generation\":1,\"protocol\":\"range\"," +
|
"leader":"leader",
|
||||||
"\"leader\":\"leader\",\"members\":[{\"memberId\":\"member-1\",\"clientId\":\"client-1\"," +
|
"members":[{"memberId":"member-1",
|
||||||
"\"clientHost\":\"host-1\",\"sessionTimeout\":1500,\"subscription\":\"AAE=\"," +
|
"clientId":"client-1",
|
||||||
"\"assignment\":\"AQI=\"}]}}}"
|
"clientHost":"host-1",
|
||||||
),
|
"sessionTimeout":1500,
|
||||||
Arguments.of(
|
"subscription":"AAE=",
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 2, GROUP_METADATA_KEY).array(),
|
"assignment":"AQI="}]}}}
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 1, GROUP_METADATA_VALUE).array(),
|
"""
|
||||||
"{\"key\":{\"type\":2,\"data\":{\"group\":\"group-id\"}},\"value\":{\"version\":1," +
|
),
|
||||||
"\"data\":{\"protocolType\":\"consumer\",\"generation\":1,\"protocol\":\"range\"," +
|
Arguments.of(
|
||||||
"\"leader\":\"leader\",\"members\":[{\"memberId\":\"member-1\",\"clientId\":\"client-1\"," +
|
MessageUtil.toVersionPrefixedByteBuffer((short) 2, GROUP_METADATA_KEY).array(),
|
||||||
"\"clientHost\":\"host-1\",\"rebalanceTimeout\":1000,\"sessionTimeout\":1500," +
|
MessageUtil.toVersionPrefixedByteBuffer((short) 1, GROUP_METADATA_VALUE).array(),
|
||||||
"\"subscription\":\"AAE=\",\"assignment\":\"AQI=\"}]}}}"
|
"""
|
||||||
),
|
{"key":{"type":2,"data":{"group":"group-id"}},
|
||||||
Arguments.of(
|
"value":{"version":1,
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 2, GROUP_METADATA_KEY).array(),
|
"data":{"protocolType":"consumer",
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 2, GROUP_METADATA_VALUE).array(),
|
"generation":1,
|
||||||
"{\"key\":{\"type\":2,\"data\":{\"group\":\"group-id\"}},\"value\":{\"version\":2," +
|
"protocol":"range",
|
||||||
"\"data\":{\"protocolType\":\"consumer\",\"generation\":1,\"protocol\":\"range\"," +
|
"leader":"leader",
|
||||||
"\"leader\":\"leader\",\"currentStateTimestamp\":1234,\"members\":[{\"memberId\":\"member-1\"," +
|
"members":[{"memberId":"member-1",
|
||||||
"\"clientId\":\"client-1\",\"clientHost\":\"host-1\",\"rebalanceTimeout\":1000," +
|
"clientId":"client-1",
|
||||||
"\"sessionTimeout\":1500,\"subscription\":\"AAE=\",\"assignment\":\"AQI=\"}]}}}"
|
"clientHost":"host-1",
|
||||||
),
|
"rebalanceTimeout":1000,
|
||||||
Arguments.of(
|
"sessionTimeout":1500,
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 2, GROUP_METADATA_KEY).array(),
|
"subscription":"AAE=",
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 3, GROUP_METADATA_VALUE).array(),
|
"assignment":"AQI="}]}}}
|
||||||
"{\"key\":{\"type\":2,\"data\":{\"group\":\"group-id\"}},\"value\":{\"version\":3," +
|
"""
|
||||||
"\"data\":{\"protocolType\":\"consumer\",\"generation\":1,\"protocol\":\"range\"," +
|
),
|
||||||
"\"leader\":\"leader\",\"currentStateTimestamp\":1234,\"members\":[{\"memberId\":\"member-1\"," +
|
Arguments.of(
|
||||||
"\"groupInstanceId\":\"group-instance-1\",\"clientId\":\"client-1\",\"clientHost\":\"host-1\"," +
|
MessageUtil.toVersionPrefixedByteBuffer((short) 2, GROUP_METADATA_KEY).array(),
|
||||||
"\"rebalanceTimeout\":1000,\"sessionTimeout\":1500,\"subscription\":\"AAE=\",\"assignment\":\"AQI=\"}]}}}"
|
MessageUtil.toVersionPrefixedByteBuffer((short) 2, GROUP_METADATA_VALUE).array(),
|
||||||
),
|
"""
|
||||||
Arguments.of(
|
{"key":{"type":2,"data":{"group":"group-id"}},
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 2, GROUP_METADATA_KEY).array(),
|
"value":{"version":2,
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 4, GROUP_METADATA_VALUE).array(),
|
"data":{"protocolType":"consumer",
|
||||||
"{\"key\":{\"type\":2,\"data\":{\"group\":\"group-id\"}},\"value\":{\"version\":4," +
|
"generation":1,
|
||||||
"\"data\":{\"protocolType\":\"consumer\",\"generation\":1,\"protocol\":\"range\"," +
|
"protocol":"range",
|
||||||
"\"leader\":\"leader\",\"currentStateTimestamp\":1234,\"members\":[{\"memberId\":\"member-1\"," +
|
"leader":"leader",
|
||||||
"\"groupInstanceId\":\"group-instance-1\",\"clientId\":\"client-1\",\"clientHost\":\"host-1\"," +
|
"currentStateTimestamp":1234,
|
||||||
"\"rebalanceTimeout\":1000,\"sessionTimeout\":1500,\"subscription\":\"AAE=\",\"assignment\":\"AQI=\"}]}}}"
|
"members":[{"memberId":"member-1",
|
||||||
),
|
"clientId":"client-1",
|
||||||
Arguments.of(
|
"clientHost":"host-1",
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 2, GROUP_METADATA_KEY).array(),
|
"rebalanceTimeout":1000,
|
||||||
null,
|
"sessionTimeout":1500,
|
||||||
"{\"key\":{\"type\":2,\"data\":{\"group\":\"group-id\"}},\"value\":null}"),
|
"subscription":"AAE=",
|
||||||
Arguments.of(
|
"assignment":"AQI="}]}}}
|
||||||
null,
|
"""
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 4, GROUP_METADATA_VALUE).array(),
|
),
|
||||||
""),
|
Arguments.of(
|
||||||
Arguments.of(null, null, ""),
|
MessageUtil.toVersionPrefixedByteBuffer((short) 2, GROUP_METADATA_KEY).array(),
|
||||||
Arguments.of(
|
MessageUtil.toVersionPrefixedByteBuffer((short) 3, GROUP_METADATA_VALUE).array(),
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 0, OFFSET_COMMIT_KEY).array(),
|
"""
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 4, OFFSET_COMMIT_VALUE).array(),
|
{"key":{"type":2,"data":{"group":"group-id"}},
|
||||||
""
|
"value":{"version":3,
|
||||||
)
|
"data":{"protocolType":"consumer",
|
||||||
|
"generation":1,
|
||||||
|
"protocol":"range",
|
||||||
|
"leader":"leader",
|
||||||
|
"currentStateTimestamp":1234,
|
||||||
|
"members":[{"memberId":"member-1",
|
||||||
|
"groupInstanceId":"group-instance-1",
|
||||||
|
"clientId":"client-1",
|
||||||
|
"clientHost":"host-1",
|
||||||
|
"rebalanceTimeout":1000,
|
||||||
|
"sessionTimeout":1500,
|
||||||
|
"subscription":"AAE=",
|
||||||
|
"assignment":"AQI="}]}}}
|
||||||
|
"""
|
||||||
|
),
|
||||||
|
Arguments.of(
|
||||||
|
MessageUtil.toVersionPrefixedByteBuffer((short) 2, GROUP_METADATA_KEY).array(),
|
||||||
|
MessageUtil.toVersionPrefixedByteBuffer((short) 4, GROUP_METADATA_VALUE).array(),
|
||||||
|
"""
|
||||||
|
{"key":{"type":2,"data":{"group":"group-id"}},
|
||||||
|
"value":{"version":4,
|
||||||
|
"data":{"protocolType":"consumer",
|
||||||
|
"generation":1,
|
||||||
|
"protocol":"range",
|
||||||
|
"leader":"leader",
|
||||||
|
"currentStateTimestamp":1234,
|
||||||
|
"members":[{"memberId":"member-1",
|
||||||
|
"groupInstanceId":"group-instance-1",
|
||||||
|
"clientId":"client-1",
|
||||||
|
"clientHost":"host-1",
|
||||||
|
"rebalanceTimeout":1000,
|
||||||
|
"sessionTimeout":1500,
|
||||||
|
"subscription":"AAE=",
|
||||||
|
"assignment":"AQI="}]}}}
|
||||||
|
"""
|
||||||
|
),
|
||||||
|
Arguments.of(
|
||||||
|
MessageUtil.toVersionPrefixedByteBuffer((short) 2, GROUP_METADATA_KEY).array(),
|
||||||
|
null,
|
||||||
|
"""
|
||||||
|
{"key":{"type":2,"data":{"group":"group-id"}},"value":null}
|
||||||
|
"""
|
||||||
|
),
|
||||||
|
Arguments.of(
|
||||||
|
null,
|
||||||
|
MessageUtil.toVersionPrefixedByteBuffer((short) 4, GROUP_METADATA_VALUE).array(),
|
||||||
|
""
|
||||||
|
),
|
||||||
|
Arguments.of(null, null, ""),
|
||||||
|
Arguments.of(
|
||||||
|
MessageUtil.toVersionPrefixedByteBuffer((short) 0, OFFSET_COMMIT_KEY).array(),
|
||||||
|
MessageUtil.toVersionPrefixedByteBuffer((short) 0, OFFSET_COMMIT_VALUE).array(),
|
||||||
|
""
|
||||||
|
),
|
||||||
|
Arguments.of(
|
||||||
|
MessageUtil.toVersionPrefixedByteBuffer(Short.MAX_VALUE, OFFSET_COMMIT_KEY).array(),
|
||||||
|
MessageUtil.toVersionPrefixedByteBuffer((short) 0, OFFSET_COMMIT_VALUE).array(),
|
||||||
|
""
|
||||||
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
|
||||||
@MethodSource("parameters")
|
|
||||||
public void testTransactionLogMessageFormatter(byte[] keyBuffer, byte[] valueBuffer, String expectedOutput) {
|
|
||||||
ConsumerRecord<byte[], byte[]> record = new ConsumerRecord<>(
|
|
||||||
TOPIC, 0, 0,
|
|
||||||
0L, TimestampType.CREATE_TIME, 0,
|
|
||||||
0, keyBuffer, valueBuffer,
|
|
||||||
new RecordHeaders(), Optional.empty());
|
|
||||||
|
|
||||||
try (MessageFormatter formatter = new GroupMetadataMessageFormatter()) {
|
|
||||||
formatter.configure(emptyMap());
|
|
||||||
ByteArrayOutputStream out = new ByteArrayOutputStream();
|
|
||||||
formatter.writeTo(record, new PrintStream(out));
|
|
||||||
assertEquals(expectedOutput, out.toString());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -16,135 +16,137 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.tools.consumer;
|
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.protocol.MessageUtil;
|
||||||
import org.apache.kafka.common.record.TimestampType;
|
|
||||||
import org.apache.kafka.coordinator.group.generated.GroupMetadataKey;
|
import org.apache.kafka.coordinator.group.generated.GroupMetadataKey;
|
||||||
import org.apache.kafka.coordinator.group.generated.GroupMetadataValue;
|
import org.apache.kafka.coordinator.group.generated.GroupMetadataValue;
|
||||||
import org.apache.kafka.coordinator.group.generated.OffsetCommitKey;
|
import org.apache.kafka.coordinator.group.generated.OffsetCommitKey;
|
||||||
import org.apache.kafka.coordinator.group.generated.OffsetCommitValue;
|
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.Arguments;
|
||||||
import org.junit.jupiter.params.provider.MethodSource;
|
|
||||||
|
|
||||||
import java.io.ByteArrayOutputStream;
|
|
||||||
import java.io.PrintStream;
|
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.Optional;
|
|
||||||
import java.util.stream.Stream;
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
import static java.util.Collections.emptyMap;
|
public class OffsetMessageFormatterTest extends CoordinatorRecordMessageFormatterTest {
|
||||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
|
||||||
|
|
||||||
public class OffsetMessageFormatterTest {
|
|
||||||
|
|
||||||
private static final OffsetCommitKey OFFSET_COMMIT_KEY = new OffsetCommitKey()
|
private static final OffsetCommitKey OFFSET_COMMIT_KEY = new OffsetCommitKey()
|
||||||
.setGroup("group-id")
|
.setGroup("group-id")
|
||||||
.setTopic("foo")
|
.setTopic("foo")
|
||||||
.setPartition(1);
|
.setPartition(1);
|
||||||
private static final OffsetCommitValue OFFSET_COMMIT_VALUE = new OffsetCommitValue()
|
private static final OffsetCommitValue OFFSET_COMMIT_VALUE = new OffsetCommitValue()
|
||||||
.setOffset(100L)
|
.setOffset(100L)
|
||||||
.setLeaderEpoch(10)
|
.setLeaderEpoch(10)
|
||||||
.setMetadata("metadata")
|
.setMetadata("metadata")
|
||||||
.setCommitTimestamp(1234L)
|
.setCommitTimestamp(1234L)
|
||||||
.setExpireTimestamp(-1L);
|
.setExpireTimestamp(5678L);
|
||||||
private static final GroupMetadataKey GROUP_METADATA_KEY = new GroupMetadataKey().setGroup("group-id");
|
private static final GroupMetadataKey GROUP_METADATA_KEY = new GroupMetadataKey().setGroup("group-id");
|
||||||
private static final GroupMetadataValue GROUP_METADATA_VALUE = new GroupMetadataValue()
|
private static final GroupMetadataValue GROUP_METADATA_VALUE = new GroupMetadataValue()
|
||||||
.setProtocolType("consumer")
|
.setProtocolType("consumer")
|
||||||
.setGeneration(1)
|
.setGeneration(1)
|
||||||
.setProtocol("range")
|
.setProtocol("range")
|
||||||
.setLeader("leader")
|
.setLeader("leader")
|
||||||
.setMembers(Collections.emptyList());
|
.setMembers(Collections.emptyList());
|
||||||
private static final String TOPIC = "TOPIC";
|
|
||||||
|
|
||||||
private static Stream<Arguments> parameters() {
|
@Override
|
||||||
|
protected CoordinatorRecordMessageFormatter formatter() {
|
||||||
|
return new OffsetsMessageFormatter();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Stream<Arguments> parameters() {
|
||||||
return Stream.of(
|
return Stream.of(
|
||||||
Arguments.of(
|
Arguments.of(
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 10, OFFSET_COMMIT_KEY).array(),
|
MessageUtil.toVersionPrefixedByteBuffer((short) 0, OFFSET_COMMIT_KEY).array(),
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 10, OFFSET_COMMIT_VALUE).array(),
|
MessageUtil.toVersionPrefixedByteBuffer((short) 0, OFFSET_COMMIT_VALUE).array(),
|
||||||
""
|
"""
|
||||||
),
|
{"key":{"type":0,"data":{"group":"group-id","topic":"foo","partition":1}},
|
||||||
Arguments.of(
|
"value":{"version":0,
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 0, OFFSET_COMMIT_KEY).array(),
|
"data":{"offset":100,
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 0, OFFSET_COMMIT_VALUE).array(),
|
"metadata":"metadata",
|
||||||
"{\"key\":{\"type\":0,\"data\":{\"group\":\"group-id\",\"topic\":\"foo\",\"partition\":1}}," +
|
"commitTimestamp":1234}}}
|
||||||
"\"value\":{\"version\":0,\"data\":{\"offset\":100,\"metadata\":\"metadata\"," +
|
"""
|
||||||
"\"commitTimestamp\":1234}}}"
|
),
|
||||||
),
|
Arguments.of(
|
||||||
Arguments.of(
|
MessageUtil.toVersionPrefixedByteBuffer((short) 1, OFFSET_COMMIT_KEY).array(),
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 0, OFFSET_COMMIT_KEY).array(),
|
MessageUtil.toVersionPrefixedByteBuffer((short) 0, OFFSET_COMMIT_VALUE).array(),
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 1, OFFSET_COMMIT_VALUE).array(),
|
"""
|
||||||
"{\"key\":{\"type\":0,\"data\":{\"group\":\"group-id\",\"topic\":\"foo\",\"partition\":1}}," +
|
{"key":{"type":1,"data":{"group":"group-id","topic":"foo","partition":1}},
|
||||||
"\"value\":{\"version\":1,\"data\":{\"offset\":100,\"metadata\":\"metadata\"," +
|
"value":{"version":0,
|
||||||
"\"commitTimestamp\":1234,\"expireTimestamp\":-1}}}"
|
"data":{"offset":100,
|
||||||
),
|
"metadata":"metadata",
|
||||||
Arguments.of(
|
"commitTimestamp":1234}}}
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 0, OFFSET_COMMIT_KEY).array(),
|
"""
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 2, OFFSET_COMMIT_VALUE).array(),
|
),
|
||||||
"{\"key\":{\"type\":0,\"data\":{\"group\":\"group-id\",\"topic\":\"foo\",\"partition\":1}}," +
|
Arguments.of(
|
||||||
"\"value\":{\"version\":2,\"data\":{\"offset\":100,\"metadata\":\"metadata\"," +
|
MessageUtil.toVersionPrefixedByteBuffer((short) 1, OFFSET_COMMIT_KEY).array(),
|
||||||
"\"commitTimestamp\":1234}}}"
|
MessageUtil.toVersionPrefixedByteBuffer((short) 1, OFFSET_COMMIT_VALUE).array(),
|
||||||
),
|
"""
|
||||||
Arguments.of(
|
{"key":{"type":1,"data":{"group":"group-id","topic":"foo","partition":1}},
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 0, OFFSET_COMMIT_KEY).array(),
|
"value":{"version":1,
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 3, OFFSET_COMMIT_VALUE).array(),
|
"data":{"offset":100,
|
||||||
"{\"key\":{\"type\":0,\"data\":{\"group\":\"group-id\",\"topic\":\"foo\",\"partition\":1}}," +
|
"metadata":"metadata",
|
||||||
"\"value\":{\"version\":3,\"data\":{\"offset\":100,\"leaderEpoch\":10," +
|
"commitTimestamp":1234,
|
||||||
"\"metadata\":\"metadata\",\"commitTimestamp\":1234}}}"
|
"expireTimestamp":5678}}}
|
||||||
),
|
"""
|
||||||
Arguments.of(
|
),
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 0, OFFSET_COMMIT_KEY).array(),
|
Arguments.of(
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 4, OFFSET_COMMIT_VALUE).array(),
|
MessageUtil.toVersionPrefixedByteBuffer((short) 1, OFFSET_COMMIT_KEY).array(),
|
||||||
"{\"key\":{\"type\":0,\"data\":{\"group\":\"group-id\",\"topic\":\"foo\",\"partition\":1}}," +
|
MessageUtil.toVersionPrefixedByteBuffer((short) 2, OFFSET_COMMIT_VALUE).array(),
|
||||||
"\"value\":{\"version\":4,\"data\":{\"offset\":100,\"leaderEpoch\":10," +
|
"""
|
||||||
"\"metadata\":\"metadata\",\"commitTimestamp\":1234}}}"
|
{"key":{"type":1,"data":{"group":"group-id","topic":"foo","partition":1}},
|
||||||
),
|
"value":{"version":2,
|
||||||
Arguments.of(
|
"data":{"offset":100,
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 5, OFFSET_COMMIT_KEY).array(),
|
"metadata":"metadata",
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 4, OFFSET_COMMIT_VALUE).array(),
|
"commitTimestamp":1234}}}
|
||||||
""
|
"""
|
||||||
),
|
),
|
||||||
Arguments.of(
|
Arguments.of(
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 0, OFFSET_COMMIT_KEY).array(),
|
MessageUtil.toVersionPrefixedByteBuffer((short) 1, OFFSET_COMMIT_KEY).array(),
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 5, OFFSET_COMMIT_VALUE).array(),
|
MessageUtil.toVersionPrefixedByteBuffer((short) 3, OFFSET_COMMIT_VALUE).array(),
|
||||||
"{\"key\":{\"type\":0,\"data\":{\"group\":\"group-id\",\"topic\":\"foo\",\"partition\":1}}," +
|
"""
|
||||||
"\"value\":{\"version\":5,\"data\":\"unknown\"}}"
|
{"key":{"type":1,"data":{"group":"group-id","topic":"foo","partition":1}},
|
||||||
),
|
"value":{"version":3,
|
||||||
Arguments.of(
|
"data":{"offset":100,
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 0, OFFSET_COMMIT_KEY).array(),
|
"leaderEpoch":10,
|
||||||
null,
|
"metadata":"metadata",
|
||||||
"{\"key\":{\"type\":0,\"data\":{\"group\":\"group-id\",\"topic\":\"foo\",\"partition\":1}}," +
|
"commitTimestamp":1234}}}
|
||||||
"\"value\":null}"),
|
"""
|
||||||
Arguments.of(
|
),
|
||||||
null,
|
Arguments.of(
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 1, OFFSET_COMMIT_VALUE).array(),
|
MessageUtil.toVersionPrefixedByteBuffer((short) 1, OFFSET_COMMIT_KEY).array(),
|
||||||
""),
|
MessageUtil.toVersionPrefixedByteBuffer((short) 4, OFFSET_COMMIT_VALUE).array(),
|
||||||
Arguments.of(null, null, ""),
|
"""
|
||||||
Arguments.of(
|
{"key":{"type":1,"data":{"group":"group-id","topic":"foo","partition":1}},
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 2, GROUP_METADATA_KEY).array(),
|
"value":{"version":4,
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 2, GROUP_METADATA_VALUE).array(),
|
"data":{"offset":100,
|
||||||
""
|
"leaderEpoch":10,
|
||||||
)
|
"metadata":"metadata",
|
||||||
|
"commitTimestamp":1234}}}
|
||||||
|
"""
|
||||||
|
),
|
||||||
|
Arguments.of(
|
||||||
|
MessageUtil.toVersionPrefixedByteBuffer((short) 1, OFFSET_COMMIT_KEY).array(),
|
||||||
|
null,
|
||||||
|
"""
|
||||||
|
{"key":{"type":1,"data":{"group":"group-id","topic":"foo","partition":1}},"value":null}
|
||||||
|
"""
|
||||||
|
),
|
||||||
|
Arguments.of(
|
||||||
|
null,
|
||||||
|
MessageUtil.toVersionPrefixedByteBuffer((short) 1, OFFSET_COMMIT_VALUE).array(),
|
||||||
|
""
|
||||||
|
),
|
||||||
|
Arguments.of(null, null, ""),
|
||||||
|
Arguments.of(
|
||||||
|
MessageUtil.toVersionPrefixedByteBuffer((short) 2, GROUP_METADATA_KEY).array(),
|
||||||
|
MessageUtil.toVersionPrefixedByteBuffer((short) 2, GROUP_METADATA_VALUE).array(),
|
||||||
|
""
|
||||||
|
),
|
||||||
|
Arguments.of(
|
||||||
|
MessageUtil.toVersionPrefixedByteBuffer(Short.MAX_VALUE, GROUP_METADATA_KEY).array(),
|
||||||
|
MessageUtil.toVersionPrefixedByteBuffer((short) 2, GROUP_METADATA_VALUE).array(),
|
||||||
|
""
|
||||||
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
|
||||||
@MethodSource("parameters")
|
|
||||||
public void testTransactionLogMessageFormatter(byte[] keyBuffer, byte[] valueBuffer, String expectedOutput) {
|
|
||||||
ConsumerRecord<byte[], byte[]> record = new ConsumerRecord<>(
|
|
||||||
TOPIC, 0, 0,
|
|
||||||
0L, TimestampType.CREATE_TIME, 0,
|
|
||||||
0, keyBuffer, valueBuffer,
|
|
||||||
new RecordHeaders(), Optional.empty());
|
|
||||||
|
|
||||||
try (MessageFormatter formatter = new OffsetsMessageFormatter()) {
|
|
||||||
formatter.configure(emptyMap());
|
|
||||||
ByteArrayOutputStream out = new ByteArrayOutputStream();
|
|
||||||
formatter.writeTo(record, new PrintStream(out));
|
|
||||||
assertEquals(expectedOutput, out.toString());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -16,93 +16,95 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.tools.consumer;
|
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.protocol.MessageUtil;
|
||||||
import org.apache.kafka.common.record.TimestampType;
|
|
||||||
import org.apache.kafka.coordinator.transaction.generated.TransactionLogKey;
|
import org.apache.kafka.coordinator.transaction.generated.TransactionLogKey;
|
||||||
import org.apache.kafka.coordinator.transaction.generated.TransactionLogValue;
|
import org.apache.kafka.coordinator.transaction.generated.TransactionLogValue;
|
||||||
|
|
||||||
import org.junit.jupiter.params.ParameterizedTest;
|
|
||||||
import org.junit.jupiter.params.provider.Arguments;
|
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 java.util.stream.Stream;
|
||||||
|
|
||||||
import static java.util.Collections.emptyList;
|
import static java.util.Collections.emptyList;
|
||||||
import static java.util.Collections.emptyMap;
|
|
||||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
|
||||||
|
|
||||||
public class TransactionLogMessageFormatterTest {
|
public class TransactionLogMessageFormatterTest extends CoordinatorRecordMessageFormatterTest {
|
||||||
|
|
||||||
private static final TransactionLogKey TXN_LOG_KEY = new TransactionLogKey()
|
private static final TransactionLogKey TXN_LOG_KEY = new TransactionLogKey()
|
||||||
.setTransactionalId("TXNID");
|
.setTransactionalId("TXNID");
|
||||||
private static final TransactionLogValue TXN_LOG_VALUE = new TransactionLogValue()
|
private static final TransactionLogValue TXN_LOG_VALUE = new TransactionLogValue()
|
||||||
.setProducerId(100)
|
.setProducerId(100)
|
||||||
.setProducerEpoch((short) 50)
|
.setProducerEpoch((short) 50)
|
||||||
.setTransactionStatus((byte) 4)
|
.setTransactionStatus((byte) 4)
|
||||||
.setTransactionStartTimestampMs(750L)
|
.setTransactionStartTimestampMs(750L)
|
||||||
.setTransactionLastUpdateTimestampMs(1000L)
|
.setTransactionLastUpdateTimestampMs(1000L)
|
||||||
.setTransactionTimeoutMs(500)
|
.setTransactionTimeoutMs(500)
|
||||||
.setTransactionPartitions(emptyList());
|
.setTransactionPartitions(emptyList());
|
||||||
private static final String TOPIC = "TOPIC";
|
|
||||||
|
@Override
|
||||||
private static Stream<Arguments> parameters() {
|
protected CoordinatorRecordMessageFormatter formatter() {
|
||||||
|
return new TransactionLogMessageFormatter();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Stream<Arguments> parameters() {
|
||||||
return Stream.of(
|
return Stream.of(
|
||||||
Arguments.of(
|
Arguments.of(
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 10, TXN_LOG_KEY).array(),
|
MessageUtil.toVersionPrefixedByteBuffer((short) 10, TXN_LOG_KEY).array(),
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 10, TXN_LOG_VALUE).array(),
|
MessageUtil.toVersionPrefixedByteBuffer((short) 10, TXN_LOG_VALUE).array(),
|
||||||
""
|
""
|
||||||
),
|
),
|
||||||
Arguments.of(
|
Arguments.of(
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 0, TXN_LOG_KEY).array(),
|
MessageUtil.toVersionPrefixedByteBuffer((short) 0, TXN_LOG_KEY).array(),
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 1, TXN_LOG_VALUE).array(),
|
MessageUtil.toVersionPrefixedByteBuffer((short) 1, TXN_LOG_VALUE).array(),
|
||||||
"{\"key\":{\"type\":0,\"data\":{\"transactionalId\":\"TXNID\"}}," +
|
"""
|
||||||
"\"value\":{\"version\":1,\"data\":{\"producerId\":100,\"producerEpoch\":50," +
|
{"key":{"type":0,"data":{"transactionalId":"TXNID"}},
|
||||||
"\"transactionTimeoutMs\":500,\"transactionStatus\":4,\"transactionPartitions\":[]," +
|
"value":{"version":1,
|
||||||
"\"transactionLastUpdateTimestampMs\":1000,\"transactionStartTimestampMs\":750}}}"
|
"data":{"producerId":100,
|
||||||
),
|
"producerEpoch":50,
|
||||||
Arguments.of(
|
"transactionTimeoutMs":500,
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 0, TXN_LOG_KEY).array(),
|
"transactionStatus":4,
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 5, TXN_LOG_VALUE).array(),
|
"transactionPartitions":[],
|
||||||
"{\"key\":{\"type\":0,\"data\":{\"transactionalId\":\"TXNID\"}}," +
|
"transactionLastUpdateTimestampMs":1000,
|
||||||
"\"value\":{\"version\":5,\"data\":\"unknown\"}}"
|
"transactionStartTimestampMs":750}}}
|
||||||
),
|
"""
|
||||||
Arguments.of(
|
),
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 1, TXN_LOG_KEY).array(),
|
Arguments.of(
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 1, TXN_LOG_VALUE).array(),
|
MessageUtil.toVersionPrefixedByteBuffer((short) 0, TXN_LOG_KEY).array(),
|
||||||
""),
|
MessageUtil.toVersionPrefixedByteBuffer((short) 1, TXN_LOG_VALUE).array(),
|
||||||
Arguments.of(
|
"""
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 0, TXN_LOG_KEY).array(),
|
{"key":{"type":0,"data":{"transactionalId":"TXNID"}},
|
||||||
null,
|
"value":{"version":1,
|
||||||
"{\"key\":{\"type\":0,\"data\":{\"transactionalId\":\"TXNID\"}}," +
|
"data":{"producerId":100,
|
||||||
"\"value\":null}"),
|
"producerEpoch":50,
|
||||||
Arguments.of(
|
"transactionTimeoutMs":500,
|
||||||
null,
|
"transactionStatus":4,
|
||||||
MessageUtil.toVersionPrefixedByteBuffer((short) 1, TXN_LOG_VALUE).array(),
|
"transactionPartitions":[],
|
||||||
""),
|
"transactionLastUpdateTimestampMs":1000,
|
||||||
Arguments.of(null, null, "")
|
"transactionStartTimestampMs":750}}}
|
||||||
|
"""
|
||||||
|
),
|
||||||
|
Arguments.of(
|
||||||
|
MessageUtil.toVersionPrefixedByteBuffer((short) 1, TXN_LOG_KEY).array(),
|
||||||
|
MessageUtil.toVersionPrefixedByteBuffer((short) 1, TXN_LOG_VALUE).array(),
|
||||||
|
""
|
||||||
|
),
|
||||||
|
Arguments.of(
|
||||||
|
MessageUtil.toVersionPrefixedByteBuffer((short) 0, TXN_LOG_KEY).array(),
|
||||||
|
null,
|
||||||
|
"""
|
||||||
|
{"key":{"type":0,"data":{"transactionalId":"TXNID"}},"value":null}
|
||||||
|
"""
|
||||||
|
),
|
||||||
|
Arguments.of(
|
||||||
|
null,
|
||||||
|
MessageUtil.toVersionPrefixedByteBuffer((short) 1, TXN_LOG_VALUE).array(),
|
||||||
|
""
|
||||||
|
),
|
||||||
|
Arguments.of(null, null, ""),
|
||||||
|
Arguments.of(
|
||||||
|
MessageUtil.toVersionPrefixedByteBuffer(Short.MAX_VALUE, TXN_LOG_KEY).array(),
|
||||||
|
MessageUtil.toVersionPrefixedByteBuffer((short) 1, TXN_LOG_VALUE).array(),
|
||||||
|
""
|
||||||
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
|
||||||
@MethodSource("parameters")
|
|
||||||
public void testTransactionLogMessageFormatter(byte[] keyBuffer, byte[] valueBuffer, String expectedOutput) {
|
|
||||||
ConsumerRecord<byte[], byte[]> record = new ConsumerRecord<>(
|
|
||||||
TOPIC, 0, 0,
|
|
||||||
0L, TimestampType.CREATE_TIME, 0,
|
|
||||||
0, keyBuffer, valueBuffer,
|
|
||||||
new RecordHeaders(), Optional.empty());
|
|
||||||
|
|
||||||
try (MessageFormatter formatter = new TransactionLogMessageFormatter()) {
|
|
||||||
formatter.configure(emptyMap());
|
|
||||||
ByteArrayOutputStream out = new ByteArrayOutputStream();
|
|
||||||
formatter.writeTo(record, new PrintStream(out));
|
|
||||||
assertEquals(expectedOutput, out.toString());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue