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

Commit

Permalink
[FEATURE] Add commit offset test for Kafka clients of different versi…
Browse files Browse the repository at this point in the history
…ons (#686)

Fixes #681 #687

### Motivation
#605 added a test framework for Kafka clients of different versions. However, it only added the basic e2e test, an important API commitOffset was not verified.

### Modifications
Add commit offset test in BasicEndToEndPulsarTest and BasicEndToEndKafkaTest in io.streamnative.pulsar.handlers.kop.compatibility package.

This feature belongs to the compatibility issue between different versions of kafka client and kop.
  • Loading branch information
wenbingshen authored Sep 1, 2021
1 parent 40e6182 commit 13004f7
Show file tree
Hide file tree
Showing 13 changed files with 487 additions and 24 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -13,14 +13,20 @@
*/
package io.streamnative.kafka.client.zero.ten;

import com.google.common.collect.Maps;
import io.streamnative.kafka.client.api.Consumer;
import io.streamnative.kafka.client.api.ConsumerConfiguration;
import io.streamnative.kafka.client.api.ConsumerRecord;
import io.streamnative.kafka.client.api.TopicOffsetAndMetadata;
import java.time.Duration;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;

/**
* The implementation of Kafka consumer 0.10.0.0.
Expand All @@ -42,4 +48,16 @@ public List<ConsumerRecord<K, V>> receive(long timeoutMs) {
public Map<String, List<PartitionInfo>> listTopics(long timeoutMS) {
return listTopics();
}

@Override
public void commitOffsetSync(List<TopicOffsetAndMetadata> offsets, Duration timeout) {
HashMap<TopicPartition, OffsetAndMetadata> offsetsMap = Maps.newHashMap();
offsets.forEach(
offsetAndMetadata -> offsetsMap.put(
offsetAndMetadata.createTopicPartition(TopicPartition.class),
offsetAndMetadata.createOffsetAndMetadata(OffsetAndMetadata.class)
)
);
commitSync(offsetsMap);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -13,14 +13,20 @@
*/
package io.streamnative.kafka.client.one.zero;

import com.google.common.collect.Maps;
import io.streamnative.kafka.client.api.Consumer;
import io.streamnative.kafka.client.api.ConsumerConfiguration;
import io.streamnative.kafka.client.api.ConsumerRecord;
import io.streamnative.kafka.client.api.TopicOffsetAndMetadata;
import java.time.Duration;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition;

/**
* The implementation of Kafka consumer 1.0.0.
Expand All @@ -42,4 +48,16 @@ public List<ConsumerRecord<K, V>> receive(long timeoutMs) {
public Map<String, List<PartitionInfo>> listTopics(long timeoutMS) {
return listTopics();
}

@Override
public void commitOffsetSync(List<TopicOffsetAndMetadata> offsets, Duration timeout) {
HashMap<TopicPartition, OffsetAndMetadata> offsetsMap = Maps.newHashMap();
offsets.forEach(
offsetAndMetadata -> offsetsMap.put(
offsetAndMetadata.createTopicPartition(TopicPartition.class),
offsetAndMetadata.createOffsetAndMetadata(OffsetAndMetadata.class)
)
);
commitSync(offsetsMap);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@
package io.streamnative.kafka.client.api;

import java.io.Closeable;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
Expand Down Expand Up @@ -61,4 +62,6 @@ default List<ConsumerRecord<K, V>> receiveUntil(int maxNumMessages, long timeout
}

Map<String, List<PartitionInfo>> listTopics(long timeoutMS);

void commitOffsetSync(List<TopicOffsetAndMetadata> offsets, Duration timeout);
}
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,8 @@ public class ConsumerConfiguration {
private String userName;
private String password;
private String requestTimeoutMs;
private Boolean enableAutoCommit;
private String sessionTimeOutMs;

public Properties toProperties() {
final Properties props = new Properties();
Expand Down Expand Up @@ -64,6 +66,12 @@ public Properties toProperties() {
if (requestTimeoutMs != null) {
props.put("request.timeout.ms", requestTimeoutMs);
}
if (enableAutoCommit != null) {
props.put("enable.auto.commit", enableAutoCommit);
}
if (sessionTimeOutMs != null) {
props.put("session.timeout.ms", sessionTimeOutMs);
}
return props;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,9 +23,14 @@ public interface Producer<K, V> extends AutoCloseable {
Future<RecordMetadata> sendAsync(ProduceContext<K, V> context);

default ProduceContext.ProduceContextBuilder<K, V> newContextBuilder(String topic, V value) {
return newContextBuilder(topic, value, null);
}

default ProduceContext.ProduceContextBuilder<K, V> newContextBuilder(String topic, V value, Integer partition) {
return ProduceContext.<K, V>builder()
.producer(this)
.topic(topic)
.value(value);
.value(value)
.partition(partition);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,57 @@
/**
* 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.kafka.client.api;

import java.lang.reflect.InvocationTargetException;
import lombok.AllArgsConstructor;
import lombok.Getter;

/**
* A completable class of org.apache.kafka.clients.consumer.OffsetAndMetadata.
*/
@AllArgsConstructor
@Getter
public class TopicOffsetAndMetadata {

private String topic;
private int partition;
private long offset;

public <T> T createTopicPartition(final Class<T> clazz) {
try {
return clazz.getConstructor(
String.class, int.class
).newInstance(topic, partition);
} catch (InvocationTargetException
| InstantiationException
| IllegalAccessException
| NoSuchMethodException e) {
throw new IllegalArgumentException(e);
}
}

public <T> T createOffsetAndMetadata(final Class<T> clazz) {
try {
return clazz.getConstructor(
long.class
).newInstance(offset);
} catch (InstantiationException
| IllegalAccessException
| InvocationTargetException
| NoSuchMethodException e) {
throw new IllegalArgumentException(e);
}

}
}
Original file line number Diff line number Diff line change
Expand Up @@ -369,11 +369,6 @@ protected ApiVersionsResponse overloadDefaultApiVersionsResponse(boolean unsuppo
for (ApiKeys apiKey : ApiKeys.values()) {
if (apiKey.minRequiredInterBrokerMagic <= RecordBatch.CURRENT_MAGIC_VALUE) {
switch (apiKey) {
case FETCH:
// V4 added MessageSets responses. We need to make sure RecordBatch format is not used
versionList.add(new ApiVersionsResponse.ApiVersion((short) 1, (short) 4,
apiKey.latestVersion()));
break;
case LIST_OFFSETS:
// V0 is needed for librdkafka
versionList.add(new ApiVersionsResponse.ApiVersion((short) 2, (short) 0,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import io.netty.util.ReferenceCounted;
import io.streamnative.pulsar.handlers.kop.exceptions.KoPMessageMetadataNotFoundException;
import io.streamnative.pulsar.handlers.kop.utils.ByteBufUtils;
import io.streamnative.pulsar.handlers.kop.utils.KopRecordsUtil;
import io.streamnative.pulsar.handlers.kop.utils.MessageIdUtils;
import java.io.IOException;
import java.util.ArrayList;
Expand All @@ -31,7 +32,7 @@
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.record.ConvertedRecords;
import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.record.RecordBatch;
import org.apache.pulsar.common.allocator.PulsarByteBufAllocator;
import org.apache.pulsar.common.api.proto.KeyValue;
import org.apache.pulsar.common.api.proto.MessageMetadata;
Expand All @@ -46,8 +47,6 @@ public class KafkaEntryFormatter implements EntryFormatter {
// These key-value identifies the entry's format as kafka
private static final String IDENTITY_KEY = "entry.format";
private static final String IDENTITY_VALUE = EntryFormatterFactory.EntryFormat.KAFKA.name().toLowerCase();
// Kafka MemoryRecords downConvert method needs time
private static final Time time = Time.SYSTEM;

@Override
public ByteBuf encode(MemoryRecords records, int numMessages) {
Expand Down Expand Up @@ -78,23 +77,24 @@ public DecodeResult decode(List<Entry> entries, byte magic) {

// batch magic greater than the magic corresponding to the version requested by the client
// need down converted
if (batchMagic > magic) {
if (batchMagic > magic || batchMagic != RecordBatch.MAGIC_VALUE_V2) {
MemoryRecords memoryRecords = MemoryRecords.readableRecords(ByteBufUtils.getNioBuffer(byteBuf));
//down converted, batch magic will be set to client magic
ConvertedRecords<MemoryRecords> convertedRecords =
memoryRecords.downConvert(magic, startOffset, time);
KopRecordsUtil.convertAndAssignOffsets(memoryRecords.batches(), magic, startOffset);

final ByteBuf kafkaBuffer = Unpooled.wrappedBuffer(convertedRecords.records().buffer());
orderedByteBuf.add(kafkaBuffer);
if (!optionalByteBufs.isPresent()) {
optionalByteBufs = Optional.of(new ArrayList<>());
}
optionalByteBufs.ifPresent(byteBufs -> byteBufs.add(byteBuf));
optionalByteBufs.ifPresent(byteBufs -> byteBufs.add(kafkaBuffer));

if (log.isTraceEnabled()) {
log.trace("[{}:{}] downConvert record, start offset {}, entry magic: {}, client magic: {}"
, entry.getLedgerId(), entry.getEntryId(), startOffset, batchMagic, magic);
log.trace("[{}:{}] convertAndAssignOffsets record for down converted"
+ " or assign offsets with v0 and v1 magic, start offset {},"
+ " entry magic: {}, client magic: {}",
entry.getLedgerId(), entry.getEntryId(), startOffset, batchMagic, magic);
}

} else {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,140 @@
/**
* 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.utils;

import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
import lombok.extern.slf4j.Slf4j;
import org.apache.kafka.common.record.AbstractRecords;
import org.apache.kafka.common.record.ConvertedRecords;
import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.MemoryRecordsBuilder;
import org.apache.kafka.common.record.Record;
import org.apache.kafka.common.record.RecordBatch;
import org.apache.kafka.common.record.TimestampType;

/**
* Utils for DownConverted and ReAssignOffset operations.
*/
@Slf4j
public class KopRecordsUtil {

public static ConvertedRecords<MemoryRecords> convertAndAssignOffsets(Iterable<? extends RecordBatch> batches,
byte toMagic,
long firstOffset) throws IOException {
// maintain the batch along with the decompressed records to avoid the need to decompress again
List<RecordBatchAndRecords> recordBatchAndRecordsList = new ArrayList<>();
int totalSizeEstimate = 0;

long batchStartOffset = firstOffset;
for (RecordBatch batch : batches) {
byte toBatchMagic = toMagic;
if (toMagic < RecordBatch.MAGIC_VALUE_V2) {
if (batch.isControlBatch()) {
continue;
}

if (batch.compressionType().name.equals("zstd")) {
throw new IOException("Down-conversion of zstandard-compressed batches "
+ "is not supported");
}
}

List<Record> records = new ArrayList<>();
long batchIndex = 0;
for (Record record : batch) {
records.add(record);
batchIndex++;
}

if (records.isEmpty()) {
continue;
}

// handle the batch.magic() <= toMagic case
// Since the internal message set of magic 0 and magic 1 has an offset,
// the internal offset may be destroyed, so we still need
// to deal with the message of batch.magic <= toMagic.
// The only thing that remains unchanged is to ensure that the magic remains unchanged.
if (batch.magic() < toMagic) {
toBatchMagic = batch.magic();
}

totalSizeEstimate += AbstractRecords.estimateSizeInBytes(
toBatchMagic, batchStartOffset, batch.compressionType(), records);
recordBatchAndRecordsList.add(new RecordBatchAndRecords(batch, records, batchStartOffset, toBatchMagic));
batchStartOffset += batchIndex;
}

ByteBuffer buffer = ByteBuffer.allocate(totalSizeEstimate);
for (RecordBatchAndRecords recordBatchAndRecords : recordBatchAndRecordsList) {
MemoryRecordsBuilder builder = convertRecordBatch(buffer, recordBatchAndRecords);
buffer = builder.buffer();
}

buffer.flip();
recordBatchAndRecordsList.clear();
return new ConvertedRecords<>(MemoryRecords.readableRecords(buffer), null);
}

private static MemoryRecordsBuilder convertRecordBatch(ByteBuffer buffer,
RecordBatchAndRecords recordBatchAndRecords) {
RecordBatch batch = recordBatchAndRecords.batch;
byte toBatchMagic = recordBatchAndRecords.toBatchMagic;
final TimestampType timestampType = batch.timestampType();
long logAppendTime = timestampType
== TimestampType.LOG_APPEND_TIME ? batch.maxTimestamp() : RecordBatch.NO_TIMESTAMP;

MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, toBatchMagic, batch.compressionType(),
timestampType, recordBatchAndRecords.baseOffset, logAppendTime);

long startOffset = recordBatchAndRecords.baseOffset;
for (Record record : recordBatchAndRecords.records) {
if (toBatchMagic > RecordBatch.MAGIC_VALUE_V1) {
builder.appendWithOffset(startOffset++,
record.timestamp(),
record.key(),
record.value(),
record.headers());
} else {
builder.appendWithOffset(startOffset++,
record.timestamp(),
record.key(),
record.value());
}
}

builder.close();
return builder;
}

private static class RecordBatchAndRecords {
private final RecordBatch batch;
private final List<Record> records;
private final Long baseOffset;
private final byte toBatchMagic;

private RecordBatchAndRecords(RecordBatch batch,
List<Record> records,
Long baseOffset,
byte toBatchMagic) {
this.batch = batch;
this.records = records;
this.baseOffset = baseOffset;
this.toBatchMagic = toBatchMagic;
}
}
}
Loading

0 comments on commit 13004f7

Please sign in to comment.