Skip to content
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

support peek broker entry metadta #9067

Merged
merged 1 commit into from
Dec 28, 2020
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -2031,6 +2031,15 @@ public static PulsarApi.BrokerEntryMetadata parseBrokerEntryMetadataIfExist(
}
}

public static PulsarApi.BrokerEntryMetadata peekBrokerEntryMetadataIfExist(
ByteBuf headerAndPayloadWithBrokerEntryMetadata) {
final int readerIndex = headerAndPayloadWithBrokerEntryMetadata.readerIndex();
PulsarApi.BrokerEntryMetadata entryMetadata =
parseBrokerEntryMetadataIfExist(headerAndPayloadWithBrokerEntryMetadata);
headerAndPayloadWithBrokerEntryMetadata.readerIndex(readerIndex);
return entryMetadata;
}

public static ByteBuf serializeMetadataAndPayload(ChecksumType checksumType,
MessageMetadata msgMetadata, ByteBuf payload) {
// / Wire format
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -202,6 +202,38 @@ public void testParseBrokerEntryMetadata() throws Exception {
assertEquals(new String(content, StandardCharsets.UTF_8), data);
}

@Test
public void testPeekBrokerEntryMetadata() throws Exception {
int MOCK_BATCH_SIZE = 10;
String data = "test-message";
ByteBuf byteBuf = PulsarByteBufAllocator.DEFAULT.buffer(data.length(), data.length());
byteBuf.writeBytes(data.getBytes(StandardCharsets.UTF_8));
ByteBuf dataWithBrokerEntryMetadata =
Commands.addBrokerEntryMetadata(byteBuf, getBrokerEntryMetadataInterceptors(), MOCK_BATCH_SIZE);
int bytesBeforePeek = dataWithBrokerEntryMetadata.readableBytes();
PulsarApi.BrokerEntryMetadata brokerMetadata =
Commands.peekBrokerEntryMetadataIfExist(dataWithBrokerEntryMetadata);

assertTrue(brokerMetadata.getBrokerTimestamp() <= System.currentTimeMillis());
assertEquals(brokerMetadata.getIndex(), MOCK_BATCH_SIZE - 1);

int bytesAfterPeek = dataWithBrokerEntryMetadata.readableBytes();
assertEquals(bytesBeforePeek, bytesAfterPeek);

// test parse logic after peek

PulsarApi.BrokerEntryMetadata brokerMetadata1 =
Commands.parseBrokerEntryMetadataIfExist(dataWithBrokerEntryMetadata);
assertTrue(brokerMetadata1.getBrokerTimestamp() <= System.currentTimeMillis());

assertEquals(brokerMetadata1.getIndex(), MOCK_BATCH_SIZE - 1);
assertEquals(data.length(), dataWithBrokerEntryMetadata.readableBytes());

byte [] content = new byte[dataWithBrokerEntryMetadata.readableBytes()];
dataWithBrokerEntryMetadata.readBytes(content);
assertEquals(new String(content, StandardCharsets.UTF_8), data);
}

public Set<BrokerEntryMetadataInterceptor> getBrokerEntryMetadataInterceptors() {
Set<String> interceptorNames = new HashSet<>();
interceptorNames.add("org.apache.pulsar.common.intercept.AppendBrokerTimestampMetadataInterceptor");
Expand Down