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

Add entry formatter to do conversions between MemoryRecords and ByteBuf #280

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.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
10 changes: 10 additions & 0 deletions kafka-impl/conf/kop.conf
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,16 @@ offsetsTopicNumPartitions=8
# Maximum number of entries that are read from cursor once per time
maxReadEntriesNum=5

# The format of an entry. The default value is pulsar.
# Optional values: [pulsar]
#
# pulsar:
# When KoP receives messages from kafka producer, it will serialize these messages to
# the format so that pulsar consumer can read directly.
# When KoP sends entries to kafka consumer, it will treat each entry as pulsar's
# format and deserialize each entry to kafka's format.
entry.format=pulsar

### --- KoP SSL configs--- ###

# Kafka ssl configuration map with: SSL_PROTOCOL_CONFIG = ssl.protocol
Expand Down
10 changes: 10 additions & 0 deletions kafka-impl/conf/kop_standalone.conf
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,16 @@ offsetsTopicNumPartitions=8
# Maximum number of entries that are read from cursor once per time
maxReadEntriesNum=1

# The format of an entry. The default value is pulsar.
# Optional values: [pulsar]
#
# pulsar:
# When KoP receives messages from kafka producer, it will serialize these messages to
# the format so that pulsar consumer can read directly.
# When KoP sends entries to kafka consumer, it will treat each entry as pulsar's
# format and deserialize each entry to kafka's format.
entry.format=pulsar

### --- KoP SSL configs--- ###

# Kafka ssl configuration map with: SSL_PROTOCOL_CONFIG = ssl.protocol
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,8 @@
import io.streamnative.pulsar.handlers.kop.coordinator.group.GroupCoordinator;
import io.streamnative.pulsar.handlers.kop.coordinator.group.GroupMetadata.GroupOverview;
import io.streamnative.pulsar.handlers.kop.coordinator.group.GroupMetadata.GroupSummary;
import io.streamnative.pulsar.handlers.kop.format.EntryFormatter;
import io.streamnative.pulsar.handlers.kop.format.EntryFormatterFactory;
import io.streamnative.pulsar.handlers.kop.offset.OffsetAndMetadata;
import io.streamnative.pulsar.handlers.kop.security.SaslAuthenticator;
import io.streamnative.pulsar.handlers.kop.utils.CoreUtils;
Expand Down Expand Up @@ -157,6 +159,8 @@ public class KafkaRequestHandler extends KafkaCommandDecoder {
private final int sslPort;
private final int defaultNumPartitions;
public final int maxReadEntriesNum;
@Getter
private final EntryFormatter entryFormatter;

private final Map<TopicPartition, PendingProduceQueue> pendingProduceQueueMap = new ConcurrentHashMap<>();

Expand Down Expand Up @@ -184,6 +188,7 @@ public KafkaRequestHandler(PulsarService pulsarService,
this.topicManager = new KafkaTopicManager(this);
this.defaultNumPartitions = kafkaConfig.getDefaultNumPartitions();
this.maxReadEntriesNum = kafkaConfig.getMaxReadEntriesNum();
this.entryFormatter = EntryFormatterFactory.create(kafkaConfig.getEntryFormat());
}

@Override
Expand Down Expand Up @@ -600,8 +605,8 @@ protected void handleProduceRequest(KafkaHeaderAndRequest produceHar,

MemoryRecords records = (MemoryRecords) entry.getValue();
String fullPartitionName = KopTopic.toString(topicPartition);
PendingProduce pendingProduce =
new PendingProduce(partitionResponse, topicManager, fullPartitionName, records, executor);
PendingProduce pendingProduce = new PendingProduce(partitionResponse, topicManager, fullPartitionName,
entryFormatter, records, executor);
PendingProduceQueue queue =
pendingProduceQueueMap.computeIfAbsent(topicPartition, ignored -> new PendingProduceQueue());
queue.add(pendingProduce);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -249,4 +249,9 @@ public class KafkaServiceConfiguration extends ServiceConfiguration {
)
private int maxReadEntriesNum = 5;

@FieldContext(
category = CATEGORY_KOP,
doc = "The format of an entry. Default: pulsar. Optional: [pulsar]"
)
private String entryFormat = "pulsar";
}
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,6 @@
*/
package io.streamnative.pulsar.handlers.kop;

import static io.streamnative.pulsar.handlers.kop.utils.MessageRecordUtils.entriesToRecords;
import static org.apache.kafka.common.protocol.CommonFields.THROTTLE_TIME_MS;

import com.google.common.collect.Lists;
Expand Down Expand Up @@ -322,9 +321,7 @@ private void readMessagesInternal(KafkaHeaderAndRequest fetch,
} else if (apiVersion <= 3) {
magic = RecordBatch.MAGIC_VALUE_V1;
}
MemoryRecords records;
// by default kafka is produced message in batched mode.
records = entriesToRecords(entries, magic);
final MemoryRecords records = requestHandler.getEntryFormatter().decode(entries, magic);

partitionData = new FetchResponse.PartitionData(
Errors.NONE,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,14 +14,13 @@
package io.streamnative.pulsar.handlers.kop;

import io.netty.buffer.ByteBuf;
import io.streamnative.pulsar.handlers.kop.utils.MessageRecordUtils;
import io.streamnative.pulsar.handlers.kop.format.EntryFormatter;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import lombok.extern.slf4j.Slf4j;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.Record;
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse;
import org.apache.pulsar.broker.service.persistent.PersistentTopic;

Expand All @@ -42,12 +41,13 @@ public class PendingProduce {
public PendingProduce(CompletableFuture<PartitionResponse> responseFuture,
KafkaTopicManager topicManager,
String partitionName,
EntryFormatter entryFormatter,
MemoryRecords memoryRecords,
ExecutorService executor) {
this.responseFuture = responseFuture;
this.topicManager = topicManager;
this.partitionName = partitionName;
this.numMessages = parseNumMessages(memoryRecords);
this.numMessages = EntryFormatter.parseNumMessages(memoryRecords);

this.topicFuture = topicManager.getTopic(partitionName).exceptionally(e -> {
log.error("Failed to getTopic for partition '{}': {}", partitionName, e);
Expand All @@ -58,10 +58,7 @@ public PendingProduce(CompletableFuture<PartitionResponse> responseFuture,
log.error("Failed to compute ByteBuf for partition '{}': {}", partitionName, e);
return null;
});
executor.execute(() -> {
ByteBuf byteBuf = MessageRecordUtils.recordsToByteBuf(memoryRecords, this.numMessages);
this.byteBufFuture.complete(byteBuf);
});
executor.execute(() -> byteBufFuture.complete(entryFormatter.encode(memoryRecords, numMessages)));
this.offsetFuture = new CompletableFuture<>();
}

Expand Down Expand Up @@ -119,12 +116,4 @@ public void publishMessages() {
byteBuf.release();
});
}

private static int parseNumMessages(MemoryRecords records) {
int n = 0;
for (Record ignored : records.records()) {
n++;
}
return n;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,63 @@
/**
* Licensed 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 io.streamnative.pulsar.handlers.kop.format;

import io.netty.buffer.ByteBuf;
import java.util.List;

import org.apache.bookkeeper.mledger.Entry;
import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.MutableRecordBatch;


/**
* The formatter for conversion between Kafka records and Bookie entries.
*/
public interface EntryFormatter {

/**
* Encode Kafka records to a ByteBuf.
*
* @param records messages with Kafka's format
* @param numMessages the number of messages
* @return the ByteBuf of an entry that is to be written to Bookie
*/
ByteBuf encode(final MemoryRecords records, final int numMessages);

/**
* Decode a stream of entries to Kafka records.
*
* @param entries the list of entries
* @param magic the Kafka record batch's magic value
* @return the Kafka records
*/
MemoryRecords decode(final List<Entry> entries, final byte magic);

/**
* Get the number of messages from MemoryRecords.
* Since MemoryRecords doesn't provide a way to get the number of messages. We need to iterate over the whole
* MemoryRecords object. So we use a helper method to get the number of messages that can be passed to
* {@link EntryFormatter#encode(MemoryRecords, int)} and metrics related methods as well.
*
* @param records messages with Kafka's format
* @return the number of messages
*/
static int parseNumMessages(final MemoryRecords records) {
int numMessages = 0;
for (MutableRecordBatch batch : records.batches()) {
numMessages += (batch.lastOffset() - batch.baseOffset() + 1);
}
return numMessages;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,40 @@
/**
* Licensed 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 io.streamnative.pulsar.handlers.kop.format;

/**
* Factory of EntryFormatter.
*
* @see EntryFormatter
*/
public class EntryFormatterFactory {

enum EntryFormat {
PULSAR
}

public static EntryFormatter create(final String format) {
try {
EntryFormat entryFormat = Enum.valueOf(EntryFormat.class, format.toUpperCase());
switch (entryFormat) {
case PULSAR:
return new PulsarEntryFormatter();
default:
throw new Exception("No EntryFormatter for " + entryFormat);
}
} catch (Exception e) {
throw new IllegalArgumentException("Unsupported entry.format '" + format + "': " + e.getMessage());
}
}
}
Loading