Skip to content
This repository has been archived by the owner on Jan 24, 2024. It is now read-only.

Commit

Permalink
Refactor APIs to allow access from the Proxy module (#710)
Browse files Browse the repository at this point in the history
This patch introduces a little refactor in order to support the implementation of a separate KOP Proxy Module:
- allow to access internal ByteBuf of KafkaHeaderAndRequest, in order to allow zero-copy request magement in the Proxy
- extract some static methods in order to use the same algorithm for mapping coordinators

Relates to #57 

Co-authored-by: Enrico Olivelli <eolivelli@apache.org>
  • Loading branch information
2 people authored and BewareMyPower committed Sep 8, 2021
1 parent 716df10 commit 14e2f1a
Show file tree
Hide file tree
Showing 3 changed files with 24 additions and 7 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -548,6 +548,10 @@ static class KafkaHeaderAndRequest implements Closeable {
this.remoteAddress = remoteAddress;
}

public ByteBuf getBuffer() {
return buffer;
}

public RequestHeader getHeader() {
return this.header;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -213,13 +213,14 @@ public GroupMetadataManager(OffsetConfig offsetConfig,
time,
// Be same with kafka: abs(groupId.hashCode) % groupMetadataTopicPartitionCount
// return a partitionId
groupId -> MathUtils.signSafeMod(
groupId.hashCode(),
offsetConfig.offsetsTopicNumPartitions()
)
groupId -> getPartitionId(groupId, offsetConfig.offsetsTopicNumPartitions())
);
}

public static int getPartitionId(String groupId, int offsetsTopicNumPartitions) {
return MathUtils.signSafeMod(groupId.hashCode(), offsetsTopicNumPartitions);
}

GroupMetadataManager(OffsetConfig offsetConfig,
ProducerBuilder<ByteBuffer> metadataTopicProducerBuilder,
ReaderBuilder<ByteBuffer> metadataTopicConsumerBuilder,
Expand Down Expand Up @@ -320,7 +321,11 @@ public String getTopicPartitionName() {
}

public String getTopicPartitionName(int partitionId) {
return offsetConfig.offsetsTopicName() + PARTITIONED_TOPIC_SUFFIX + partitionId;
return getTopicPartitionName(offsetConfig.offsetsTopicName(), partitionId);
}

public static String getTopicPartitionName(String offsetsTopicName, int partitionId) {
return offsetsTopicName + PARTITIONED_TOPIC_SUFFIX + partitionId;
}

public int getGroupMetadataTopicPartitionCount() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -105,9 +105,13 @@ public CompletableFuture<Void> startup() {
}

public int partitionFor(String transactionalId) {
return partitionFor(transactionalId, transactionConfig.getTransactionLogNumPartitions());
}

public static int partitionFor(String transactionalId, int transactionLogNumPartitions) {
return MathUtils.signSafeMod(
transactionalId.hashCode(),
transactionConfig.getTransactionLogNumPartitions()
transactionLogNumPartitions
);
}

Expand All @@ -116,7 +120,11 @@ public String getTopicPartitionName() {
}

public String getTopicPartitionName(int partitionId) {
return getTopicPartitionName() + PARTITIONED_TOPIC_SUFFIX + partitionId;
return getTopicPartitionName(getTopicPartitionName(), partitionId);
}

public static String getTopicPartitionName(String topicPartitionName, int partitionId) {
return topicPartitionName + PARTITIONED_TOPIC_SUFFIX + partitionId;
}

public void handleInitProducerId(String transactionalId, int transactionTimeoutMs,
Expand Down

0 comments on commit 14e2f1a

Please sign in to comment.