-
Notifications
You must be signed in to change notification settings - Fork 14.1k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
(WIP) KAFKA-18616; Refactor Tools's ApiMessageFormatter #18695
base: trunk
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -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(); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Maybe we can move |
||
|
||
@Override | ||
protected CoordinatorRecord deserialize(ByteBuffer key, ByteBuffer value) { | ||
return serde.deserialize(key, value); | ||
} | ||
|
||
@Override | ||
protected JsonNode readToKeyJson(ByteBuffer byteBuffer) { | ||
try { | ||
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 | ||
); | ||
protected boolean shouldPrint(short recordType) { | ||
return CoordinatorRecordType.OFFSET_COMMIT.id() == recordType || | ||
CoordinatorRecordType.LEGACY_OFFSET_COMMIT.id() == recordType; | ||
} | ||
|
||
default: | ||
return NullNode.getInstance(); | ||
} | ||
} catch (UnsupportedVersionException ex) { | ||
return NullNode.getInstance(); | ||
} | ||
@Override | ||
protected JsonNode keyAsJson(ApiMessage message) { | ||
return CoordinatorRecordJsonConverters.writeRecordKeyAsJson(message); | ||
} | ||
|
||
@Override | ||
protected JsonNode readToValueJson(ByteBuffer byteBuffer) { | ||
short version = byteBuffer.getShort(); | ||
if (version >= OffsetCommitValue.LOWEST_SUPPORTED_VERSION && version <= OffsetCommitValue.HIGHEST_SUPPORTED_VERSION) { | ||
return OffsetCommitValueJsonConverter.write(new OffsetCommitValue(new ByteBufferAccessor(byteBuffer), version), version); | ||
} | ||
return new TextNode(UNKNOWN); | ||
protected JsonNode valueAsJson(ApiMessage message, short version) { | ||
return CoordinatorRecordJsonConverters.writeRecordValueAsJson(message, version); | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -16,45 +16,36 @@ | |
*/ | ||
package org.apache.kafka.tools.consumer; | ||
|
||
import org.apache.kafka.common.errors.UnsupportedVersionException; | ||
import org.apache.kafka.common.protocol.ByteBufferAccessor; | ||
import org.apache.kafka.coordinator.transaction.generated.CoordinatorRecordType; | ||
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 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.transaction.TransactionCoordinatorRecordSerde; | ||
import org.apache.kafka.coordinator.transaction.generated.CoordinatorRecordJsonConverters; | ||
|
||
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 ApiMessageFormatter { | ||
public class TransactionLogMessageFormatter extends CoordinatorRecordMessageFormatter { | ||
private CoordinatorRecordSerde serde = new TransactionCoordinatorRecordSerde(); | ||
|
||
@Override | ||
protected CoordinatorRecord deserialize(ByteBuffer key, ByteBuffer value) { | ||
return serde.deserialize(key, value); | ||
} | ||
|
||
@Override | ||
protected boolean shouldPrint(short recordType) { | ||
return true; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I have a question, could you help me understand why this is There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think the reason is that there is only one element, TRANSACTION_LOG, in CoordinatorType, so there is no need to add extra checking here. |
||
} | ||
|
||
@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(); | ||
} | ||
protected JsonNode keyAsJson(ApiMessage message) { | ||
return CoordinatorRecordJsonConverters.writeRecordKeyAsJson(message); | ||
} | ||
|
||
@Override | ||
protected JsonNode readToValueJson(ByteBuffer byteBuffer) { | ||
short version = byteBuffer.getShort(); | ||
if (version >= TransactionLogValue.LOWEST_SUPPORTED_VERSION && version <= TransactionLogValue.HIGHEST_SUPPORTED_VERSION) { | ||
return TransactionLogValueJsonConverter.write(new TransactionLogValue(new ByteBufferAccessor(byteBuffer), version), version); | ||
} | ||
return new TextNode(UNKNOWN); | ||
protected JsonNode valueAsJson(ApiMessage message, short version) { | ||
return CoordinatorRecordJsonConverters.writeRecordValueAsJson(message, version); | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
line#47 has checked the null, so this check is redundant.