KAFKA-7632: Support Compression Levels (KIP-390) (#15516)

Reviewers: Jun Rao <jun@confluent.io>,  Luke Chen <showuon@gmail.com>
Co-authored-by: Lee Dongjin <dongjin@apache.org>
This commit is contained in:
Mickael Maison 2024-05-21 17:58:49 +02:00 committed by GitHub
parent 9fe3932e5c
commit affe8da54c
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
134 changed files with 2374 additions and 1366 deletions

View File

@ -51,6 +51,7 @@
<allow pkg="org.apache.kafka.clients.consumer" /> <allow pkg="org.apache.kafka.clients.consumer" />
<allow pkg="org.apache.kafka.common.annotation" /> <allow pkg="org.apache.kafka.common.annotation" />
<allow pkg="org.apache.kafka.common.config" /> <allow pkg="org.apache.kafka.common.config" />
<allow pkg="org.apache.kafka.common.compress" />
<allow pkg="org.apache.kafka.common.internals" /> <allow pkg="org.apache.kafka.common.internals" />
<allow pkg="org.apache.kafka.common.message" /> <allow pkg="org.apache.kafka.common.message" />
<allow pkg="org.apache.kafka.common.metadata" /> <allow pkg="org.apache.kafka.common.metadata" />

View File

@ -173,18 +173,22 @@
<allow pkg="org.apache.kafka.metadata" /> <allow pkg="org.apache.kafka.metadata" />
<allow pkg="org.apache.kafka.common.internals" /> <allow pkg="org.apache.kafka.common.internals" />
</subpackage> </subpackage>
<subpackage name="migration">
<allow pkg="org.apache.kafka.controller" />
</subpackage>
<subpackage name="bootstrap"> <subpackage name="bootstrap">
<allow pkg="org.apache.kafka.snapshot" /> <allow pkg="org.apache.kafka.snapshot" />
</subpackage> </subpackage>
<subpackage name="fault"> <subpackage name="fault">
<allow pkg="org.apache.kafka.server.fault" /> <allow pkg="org.apache.kafka.server.fault" />
</subpackage> </subpackage>
<subpackage name="migration">
<allow pkg="org.apache.kafka.controller" />
</subpackage>
<subpackage name="util">
<allow class="org.apache.kafka.common.compress.Compression" exact-match="true" />
</subpackage>
</subpackage> </subpackage>
<subpackage name="metalog"> <subpackage name="metalog">
<allow class="org.apache.kafka.common.compress.Compression" exact-match="true" />
<allow pkg="org.apache.kafka.common.metadata" /> <allow pkg="org.apache.kafka.common.metadata" />
<allow pkg="org.apache.kafka.common.protocol" /> <allow pkg="org.apache.kafka.common.protocol" />
<allow pkg="org.apache.kafka.common.record" /> <allow pkg="org.apache.kafka.common.record" />

View File

@ -76,8 +76,10 @@
<allow pkg="com.github.luben.zstd" /> <allow pkg="com.github.luben.zstd" />
<allow pkg="net.jpountz.lz4" /> <allow pkg="net.jpountz.lz4" />
<allow pkg="net.jpountz.xxhash" /> <allow pkg="net.jpountz.xxhash" />
<allow pkg="org.apache.kafka.common.compress" />
<allow pkg="org.xerial.snappy" /> <allow pkg="org.xerial.snappy" />
<allow pkg="org.apache.kafka.common.compress" />
<allow class="org.apache.kafka.common.record.CompressionType" exact-match="true" />
<allow class="org.apache.kafka.common.record.RecordBatch" exact-match="true" />
</subpackage> </subpackage>
<subpackage name="message"> <subpackage name="message">
@ -138,6 +140,7 @@
<subpackage name="protocol"> <subpackage name="protocol">
<allow pkg="org.apache.kafka.common.errors" /> <allow pkg="org.apache.kafka.common.errors" />
<allow class="org.apache.kafka.common.compress.Compression" exact-match="true" />
<allow pkg="org.apache.kafka.common.message" /> <allow pkg="org.apache.kafka.common.message" />
<allow pkg="org.apache.kafka.common.network" /> <allow pkg="org.apache.kafka.common.network" />
<allow pkg="org.apache.kafka.common.protocol" /> <allow pkg="org.apache.kafka.common.protocol" />
@ -166,6 +169,7 @@
<subpackage name="requests"> <subpackage name="requests">
<allow pkg="org.apache.kafka.common.acl" /> <allow pkg="org.apache.kafka.common.acl" />
<allow class="org.apache.kafka.common.compress.Compression" exact-match="true" />
<allow pkg="org.apache.kafka.common.feature" /> <allow pkg="org.apache.kafka.common.feature" />
<allow pkg="org.apache.kafka.common.protocol" /> <allow pkg="org.apache.kafka.common.protocol" />
<allow pkg="org.apache.kafka.common.message" /> <allow pkg="org.apache.kafka.common.message" />
@ -426,6 +430,7 @@
<allow pkg="org.apache.kafka.metadata" /> <allow pkg="org.apache.kafka.metadata" />
<allow pkg="org.apache.kafka.snapshot" /> <allow pkg="org.apache.kafka.snapshot" />
<allow pkg="org.apache.kafka.clients" /> <allow pkg="org.apache.kafka.clients" />
<allow class="org.apache.kafka.common.compress.Compression" exact-match="true" />
<allow pkg="org.apache.kafka.common.config" /> <allow pkg="org.apache.kafka.common.config" />
<allow pkg="org.apache.kafka.common.feature" /> <allow pkg="org.apache.kafka.common.feature" />
<allow pkg="org.apache.kafka.common.message" /> <allow pkg="org.apache.kafka.common.message" />
@ -444,8 +449,9 @@
</subpackage> </subpackage>
<subpackage name="snapshot"> <subpackage name="snapshot">
<allow pkg="org.apache.kafka.common.record" /> <allow class="org.apache.kafka.common.compress.Compression" exact-match="true" />
<allow pkg="org.apache.kafka.common.message" /> <allow pkg="org.apache.kafka.common.message" />
<allow pkg="org.apache.kafka.common.record" />
<allow pkg="org.apache.kafka.raft" /> <allow pkg="org.apache.kafka.raft" />
<allow pkg="org.apache.kafka.server.common" /> <allow pkg="org.apache.kafka.server.common" />
<allow pkg="org.apache.kafka.test"/> <allow pkg="org.apache.kafka.test"/>

View File

@ -90,7 +90,7 @@
files="(Errors|SaslAuthenticatorTest|AgentTest|CoordinatorTest|NetworkClientTest).java"/> files="(Errors|SaslAuthenticatorTest|AgentTest|CoordinatorTest|NetworkClientTest).java"/>
<suppress checks="BooleanExpressionComplexity" <suppress checks="BooleanExpressionComplexity"
files="(Utils|Topic|KafkaLZ4BlockOutputStream|AclData|JoinGroupRequest).java"/> files="(Utils|Topic|Lz4BlockOutputStream|AclData|JoinGroupRequest).java"/>
<suppress checks="CyclomaticComplexity" <suppress checks="CyclomaticComplexity"
files="(AbstractFetch|ClientTelemetryReporter|ConsumerCoordinator|CommitRequestManager|FetchCollector|OffsetFetcherUtils|KafkaProducer|Sender|ConfigDef|KerberosLogin|AbstractRequest|AbstractResponse|Selector|SslFactory|SslTransportLayer|SaslClientAuthenticator|SaslClientCallbackHandler|SaslServerAuthenticator|AbstractCoordinator|TransactionManager|AbstractStickyAssignor|DefaultSslEngineFactory|Authorizer|RecordAccumulator|MemoryRecords|FetchSessionHandler|MockAdminClient).java"/> files="(AbstractFetch|ClientTelemetryReporter|ConsumerCoordinator|CommitRequestManager|FetchCollector|OffsetFetcherUtils|KafkaProducer|Sender|ConfigDef|KerberosLogin|AbstractRequest|AbstractResponse|Selector|SslFactory|SslTransportLayer|SaslClientAuthenticator|SaslClientCallbackHandler|SaslServerAuthenticator|AbstractCoordinator|TransactionManager|AbstractStickyAssignor|DefaultSslEngineFactory|Authorizer|RecordAccumulator|MemoryRecords|FetchSessionHandler|MockAdminClient).java"/>

View File

@ -41,6 +41,7 @@ import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.Uuid; import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.common.errors.ApiException; import org.apache.kafka.common.errors.ApiException;
import org.apache.kafka.common.errors.AuthenticationException; import org.apache.kafka.common.errors.AuthenticationException;
@ -250,7 +251,7 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
private final RecordAccumulator accumulator; private final RecordAccumulator accumulator;
private final Sender sender; private final Sender sender;
private final Thread ioThread; private final Thread ioThread;
private final CompressionType compressionType; private final Compression compression;
private final Sensor errors; private final Sensor errors;
private final Time time; private final Time time;
private final Serializer<K> keySerializer; private final Serializer<K> keySerializer;
@ -413,7 +414,7 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
Arrays.asList(this.keySerializer, this.valueSerializer)); Arrays.asList(this.keySerializer, this.valueSerializer));
this.maxRequestSize = config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG); this.maxRequestSize = config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG);
this.totalMemorySize = config.getLong(ProducerConfig.BUFFER_MEMORY_CONFIG); this.totalMemorySize = config.getLong(ProducerConfig.BUFFER_MEMORY_CONFIG);
this.compressionType = CompressionType.forName(config.getString(ProducerConfig.COMPRESSION_TYPE_CONFIG)); this.compression = configureCompression(config);
this.maxBlockTimeMs = config.getLong(ProducerConfig.MAX_BLOCK_MS_CONFIG); this.maxBlockTimeMs = config.getLong(ProducerConfig.MAX_BLOCK_MS_CONFIG);
int deliveryTimeoutMs = configureDeliveryTimeout(config, log); int deliveryTimeoutMs = configureDeliveryTimeout(config, log);
@ -432,7 +433,7 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
int batchSize = Math.max(1, config.getInt(ProducerConfig.BATCH_SIZE_CONFIG)); int batchSize = Math.max(1, config.getInt(ProducerConfig.BATCH_SIZE_CONFIG));
this.accumulator = new RecordAccumulator(logContext, this.accumulator = new RecordAccumulator(logContext,
batchSize, batchSize,
this.compressionType, compression,
lingerMs(config), lingerMs(config),
retryBackoffMs, retryBackoffMs,
retryBackoffMaxMs, retryBackoffMaxMs,
@ -501,7 +502,7 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
this.interceptors = interceptors; this.interceptors = interceptors;
this.maxRequestSize = config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG); this.maxRequestSize = config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG);
this.totalMemorySize = config.getLong(ProducerConfig.BUFFER_MEMORY_CONFIG); this.totalMemorySize = config.getLong(ProducerConfig.BUFFER_MEMORY_CONFIG);
this.compressionType = CompressionType.forName(config.getString(ProducerConfig.COMPRESSION_TYPE_CONFIG)); this.compression = configureCompression(config);
this.maxBlockTimeMs = config.getLong(ProducerConfig.MAX_BLOCK_MS_CONFIG); this.maxBlockTimeMs = config.getLong(ProducerConfig.MAX_BLOCK_MS_CONFIG);
this.partitionerIgnoreKeys = config.getBoolean(ProducerConfig.PARTITIONER_IGNORE_KEYS_CONFIG); this.partitionerIgnoreKeys = config.getBoolean(ProducerConfig.PARTITIONER_IGNORE_KEYS_CONFIG);
this.apiVersions = new ApiVersions(); this.apiVersions = new ApiVersions();
@ -548,6 +549,29 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
apiVersions); apiVersions);
} }
private static Compression configureCompression(ProducerConfig config) {
CompressionType type = CompressionType.forName(config.getString(ProducerConfig.COMPRESSION_TYPE_CONFIG));
switch (type) {
case GZIP: {
return Compression.gzip()
.level(config.getInt(ProducerConfig.COMPRESSION_GZIP_LEVEL_CONFIG))
.build();
}
case LZ4: {
return Compression.lz4()
.level(config.getInt(ProducerConfig.COMPRESSION_LZ4_LEVEL_CONFIG))
.build();
}
case ZSTD: {
return Compression.zstd()
.level(config.getInt(ProducerConfig.COMPRESSION_ZSTD_LEVEL_CONFIG))
.build();
}
default:
return Compression.of(type).build();
}
}
private static int lingerMs(ProducerConfig config) { private static int lingerMs(ProducerConfig config) {
return (int) Math.min(config.getLong(ProducerConfig.LINGER_MS_CONFIG), Integer.MAX_VALUE); return (int) Math.min(config.getLong(ProducerConfig.LINGER_MS_CONFIG), Integer.MAX_VALUE);
} }
@ -1033,7 +1057,7 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
Header[] headers = record.headers().toArray(); Header[] headers = record.headers().toArray();
int serializedSize = AbstractRecords.estimateSizeInBytesUpperBound(apiVersions.maxUsableProduceMagic(), int serializedSize = AbstractRecords.estimateSizeInBytesUpperBound(apiVersions.maxUsableProduceMagic(),
compressionType, serializedKey, serializedValue, headers); compression.type(), serializedKey, serializedValue, headers);
ensureValidRecordSize(serializedSize); ensureValidRecordSize(serializedSize);
long timestamp = record.timestamp() == null ? nowMs : record.timestamp(); long timestamp = record.timestamp() == null ? nowMs : record.timestamp();

View File

@ -18,6 +18,9 @@ package org.apache.kafka.clients.producer;
import org.apache.kafka.clients.ClientDnsLookup; import org.apache.kafka.clients.ClientDnsLookup;
import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.common.compress.GzipCompression;
import org.apache.kafka.common.compress.Lz4Compression;
import org.apache.kafka.common.compress.ZstdCompression;
import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.AbstractConfig;
import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.config.ConfigDef.Importance; import org.apache.kafka.common.config.ConfigDef.Importance;
@ -225,6 +228,18 @@ public class ProducerConfig extends AbstractConfig {
+ " values are <code>none</code>, <code>gzip</code>, <code>snappy</code>, <code>lz4</code>, or <code>zstd</code>. " + " values are <code>none</code>, <code>gzip</code>, <code>snappy</code>, <code>lz4</code>, or <code>zstd</code>. "
+ "Compression is of full batches of data, so the efficacy of batching will also impact the compression ratio (more batching means better compression)."; + "Compression is of full batches of data, so the efficacy of batching will also impact the compression ratio (more batching means better compression).";
/** <code>compression.gzip.level</code> */
public static final String COMPRESSION_GZIP_LEVEL_CONFIG = "compression.gzip.level";
private static final String COMPRESSION_GZIP_LEVEL_DOC = "The compression level to use if " + COMPRESSION_TYPE_CONFIG + " is set to <code>gzip</code>.";
/** <code>compression.lz4.level</code> */
public static final String COMPRESSION_LZ4_LEVEL_CONFIG = "compression.lz4.level";
private static final String COMPRESSION_LZ4_LEVEL_DOC = "The compression level to use if " + COMPRESSION_TYPE_CONFIG + " is set to <code>lz4</code>.";
/** <code>compression.zstd.level</code> */
public static final String COMPRESSION_ZSTD_LEVEL_CONFIG = "compression.zstd.level";
private static final String COMPRESSION_ZSTD_LEVEL_DOC = "The compression level to use if " + COMPRESSION_TYPE_CONFIG + " is set to <code>zstd</code>.";
/** <code>metrics.sample.window.ms</code> */ /** <code>metrics.sample.window.ms</code> */
public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG; public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG;
@ -364,6 +379,9 @@ public class ProducerConfig extends AbstractConfig {
Importance.LOW, Importance.LOW,
ACKS_DOC) ACKS_DOC)
.define(COMPRESSION_TYPE_CONFIG, Type.STRING, CompressionType.NONE.name, in(Utils.enumOptions(CompressionType.class)), Importance.HIGH, COMPRESSION_TYPE_DOC) .define(COMPRESSION_TYPE_CONFIG, Type.STRING, CompressionType.NONE.name, in(Utils.enumOptions(CompressionType.class)), Importance.HIGH, COMPRESSION_TYPE_DOC)
.define(COMPRESSION_GZIP_LEVEL_CONFIG, Type.INT, GzipCompression.DEFAULT_LEVEL, new GzipCompression.LevelValidator(), Importance.MEDIUM, COMPRESSION_GZIP_LEVEL_DOC)
.define(COMPRESSION_LZ4_LEVEL_CONFIG, Type.INT, Lz4Compression.DEFAULT_LEVEL, between(Lz4Compression.MIN_LEVEL, Lz4Compression.MAX_LEVEL), Importance.MEDIUM, COMPRESSION_LZ4_LEVEL_DOC)
.define(COMPRESSION_ZSTD_LEVEL_CONFIG, Type.INT, ZstdCompression.DEFAULT_LEVEL, between(ZstdCompression.MIN_LEVEL, ZstdCompression.MAX_LEVEL), Importance.MEDIUM, COMPRESSION_ZSTD_LEVEL_DOC)
.define(BATCH_SIZE_CONFIG, Type.INT, 16384, atLeast(0), Importance.MEDIUM, BATCH_SIZE_DOC) .define(BATCH_SIZE_CONFIG, Type.INT, 16384, atLeast(0), Importance.MEDIUM, BATCH_SIZE_DOC)
.define(PARTITIONER_ADPATIVE_PARTITIONING_ENABLE_CONFIG, Type.BOOLEAN, true, Importance.LOW, PARTITIONER_ADPATIVE_PARTITIONING_ENABLE_DOC) .define(PARTITIONER_ADPATIVE_PARTITIONING_ENABLE_CONFIG, Type.BOOLEAN, true, Importance.LOW, PARTITIONER_ADPATIVE_PARTITIONING_ENABLE_DOC)
.define(PARTITIONER_AVAILABILITY_TIMEOUT_MS_CONFIG, Type.LONG, 0, atLeast(0), Importance.LOW, PARTITIONER_AVAILABILITY_TIMEOUT_MS_DOC) .define(PARTITIONER_AVAILABILITY_TIMEOUT_MS_CONFIG, Type.LONG, 0, atLeast(0), Importance.LOW, PARTITIONER_AVAILABILITY_TIMEOUT_MS_DOC)

View File

@ -99,7 +99,7 @@ public final class ProducerBatch {
this.retry = false; this.retry = false;
this.isSplitBatch = isSplitBatch; this.isSplitBatch = isSplitBatch;
float compressionRatioEstimation = CompressionRatioEstimator.estimation(topicPartition.topic(), float compressionRatioEstimation = CompressionRatioEstimator.estimation(topicPartition.topic(),
recordsBuilder.compressionType()); recordsBuilder.compression().type());
this.currentLeaderEpoch = OptionalInt.empty(); this.currentLeaderEpoch = OptionalInt.empty();
this.attemptsWhenLeaderLastChanged = 0; this.attemptsWhenLeaderLastChanged = 0;
recordsBuilder.setEstimatedCompressionRatio(compressionRatioEstimation); recordsBuilder.setEstimatedCompressionRatio(compressionRatioEstimation);
@ -146,7 +146,7 @@ public final class ProducerBatch {
} else { } else {
this.recordsBuilder.append(timestamp, key, value, headers); this.recordsBuilder.append(timestamp, key, value, headers);
this.maxRecordSize = Math.max(this.maxRecordSize, AbstractRecords.estimateSizeInBytesUpperBound(magic(), this.maxRecordSize = Math.max(this.maxRecordSize, AbstractRecords.estimateSizeInBytesUpperBound(magic(),
recordsBuilder.compressionType(), key, value, headers)); recordsBuilder.compression().type(), key, value, headers));
this.lastAppendTime = now; this.lastAppendTime = now;
FutureRecordMetadata future = new FutureRecordMetadata(this.produceFuture, this.recordCount, FutureRecordMetadata future = new FutureRecordMetadata(this.produceFuture, this.recordCount,
timestamp, timestamp,
@ -172,7 +172,7 @@ public final class ProducerBatch {
// No need to get the CRC. // No need to get the CRC.
this.recordsBuilder.append(timestamp, key, value, headers); this.recordsBuilder.append(timestamp, key, value, headers);
this.maxRecordSize = Math.max(this.maxRecordSize, AbstractRecords.estimateSizeInBytesUpperBound(magic(), this.maxRecordSize = Math.max(this.maxRecordSize, AbstractRecords.estimateSizeInBytesUpperBound(magic(),
recordsBuilder.compressionType(), key, value, headers)); recordsBuilder.compression().type(), key, value, headers));
FutureRecordMetadata future = new FutureRecordMetadata(this.produceFuture, this.recordCount, FutureRecordMetadata future = new FutureRecordMetadata(this.produceFuture, this.recordCount,
timestamp, timestamp,
key == null ? -1 : key.remaining(), key == null ? -1 : key.remaining(),
@ -377,19 +377,19 @@ public final class ProducerBatch {
private ProducerBatch createBatchOffAccumulatorForRecord(Record record, int batchSize) { private ProducerBatch createBatchOffAccumulatorForRecord(Record record, int batchSize) {
int initialSize = Math.max(AbstractRecords.estimateSizeInBytesUpperBound(magic(), int initialSize = Math.max(AbstractRecords.estimateSizeInBytesUpperBound(magic(),
recordsBuilder.compressionType(), record.key(), record.value(), record.headers()), batchSize); recordsBuilder.compression().type(), record.key(), record.value(), record.headers()), batchSize);
ByteBuffer buffer = ByteBuffer.allocate(initialSize); ByteBuffer buffer = ByteBuffer.allocate(initialSize);
// Note that we intentionally do not set producer state (producerId, epoch, sequence, and isTransactional) // Note that we intentionally do not set producer state (producerId, epoch, sequence, and isTransactional)
// for the newly created batch. This will be set when the batch is dequeued for sending (which is consistent // for the newly created batch. This will be set when the batch is dequeued for sending (which is consistent
// with how normal batches are handled). // with how normal batches are handled).
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic(), recordsBuilder.compressionType(), MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic(), recordsBuilder.compression(),
TimestampType.CREATE_TIME, 0L); TimestampType.CREATE_TIME, 0L);
return new ProducerBatch(topicPartition, builder, this.createdMs, true); return new ProducerBatch(topicPartition, builder, this.createdMs, true);
} }
public boolean isCompressed() { public boolean isCompressed() {
return recordsBuilder.compressionType() != CompressionType.NONE; return recordsBuilder.compression().type() != CompressionType.NONE;
} }
/** /**
@ -491,7 +491,7 @@ public final class ProducerBatch {
recordsBuilder.close(); recordsBuilder.close();
if (!recordsBuilder.isControlBatch()) { if (!recordsBuilder.isControlBatch()) {
CompressionRatioEstimator.updateEstimation(topicPartition.topic(), CompressionRatioEstimator.updateEstimation(topicPartition.topic(),
recordsBuilder.compressionType(), recordsBuilder.compression().type(),
(float) recordsBuilder.compressionRatio()); (float) recordsBuilder.compressionRatio());
} }
reopened = false; reopened = false;

View File

@ -36,6 +36,7 @@ import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.clients.MetadataSnapshot; import org.apache.kafka.clients.MetadataSnapshot;
import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.Callback;
import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.utils.ExponentialBackoff; import org.apache.kafka.common.utils.ExponentialBackoff;
import org.apache.kafka.common.utils.ProducerIdAndEpoch; import org.apache.kafka.common.utils.ProducerIdAndEpoch;
import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Cluster;
@ -48,7 +49,6 @@ import org.apache.kafka.common.header.Header;
import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.record.AbstractRecords; import org.apache.kafka.common.record.AbstractRecords;
import org.apache.kafka.common.record.CompressionRatioEstimator; import org.apache.kafka.common.record.CompressionRatioEstimator;
import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.MemoryRecordsBuilder; import org.apache.kafka.common.record.MemoryRecordsBuilder;
import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.Record;
@ -74,7 +74,7 @@ public class RecordAccumulator {
private final AtomicInteger flushesInProgress; private final AtomicInteger flushesInProgress;
private final AtomicInteger appendsInProgress; private final AtomicInteger appendsInProgress;
private final int batchSize; private final int batchSize;
private final CompressionType compression; private final Compression compression;
private final int lingerMs; private final int lingerMs;
private final ExponentialBackoff retryBackoff; private final ExponentialBackoff retryBackoff;
private final int deliveryTimeoutMs; private final int deliveryTimeoutMs;
@ -116,7 +116,7 @@ public class RecordAccumulator {
*/ */
public RecordAccumulator(LogContext logContext, public RecordAccumulator(LogContext logContext,
int batchSize, int batchSize,
CompressionType compression, Compression compression,
int lingerMs, int lingerMs,
long retryBackoffMs, long retryBackoffMs,
long retryBackoffMaxMs, long retryBackoffMaxMs,
@ -176,7 +176,7 @@ public class RecordAccumulator {
*/ */
public RecordAccumulator(LogContext logContext, public RecordAccumulator(LogContext logContext,
int batchSize, int batchSize,
CompressionType compression, Compression compression,
int lingerMs, int lingerMs,
long retryBackoffMs, long retryBackoffMs,
long retryBackoffMaxMs, long retryBackoffMaxMs,
@ -344,7 +344,7 @@ public class RecordAccumulator {
if (buffer == null) { if (buffer == null) {
byte maxUsableMagic = apiVersions.maxUsableProduceMagic(); byte maxUsableMagic = apiVersions.maxUsableProduceMagic();
int size = Math.max(this.batchSize, AbstractRecords.estimateSizeInBytesUpperBound(maxUsableMagic, compression, key, value, headers)); int size = Math.max(this.batchSize, AbstractRecords.estimateSizeInBytesUpperBound(maxUsableMagic, compression.type(), key, value, headers));
log.trace("Allocating a new {} byte message buffer for topic {} partition {} with remaining timeout {}ms", size, topic, partition, maxTimeToBlock); log.trace("Allocating a new {} byte message buffer for topic {} partition {} with remaining timeout {}ms", size, topic, partition, maxTimeToBlock);
// This call may block if we exhausted buffer space. // This call may block if we exhausted buffer space.
buffer = free.allocate(size, maxTimeToBlock); buffer = free.allocate(size, maxTimeToBlock);
@ -533,7 +533,7 @@ public class RecordAccumulator {
// Reset the estimated compression ratio to the initial value or the big batch compression ratio, whichever // Reset the estimated compression ratio to the initial value or the big batch compression ratio, whichever
// is bigger. There are several different ways to do the reset. We chose the most conservative one to ensure // is bigger. There are several different ways to do the reset. We chose the most conservative one to ensure
// the split doesn't happen too often. // the split doesn't happen too often.
CompressionRatioEstimator.setEstimation(bigBatch.topicPartition.topic(), compression, CompressionRatioEstimator.setEstimation(bigBatch.topicPartition.topic(), compression.type(),
Math.max(1.0f, (float) bigBatch.compressionRatio())); Math.max(1.0f, (float) bigBatch.compressionRatio()));
Deque<ProducerBatch> dq = bigBatch.split(this.batchSize); Deque<ProducerBatch> dq = bigBatch.split(this.batchSize);
int numSplitBatches = dq.size(); int numSplitBatches = dq.size();

View File

@ -0,0 +1,112 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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 org.apache.kafka.common.compress;
import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.utils.BufferSupplier;
import org.apache.kafka.common.utils.ByteBufferOutputStream;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.ByteBuffer;
public interface Compression {
/**
* The compression type for this compression codec
*/
CompressionType type();
/**
* Wrap bufferStream with an OutputStream that will compress data with this Compression.
*
* @param bufferStream The buffer to write the compressed data to
* @param messageVersion The record format version to use.
* Note: Unlike {@link #wrapForInput}, this cannot take {@link ByteBuffer}s directly.
* Currently, MemoryRecordsBuilder writes to the underlying buffer in the given {@link ByteBufferOutputStream} after the compressed data has been written.
* In the event that the buffer needs to be expanded while writing the data, access to the underlying buffer needs to be preserved.
*/
OutputStream wrapForOutput(ByteBufferOutputStream bufferStream, byte messageVersion);
/**
* Wrap buffer with an InputStream that will decompress data with this Compression.
*
* @param buffer The {@link ByteBuffer} instance holding the data to decompress.
* @param messageVersion The record format version to use.
* @param decompressionBufferSupplier The supplier of ByteBuffer(s) used for decompression if supported.
* For small record batches, allocating a potentially large buffer (64 KB for LZ4)
* will dominate the cost of decompressing and iterating over the records in the
* batch. As such, a supplier that reuses buffers will have a significant
* performance impact.
*/
InputStream wrapForInput(ByteBuffer buffer, byte messageVersion, BufferSupplier decompressionBufferSupplier);
/**
* Recommended size of buffer for storing decompressed output.
*/
default int decompressionOutputSize() {
throw new UnsupportedOperationException("Size of decompression buffer is not defined for this compression type=" + type().name);
}
interface Builder<T extends Compression> {
T build();
}
static Builder<? extends Compression> of(final String compressionName) {
CompressionType compressionType = CompressionType.forName(compressionName);
return of(compressionType);
}
static Builder<? extends Compression> of(final CompressionType compressionType) {
switch (compressionType) {
case NONE:
return none();
case GZIP:
return gzip();
case SNAPPY:
return snappy();
case LZ4:
return lz4();
case ZSTD:
return zstd();
default:
throw new IllegalArgumentException("Unknown compression type: " + compressionType.name);
}
}
NoCompression NONE = none().build();
static NoCompression.Builder none() {
return new NoCompression.Builder();
}
static GzipCompression.Builder gzip() {
return new GzipCompression.Builder();
}
static SnappyCompression.Builder snappy() {
return new SnappyCompression.Builder();
}
static Lz4Compression.Builder lz4() {
return new Lz4Compression.Builder();
}
static ZstdCompression.Builder zstd() {
return new ZstdCompression.Builder();
}
}

View File

@ -0,0 +1,138 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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 org.apache.kafka.common.compress;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.utils.BufferSupplier;
import org.apache.kafka.common.utils.ByteBufferInputStream;
import org.apache.kafka.common.utils.ByteBufferOutputStream;
import org.apache.kafka.common.utils.ChunkedBytesStream;
import java.io.BufferedOutputStream;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.ByteBuffer;
import java.util.Objects;
import java.util.zip.Deflater;
import java.util.zip.GZIPInputStream;
public class GzipCompression implements Compression {
public static final int MIN_LEVEL = Deflater.BEST_SPEED;
public static final int MAX_LEVEL = Deflater.BEST_COMPRESSION;
public static final int DEFAULT_LEVEL = Deflater.DEFAULT_COMPRESSION;
private final int level;
private GzipCompression(int level) {
this.level = level;
}
@Override
public CompressionType type() {
return CompressionType.GZIP;
}
@Override
public OutputStream wrapForOutput(ByteBufferOutputStream buffer, byte messageVersion) {
try {
// Set input buffer (uncompressed) to 16 KB (none by default) and output buffer (compressed) to
// 8 KB (0.5 KB by default) to ensure reasonable performance in cases where the caller passes a small
// number of bytes to write (potentially a single byte)
return new BufferedOutputStream(new GzipOutputStream(buffer, 8 * 1024, level), 16 * 1024);
} catch (Exception e) {
throw new KafkaException(e);
}
}
@Override
public InputStream wrapForInput(ByteBuffer buffer, byte messageVersion, BufferSupplier decompressionBufferSupplier) {
try {
// Set input buffer (compressed) to 8 KB (GZIPInputStream uses 0.5 KB by default) to ensure reasonable
// performance in cases where the caller reads a small number of bytes (potentially a single byte).
//
// Size of output buffer (uncompressed) is provided by decompressionOutputSize.
//
// ChunkedBytesStream is used to wrap the GZIPInputStream because the default implementation of
// GZIPInputStream does not use an intermediate buffer for decompression in chunks.
return new ChunkedBytesStream(new GZIPInputStream(new ByteBufferInputStream(buffer), 8 * 1024),
decompressionBufferSupplier,
decompressionOutputSize(),
false);
} catch (Exception e) {
throw new KafkaException(e);
}
}
@Override
public int decompressionOutputSize() {
// 16KB has been chosen based on legacy implementation introduced in https://github.com/apache/kafka/pull/6785
return 16 * 1024;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
GzipCompression that = (GzipCompression) o;
return level == that.level;
}
@Override
public int hashCode() {
return Objects.hash(level);
}
public static class Builder implements Compression.Builder<GzipCompression> {
private int level = DEFAULT_LEVEL;
public Builder level(int level) {
if ((level < MIN_LEVEL || MAX_LEVEL < level) && level != DEFAULT_LEVEL) {
throw new IllegalArgumentException("gzip doesn't support given compression level: " + level);
}
this.level = level;
return this;
}
@Override
public GzipCompression build() {
return new GzipCompression(level);
}
}
public static class LevelValidator implements ConfigDef.Validator {
@Override
public void ensureValid(String name, Object o) {
if (o == null)
throw new ConfigException(name, null, "Value must be non-null");
int level = ((Number) o).intValue();
if (level > MAX_LEVEL || (level < MIN_LEVEL && level != DEFAULT_LEVEL)) {
throw new ConfigException(name, o, "Value must be between " + MIN_LEVEL + " and " + MAX_LEVEL + " or equal to " + DEFAULT_LEVEL);
}
}
@Override
public String toString() {
return "[" + MIN_LEVEL + ",...," + MAX_LEVEL + "] or " + DEFAULT_LEVEL;
}
}
}

View File

@ -0,0 +1,48 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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 org.apache.kafka.common.compress;
import java.io.IOException;
import java.io.OutputStream;
import java.util.zip.GZIPOutputStream;
/**
* An extension of {@link GZIPOutputStream}, with compression level functionality.
*/
public class GzipOutputStream extends GZIPOutputStream {
/**
* Creates a new {@link OutputStream} with the specified buffer size and compression level.
*
* @param out the output stream
* @param size the output buffer size
* @param level the compression level
* @throws IOException If an I/O error has occurred.
*/
public GzipOutputStream(OutputStream out, int size, int level) throws IOException {
super(out, size);
setLevel(level);
}
/**
* Sets the compression level.
*
* @param level the compression level
*/
private void setLevel(int level) {
def.setLevel(level);
}
}

View File

@ -23,8 +23,8 @@ import net.jpountz.lz4.LZ4SafeDecompressor;
import net.jpountz.xxhash.XXHash32; import net.jpountz.xxhash.XXHash32;
import net.jpountz.xxhash.XXHashFactory; import net.jpountz.xxhash.XXHashFactory;
import org.apache.kafka.common.compress.KafkaLZ4BlockOutputStream.BD; import org.apache.kafka.common.compress.Lz4BlockOutputStream.BD;
import org.apache.kafka.common.compress.KafkaLZ4BlockOutputStream.FLG; import org.apache.kafka.common.compress.Lz4BlockOutputStream.FLG;
import org.apache.kafka.common.utils.BufferSupplier; import org.apache.kafka.common.utils.BufferSupplier;
import java.io.IOException; import java.io.IOException;
@ -32,8 +32,8 @@ import java.io.InputStream;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.nio.ByteOrder; import java.nio.ByteOrder;
import static org.apache.kafka.common.compress.KafkaLZ4BlockOutputStream.LZ4_FRAME_INCOMPRESSIBLE_MASK; import static org.apache.kafka.common.compress.Lz4BlockOutputStream.LZ4_FRAME_INCOMPRESSIBLE_MASK;
import static org.apache.kafka.common.compress.KafkaLZ4BlockOutputStream.MAGIC; import static org.apache.kafka.common.compress.Lz4BlockOutputStream.MAGIC;
/** /**
* A partial implementation of the v1.5.1 LZ4 Frame format. * A partial implementation of the v1.5.1 LZ4 Frame format.
@ -42,7 +42,7 @@ import static org.apache.kafka.common.compress.KafkaLZ4BlockOutputStream.MAGIC;
* *
* This class is not thread-safe. * This class is not thread-safe.
*/ */
public final class KafkaLZ4BlockInputStream extends InputStream { public final class Lz4BlockInputStream extends InputStream {
public static final String PREMATURE_EOS = "Stream ended prematurely"; public static final String PREMATURE_EOS = "Stream ended prematurely";
public static final String NOT_SUPPORTED = "Stream unsupported (invalid magic bytes)"; public static final String NOT_SUPPORTED = "Stream unsupported (invalid magic bytes)";
@ -86,7 +86,7 @@ public final class KafkaLZ4BlockInputStream extends InputStream {
* @param ignoreFlagDescriptorChecksum for compatibility with old kafka clients, ignore incorrect HC byte * @param ignoreFlagDescriptorChecksum for compatibility with old kafka clients, ignore incorrect HC byte
* @throws IOException * @throws IOException
*/ */
public KafkaLZ4BlockInputStream(ByteBuffer in, BufferSupplier bufferSupplier, boolean ignoreFlagDescriptorChecksum) throws IOException { public Lz4BlockInputStream(ByteBuffer in, BufferSupplier bufferSupplier, boolean ignoreFlagDescriptorChecksum) throws IOException {
if (BROKEN_LZ4_EXCEPTION != null) { if (BROKEN_LZ4_EXCEPTION != null) {
throw BROKEN_LZ4_EXCEPTION; throw BROKEN_LZ4_EXCEPTION;
} }

View File

@ -33,18 +33,14 @@ import net.jpountz.xxhash.XXHashFactory;
* *
* This class is not thread-safe. * This class is not thread-safe.
*/ */
public final class KafkaLZ4BlockOutputStream extends OutputStream { public final class Lz4BlockOutputStream extends OutputStream {
public static final int MAGIC = 0x184D2204; public static final int MAGIC = 0x184D2204;
public static final int LZ4_MAX_HEADER_LENGTH = 19;
public static final int LZ4_FRAME_INCOMPRESSIBLE_MASK = 0x80000000; public static final int LZ4_FRAME_INCOMPRESSIBLE_MASK = 0x80000000;
public static final String CLOSED_STREAM = "The stream is already closed"; public static final String CLOSED_STREAM = "The stream is already closed";
public static final int BLOCKSIZE_64KB = 4; public static final int BLOCKSIZE_64KB = 4;
public static final int BLOCKSIZE_256KB = 5;
public static final int BLOCKSIZE_1MB = 6;
public static final int BLOCKSIZE_4MB = 7;
private final LZ4Compressor compressor; private final LZ4Compressor compressor;
private final XXHash32 checksum; private final XXHash32 checksum;
@ -64,15 +60,22 @@ public final class KafkaLZ4BlockOutputStream extends OutputStream {
* @param out The output stream to compress * @param out The output stream to compress
* @param blockSize Default: 4. The block size used during compression. 4=64kb, 5=256kb, 6=1mb, 7=4mb. All other * @param blockSize Default: 4. The block size used during compression. 4=64kb, 5=256kb, 6=1mb, 7=4mb. All other
* values will generate an exception * values will generate an exception
* @param level The compression level to use
* @param blockChecksum Default: false. When true, a XXHash32 checksum is computed and appended to the stream for * @param blockChecksum Default: false. When true, a XXHash32 checksum is computed and appended to the stream for
* every block of data * every block of data
* @param useBrokenFlagDescriptorChecksum Default: false. When true, writes an incorrect FrameDescriptor checksum * @param useBrokenFlagDescriptorChecksum Default: false. When true, writes an incorrect FrameDescriptor checksum
* compatible with older kafka clients. * compatible with older kafka clients.
* @throws IOException * @throws IOException
*/ */
public KafkaLZ4BlockOutputStream(OutputStream out, int blockSize, boolean blockChecksum, boolean useBrokenFlagDescriptorChecksum) throws IOException { public Lz4BlockOutputStream(OutputStream out, int blockSize, int level, boolean blockChecksum, boolean useBrokenFlagDescriptorChecksum) throws IOException {
this.out = out; this.out = out;
compressor = LZ4Factory.fastestInstance().fastCompressor(); /*
* lz4-java provides two types of compressors; fastCompressor, which requires less memory but fast compression speed (with default compression level only),
* and highCompressor which requires more memory and slower speed but compresses more efficiently (with various compression level).
*
* For backward compatibility, Lz4BlockOutputStream uses fastCompressor with default compression level but, with the other level, it uses highCompressor.
*/
compressor = level == Lz4Compression.DEFAULT_LEVEL ? LZ4Factory.fastestInstance().fastCompressor() : LZ4Factory.fastestInstance().highCompressor(level);
checksum = XXHashFactory.fastestInstance().hash32(); checksum = XXHashFactory.fastestInstance().hash32();
this.useBrokenFlagDescriptorChecksum = useBrokenFlagDescriptorChecksum; this.useBrokenFlagDescriptorChecksum = useBrokenFlagDescriptorChecksum;
bd = new BD(blockSize); bd = new BD(blockSize);
@ -89,40 +92,14 @@ public final class KafkaLZ4BlockOutputStream extends OutputStream {
* Create a new {@link OutputStream} that will compress data using the LZ4 algorithm. * Create a new {@link OutputStream} that will compress data using the LZ4 algorithm.
* *
* @param out The output stream to compress * @param out The output stream to compress
* @param blockSize Default: 4. The block size used during compression. 4=64kb, 5=256kb, 6=1mb, 7=4mb. All other * @param level The compression level to use
* values will generate an exception * @param useBrokenFlagDescriptorChecksum Default: false. When true, writes an incorrect FrameDescriptor checksum
* @param blockChecksum Default: false. When true, a XXHash32 checksum is computed and appended to the stream for * compatible with older kafka clients.
* every block of data
* @throws IOException * @throws IOException
*/ */
public KafkaLZ4BlockOutputStream(OutputStream out, int blockSize, boolean blockChecksum) throws IOException {
this(out, blockSize, blockChecksum, false);
}
/** public Lz4BlockOutputStream(OutputStream out, int level, boolean useBrokenFlagDescriptorChecksum) throws IOException {
* Create a new {@link OutputStream} that will compress data using the LZ4 algorithm. this(out, BLOCKSIZE_64KB, level, false, useBrokenFlagDescriptorChecksum);
*
* @param out The stream to compress
* @param blockSize Default: 4. The block size used during compression. 4=64kb, 5=256kb, 6=1mb, 7=4mb. All other
* values will generate an exception
* @throws IOException
*/
public KafkaLZ4BlockOutputStream(OutputStream out, int blockSize) throws IOException {
this(out, blockSize, false, false);
}
/**
* Create a new {@link OutputStream} that will compress data using the LZ4 algorithm.
*
* @param out The output stream to compress
* @throws IOException
*/
public KafkaLZ4BlockOutputStream(OutputStream out) throws IOException {
this(out, BLOCKSIZE_64KB);
}
public KafkaLZ4BlockOutputStream(OutputStream out, boolean useBrokenHC) throws IOException {
this(out, BLOCKSIZE_64KB, false, useBrokenHC);
} }
/** /**
@ -292,10 +269,6 @@ public final class KafkaLZ4BlockOutputStream extends OutputStream {
private final int blockIndependence; private final int blockIndependence;
private final int version; private final int version;
public FLG() {
this(false);
}
public FLG(boolean blockChecksum) { public FLG(boolean blockChecksum) {
this(0, 0, 0, blockChecksum ? 1 : 0, 1, VERSION); this(0, 0, 0, blockChecksum ? 1 : 0, 1, VERSION);
} }
@ -375,10 +348,6 @@ public final class KafkaLZ4BlockOutputStream extends OutputStream {
private final int blockSizeValue; private final int blockSizeValue;
private final int reserved3; private final int reserved3;
public BD() {
this(0, BLOCKSIZE_64KB, 0);
}
public BD(int blockSizeValue) { public BD(int blockSizeValue) {
this(0, blockSizeValue, 0); this(0, blockSizeValue, 0);
} }

View File

@ -0,0 +1,108 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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 org.apache.kafka.common.compress;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.record.RecordBatch;
import org.apache.kafka.common.utils.BufferSupplier;
import org.apache.kafka.common.utils.ByteBufferOutputStream;
import org.apache.kafka.common.utils.ChunkedBytesStream;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.ByteBuffer;
import java.util.Objects;
public class Lz4Compression implements Compression {
// These values come from net.jpountz.lz4.LZ4Constants
// We may need to update them if the lz4 library changes these values.
public static final int MIN_LEVEL = 1;
public static final int MAX_LEVEL = 17;
public static final int DEFAULT_LEVEL = 9;
private final int level;
private Lz4Compression(int level) {
this.level = level;
}
@Override
public CompressionType type() {
return CompressionType.LZ4;
}
@Override
public OutputStream wrapForOutput(ByteBufferOutputStream buffer, byte messageVersion) {
try {
return new Lz4BlockOutputStream(buffer, level, messageVersion == RecordBatch.MAGIC_VALUE_V0);
} catch (Throwable e) {
throw new KafkaException(e);
}
}
@Override
public InputStream wrapForInput(ByteBuffer inputBuffer, byte messageVersion, BufferSupplier decompressionBufferSupplier) {
try {
return new ChunkedBytesStream(
new Lz4BlockInputStream(inputBuffer, decompressionBufferSupplier, messageVersion == RecordBatch.MAGIC_VALUE_V0),
decompressionBufferSupplier, decompressionOutputSize(), true);
} catch (Throwable e) {
throw new KafkaException(e);
}
}
@Override
public int decompressionOutputSize() {
// Lz4BlockInputStream uses an internal intermediate buffer to store decompressed data. The size
// of this buffer is based on legacy implementation based on skipArray introduced in
// https://github.com/apache/kafka/pull/6785
return 2 * 1024; // 2KB
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
Lz4Compression that = (Lz4Compression) o;
return level == that.level;
}
@Override
public int hashCode() {
return Objects.hash(level);
}
public static class Builder implements Compression.Builder<Lz4Compression> {
private int level = DEFAULT_LEVEL;
public Builder level(int level) {
if (level < MIN_LEVEL || MAX_LEVEL < level) {
throw new IllegalArgumentException("lz4 doesn't support given compression level: " + level);
}
this.level = level;
return this;
}
@Override
public Lz4Compression build() {
return new Lz4Compression(level);
}
}
}

View File

@ -14,37 +14,41 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.kafka.common.compress; package org.apache.kafka.common.compress;
import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.utils.BufferSupplier;
import org.apache.kafka.common.utils.ByteBufferInputStream; import org.apache.kafka.common.utils.ByteBufferInputStream;
import org.apache.kafka.common.utils.ByteBufferOutputStream; import org.apache.kafka.common.utils.ByteBufferOutputStream;
import org.xerial.snappy.SnappyInputStream;
import org.xerial.snappy.SnappyOutputStream;
import java.io.InputStream; import java.io.InputStream;
import java.io.OutputStream; import java.io.OutputStream;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
public class SnappyFactory { public class NoCompression implements Compression {
private SnappyFactory() { } private NoCompression() {}
public static OutputStream wrapForOutput(ByteBufferOutputStream buffer) { @Override
try { public CompressionType type() {
return new SnappyOutputStream(buffer); return CompressionType.NONE;
} catch (Throwable e) {
throw new KafkaException(e);
}
} }
public static InputStream wrapForInput(ByteBuffer buffer) { @Override
try { public OutputStream wrapForOutput(ByteBufferOutputStream bufferStream, byte messageVersion) {
return new SnappyInputStream(new ByteBufferInputStream(buffer)); return bufferStream;
} catch (Throwable e) {
throw new KafkaException(e);
}
} }
@Override
public InputStream wrapForInput(ByteBuffer buffer, byte messageVersion, BufferSupplier decompressionBufferSupplier) {
return new ByteBufferInputStream(buffer);
}
public static class Builder implements Compression.Builder<NoCompression> {
@Override
public NoCompression build() {
return new NoCompression();
}
}
} }

View File

@ -0,0 +1,91 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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 org.apache.kafka.common.compress;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.utils.BufferSupplier;
import org.apache.kafka.common.utils.ByteBufferInputStream;
import org.apache.kafka.common.utils.ByteBufferOutputStream;
import org.apache.kafka.common.utils.ChunkedBytesStream;
import org.xerial.snappy.SnappyInputStream;
import org.xerial.snappy.SnappyOutputStream;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.ByteBuffer;
public class SnappyCompression implements Compression {
private SnappyCompression() {}
@Override
public CompressionType type() {
return CompressionType.SNAPPY;
}
@Override
public OutputStream wrapForOutput(ByteBufferOutputStream bufferStream, byte messageVersion) {
try {
return new SnappyOutputStream(bufferStream);
} catch (Throwable e) {
throw new KafkaException(e);
}
}
@Override
public InputStream wrapForInput(ByteBuffer buffer, byte messageVersion, BufferSupplier decompressionBufferSupplier) {
// SnappyInputStream uses default implementation of InputStream for skip. Default implementation of
// SnappyInputStream allocates a new skip buffer every time, hence, we prefer our own implementation.
try {
return new ChunkedBytesStream(new SnappyInputStream(new ByteBufferInputStream(buffer)),
decompressionBufferSupplier,
decompressionOutputSize(),
false);
} catch (Throwable e) {
throw new KafkaException(e);
}
}
@Override
public int decompressionOutputSize() {
// SnappyInputStream already uses an intermediate buffer internally. The size
// of this buffer is based on legacy implementation based on skipArray introduced in
// https://github.com/apache/kafka/pull/6785
return 2 * 1024; // 2KB
}
@Override
public boolean equals(Object o) {
return o instanceof SnappyCompression;
}
@Override
public int hashCode() {
return super.hashCode();
}
public static class Builder implements Compression.Builder<SnappyCompression> {
@Override
public SnappyCompression build() {
return new SnappyCompression();
}
}
}

View File

@ -0,0 +1,143 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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 org.apache.kafka.common.compress;
import com.github.luben.zstd.BufferPool;
import com.github.luben.zstd.RecyclingBufferPool;
import com.github.luben.zstd.Zstd;
import com.github.luben.zstd.ZstdInputStreamNoFinalizer;
import com.github.luben.zstd.ZstdOutputStreamNoFinalizer;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.utils.BufferSupplier;
import org.apache.kafka.common.utils.ByteBufferInputStream;
import org.apache.kafka.common.utils.ByteBufferOutputStream;
import org.apache.kafka.common.utils.ChunkedBytesStream;
import java.io.BufferedOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.ByteBuffer;
import java.util.Objects;
public class ZstdCompression implements Compression {
public static final int MIN_LEVEL = Zstd.minCompressionLevel();
public static final int MAX_LEVEL = Zstd.maxCompressionLevel();
public static final int DEFAULT_LEVEL = Zstd.defaultCompressionLevel();
private final int level;
private ZstdCompression(int level) {
this.level = level;
}
@Override
public CompressionType type() {
return CompressionType.ZSTD;
}
@Override
public OutputStream wrapForOutput(ByteBufferOutputStream bufferStream, byte messageVersion) {
try {
// Set input buffer (uncompressed) to 16 KB (none by default) to ensure reasonable performance
// in cases where the caller passes a small number of bytes to write (potentially a single byte).
return new BufferedOutputStream(new ZstdOutputStreamNoFinalizer(bufferStream, RecyclingBufferPool.INSTANCE, level), 16 * 1024);
} catch (Throwable e) {
throw new KafkaException(e);
}
}
@Override
public InputStream wrapForInput(ByteBuffer buffer, byte messageVersion, BufferSupplier decompressionBufferSupplier) {
try {
return new ChunkedBytesStream(wrapForZstdInput(buffer, decompressionBufferSupplier),
decompressionBufferSupplier,
decompressionOutputSize(),
false);
} catch (Throwable e) {
throw new KafkaException(e);
}
}
// visible for testing
public static ZstdInputStreamNoFinalizer wrapForZstdInput(ByteBuffer buffer, BufferSupplier decompressionBufferSupplier) throws IOException {
// We use our own BufferSupplier instead of com.github.luben.zstd.RecyclingBufferPool since our
// implementation doesn't require locking or soft references. The buffer allocated by this buffer pool is
// used by zstd-jni for 1\ reading compressed data from input stream into a buffer before passing it over JNI
// 2\ implementation of skip inside zstd-jni where buffer is obtained and released with every call
final BufferPool bufferPool = new BufferPool() {
@Override
public ByteBuffer get(int capacity) {
return decompressionBufferSupplier.get(capacity);
}
@Override
public void release(ByteBuffer buffer) {
decompressionBufferSupplier.release(buffer);
}
};
// Ideally, data from ZstdInputStreamNoFinalizer should be read in a bulk because every call to
// `ZstdInputStreamNoFinalizer#read()` is a JNI call. The caller is expected to
// balance the tradeoff between reading large amount of data vs. making multiple JNI calls.
return new ZstdInputStreamNoFinalizer(new ByteBufferInputStream(buffer), bufferPool);
}
/**
* Size of intermediate buffer which contains uncompressed data.
* This size should be <= ZSTD_BLOCKSIZE_MAX
* see: https://github.com/facebook/zstd/blob/189653a9c10c9f4224a5413a6d6a69dd01d7c3bd/lib/zstd.h#L854
*/
@Override
public int decompressionOutputSize() {
// 16KB has been chosen based on legacy implementation introduced in https://github.com/apache/kafka/pull/6785
return 16 * 1024;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
ZstdCompression that = (ZstdCompression) o;
return level == that.level;
}
@Override
public int hashCode() {
return Objects.hash(level);
}
public static class Builder implements Compression.Builder<ZstdCompression> {
private int level = DEFAULT_LEVEL;
public Builder level(int level) {
if (MAX_LEVEL < level || level < MIN_LEVEL) {
throw new IllegalArgumentException("zstd doesn't support given compression level: " + level);
}
this.level = level;
return this;
}
@Override
public ZstdCompression build() {
return new ZstdCompression(level);
}
}
}

View File

@ -1,73 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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 org.apache.kafka.common.compress;
import com.github.luben.zstd.BufferPool;
import com.github.luben.zstd.RecyclingBufferPool;
import com.github.luben.zstd.ZstdInputStreamNoFinalizer;
import com.github.luben.zstd.ZstdOutputStreamNoFinalizer;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.utils.BufferSupplier;
import org.apache.kafka.common.utils.ByteBufferInputStream;
import org.apache.kafka.common.utils.ByteBufferOutputStream;
import java.io.BufferedOutputStream;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.ByteBuffer;
public class ZstdFactory {
private ZstdFactory() { }
public static OutputStream wrapForOutput(ByteBufferOutputStream buffer) {
try {
// Set input buffer (uncompressed) to 16 KB (none by default) to ensure reasonable performance
// in cases where the caller passes a small number of bytes to write (potentially a single byte).
return new BufferedOutputStream(new ZstdOutputStreamNoFinalizer(buffer, RecyclingBufferPool.INSTANCE), 16 * 1024);
} catch (Throwable e) {
throw new KafkaException(e);
}
}
public static InputStream wrapForInput(ByteBuffer buffer, byte messageVersion, BufferSupplier decompressionBufferSupplier) {
try {
// We use our own BufferSupplier instead of com.github.luben.zstd.RecyclingBufferPool since our
// implementation doesn't require locking or soft references. The buffer allocated by this buffer pool is
// used by zstd-jni for 1\ reading compressed data from input stream into a buffer before passing it over JNI
// 2\ implementation of skip inside zstd-jni where buffer is obtained and released with every call
final BufferPool bufferPool = new BufferPool() {
@Override
public ByteBuffer get(int capacity) {
return decompressionBufferSupplier.get(capacity);
}
@Override
public void release(ByteBuffer buffer) {
decompressionBufferSupplier.release(buffer);
}
};
// Ideally, data from ZstdInputStreamNoFinalizer should be read in a bulk because every call to
// `ZstdInputStreamNoFinalizer#read()` is a JNI call. The caller is expected to
// balance the tradeoff between reading large amount of data vs. making multiple JNI calls.
return new ZstdInputStreamNoFinalizer(new ByteBufferInputStream(buffer), bufferPool);
} catch (Throwable e) {
throw new KafkaException(e);
}
}
}

View File

@ -173,6 +173,14 @@ public class TopicConfig {
"accepts 'uncompressed' which is equivalent to no compression; and 'producer' which means retain the " + "accepts 'uncompressed' which is equivalent to no compression; and 'producer' which means retain the " +
"original compression codec set by the producer."; "original compression codec set by the producer.";
public static final String COMPRESSION_GZIP_LEVEL_CONFIG = "compression.gzip.level";
public static final String COMPRESSION_GZIP_LEVEL_DOC = "The compression level to use if " + COMPRESSION_TYPE_CONFIG + " is set to <code>gzip</code>.";
public static final String COMPRESSION_LZ4_LEVEL_CONFIG = "compression.lz4.level";
public static final String COMPRESSION_LZ4_LEVEL_DOC = "The compression level to use if " + COMPRESSION_TYPE_CONFIG + " is set to <code>lz4</code>.";
public static final String COMPRESSION_ZSTD_LEVEL_CONFIG = "compression.zstd.level";
public static final String COMPRESSION_ZSTD_LEVEL_DOC = "The compression level to use if " + COMPRESSION_TYPE_CONFIG + " is set to <code>zstd</code>.";
public static final String PREALLOCATE_CONFIG = "preallocate"; public static final String PREALLOCATE_CONFIG = "preallocate";
public static final String PREALLOCATE_DOC = "True if we should preallocate the file on disk when " + public static final String PREALLOCATE_DOC = "True if we should preallocate the file on disk when " +
"creating a new log segment."; "creating a new log segment.";

View File

@ -18,6 +18,7 @@ package org.apache.kafka.common.record;
import org.apache.kafka.common.InvalidRecordException; import org.apache.kafka.common.InvalidRecordException;
import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.errors.CorruptRecordException; import org.apache.kafka.common.errors.CorruptRecordException;
import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.Header;
import org.apache.kafka.common.utils.AbstractIterator; import org.apache.kafka.common.utils.AbstractIterator;
@ -332,7 +333,7 @@ public abstract class AbstractLegacyRecordBatch extends AbstractRecordBatch impl
throw new InvalidRecordException("Found invalid compressed record set with null value (magic = " + throw new InvalidRecordException("Found invalid compressed record set with null value (magic = " +
wrapperMagic + ")"); wrapperMagic + ")");
InputStream stream = compressionType.wrapForInput(wrapperValue, wrapperRecord.magic(), bufferSupplier); InputStream stream = Compression.of(compressionType).build().wrapForInput(wrapperValue, wrapperRecord.magic(), bufferSupplier);
LogInputStream<AbstractLegacyRecordBatch> logStream = new DataLogInputStream(stream, maxMessageSize); LogInputStream<AbstractLegacyRecordBatch> logStream = new DataLogInputStream(stream, maxMessageSize);
long lastOffsetFromWrapper = wrapperEntry.lastOffset(); long lastOffsetFromWrapper = wrapperEntry.lastOffset();

View File

@ -16,158 +16,23 @@
*/ */
package org.apache.kafka.common.record; package org.apache.kafka.common.record;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.compress.KafkaLZ4BlockInputStream;
import org.apache.kafka.common.compress.KafkaLZ4BlockOutputStream;
import org.apache.kafka.common.compress.SnappyFactory;
import org.apache.kafka.common.compress.ZstdFactory;
import org.apache.kafka.common.utils.BufferSupplier;
import org.apache.kafka.common.utils.ByteBufferInputStream;
import org.apache.kafka.common.utils.ByteBufferOutputStream;
import org.apache.kafka.common.utils.ChunkedBytesStream;
import java.io.BufferedOutputStream;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.ByteBuffer;
import java.util.zip.GZIPInputStream;
import java.util.zip.GZIPOutputStream;
/** /**
* The compression type to use * The compression type to use
*/ */
public enum CompressionType { public enum CompressionType {
NONE((byte) 0, "none", 1.0f) { NONE((byte) 0, "none", 1.0f),
@Override
public OutputStream wrapForOutput(ByteBufferOutputStream buffer, byte messageVersion) {
return buffer;
}
@Override
public InputStream wrapForInput(ByteBuffer buffer, byte messageVersion, BufferSupplier decompressionBufferSupplier) {
return new ByteBufferInputStream(buffer);
}
},
// Shipped with the JDK // Shipped with the JDK
GZIP((byte) 1, "gzip", 1.0f) { GZIP((byte) 1, "gzip", 1.0f),
@Override
public OutputStream wrapForOutput(ByteBufferOutputStream buffer, byte messageVersion) {
try {
// Set input buffer (uncompressed) to 16 KB (none by default) and output buffer (compressed) to
// 8 KB (0.5 KB by default) to ensure reasonable performance in cases where the caller passes a small
// number of bytes to write (potentially a single byte)
return new BufferedOutputStream(new GZIPOutputStream(buffer, 8 * 1024), 16 * 1024);
} catch (Exception e) {
throw new KafkaException(e);
}
}
@Override
public InputStream wrapForInput(ByteBuffer buffer, byte messageVersion, BufferSupplier decompressionBufferSupplier) {
try {
// Set input buffer (compressed) to 8 KB (GZIPInputStream uses 0.5 KB by default) to ensure reasonable
// performance in cases where the caller reads a small number of bytes (potentially a single byte).
//
// Size of output buffer (uncompressed) is provided by decompressionOutputSize.
//
// ChunkedBytesStream is used to wrap the GZIPInputStream because the default implementation of
// GZIPInputStream does not use an intermediate buffer for decompression in chunks.
return new ChunkedBytesStream(new GZIPInputStream(new ByteBufferInputStream(buffer), 8 * 1024), decompressionBufferSupplier, decompressionOutputSize(), false);
} catch (Exception e) {
throw new KafkaException(e);
}
}
@Override
public int decompressionOutputSize() {
// 16KB has been chosen based on legacy implementation introduced in https://github.com/apache/kafka/pull/6785
return 16 * 1024;
}
},
// We should only load classes from a given compression library when we actually use said compression library. This // We should only load classes from a given compression library when we actually use said compression library. This
// is because compression libraries include native code for a set of platforms and we want to avoid errors // is because compression libraries include native code for a set of platforms and we want to avoid errors
// in case the platform is not supported and the compression library is not actually used. // in case the platform is not supported and the compression library is not actually used.
// To ensure this, we only reference compression library code from classes that are only invoked when actual usage // To ensure this, we only reference compression library code from classes that are only invoked when actual usage
// happens. // happens.
SNAPPY((byte) 2, "snappy", 1.0f),
SNAPPY((byte) 2, "snappy", 1.0f) { LZ4((byte) 3, "lz4", 1.0f),
@Override ZSTD((byte) 4, "zstd", 1.0f);
public OutputStream wrapForOutput(ByteBufferOutputStream buffer, byte messageVersion) {
return SnappyFactory.wrapForOutput(buffer);
}
@Override
public InputStream wrapForInput(ByteBuffer buffer, byte messageVersion, BufferSupplier decompressionBufferSupplier) {
// SnappyInputStream uses default implementation of InputStream for skip. Default implementation of
// SnappyInputStream allocates a new skip buffer every time, hence, we prefer our own implementation.
return new ChunkedBytesStream(SnappyFactory.wrapForInput(buffer), decompressionBufferSupplier, decompressionOutputSize(), false);
}
@Override
public int decompressionOutputSize() {
// SnappyInputStream already uses an intermediate buffer internally. The size
// of this buffer is based on legacy implementation based on skipArray introduced in
// https://github.com/apache/kafka/pull/6785
return 2 * 1024; // 2KB
}
},
LZ4((byte) 3, "lz4", 1.0f) {
@Override
public OutputStream wrapForOutput(ByteBufferOutputStream buffer, byte messageVersion) {
try {
return new KafkaLZ4BlockOutputStream(buffer, messageVersion == RecordBatch.MAGIC_VALUE_V0);
} catch (Throwable e) {
throw new KafkaException(e);
}
}
@Override
public InputStream wrapForInput(ByteBuffer inputBuffer, byte messageVersion, BufferSupplier decompressionBufferSupplier) {
try {
return new ChunkedBytesStream(
new KafkaLZ4BlockInputStream(inputBuffer, decompressionBufferSupplier, messageVersion == RecordBatch.MAGIC_VALUE_V0),
decompressionBufferSupplier, decompressionOutputSize(), true);
} catch (Throwable e) {
throw new KafkaException(e);
}
}
@Override
public int decompressionOutputSize() {
// KafkaLZ4BlockInputStream uses an internal intermediate buffer to store decompressed data. The size
// of this buffer is based on legacy implementation based on skipArray introduced in
// https://github.com/apache/kafka/pull/6785
return 2 * 1024; // 2KB
}
},
ZSTD((byte) 4, "zstd", 1.0f) {
@Override
public OutputStream wrapForOutput(ByteBufferOutputStream buffer, byte messageVersion) {
return ZstdFactory.wrapForOutput(buffer);
}
@Override
public InputStream wrapForInput(ByteBuffer buffer, byte messageVersion, BufferSupplier decompressionBufferSupplier) {
return new ChunkedBytesStream(ZstdFactory.wrapForInput(buffer, messageVersion, decompressionBufferSupplier), decompressionBufferSupplier, decompressionOutputSize(), false);
}
/**
* Size of intermediate buffer which contains uncompressed data.
* This size should be <= ZSTD_BLOCKSIZE_MAX
* see: https://github.com/facebook/zstd/blob/189653a9c10c9f4224a5413a6d6a69dd01d7c3bd/lib/zstd.h#L854
*/
@Override
public int decompressionOutputSize() {
// 16KB has been chosen based on legacy implementation introduced in https://github.com/apache/kafka/pull/6785
return 16 * 1024;
}
};
// compression type is represented by two bits in the attributes field of the record batch header, so `byte` is // compression type is represented by two bits in the attributes field of the record batch header, so `byte` is
// large enough // large enough
@ -181,34 +46,6 @@ public enum CompressionType {
this.rate = rate; this.rate = rate;
} }
/**
* Wrap bufferStream with an OutputStream that will compress data with this CompressionType.
* <p>
* Note: Unlike {@link #wrapForInput}, {@link #wrapForOutput} cannot take {@link ByteBuffer}s directly.
* Currently, {@link MemoryRecordsBuilder#writeDefaultBatchHeader()} and {@link MemoryRecordsBuilder#writeLegacyCompressedWrapperHeader()}
* write to the underlying buffer in the given {@link ByteBufferOutputStream} after the compressed data has been written.
* In the event that the buffer needs to be expanded while writing the data, access to the underlying buffer needs to be preserved.
*/
public abstract OutputStream wrapForOutput(ByteBufferOutputStream bufferStream, byte messageVersion);
/**
* Wrap buffer with an InputStream that will decompress data with this CompressionType.
*
* @param decompressionBufferSupplier The supplier of ByteBuffer(s) used for decompression if supported.
* For small record batches, allocating a potentially large buffer (64 KB for LZ4)
* will dominate the cost of decompressing and iterating over the records in the
* batch. As such, a supplier that reuses buffers will have a significant
* performance impact.
*/
public abstract InputStream wrapForInput(ByteBuffer buffer, byte messageVersion, BufferSupplier decompressionBufferSupplier);
/**
* Recommended size of buffer for storing decompressed output.
*/
public int decompressionOutputSize() {
throw new UnsupportedOperationException("Size of decompression buffer is not defined for this compression type=" + this.name);
}
public static CompressionType forId(int id) { public static CompressionType forId(int id) {
switch (id) { switch (id) {
case 0: case 0:

View File

@ -18,6 +18,7 @@ package org.apache.kafka.common.record;
import org.apache.kafka.common.InvalidRecordException; import org.apache.kafka.common.InvalidRecordException;
import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.errors.CorruptRecordException; import org.apache.kafka.common.errors.CorruptRecordException;
import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.Header;
import org.apache.kafka.common.utils.BufferSupplier; import org.apache.kafka.common.utils.BufferSupplier;
@ -270,7 +271,7 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe
public InputStream recordInputStream(BufferSupplier bufferSupplier) { public InputStream recordInputStream(BufferSupplier bufferSupplier) {
final ByteBuffer buffer = this.buffer.duplicate(); final ByteBuffer buffer = this.buffer.duplicate();
buffer.position(RECORDS_OFFSET); buffer.position(RECORDS_OFFSET);
return compressionType().wrapForInput(buffer, magic(), bufferSupplier); return Compression.of(compressionType()).build().wrapForInput(buffer, magic(), bufferSupplier);
} }
private CloseableIterator<Record> compressedIterator(BufferSupplier bufferSupplier, boolean skipKeyValue) { private CloseableIterator<Record> compressedIterator(BufferSupplier bufferSupplier, boolean skipKeyValue) {

View File

@ -17,6 +17,7 @@
package org.apache.kafka.common.record; package org.apache.kafka.common.record;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.errors.CorruptRecordException; import org.apache.kafka.common.errors.CorruptRecordException;
import org.apache.kafka.common.message.KRaftVersionRecord; import org.apache.kafka.common.message.KRaftVersionRecord;
import org.apache.kafka.common.message.LeaderChangeMessage; import org.apache.kafka.common.message.LeaderChangeMessage;
@ -47,7 +48,7 @@ import java.util.Objects;
/** /**
* A {@link Records} implementation backed by a ByteBuffer. This is used only for reading or * A {@link Records} implementation backed by a ByteBuffer. This is used only for reading or
* modifying in-place an existing buffer of record batches. To create a new buffer see {@link MemoryRecordsBuilder}, * modifying in-place an existing buffer of record batches. To create a new buffer see {@link MemoryRecordsBuilder},
* or one of the {@link #builder(ByteBuffer, byte, CompressionType, TimestampType, long)} variants. * or one of the {@link #builder(ByteBuffer, byte, Compression, TimestampType, long)} variants.
*/ */
public class MemoryRecords extends AbstractRecords { public class MemoryRecords extends AbstractRecords {
private static final Logger log = LoggerFactory.getLogger(MemoryRecords.class); private static final Logger log = LoggerFactory.getLogger(MemoryRecords.class);
@ -293,6 +294,7 @@ public class MemoryRecords extends AbstractRecords {
ByteBufferOutputStream bufferOutputStream, ByteBufferOutputStream bufferOutputStream,
final long deleteHorizonMs) { final long deleteHorizonMs) {
byte magic = originalBatch.magic(); byte magic = originalBatch.magic();
Compression compression = Compression.of(originalBatch.compressionType()).build();
TimestampType timestampType = originalBatch.timestampType(); TimestampType timestampType = originalBatch.timestampType();
long logAppendTime = timestampType == TimestampType.LOG_APPEND_TIME ? long logAppendTime = timestampType == TimestampType.LOG_APPEND_TIME ?
originalBatch.maxTimestamp() : RecordBatch.NO_TIMESTAMP; originalBatch.maxTimestamp() : RecordBatch.NO_TIMESTAMP;
@ -300,7 +302,7 @@ public class MemoryRecords extends AbstractRecords {
originalBatch.baseOffset() : retainedRecords.get(0).offset(); originalBatch.baseOffset() : retainedRecords.get(0).offset();
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(bufferOutputStream, magic, MemoryRecordsBuilder builder = new MemoryRecordsBuilder(bufferOutputStream, magic,
originalBatch.compressionType(), timestampType, baseOffset, logAppendTime, originalBatch.producerId(), compression, timestampType, baseOffset, logAppendTime, originalBatch.producerId(),
originalBatch.producerEpoch(), originalBatch.baseSequence(), originalBatch.isTransactional(), originalBatch.producerEpoch(), originalBatch.baseSequence(), originalBatch.isTransactional(),
originalBatch.isControlBatch(), originalBatch.partitionLeaderEpoch(), bufferOutputStream.limit(), deleteHorizonMs); originalBatch.isControlBatch(), originalBatch.partitionLeaderEpoch(), bufferOutputStream.limit(), deleteHorizonMs);
@ -470,14 +472,14 @@ public class MemoryRecords extends AbstractRecords {
} }
public static MemoryRecordsBuilder builder(ByteBuffer buffer, public static MemoryRecordsBuilder builder(ByteBuffer buffer,
CompressionType compressionType, Compression compression,
TimestampType timestampType, TimestampType timestampType,
long baseOffset) { long baseOffset) {
return builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compressionType, timestampType, baseOffset); return builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compression, timestampType, baseOffset);
} }
public static MemoryRecordsBuilder builder(ByteBuffer buffer, public static MemoryRecordsBuilder builder(ByteBuffer buffer,
CompressionType compressionType, Compression compression,
TimestampType timestampType, TimestampType timestampType,
long baseOffset, long baseOffset,
int maxSize) { int maxSize) {
@ -485,84 +487,84 @@ public class MemoryRecords extends AbstractRecords {
if (timestampType == TimestampType.LOG_APPEND_TIME) if (timestampType == TimestampType.LOG_APPEND_TIME)
logAppendTime = System.currentTimeMillis(); logAppendTime = System.currentTimeMillis();
return new MemoryRecordsBuilder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compressionType, timestampType, baseOffset, return new MemoryRecordsBuilder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compression, timestampType, baseOffset,
logAppendTime, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, logAppendTime, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE,
false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, maxSize); false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, maxSize);
} }
public static MemoryRecordsBuilder idempotentBuilder(ByteBuffer buffer, public static MemoryRecordsBuilder idempotentBuilder(ByteBuffer buffer,
CompressionType compressionType, Compression compression,
long baseOffset, long baseOffset,
long producerId, long producerId,
short producerEpoch, short producerEpoch,
int baseSequence) { int baseSequence) {
return builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compressionType, TimestampType.CREATE_TIME, return builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compression, TimestampType.CREATE_TIME,
baseOffset, System.currentTimeMillis(), producerId, producerEpoch, baseSequence); baseOffset, System.currentTimeMillis(), producerId, producerEpoch, baseSequence);
} }
public static MemoryRecordsBuilder builder(ByteBuffer buffer, public static MemoryRecordsBuilder builder(ByteBuffer buffer,
byte magic, byte magic,
CompressionType compressionType, Compression compression,
TimestampType timestampType, TimestampType timestampType,
long baseOffset, long baseOffset,
long logAppendTime) { long logAppendTime) {
return builder(buffer, magic, compressionType, timestampType, baseOffset, logAppendTime, return builder(buffer, magic, compression, timestampType, baseOffset, logAppendTime,
RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, false, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, false,
RecordBatch.NO_PARTITION_LEADER_EPOCH); RecordBatch.NO_PARTITION_LEADER_EPOCH);
} }
public static MemoryRecordsBuilder builder(ByteBuffer buffer, public static MemoryRecordsBuilder builder(ByteBuffer buffer,
byte magic, byte magic,
CompressionType compressionType, Compression compression,
TimestampType timestampType, TimestampType timestampType,
long baseOffset) { long baseOffset) {
long logAppendTime = RecordBatch.NO_TIMESTAMP; long logAppendTime = RecordBatch.NO_TIMESTAMP;
if (timestampType == TimestampType.LOG_APPEND_TIME) if (timestampType == TimestampType.LOG_APPEND_TIME)
logAppendTime = System.currentTimeMillis(); logAppendTime = System.currentTimeMillis();
return builder(buffer, magic, compressionType, timestampType, baseOffset, logAppendTime, return builder(buffer, magic, compression, timestampType, baseOffset, logAppendTime,
RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, false, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, false,
RecordBatch.NO_PARTITION_LEADER_EPOCH); RecordBatch.NO_PARTITION_LEADER_EPOCH);
} }
public static MemoryRecordsBuilder builder(ByteBuffer buffer, public static MemoryRecordsBuilder builder(ByteBuffer buffer,
byte magic, byte magic,
CompressionType compressionType, Compression compression,
TimestampType timestampType, TimestampType timestampType,
long baseOffset, long baseOffset,
long logAppendTime, long logAppendTime,
int partitionLeaderEpoch) { int partitionLeaderEpoch) {
return builder(buffer, magic, compressionType, timestampType, baseOffset, logAppendTime, return builder(buffer, magic, compression, timestampType, baseOffset, logAppendTime,
RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, false, partitionLeaderEpoch); RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, false, partitionLeaderEpoch);
} }
public static MemoryRecordsBuilder builder(ByteBuffer buffer, public static MemoryRecordsBuilder builder(ByteBuffer buffer,
CompressionType compressionType, Compression compression,
long baseOffset, long baseOffset,
long producerId, long producerId,
short producerEpoch, short producerEpoch,
int baseSequence, int baseSequence,
boolean isTransactional) { boolean isTransactional) {
return builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compressionType, TimestampType.CREATE_TIME, baseOffset, return builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compression, TimestampType.CREATE_TIME, baseOffset,
RecordBatch.NO_TIMESTAMP, producerId, producerEpoch, baseSequence, isTransactional, RecordBatch.NO_TIMESTAMP, producerId, producerEpoch, baseSequence, isTransactional,
RecordBatch.NO_PARTITION_LEADER_EPOCH); RecordBatch.NO_PARTITION_LEADER_EPOCH);
} }
public static MemoryRecordsBuilder builder(ByteBuffer buffer, public static MemoryRecordsBuilder builder(ByteBuffer buffer,
byte magic, byte magic,
CompressionType compressionType, Compression compression,
TimestampType timestampType, TimestampType timestampType,
long baseOffset, long baseOffset,
long logAppendTime, long logAppendTime,
long producerId, long producerId,
short producerEpoch, short producerEpoch,
int baseSequence) { int baseSequence) {
return builder(buffer, magic, compressionType, timestampType, baseOffset, logAppendTime, return builder(buffer, magic, compression, timestampType, baseOffset, logAppendTime,
producerId, producerEpoch, baseSequence, false, RecordBatch.NO_PARTITION_LEADER_EPOCH); producerId, producerEpoch, baseSequence, false, RecordBatch.NO_PARTITION_LEADER_EPOCH);
} }
public static MemoryRecordsBuilder builder(ByteBuffer buffer, public static MemoryRecordsBuilder builder(ByteBuffer buffer,
byte magic, byte magic,
CompressionType compressionType, Compression compression,
TimestampType timestampType, TimestampType timestampType,
long baseOffset, long baseOffset,
long logAppendTime, long logAppendTime,
@ -571,13 +573,13 @@ public class MemoryRecords extends AbstractRecords {
int baseSequence, int baseSequence,
boolean isTransactional, boolean isTransactional,
int partitionLeaderEpoch) { int partitionLeaderEpoch) {
return builder(buffer, magic, compressionType, timestampType, baseOffset, return builder(buffer, magic, compression, timestampType, baseOffset,
logAppendTime, producerId, producerEpoch, baseSequence, isTransactional, false, partitionLeaderEpoch); logAppendTime, producerId, producerEpoch, baseSequence, isTransactional, false, partitionLeaderEpoch);
} }
public static MemoryRecordsBuilder builder(ByteBuffer buffer, public static MemoryRecordsBuilder builder(ByteBuffer buffer,
byte magic, byte magic,
CompressionType compressionType, Compression compression,
TimestampType timestampType, TimestampType timestampType,
long baseOffset, long baseOffset,
long logAppendTime, long logAppendTime,
@ -587,98 +589,98 @@ public class MemoryRecords extends AbstractRecords {
boolean isTransactional, boolean isTransactional,
boolean isControlBatch, boolean isControlBatch,
int partitionLeaderEpoch) { int partitionLeaderEpoch) {
return new MemoryRecordsBuilder(buffer, magic, compressionType, timestampType, baseOffset, return new MemoryRecordsBuilder(buffer, magic, compression, timestampType, baseOffset,
logAppendTime, producerId, producerEpoch, baseSequence, isTransactional, isControlBatch, partitionLeaderEpoch, logAppendTime, producerId, producerEpoch, baseSequence, isTransactional, isControlBatch, partitionLeaderEpoch,
buffer.remaining()); buffer.remaining());
} }
public static MemoryRecords withRecords(CompressionType compressionType, SimpleRecord... records) { public static MemoryRecords withRecords(Compression compression, SimpleRecord... records) {
return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, compressionType, records); return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, compression, records);
} }
public static MemoryRecords withRecords(CompressionType compressionType, int partitionLeaderEpoch, SimpleRecord... records) { public static MemoryRecords withRecords(Compression compression, int partitionLeaderEpoch, SimpleRecord... records) {
return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, 0L, compressionType, TimestampType.CREATE_TIME, return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, 0L, compression, TimestampType.CREATE_TIME,
RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE,
partitionLeaderEpoch, false, records); partitionLeaderEpoch, false, records);
} }
public static MemoryRecords withRecords(byte magic, CompressionType compressionType, SimpleRecord... records) { public static MemoryRecords withRecords(byte magic, Compression compression, SimpleRecord... records) {
return withRecords(magic, 0L, compressionType, TimestampType.CREATE_TIME, records); return withRecords(magic, 0L, compression, TimestampType.CREATE_TIME, records);
} }
public static MemoryRecords withRecords(long initialOffset, CompressionType compressionType, SimpleRecord... records) { public static MemoryRecords withRecords(long initialOffset, Compression compression, SimpleRecord... records) {
return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, initialOffset, compressionType, TimestampType.CREATE_TIME, return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, initialOffset, compression, TimestampType.CREATE_TIME,
records); records);
} }
public static MemoryRecords withRecords(byte magic, long initialOffset, CompressionType compressionType, SimpleRecord... records) { public static MemoryRecords withRecords(byte magic, long initialOffset, Compression compression, SimpleRecord... records) {
return withRecords(magic, initialOffset, compressionType, TimestampType.CREATE_TIME, records); return withRecords(magic, initialOffset, compression, TimestampType.CREATE_TIME, records);
} }
public static MemoryRecords withRecords(long initialOffset, CompressionType compressionType, Integer partitionLeaderEpoch, SimpleRecord... records) { public static MemoryRecords withRecords(long initialOffset, Compression compression, Integer partitionLeaderEpoch, SimpleRecord... records) {
return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, initialOffset, compressionType, TimestampType.CREATE_TIME, RecordBatch.NO_PRODUCER_ID, return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, initialOffset, compression, TimestampType.CREATE_TIME, RecordBatch.NO_PRODUCER_ID,
RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, partitionLeaderEpoch, false, records); RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, partitionLeaderEpoch, false, records);
} }
public static MemoryRecords withIdempotentRecords(CompressionType compressionType, long producerId, public static MemoryRecords withIdempotentRecords(Compression compression, long producerId,
short producerEpoch, int baseSequence, SimpleRecord... records) { short producerEpoch, int baseSequence, SimpleRecord... records) {
return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, 0L, compressionType, TimestampType.CREATE_TIME, producerId, producerEpoch, return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, 0L, compression, TimestampType.CREATE_TIME, producerId, producerEpoch,
baseSequence, RecordBatch.NO_PARTITION_LEADER_EPOCH, false, records); baseSequence, RecordBatch.NO_PARTITION_LEADER_EPOCH, false, records);
} }
public static MemoryRecords withIdempotentRecords(byte magic, long initialOffset, CompressionType compressionType, public static MemoryRecords withIdempotentRecords(byte magic, long initialOffset, Compression compression,
long producerId, short producerEpoch, int baseSequence, long producerId, short producerEpoch, int baseSequence,
int partitionLeaderEpoch, SimpleRecord... records) { int partitionLeaderEpoch, SimpleRecord... records) {
return withRecords(magic, initialOffset, compressionType, TimestampType.CREATE_TIME, producerId, producerEpoch, return withRecords(magic, initialOffset, compression, TimestampType.CREATE_TIME, producerId, producerEpoch,
baseSequence, partitionLeaderEpoch, false, records); baseSequence, partitionLeaderEpoch, false, records);
} }
public static MemoryRecords withIdempotentRecords(long initialOffset, CompressionType compressionType, long producerId, public static MemoryRecords withIdempotentRecords(long initialOffset, Compression compression, long producerId,
short producerEpoch, int baseSequence, int partitionLeaderEpoch, short producerEpoch, int baseSequence, int partitionLeaderEpoch,
SimpleRecord... records) { SimpleRecord... records) {
return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, initialOffset, compressionType, TimestampType.CREATE_TIME, return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, initialOffset, compression, TimestampType.CREATE_TIME,
producerId, producerEpoch, baseSequence, partitionLeaderEpoch, false, records); producerId, producerEpoch, baseSequence, partitionLeaderEpoch, false, records);
} }
public static MemoryRecords withTransactionalRecords(CompressionType compressionType, long producerId, public static MemoryRecords withTransactionalRecords(Compression compression, long producerId,
short producerEpoch, int baseSequence, SimpleRecord... records) { short producerEpoch, int baseSequence, SimpleRecord... records) {
return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, 0L, compressionType, TimestampType.CREATE_TIME, return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, 0L, compression, TimestampType.CREATE_TIME,
producerId, producerEpoch, baseSequence, RecordBatch.NO_PARTITION_LEADER_EPOCH, true, records); producerId, producerEpoch, baseSequence, RecordBatch.NO_PARTITION_LEADER_EPOCH, true, records);
} }
public static MemoryRecords withTransactionalRecords(byte magic, long initialOffset, CompressionType compressionType, public static MemoryRecords withTransactionalRecords(byte magic, long initialOffset, Compression compression,
long producerId, short producerEpoch, int baseSequence, long producerId, short producerEpoch, int baseSequence,
int partitionLeaderEpoch, SimpleRecord... records) { int partitionLeaderEpoch, SimpleRecord... records) {
return withRecords(magic, initialOffset, compressionType, TimestampType.CREATE_TIME, producerId, producerEpoch, return withRecords(magic, initialOffset, compression, TimestampType.CREATE_TIME, producerId, producerEpoch,
baseSequence, partitionLeaderEpoch, true, records); baseSequence, partitionLeaderEpoch, true, records);
} }
public static MemoryRecords withTransactionalRecords(long initialOffset, CompressionType compressionType, long producerId, public static MemoryRecords withTransactionalRecords(long initialOffset, Compression compression, long producerId,
short producerEpoch, int baseSequence, int partitionLeaderEpoch, short producerEpoch, int baseSequence, int partitionLeaderEpoch,
SimpleRecord... records) { SimpleRecord... records) {
return withTransactionalRecords(RecordBatch.CURRENT_MAGIC_VALUE, initialOffset, compressionType, return withTransactionalRecords(RecordBatch.CURRENT_MAGIC_VALUE, initialOffset, compression,
producerId, producerEpoch, baseSequence, partitionLeaderEpoch, records); producerId, producerEpoch, baseSequence, partitionLeaderEpoch, records);
} }
public static MemoryRecords withRecords(byte magic, long initialOffset, CompressionType compressionType, public static MemoryRecords withRecords(byte magic, long initialOffset, Compression compression,
TimestampType timestampType, SimpleRecord... records) { TimestampType timestampType, SimpleRecord... records) {
return withRecords(magic, initialOffset, compressionType, timestampType, RecordBatch.NO_PRODUCER_ID, return withRecords(magic, initialOffset, compression, timestampType, RecordBatch.NO_PRODUCER_ID,
RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, RecordBatch.NO_PARTITION_LEADER_EPOCH, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, RecordBatch.NO_PARTITION_LEADER_EPOCH,
false, records); false, records);
} }
public static MemoryRecords withRecords(byte magic, long initialOffset, CompressionType compressionType, public static MemoryRecords withRecords(byte magic, long initialOffset, Compression compression,
TimestampType timestampType, long producerId, short producerEpoch, TimestampType timestampType, long producerId, short producerEpoch,
int baseSequence, int partitionLeaderEpoch, boolean isTransactional, int baseSequence, int partitionLeaderEpoch, boolean isTransactional,
SimpleRecord... records) { SimpleRecord... records) {
if (records.length == 0) if (records.length == 0)
return MemoryRecords.EMPTY; return MemoryRecords.EMPTY;
int sizeEstimate = AbstractRecords.estimateSizeInBytes(magic, compressionType, Arrays.asList(records)); int sizeEstimate = AbstractRecords.estimateSizeInBytes(magic, compression.type(), Arrays.asList(records));
ByteBufferOutputStream bufferStream = new ByteBufferOutputStream(sizeEstimate); ByteBufferOutputStream bufferStream = new ByteBufferOutputStream(sizeEstimate);
long logAppendTime = RecordBatch.NO_TIMESTAMP; long logAppendTime = RecordBatch.NO_TIMESTAMP;
if (timestampType == TimestampType.LOG_APPEND_TIME) if (timestampType == TimestampType.LOG_APPEND_TIME)
logAppendTime = System.currentTimeMillis(); logAppendTime = System.currentTimeMillis();
try (final MemoryRecordsBuilder builder = new MemoryRecordsBuilder(bufferStream, magic, compressionType, timestampType, try (final MemoryRecordsBuilder builder = new MemoryRecordsBuilder(bufferStream, magic, compression, timestampType,
initialOffset, logAppendTime, producerId, producerEpoch, baseSequence, isTransactional, false, initialOffset, logAppendTime, producerId, producerEpoch, baseSequence, isTransactional, false,
partitionLeaderEpoch, sizeEstimate)) { partitionLeaderEpoch, sizeEstimate)) {
for (SimpleRecord record : records) for (SimpleRecord record : records)
@ -714,7 +716,7 @@ public class MemoryRecords extends AbstractRecords {
int partitionLeaderEpoch, long producerId, short producerEpoch, int partitionLeaderEpoch, long producerId, short producerEpoch,
EndTransactionMarker marker) { EndTransactionMarker marker) {
boolean isTransactional = true; boolean isTransactional = true;
try (MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, try (MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, Compression.NONE,
TimestampType.CREATE_TIME, initialOffset, timestamp, producerId, producerEpoch, TimestampType.CREATE_TIME, initialOffset, timestamp, producerId, producerEpoch,
RecordBatch.NO_SEQUENCE, isTransactional, true, partitionLeaderEpoch, RecordBatch.NO_SEQUENCE, isTransactional, true, partitionLeaderEpoch,
buffer.capacity()) buffer.capacity())
@ -827,7 +829,7 @@ public class MemoryRecords extends AbstractRecords {
return new MemoryRecordsBuilder( return new MemoryRecordsBuilder(
buffer, buffer,
RecordBatch.CURRENT_MAGIC_VALUE, RecordBatch.CURRENT_MAGIC_VALUE,
CompressionType.NONE, Compression.NONE,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
initialOffset, initialOffset,
timestamp, timestamp,

View File

@ -17,6 +17,7 @@
package org.apache.kafka.common.record; package org.apache.kafka.common.record;
import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.Header;
import org.apache.kafka.common.message.KRaftVersionRecord; import org.apache.kafka.common.message.KRaftVersionRecord;
import org.apache.kafka.common.message.LeaderChangeMessage; import org.apache.kafka.common.message.LeaderChangeMessage;
@ -54,7 +55,7 @@ public class MemoryRecordsBuilder implements AutoCloseable {
}); });
private final TimestampType timestampType; private final TimestampType timestampType;
private final CompressionType compressionType; private final Compression compression;
// Used to hold a reference to the underlying ByteBuffer so that we can write the record batch header and access // Used to hold a reference to the underlying ByteBuffer so that we can write the record batch header and access
// the written bytes. ByteBufferOutputStream allocates a new ByteBuffer if the existing one is not large enough, // the written bytes. ByteBufferOutputStream allocates a new ByteBuffer if the existing one is not large enough,
// so it's not safe to hold a direct reference to the underlying ByteBuffer. // so it's not safe to hold a direct reference to the underlying ByteBuffer.
@ -92,7 +93,7 @@ public class MemoryRecordsBuilder implements AutoCloseable {
public MemoryRecordsBuilder(ByteBufferOutputStream bufferStream, public MemoryRecordsBuilder(ByteBufferOutputStream bufferStream,
byte magic, byte magic,
CompressionType compressionType, Compression compression,
TimestampType timestampType, TimestampType timestampType,
long baseOffset, long baseOffset,
long logAppendTime, long logAppendTime,
@ -111,7 +112,7 @@ public class MemoryRecordsBuilder implements AutoCloseable {
throw new IllegalArgumentException("Transactional records are not supported for magic " + magic); throw new IllegalArgumentException("Transactional records are not supported for magic " + magic);
if (isControlBatch) if (isControlBatch)
throw new IllegalArgumentException("Control records are not supported for magic " + magic); throw new IllegalArgumentException("Control records are not supported for magic " + magic);
if (compressionType == CompressionType.ZSTD) if (compression.type() == CompressionType.ZSTD)
throw new IllegalArgumentException("ZStandard compression is not supported for magic " + magic); throw new IllegalArgumentException("ZStandard compression is not supported for magic " + magic);
if (deleteHorizonMs != RecordBatch.NO_TIMESTAMP) if (deleteHorizonMs != RecordBatch.NO_TIMESTAMP)
throw new IllegalArgumentException("Delete horizon timestamp is not supported for magic " + magic); throw new IllegalArgumentException("Delete horizon timestamp is not supported for magic " + magic);
@ -119,7 +120,7 @@ public class MemoryRecordsBuilder implements AutoCloseable {
this.magic = magic; this.magic = magic;
this.timestampType = timestampType; this.timestampType = timestampType;
this.compressionType = compressionType; this.compression = compression;
this.baseOffset = baseOffset; this.baseOffset = baseOffset;
this.logAppendTime = logAppendTime; this.logAppendTime = logAppendTime;
this.numRecords = 0; this.numRecords = 0;
@ -135,11 +136,11 @@ public class MemoryRecordsBuilder implements AutoCloseable {
this.partitionLeaderEpoch = partitionLeaderEpoch; this.partitionLeaderEpoch = partitionLeaderEpoch;
this.writeLimit = writeLimit; this.writeLimit = writeLimit;
this.initialPosition = bufferStream.position(); this.initialPosition = bufferStream.position();
this.batchHeaderSizeInBytes = AbstractRecords.recordBatchHeaderSizeInBytes(magic, compressionType); this.batchHeaderSizeInBytes = AbstractRecords.recordBatchHeaderSizeInBytes(magic, compression.type());
bufferStream.position(initialPosition + batchHeaderSizeInBytes); bufferStream.position(initialPosition + batchHeaderSizeInBytes);
this.bufferStream = bufferStream; this.bufferStream = bufferStream;
this.appendStream = new DataOutputStream(compressionType.wrapForOutput(this.bufferStream, magic)); this.appendStream = new DataOutputStream(compression.wrapForOutput(this.bufferStream, magic));
if (hasDeleteHorizonMs()) { if (hasDeleteHorizonMs()) {
this.baseTimestamp = deleteHorizonMs; this.baseTimestamp = deleteHorizonMs;
@ -148,7 +149,7 @@ public class MemoryRecordsBuilder implements AutoCloseable {
public MemoryRecordsBuilder(ByteBufferOutputStream bufferStream, public MemoryRecordsBuilder(ByteBufferOutputStream bufferStream,
byte magic, byte magic,
CompressionType compressionType, Compression compression,
TimestampType timestampType, TimestampType timestampType,
long baseOffset, long baseOffset,
long logAppendTime, long logAppendTime,
@ -159,7 +160,7 @@ public class MemoryRecordsBuilder implements AutoCloseable {
boolean isControlBatch, boolean isControlBatch,
int partitionLeaderEpoch, int partitionLeaderEpoch,
int writeLimit) { int writeLimit) {
this(bufferStream, magic, compressionType, timestampType, baseOffset, logAppendTime, producerId, this(bufferStream, magic, compression, timestampType, baseOffset, logAppendTime, producerId,
producerEpoch, baseSequence, isTransactional, isControlBatch, partitionLeaderEpoch, writeLimit, producerEpoch, baseSequence, isTransactional, isControlBatch, partitionLeaderEpoch, writeLimit,
RecordBatch.NO_TIMESTAMP); RecordBatch.NO_TIMESTAMP);
} }
@ -170,7 +171,7 @@ public class MemoryRecordsBuilder implements AutoCloseable {
* @param buffer The underlying buffer to use (note that this class will allocate a new buffer if necessary * @param buffer The underlying buffer to use (note that this class will allocate a new buffer if necessary
* to fit the records appended) * to fit the records appended)
* @param magic The magic value to use * @param magic The magic value to use
* @param compressionType The compression codec to use * @param compression The compression codec to use
* @param timestampType The desired timestamp type. For magic > 0, this cannot be {@link TimestampType#NO_TIMESTAMP_TYPE}. * @param timestampType The desired timestamp type. For magic > 0, this cannot be {@link TimestampType#NO_TIMESTAMP_TYPE}.
* @param baseOffset The initial offset to use for * @param baseOffset The initial offset to use for
* @param logAppendTime The log append time of this record set. Can be set to NO_TIMESTAMP if CREATE_TIME is used. * @param logAppendTime The log append time of this record set. Can be set to NO_TIMESTAMP if CREATE_TIME is used.
@ -186,7 +187,7 @@ public class MemoryRecordsBuilder implements AutoCloseable {
*/ */
public MemoryRecordsBuilder(ByteBuffer buffer, public MemoryRecordsBuilder(ByteBuffer buffer,
byte magic, byte magic,
CompressionType compressionType, Compression compression,
TimestampType timestampType, TimestampType timestampType,
long baseOffset, long baseOffset,
long logAppendTime, long logAppendTime,
@ -197,7 +198,7 @@ public class MemoryRecordsBuilder implements AutoCloseable {
boolean isControlBatch, boolean isControlBatch,
int partitionLeaderEpoch, int partitionLeaderEpoch,
int writeLimit) { int writeLimit) {
this(new ByteBufferOutputStream(buffer), magic, compressionType, timestampType, baseOffset, logAppendTime, this(new ByteBufferOutputStream(buffer), magic, compression, timestampType, baseOffset, logAppendTime,
producerId, producerEpoch, baseSequence, isTransactional, isControlBatch, partitionLeaderEpoch, producerId, producerEpoch, baseSequence, isTransactional, isControlBatch, partitionLeaderEpoch,
writeLimit); writeLimit);
} }
@ -214,8 +215,8 @@ public class MemoryRecordsBuilder implements AutoCloseable {
return actualCompressionRatio; return actualCompressionRatio;
} }
public CompressionType compressionType() { public Compression compression() {
return compressionType; return compression;
} }
public boolean isControlBatch() { public boolean isControlBatch() {
@ -269,7 +270,7 @@ public class MemoryRecordsBuilder implements AutoCloseable {
*/ */
public RecordsInfo info() { public RecordsInfo info() {
if (timestampType == TimestampType.LOG_APPEND_TIME) { if (timestampType == TimestampType.LOG_APPEND_TIME) {
if (compressionType != CompressionType.NONE || magic >= RecordBatch.MAGIC_VALUE_V2) if (compression.type() != CompressionType.NONE || magic >= RecordBatch.MAGIC_VALUE_V2)
// maxTimestamp => case 2 // maxTimestamp => case 2
// shallowOffsetOfMaxTimestamp => case 2 // shallowOffsetOfMaxTimestamp => case 2
return new RecordsInfo(logAppendTime, lastOffset); return new RecordsInfo(logAppendTime, lastOffset);
@ -282,7 +283,7 @@ public class MemoryRecordsBuilder implements AutoCloseable {
// shallowOffsetOfMaxTimestamp => case 1 // shallowOffsetOfMaxTimestamp => case 1
return new RecordsInfo(RecordBatch.NO_TIMESTAMP, -1); return new RecordsInfo(RecordBatch.NO_TIMESTAMP, -1);
} else { } else {
if (compressionType != CompressionType.NONE || magic >= RecordBatch.MAGIC_VALUE_V2) if (compression.type() != CompressionType.NONE || magic >= RecordBatch.MAGIC_VALUE_V2)
// maxTimestamp => case 3 // maxTimestamp => case 3
// shallowOffsetOfMaxTimestamp => case 4 // shallowOffsetOfMaxTimestamp => case 4
return new RecordsInfo(maxTimestamp, lastOffset); return new RecordsInfo(maxTimestamp, lastOffset);
@ -374,7 +375,7 @@ public class MemoryRecordsBuilder implements AutoCloseable {
} else { } else {
if (magic > RecordBatch.MAGIC_VALUE_V1) if (magic > RecordBatch.MAGIC_VALUE_V1)
this.actualCompressionRatio = (float) writeDefaultBatchHeader() / this.uncompressedRecordsSizeInBytes; this.actualCompressionRatio = (float) writeDefaultBatchHeader() / this.uncompressedRecordsSizeInBytes;
else if (compressionType != CompressionType.NONE) else if (compression.type() != CompressionType.NONE)
this.actualCompressionRatio = (float) writeLegacyCompressedWrapperHeader() / this.uncompressedRecordsSizeInBytes; this.actualCompressionRatio = (float) writeLegacyCompressedWrapperHeader() / this.uncompressedRecordsSizeInBytes;
ByteBuffer buffer = buffer().duplicate(); ByteBuffer buffer = buffer().duplicate();
@ -419,7 +420,7 @@ public class MemoryRecordsBuilder implements AutoCloseable {
else else
maxTimestamp = this.maxTimestamp; maxTimestamp = this.maxTimestamp;
DefaultRecordBatch.writeHeader(buffer, baseOffset, offsetDelta, size, magic, compressionType, timestampType, DefaultRecordBatch.writeHeader(buffer, baseOffset, offsetDelta, size, magic, compression.type(), timestampType,
baseTimestamp, maxTimestamp, producerId, producerEpoch, baseSequence, isTransactional, isControlBatch, baseTimestamp, maxTimestamp, producerId, producerEpoch, baseSequence, isTransactional, isControlBatch,
hasDeleteHorizonMs(), partitionLeaderEpoch, numRecords); hasDeleteHorizonMs(), partitionLeaderEpoch, numRecords);
@ -442,7 +443,7 @@ public class MemoryRecordsBuilder implements AutoCloseable {
AbstractLegacyRecordBatch.writeHeader(buffer, lastOffset, wrapperSize); AbstractLegacyRecordBatch.writeHeader(buffer, lastOffset, wrapperSize);
long timestamp = timestampType == TimestampType.LOG_APPEND_TIME ? logAppendTime : maxTimestamp; long timestamp = timestampType == TimestampType.LOG_APPEND_TIME ? logAppendTime : maxTimestamp;
LegacyRecord.writeCompressedRecordHeader(buffer, magic, wrapperSize, timestamp, compressionType, timestampType); LegacyRecord.writeCompressedRecordHeader(buffer, magic, wrapperSize, timestamp, compression.type(), timestampType);
buffer.position(pos); buffer.position(pos);
return writtenCompressed; return writtenCompressed;
@ -778,7 +779,7 @@ public class MemoryRecordsBuilder implements AutoCloseable {
private long toInnerOffset(long offset) { private long toInnerOffset(long offset) {
// use relative offsets for compressed messages with magic v1 // use relative offsets for compressed messages with magic v1
if (magic > 0 && compressionType != CompressionType.NONE) if (magic > 0 && compression.type() != CompressionType.NONE)
return offset - baseOffset; return offset - baseOffset;
return offset; return offset;
} }
@ -817,7 +818,7 @@ public class MemoryRecordsBuilder implements AutoCloseable {
* @return The estimated number of bytes written * @return The estimated number of bytes written
*/ */
private int estimatedBytesWritten() { private int estimatedBytesWritten() {
if (compressionType == CompressionType.NONE) { if (compression.type() == CompressionType.NONE) {
return batchHeaderSizeInBytes + uncompressedRecordsSizeInBytes; return batchHeaderSizeInBytes + uncompressedRecordsSizeInBytes;
} else { } else {
// estimate the written bytes to the underlying byte buffer based on uncompressed written bytes // estimate the written bytes to the underlying byte buffer based on uncompressed written bytes

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.common.record; package org.apache.kafka.common.record;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.errors.UnsupportedCompressionTypeException; import org.apache.kafka.common.errors.UnsupportedCompressionTypeException;
import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.utils.Utils;
@ -110,7 +111,7 @@ public class RecordsUtil {
final TimestampType timestampType = batch.timestampType(); final TimestampType timestampType = batch.timestampType();
long logAppendTime = timestampType == TimestampType.LOG_APPEND_TIME ? batch.maxTimestamp() : RecordBatch.NO_TIMESTAMP; long logAppendTime = timestampType == TimestampType.LOG_APPEND_TIME ? batch.maxTimestamp() : RecordBatch.NO_TIMESTAMP;
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic, batch.compressionType(), MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic, Compression.of(batch.compressionType()).build(),
timestampType, recordBatchAndRecords.baseOffset, logAppendTime); timestampType, recordBatchAndRecords.baseOffset, logAppendTime);
for (Record record : recordBatchAndRecords.records) { for (Record record : recordBatchAndRecords.records) {
// Down-convert this record. Ignore headers when down-converting to V0 and V1 since they are not supported // Down-convert this record. Ignore headers when down-converting to V0 and V1 since they are not supported

View File

@ -20,6 +20,7 @@ import io.opentelemetry.proto.metrics.v1.MetricsData;
import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.Uuid; import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.metrics.MetricsContext; import org.apache.kafka.common.metrics.MetricsContext;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.CompressionType;
@ -190,7 +191,8 @@ public class ClientTelemetryUtils {
public static byte[] compress(byte[] raw, CompressionType compressionType) throws IOException { public static byte[] compress(byte[] raw, CompressionType compressionType) throws IOException {
try (ByteBufferOutputStream compressedOut = new ByteBufferOutputStream(512)) { try (ByteBufferOutputStream compressedOut = new ByteBufferOutputStream(512)) {
try (OutputStream out = compressionType.wrapForOutput(compressedOut, RecordBatch.CURRENT_MAGIC_VALUE)) { Compression compression = Compression.of(compressionType).build();
try (OutputStream out = compression.wrapForOutput(compressedOut, RecordBatch.CURRENT_MAGIC_VALUE)) {
out.write(raw); out.write(raw);
out.flush(); out.flush();
} }
@ -201,7 +203,8 @@ public class ClientTelemetryUtils {
public static ByteBuffer decompress(byte[] metrics, CompressionType compressionType) { public static ByteBuffer decompress(byte[] metrics, CompressionType compressionType) {
ByteBuffer data = ByteBuffer.wrap(metrics); ByteBuffer data = ByteBuffer.wrap(metrics);
try (InputStream in = compressionType.wrapForInput(data, RecordBatch.CURRENT_MAGIC_VALUE, BufferSupplier.create()); Compression compression = Compression.of(compressionType).build();
try (InputStream in = compression.wrapForInput(data, RecordBatch.CURRENT_MAGIC_VALUE, BufferSupplier.create());
ByteBufferOutputStream out = new ByteBufferOutputStream(512)) { ByteBufferOutputStream out = new ByteBufferOutputStream(512)) {
byte[] bytes = new byte[data.capacity() * 2]; byte[] bytes = new byte[data.capacity() * 2];
int nRead; int nRead;

View File

@ -34,6 +34,7 @@ import org.apache.kafka.common.Node;
import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicIdPartition;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.Uuid; import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.config.SslConfigs; import org.apache.kafka.common.config.SslConfigs;
import org.apache.kafka.common.errors.AuthenticationException; import org.apache.kafka.common.errors.AuthenticationException;
import org.apache.kafka.common.errors.InterruptException; import org.apache.kafka.common.errors.InterruptException;
@ -63,7 +64,6 @@ import org.apache.kafka.common.metrics.stats.Avg;
import org.apache.kafka.common.network.Selectable; import org.apache.kafka.common.network.Selectable;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.MemoryRecordsBuilder; import org.apache.kafka.common.record.MemoryRecordsBuilder;
import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.record.TimestampType;
@ -2760,7 +2760,7 @@ public class KafkaConsumerTest {
if (fetchCount == 0) { if (fetchCount == 0) {
records = MemoryRecords.EMPTY; records = MemoryRecords.EMPTY;
} else { } else {
try (MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, try (MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), Compression.NONE,
TimestampType.CREATE_TIME, fetchOffset)) { TimestampType.CREATE_TIME, fetchOffset)) {
for (int i = 0; i < fetchCount; i++) for (int i = 0; i < fetchCount; i++)
builder.append(0L, ("key-" + i).getBytes(), ("value-" + i).getBytes()); builder.append(0L, ("key-" + i).getBytes(), ("value-" + i).getBytes());

View File

@ -21,11 +21,11 @@ import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.clients.consumer.OffsetResetStrategy;
import org.apache.kafka.common.IsolationLevel; import org.apache.kafka.common.IsolationLevel;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.errors.RecordDeserializationException; import org.apache.kafka.common.errors.RecordDeserializationException;
import org.apache.kafka.common.message.FetchResponseData; import org.apache.kafka.common.message.FetchResponseData;
import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.record.ControlRecordType; import org.apache.kafka.common.record.ControlRecordType;
import org.apache.kafka.common.record.EndTransactionMarker; import org.apache.kafka.common.record.EndTransactionMarker;
import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecords;
@ -157,7 +157,7 @@ public class CompletedFetchTest {
@Test @Test
public void testCorruptedMessage() { public void testCorruptedMessage() {
// Create one good record and then one "corrupted" record. // Create one good record and then one "corrupted" record.
try (final MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME, 0); try (final MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), Compression.NONE, TimestampType.CREATE_TIME, 0);
final UUIDSerializer serializer = new UUIDSerializer()) { final UUIDSerializer serializer = new UUIDSerializer()) {
builder.append(new SimpleRecord(serializer.serialize(TOPIC_NAME, UUID.randomUUID()))); builder.append(new SimpleRecord(serializer.serialize(TOPIC_NAME, UUID.randomUUID())));
builder.append(0L, "key".getBytes(), "value".getBytes()); builder.append(0L, "key".getBytes(), "value".getBytes());
@ -223,7 +223,7 @@ public class CompletedFetchTest {
} }
private Records newRecords(long baseOffset, int count, long firstMessageId) { private Records newRecords(long baseOffset, int count, long firstMessageId) {
try (final MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME, baseOffset)) { try (final MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), Compression.NONE, TimestampType.CREATE_TIME, baseOffset)) {
for (int i = 0; i < count; i++) for (int i = 0; i < count; i++)
builder.append(0L, "key".getBytes(), ("value-" + (firstMessageId + i)).getBytes()); builder.append(0L, "key".getBytes(), ("value-" + (firstMessageId + i)).getBytes());
return builder.build(); return builder.build();
@ -236,7 +236,7 @@ public class CompletedFetchTest {
try (MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, try (MemoryRecordsBuilder builder = MemoryRecords.builder(buffer,
RecordBatch.CURRENT_MAGIC_VALUE, RecordBatch.CURRENT_MAGIC_VALUE,
CompressionType.NONE, Compression.NONE,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
0, 0,
time.milliseconds(), time.milliseconds(),

View File

@ -19,13 +19,13 @@ package org.apache.kafka.clients.consumer.internals;
import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.errors.TopicAuthorizationException; import org.apache.kafka.common.errors.TopicAuthorizationException;
import org.apache.kafka.common.internals.ClusterResourceListeners; import org.apache.kafka.common.internals.ClusterResourceListeners;
import org.apache.kafka.common.message.FetchResponseData; import org.apache.kafka.common.message.FetchResponseData;
import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.MemoryRecordsBuilder; import org.apache.kafka.common.record.MemoryRecordsBuilder;
import org.apache.kafka.common.record.Records; import org.apache.kafka.common.record.Records;
@ -833,7 +833,7 @@ public class FetchCollectorTest {
ByteBuffer allocate = ByteBuffer.allocate(1024); ByteBuffer allocate = ByteBuffer.allocate(1024);
try (MemoryRecordsBuilder builder = MemoryRecords.builder(allocate, try (MemoryRecordsBuilder builder = MemoryRecords.builder(allocate,
CompressionType.NONE, Compression.NONE,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
0)) { 0)) {
for (int i = 0; i < recordCount; i++) for (int i = 0; i < recordCount; i++)

View File

@ -37,6 +37,7 @@ import org.apache.kafka.common.Node;
import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicIdPartition;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.Uuid; import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.errors.DisconnectException; import org.apache.kafka.common.errors.DisconnectException;
import org.apache.kafka.common.errors.RecordTooLargeException; import org.apache.kafka.common.errors.RecordTooLargeException;
import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.errors.SerializationException;
@ -731,7 +732,7 @@ public class FetchRequestManagerTest {
ByteBuffer buffer = ByteBuffer.allocate(1024); ByteBuffer buffer = ByteBuffer.allocate(1024);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V0, MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V0,
CompressionType.NONE, TimestampType.CREATE_TIME, 0L, System.currentTimeMillis(), Compression.NONE, TimestampType.CREATE_TIME, 0L, System.currentTimeMillis(),
RecordBatch.NO_PARTITION_LEADER_EPOCH); RecordBatch.NO_PARTITION_LEADER_EPOCH);
builder.append(0L, "key".getBytes(), "1".getBytes()); builder.append(0L, "key".getBytes(), "1".getBytes());
builder.append(0L, "key".getBytes(), "2".getBytes()); builder.append(0L, "key".getBytes(), "2".getBytes());
@ -764,7 +765,7 @@ public class FetchRequestManagerTest {
ByteBuffer buffer = ByteBuffer.allocate(1024); ByteBuffer buffer = ByteBuffer.allocate(1024);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE,
CompressionType.NONE, TimestampType.CREATE_TIME, 0L, System.currentTimeMillis(), Compression.NONE, TimestampType.CREATE_TIME, 0L, System.currentTimeMillis(),
partitionLeaderEpoch); partitionLeaderEpoch);
builder.append(0L, "key".getBytes(), Integer.toString(partitionLeaderEpoch).getBytes()); builder.append(0L, "key".getBytes(), Integer.toString(partitionLeaderEpoch).getBytes());
builder.append(0L, "key".getBytes(), Integer.toString(partitionLeaderEpoch).getBytes()); builder.append(0L, "key".getBytes(), Integer.toString(partitionLeaderEpoch).getBytes());
@ -772,13 +773,13 @@ public class FetchRequestManagerTest {
partitionLeaderEpoch += 7; partitionLeaderEpoch += 7;
builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, Compression.NONE,
TimestampType.CREATE_TIME, 2L, System.currentTimeMillis(), partitionLeaderEpoch); TimestampType.CREATE_TIME, 2L, System.currentTimeMillis(), partitionLeaderEpoch);
builder.append(0L, "key".getBytes(), Integer.toString(partitionLeaderEpoch).getBytes()); builder.append(0L, "key".getBytes(), Integer.toString(partitionLeaderEpoch).getBytes());
builder.close(); builder.close();
partitionLeaderEpoch += 5; partitionLeaderEpoch += 5;
builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, Compression.NONE,
TimestampType.CREATE_TIME, 3L, System.currentTimeMillis(), partitionLeaderEpoch); TimestampType.CREATE_TIME, 3L, System.currentTimeMillis(), partitionLeaderEpoch);
builder.append(0L, "key".getBytes(), Integer.toString(partitionLeaderEpoch).getBytes()); builder.append(0L, "key".getBytes(), Integer.toString(partitionLeaderEpoch).getBytes());
builder.append(0L, "key".getBytes(), Integer.toString(partitionLeaderEpoch).getBytes()); builder.append(0L, "key".getBytes(), Integer.toString(partitionLeaderEpoch).getBytes());
@ -858,7 +859,7 @@ public class FetchRequestManagerTest {
int partitionLeaderEpoch = 0; int partitionLeaderEpoch = 0;
ByteBuffer buffer = ByteBuffer.allocate(1024); ByteBuffer buffer = ByteBuffer.allocate(1024);
MemoryRecordsBuilder builder = MemoryRecords.idempotentBuilder(buffer, CompressionType.NONE, 0L, producerId, MemoryRecordsBuilder builder = MemoryRecords.idempotentBuilder(buffer, Compression.NONE, 0L, producerId,
producerEpoch, baseSequence); producerEpoch, baseSequence);
builder.append(0L, "key".getBytes(), null); builder.append(0L, "key".getBytes(), null);
builder.close(); builder.close();
@ -1046,7 +1047,7 @@ public class FetchRequestManagerTest {
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(out, MemoryRecordsBuilder builder = new MemoryRecordsBuilder(out,
DefaultRecordBatch.CURRENT_MAGIC_VALUE, DefaultRecordBatch.CURRENT_MAGIC_VALUE,
CompressionType.NONE, Compression.NONE,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
0L, 10L, 0L, (short) 0, 0, false, false, 0, 1024); 0L, 10L, 0L, (short) 0, 0, false, false, 0, 1024);
builder.append(10L, "key".getBytes(), "value".getBytes()); builder.append(10L, "key".getBytes(), "value".getBytes());
@ -1077,7 +1078,7 @@ public class FetchRequestManagerTest {
public void testParseInvalidRecordBatch() { public void testParseInvalidRecordBatch() {
buildFetcher(); buildFetcher();
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, 0L, MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, 0L,
CompressionType.NONE, TimestampType.CREATE_TIME, Compression.NONE, TimestampType.CREATE_TIME,
new SimpleRecord(1L, "a".getBytes(), "1".getBytes()), new SimpleRecord(1L, "a".getBytes(), "1".getBytes()),
new SimpleRecord(2L, "b".getBytes(), "2".getBytes()), new SimpleRecord(2L, "b".getBytes(), "2".getBytes()),
new SimpleRecord(3L, "c".getBytes(), "3".getBytes())); new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
@ -1103,7 +1104,7 @@ public class FetchRequestManagerTest {
public void testHeaders() { public void testHeaders() {
buildFetcher(); buildFetcher();
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME, 1L); MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), Compression.NONE, TimestampType.CREATE_TIME, 1L);
builder.append(0L, "key".getBytes(), "value-1".getBytes()); builder.append(0L, "key".getBytes(), "value-1".getBytes());
Header[] headersArray = new Header[1]; Header[] headersArray = new Header[1];
@ -1226,7 +1227,7 @@ public class FetchRequestManagerTest {
// this test verifies the fetcher updates the current fetched/consumed positions correctly for this case // this test verifies the fetcher updates the current fetched/consumed positions correctly for this case
buildFetcher(); buildFetcher();
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), Compression.NONE,
TimestampType.CREATE_TIME, 0L); TimestampType.CREATE_TIME, 0L);
builder.appendWithOffset(15L, 0L, "key".getBytes(), "value-1".getBytes()); builder.appendWithOffset(15L, 0L, "key".getBytes(), "value-1".getBytes());
builder.appendWithOffset(20L, 0L, "key".getBytes(), "value-2".getBytes()); builder.appendWithOffset(20L, 0L, "key".getBytes(), "value-2".getBytes());
@ -1975,7 +1976,7 @@ public class FetchRequestManagerTest {
assertEquals(100, (Double) partitionLag.metricValue(), EPSILON); assertEquals(100, (Double) partitionLag.metricValue(), EPSILON);
// recordsFetchLagMax should be hw - offset of the last message after receiving a non-empty FetchResponse // recordsFetchLagMax should be hw - offset of the last message after receiving a non-empty FetchResponse
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), Compression.NONE,
TimestampType.CREATE_TIME, 0L); TimestampType.CREATE_TIME, 0L);
for (int v = 0; v < 3; v++) for (int v = 0; v < 3; v++)
builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes()); builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes());
@ -2016,7 +2017,7 @@ public class FetchRequestManagerTest {
assertEquals(0L, (Double) partitionLead.metricValue(), EPSILON); assertEquals(0L, (Double) partitionLead.metricValue(), EPSILON);
// recordsFetchLeadMin should be position - logStartOffset after receiving a non-empty FetchResponse // recordsFetchLeadMin should be position - logStartOffset after receiving a non-empty FetchResponse
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), Compression.NONE,
TimestampType.CREATE_TIME, 0L); TimestampType.CREATE_TIME, 0L);
for (int v = 0; v < 3; v++) { for (int v = 0; v < 3; v++) {
builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes()); builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes());
@ -2060,7 +2061,7 @@ public class FetchRequestManagerTest {
assertEquals(50, (Double) partitionLag.metricValue(), EPSILON); assertEquals(50, (Double) partitionLag.metricValue(), EPSILON);
// recordsFetchLagMax should be lso - offset of the last message after receiving a non-empty FetchResponse // recordsFetchLagMax should be lso - offset of the last message after receiving a non-empty FetchResponse
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), Compression.NONE,
TimestampType.CREATE_TIME, 0L); TimestampType.CREATE_TIME, 0L);
for (int v = 0; v < 3; v++) for (int v = 0; v < 3; v++)
builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes()); builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes());
@ -2100,7 +2101,7 @@ public class FetchRequestManagerTest {
for (TopicIdPartition tp : mkSet(tidp1, tidp2)) { for (TopicIdPartition tp : mkSet(tidp1, tidp2)) {
subscriptions.seek(tp.topicPartition(), 0); subscriptions.seek(tp.topicPartition(), 0);
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), Compression.NONE,
TimestampType.CREATE_TIME, 0L); TimestampType.CREATE_TIME, 0L);
for (int v = 0; v < 3; v++) for (int v = 0; v < 3; v++)
builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes()); builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes());
@ -2141,7 +2142,7 @@ public class FetchRequestManagerTest {
KafkaMetric fetchSizeAverage = allMetrics.get(metrics.metricInstance(metricsRegistry.fetchSizeAvg)); KafkaMetric fetchSizeAverage = allMetrics.get(metrics.metricInstance(metricsRegistry.fetchSizeAvg));
KafkaMetric recordsCountAverage = allMetrics.get(metrics.metricInstance(metricsRegistry.recordsPerRequestAvg)); KafkaMetric recordsCountAverage = allMetrics.get(metrics.metricInstance(metricsRegistry.recordsPerRequestAvg));
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), Compression.NONE,
TimestampType.CREATE_TIME, 0L); TimestampType.CREATE_TIME, 0L);
for (int v = 0; v < 3; v++) for (int v = 0; v < 3; v++)
builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes()); builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes());
@ -2169,7 +2170,7 @@ public class FetchRequestManagerTest {
KafkaMetric fetchSizeAverage = allMetrics.get(metrics.metricInstance(metricsRegistry.fetchSizeAvg)); KafkaMetric fetchSizeAverage = allMetrics.get(metrics.metricInstance(metricsRegistry.fetchSizeAvg));
KafkaMetric recordsCountAverage = allMetrics.get(metrics.metricInstance(metricsRegistry.recordsPerRequestAvg)); KafkaMetric recordsCountAverage = allMetrics.get(metrics.metricInstance(metricsRegistry.recordsPerRequestAvg));
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), Compression.NONE,
TimestampType.CREATE_TIME, 0L); TimestampType.CREATE_TIME, 0L);
for (int v = 0; v < 3; v++) for (int v = 0; v < 3; v++)
builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes()); builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes());
@ -2216,7 +2217,7 @@ public class FetchRequestManagerTest {
assertEquals(1, sendFetches()); assertEquals(1, sendFetches());
subscriptions.seek(tp1, 5); subscriptions.seek(tp1, 5);
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), Compression.NONE,
TimestampType.CREATE_TIME, 0L); TimestampType.CREATE_TIME, 0L);
for (int v = 0; v < 3; v++) for (int v = 0; v < 3; v++)
builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes()); builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes());
@ -2232,7 +2233,7 @@ public class FetchRequestManagerTest {
.setPartitionIndex(tp1.partition()) .setPartitionIndex(tp1.partition())
.setHighWatermark(100) .setHighWatermark(100)
.setLogStartOffset(0) .setLogStartOffset(0)
.setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("val".getBytes())))); .setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("val".getBytes()))));
client.prepareResponse(FetchResponse.of(Errors.NONE, 0, INVALID_SESSION_ID, new LinkedHashMap<>(partitions))); client.prepareResponse(FetchResponse.of(Errors.NONE, 0, INVALID_SESSION_ID, new LinkedHashMap<>(partitions)));
networkClientDelegate.poll(time.timer(0)); networkClientDelegate.poll(time.timer(0));
@ -2534,7 +2535,7 @@ public class FetchRequestManagerTest {
public void testUpdatePositionWithLastRecordMissingFromBatch() { public void testUpdatePositionWithLastRecordMissingFromBatch() {
buildFetcher(); buildFetcher();
MemoryRecords records = MemoryRecords.withRecords(CompressionType.NONE, MemoryRecords records = MemoryRecords.withRecords(Compression.NONE,
new SimpleRecord("0".getBytes(), "v".getBytes()), new SimpleRecord("0".getBytes(), "v".getBytes()),
new SimpleRecord("1".getBytes(), "v".getBytes()), new SimpleRecord("1".getBytes(), "v".getBytes()),
new SimpleRecord("2".getBytes(), "v".getBytes()), new SimpleRecord("2".getBytes(), "v".getBytes()),
@ -2861,14 +2862,14 @@ public class FetchRequestManagerTest {
} }
private MemoryRecords buildRecords(long baseOffset, int count, long firstMessageId) { private MemoryRecords buildRecords(long baseOffset, int count, long firstMessageId) {
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME, baseOffset); MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), Compression.NONE, TimestampType.CREATE_TIME, baseOffset);
for (int i = 0; i < count; i++) for (int i = 0; i < count; i++)
builder.append(0L, "key".getBytes(), ("value-" + (firstMessageId + i)).getBytes()); builder.append(0L, "key".getBytes(), ("value-" + (firstMessageId + i)).getBytes());
return builder.build(); return builder.build();
} }
private int appendTransactionalRecords(ByteBuffer buffer, long pid, long baseOffset, int baseSequence, SimpleRecord... records) { private int appendTransactionalRecords(ByteBuffer buffer, long pid, long baseOffset, int baseSequence, SimpleRecord... records) {
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, Compression.NONE,
TimestampType.CREATE_TIME, baseOffset, time.milliseconds(), pid, (short) 0, baseSequence, true, TimestampType.CREATE_TIME, baseOffset, time.milliseconds(), pid, (short) 0, baseSequence, true,
RecordBatch.NO_PARTITION_LEADER_EPOCH); RecordBatch.NO_PARTITION_LEADER_EPOCH);
@ -2904,7 +2905,7 @@ public class FetchRequestManagerTest {
MemoryRecordsBuilder builder = MemoryRecords.builder( MemoryRecordsBuilder builder = MemoryRecords.builder(
ByteBuffer.allocate(1024), ByteBuffer.allocate(1024),
RecordBatch.CURRENT_MAGIC_VALUE, RecordBatch.CURRENT_MAGIC_VALUE,
CompressionType.NONE, Compression.NONE,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
0L, 0L,
RecordBatch.NO_TIMESTAMP, RecordBatch.NO_TIMESTAMP,

View File

@ -36,6 +36,7 @@ import org.apache.kafka.common.Node;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicIdPartition;
import org.apache.kafka.common.Uuid; import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.errors.RecordTooLargeException; import org.apache.kafka.common.errors.RecordTooLargeException;
import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.errors.SerializationException;
import org.apache.kafka.common.errors.TopicAuthorizationException; import org.apache.kafka.common.errors.TopicAuthorizationException;
@ -680,7 +681,7 @@ public class FetcherTest {
ByteBuffer buffer = ByteBuffer.allocate(1024); ByteBuffer buffer = ByteBuffer.allocate(1024);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V0, MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V0,
CompressionType.NONE, TimestampType.CREATE_TIME, 0L, System.currentTimeMillis(), Compression.NONE, TimestampType.CREATE_TIME, 0L, System.currentTimeMillis(),
RecordBatch.NO_PARTITION_LEADER_EPOCH); RecordBatch.NO_PARTITION_LEADER_EPOCH);
builder.append(0L, "key".getBytes(), "1".getBytes()); builder.append(0L, "key".getBytes(), "1".getBytes());
builder.append(0L, "key".getBytes(), "2".getBytes()); builder.append(0L, "key".getBytes(), "2".getBytes());
@ -713,7 +714,7 @@ public class FetcherTest {
ByteBuffer buffer = ByteBuffer.allocate(1024); ByteBuffer buffer = ByteBuffer.allocate(1024);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE,
CompressionType.NONE, TimestampType.CREATE_TIME, 0L, System.currentTimeMillis(), Compression.NONE, TimestampType.CREATE_TIME, 0L, System.currentTimeMillis(),
partitionLeaderEpoch); partitionLeaderEpoch);
builder.append(0L, "key".getBytes(), Integer.toString(partitionLeaderEpoch).getBytes()); builder.append(0L, "key".getBytes(), Integer.toString(partitionLeaderEpoch).getBytes());
builder.append(0L, "key".getBytes(), Integer.toString(partitionLeaderEpoch).getBytes()); builder.append(0L, "key".getBytes(), Integer.toString(partitionLeaderEpoch).getBytes());
@ -721,13 +722,13 @@ public class FetcherTest {
partitionLeaderEpoch += 7; partitionLeaderEpoch += 7;
builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, Compression.NONE,
TimestampType.CREATE_TIME, 2L, System.currentTimeMillis(), partitionLeaderEpoch); TimestampType.CREATE_TIME, 2L, System.currentTimeMillis(), partitionLeaderEpoch);
builder.append(0L, "key".getBytes(), Integer.toString(partitionLeaderEpoch).getBytes()); builder.append(0L, "key".getBytes(), Integer.toString(partitionLeaderEpoch).getBytes());
builder.close(); builder.close();
partitionLeaderEpoch += 5; partitionLeaderEpoch += 5;
builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, Compression.NONE,
TimestampType.CREATE_TIME, 3L, System.currentTimeMillis(), partitionLeaderEpoch); TimestampType.CREATE_TIME, 3L, System.currentTimeMillis(), partitionLeaderEpoch);
builder.append(0L, "key".getBytes(), Integer.toString(partitionLeaderEpoch).getBytes()); builder.append(0L, "key".getBytes(), Integer.toString(partitionLeaderEpoch).getBytes());
builder.append(0L, "key".getBytes(), Integer.toString(partitionLeaderEpoch).getBytes()); builder.append(0L, "key".getBytes(), Integer.toString(partitionLeaderEpoch).getBytes());
@ -807,7 +808,7 @@ public class FetcherTest {
int partitionLeaderEpoch = 0; int partitionLeaderEpoch = 0;
ByteBuffer buffer = ByteBuffer.allocate(1024); ByteBuffer buffer = ByteBuffer.allocate(1024);
MemoryRecordsBuilder builder = MemoryRecords.idempotentBuilder(buffer, CompressionType.NONE, 0L, producerId, MemoryRecordsBuilder builder = MemoryRecords.idempotentBuilder(buffer, Compression.NONE, 0L, producerId,
producerEpoch, baseSequence); producerEpoch, baseSequence);
builder.append(0L, "key".getBytes(), null); builder.append(0L, "key".getBytes(), null);
builder.close(); builder.close();
@ -995,7 +996,7 @@ public class FetcherTest {
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(out, MemoryRecordsBuilder builder = new MemoryRecordsBuilder(out,
DefaultRecordBatch.CURRENT_MAGIC_VALUE, DefaultRecordBatch.CURRENT_MAGIC_VALUE,
CompressionType.NONE, Compression.NONE,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
0L, 10L, 0L, (short) 0, 0, false, false, 0, 1024); 0L, 10L, 0L, (short) 0, 0, false, false, 0, 1024);
builder.append(10L, "key".getBytes(), "value".getBytes()); builder.append(10L, "key".getBytes(), "value".getBytes());
@ -1026,7 +1027,7 @@ public class FetcherTest {
public void testParseInvalidRecordBatch() { public void testParseInvalidRecordBatch() {
buildFetcher(); buildFetcher();
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, 0L, MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, 0L,
CompressionType.NONE, TimestampType.CREATE_TIME, Compression.NONE, TimestampType.CREATE_TIME,
new SimpleRecord(1L, "a".getBytes(), "1".getBytes()), new SimpleRecord(1L, "a".getBytes(), "1".getBytes()),
new SimpleRecord(2L, "b".getBytes(), "2".getBytes()), new SimpleRecord(2L, "b".getBytes(), "2".getBytes()),
new SimpleRecord(3L, "c".getBytes(), "3".getBytes())); new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
@ -1052,7 +1053,7 @@ public class FetcherTest {
public void testHeaders() { public void testHeaders() {
buildFetcher(); buildFetcher();
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME, 1L); MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), Compression.NONE, TimestampType.CREATE_TIME, 1L);
builder.append(0L, "key".getBytes(), "value-1".getBytes()); builder.append(0L, "key".getBytes(), "value-1".getBytes());
Header[] headersArray = new Header[1]; Header[] headersArray = new Header[1];
@ -1235,7 +1236,7 @@ public class FetcherTest {
// this test verifies the fetcher updates the current fetched/consumed positions correctly for this case // this test verifies the fetcher updates the current fetched/consumed positions correctly for this case
buildFetcher(); buildFetcher();
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), Compression.NONE,
TimestampType.CREATE_TIME, 0L); TimestampType.CREATE_TIME, 0L);
builder.appendWithOffset(15L, 0L, "key".getBytes(), "value-1".getBytes()); builder.appendWithOffset(15L, 0L, "key".getBytes(), "value-1".getBytes());
builder.appendWithOffset(20L, 0L, "key".getBytes(), "value-2".getBytes()); builder.appendWithOffset(20L, 0L, "key".getBytes(), "value-2".getBytes());
@ -1971,7 +1972,7 @@ public class FetcherTest {
assertEquals(100, (Double) partitionLag.metricValue(), EPSILON); assertEquals(100, (Double) partitionLag.metricValue(), EPSILON);
// recordsFetchLagMax should be hw - offset of the last message after receiving a non-empty FetchResponse // recordsFetchLagMax should be hw - offset of the last message after receiving a non-empty FetchResponse
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), Compression.NONE,
TimestampType.CREATE_TIME, 0L); TimestampType.CREATE_TIME, 0L);
for (int v = 0; v < 3; v++) for (int v = 0; v < 3; v++)
builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes()); builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes());
@ -2012,7 +2013,7 @@ public class FetcherTest {
assertEquals(0L, (Double) partitionLead.metricValue(), EPSILON); assertEquals(0L, (Double) partitionLead.metricValue(), EPSILON);
// recordsFetchLeadMin should be position - logStartOffset after receiving a non-empty FetchResponse // recordsFetchLeadMin should be position - logStartOffset after receiving a non-empty FetchResponse
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), Compression.NONE,
TimestampType.CREATE_TIME, 0L); TimestampType.CREATE_TIME, 0L);
for (int v = 0; v < 3; v++) { for (int v = 0; v < 3; v++) {
builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes()); builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes());
@ -2056,7 +2057,7 @@ public class FetcherTest {
assertEquals(50, (Double) partitionLag.metricValue(), EPSILON); assertEquals(50, (Double) partitionLag.metricValue(), EPSILON);
// recordsFetchLagMax should be lso - offset of the last message after receiving a non-empty FetchResponse // recordsFetchLagMax should be lso - offset of the last message after receiving a non-empty FetchResponse
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), Compression.NONE,
TimestampType.CREATE_TIME, 0L); TimestampType.CREATE_TIME, 0L);
for (int v = 0; v < 3; v++) for (int v = 0; v < 3; v++)
builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes()); builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes());
@ -2096,7 +2097,7 @@ public class FetcherTest {
for (TopicIdPartition tp : mkSet(tidp1, tidp2)) { for (TopicIdPartition tp : mkSet(tidp1, tidp2)) {
subscriptions.seek(tp.topicPartition(), 0); subscriptions.seek(tp.topicPartition(), 0);
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), Compression.NONE,
TimestampType.CREATE_TIME, 0L); TimestampType.CREATE_TIME, 0L);
for (int v = 0; v < 3; v++) for (int v = 0; v < 3; v++)
builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes()); builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes());
@ -2137,7 +2138,7 @@ public class FetcherTest {
KafkaMetric fetchSizeAverage = allMetrics.get(metrics.metricInstance(metricsRegistry.fetchSizeAvg)); KafkaMetric fetchSizeAverage = allMetrics.get(metrics.metricInstance(metricsRegistry.fetchSizeAvg));
KafkaMetric recordsCountAverage = allMetrics.get(metrics.metricInstance(metricsRegistry.recordsPerRequestAvg)); KafkaMetric recordsCountAverage = allMetrics.get(metrics.metricInstance(metricsRegistry.recordsPerRequestAvg));
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), Compression.NONE,
TimestampType.CREATE_TIME, 0L); TimestampType.CREATE_TIME, 0L);
for (int v = 0; v < 3; v++) for (int v = 0; v < 3; v++)
builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes()); builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes());
@ -2165,7 +2166,7 @@ public class FetcherTest {
KafkaMetric fetchSizeAverage = allMetrics.get(metrics.metricInstance(metricsRegistry.fetchSizeAvg)); KafkaMetric fetchSizeAverage = allMetrics.get(metrics.metricInstance(metricsRegistry.fetchSizeAvg));
KafkaMetric recordsCountAverage = allMetrics.get(metrics.metricInstance(metricsRegistry.recordsPerRequestAvg)); KafkaMetric recordsCountAverage = allMetrics.get(metrics.metricInstance(metricsRegistry.recordsPerRequestAvg));
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), Compression.NONE,
TimestampType.CREATE_TIME, 0L); TimestampType.CREATE_TIME, 0L);
for (int v = 0; v < 3; v++) for (int v = 0; v < 3; v++)
builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes()); builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes());
@ -2212,7 +2213,7 @@ public class FetcherTest {
assertEquals(1, sendFetches()); assertEquals(1, sendFetches());
subscriptions.seek(tp1, 5); subscriptions.seek(tp1, 5);
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), Compression.NONE,
TimestampType.CREATE_TIME, 0L); TimestampType.CREATE_TIME, 0L);
for (int v = 0; v < 3; v++) for (int v = 0; v < 3; v++)
builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes()); builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes());
@ -2228,7 +2229,7 @@ public class FetcherTest {
.setPartitionIndex(tp1.partition()) .setPartitionIndex(tp1.partition())
.setHighWatermark(100) .setHighWatermark(100)
.setLogStartOffset(0) .setLogStartOffset(0)
.setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("val".getBytes())))); .setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("val".getBytes()))));
client.prepareResponse(FetchResponse.of(Errors.NONE, 0, INVALID_SESSION_ID, new LinkedHashMap<>(partitions))); client.prepareResponse(FetchResponse.of(Errors.NONE, 0, INVALID_SESSION_ID, new LinkedHashMap<>(partitions)));
consumerClient.poll(time.timer(0)); consumerClient.poll(time.timer(0));
@ -2530,7 +2531,7 @@ public class FetcherTest {
public void testUpdatePositionWithLastRecordMissingFromBatch() { public void testUpdatePositionWithLastRecordMissingFromBatch() {
buildFetcher(); buildFetcher();
MemoryRecords records = MemoryRecords.withRecords(CompressionType.NONE, MemoryRecords records = MemoryRecords.withRecords(Compression.NONE,
new SimpleRecord("0".getBytes(), "v".getBytes()), new SimpleRecord("0".getBytes(), "v".getBytes()),
new SimpleRecord("1".getBytes(), "v".getBytes()), new SimpleRecord("1".getBytes(), "v".getBytes()),
new SimpleRecord("2".getBytes(), "v".getBytes()), new SimpleRecord("2".getBytes(), "v".getBytes()),
@ -3065,14 +3066,14 @@ public class FetcherTest {
} }
private MemoryRecords buildRecords(long baseOffset, int count, long firstMessageId) { private MemoryRecords buildRecords(long baseOffset, int count, long firstMessageId) {
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME, baseOffset); MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), Compression.NONE, TimestampType.CREATE_TIME, baseOffset);
for (int i = 0; i < count; i++) for (int i = 0; i < count; i++)
builder.append(0L, "key".getBytes(), ("value-" + (firstMessageId + i)).getBytes()); builder.append(0L, "key".getBytes(), ("value-" + (firstMessageId + i)).getBytes());
return builder.build(); return builder.build();
} }
private int appendTransactionalRecords(ByteBuffer buffer, long pid, long baseOffset, int baseSequence, SimpleRecord... records) { private int appendTransactionalRecords(ByteBuffer buffer, long pid, long baseOffset, int baseSequence, SimpleRecord... records) {
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, Compression.NONE,
TimestampType.CREATE_TIME, baseOffset, time.milliseconds(), pid, (short) 0, baseSequence, true, TimestampType.CREATE_TIME, baseOffset, time.milliseconds(), pid, (short) 0, baseSequence, true,
RecordBatch.NO_PARTITION_LEADER_EPOCH); RecordBatch.NO_PARTITION_LEADER_EPOCH);
@ -3108,7 +3109,7 @@ public class FetcherTest {
MemoryRecordsBuilder builder = MemoryRecords.builder( MemoryRecordsBuilder builder = MemoryRecords.builder(
ByteBuffer.allocate(1024), ByteBuffer.allocate(1024),
RecordBatch.CURRENT_MAGIC_VALUE, RecordBatch.CURRENT_MAGIC_VALUE,
CompressionType.NONE, Compression.NONE,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
0L, 0L,
RecordBatch.NO_TIMESTAMP, RecordBatch.NO_TIMESTAMP,
@ -3156,7 +3157,7 @@ public class FetcherTest {
MemoryRecordsBuilder builder = MemoryRecords.builder( MemoryRecordsBuilder builder = MemoryRecords.builder(
ByteBuffer.allocate(1024), ByteBuffer.allocate(1024),
RecordBatch.CURRENT_MAGIC_VALUE, RecordBatch.CURRENT_MAGIC_VALUE,
CompressionType.NONE, Compression.NONE,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
0L, 0L,
RecordBatch.NO_TIMESTAMP, RecordBatch.NO_TIMESTAMP,

View File

@ -21,6 +21,7 @@ import org.apache.kafka.clients.producer.Callback;
import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.Header;
import org.apache.kafka.common.header.internals.RecordHeader; import org.apache.kafka.common.header.internals.RecordHeader;
import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.CompressionType;
@ -58,7 +59,7 @@ public class ProducerBatchTest {
private final long now = 1488748346917L; private final long now = 1488748346917L;
private final MemoryRecordsBuilder memoryRecordsBuilder = MemoryRecords.builder(ByteBuffer.allocate(512), private final MemoryRecordsBuilder memoryRecordsBuilder = MemoryRecords.builder(ByteBuffer.allocate(512),
CompressionType.NONE, TimestampType.CREATE_TIME, 128); Compression.NONE, TimestampType.CREATE_TIME, 128);
@Test @Test
public void testBatchAbort() throws Exception { public void testBatchAbort() throws Exception {
@ -136,7 +137,7 @@ public class ProducerBatchTest {
MemoryRecordsBuilder builder = MemoryRecords.builder( MemoryRecordsBuilder builder = MemoryRecords.builder(
ByteBuffer.allocate(1024), ByteBuffer.allocate(1024),
MAGIC_VALUE_V2, MAGIC_VALUE_V2,
compressionType, Compression.of(compressionType).build(),
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
0L); 0L);
ProducerBatch batch = new ProducerBatch(new TopicPartition("topic", 1), builder, now); ProducerBatch batch = new ProducerBatch(new TopicPartition("topic", 1), builder, now);
@ -176,7 +177,7 @@ public class ProducerBatchTest {
continue; continue;
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), magic, MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), magic,
compressionType, TimestampType.CREATE_TIME, 0L); Compression.of(compressionType).build(), TimestampType.CREATE_TIME, 0L);
ProducerBatch batch = new ProducerBatch(new TopicPartition("topic", 1), builder, now); ProducerBatch batch = new ProducerBatch(new TopicPartition("topic", 1), builder, now);
while (true) { while (true) {

View File

@ -31,6 +31,7 @@ import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.Node; import org.apache.kafka.common.Node;
import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
@ -157,7 +158,7 @@ public class RecordAccumulatorTest {
Collections.emptyMap(), Collections.emptyMap(),
cluster); cluster);
long batchSize = value.length + DefaultRecordBatch.RECORD_BATCH_OVERHEAD; long batchSize = value.length + DefaultRecordBatch.RECORD_BATCH_OVERHEAD;
RecordAccumulator accum = createTestRecordAccumulator((int) batchSize, Integer.MAX_VALUE, CompressionType.NONE, 10); RecordAccumulator accum = createTestRecordAccumulator((int) batchSize, Integer.MAX_VALUE, Compression.NONE, 10);
// initial data // initial data
@ -225,7 +226,7 @@ public class RecordAccumulatorTest {
int batchSize = 1025; int batchSize = 1025;
RecordAccumulator accum = createTestRecordAccumulator( RecordAccumulator accum = createTestRecordAccumulator(
batchSize + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 10L * batchSize, CompressionType.NONE, 10); batchSize + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 10L * batchSize, Compression.NONE, 10);
int appends = expectedNumAppends(batchSize); int appends = expectedNumAppends(batchSize);
for (int i = 0; i < appends; i++) { for (int i = 0; i < appends; i++) {
// append to the first batch // append to the first batch
@ -262,19 +263,19 @@ public class RecordAccumulatorTest {
@Test @Test
public void testAppendLargeCompressed() throws Exception { public void testAppendLargeCompressed() throws Exception {
testAppendLarge(CompressionType.GZIP); testAppendLarge(Compression.gzip().build());
} }
@Test @Test
public void testAppendLargeNonCompressed() throws Exception { public void testAppendLargeNonCompressed() throws Exception {
testAppendLarge(CompressionType.NONE); testAppendLarge(Compression.NONE);
} }
private void testAppendLarge(CompressionType compressionType) throws Exception { private void testAppendLarge(Compression compression) throws Exception {
int batchSize = 512; int batchSize = 512;
byte[] value = new byte[2 * batchSize]; byte[] value = new byte[2 * batchSize];
RecordAccumulator accum = createTestRecordAccumulator( RecordAccumulator accum = createTestRecordAccumulator(
batchSize + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 10 * 1024, compressionType, 0); batchSize + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 10 * 1024, compression, 0);
accum.append(topic, partition1, 0L, key, value, Record.EMPTY_HEADERS, null, maxBlockTimeMs, false, time.milliseconds(), metadataCache.cluster()); accum.append(topic, partition1, 0L, key, value, Record.EMPTY_HEADERS, null, maxBlockTimeMs, false, time.milliseconds(), metadataCache.cluster());
assertEquals(Collections.singleton(node1), accum.ready(metadataCache, time.milliseconds()).readyNodes, "Our partition's leader should be ready"); assertEquals(Collections.singleton(node1), accum.ready(metadataCache, time.milliseconds()).readyNodes, "Our partition's leader should be ready");
@ -296,15 +297,15 @@ public class RecordAccumulatorTest {
@Test @Test
public void testAppendLargeOldMessageFormatCompressed() throws Exception { public void testAppendLargeOldMessageFormatCompressed() throws Exception {
testAppendLargeOldMessageFormat(CompressionType.GZIP); testAppendLargeOldMessageFormat(Compression.gzip().build());
} }
@Test @Test
public void testAppendLargeOldMessageFormatNonCompressed() throws Exception { public void testAppendLargeOldMessageFormatNonCompressed() throws Exception {
testAppendLargeOldMessageFormat(CompressionType.NONE); testAppendLargeOldMessageFormat(Compression.NONE);
} }
private void testAppendLargeOldMessageFormat(CompressionType compressionType) throws Exception { private void testAppendLargeOldMessageFormat(Compression compression) throws Exception {
int batchSize = 512; int batchSize = 512;
byte[] value = new byte[2 * batchSize]; byte[] value = new byte[2 * batchSize];
@ -312,7 +313,7 @@ public class RecordAccumulatorTest {
apiVersions.update(node1.idString(), NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 2)); apiVersions.update(node1.idString(), NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 2));
RecordAccumulator accum = createTestRecordAccumulator( RecordAccumulator accum = createTestRecordAccumulator(
batchSize + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 10 * 1024, compressionType, 0); batchSize + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 10 * 1024, compression, 0);
accum.append(topic, partition1, 0L, key, value, Record.EMPTY_HEADERS, null, maxBlockTimeMs, false, time.milliseconds(), metadataCache.cluster()); accum.append(topic, partition1, 0L, key, value, Record.EMPTY_HEADERS, null, maxBlockTimeMs, false, time.milliseconds(), metadataCache.cluster());
assertEquals(Collections.singleton(node1), accum.ready(metadataCache, time.milliseconds()).readyNodes, "Our partition's leader should be ready"); assertEquals(Collections.singleton(node1), accum.ready(metadataCache, time.milliseconds()).readyNodes, "Our partition's leader should be ready");
@ -336,7 +337,7 @@ public class RecordAccumulatorTest {
public void testLinger() throws Exception { public void testLinger() throws Exception {
int lingerMs = 10; int lingerMs = 10;
RecordAccumulator accum = createTestRecordAccumulator( RecordAccumulator accum = createTestRecordAccumulator(
1024 + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 10 * 1024, CompressionType.NONE, lingerMs); 1024 + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 10 * 1024, Compression.NONE, lingerMs);
accum.append(topic, partition1, 0L, key, value, Record.EMPTY_HEADERS, null, maxBlockTimeMs, false, time.milliseconds(), cluster); accum.append(topic, partition1, 0L, key, value, Record.EMPTY_HEADERS, null, maxBlockTimeMs, false, time.milliseconds(), cluster);
assertEquals(0, accum.ready(metadataCache, time.milliseconds()).readyNodes.size(), "No partitions should be ready"); assertEquals(0, accum.ready(metadataCache, time.milliseconds()).readyNodes.size(), "No partitions should be ready");
time.sleep(10); time.sleep(10);
@ -355,7 +356,7 @@ public class RecordAccumulatorTest {
@Test @Test
public void testPartialDrain() throws Exception { public void testPartialDrain() throws Exception {
RecordAccumulator accum = createTestRecordAccumulator( RecordAccumulator accum = createTestRecordAccumulator(
1024 + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 10 * 1024, CompressionType.NONE, 10); 1024 + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 10 * 1024, Compression.NONE, 10);
int appends = 1024 / msgSize + 1; int appends = 1024 / msgSize + 1;
List<TopicPartition> partitions = asList(tp1, tp2); List<TopicPartition> partitions = asList(tp1, tp2);
for (TopicPartition tp : partitions) { for (TopicPartition tp : partitions) {
@ -375,7 +376,7 @@ public class RecordAccumulatorTest {
final int msgs = 10000; final int msgs = 10000;
final int numParts = 2; final int numParts = 2;
final RecordAccumulator accum = createTestRecordAccumulator( final RecordAccumulator accum = createTestRecordAccumulator(
1024 + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 10 * 1024, CompressionType.NONE, 0); 1024 + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 10 * 1024, Compression.NONE, 0);
List<Thread> threads = new ArrayList<>(); List<Thread> threads = new ArrayList<>();
for (int i = 0; i < numThreads; i++) { for (int i = 0; i < numThreads; i++) {
threads.add(new Thread(() -> { threads.add(new Thread(() -> {
@ -418,7 +419,7 @@ public class RecordAccumulatorTest {
int batchSize = 1025; int batchSize = 1025;
RecordAccumulator accum = createTestRecordAccumulator(batchSize + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, RecordAccumulator accum = createTestRecordAccumulator(batchSize + DefaultRecordBatch.RECORD_BATCH_OVERHEAD,
10 * batchSize, CompressionType.NONE, lingerMs); 10 * batchSize, Compression.NONE, lingerMs);
// Just short of going over the limit so we trigger linger time // Just short of going over the limit so we trigger linger time
int appends = expectedNumAppends(batchSize); int appends = expectedNumAppends(batchSize);
@ -459,7 +460,7 @@ public class RecordAccumulatorTest {
String metricGrpName = "producer-metrics"; String metricGrpName = "producer-metrics";
final RecordAccumulator accum = new RecordAccumulator(logContext, batchSize, final RecordAccumulator accum = new RecordAccumulator(logContext, batchSize,
CompressionType.NONE, lingerMs, retryBackoffMs, retryBackoffMaxMs, Compression.NONE, lingerMs, retryBackoffMs, retryBackoffMaxMs,
deliveryTimeoutMs, metrics, metricGrpName, time, new ApiVersions(), null, deliveryTimeoutMs, metrics, metricGrpName, time, new ApiVersions(), null,
new BufferPool(totalSize, batchSize, metrics, time, metricGrpName)); new BufferPool(totalSize, batchSize, metrics, time, metricGrpName));
@ -524,7 +525,7 @@ public class RecordAccumulatorTest {
String metricGrpName = "producer-metrics"; String metricGrpName = "producer-metrics";
final RecordAccumulator accum = new RecordAccumulator(logContext, batchSize, final RecordAccumulator accum = new RecordAccumulator(logContext, batchSize,
CompressionType.NONE, lingerMs, retryBackoffMs, retryBackoffMaxMs, Compression.NONE, lingerMs, retryBackoffMs, retryBackoffMaxMs,
deliveryTimeoutMs, metrics, metricGrpName, time, new ApiVersions(), null, deliveryTimeoutMs, metrics, metricGrpName, time, new ApiVersions(), null,
new BufferPool(totalSize, batchSize, metrics, time, metricGrpName)); new BufferPool(totalSize, batchSize, metrics, time, metricGrpName));
@ -585,7 +586,7 @@ public class RecordAccumulatorTest {
Collections.emptyMap()); Collections.emptyMap());
final RecordAccumulator accum = new RecordAccumulator(logContext, batchSize, final RecordAccumulator accum = new RecordAccumulator(logContext, batchSize,
CompressionType.NONE, lingerMs, retryBackoffMs, retryBackoffMaxMs, Compression.NONE, lingerMs, retryBackoffMs, retryBackoffMaxMs,
deliveryTimeoutMs, metrics, metricGrpName, time, new ApiVersions(), null, deliveryTimeoutMs, metrics, metricGrpName, time, new ApiVersions(), null,
new BufferPool(totalSize, batchSize, metrics, time, metricGrpName)); new BufferPool(totalSize, batchSize, metrics, time, metricGrpName));
@ -645,7 +646,7 @@ public class RecordAccumulatorTest {
public void testFlush() throws Exception { public void testFlush() throws Exception {
int lingerMs = Integer.MAX_VALUE; int lingerMs = Integer.MAX_VALUE;
final RecordAccumulator accum = createTestRecordAccumulator( final RecordAccumulator accum = createTestRecordAccumulator(
4 * 1024 + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 64 * 1024, CompressionType.NONE, lingerMs); 4 * 1024 + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 64 * 1024, Compression.NONE, lingerMs);
for (int i = 0; i < 100; i++) { for (int i = 0; i < 100; i++) {
accum.append(topic, i % 3, 0L, key, value, Record.EMPTY_HEADERS, null, maxBlockTimeMs, false, time.milliseconds(), cluster); accum.append(topic, i % 3, 0L, key, value, Record.EMPTY_HEADERS, null, maxBlockTimeMs, false, time.milliseconds(), cluster);
@ -683,7 +684,7 @@ public class RecordAccumulatorTest {
@Test @Test
public void testAwaitFlushComplete() throws Exception { public void testAwaitFlushComplete() throws Exception {
RecordAccumulator accum = createTestRecordAccumulator( RecordAccumulator accum = createTestRecordAccumulator(
4 * 1024 + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 64 * 1024, CompressionType.NONE, Integer.MAX_VALUE); 4 * 1024 + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 64 * 1024, Compression.NONE, Integer.MAX_VALUE);
accum.append(topic, 0, 0L, key, value, Record.EMPTY_HEADERS, null, maxBlockTimeMs, false, time.milliseconds(), cluster); accum.append(topic, 0, 0L, key, value, Record.EMPTY_HEADERS, null, maxBlockTimeMs, false, time.milliseconds(), cluster);
accum.beginFlush(); accum.beginFlush();
@ -704,7 +705,7 @@ public class RecordAccumulatorTest {
final AtomicInteger numExceptionReceivedInCallback = new AtomicInteger(0); final AtomicInteger numExceptionReceivedInCallback = new AtomicInteger(0);
final RecordAccumulator accum = createTestRecordAccumulator( final RecordAccumulator accum = createTestRecordAccumulator(
128 + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 64 * 1024, CompressionType.NONE, lingerMs); 128 + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 64 * 1024, Compression.NONE, lingerMs);
class TestCallback implements RecordAccumulator.AppendCallbacks { class TestCallback implements RecordAccumulator.AppendCallbacks {
@Override @Override
public void onCompletion(RecordMetadata metadata, Exception exception) { public void onCompletion(RecordMetadata metadata, Exception exception) {
@ -747,7 +748,7 @@ public class RecordAccumulatorTest {
final AtomicInteger numExceptionReceivedInCallback = new AtomicInteger(0); final AtomicInteger numExceptionReceivedInCallback = new AtomicInteger(0);
final RecordAccumulator accum = createTestRecordAccumulator( final RecordAccumulator accum = createTestRecordAccumulator(
128 + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 64 * 1024, CompressionType.NONE, lingerMs); 128 + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 64 * 1024, Compression.NONE, lingerMs);
final KafkaException cause = new KafkaException(); final KafkaException cause = new KafkaException();
class TestCallback implements RecordAccumulator.AppendCallbacks { class TestCallback implements RecordAccumulator.AppendCallbacks {
@ -795,7 +796,7 @@ public class RecordAccumulatorTest {
// test case assumes that the records do not fill the batch completely // test case assumes that the records do not fill the batch completely
int batchSize = 1025; int batchSize = 1025;
RecordAccumulator accum = createTestRecordAccumulator(deliveryTimeoutMs, RecordAccumulator accum = createTestRecordAccumulator(deliveryTimeoutMs,
batchSize + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 10 * batchSize, CompressionType.NONE, lingerMs); batchSize + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 10 * batchSize, Compression.NONE, lingerMs);
// Make the batches ready due to linger. These batches are not in retry // Make the batches ready due to linger. These batches are not in retry
for (Boolean mute: muteStates) { for (Boolean mute: muteStates) {
@ -845,7 +846,7 @@ public class RecordAccumulatorTest {
int batchSize = 1025; int batchSize = 1025;
RecordAccumulator accum = createTestRecordAccumulator( RecordAccumulator accum = createTestRecordAccumulator(
deliveryTimeoutMs, batchSize + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 10 * batchSize, CompressionType.NONE, lingerMs); deliveryTimeoutMs, batchSize + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 10 * batchSize, Compression.NONE, lingerMs);
int appends = expectedNumAppends(batchSize); int appends = expectedNumAppends(batchSize);
// Test batches not in retry // Test batches not in retry
@ -938,7 +939,7 @@ public class RecordAccumulatorTest {
int batchSize = 1025; int batchSize = 1025;
RecordAccumulator accum = createTestRecordAccumulator( RecordAccumulator accum = createTestRecordAccumulator(
batchSize + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 10 * batchSize, CompressionType.NONE, 10); batchSize + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 10 * batchSize, Compression.NONE, 10);
int appends = expectedNumAppends(batchSize); int appends = expectedNumAppends(batchSize);
for (int i = 0; i < appends; i++) { for (int i = 0; i < appends; i++) {
accum.append(topic, partition1, 0L, key, value, Record.EMPTY_HEADERS, null, maxBlockTimeMs, false, time.milliseconds(), cluster); accum.append(topic, partition1, 0L, key, value, Record.EMPTY_HEADERS, null, maxBlockTimeMs, false, time.milliseconds(), cluster);
@ -981,7 +982,7 @@ public class RecordAccumulatorTest {
apiVersions.update("foobar", NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 2)); apiVersions.update("foobar", NodeApiVersions.create(ApiKeys.PRODUCE.id, (short) 0, (short) 2));
TransactionManager transactionManager = new TransactionManager(new LogContext(), null, 0, retryBackoffMs, apiVersions); TransactionManager transactionManager = new TransactionManager(new LogContext(), null, 0, retryBackoffMs, apiVersions);
RecordAccumulator accum = new RecordAccumulator(logContext, batchSize + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, RecordAccumulator accum = new RecordAccumulator(logContext, batchSize + DefaultRecordBatch.RECORD_BATCH_OVERHEAD,
CompressionType.NONE, lingerMs, retryBackoffMs, retryBackoffMs, deliveryTimeoutMs, metrics, metricGrpName, time, apiVersions, transactionManager, Compression.NONE, lingerMs, retryBackoffMs, retryBackoffMs, deliveryTimeoutMs, metrics, metricGrpName, time, apiVersions, transactionManager,
new BufferPool(totalSize, batchSize, metrics, time, metricGrpName)); new BufferPool(totalSize, batchSize, metrics, time, metricGrpName));
assertThrows(UnsupportedVersionException.class, assertThrows(UnsupportedVersionException.class,
() -> accum.append(topic, partition1, 0L, key, value, Record.EMPTY_HEADERS, null, 0, false, time.milliseconds(), cluster)); () -> accum.append(topic, partition1, 0L, key, value, Record.EMPTY_HEADERS, null, 0, false, time.milliseconds(), cluster));
@ -996,7 +997,7 @@ public class RecordAccumulatorTest {
TransactionManager transactionManager = Mockito.mock(TransactionManager.class); TransactionManager transactionManager = Mockito.mock(TransactionManager.class);
RecordAccumulator accumulator = createTestRecordAccumulator(transactionManager, deliveryTimeoutMs, RecordAccumulator accumulator = createTestRecordAccumulator(transactionManager, deliveryTimeoutMs,
batchSize, totalSize, CompressionType.NONE, lingerMs); batchSize, totalSize, Compression.NONE, lingerMs);
ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(12345L, (short) 5); ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(12345L, (short) 5);
Mockito.when(transactionManager.producerIdAndEpoch()).thenReturn(producerIdAndEpoch); Mockito.when(transactionManager.producerIdAndEpoch()).thenReturn(producerIdAndEpoch);
@ -1036,11 +1037,11 @@ public class RecordAccumulatorTest {
@Test @Test
public void testSplitAndReenqueue() throws ExecutionException, InterruptedException { public void testSplitAndReenqueue() throws ExecutionException, InterruptedException {
long now = time.milliseconds(); long now = time.milliseconds();
RecordAccumulator accum = createTestRecordAccumulator(1024, 10 * 1024, CompressionType.GZIP, 10); RecordAccumulator accum = createTestRecordAccumulator(1024, 10 * 1024, Compression.gzip().build(), 10);
// Create a big batch // Create a big batch
ByteBuffer buffer = ByteBuffer.allocate(4096); ByteBuffer buffer = ByteBuffer.allocate(4096);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, CompressionType.NONE, TimestampType.CREATE_TIME, 0L); MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, Compression.NONE, TimestampType.CREATE_TIME, 0L);
ProducerBatch batch = new ProducerBatch(tp1, builder, now, true); ProducerBatch batch = new ProducerBatch(tp1, builder, now, true);
byte[] value = new byte[1024]; byte[] value = new byte[1024];
@ -1091,7 +1092,7 @@ public class RecordAccumulatorTest {
// First set the compression ratio estimation to be good. // First set the compression ratio estimation to be good.
CompressionRatioEstimator.setEstimation(tp1.topic(), CompressionType.GZIP, 0.1f); CompressionRatioEstimator.setEstimation(tp1.topic(), CompressionType.GZIP, 0.1f);
RecordAccumulator accum = createTestRecordAccumulator(batchSize, bufferCapacity, CompressionType.GZIP, 0); RecordAccumulator accum = createTestRecordAccumulator(batchSize, bufferCapacity, Compression.gzip().build(), 0);
int numSplitBatches = prepareSplitBatches(accum, seed, 100, 20); int numSplitBatches = prepareSplitBatches(accum, seed, 100, 20);
assertTrue(numSplitBatches > 0, "There should be some split batches"); assertTrue(numSplitBatches > 0, "There should be some split batches");
// Drain all the split batches. // Drain all the split batches.
@ -1115,7 +1116,7 @@ public class RecordAccumulatorTest {
final int batchSize = 1024; final int batchSize = 1024;
final int numMessages = 1000; final int numMessages = 1000;
RecordAccumulator accum = createTestRecordAccumulator(batchSize, 3 * 1024, CompressionType.GZIP, 10); RecordAccumulator accum = createTestRecordAccumulator(batchSize, 3 * 1024, Compression.gzip().build(), 10);
// Adjust the high and low compression ratio message percentage // Adjust the high and low compression ratio message percentage
for (int goodCompRatioPercentage = 1; goodCompRatioPercentage < 100; goodCompRatioPercentage++) { for (int goodCompRatioPercentage = 1; goodCompRatioPercentage < 100; goodCompRatioPercentage++) {
int numSplit = 0; int numSplit = 0;
@ -1146,7 +1147,7 @@ public class RecordAccumulatorTest {
int batchSize = 1025; int batchSize = 1025;
RecordAccumulator accum = createTestRecordAccumulator( RecordAccumulator accum = createTestRecordAccumulator(
batchSize + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 10 * batchSize, CompressionType.NONE, lingerMs); batchSize + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 10 * batchSize, Compression.NONE, lingerMs);
accum.append(topic, partition1, 0L, key, value, Record.EMPTY_HEADERS, null, maxBlockTimeMs, false, time.milliseconds(), cluster); accum.append(topic, partition1, 0L, key, value, Record.EMPTY_HEADERS, null, maxBlockTimeMs, false, time.milliseconds(), cluster);
Set<Node> readyNodes = accum.ready(metadataCache, time.milliseconds()).readyNodes; Set<Node> readyNodes = accum.ready(metadataCache, time.milliseconds()).readyNodes;
@ -1184,7 +1185,7 @@ public class RecordAccumulatorTest {
// test case assumes that the records do not fill the batch completely // test case assumes that the records do not fill the batch completely
int batchSize = 1025; int batchSize = 1025;
RecordAccumulator accum = createTestRecordAccumulator( RecordAccumulator accum = createTestRecordAccumulator(
batchSize + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 10 * batchSize, CompressionType.NONE, lingerMs); batchSize + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 10 * batchSize, Compression.NONE, lingerMs);
// Test batches in retry. // Test batches in retry.
for (Boolean mute : muteStates) { for (Boolean mute : muteStates) {
@ -1220,7 +1221,7 @@ public class RecordAccumulatorTest {
Partitioner partitioner = new DefaultPartitioner(); Partitioner partitioner = new DefaultPartitioner();
RecordAccumulator accum = createTestRecordAccumulator(3200, RecordAccumulator accum = createTestRecordAccumulator(3200,
batchSize + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 10L * batchSize, CompressionType.NONE, 10); batchSize + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 10L * batchSize, Compression.NONE, 10);
int expectedAppends = expectedNumAppendsNoKey(batchSize); int expectedAppends = expectedNumAppendsNoKey(batchSize);
// Create first batch // Create first batch
@ -1293,7 +1294,7 @@ public class RecordAccumulatorTest {
long totalSize = 1024 * 1024; long totalSize = 1024 * 1024;
int batchSize = 1024; // note that this is also a "sticky" limit for the partitioner int batchSize = 1024; // note that this is also a "sticky" limit for the partitioner
RecordAccumulator accum = createTestRecordAccumulator(batchSize, totalSize, CompressionType.NONE, 0); RecordAccumulator accum = createTestRecordAccumulator(batchSize, totalSize, Compression.NONE, 0);
// Set up callbacks so that we know what partition is chosen. // Set up callbacks so that we know what partition is chosen.
final AtomicInteger partition = new AtomicInteger(RecordMetadata.UNKNOWN_PARTITION); final AtomicInteger partition = new AtomicInteger(RecordMetadata.UNKNOWN_PARTITION);
@ -1366,7 +1367,7 @@ public class RecordAccumulatorTest {
RecordAccumulator.PartitionerConfig config = new RecordAccumulator.PartitionerConfig(true, 100); RecordAccumulator.PartitionerConfig config = new RecordAccumulator.PartitionerConfig(true, 100);
long totalSize = 1024 * 1024; long totalSize = 1024 * 1024;
int batchSize = 128; int batchSize = 128;
RecordAccumulator accum = new RecordAccumulator(logContext, batchSize, CompressionType.NONE, 0, 0L, 0L, RecordAccumulator accum = new RecordAccumulator(logContext, batchSize, Compression.NONE, 0, 0L, 0L,
3200, config, metrics, "producer-metrics", time, new ApiVersions(), null, 3200, config, metrics, "producer-metrics", time, new ApiVersions(), null,
new BufferPool(totalSize, batchSize, metrics, time, "producer-internal-metrics")); new BufferPool(totalSize, batchSize, metrics, time, "producer-internal-metrics"));
@ -1451,7 +1452,7 @@ public class RecordAccumulatorTest {
long totalSize = 1024 * 1024; long totalSize = 1024 * 1024;
int batchSize = 512; // note that this is also a "sticky" limit for the partitioner int batchSize = 512; // note that this is also a "sticky" limit for the partitioner
int valSize = 32; int valSize = 32;
RecordAccumulator accum = createTestRecordAccumulator(batchSize, totalSize, CompressionType.NONE, 10); RecordAccumulator accum = createTestRecordAccumulator(batchSize, totalSize, Compression.NONE, 10);
byte[] value = new byte[valSize]; byte[] value = new byte[valSize];
for (int c = 10; c-- > 0; ) { for (int c = 10; c-- > 0; ) {
@ -1495,7 +1496,7 @@ public class RecordAccumulatorTest {
long totalSize = 10 * 1024; long totalSize = 10 * 1024;
String metricGrpName = "producer-metrics"; String metricGrpName = "producer-metrics";
final RecordAccumulator accum = new RecordAccumulator(logContext, batchSize, final RecordAccumulator accum = new RecordAccumulator(logContext, batchSize,
CompressionType.NONE, lingerMs, retryBackoffMs, retryBackoffMaxMs, Compression.NONE, lingerMs, retryBackoffMs, retryBackoffMaxMs,
deliveryTimeoutMs, metrics, metricGrpName, time, new ApiVersions(), null, deliveryTimeoutMs, metrics, metricGrpName, time, new ApiVersions(), null,
new BufferPool(totalSize, batchSize, metrics, time, metricGrpName)); new BufferPool(totalSize, batchSize, metrics, time, metricGrpName));
@ -1600,8 +1601,7 @@ public class RecordAccumulatorTest {
int batchSize = 10; int batchSize = 10;
int lingerMs = 10; int lingerMs = 10;
long totalSize = 10 * 1024; long totalSize = 10 * 1024;
RecordAccumulator accum = createTestRecordAccumulator(batchSize, totalSize, RecordAccumulator accum = createTestRecordAccumulator(batchSize, totalSize, Compression.NONE, lingerMs);
CompressionType.NONE, lingerMs);
// Create cluster metadata, node2 doesn't host any partitions. // Create cluster metadata, node2 doesn't host any partitions.
PartitionMetadata part1Metadata = new PartitionMetadata(Errors.NONE, tp1, Optional.of(node1.id()), Optional.empty(), null, null, null); PartitionMetadata part1Metadata = new PartitionMetadata(Errors.NONE, tp1, Optional.of(node1.id()), Optional.empty(), null, null, null);
@ -1724,13 +1724,13 @@ public class RecordAccumulatorTest {
} }
} }
private RecordAccumulator createTestRecordAccumulator(int batchSize, long totalSize, CompressionType type, int lingerMs) { private RecordAccumulator createTestRecordAccumulator(int batchSize, long totalSize, Compression compression, int lingerMs) {
int deliveryTimeoutMs = 3200; int deliveryTimeoutMs = 3200;
return createTestRecordAccumulator(deliveryTimeoutMs, batchSize, totalSize, type, lingerMs); return createTestRecordAccumulator(deliveryTimeoutMs, batchSize, totalSize, compression, lingerMs);
} }
private RecordAccumulator createTestRecordAccumulator(int deliveryTimeoutMs, int batchSize, long totalSize, CompressionType type, int lingerMs) { private RecordAccumulator createTestRecordAccumulator(int deliveryTimeoutMs, int batchSize, long totalSize, Compression compression, int lingerMs) {
return createTestRecordAccumulator(null, deliveryTimeoutMs, batchSize, totalSize, type, lingerMs); return createTestRecordAccumulator(null, deliveryTimeoutMs, batchSize, totalSize, compression, lingerMs);
} }
/** /**
@ -1741,7 +1741,7 @@ public class RecordAccumulatorTest {
int deliveryTimeoutMs, int deliveryTimeoutMs,
int batchSize, int batchSize,
long totalSize, long totalSize,
CompressionType type, Compression compression,
int lingerMs int lingerMs
) { ) {
long retryBackoffMs = 100L; long retryBackoffMs = 100L;
@ -1751,7 +1751,7 @@ public class RecordAccumulatorTest {
return new RecordAccumulator( return new RecordAccumulator(
logContext, logContext,
batchSize, batchSize,
type, compression,
lingerMs, lingerMs,
retryBackoffMs, retryBackoffMs,
retryBackoffMaxMs, retryBackoffMaxMs,

View File

@ -32,6 +32,7 @@ import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.MetricNameTemplate; import org.apache.kafka.common.MetricNameTemplate;
import org.apache.kafka.common.Node; import org.apache.kafka.common.Node;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.errors.ClusterAuthorizationException; import org.apache.kafka.common.errors.ClusterAuthorizationException;
import org.apache.kafka.common.errors.InvalidRequestException; import org.apache.kafka.common.errors.InvalidRequestException;
import org.apache.kafka.common.errors.InvalidTxnStateException; import org.apache.kafka.common.errors.InvalidTxnStateException;
@ -569,7 +570,7 @@ public class SenderTest {
// otherwise it wouldn't update the stats. // otherwise it wouldn't update the stats.
RecordAccumulator.PartitionerConfig config = new RecordAccumulator.PartitionerConfig(false, 42); RecordAccumulator.PartitionerConfig config = new RecordAccumulator.PartitionerConfig(false, 42);
long totalSize = 1024 * 1024; long totalSize = 1024 * 1024;
accumulator = new RecordAccumulator(logContext, batchSize, CompressionType.NONE, 0, 0L, 0L, accumulator = new RecordAccumulator(logContext, batchSize, Compression.NONE, 0, 0L, 0L,
DELIVERY_TIMEOUT_MS, config, m, "producer-metrics", time, apiVersions, null, DELIVERY_TIMEOUT_MS, config, m, "producer-metrics", time, apiVersions, null,
new BufferPool(totalSize, batchSize, m, time, "producer-internal-metrics")); new BufferPool(totalSize, batchSize, m, time, "producer-internal-metrics"));
@ -2417,7 +2418,7 @@ public class SenderTest {
// Set a good compression ratio. // Set a good compression ratio.
CompressionRatioEstimator.setEstimation(topic, CompressionType.GZIP, 0.2f); CompressionRatioEstimator.setEstimation(topic, CompressionType.GZIP, 0.2f);
try (Metrics m = new Metrics()) { try (Metrics m = new Metrics()) {
accumulator = new RecordAccumulator(logContext, batchSize, CompressionType.GZIP, accumulator = new RecordAccumulator(logContext, batchSize, Compression.gzip().build(),
0, 0L, 0L, deliveryTimeoutMs, m, metricGrpName, time, new ApiVersions(), txnManager, 0, 0L, 0L, deliveryTimeoutMs, m, metricGrpName, time, new ApiVersions(), txnManager,
new BufferPool(totalSize, batchSize, metrics, time, "producer-internal-metrics")); new BufferPool(totalSize, batchSize, metrics, time, "producer-internal-metrics"));
SenderMetricsRegistry senderMetrics = new SenderMetricsRegistry(m); SenderMetricsRegistry senderMetrics = new SenderMetricsRegistry(m);
@ -3200,7 +3201,7 @@ public class SenderTest {
long retryBackoffMaxMs = 100L; long retryBackoffMaxMs = 100L;
// lingerMs is 0 to send batch as soon as any records are available on it. // lingerMs is 0 to send batch as soon as any records are available on it.
this.accumulator = new RecordAccumulator(logContext, batchSize, this.accumulator = new RecordAccumulator(logContext, batchSize,
CompressionType.NONE, 0, 10L, retryBackoffMaxMs, Compression.NONE, 0, 10L, retryBackoffMaxMs,
DELIVERY_TIMEOUT_MS, metrics, metricGrpName, time, apiVersions, null, pool); DELIVERY_TIMEOUT_MS, metrics, metricGrpName, time, apiVersions, null, pool);
Sender sender = new Sender(logContext, client, metadata, this.accumulator, false, Sender sender = new Sender(logContext, client, metadata, this.accumulator, false,
MAX_REQUEST_SIZE, ACKS_ALL, MAX_REQUEST_SIZE, ACKS_ALL,
@ -3314,7 +3315,7 @@ public class SenderTest {
long retryBackoffMaxMs = 100L; long retryBackoffMaxMs = 100L;
// lingerMs is 0 to send batch as soon as any records are available on it. // lingerMs is 0 to send batch as soon as any records are available on it.
this.accumulator = new RecordAccumulator(logContext, batchSize, this.accumulator = new RecordAccumulator(logContext, batchSize,
CompressionType.NONE, 0, 10L, retryBackoffMaxMs, Compression.NONE, 0, 10L, retryBackoffMaxMs,
DELIVERY_TIMEOUT_MS, metrics, metricGrpName, time, apiVersions, null, pool); DELIVERY_TIMEOUT_MS, metrics, metricGrpName, time, apiVersions, null, pool);
Sender sender = new Sender(logContext, client, metadata, this.accumulator, false, Sender sender = new Sender(logContext, client, metadata, this.accumulator, false,
MAX_REQUEST_SIZE, ACKS_ALL, MAX_REQUEST_SIZE, ACKS_ALL,
@ -3394,7 +3395,7 @@ public class SenderTest {
long retryBackoffMaxMs = 100L; long retryBackoffMaxMs = 100L;
// lingerMs is 0 to send batch as soon as any records are available on it. // lingerMs is 0 to send batch as soon as any records are available on it.
this.accumulator = new RecordAccumulator(logContext, batchSize, this.accumulator = new RecordAccumulator(logContext, batchSize,
CompressionType.NONE, 0, 10L, retryBackoffMaxMs, Compression.NONE, 0, 10L, retryBackoffMaxMs,
DELIVERY_TIMEOUT_MS, metrics, metricGrpName, time, apiVersions, null, pool); DELIVERY_TIMEOUT_MS, metrics, metricGrpName, time, apiVersions, null, pool);
Sender sender = new Sender(logContext, client, metadata, this.accumulator, false, Sender sender = new Sender(logContext, client, metadata, this.accumulator, false,
MAX_REQUEST_SIZE, ACKS_ALL, MAX_REQUEST_SIZE, ACKS_ALL,
@ -3696,7 +3697,7 @@ public class SenderTest {
this.metrics = new Metrics(metricConfig, time); this.metrics = new Metrics(metricConfig, time);
BufferPool pool = (customPool == null) ? new BufferPool(totalSize, batchSize, metrics, time, metricGrpName) : customPool; BufferPool pool = (customPool == null) ? new BufferPool(totalSize, batchSize, metrics, time, metricGrpName) : customPool;
this.accumulator = new RecordAccumulator(logContext, batchSize, CompressionType.NONE, lingerMs, 0L, 0L, this.accumulator = new RecordAccumulator(logContext, batchSize, Compression.NONE, lingerMs, 0L, 0L,
DELIVERY_TIMEOUT_MS, metrics, metricGrpName, time, apiVersions, transactionManager, pool); DELIVERY_TIMEOUT_MS, metrics, metricGrpName, time, apiVersions, transactionManager, pool);
this.senderMetricsRegistry = new SenderMetricsRegistry(this.metrics); this.senderMetricsRegistry = new SenderMetricsRegistry(this.metrics);
this.sender = new Sender(logContext, this.client, this.metadata, this.accumulator, guaranteeOrder, MAX_REQUEST_SIZE, ACKS_ALL, this.sender = new Sender(logContext, this.client, this.metadata, this.accumulator, guaranteeOrder, MAX_REQUEST_SIZE, ACKS_ALL,

View File

@ -27,6 +27,7 @@ import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.Node; import org.apache.kafka.common.Node;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.errors.FencedInstanceIdException; import org.apache.kafka.common.errors.FencedInstanceIdException;
import org.apache.kafka.common.errors.GroupAuthorizationException; import org.apache.kafka.common.errors.GroupAuthorizationException;
import org.apache.kafka.common.errors.InvalidTxnStateException; import org.apache.kafka.common.errors.InvalidTxnStateException;
@ -49,7 +50,6 @@ import org.apache.kafka.common.message.InitProducerIdResponseData;
import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.MemoryRecordsBuilder; import org.apache.kafka.common.record.MemoryRecordsBuilder;
import org.apache.kafka.common.record.MutableRecordBatch; import org.apache.kafka.common.record.MutableRecordBatch;
@ -178,7 +178,7 @@ public class TransactionManagerTest {
String metricGrpName = "producer-metrics"; String metricGrpName = "producer-metrics";
this.brokerNode = new Node(0, "localhost", 2211); this.brokerNode = new Node(0, "localhost", 2211);
this.accumulator = new RecordAccumulator(logContext, batchSize, CompressionType.NONE, 0, 0L, 0L, this.accumulator = new RecordAccumulator(logContext, batchSize, Compression.NONE, 0, 0L, 0L,
deliveryTimeoutMs, metrics, metricGrpName, time, apiVersions, transactionManager, deliveryTimeoutMs, metrics, metricGrpName, time, apiVersions, transactionManager,
new BufferPool(totalSize, batchSize, metrics, time, metricGrpName)); new BufferPool(totalSize, batchSize, metrics, time, metricGrpName));
@ -690,7 +690,7 @@ public class TransactionManagerTest {
final int requestTimeout = 10000; final int requestTimeout = 10000;
final int deliveryTimeout = 15000; final int deliveryTimeout = 15000;
RecordAccumulator accumulator = new RecordAccumulator(logContext, 16 * 1024, CompressionType.NONE, 0, 0L, 0L, RecordAccumulator accumulator = new RecordAccumulator(logContext, 16 * 1024, Compression.NONE, 0, 0L, 0L,
deliveryTimeout, metrics, "", time, apiVersions, transactionManager, deliveryTimeout, metrics, "", time, apiVersions, transactionManager,
new BufferPool(1024 * 1024, 16 * 1024, metrics, time, "")); new BufferPool(1024 * 1024, 16 * 1024, metrics, time, ""));
@ -760,7 +760,7 @@ public class TransactionManagerTest {
private ProducerBatch batchWithValue(TopicPartition tp, String value) { private ProducerBatch batchWithValue(TopicPartition tp, String value) {
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(64), MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(64),
CompressionType.NONE, TimestampType.CREATE_TIME, 0L); Compression.NONE, TimestampType.CREATE_TIME, 0L);
long currentTimeMs = time.milliseconds(); long currentTimeMs = time.milliseconds();
ProducerBatch batch = new ProducerBatch(tp, builder, currentTimeMs); ProducerBatch batch = new ProducerBatch(tp, builder, currentTimeMs);
batch.tryAppend(currentTimeMs, new byte[0], value.getBytes(), new Header[0], null, currentTimeMs); batch.tryAppend(currentTimeMs, new byte[0], value.getBytes(), new Header[0], null, currentTimeMs);

View File

@ -0,0 +1,84 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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 org.apache.kafka.common.compress;
import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.common.record.RecordBatch;
import org.apache.kafka.common.utils.BufferSupplier;
import org.apache.kafka.common.utils.ByteBufferOutputStream;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.Collections;
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;
public class GzipCompressionTest {
@Test
public void testCompressionDecompression() throws IOException {
GzipCompression.Builder builder = Compression.gzip();
byte[] data = String.join("", Collections.nCopies(256, "data")).getBytes(StandardCharsets.UTF_8);
for (byte magic : Arrays.asList(RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V2)) {
for (int level : Arrays.asList(GzipCompression.MIN_LEVEL, GzipCompression.DEFAULT_LEVEL, GzipCompression.MAX_LEVEL)) {
GzipCompression compression = builder.level(level).build();
ByteBufferOutputStream bufferStream = new ByteBufferOutputStream(4);
try (OutputStream out = compression.wrapForOutput(bufferStream, magic)) {
out.write(data);
out.flush();
}
bufferStream.buffer().flip();
try (InputStream inputStream = compression.wrapForInput(bufferStream.buffer(), magic, BufferSupplier.create())) {
byte[] result = new byte[data.length];
int read = inputStream.read(result);
assertEquals(data.length, read);
assertArrayEquals(data, result);
}
}
}
}
@Test
public void testCompressionLevels() {
GzipCompression.Builder builder = Compression.gzip();
assertThrows(IllegalArgumentException.class, () -> builder.level(GzipCompression.MIN_LEVEL - 1));
assertThrows(IllegalArgumentException.class, () -> builder.level(GzipCompression.MAX_LEVEL + 1));
builder.level(GzipCompression.MIN_LEVEL);
builder.level(GzipCompression.MAX_LEVEL);
}
@Test
public void testLevelValidator() {
GzipCompression.LevelValidator validator = new GzipCompression.LevelValidator();
for (int level = GzipCompression.MIN_LEVEL; level <= GzipCompression.MAX_LEVEL; level++) {
validator.ensureValid("", level);
}
validator.ensureValid("", GzipCompression.DEFAULT_LEVEL);
assertThrows(ConfigException.class, () -> validator.ensureValid("", GzipCompression.MIN_LEVEL - 1));
assertThrows(ConfigException.class, () -> validator.ensureValid("", GzipCompression.MAX_LEVEL + 1));
}
}

View File

@ -18,7 +18,11 @@ package org.apache.kafka.common.compress;
import net.jpountz.xxhash.XXHashFactory; import net.jpountz.xxhash.XXHashFactory;
import org.apache.kafka.common.record.RecordBatch;
import org.apache.kafka.common.utils.BufferSupplier; import org.apache.kafka.common.utils.BufferSupplier;
import org.apache.kafka.common.utils.ByteBufferOutputStream;
import org.apache.kafka.common.utils.ChunkedBytesStream;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtensionContext; import org.junit.jupiter.api.extension.ExtensionContext;
import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.Arguments;
@ -27,25 +31,94 @@ import org.junit.jupiter.params.provider.ArgumentsSource;
import java.io.ByteArrayOutputStream; import java.io.ByteArrayOutputStream;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.nio.ByteOrder; import java.nio.ByteOrder;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Random; import java.util.Random;
import java.util.stream.Stream; import java.util.stream.Stream;
import static org.apache.kafka.common.compress.KafkaLZ4BlockOutputStream.LZ4_FRAME_INCOMPRESSIBLE_MASK; import static org.apache.kafka.common.compress.Lz4BlockOutputStream.LZ4_FRAME_INCOMPRESSIBLE_MASK;
import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertArrayEquals;
import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertTrue;
public class KafkaLZ4Test { public class Lz4CompressionTest {
private final static Random RANDOM = new Random(0); private final static Random RANDOM = new Random(0);
@Test
public void testLz4FramingMagicV0() {
ByteBuffer buffer = ByteBuffer.allocate(256);
Lz4Compression compression = new Lz4Compression.Builder().build();
Lz4BlockOutputStream out = (Lz4BlockOutputStream) compression.wrapForOutput(
new ByteBufferOutputStream(buffer), RecordBatch.MAGIC_VALUE_V0);
assertTrue(out.useBrokenFlagDescriptorChecksum());
buffer.rewind();
ChunkedBytesStream in = (ChunkedBytesStream) compression.wrapForInput(buffer, RecordBatch.MAGIC_VALUE_V0, BufferSupplier.NO_CACHING);
assertTrue(((Lz4BlockInputStream) in.sourceStream()).ignoreFlagDescriptorChecksum());
}
@Test
public void testLz4FramingMagicV1() {
ByteBuffer buffer = ByteBuffer.allocate(256);
Lz4Compression compression = new Lz4Compression.Builder().build();
Lz4BlockOutputStream out = (Lz4BlockOutputStream) compression.wrapForOutput(
new ByteBufferOutputStream(buffer), RecordBatch.MAGIC_VALUE_V1);
assertFalse(out.useBrokenFlagDescriptorChecksum());
buffer.rewind();
ChunkedBytesStream in = (ChunkedBytesStream) compression.wrapForInput(buffer, RecordBatch.MAGIC_VALUE_V1, BufferSupplier.create());
assertFalse(((Lz4BlockInputStream) in.sourceStream()).ignoreFlagDescriptorChecksum());
}
@Test
public void testCompressionDecompression() throws IOException {
Lz4Compression.Builder builder = Compression.lz4();
byte[] data = String.join("", Collections.nCopies(256, "data")).getBytes(StandardCharsets.UTF_8);
for (byte magic : Arrays.asList(RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V2)) {
for (int level : Arrays.asList(Lz4Compression.MIN_LEVEL, Lz4Compression.DEFAULT_LEVEL, Lz4Compression.MAX_LEVEL)) {
Lz4Compression compression = builder.level(level).build();
ByteBufferOutputStream bufferStream = new ByteBufferOutputStream(4);
try (OutputStream out = compression.wrapForOutput(bufferStream, magic)) {
out.write(data);
out.flush();
}
bufferStream.buffer().flip();
try (InputStream inputStream = compression.wrapForInput(bufferStream.buffer(), magic, BufferSupplier.create())) {
byte[] result = new byte[data.length];
int read = inputStream.read(result);
assertEquals(data.length, read);
assertArrayEquals(data, result);
}
}
}
}
@Test
public void testCompressionLevels() {
Lz4Compression.Builder builder = Compression.lz4();
assertThrows(IllegalArgumentException.class, () -> builder.level(Lz4Compression.MIN_LEVEL - 1));
assertThrows(IllegalArgumentException.class, () -> builder.level(Lz4Compression.MAX_LEVEL + 1));
builder.level(Lz4Compression.MIN_LEVEL);
builder.level(Lz4Compression.MAX_LEVEL);
}
private static class Payload { private static class Payload {
String name; String name;
byte[] payload; byte[] payload;
@ -67,14 +140,16 @@ public class KafkaLZ4Test {
private static class Args { private static class Args {
final boolean useBrokenFlagDescriptorChecksum; final boolean useBrokenFlagDescriptorChecksum;
final boolean ignoreFlagDescriptorChecksum; final boolean ignoreFlagDescriptorChecksum;
final int level;
final byte[] payload; final byte[] payload;
final boolean close; final boolean close;
final boolean blockChecksum; final boolean blockChecksum;
Args(boolean useBrokenFlagDescriptorChecksum, boolean ignoreFlagDescriptorChecksum, Args(boolean useBrokenFlagDescriptorChecksum, boolean ignoreFlagDescriptorChecksum,
boolean blockChecksum, boolean close, Payload payload) { int level, boolean blockChecksum, boolean close, Payload payload) {
this.useBrokenFlagDescriptorChecksum = useBrokenFlagDescriptorChecksum; this.useBrokenFlagDescriptorChecksum = useBrokenFlagDescriptorChecksum;
this.ignoreFlagDescriptorChecksum = ignoreFlagDescriptorChecksum; this.ignoreFlagDescriptorChecksum = ignoreFlagDescriptorChecksum;
this.level = level;
this.blockChecksum = blockChecksum; this.blockChecksum = blockChecksum;
this.close = close; this.close = close;
this.payload = payload.payload; this.payload = payload.payload;
@ -84,6 +159,7 @@ public class KafkaLZ4Test {
public String toString() { public String toString() {
return "useBrokenFlagDescriptorChecksum=" + useBrokenFlagDescriptorChecksum + return "useBrokenFlagDescriptorChecksum=" + useBrokenFlagDescriptorChecksum +
", ignoreFlagDescriptorChecksum=" + ignoreFlagDescriptorChecksum + ", ignoreFlagDescriptorChecksum=" + ignoreFlagDescriptorChecksum +
", level=" + level +
", blockChecksum=" + blockChecksum + ", blockChecksum=" + blockChecksum +
", close=" + close + ", close=" + close +
", payload=" + Arrays.toString(payload); ", payload=" + Arrays.toString(payload);
@ -115,7 +191,8 @@ public class KafkaLZ4Test {
for (boolean ignore : Arrays.asList(false, true)) for (boolean ignore : Arrays.asList(false, true))
for (boolean blockChecksum : Arrays.asList(false, true)) for (boolean blockChecksum : Arrays.asList(false, true))
for (boolean close : Arrays.asList(false, true)) for (boolean close : Arrays.asList(false, true))
arguments.add(Arguments.of(new Args(broken, ignore, blockChecksum, close, payload))); for (int level : Arrays.asList(Lz4Compression.MIN_LEVEL, Lz4Compression.DEFAULT_LEVEL, Lz4Compression.MAX_LEVEL))
arguments.add(Arguments.of(new Args(broken, ignore, level, blockChecksum, close, payload)));
return arguments.stream(); return arguments.stream();
} }
@ -126,11 +203,11 @@ public class KafkaLZ4Test {
public void testHeaderPrematureEnd(Args args) { public void testHeaderPrematureEnd(Args args) {
ByteBuffer buffer = ByteBuffer.allocate(2); ByteBuffer buffer = ByteBuffer.allocate(2);
IOException e = assertThrows(IOException.class, () -> makeInputStream(buffer, args.ignoreFlagDescriptorChecksum)); IOException e = assertThrows(IOException.class, () -> makeInputStream(buffer, args.ignoreFlagDescriptorChecksum));
assertEquals(KafkaLZ4BlockInputStream.PREMATURE_EOS, e.getMessage()); assertEquals(Lz4BlockInputStream.PREMATURE_EOS, e.getMessage());
} }
private KafkaLZ4BlockInputStream makeInputStream(ByteBuffer buffer, boolean ignoreFlagDescriptorChecksum) throws IOException { private Lz4BlockInputStream makeInputStream(ByteBuffer buffer, boolean ignoreFlagDescriptorChecksum) throws IOException {
return new KafkaLZ4BlockInputStream(buffer, BufferSupplier.create(), ignoreFlagDescriptorChecksum); return new Lz4BlockInputStream(buffer, BufferSupplier.create(), ignoreFlagDescriptorChecksum);
} }
@ParameterizedTest @ParameterizedTest
@ -140,7 +217,7 @@ public class KafkaLZ4Test {
compressed[0] = 0x00; compressed[0] = 0x00;
ByteBuffer buffer = ByteBuffer.wrap(compressed); ByteBuffer buffer = ByteBuffer.wrap(compressed);
IOException e = assertThrows(IOException.class, () -> makeInputStream(buffer, args.ignoreFlagDescriptorChecksum)); IOException e = assertThrows(IOException.class, () -> makeInputStream(buffer, args.ignoreFlagDescriptorChecksum));
assertEquals(KafkaLZ4BlockInputStream.NOT_SUPPORTED, e.getMessage()); assertEquals(Lz4BlockInputStream.NOT_SUPPORTED, e.getMessage());
} }
@ParameterizedTest @ParameterizedTest
@ -154,7 +231,7 @@ public class KafkaLZ4Test {
makeInputStream(buffer, args.ignoreFlagDescriptorChecksum); makeInputStream(buffer, args.ignoreFlagDescriptorChecksum);
} else { } else {
IOException e = assertThrows(IOException.class, () -> makeInputStream(buffer, args.ignoreFlagDescriptorChecksum)); IOException e = assertThrows(IOException.class, () -> makeInputStream(buffer, args.ignoreFlagDescriptorChecksum));
assertEquals(KafkaLZ4BlockInputStream.DESCRIPTOR_HASH_MISMATCH, e.getMessage()); assertEquals(Lz4BlockInputStream.DESCRIPTOR_HASH_MISMATCH, e.getMessage());
} }
} }
@ -302,7 +379,7 @@ public class KafkaLZ4Test {
public void testSkip(Args args) throws Exception { public void testSkip(Args args) throws Exception {
if (!args.close || (args.useBrokenFlagDescriptorChecksum && !args.ignoreFlagDescriptorChecksum)) return; if (!args.close || (args.useBrokenFlagDescriptorChecksum && !args.ignoreFlagDescriptorChecksum)) return;
final KafkaLZ4BlockInputStream in = makeInputStream(ByteBuffer.wrap(compressedBytes(args)), final Lz4BlockInputStream in = makeInputStream(ByteBuffer.wrap(compressedBytes(args)),
args.ignoreFlagDescriptorChecksum); args.ignoreFlagDescriptorChecksum);
int n = 100; int n = 100;
@ -319,7 +396,7 @@ public class KafkaLZ4Test {
private void testDecompression(ByteBuffer buffer, Args args) throws IOException { private void testDecompression(ByteBuffer buffer, Args args) throws IOException {
IOException error = null; IOException error = null;
try { try {
KafkaLZ4BlockInputStream decompressed = makeInputStream(buffer, args.ignoreFlagDescriptorChecksum); Lz4BlockInputStream decompressed = makeInputStream(buffer, args.ignoreFlagDescriptorChecksum);
byte[] testPayload = new byte[args.payload.length]; byte[] testPayload = new byte[args.payload.length];
@ -341,10 +418,10 @@ public class KafkaLZ4Test {
assertArrayEquals(args.payload, testPayload); assertArrayEquals(args.payload, testPayload);
} catch (IOException e) { } catch (IOException e) {
if (!args.ignoreFlagDescriptorChecksum && args.useBrokenFlagDescriptorChecksum) { if (!args.ignoreFlagDescriptorChecksum && args.useBrokenFlagDescriptorChecksum) {
assertEquals(KafkaLZ4BlockInputStream.DESCRIPTOR_HASH_MISMATCH, e.getMessage()); assertEquals(Lz4BlockInputStream.DESCRIPTOR_HASH_MISMATCH, e.getMessage());
error = e; error = e;
} else if (!args.close) { } else if (!args.close) {
assertEquals(KafkaLZ4BlockInputStream.PREMATURE_EOS, e.getMessage()); assertEquals(Lz4BlockInputStream.PREMATURE_EOS, e.getMessage());
error = e; error = e;
} else { } else {
throw e; throw e;
@ -356,9 +433,10 @@ public class KafkaLZ4Test {
private byte[] compressedBytes(Args args) throws IOException { private byte[] compressedBytes(Args args) throws IOException {
ByteArrayOutputStream output = new ByteArrayOutputStream(); ByteArrayOutputStream output = new ByteArrayOutputStream();
KafkaLZ4BlockOutputStream lz4 = new KafkaLZ4BlockOutputStream( Lz4BlockOutputStream lz4 = new Lz4BlockOutputStream(
output, output,
KafkaLZ4BlockOutputStream.BLOCKSIZE_64KB, Lz4BlockOutputStream.BLOCKSIZE_64KB,
args.level,
args.blockChecksum, args.blockChecksum,
args.useBrokenFlagDescriptorChecksum args.useBrokenFlagDescriptorChecksum
); );

View File

@ -0,0 +1,58 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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 org.apache.kafka.common.compress;
import org.apache.kafka.common.record.RecordBatch;
import org.apache.kafka.common.utils.BufferSupplier;
import org.apache.kafka.common.utils.ByteBufferOutputStream;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.Collections;
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
import static org.junit.jupiter.api.Assertions.assertEquals;
public class NoCompressionTest {
@Test
public void testCompressionDecompression() throws IOException {
NoCompression compression = Compression.NONE;
byte[] data = String.join("", Collections.nCopies(256, "data")).getBytes(StandardCharsets.UTF_8);
for (byte magic : Arrays.asList(RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V2)) {
ByteBufferOutputStream bufferStream = new ByteBufferOutputStream(4);
try (OutputStream out = compression.wrapForOutput(bufferStream, magic)) {
out.write(data);
out.flush();
}
bufferStream.buffer().flip();
assertArrayEquals(data, bufferStream.buffer().array());
try (InputStream inputStream = compression.wrapForInput(bufferStream.buffer(), magic, BufferSupplier.create())) {
byte[] result = new byte[data.length];
int read = inputStream.read(result);
assertEquals(data.length, read);
assertArrayEquals(data, result);
}
}
}
}

View File

@ -0,0 +1,57 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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 org.apache.kafka.common.compress;
import org.apache.kafka.common.record.RecordBatch;
import org.apache.kafka.common.utils.BufferSupplier;
import org.apache.kafka.common.utils.ByteBufferOutputStream;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.Collections;
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
import static org.junit.jupiter.api.Assertions.assertEquals;
public class SnappyCompressionTest {
@Test
public void testCompressionDecompression() throws IOException {
SnappyCompression compression = Compression.snappy().build();
byte[] data = String.join("", Collections.nCopies(256, "data")).getBytes(StandardCharsets.UTF_8);
for (byte magic : Arrays.asList(RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V2)) {
ByteBufferOutputStream bufferStream = new ByteBufferOutputStream(4);
try (OutputStream out = compression.wrapForOutput(bufferStream, magic)) {
out.write(data);
out.flush();
}
bufferStream.buffer().flip();
try (InputStream inputStream = compression.wrapForInput(bufferStream.buffer(), magic, BufferSupplier.create())) {
byte[] result = new byte[data.length];
int read = inputStream.read(result);
assertEquals(data.length, read);
assertArrayEquals(data, result);
}
}
}
}

View File

@ -0,0 +1,72 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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 org.apache.kafka.common.compress;
import org.apache.kafka.common.record.RecordBatch;
import org.apache.kafka.common.utils.BufferSupplier;
import org.apache.kafka.common.utils.ByteBufferOutputStream;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.Collections;
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;
public class ZstdCompressionTest {
@Test
public void testCompressionDecompression() throws IOException {
ZstdCompression.Builder builder = Compression.zstd();
byte[] data = String.join("", Collections.nCopies(256, "data")).getBytes(StandardCharsets.UTF_8);
for (byte magic : Arrays.asList(RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V2)) {
for (int level : Arrays.asList(ZstdCompression.MIN_LEVEL, ZstdCompression.DEFAULT_LEVEL, ZstdCompression.MAX_LEVEL)) {
ZstdCompression compression = builder.level(level).build();
ByteBufferOutputStream bufferStream = new ByteBufferOutputStream(4);
try (OutputStream out = compression.wrapForOutput(bufferStream, magic)) {
out.write(data);
out.flush();
}
bufferStream.buffer().flip();
try (InputStream inputStream = compression.wrapForInput(bufferStream.buffer(), magic, BufferSupplier.create())) {
byte[] result = new byte[data.length];
int read = inputStream.read(result);
assertEquals(data.length, read);
assertArrayEquals(data, result);
}
}
}
}
@Test
public void testCompressionLevels() {
ZstdCompression.Builder builder = Compression.zstd();
assertThrows(IllegalArgumentException.class, () -> builder.level(ZstdCompression.MIN_LEVEL - 1));
assertThrows(IllegalArgumentException.class, () -> builder.level(ZstdCompression.MAX_LEVEL + 1));
builder.level(ZstdCompression.MIN_LEVEL);
builder.level(ZstdCompression.MAX_LEVEL);
}
}

View File

@ -16,9 +16,9 @@
*/ */
package org.apache.kafka.common.message; package org.apache.kafka.common.message;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.ByteBufferAccessor;
import org.apache.kafka.common.protocol.MessageUtil; import org.apache.kafka.common.protocol.MessageUtil;
import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.SimpleRecord; import org.apache.kafka.common.record.SimpleRecord;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
@ -32,7 +32,7 @@ public class RecordsSerdeTest {
@Test @Test
public void testSerdeRecords() { public void testSerdeRecords() {
MemoryRecords records = MemoryRecords.withRecords(CompressionType.NONE, MemoryRecords records = MemoryRecords.withRecords(Compression.NONE,
new SimpleRecord("foo".getBytes()), new SimpleRecord("foo".getBytes()),
new SimpleRecord("bar".getBytes())); new SimpleRecord("bar".getBytes()));

View File

@ -16,8 +16,8 @@
*/ */
package org.apache.kafka.common.protocol; package org.apache.kafka.common.protocol;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.network.Send; import org.apache.kafka.common.network.Send;
import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.MemoryRecordsBuilder; import org.apache.kafka.common.record.MemoryRecordsBuilder;
import org.apache.kafka.common.record.SimpleRecord; import org.apache.kafka.common.record.SimpleRecord;
@ -155,7 +155,7 @@ public class SendBuilderTest {
private MemoryRecords createRecords(ByteBuffer buffer, String value) { private MemoryRecords createRecords(ByteBuffer buffer, String value) {
MemoryRecordsBuilder recordsBuilder = MemoryRecords.builder( MemoryRecordsBuilder recordsBuilder = MemoryRecords.builder(
buffer, buffer,
CompressionType.NONE, Compression.NONE,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
0L 0L
); );

View File

@ -16,7 +16,7 @@
*/ */
package org.apache.kafka.common.protocol.types; package org.apache.kafka.common.protocol.types;
import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.SimpleRecord; import org.apache.kafka.common.record.SimpleRecord;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
@ -50,7 +50,7 @@ public class TypeTest {
@Test @Test
public void testRecordsSerde() { public void testRecordsSerde() {
MemoryRecords records = MemoryRecords.withRecords(CompressionType.NONE, MemoryRecords records = MemoryRecords.withRecords(Compression.NONE,
new SimpleRecord("foo".getBytes()), new SimpleRecord("foo".getBytes()),
new SimpleRecord("bar".getBytes())); new SimpleRecord("bar".getBytes()));
ByteBuffer buffer = ByteBuffer.allocate(Type.RECORDS.sizeOf(records)); ByteBuffer buffer = ByteBuffer.allocate(Type.RECORDS.sizeOf(records));
@ -81,7 +81,7 @@ public class TypeTest {
@Test @Test
public void testCompactRecordsSerde() { public void testCompactRecordsSerde() {
MemoryRecords records = MemoryRecords.withRecords(CompressionType.NONE, MemoryRecords records = MemoryRecords.withRecords(Compression.NONE,
new SimpleRecord("foo".getBytes()), new SimpleRecord("foo".getBytes()),
new SimpleRecord("bar".getBytes())); new SimpleRecord("bar".getBytes()));
ByteBuffer buffer = ByteBuffer.allocate(Type.COMPACT_RECORDS.sizeOf(records)); ByteBuffer buffer = ByteBuffer.allocate(Type.COMPACT_RECORDS.sizeOf(records));

View File

@ -17,6 +17,7 @@
package org.apache.kafka.common.record; package org.apache.kafka.common.record;
import org.apache.kafka.common.InvalidRecordException; import org.apache.kafka.common.InvalidRecordException;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.record.AbstractLegacyRecordBatch.ByteBufferLegacyRecordBatch; import org.apache.kafka.common.record.AbstractLegacyRecordBatch.ByteBufferLegacyRecordBatch;
import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.utils.Utils;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
@ -40,7 +41,7 @@ public class AbstractLegacyRecordBatchTest {
}; };
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, 0L, MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, 0L,
CompressionType.GZIP, TimestampType.CREATE_TIME, simpleRecords); Compression.gzip().build(), TimestampType.CREATE_TIME, simpleRecords);
long lastOffset = 500L; long lastOffset = 500L;
long firstOffset = lastOffset - simpleRecords.length + 1; long firstOffset = lastOffset - simpleRecords.length + 1;
@ -75,7 +76,7 @@ public class AbstractLegacyRecordBatchTest {
}; };
MemoryRecords records = MemoryRecords.withRecords(magic, 0L, MemoryRecords records = MemoryRecords.withRecords(magic, 0L,
CompressionType.GZIP, TimestampType.CREATE_TIME, simpleRecords); Compression.gzip().build(), TimestampType.CREATE_TIME, simpleRecords);
ByteBufferLegacyRecordBatch batch = new ByteBufferLegacyRecordBatch(records.buffer()); ByteBufferLegacyRecordBatch batch = new ByteBufferLegacyRecordBatch(records.buffer());
batch.setLastOffset(0L); batch.setLastOffset(0L);
@ -95,7 +96,7 @@ public class AbstractLegacyRecordBatchTest {
}; };
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, 0L, MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, 0L,
CompressionType.GZIP, TimestampType.CREATE_TIME, simpleRecords); Compression.gzip().build(), TimestampType.CREATE_TIME, simpleRecords);
ByteBufferLegacyRecordBatch batch = new ByteBufferLegacyRecordBatch(records.buffer()); ByteBufferLegacyRecordBatch batch = new ByteBufferLegacyRecordBatch(records.buffer());
batch.setLastOffset(1L); batch.setLastOffset(1L);
@ -106,7 +107,7 @@ public class AbstractLegacyRecordBatchTest {
@Test @Test
public void testSetNoTimestampTypeNotAllowed() { public void testSetNoTimestampTypeNotAllowed() {
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, 0L, MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, 0L,
CompressionType.GZIP, TimestampType.CREATE_TIME, Compression.gzip().build(), TimestampType.CREATE_TIME,
new SimpleRecord(1L, "a".getBytes(), "1".getBytes()), new SimpleRecord(1L, "a".getBytes(), "1".getBytes()),
new SimpleRecord(2L, "b".getBytes(), "2".getBytes()), new SimpleRecord(2L, "b".getBytes(), "2".getBytes()),
new SimpleRecord(3L, "c".getBytes(), "3".getBytes())); new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
@ -117,7 +118,7 @@ public class AbstractLegacyRecordBatchTest {
@Test @Test
public void testSetLogAppendTimeNotAllowedV0() { public void testSetLogAppendTimeNotAllowedV0() {
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V0, 0L, MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V0, 0L,
CompressionType.GZIP, TimestampType.CREATE_TIME, Compression.gzip().build(), TimestampType.CREATE_TIME,
new SimpleRecord(1L, "a".getBytes(), "1".getBytes()), new SimpleRecord(1L, "a".getBytes(), "1".getBytes()),
new SimpleRecord(2L, "b".getBytes(), "2".getBytes()), new SimpleRecord(2L, "b".getBytes(), "2".getBytes()),
new SimpleRecord(3L, "c".getBytes(), "3".getBytes())); new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
@ -129,7 +130,7 @@ public class AbstractLegacyRecordBatchTest {
@Test @Test
public void testSetCreateTimeNotAllowedV0() { public void testSetCreateTimeNotAllowedV0() {
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V0, 0L, MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V0, 0L,
CompressionType.GZIP, TimestampType.CREATE_TIME, Compression.gzip().build(), TimestampType.CREATE_TIME,
new SimpleRecord(1L, "a".getBytes(), "1".getBytes()), new SimpleRecord(1L, "a".getBytes(), "1".getBytes()),
new SimpleRecord(2L, "b".getBytes(), "2".getBytes()), new SimpleRecord(2L, "b".getBytes(), "2".getBytes()),
new SimpleRecord(3L, "c".getBytes(), "3".getBytes())); new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
@ -141,7 +142,7 @@ public class AbstractLegacyRecordBatchTest {
@Test @Test
public void testSetPartitionLeaderEpochNotAllowedV0() { public void testSetPartitionLeaderEpochNotAllowedV0() {
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V0, 0L, MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V0, 0L,
CompressionType.GZIP, TimestampType.CREATE_TIME, Compression.gzip().build(), TimestampType.CREATE_TIME,
new SimpleRecord(1L, "a".getBytes(), "1".getBytes()), new SimpleRecord(1L, "a".getBytes(), "1".getBytes()),
new SimpleRecord(2L, "b".getBytes(), "2".getBytes()), new SimpleRecord(2L, "b".getBytes(), "2".getBytes()),
new SimpleRecord(3L, "c".getBytes(), "3".getBytes())); new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
@ -152,7 +153,7 @@ public class AbstractLegacyRecordBatchTest {
@Test @Test
public void testSetPartitionLeaderEpochNotAllowedV1() { public void testSetPartitionLeaderEpochNotAllowedV1() {
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, 0L, MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, 0L,
CompressionType.GZIP, TimestampType.CREATE_TIME, Compression.gzip().build(), TimestampType.CREATE_TIME,
new SimpleRecord(1L, "a".getBytes(), "1".getBytes()), new SimpleRecord(1L, "a".getBytes(), "1".getBytes()),
new SimpleRecord(2L, "b".getBytes(), "2".getBytes()), new SimpleRecord(2L, "b".getBytes(), "2".getBytes()),
new SimpleRecord(3L, "c".getBytes(), "3".getBytes())); new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
@ -163,7 +164,7 @@ public class AbstractLegacyRecordBatchTest {
@Test @Test
public void testSetLogAppendTimeV1() { public void testSetLogAppendTimeV1() {
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, 0L, MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, 0L,
CompressionType.GZIP, TimestampType.CREATE_TIME, Compression.gzip().build(), TimestampType.CREATE_TIME,
new SimpleRecord(1L, "a".getBytes(), "1".getBytes()), new SimpleRecord(1L, "a".getBytes(), "1".getBytes()),
new SimpleRecord(2L, "b".getBytes(), "2".getBytes()), new SimpleRecord(2L, "b".getBytes(), "2".getBytes()),
new SimpleRecord(3L, "c".getBytes(), "3".getBytes())); new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
@ -188,7 +189,7 @@ public class AbstractLegacyRecordBatchTest {
@Test @Test
public void testSetCreateTimeV1() { public void testSetCreateTimeV1() {
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, 0L, MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, 0L,
CompressionType.GZIP, TimestampType.CREATE_TIME, Compression.gzip().build(), TimestampType.CREATE_TIME,
new SimpleRecord(1L, "a".getBytes(), "1".getBytes()), new SimpleRecord(1L, "a".getBytes(), "1".getBytes()),
new SimpleRecord(2L, "b".getBytes(), "2".getBytes()), new SimpleRecord(2L, "b".getBytes(), "2".getBytes()),
new SimpleRecord(3L, "c".getBytes(), "3".getBytes())); new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
@ -222,7 +223,7 @@ public class AbstractLegacyRecordBatchTest {
// Check V0 // Check V0
try { try {
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V0, 0L, MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V0, 0L,
CompressionType.ZSTD, TimestampType.CREATE_TIME, simpleRecords); Compression.zstd().build(), TimestampType.CREATE_TIME, simpleRecords);
ByteBufferLegacyRecordBatch batch = new ByteBufferLegacyRecordBatch(records.buffer()); ByteBufferLegacyRecordBatch batch = new ByteBufferLegacyRecordBatch(records.buffer());
batch.setLastOffset(1L); batch.setLastOffset(1L);
@ -236,7 +237,7 @@ public class AbstractLegacyRecordBatchTest {
// Check V1 // Check V1
try { try {
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, 0L, MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, 0L,
CompressionType.ZSTD, TimestampType.CREATE_TIME, simpleRecords); Compression.zstd().build(), TimestampType.CREATE_TIME, simpleRecords);
ByteBufferLegacyRecordBatch batch = new ByteBufferLegacyRecordBatch(records.buffer()); ByteBufferLegacyRecordBatch batch = new ByteBufferLegacyRecordBatch(records.buffer());
batch.setLastOffset(1L); batch.setLastOffset(1L);

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.common.record; package org.apache.kafka.common.record;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.errors.CorruptRecordException; import org.apache.kafka.common.errors.CorruptRecordException;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
@ -33,12 +34,12 @@ public class ByteBufferLogInputStreamTest {
@Test @Test
public void iteratorIgnoresIncompleteEntries() { public void iteratorIgnoresIncompleteEntries() {
ByteBuffer buffer = ByteBuffer.allocate(1024); ByteBuffer buffer = ByteBuffer.allocate(1024);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, CompressionType.NONE, TimestampType.CREATE_TIME, 0L); MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, Compression.NONE, TimestampType.CREATE_TIME, 0L);
builder.append(15L, "a".getBytes(), "1".getBytes()); builder.append(15L, "a".getBytes(), "1".getBytes());
builder.append(20L, "b".getBytes(), "2".getBytes()); builder.append(20L, "b".getBytes(), "2".getBytes());
builder.close(); builder.close();
builder = MemoryRecords.builder(buffer, CompressionType.NONE, TimestampType.CREATE_TIME, 2L); builder = MemoryRecords.builder(buffer, Compression.NONE, TimestampType.CREATE_TIME, 2L);
builder.append(30L, "c".getBytes(), "3".getBytes()); builder.append(30L, "c".getBytes(), "3".getBytes());
builder.append(40L, "d".getBytes(), "4".getBytes()); builder.append(40L, "d".getBytes(), "4".getBytes());
builder.close(); builder.close();
@ -58,14 +59,14 @@ public class ByteBufferLogInputStreamTest {
@Test @Test
public void iteratorRaisesOnTooSmallRecords() { public void iteratorRaisesOnTooSmallRecords() {
ByteBuffer buffer = ByteBuffer.allocate(1024); ByteBuffer buffer = ByteBuffer.allocate(1024);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, CompressionType.NONE, TimestampType.CREATE_TIME, 0L); MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, Compression.NONE, TimestampType.CREATE_TIME, 0L);
builder.append(15L, "a".getBytes(), "1".getBytes()); builder.append(15L, "a".getBytes(), "1".getBytes());
builder.append(20L, "b".getBytes(), "2".getBytes()); builder.append(20L, "b".getBytes(), "2".getBytes());
builder.close(); builder.close();
int position = buffer.position(); int position = buffer.position();
builder = MemoryRecords.builder(buffer, CompressionType.NONE, TimestampType.CREATE_TIME, 2L); builder = MemoryRecords.builder(buffer, Compression.NONE, TimestampType.CREATE_TIME, 2L);
builder.append(30L, "c".getBytes(), "3".getBytes()); builder.append(30L, "c".getBytes(), "3".getBytes());
builder.append(40L, "d".getBytes(), "4".getBytes()); builder.append(40L, "d".getBytes(), "4".getBytes());
builder.close(); builder.close();
@ -81,14 +82,14 @@ public class ByteBufferLogInputStreamTest {
@Test @Test
public void iteratorRaisesOnInvalidMagic() { public void iteratorRaisesOnInvalidMagic() {
ByteBuffer buffer = ByteBuffer.allocate(1024); ByteBuffer buffer = ByteBuffer.allocate(1024);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, CompressionType.NONE, TimestampType.CREATE_TIME, 0L); MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, Compression.NONE, TimestampType.CREATE_TIME, 0L);
builder.append(15L, "a".getBytes(), "1".getBytes()); builder.append(15L, "a".getBytes(), "1".getBytes());
builder.append(20L, "b".getBytes(), "2".getBytes()); builder.append(20L, "b".getBytes(), "2".getBytes());
builder.close(); builder.close();
int position = buffer.position(); int position = buffer.position();
builder = MemoryRecords.builder(buffer, CompressionType.NONE, TimestampType.CREATE_TIME, 2L); builder = MemoryRecords.builder(buffer, Compression.NONE, TimestampType.CREATE_TIME, 2L);
builder.append(30L, "c".getBytes(), "3".getBytes()); builder.append(30L, "c".getBytes(), "3".getBytes());
builder.append(40L, "d".getBytes(), "4".getBytes()); builder.append(40L, "d".getBytes(), "4".getBytes());
builder.close(); builder.close();
@ -104,11 +105,11 @@ public class ByteBufferLogInputStreamTest {
@Test @Test
public void iteratorRaisesOnTooLargeRecords() { public void iteratorRaisesOnTooLargeRecords() {
ByteBuffer buffer = ByteBuffer.allocate(1024); ByteBuffer buffer = ByteBuffer.allocate(1024);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, CompressionType.NONE, TimestampType.CREATE_TIME, 0L); MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, Compression.NONE, TimestampType.CREATE_TIME, 0L);
builder.append(15L, "a".getBytes(), "1".getBytes()); builder.append(15L, "a".getBytes(), "1".getBytes());
builder.close(); builder.close();
builder = MemoryRecords.builder(buffer, CompressionType.NONE, TimestampType.CREATE_TIME, 2L); builder = MemoryRecords.builder(buffer, Compression.NONE, TimestampType.CREATE_TIME, 2L);
builder.append(30L, "c".getBytes(), "3".getBytes()); builder.append(30L, "c".getBytes(), "3".getBytes());
builder.append(40L, "d".getBytes(), "4".getBytes()); builder.append(40L, "d".getBytes(), "4".getBytes());
builder.close(); builder.close();

View File

@ -1,58 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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 org.apache.kafka.common.record;
import org.apache.kafka.common.compress.KafkaLZ4BlockInputStream;
import org.apache.kafka.common.compress.KafkaLZ4BlockOutputStream;
import org.apache.kafka.common.utils.BufferSupplier;
import org.apache.kafka.common.utils.ByteBufferOutputStream;
import org.apache.kafka.common.utils.ChunkedBytesStream;
import org.junit.jupiter.api.Test;
import java.nio.ByteBuffer;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
public class CompressionTypeTest {
@Test
public void testLZ4FramingMagicV0() {
ByteBuffer buffer = ByteBuffer.allocate(256);
KafkaLZ4BlockOutputStream out = (KafkaLZ4BlockOutputStream) CompressionType.LZ4.wrapForOutput(
new ByteBufferOutputStream(buffer), RecordBatch.MAGIC_VALUE_V0);
assertTrue(out.useBrokenFlagDescriptorChecksum());
buffer.rewind();
ChunkedBytesStream in = (ChunkedBytesStream) CompressionType.LZ4.wrapForInput(buffer, RecordBatch.MAGIC_VALUE_V0, BufferSupplier.NO_CACHING);
assertTrue(((KafkaLZ4BlockInputStream) in.sourceStream()).ignoreFlagDescriptorChecksum());
}
@Test
public void testLZ4FramingMagicV1() {
ByteBuffer buffer = ByteBuffer.allocate(256);
KafkaLZ4BlockOutputStream out = (KafkaLZ4BlockOutputStream) CompressionType.LZ4.wrapForOutput(
new ByteBufferOutputStream(buffer), RecordBatch.MAGIC_VALUE_V1);
assertFalse(out.useBrokenFlagDescriptorChecksum());
buffer.rewind();
ChunkedBytesStream in = (ChunkedBytesStream) CompressionType.LZ4.wrapForInput(buffer, RecordBatch.MAGIC_VALUE_V1, BufferSupplier.create());
assertFalse(((KafkaLZ4BlockInputStream) in.sourceStream()).ignoreFlagDescriptorChecksum());
}
}

View File

@ -17,7 +17,8 @@
package org.apache.kafka.common.record; package org.apache.kafka.common.record;
import org.apache.kafka.common.InvalidRecordException; import org.apache.kafka.common.InvalidRecordException;
import org.apache.kafka.common.compress.ZstdFactory; import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.compress.ZstdCompression;
import org.apache.kafka.common.errors.CorruptRecordException; import org.apache.kafka.common.errors.CorruptRecordException;
import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.Header;
import org.apache.kafka.common.header.internals.RecordHeader; import org.apache.kafka.common.header.internals.RecordHeader;
@ -50,16 +51,13 @@ import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyByte;
import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyInt;
import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.anyLong;
import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.never; import static org.mockito.Mockito.never;
import static org.mockito.Mockito.spy; import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.times; import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
public class DefaultRecordBatchTest { public class DefaultRecordBatchTest {
// We avoid SecureRandom.getInstanceStrong() here because it reads from /dev/random and blocks on Linux. Since these // We avoid SecureRandom.getInstanceStrong() here because it reads from /dev/random and blocks on Linux. Since these
@ -106,7 +104,7 @@ public class DefaultRecordBatchTest {
public void buildDefaultRecordBatch() { public void buildDefaultRecordBatch() {
ByteBuffer buffer = ByteBuffer.allocate(2048); ByteBuffer buffer = ByteBuffer.allocate(2048);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE, MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, Compression.NONE,
TimestampType.CREATE_TIME, 1234567L); TimestampType.CREATE_TIME, 1234567L);
builder.appendWithOffset(1234567, 1L, "a".getBytes(), "v".getBytes()); builder.appendWithOffset(1234567, 1L, "a".getBytes(), "v".getBytes());
builder.appendWithOffset(1234568, 2L, "b".getBytes(), "v".getBytes()); builder.appendWithOffset(1234568, 2L, "b".getBytes(), "v".getBytes());
@ -134,7 +132,7 @@ public class DefaultRecordBatchTest {
ByteBuffer buffer = ByteBuffer.allocate(2048); ByteBuffer buffer = ByteBuffer.allocate(2048);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE, MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, Compression.NONE,
TimestampType.CREATE_TIME, 1234567L, RecordBatch.NO_TIMESTAMP, pid, epoch, baseSequence); TimestampType.CREATE_TIME, 1234567L, RecordBatch.NO_TIMESTAMP, pid, epoch, baseSequence);
builder.appendWithOffset(1234567, 1L, "a".getBytes(), "v".getBytes()); builder.appendWithOffset(1234567, 1L, "a".getBytes(), "v".getBytes());
builder.appendWithOffset(1234568, 2L, "b".getBytes(), "v".getBytes()); builder.appendWithOffset(1234568, 2L, "b".getBytes(), "v".getBytes());
@ -161,7 +159,7 @@ public class DefaultRecordBatchTest {
int baseSequence = Integer.MAX_VALUE - 1; int baseSequence = Integer.MAX_VALUE - 1;
ByteBuffer buffer = ByteBuffer.allocate(2048); ByteBuffer buffer = ByteBuffer.allocate(2048);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE, MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, Compression.NONE,
TimestampType.CREATE_TIME, 1234567L, RecordBatch.NO_TIMESTAMP, pid, epoch, baseSequence); TimestampType.CREATE_TIME, 1234567L, RecordBatch.NO_TIMESTAMP, pid, epoch, baseSequence);
builder.appendWithOffset(1234567, 1L, "a".getBytes(), "v".getBytes()); builder.appendWithOffset(1234567, 1L, "a".getBytes(), "v".getBytes());
builder.appendWithOffset(1234568, 2L, "b".getBytes(), "v".getBytes()); builder.appendWithOffset(1234568, 2L, "b".getBytes(), "v".getBytes());
@ -197,14 +195,14 @@ public class DefaultRecordBatchTest {
new SimpleRecord(timestamp + 60000, "key".getBytes(), null), new SimpleRecord(timestamp + 60000, "key".getBytes(), null),
new SimpleRecord(timestamp + 60000, "key".getBytes(), "value".getBytes(), headers) new SimpleRecord(timestamp + 60000, "key".getBytes(), "value".getBytes(), headers)
}; };
int actualSize = MemoryRecords.withRecords(CompressionType.NONE, records).sizeInBytes(); int actualSize = MemoryRecords.withRecords(Compression.NONE, records).sizeInBytes();
assertEquals(actualSize, DefaultRecordBatch.sizeInBytes(Arrays.asList(records))); assertEquals(actualSize, DefaultRecordBatch.sizeInBytes(Arrays.asList(records)));
} }
@Test @Test
public void testInvalidRecordSize() { public void testInvalidRecordSize() {
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, 0L, MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, 0L,
CompressionType.NONE, TimestampType.CREATE_TIME, Compression.NONE, TimestampType.CREATE_TIME,
new SimpleRecord(1L, "a".getBytes(), "1".getBytes()), new SimpleRecord(1L, "a".getBytes(), "1".getBytes()),
new SimpleRecord(2L, "b".getBytes(), "2".getBytes()), new SimpleRecord(2L, "b".getBytes(), "2".getBytes()),
new SimpleRecord(3L, "c".getBytes(), "3".getBytes())); new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
@ -256,7 +254,7 @@ public class DefaultRecordBatchTest {
@Test @Test
public void testInvalidCrc() { public void testInvalidCrc() {
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, 0L, MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, 0L,
CompressionType.NONE, TimestampType.CREATE_TIME, Compression.NONE, TimestampType.CREATE_TIME,
new SimpleRecord(1L, "a".getBytes(), "1".getBytes()), new SimpleRecord(1L, "a".getBytes(), "1".getBytes()),
new SimpleRecord(2L, "b".getBytes(), "2".getBytes()), new SimpleRecord(2L, "b".getBytes(), "2".getBytes()),
new SimpleRecord(3L, "c".getBytes(), "3".getBytes())); new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
@ -277,7 +275,7 @@ public class DefaultRecordBatchTest {
new SimpleRecord(3L, "c".getBytes(), "3".getBytes()) new SimpleRecord(3L, "c".getBytes(), "3".getBytes())
}; };
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, 0L, MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, 0L,
CompressionType.NONE, TimestampType.CREATE_TIME, simpleRecords); Compression.NONE, TimestampType.CREATE_TIME, simpleRecords);
long lastOffset = 500L; long lastOffset = 500L;
long firstOffset = lastOffset - simpleRecords.length + 1; long firstOffset = lastOffset - simpleRecords.length + 1;
@ -300,7 +298,7 @@ public class DefaultRecordBatchTest {
@Test @Test
public void testSetPartitionLeaderEpoch() { public void testSetPartitionLeaderEpoch() {
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, 0L, MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, 0L,
CompressionType.NONE, TimestampType.CREATE_TIME, Compression.NONE, TimestampType.CREATE_TIME,
new SimpleRecord(1L, "a".getBytes(), "1".getBytes()), new SimpleRecord(1L, "a".getBytes(), "1".getBytes()),
new SimpleRecord(2L, "b".getBytes(), "2".getBytes()), new SimpleRecord(2L, "b".getBytes(), "2".getBytes()),
new SimpleRecord(3L, "c".getBytes(), "3".getBytes())); new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
@ -320,7 +318,7 @@ public class DefaultRecordBatchTest {
@Test @Test
public void testSetLogAppendTime() { public void testSetLogAppendTime() {
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, 0L, MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, 0L,
CompressionType.NONE, TimestampType.CREATE_TIME, Compression.NONE, TimestampType.CREATE_TIME,
new SimpleRecord(1L, "a".getBytes(), "1".getBytes()), new SimpleRecord(1L, "a".getBytes(), "1".getBytes()),
new SimpleRecord(2L, "b".getBytes(), "2".getBytes()), new SimpleRecord(2L, "b".getBytes(), "2".getBytes()),
new SimpleRecord(3L, "c".getBytes(), "3".getBytes())); new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
@ -345,7 +343,7 @@ public class DefaultRecordBatchTest {
@Test @Test
public void testSetNoTimestampTypeNotAllowed() { public void testSetNoTimestampTypeNotAllowed() {
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, 0L, MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, 0L,
CompressionType.NONE, TimestampType.CREATE_TIME, Compression.NONE, TimestampType.CREATE_TIME,
new SimpleRecord(1L, "a".getBytes(), "1".getBytes()), new SimpleRecord(1L, "a".getBytes(), "1".getBytes()),
new SimpleRecord(2L, "b".getBytes(), "2".getBytes()), new SimpleRecord(2L, "b".getBytes(), "2".getBytes()),
new SimpleRecord(3L, "c".getBytes(), "3".getBytes())); new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
@ -361,7 +359,7 @@ public class DefaultRecordBatchTest {
ByteBuffer buffer = ByteBuffer.allocate(128); ByteBuffer buffer = ByteBuffer.allocate(128);
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, RecordBatch.CURRENT_MAGIC_VALUE,
CompressionType.NONE, TimestampType.CREATE_TIME, 0L, RecordBatch.NO_TIMESTAMP, producerId, Compression.NONE, TimestampType.CREATE_TIME, 0L, RecordBatch.NO_TIMESTAMP, producerId,
producerEpoch, RecordBatch.NO_SEQUENCE, true, true, RecordBatch.NO_PARTITION_LEADER_EPOCH, producerEpoch, RecordBatch.NO_SEQUENCE, true, true, RecordBatch.NO_PARTITION_LEADER_EPOCH,
buffer.remaining()); buffer.remaining());
@ -385,8 +383,9 @@ public class DefaultRecordBatchTest {
@ParameterizedTest @ParameterizedTest
@EnumSource(value = CompressionType.class) @EnumSource(value = CompressionType.class)
public void testStreamingIteratorConsistency(CompressionType compressionType) { public void testStreamingIteratorConsistency(CompressionType compressionType) {
Compression compression = Compression.of(compressionType).build();
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, 0L, MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, 0L,
compressionType, TimestampType.CREATE_TIME, compression, TimestampType.CREATE_TIME,
new SimpleRecord(1L, "a".getBytes(), "1".getBytes()), new SimpleRecord(1L, "a".getBytes(), "1".getBytes()),
new SimpleRecord(2L, "b".getBytes(), "2".getBytes()), new SimpleRecord(2L, "b".getBytes(), "2".getBytes()),
new SimpleRecord(3L, "c".getBytes(), "3".getBytes())); new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
@ -399,12 +398,13 @@ public class DefaultRecordBatchTest {
@ParameterizedTest @ParameterizedTest
@EnumSource(value = CompressionType.class) @EnumSource(value = CompressionType.class)
public void testSkipKeyValueIteratorCorrectness(CompressionType compressionType) { public void testSkipKeyValueIteratorCorrectness(CompressionType compressionType) {
Compression compression = Compression.of(compressionType).build();
Header[] headers = {new RecordHeader("k1", "v1".getBytes()), new RecordHeader("k2", null)}; Header[] headers = {new RecordHeader("k1", "v1".getBytes()), new RecordHeader("k2", null)};
byte[] largeRecordValue = new byte[200 * 1024]; // 200KB byte[] largeRecordValue = new byte[200 * 1024]; // 200KB
RANDOM.nextBytes(largeRecordValue); RANDOM.nextBytes(largeRecordValue);
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, 0L, MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, 0L,
compressionType, TimestampType.CREATE_TIME, compression, TimestampType.CREATE_TIME,
// one sample with small value size // one sample with small value size
new SimpleRecord(1L, "a".getBytes(), "1".getBytes()), new SimpleRecord(1L, "a".getBytes(), "1".getBytes()),
// one sample with null value // one sample with null value
@ -448,8 +448,9 @@ public class DefaultRecordBatchTest {
@ParameterizedTest @ParameterizedTest
@MethodSource @MethodSource
public void testBufferReuseInSkipKeyValueIterator(CompressionType compressionType, int expectedNumBufferAllocations, byte[] recordValue) { public void testBufferReuseInSkipKeyValueIterator(CompressionType compressionType, int expectedNumBufferAllocations, byte[] recordValue) {
Compression compression = Compression.of(compressionType).build();
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, 0L, MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, 0L,
compressionType, TimestampType.CREATE_TIME, compression, TimestampType.CREATE_TIME,
new SimpleRecord(1000L, "a".getBytes(), "0".getBytes()), new SimpleRecord(1000L, "a".getBytes(), "0".getBytes()),
new SimpleRecord(9999L, "b".getBytes(), recordValue) new SimpleRecord(9999L, "b".getBytes(), recordValue)
); );
@ -500,7 +501,7 @@ public class DefaultRecordBatchTest {
@MethodSource @MethodSource
public void testZstdJniForSkipKeyValueIterator(int expectedJniCalls, byte[] recordValue) throws IOException { public void testZstdJniForSkipKeyValueIterator(int expectedJniCalls, byte[] recordValue) throws IOException {
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, 0L, MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, 0L,
CompressionType.ZSTD, TimestampType.CREATE_TIME, Compression.zstd().build(), TimestampType.CREATE_TIME,
new SimpleRecord(9L, "hakuna-matata".getBytes(), recordValue) new SimpleRecord(9L, "hakuna-matata".getBytes(), recordValue)
); );
@ -508,19 +509,15 @@ public class DefaultRecordBatchTest {
final ByteBuffer compressedBuf = records.buffer(); final ByteBuffer compressedBuf = records.buffer();
// Create a RecordBatch object // Create a RecordBatch object
final DefaultRecordBatch batch = spy(new DefaultRecordBatch(compressedBuf.duplicate())); final DefaultRecordBatch batch = spy(new DefaultRecordBatch(compressedBuf.duplicate()));
final CompressionType mockCompression = mock(CompressionType.ZSTD.getClass());
doReturn(mockCompression).when(batch).compressionType();
// Buffer containing compressed records to be used for creating zstd-jni stream // Buffer containing compressed records to be used for creating zstd-jni stream
ByteBuffer recordsBuffer = compressedBuf.duplicate(); ByteBuffer recordsBuffer = compressedBuf.duplicate();
recordsBuffer.position(RECORDS_OFFSET); recordsBuffer.position(RECORDS_OFFSET);
try (final BufferSupplier bufferSupplier = BufferSupplier.create(); try (final BufferSupplier bufferSupplier = BufferSupplier.create();
final InputStream zstdStream = spy(ZstdFactory.wrapForInput(recordsBuffer, batch.magic(), bufferSupplier)); final InputStream zstdStream = spy(ZstdCompression.wrapForZstdInput(recordsBuffer, bufferSupplier));
final InputStream chunkedStream = new ChunkedBytesStream(zstdStream, bufferSupplier, 16 * 1024, false)) { final InputStream chunkedStream = new ChunkedBytesStream(zstdStream, bufferSupplier, 16 * 1024, false)
) {
when(mockCompression.wrapForInput(any(ByteBuffer.class), anyByte(), any(BufferSupplier.class))).thenReturn(chunkedStream); doReturn(chunkedStream).when(batch).recordInputStream(any());
try (CloseableIterator<Record> streamingIterator = batch.skipKeyValueIterator(bufferSupplier)) { try (CloseableIterator<Record> streamingIterator = batch.skipKeyValueIterator(bufferSupplier)) {
assertNotNull(streamingIterator); assertNotNull(streamingIterator);
Utils.toList(streamingIterator); Utils.toList(streamingIterator);
@ -571,7 +568,7 @@ public class DefaultRecordBatchTest {
private static DefaultRecordBatch recordsWithInvalidRecordCount(Byte magicValue, long timestamp, private static DefaultRecordBatch recordsWithInvalidRecordCount(Byte magicValue, long timestamp,
CompressionType codec, int invalidCount) { CompressionType codec, int invalidCount) {
ByteBuffer buf = ByteBuffer.allocate(512); ByteBuffer buf = ByteBuffer.allocate(512);
MemoryRecordsBuilder builder = MemoryRecords.builder(buf, magicValue, codec, TimestampType.CREATE_TIME, 0L); MemoryRecordsBuilder builder = MemoryRecords.builder(buf, magicValue, Compression.of(codec).build(), TimestampType.CREATE_TIME, 0L);
builder.appendWithOffset(0, timestamp, null, "hello".getBytes()); builder.appendWithOffset(0, timestamp, null, "hello".getBytes());
builder.appendWithOffset(1, timestamp, null, "there".getBytes()); builder.appendWithOffset(1, timestamp, null, "there".getBytes());
builder.appendWithOffset(2, timestamp, null, "beautiful".getBytes()); builder.appendWithOffset(2, timestamp, null, "beautiful".getBytes());

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.common.record; package org.apache.kafka.common.record;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.record.FileLogInputStream.FileChannelRecordBatch; import org.apache.kafka.common.record.FileLogInputStream.FileChannelRecordBatch;
import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.test.TestUtils; import org.apache.kafka.test.TestUtils;
@ -50,9 +51,9 @@ public class FileLogInputStreamTest {
private static class Args { private static class Args {
final byte magic; final byte magic;
final CompressionType compression; final Compression compression;
public Args(byte magic, CompressionType compression) { public Args(byte magic, Compression compression) {
this.magic = magic; this.magic = magic;
this.compression = compression; this.compression = compression;
} }
@ -71,7 +72,7 @@ public class FileLogInputStreamTest {
List<Arguments> arguments = new ArrayList<>(); List<Arguments> arguments = new ArrayList<>();
for (byte magic : asList(MAGIC_VALUE_V0, MAGIC_VALUE_V1, MAGIC_VALUE_V2)) for (byte magic : asList(MAGIC_VALUE_V0, MAGIC_VALUE_V1, MAGIC_VALUE_V2))
for (CompressionType type: CompressionType.values()) for (CompressionType type: CompressionType.values())
arguments.add(Arguments.of(new Args(magic, type))); arguments.add(Arguments.of(new Args(magic, Compression.of(type).build())));
return arguments.stream(); return arguments.stream();
} }
} }
@ -79,9 +80,9 @@ public class FileLogInputStreamTest {
@ParameterizedTest @ParameterizedTest
@ArgumentsSource(FileLogInputStreamArgumentsProvider.class) @ArgumentsSource(FileLogInputStreamArgumentsProvider.class)
public void testWriteTo(Args args) throws IOException { public void testWriteTo(Args args) throws IOException {
CompressionType compression = args.compression; Compression compression = args.compression;
byte magic = args.magic; byte magic = args.magic;
if (compression == CompressionType.ZSTD && magic < MAGIC_VALUE_V2) if (compression.type() == CompressionType.ZSTD && magic < MAGIC_VALUE_V2)
return; return;
try (FileRecords fileRecords = FileRecords.open(tempFile())) { try (FileRecords fileRecords = FileRecords.open(tempFile())) {
@ -110,9 +111,9 @@ public class FileLogInputStreamTest {
@ParameterizedTest @ParameterizedTest
@ArgumentsSource(FileLogInputStreamArgumentsProvider.class) @ArgumentsSource(FileLogInputStreamArgumentsProvider.class)
public void testSimpleBatchIteration(Args args) throws IOException { public void testSimpleBatchIteration(Args args) throws IOException {
CompressionType compression = args.compression; Compression compression = args.compression;
byte magic = args.magic; byte magic = args.magic;
if (compression == CompressionType.ZSTD && magic < MAGIC_VALUE_V2) if (compression.type() == CompressionType.ZSTD && magic < MAGIC_VALUE_V2)
return; return;
try (FileRecords fileRecords = FileRecords.open(tempFile())) { try (FileRecords fileRecords = FileRecords.open(tempFile())) {
@ -140,12 +141,12 @@ public class FileLogInputStreamTest {
@ParameterizedTest @ParameterizedTest
@ArgumentsSource(FileLogInputStreamArgumentsProvider.class) @ArgumentsSource(FileLogInputStreamArgumentsProvider.class)
public void testBatchIterationWithMultipleRecordsPerBatch(Args args) throws IOException { public void testBatchIterationWithMultipleRecordsPerBatch(Args args) throws IOException {
CompressionType compression = args.compression; Compression compression = args.compression;
byte magic = args.magic; byte magic = args.magic;
if (magic < MAGIC_VALUE_V2 && compression == CompressionType.NONE) if (magic < MAGIC_VALUE_V2 && compression.type() == CompressionType.NONE)
return; return;
if (compression == CompressionType.ZSTD && magic < MAGIC_VALUE_V2) if (compression.type() == CompressionType.ZSTD && magic < MAGIC_VALUE_V2)
return; return;
try (FileRecords fileRecords = FileRecords.open(tempFile())) { try (FileRecords fileRecords = FileRecords.open(tempFile())) {
@ -181,7 +182,7 @@ public class FileLogInputStreamTest {
@ParameterizedTest @ParameterizedTest
@ArgumentsSource(FileLogInputStreamArgumentsProvider.class) @ArgumentsSource(FileLogInputStreamArgumentsProvider.class)
public void testBatchIterationV2(Args args) throws IOException { public void testBatchIterationV2(Args args) throws IOException {
CompressionType compression = args.compression; Compression compression = args.compression;
byte magic = args.magic; byte magic = args.magic;
if (magic != MAGIC_VALUE_V2) if (magic != MAGIC_VALUE_V2)
return; return;
@ -229,9 +230,9 @@ public class FileLogInputStreamTest {
@ParameterizedTest @ParameterizedTest
@ArgumentsSource(FileLogInputStreamArgumentsProvider.class) @ArgumentsSource(FileLogInputStreamArgumentsProvider.class)
public void testBatchIterationIncompleteBatch(Args args) throws IOException { public void testBatchIterationIncompleteBatch(Args args) throws IOException {
CompressionType compression = args.compression; Compression compression = args.compression;
byte magic = args.magic; byte magic = args.magic;
if (compression == CompressionType.ZSTD && magic < MAGIC_VALUE_V2) if (compression.type() == CompressionType.ZSTD && magic < MAGIC_VALUE_V2)
return; return;
try (FileRecords fileRecords = FileRecords.open(tempFile())) { try (FileRecords fileRecords = FileRecords.open(tempFile())) {
@ -288,10 +289,10 @@ public class FileLogInputStreamTest {
private void assertGenericRecordBatchData(Args args, RecordBatch batch, long baseOffset, long maxTimestamp, private void assertGenericRecordBatchData(Args args, RecordBatch batch, long baseOffset, long maxTimestamp,
SimpleRecord... records) { SimpleRecord... records) {
CompressionType compression = args.compression; Compression compression = args.compression;
byte magic = args.magic; byte magic = args.magic;
assertEquals(magic, batch.magic()); assertEquals(magic, batch.magic());
assertEquals(compression, batch.compressionType()); assertEquals(compression.type(), batch.compressionType());
if (magic == MAGIC_VALUE_V0) { if (magic == MAGIC_VALUE_V0) {
assertEquals(NO_TIMESTAMP_TYPE, batch.timestampType()); assertEquals(NO_TIMESTAMP_TYPE, batch.timestampType());

View File

@ -18,6 +18,8 @@ package org.apache.kafka.common.record;
import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.compress.GzipCompression;
import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.Header;
import org.apache.kafka.common.header.internals.RecordHeader; import org.apache.kafka.common.header.internals.RecordHeader;
import org.apache.kafka.common.network.TransferableChannel; import org.apache.kafka.common.network.TransferableChannel;
@ -116,7 +118,7 @@ public class FileRecordsTest {
public void testFileSize() throws IOException { public void testFileSize() throws IOException {
assertEquals(fileRecords.channel().size(), fileRecords.sizeInBytes()); assertEquals(fileRecords.channel().size(), fileRecords.sizeInBytes());
for (int i = 0; i < 20; i++) { for (int i = 0; i < 20; i++) {
fileRecords.append(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("abcd".getBytes()))); fileRecords.append(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("abcd".getBytes())));
assertEquals(fileRecords.channel().size(), fileRecords.sizeInBytes()); assertEquals(fileRecords.channel().size(), fileRecords.sizeInBytes());
} }
} }
@ -247,7 +249,7 @@ public class FileRecordsTest {
public void testSearch() throws IOException { public void testSearch() throws IOException {
// append a new message with a high offset // append a new message with a high offset
SimpleRecord lastMessage = new SimpleRecord("test".getBytes()); SimpleRecord lastMessage = new SimpleRecord("test".getBytes());
fileRecords.append(MemoryRecords.withRecords(50L, CompressionType.NONE, lastMessage)); fileRecords.append(MemoryRecords.withRecords(50L, Compression.NONE, lastMessage));
List<RecordBatch> batches = batches(fileRecords); List<RecordBatch> batches = batches(fileRecords);
int position = 0; int position = 0;
@ -484,7 +486,7 @@ public class FileRecordsTest {
int leaderEpoch) throws IOException { int leaderEpoch) throws IOException {
ByteBuffer buffer = ByteBuffer.allocate(128); ByteBuffer buffer = ByteBuffer.allocate(128);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, recordVersion.value, MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, recordVersion.value,
CompressionType.NONE, TimestampType.CREATE_TIME, offset, timestamp, leaderEpoch); Compression.NONE, TimestampType.CREATE_TIME, offset, timestamp, leaderEpoch);
builder.append(new SimpleRecord(timestamp, new byte[0], new byte[0])); builder.append(new SimpleRecord(timestamp, new byte[0], new byte[0]));
fileRecords.append(builder.build()); fileRecords.append(builder.build());
} }
@ -497,7 +499,7 @@ public class FileRecordsTest {
random.nextBytes(bytes); random.nextBytes(bytes);
// records // records
CompressionType compressionType = CompressionType.GZIP; GzipCompression compression = Compression.gzip().build();
List<Long> offsets = asList(0L, 1L); List<Long> offsets = asList(0L, 1L);
List<Byte> magic = asList(RecordBatch.MAGIC_VALUE_V2, RecordBatch.MAGIC_VALUE_V1); // downgrade message format from v2 to v1 List<Byte> magic = asList(RecordBatch.MAGIC_VALUE_V2, RecordBatch.MAGIC_VALUE_V1); // downgrade message format from v2 to v1
List<SimpleRecord> records = asList( List<SimpleRecord> records = asList(
@ -508,7 +510,7 @@ public class FileRecordsTest {
// create MemoryRecords // create MemoryRecords
ByteBuffer buffer = ByteBuffer.allocate(8000); ByteBuffer buffer = ByteBuffer.allocate(8000);
for (int i = 0; i < records.size(); i++) { for (int i = 0; i < records.size(); i++) {
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic.get(i), compressionType, TimestampType.CREATE_TIME, 0L); MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic.get(i), compression, TimestampType.CREATE_TIME, 0L);
builder.appendWithOffset(offsets.get(i), records.get(i)); builder.appendWithOffset(offsets.get(i), records.get(i));
builder.close(); builder.close();
} }
@ -518,18 +520,18 @@ public class FileRecordsTest {
try (FileRecords fileRecords = FileRecords.open(tempFile())) { try (FileRecords fileRecords = FileRecords.open(tempFile())) {
fileRecords.append(MemoryRecords.readableRecords(buffer)); fileRecords.append(MemoryRecords.readableRecords(buffer));
fileRecords.flush(); fileRecords.flush();
downConvertAndVerifyRecords(records, offsets, fileRecords, compressionType, toMagic, 0L, time); downConvertAndVerifyRecords(records, offsets, fileRecords, compression, toMagic, 0L, time);
} }
} }
@Test @Test
public void testConversion() throws IOException { public void testConversion() throws IOException {
doTestConversion(CompressionType.NONE, RecordBatch.MAGIC_VALUE_V0); doTestConversion(Compression.NONE, RecordBatch.MAGIC_VALUE_V0);
doTestConversion(CompressionType.GZIP, RecordBatch.MAGIC_VALUE_V0); doTestConversion(Compression.gzip().build(), RecordBatch.MAGIC_VALUE_V0);
doTestConversion(CompressionType.NONE, RecordBatch.MAGIC_VALUE_V1); doTestConversion(Compression.NONE, RecordBatch.MAGIC_VALUE_V1);
doTestConversion(CompressionType.GZIP, RecordBatch.MAGIC_VALUE_V1); doTestConversion(Compression.gzip().build(), RecordBatch.MAGIC_VALUE_V1);
doTestConversion(CompressionType.NONE, RecordBatch.MAGIC_VALUE_V2); doTestConversion(Compression.NONE, RecordBatch.MAGIC_VALUE_V2);
doTestConversion(CompressionType.GZIP, RecordBatch.MAGIC_VALUE_V2); doTestConversion(Compression.gzip().build(), RecordBatch.MAGIC_VALUE_V2);
} }
@Test @Test
@ -551,7 +553,7 @@ public class FileRecordsTest {
verify(channel).transferFrom(any(), anyLong(), eq((long) size - firstWritten)); verify(channel).transferFrom(any(), anyLong(), eq((long) size - firstWritten));
} }
private void doTestConversion(CompressionType compressionType, byte toMagic) throws IOException { private void doTestConversion(Compression compression, byte toMagic) throws IOException {
List<Long> offsets = asList(0L, 2L, 3L, 9L, 11L, 15L, 16L, 17L, 22L, 24L); List<Long> offsets = asList(0L, 2L, 3L, 9L, 11L, 15L, 16L, 17L, 22L, 24L);
Header[] headers = {new RecordHeader("headerKey1", "headerValue1".getBytes()), Header[] headers = {new RecordHeader("headerKey1", "headerValue1".getBytes()),
@ -572,19 +574,19 @@ public class FileRecordsTest {
assertEquals(offsets.size(), records.size(), "incorrect test setup"); assertEquals(offsets.size(), records.size(), "incorrect test setup");
ByteBuffer buffer = ByteBuffer.allocate(1024); ByteBuffer buffer = ByteBuffer.allocate(1024);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V0, compressionType, MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V0, compression,
TimestampType.CREATE_TIME, 0L); TimestampType.CREATE_TIME, 0L);
for (int i = 0; i < 3; i++) for (int i = 0; i < 3; i++)
builder.appendWithOffset(offsets.get(i), records.get(i)); builder.appendWithOffset(offsets.get(i), records.get(i));
builder.close(); builder.close();
builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V1, compressionType, TimestampType.CREATE_TIME, builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V1, compression, TimestampType.CREATE_TIME,
0L); 0L);
for (int i = 3; i < 6; i++) for (int i = 3; i < 6; i++)
builder.appendWithOffset(offsets.get(i), records.get(i)); builder.appendWithOffset(offsets.get(i), records.get(i));
builder.close(); builder.close();
builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, compressionType, TimestampType.CREATE_TIME, 0L); builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, compression, TimestampType.CREATE_TIME, 0L);
for (int i = 6; i < 10; i++) for (int i = 6; i < 10; i++)
builder.appendWithOffset(offsets.get(i), records.get(i)); builder.appendWithOffset(offsets.get(i), records.get(i));
builder.close(); builder.close();
@ -594,9 +596,9 @@ public class FileRecordsTest {
try (FileRecords fileRecords = FileRecords.open(tempFile())) { try (FileRecords fileRecords = FileRecords.open(tempFile())) {
fileRecords.append(MemoryRecords.readableRecords(buffer)); fileRecords.append(MemoryRecords.readableRecords(buffer));
fileRecords.flush(); fileRecords.flush();
downConvertAndVerifyRecords(records, offsets, fileRecords, compressionType, toMagic, 0L, time); downConvertAndVerifyRecords(records, offsets, fileRecords, compression, toMagic, 0L, time);
if (toMagic <= RecordBatch.MAGIC_VALUE_V1 && compressionType == CompressionType.NONE) { if (toMagic <= RecordBatch.MAGIC_VALUE_V1 && compression.type() == CompressionType.NONE) {
long firstOffset; long firstOffset;
if (toMagic == RecordBatch.MAGIC_VALUE_V0) if (toMagic == RecordBatch.MAGIC_VALUE_V0)
firstOffset = 11L; // v1 record firstOffset = 11L; // v1 record
@ -607,10 +609,10 @@ public class FileRecordsTest {
int index = filteredOffsets.indexOf(firstOffset) - 1; int index = filteredOffsets.indexOf(firstOffset) - 1;
filteredRecords.remove(index); filteredRecords.remove(index);
filteredOffsets.remove(index); filteredOffsets.remove(index);
downConvertAndVerifyRecords(filteredRecords, filteredOffsets, fileRecords, compressionType, toMagic, firstOffset, time); downConvertAndVerifyRecords(filteredRecords, filteredOffsets, fileRecords, compression, toMagic, firstOffset, time);
} else { } else {
// firstOffset doesn't have any effect in this case // firstOffset doesn't have any effect in this case
downConvertAndVerifyRecords(records, offsets, fileRecords, compressionType, toMagic, 10L, time); downConvertAndVerifyRecords(records, offsets, fileRecords, compression, toMagic, 10L, time);
} }
} }
} }
@ -618,7 +620,7 @@ public class FileRecordsTest {
private void downConvertAndVerifyRecords(List<SimpleRecord> initialRecords, private void downConvertAndVerifyRecords(List<SimpleRecord> initialRecords,
List<Long> initialOffsets, List<Long> initialOffsets,
FileRecords fileRecords, FileRecords fileRecords,
CompressionType compressionType, Compression compression,
byte toMagic, byte toMagic,
long firstOffset, long firstOffset,
Time time) { Time time) {
@ -632,7 +634,7 @@ public class FileRecordsTest {
// Test the normal down-conversion path // Test the normal down-conversion path
List<Records> convertedRecords = new ArrayList<>(); List<Records> convertedRecords = new ArrayList<>();
convertedRecords.add(fileRecords.downConvert(toMagic, firstOffset, time).records()); convertedRecords.add(fileRecords.downConvert(toMagic, firstOffset, time).records());
verifyConvertedRecords(initialRecords, initialOffsets, convertedRecords, compressionType, toMagic); verifyConvertedRecords(initialRecords, initialOffsets, convertedRecords, compression, toMagic);
convertedRecords.clear(); convertedRecords.clear();
// Test the lazy down-conversion path // Test the lazy down-conversion path
@ -648,7 +650,7 @@ public class FileRecordsTest {
Iterator<ConvertedRecords<?>> it = lazyRecords.iterator(readSize); Iterator<ConvertedRecords<?>> it = lazyRecords.iterator(readSize);
while (it.hasNext()) while (it.hasNext())
convertedRecords.add(it.next().records()); convertedRecords.add(it.next().records());
verifyConvertedRecords(initialRecords, initialOffsets, convertedRecords, compressionType, toMagic); verifyConvertedRecords(initialRecords, initialOffsets, convertedRecords, compression, toMagic);
convertedRecords.clear(); convertedRecords.clear();
} }
} }
@ -656,7 +658,7 @@ public class FileRecordsTest {
private void verifyConvertedRecords(List<SimpleRecord> initialRecords, private void verifyConvertedRecords(List<SimpleRecord> initialRecords,
List<Long> initialOffsets, List<Long> initialOffsets,
List<Records> convertedRecordsList, List<Records> convertedRecordsList,
CompressionType compressionType, Compression compression,
byte magicByte) { byte magicByte) {
int i = 0; int i = 0;
@ -667,7 +669,7 @@ public class FileRecordsTest {
assertEquals(TimestampType.NO_TIMESTAMP_TYPE, batch.timestampType()); assertEquals(TimestampType.NO_TIMESTAMP_TYPE, batch.timestampType());
else else
assertEquals(TimestampType.CREATE_TIME, batch.timestampType()); assertEquals(TimestampType.CREATE_TIME, batch.timestampType());
assertEquals(compressionType, batch.compressionType(), "Compression type should not be affected by conversion"); assertEquals(compression.type(), batch.compressionType(), "Compression type should not be affected by conversion");
for (Record record : batch) { for (Record record : batch) {
assertTrue(record.hasMagic(batch.magic()), "Inner record should have magic " + magicByte); assertTrue(record.hasMagic(batch.magic()), "Inner record should have magic " + magicByte);
assertEquals(initialOffsets.get(i).longValue(), record.offset(), "Offset should not change"); assertEquals(initialOffsets.get(i).longValue(), record.offset(), "Offset should not change");
@ -710,7 +712,7 @@ public class FileRecordsTest {
for (byte[] value : values) { for (byte[] value : values) {
ByteBuffer buffer = ByteBuffer.allocate(128); ByteBuffer buffer = ByteBuffer.allocate(128);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE,
CompressionType.NONE, TimestampType.CREATE_TIME, offset); Compression.NONE, TimestampType.CREATE_TIME, offset);
builder.appendWithOffset(offset++, System.currentTimeMillis(), null, value); builder.appendWithOffset(offset++, System.currentTimeMillis(), null, value);
fileRecords.append(builder.build()); fileRecords.append(builder.build());
} }

View File

@ -17,6 +17,7 @@
package org.apache.kafka.common.record; package org.apache.kafka.common.record;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.Header;
import org.apache.kafka.common.header.internals.RecordHeader; import org.apache.kafka.common.header.internals.RecordHeader;
import org.apache.kafka.common.network.TransferableChannel; import org.apache.kafka.common.network.TransferableChannel;
@ -112,19 +113,20 @@ public class LazyDownConversionRecordsTest {
assertEquals(offsets.size(), records.size(), "incorrect test setup"); assertEquals(offsets.size(), records.size(), "incorrect test setup");
ByteBuffer buffer = ByteBuffer.allocate(1024); ByteBuffer buffer = ByteBuffer.allocate(1024);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compressionType, Compression compression = Compression.of(compressionType).build();
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compression,
TimestampType.CREATE_TIME, 0L); TimestampType.CREATE_TIME, 0L);
for (int i = 0; i < 3; i++) for (int i = 0; i < 3; i++)
builder.appendWithOffset(offsets.get(i), records.get(i)); builder.appendWithOffset(offsets.get(i), records.get(i));
builder.close(); builder.close();
builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compressionType, TimestampType.CREATE_TIME, builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compression, TimestampType.CREATE_TIME,
0L); 0L);
for (int i = 3; i < 6; i++) for (int i = 3; i < 6; i++)
builder.appendWithOffset(offsets.get(i), records.get(i)); builder.appendWithOffset(offsets.get(i), records.get(i));
builder.close(); builder.close();
builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compressionType, TimestampType.CREATE_TIME, builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, compression, TimestampType.CREATE_TIME,
0L); 0L);
for (int i = 6; i < 10; i++) for (int i = 6; i < 10; i++)
builder.appendWithOffset(offsets.get(i), records.get(i)); builder.appendWithOffset(offsets.get(i), records.get(i));

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.common.record; package org.apache.kafka.common.record;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.errors.UnsupportedCompressionTypeException; import org.apache.kafka.common.errors.UnsupportedCompressionTypeException;
import org.apache.kafka.common.message.LeaderChangeMessage; import org.apache.kafka.common.message.LeaderChangeMessage;
import org.apache.kafka.common.message.LeaderChangeMessage.Voter; import org.apache.kafka.common.message.LeaderChangeMessage.Voter;
@ -60,12 +61,12 @@ public class MemoryRecordsBuilderTest {
private static class Args { private static class Args {
final int bufferOffset; final int bufferOffset;
final CompressionType compressionType; final Compression compression;
final byte magic; final byte magic;
public Args(int bufferOffset, CompressionType compressionType, byte magic) { public Args(int bufferOffset, Compression compression, byte magic) {
this.bufferOffset = bufferOffset; this.bufferOffset = bufferOffset;
this.compressionType = compressionType; this.compression = compression;
this.magic = magic; this.magic = magic;
} }
@ -73,7 +74,7 @@ public class MemoryRecordsBuilderTest {
public String toString() { public String toString() {
return "magic=" + magic + return "magic=" + magic +
", bufferOffset=" + bufferOffset + ", bufferOffset=" + bufferOffset +
", compressionType=" + compressionType; ", compression=" + compression;
} }
} }
@ -87,7 +88,7 @@ public class MemoryRecordsBuilderTest {
? Collections.singletonList(RecordBatch.MAGIC_VALUE_V2) ? Collections.singletonList(RecordBatch.MAGIC_VALUE_V2)
: asList(RecordBatch.MAGIC_VALUE_V0, MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V2); : asList(RecordBatch.MAGIC_VALUE_V0, MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V2);
for (byte magic : magics) for (byte magic : magics)
values.add(Arguments.of(new Args(bufferOffset, type, magic))); values.add(Arguments.of(new Args(bufferOffset, Compression.of(type).build(), magic)));
} }
return values.stream(); return values.stream();
} }
@ -97,13 +98,13 @@ public class MemoryRecordsBuilderTest {
@Test @Test
public void testUnsupportedCompress() { public void testUnsupportedCompress() {
BiFunction<Byte, CompressionType, MemoryRecordsBuilder> builderBiFunction = (magic, compressionType) -> BiFunction<Byte, Compression, MemoryRecordsBuilder> builderBiFunction = (magic, compression) ->
new MemoryRecordsBuilder(ByteBuffer.allocate(128), magic, compressionType, TimestampType.CREATE_TIME, 0L, 0L, new MemoryRecordsBuilder(ByteBuffer.allocate(128), magic, compression, TimestampType.CREATE_TIME, 0L, 0L,
RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE,
false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, 128); false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, 128);
Arrays.asList(MAGIC_VALUE_V0, MAGIC_VALUE_V1).forEach(magic -> { Arrays.asList(MAGIC_VALUE_V0, MAGIC_VALUE_V1).forEach(magic -> {
Exception e = assertThrows(IllegalArgumentException.class, () -> builderBiFunction.apply(magic, CompressionType.ZSTD)); Exception e = assertThrows(IllegalArgumentException.class, () -> builderBiFunction.apply(magic, Compression.zstd().build()));
assertEquals(e.getMessage(), "ZStandard compression is not supported for magic " + magic); assertEquals(e.getMessage(), "ZStandard compression is not supported for magic " + magic);
}); });
} }
@ -115,7 +116,7 @@ public class MemoryRecordsBuilderTest {
ByteBuffer buffer = allocateBuffer(128, args); ByteBuffer buffer = allocateBuffer(128, args);
MemoryRecords records = new MemoryRecordsBuilder(buffer, magic, MemoryRecords records = new MemoryRecordsBuilder(buffer, magic,
args.compressionType, TimestampType.CREATE_TIME, 0L, 0L, args.compression, TimestampType.CREATE_TIME, 0L, 0L,
RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE,
false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity()).build(); false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity()).build();
@ -131,7 +132,7 @@ public class MemoryRecordsBuilderTest {
short epoch = 15; short epoch = 15;
int sequence = 2342; int sequence = 2342;
Supplier<MemoryRecordsBuilder> supplier = () -> new MemoryRecordsBuilder(buffer, args.magic, args.compressionType, Supplier<MemoryRecordsBuilder> supplier = () -> new MemoryRecordsBuilder(buffer, args.magic, args.compression,
TimestampType.CREATE_TIME, 0L, 0L, pid, epoch, sequence, true, false, TimestampType.CREATE_TIME, 0L, 0L, pid, epoch, sequence, true, false,
RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity()); RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
@ -156,7 +157,7 @@ public class MemoryRecordsBuilderTest {
short epoch = 15; short epoch = 15;
int sequence = 2342; int sequence = 2342;
Supplier<MemoryRecordsBuilder> supplier = () -> new MemoryRecordsBuilder(buffer, args.magic, args.compressionType, TimestampType.CREATE_TIME, Supplier<MemoryRecordsBuilder> supplier = () -> new MemoryRecordsBuilder(buffer, args.magic, args.compression, TimestampType.CREATE_TIME,
0L, 0L, pid, epoch, sequence, true, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity()); 0L, 0L, pid, epoch, sequence, true, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
if (args.magic < MAGIC_VALUE_V2) { if (args.magic < MAGIC_VALUE_V2) {
assertThrows(IllegalArgumentException.class, supplier::get); assertThrows(IllegalArgumentException.class, supplier::get);
@ -174,7 +175,7 @@ public class MemoryRecordsBuilderTest {
short epoch = RecordBatch.NO_PRODUCER_EPOCH; short epoch = RecordBatch.NO_PRODUCER_EPOCH;
int sequence = 2342; int sequence = 2342;
Supplier<MemoryRecordsBuilder> supplier = () -> new MemoryRecordsBuilder(buffer, args.magic, args.compressionType, TimestampType.CREATE_TIME, Supplier<MemoryRecordsBuilder> supplier = () -> new MemoryRecordsBuilder(buffer, args.magic, args.compression, TimestampType.CREATE_TIME,
0L, 0L, pid, epoch, sequence, true, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity()); 0L, 0L, pid, epoch, sequence, true, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
if (args.magic < MAGIC_VALUE_V2) { if (args.magic < MAGIC_VALUE_V2) {
@ -193,7 +194,7 @@ public class MemoryRecordsBuilderTest {
short epoch = 15; short epoch = 15;
int sequence = RecordBatch.NO_SEQUENCE; int sequence = RecordBatch.NO_SEQUENCE;
Supplier<MemoryRecordsBuilder> supplier = () -> new MemoryRecordsBuilder(buffer, args.magic, args.compressionType, TimestampType.CREATE_TIME, Supplier<MemoryRecordsBuilder> supplier = () -> new MemoryRecordsBuilder(buffer, args.magic, args.compression, TimestampType.CREATE_TIME,
0L, 0L, pid, epoch, sequence, true, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity()); 0L, 0L, pid, epoch, sequence, true, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
if (args.magic < MAGIC_VALUE_V2) { if (args.magic < MAGIC_VALUE_V2) {
@ -212,7 +213,7 @@ public class MemoryRecordsBuilderTest {
short epoch = 15; short epoch = 15;
int sequence = RecordBatch.NO_SEQUENCE; int sequence = RecordBatch.NO_SEQUENCE;
Supplier<MemoryRecordsBuilder> supplier = () -> new MemoryRecordsBuilder(buffer, args.magic, args.compressionType, Supplier<MemoryRecordsBuilder> supplier = () -> new MemoryRecordsBuilder(buffer, args.magic, args.compression,
TimestampType.CREATE_TIME, 0L, 0L, pid, epoch, sequence, false, true, TimestampType.CREATE_TIME, 0L, 0L, pid, epoch, sequence, false, true,
RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity()); RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
@ -233,7 +234,7 @@ public class MemoryRecordsBuilderTest {
short epoch = 15; short epoch = 15;
int sequence = RecordBatch.NO_SEQUENCE; int sequence = RecordBatch.NO_SEQUENCE;
Supplier<MemoryRecordsBuilder> supplier = () -> new MemoryRecordsBuilder(buffer, args.magic, args.compressionType, TimestampType.CREATE_TIME, Supplier<MemoryRecordsBuilder> supplier = () -> new MemoryRecordsBuilder(buffer, args.magic, args.compression, TimestampType.CREATE_TIME,
0L, 0L, pid, epoch, sequence, true, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity()); 0L, 0L, pid, epoch, sequence, true, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
if (args.magic < MAGIC_VALUE_V2) { if (args.magic < MAGIC_VALUE_V2) {
@ -249,7 +250,7 @@ public class MemoryRecordsBuilderTest {
@ArgumentsSource(MemoryRecordsBuilderArgumentsProvider.class) @ArgumentsSource(MemoryRecordsBuilderArgumentsProvider.class)
public void testWriteLeaderChangeControlBatchWithoutLeaderEpoch(Args args) { public void testWriteLeaderChangeControlBatchWithoutLeaderEpoch(Args args) {
ByteBuffer buffer = allocateBuffer(128, args); ByteBuffer buffer = allocateBuffer(128, args);
Supplier<MemoryRecordsBuilder> supplier = () -> new MemoryRecordsBuilder(buffer, args.magic, args.compressionType, Supplier<MemoryRecordsBuilder> supplier = () -> new MemoryRecordsBuilder(buffer, args.magic, args.compression,
TimestampType.CREATE_TIME, 0L, 0L, TimestampType.CREATE_TIME, 0L, 0L,
RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE,
false, true, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity()); false, true, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
@ -272,7 +273,7 @@ public class MemoryRecordsBuilderTest {
final int leaderEpoch = 5; final int leaderEpoch = 5;
final List<Integer> voters = Arrays.asList(2, 3); final List<Integer> voters = Arrays.asList(2, 3);
Supplier<MemoryRecordsBuilder> supplier = () -> new MemoryRecordsBuilder(buffer, args.magic, args.compressionType, Supplier<MemoryRecordsBuilder> supplier = () -> new MemoryRecordsBuilder(buffer, args.magic, args.compression,
TimestampType.CREATE_TIME, 0L, 0L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, TimestampType.CREATE_TIME, 0L, 0L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH,
RecordBatch.NO_SEQUENCE, false, true, leaderEpoch, buffer.capacity()); RecordBatch.NO_SEQUENCE, false, true, leaderEpoch, buffer.capacity());
@ -313,7 +314,7 @@ public class MemoryRecordsBuilderTest {
} else { } else {
LegacyRecord[] records = supplier.get(); LegacyRecord[] records = supplier.get();
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, magic, args.compressionType, MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, magic, args.compression,
TimestampType.CREATE_TIME, 0L, 0L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, TimestampType.CREATE_TIME, 0L, 0L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE,
false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity()); false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
@ -324,7 +325,7 @@ public class MemoryRecordsBuilderTest {
} }
MemoryRecords built = builder.build(); MemoryRecords built = builder.build();
if (args.compressionType == CompressionType.NONE) { if (args.compression.type() == CompressionType.NONE) {
assertEquals(1.0, builder.compressionRatio(), 0.00001); assertEquals(1.0, builder.compressionRatio(), 0.00001);
} else { } else {
int recordHead = magic == MAGIC_VALUE_V0 ? LegacyRecord.RECORD_OVERHEAD_V0 : LegacyRecord.RECORD_OVERHEAD_V1; int recordHead = magic == MAGIC_VALUE_V0 ? LegacyRecord.RECORD_OVERHEAD_V0 : LegacyRecord.RECORD_OVERHEAD_V1;
@ -340,7 +341,7 @@ public class MemoryRecordsBuilderTest {
public void testEstimatedSizeInBytes(Args args) { public void testEstimatedSizeInBytes(Args args) {
ByteBuffer buffer = allocateBuffer(1024, args); ByteBuffer buffer = allocateBuffer(1024, args);
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, args.magic, args.compressionType, MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, args.magic, args.compression,
TimestampType.CREATE_TIME, 0L, 0L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, TimestampType.CREATE_TIME, 0L, 0L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE,
false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity()); false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
@ -355,7 +356,7 @@ public class MemoryRecordsBuilderTest {
int bytesWrittenBeforeClose = builder.estimatedSizeInBytes(); int bytesWrittenBeforeClose = builder.estimatedSizeInBytes();
MemoryRecords records = builder.build(); MemoryRecords records = builder.build();
assertEquals(records.sizeInBytes(), builder.estimatedSizeInBytes()); assertEquals(records.sizeInBytes(), builder.estimatedSizeInBytes());
if (args.compressionType == CompressionType.NONE) if (args.compression.type() == CompressionType.NONE)
assertEquals(records.sizeInBytes(), bytesWrittenBeforeClose); assertEquals(records.sizeInBytes(), bytesWrittenBeforeClose);
} }
@ -367,7 +368,7 @@ public class MemoryRecordsBuilderTest {
ByteBuffer buffer = allocateBuffer(1024, args); ByteBuffer buffer = allocateBuffer(1024, args);
long logAppendTime = System.currentTimeMillis(); long logAppendTime = System.currentTimeMillis();
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, magic, args.compressionType, MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, magic, args.compression,
TimestampType.LOG_APPEND_TIME, 0L, logAppendTime, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, TimestampType.LOG_APPEND_TIME, 0L, logAppendTime, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH,
RecordBatch.NO_SEQUENCE, false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity()); RecordBatch.NO_SEQUENCE, false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
builder.append(0L, "a".getBytes(), "1".getBytes()); builder.append(0L, "a".getBytes(), "1".getBytes());
@ -378,7 +379,7 @@ public class MemoryRecordsBuilderTest {
MemoryRecordsBuilder.RecordsInfo info = builder.info(); MemoryRecordsBuilder.RecordsInfo info = builder.info();
assertEquals(logAppendTime, info.maxTimestamp); assertEquals(logAppendTime, info.maxTimestamp);
if (args.compressionType == CompressionType.NONE && magic <= MAGIC_VALUE_V1) if (args.compression.type() == CompressionType.NONE && magic <= MAGIC_VALUE_V1)
assertEquals(0L, info.shallowOffsetOfMaxTimestamp); assertEquals(0L, info.shallowOffsetOfMaxTimestamp);
else else
assertEquals(2L, info.shallowOffsetOfMaxTimestamp); assertEquals(2L, info.shallowOffsetOfMaxTimestamp);
@ -400,7 +401,7 @@ public class MemoryRecordsBuilderTest {
ByteBuffer buffer = allocateBuffer(1024, args); ByteBuffer buffer = allocateBuffer(1024, args);
long logAppendTime = System.currentTimeMillis(); long logAppendTime = System.currentTimeMillis();
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, magic, args.compressionType, MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, magic, args.compression,
TimestampType.CREATE_TIME, 0L, logAppendTime, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, TimestampType.CREATE_TIME, 0L, logAppendTime, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE,
false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity()); false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
builder.append(0L, "a".getBytes(), "1".getBytes()); builder.append(0L, "a".getBytes(), "1".getBytes());
@ -417,7 +418,7 @@ public class MemoryRecordsBuilderTest {
if (magic == MAGIC_VALUE_V0) if (magic == MAGIC_VALUE_V0)
assertEquals(-1, info.shallowOffsetOfMaxTimestamp); assertEquals(-1, info.shallowOffsetOfMaxTimestamp);
else if (args.compressionType == CompressionType.NONE && magic == MAGIC_VALUE_V1) else if (args.compression.type() == CompressionType.NONE && magic == MAGIC_VALUE_V1)
assertEquals(1L, info.shallowOffsetOfMaxTimestamp); assertEquals(1L, info.shallowOffsetOfMaxTimestamp);
else else
assertEquals(2L, info.shallowOffsetOfMaxTimestamp); assertEquals(2L, info.shallowOffsetOfMaxTimestamp);
@ -439,7 +440,7 @@ public class MemoryRecordsBuilderTest {
@ArgumentsSource(MemoryRecordsBuilderArgumentsProvider.class) @ArgumentsSource(MemoryRecordsBuilderArgumentsProvider.class)
public void testAppendedChecksumConsistency(Args args) { public void testAppendedChecksumConsistency(Args args) {
ByteBuffer buffer = ByteBuffer.allocate(512); ByteBuffer buffer = ByteBuffer.allocate(512);
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, args.magic, args.compressionType, MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, args.magic, args.compression,
TimestampType.CREATE_TIME, 0L, LegacyRecord.NO_TIMESTAMP, RecordBatch.NO_PRODUCER_ID, TimestampType.CREATE_TIME, 0L, LegacyRecord.NO_TIMESTAMP, RecordBatch.NO_PRODUCER_ID,
RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, false, false, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, false, false,
RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity()); RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
@ -458,7 +459,7 @@ public class MemoryRecordsBuilderTest {
byte[] value = "bar".getBytes(); byte[] value = "bar".getBytes();
int writeLimit = 0; int writeLimit = 0;
ByteBuffer buffer = ByteBuffer.allocate(512); ByteBuffer buffer = ByteBuffer.allocate(512);
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, args.magic, args.compressionType, MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, args.magic, args.compression,
TimestampType.CREATE_TIME, 0L, LegacyRecord.NO_TIMESTAMP, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, TimestampType.CREATE_TIME, 0L, LegacyRecord.NO_TIMESTAMP, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH,
RecordBatch.NO_SEQUENCE, false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, writeLimit); RecordBatch.NO_SEQUENCE, false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, writeLimit);
@ -485,7 +486,7 @@ public class MemoryRecordsBuilderTest {
ByteBuffer buffer = allocateBuffer(64, args); ByteBuffer buffer = allocateBuffer(64, args);
long logAppendTime = System.currentTimeMillis(); long logAppendTime = System.currentTimeMillis();
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, magic, args.compressionType, MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, magic, args.compression,
TimestampType.CREATE_TIME, 0L, logAppendTime, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, TimestampType.CREATE_TIME, 0L, logAppendTime, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE,
false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity()); false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
builder.setEstimatedCompressionRatio(0.5f); builder.setEstimatedCompressionRatio(0.5f);
@ -523,7 +524,7 @@ public class MemoryRecordsBuilderTest {
ByteBuffer buffer = allocateBuffer(1024, args); ByteBuffer buffer = allocateBuffer(1024, args);
long logAppendTime = System.currentTimeMillis(); long logAppendTime = System.currentTimeMillis();
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, args.magic, args.compressionType, MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, args.magic, args.compression,
TimestampType.CREATE_TIME, 0L, logAppendTime, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, TimestampType.CREATE_TIME, 0L, logAppendTime, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE,
false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity()); false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
@ -538,8 +539,9 @@ public class MemoryRecordsBuilderTest {
@EnumSource(CompressionType.class) @EnumSource(CompressionType.class)
public void convertV2ToV1UsingMixedCreateAndLogAppendTime(CompressionType compressionType) { public void convertV2ToV1UsingMixedCreateAndLogAppendTime(CompressionType compressionType) {
ByteBuffer buffer = ByteBuffer.allocate(512); ByteBuffer buffer = ByteBuffer.allocate(512);
Compression compression = Compression.of(compressionType).build();
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2,
compressionType, TimestampType.LOG_APPEND_TIME, 0L); compression, TimestampType.LOG_APPEND_TIME, 0L);
builder.append(10L, "1".getBytes(), "a".getBytes()); builder.append(10L, "1".getBytes(), "a".getBytes());
builder.close(); builder.close();
@ -550,7 +552,7 @@ public class MemoryRecordsBuilderTest {
int position = buffer.position(); int position = buffer.position();
builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, compressionType, builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, compression,
TimestampType.CREATE_TIME, 1L); TimestampType.CREATE_TIME, 1L);
builder.append(12L, "2".getBytes(), "b".getBytes()); builder.append(12L, "2".getBytes(), "b".getBytes());
builder.append(13L, "3".getBytes(), "c".getBytes()); builder.append(13L, "3".getBytes(), "c".getBytes());
@ -566,16 +568,16 @@ public class MemoryRecordsBuilderTest {
Supplier<ConvertedRecords<MemoryRecords>> convertedRecordsSupplier = () -> Supplier<ConvertedRecords<MemoryRecords>> convertedRecordsSupplier = () ->
MemoryRecords.readableRecords(buffer).downConvert(MAGIC_VALUE_V1, 0, time); MemoryRecords.readableRecords(buffer).downConvert(MAGIC_VALUE_V1, 0, time);
if (compressionType != CompressionType.ZSTD) { if (compression.type() != CompressionType.ZSTD) {
ConvertedRecords<MemoryRecords> convertedRecords = convertedRecordsSupplier.get(); ConvertedRecords<MemoryRecords> convertedRecords = convertedRecordsSupplier.get();
MemoryRecords records = convertedRecords.records(); MemoryRecords records = convertedRecords.records();
// Transactional markers are skipped when down converting to V1, so exclude them from size // Transactional markers are skipped when down converting to V1, so exclude them from size
verifyRecordsProcessingStats(compressionType, convertedRecords.recordConversionStats(), verifyRecordsProcessingStats(compression, convertedRecords.recordConversionStats(),
3, 3, records.sizeInBytes(), sizeExcludingTxnMarkers); 3, 3, records.sizeInBytes(), sizeExcludingTxnMarkers);
List<? extends RecordBatch> batches = Utils.toList(records.batches().iterator()); List<? extends RecordBatch> batches = Utils.toList(records.batches().iterator());
if (compressionType != CompressionType.NONE) { if (compression.type() != CompressionType.NONE) {
assertEquals(2, batches.size()); assertEquals(2, batches.size());
assertEquals(TimestampType.LOG_APPEND_TIME, batches.get(0).timestampType()); assertEquals(TimestampType.LOG_APPEND_TIME, batches.get(0).timestampType());
assertEquals(TimestampType.CREATE_TIME, batches.get(1).timestampType()); assertEquals(TimestampType.CREATE_TIME, batches.get(1).timestampType());
@ -602,8 +604,9 @@ public class MemoryRecordsBuilderTest {
public void convertToV1WithMixedV0AndV2Data(CompressionType compressionType) { public void convertToV1WithMixedV0AndV2Data(CompressionType compressionType) {
ByteBuffer buffer = ByteBuffer.allocate(512); ByteBuffer buffer = ByteBuffer.allocate(512);
Compression compression = Compression.of(compressionType).build();
Supplier<MemoryRecordsBuilder> supplier = () -> MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V0, Supplier<MemoryRecordsBuilder> supplier = () -> MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V0,
compressionType, TimestampType.NO_TIMESTAMP_TYPE, 0L); compression, TimestampType.NO_TIMESTAMP_TYPE, 0L);
if (compressionType == CompressionType.ZSTD) { if (compressionType == CompressionType.ZSTD) {
assertThrows(IllegalArgumentException.class, supplier::get); assertThrows(IllegalArgumentException.class, supplier::get);
@ -612,7 +615,7 @@ public class MemoryRecordsBuilderTest {
builder.append(RecordBatch.NO_TIMESTAMP, "1".getBytes(), "a".getBytes()); builder.append(RecordBatch.NO_TIMESTAMP, "1".getBytes(), "a".getBytes());
builder.close(); builder.close();
builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, compressionType, builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, compression,
TimestampType.CREATE_TIME, 1L); TimestampType.CREATE_TIME, 1L);
builder.append(11L, "2".getBytes(), "b".getBytes()); builder.append(11L, "2".getBytes(), "b".getBytes());
builder.append(12L, "3".getBytes(), "c".getBytes()); builder.append(12L, "3".getBytes(), "c".getBytes());
@ -623,7 +626,7 @@ public class MemoryRecordsBuilderTest {
ConvertedRecords<MemoryRecords> convertedRecords = MemoryRecords.readableRecords(buffer) ConvertedRecords<MemoryRecords> convertedRecords = MemoryRecords.readableRecords(buffer)
.downConvert(MAGIC_VALUE_V1, 0, time); .downConvert(MAGIC_VALUE_V1, 0, time);
MemoryRecords records = convertedRecords.records(); MemoryRecords records = convertedRecords.records();
verifyRecordsProcessingStats(compressionType, convertedRecords.recordConversionStats(), 3, 2, verifyRecordsProcessingStats(compression, convertedRecords.recordConversionStats(), 3, 2,
records.sizeInBytes(), buffer.limit()); records.sizeInBytes(), buffer.limit());
List<? extends RecordBatch> batches = Utils.toList(records.batches().iterator()); List<? extends RecordBatch> batches = Utils.toList(records.batches().iterator());
@ -663,7 +666,7 @@ public class MemoryRecordsBuilderTest {
assertEquals("1", utf8(logRecords.get(0).key())); assertEquals("1", utf8(logRecords.get(0).key()));
assertEquals("2", utf8(logRecords.get(1).key())); assertEquals("2", utf8(logRecords.get(1).key()));
assertEquals("3", utf8(logRecords.get(2).key())); assertEquals("3", utf8(logRecords.get(2).key()));
verifyRecordsProcessingStats(compressionType, convertedRecords.recordConversionStats(), 3, 2, verifyRecordsProcessingStats(compression, convertedRecords.recordConversionStats(), 3, 2,
records.sizeInBytes(), buffer.limit()); records.sizeInBytes(), buffer.limit());
} else { } else {
assertEquals(2, batches.size()); assertEquals(2, batches.size());
@ -673,7 +676,7 @@ public class MemoryRecordsBuilderTest {
assertEquals(2, batches.get(1).baseOffset()); assertEquals(2, batches.get(1).baseOffset());
assertEquals("1", utf8(logRecords.get(0).key())); assertEquals("1", utf8(logRecords.get(0).key()));
assertEquals("3", utf8(logRecords.get(1).key())); assertEquals("3", utf8(logRecords.get(1).key()));
verifyRecordsProcessingStats(compressionType, convertedRecords.recordConversionStats(), 3, 1, verifyRecordsProcessingStats(compression, convertedRecords.recordConversionStats(), 3, 1,
records.sizeInBytes(), buffer.limit()); records.sizeInBytes(), buffer.limit());
} }
} }
@ -684,7 +687,7 @@ public class MemoryRecordsBuilderTest {
public void shouldThrowIllegalStateExceptionOnBuildWhenAborted(Args args) { public void shouldThrowIllegalStateExceptionOnBuildWhenAborted(Args args) {
ByteBuffer buffer = allocateBuffer(128, args); ByteBuffer buffer = allocateBuffer(128, args);
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, args.magic, args.compressionType, MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, args.magic, args.compression,
TimestampType.CREATE_TIME, 0L, 0L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, TimestampType.CREATE_TIME, 0L, 0L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH,
RecordBatch.NO_SEQUENCE, false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity()); RecordBatch.NO_SEQUENCE, false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
builder.abort(); builder.abort();
@ -696,7 +699,7 @@ public class MemoryRecordsBuilderTest {
public void shouldResetBufferToInitialPositionOnAbort(Args args) { public void shouldResetBufferToInitialPositionOnAbort(Args args) {
ByteBuffer buffer = allocateBuffer(128, args); ByteBuffer buffer = allocateBuffer(128, args);
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, args.magic, args.compressionType, MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, args.magic, args.compression,
TimestampType.CREATE_TIME, 0L, 0L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, TimestampType.CREATE_TIME, 0L, 0L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE,
false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity()); false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
builder.append(0L, "a".getBytes(), "1".getBytes()); builder.append(0L, "a".getBytes(), "1".getBytes());
@ -709,7 +712,7 @@ public class MemoryRecordsBuilderTest {
public void shouldThrowIllegalStateExceptionOnCloseWhenAborted(Args args) { public void shouldThrowIllegalStateExceptionOnCloseWhenAborted(Args args) {
ByteBuffer buffer = allocateBuffer(128, args); ByteBuffer buffer = allocateBuffer(128, args);
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, args.magic, args.compressionType, MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, args.magic, args.compression,
TimestampType.CREATE_TIME, 0L, 0L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, TimestampType.CREATE_TIME, 0L, 0L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE,
false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity()); false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
builder.abort(); builder.abort();
@ -721,7 +724,7 @@ public class MemoryRecordsBuilderTest {
public void shouldThrowIllegalStateExceptionOnAppendWhenAborted(Args args) { public void shouldThrowIllegalStateExceptionOnAppendWhenAborted(Args args) {
ByteBuffer buffer = allocateBuffer(128, args); ByteBuffer buffer = allocateBuffer(128, args);
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, args.magic, args.compressionType, MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, args.magic, args.compression,
TimestampType.CREATE_TIME, 0L, 0L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, TimestampType.CREATE_TIME, 0L, 0L, RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE,
false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity()); false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
builder.abort(); builder.abort();
@ -743,7 +746,7 @@ public class MemoryRecordsBuilderTest {
int iterations = 0; int iterations = 0;
while (iterations++ < 100) { while (iterations++ < 100) {
buffer.rewind(); buffer.rewind();
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, args.magic, args.compressionType, MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, args.magic, args.compression,
TimestampType.CREATE_TIME, 0L, 0L, RecordBatch.NO_PRODUCER_ID, TimestampType.CREATE_TIME, 0L, 0L, RecordBatch.NO_PRODUCER_ID,
RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, false, false, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, false, false,
RecordBatch.NO_PARTITION_LEADER_EPOCH, 0); RecordBatch.NO_PARTITION_LEADER_EPOCH, 0);
@ -769,7 +772,7 @@ public class MemoryRecordsBuilderTest {
int payloadLen = 1024 * 1024; int payloadLen = 1024 * 1024;
ByteBuffer buffer = ByteBuffer.allocate(payloadLen * 2); ByteBuffer buffer = ByteBuffer.allocate(payloadLen * 2);
ByteBufferOutputStream byteBufferOutputStream = new ByteBufferOutputStream(buffer); ByteBufferOutputStream byteBufferOutputStream = new ByteBufferOutputStream(buffer);
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(byteBufferOutputStream, args.magic, args.compressionType, MemoryRecordsBuilder builder = new MemoryRecordsBuilder(byteBufferOutputStream, args.magic, args.compression,
TimestampType.CREATE_TIME, 0L, 0L, RecordBatch.NO_PRODUCER_ID, TimestampType.CREATE_TIME, 0L, 0L, RecordBatch.NO_PRODUCER_ID,
RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, false, false, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, false, false,
RecordBatch.NO_PARTITION_LEADER_EPOCH, 0, deleteHorizon); RecordBatch.NO_PARTITION_LEADER_EPOCH, 0, deleteHorizon);
@ -798,13 +801,13 @@ public class MemoryRecordsBuilderTest {
List<Arguments> values = new ArrayList<>(); List<Arguments> values = new ArrayList<>();
for (int bufferOffset : Arrays.asList(0, 15)) for (int bufferOffset : Arrays.asList(0, 15))
for (CompressionType type: CompressionType.values()) { for (CompressionType type: CompressionType.values()) {
values.add(Arguments.of(new Args(bufferOffset, type, MAGIC_VALUE_V2))); values.add(Arguments.of(new Args(bufferOffset, Compression.of(type).build(), MAGIC_VALUE_V2)));
} }
return values.stream(); return values.stream();
} }
} }
private void verifyRecordsProcessingStats(CompressionType compressionType, RecordValidationStats processingStats, private void verifyRecordsProcessingStats(Compression compression, RecordValidationStats processingStats,
int numRecords, int numRecordsConverted, long finalBytes, int numRecords, int numRecordsConverted, long finalBytes,
long preConvertedBytes) { long preConvertedBytes) {
assertNotNull(processingStats, "Records processing info is null"); assertNotNull(processingStats, "Records processing info is null");
@ -813,7 +816,7 @@ public class MemoryRecordsBuilderTest {
// only check if the value >= 0. Default is -1, so this checks if time has been recorded. // only check if the value >= 0. Default is -1, so this checks if time has been recorded.
assertTrue(processingStats.conversionTimeNanos() >= 0, "Processing time not recorded: " + processingStats); assertTrue(processingStats.conversionTimeNanos() >= 0, "Processing time not recorded: " + processingStats);
long tempBytes = processingStats.temporaryMemoryBytes(); long tempBytes = processingStats.temporaryMemoryBytes();
if (compressionType == CompressionType.NONE) { if (compression.type() == CompressionType.NONE) {
if (numRecordsConverted == 0) if (numRecordsConverted == 0)
assertEquals(finalBytes, tempBytes); assertEquals(finalBytes, tempBytes);
else if (numRecordsConverted == numRecords) else if (numRecordsConverted == numRecords)

View File

@ -17,6 +17,7 @@
package org.apache.kafka.common.record; package org.apache.kafka.common.record;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.errors.CorruptRecordException; import org.apache.kafka.common.errors.CorruptRecordException;
import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.header.internals.RecordHeaders;
import org.apache.kafka.common.message.LeaderChangeMessage; import org.apache.kafka.common.message.LeaderChangeMessage;
@ -58,14 +59,14 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
public class MemoryRecordsTest { public class MemoryRecordsTest {
private static class Args { private static class Args {
final CompressionType compression; final Compression compression;
final byte magic; final byte magic;
final long firstOffset; final long firstOffset;
final long pid; final long pid;
final short epoch; final short epoch;
final int firstSequence; final int firstSequence;
public Args(byte magic, long firstOffset, CompressionType compression) { public Args(byte magic, long firstOffset, Compression compression) {
this.magic = magic; this.magic = magic;
this.compression = compression; this.compression = compression;
this.firstOffset = firstOffset; this.firstOffset = firstOffset;
@ -84,7 +85,7 @@ public class MemoryRecordsTest {
public String toString() { public String toString() {
return "magic=" + magic + return "magic=" + magic +
", firstOffset=" + firstOffset + ", firstOffset=" + firstOffset +
", compressionType=" + compression; ", compression=" + compression;
} }
} }
@ -98,7 +99,7 @@ public class MemoryRecordsTest {
? Collections.singletonList(RecordBatch.MAGIC_VALUE_V2) ? Collections.singletonList(RecordBatch.MAGIC_VALUE_V2)
: asList(RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V2); : asList(RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V2);
for (byte magic : magics) for (byte magic : magics)
arguments.add(Arguments.of(new Args(magic, firstOffset, type))); arguments.add(Arguments.of(new Args(magic, firstOffset, Compression.of(type).build())));
} }
return arguments.stream(); return arguments.stream();
} }
@ -110,7 +111,7 @@ public class MemoryRecordsTest {
List<Arguments> arguments = new ArrayList<>(); List<Arguments> arguments = new ArrayList<>();
for (long firstOffset : asList(0L, 57L)) for (long firstOffset : asList(0L, 57L))
for (CompressionType type: CompressionType.values()) { for (CompressionType type: CompressionType.values()) {
arguments.add(Arguments.of(new Args(RecordBatch.MAGIC_VALUE_V2, firstOffset, type))); arguments.add(Arguments.of(new Args(RecordBatch.MAGIC_VALUE_V2, firstOffset, Compression.of(type).build())));
} }
return arguments.stream(); return arguments.stream();
} }
@ -121,7 +122,7 @@ public class MemoryRecordsTest {
@ParameterizedTest @ParameterizedTest
@ArgumentsSource(MemoryRecordsArgumentsProvider.class) @ArgumentsSource(MemoryRecordsArgumentsProvider.class)
public void testIterator(Args args) { public void testIterator(Args args) {
CompressionType compression = args.compression; Compression compression = args.compression;
byte magic = args.magic; byte magic = args.magic;
long pid = args.pid; long pid = args.pid;
short epoch = args.epoch; short epoch = args.epoch;
@ -151,7 +152,7 @@ public class MemoryRecordsTest {
int total = 0; int total = 0;
for (RecordBatch batch : memoryRecords.batches()) { for (RecordBatch batch : memoryRecords.batches()) {
assertTrue(batch.isValid()); assertTrue(batch.isValid());
assertEquals(compression, batch.compressionType()); assertEquals(compression.type(), batch.compressionType());
assertEquals(firstOffset + total, batch.baseOffset()); assertEquals(firstOffset + total, batch.baseOffset());
if (magic >= RecordBatch.MAGIC_VALUE_V2) { if (magic >= RecordBatch.MAGIC_VALUE_V2) {
@ -243,10 +244,10 @@ public class MemoryRecordsTest {
@ParameterizedTest @ParameterizedTest
@ArgumentsSource(MemoryRecordsArgumentsProvider.class) @ArgumentsSource(MemoryRecordsArgumentsProvider.class)
public void testChecksum(Args args) { public void testChecksum(Args args) {
CompressionType compression = args.compression; Compression compression = args.compression;
byte magic = args.magic; byte magic = args.magic;
// we get reasonable coverage with uncompressed and one compression type // we get reasonable coverage with uncompressed and one compression type
if (compression != CompressionType.NONE && compression != CompressionType.LZ4) if (compression.type() != CompressionType.NONE && compression.type() != CompressionType.LZ4)
return; return;
SimpleRecord[] records = { SimpleRecord[] records = {
@ -256,17 +257,17 @@ public class MemoryRecordsTest {
RecordBatch batch = MemoryRecords.withRecords(magic, compression, records).batches().iterator().next(); RecordBatch batch = MemoryRecords.withRecords(magic, compression, records).batches().iterator().next();
long expectedChecksum; long expectedChecksum;
if (magic == RecordBatch.MAGIC_VALUE_V0) { if (magic == RecordBatch.MAGIC_VALUE_V0) {
if (compression == CompressionType.NONE) if (compression.type() == CompressionType.NONE)
expectedChecksum = 1978725405L; expectedChecksum = 1978725405L;
else else
expectedChecksum = 66944826L; expectedChecksum = 66944826L;
} else if (magic == RecordBatch.MAGIC_VALUE_V1) { } else if (magic == RecordBatch.MAGIC_VALUE_V1) {
if (compression == CompressionType.NONE) if (compression.type() == CompressionType.NONE)
expectedChecksum = 109425508L; expectedChecksum = 109425508L;
else else
expectedChecksum = 1407303399L; expectedChecksum = 1407303399L;
} else { } else {
if (compression == CompressionType.NONE) if (compression.type() == CompressionType.NONE)
expectedChecksum = 3851219455L; expectedChecksum = 3851219455L;
else else
expectedChecksum = 2745969314L; expectedChecksum = 2745969314L;
@ -590,7 +591,7 @@ public class MemoryRecordsTest {
@ParameterizedTest @ParameterizedTest
@ArgumentsSource(MemoryRecordsArgumentsProvider.class) @ArgumentsSource(MemoryRecordsArgumentsProvider.class)
public void testFilterToBatchDiscard(Args args) { public void testFilterToBatchDiscard(Args args) {
CompressionType compression = args.compression; Compression compression = args.compression;
byte magic = args.magic; byte magic = args.magic;
ByteBuffer buffer = ByteBuffer.allocate(2048); ByteBuffer buffer = ByteBuffer.allocate(2048);
@ -635,7 +636,7 @@ public class MemoryRecordsTest {
MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered);
List<MutableRecordBatch> batches = TestUtils.toList(filteredRecords.batches()); List<MutableRecordBatch> batches = TestUtils.toList(filteredRecords.batches());
if (compression != CompressionType.NONE || magic >= MAGIC_VALUE_V2) { if (compression.type() != CompressionType.NONE || magic >= MAGIC_VALUE_V2) {
assertEquals(2, batches.size()); assertEquals(2, batches.size());
assertEquals(0, batches.get(0).lastOffset()); assertEquals(0, batches.get(0).lastOffset());
assertEquals(5, batches.get(1).lastOffset()); assertEquals(5, batches.get(1).lastOffset());
@ -650,7 +651,7 @@ public class MemoryRecordsTest {
@ArgumentsSource(MemoryRecordsArgumentsProvider.class) @ArgumentsSource(MemoryRecordsArgumentsProvider.class)
public void testFilterToAlreadyCompactedLog(Args args) { public void testFilterToAlreadyCompactedLog(Args args) {
byte magic = args.magic; byte magic = args.magic;
CompressionType compression = args.compression; Compression compression = args.compression;
ByteBuffer buffer = ByteBuffer.allocate(2048); ByteBuffer buffer = ByteBuffer.allocate(2048);
@ -699,7 +700,7 @@ public class MemoryRecordsTest {
@ArgumentsSource(MemoryRecordsArgumentsProvider.class) @ArgumentsSource(MemoryRecordsArgumentsProvider.class)
public void testFilterToPreservesProducerInfo(Args args) { public void testFilterToPreservesProducerInfo(Args args) {
byte magic = args.magic; byte magic = args.magic;
CompressionType compression = args.compression; Compression compression = args.compression;
ByteBuffer buffer = ByteBuffer.allocate(2048); ByteBuffer buffer = ByteBuffer.allocate(2048);
// non-idempotent, non-transactional // non-idempotent, non-transactional
@ -802,7 +803,7 @@ public class MemoryRecordsTest {
@ArgumentsSource(MemoryRecordsArgumentsProvider.class) @ArgumentsSource(MemoryRecordsArgumentsProvider.class)
public void testFilterToWithUndersizedBuffer(Args args) { public void testFilterToWithUndersizedBuffer(Args args) {
byte magic = args.magic; byte magic = args.magic;
CompressionType compression = args.compression; Compression compression = args.compression;
ByteBuffer buffer = ByteBuffer.allocate(1024); ByteBuffer buffer = ByteBuffer.allocate(1024);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME, 0L); MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME, 0L);
@ -856,7 +857,7 @@ public class MemoryRecordsTest {
@ArgumentsSource(MemoryRecordsArgumentsProvider.class) @ArgumentsSource(MemoryRecordsArgumentsProvider.class)
public void testFilterTo(Args args) { public void testFilterTo(Args args) {
byte magic = args.magic; byte magic = args.magic;
CompressionType compression = args.compression; Compression compression = args.compression;
ByteBuffer buffer = ByteBuffer.allocate(2048); ByteBuffer buffer = ByteBuffer.allocate(2048);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME, 0L); MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME, 0L);
@ -893,7 +894,7 @@ public class MemoryRecordsTest {
assertEquals(filtered.limit(), result.bytesRetained()); assertEquals(filtered.limit(), result.bytesRetained());
if (magic > RecordBatch.MAGIC_VALUE_V0) { if (magic > RecordBatch.MAGIC_VALUE_V0) {
assertEquals(20L, result.maxTimestamp()); assertEquals(20L, result.maxTimestamp());
if (compression == CompressionType.NONE && magic < RecordBatch.MAGIC_VALUE_V2) if (compression.type() == CompressionType.NONE && magic < RecordBatch.MAGIC_VALUE_V2)
assertEquals(4L, result.shallowOffsetOfMaxTimestamp()); assertEquals(4L, result.shallowOffsetOfMaxTimestamp());
else else
assertEquals(5L, result.shallowOffsetOfMaxTimestamp()); assertEquals(5L, result.shallowOffsetOfMaxTimestamp());
@ -906,7 +907,7 @@ public class MemoryRecordsTest {
final List<Long> expectedStartOffsets; final List<Long> expectedStartOffsets;
final List<Long> expectedMaxTimestamps; final List<Long> expectedMaxTimestamps;
if (magic < RecordBatch.MAGIC_VALUE_V2 && compression == CompressionType.NONE) { if (magic < RecordBatch.MAGIC_VALUE_V2 && compression.type() == CompressionType.NONE) {
expectedEndOffsets = asList(1L, 4L, 5L, 6L); expectedEndOffsets = asList(1L, 4L, 5L, 6L);
expectedStartOffsets = asList(1L, 4L, 5L, 6L); expectedStartOffsets = asList(1L, 4L, 5L, 6L);
expectedMaxTimestamps = asList(11L, 20L, 15L, 16L); expectedMaxTimestamps = asList(11L, 20L, 15L, 16L);
@ -927,7 +928,7 @@ public class MemoryRecordsTest {
assertEquals(expectedStartOffsets.get(i).longValue(), batch.baseOffset()); assertEquals(expectedStartOffsets.get(i).longValue(), batch.baseOffset());
assertEquals(expectedEndOffsets.get(i).longValue(), batch.lastOffset()); assertEquals(expectedEndOffsets.get(i).longValue(), batch.lastOffset());
assertEquals(magic, batch.magic()); assertEquals(magic, batch.magic());
assertEquals(compression, batch.compressionType()); assertEquals(compression.type(), batch.compressionType());
if (magic >= RecordBatch.MAGIC_VALUE_V1) { if (magic >= RecordBatch.MAGIC_VALUE_V1) {
assertEquals(expectedMaxTimestamps.get(i).longValue(), batch.maxTimestamp()); assertEquals(expectedMaxTimestamps.get(i).longValue(), batch.maxTimestamp());
assertEquals(TimestampType.CREATE_TIME, batch.timestampType()); assertEquals(TimestampType.CREATE_TIME, batch.timestampType());
@ -973,7 +974,7 @@ public class MemoryRecordsTest {
@ArgumentsSource(MemoryRecordsArgumentsProvider.class) @ArgumentsSource(MemoryRecordsArgumentsProvider.class)
public void testFilterToPreservesLogAppendTime(Args args) { public void testFilterToPreservesLogAppendTime(Args args) {
byte magic = args.magic; byte magic = args.magic;
CompressionType compression = args.compression; Compression compression = args.compression;
long pid = args.pid; long pid = args.pid;
short epoch = args.epoch; short epoch = args.epoch;
int firstSequence = args.firstSequence; int firstSequence = args.firstSequence;
@ -1008,10 +1009,10 @@ public class MemoryRecordsTest {
MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered);
List<MutableRecordBatch> batches = TestUtils.toList(filteredRecords.batches()); List<MutableRecordBatch> batches = TestUtils.toList(filteredRecords.batches());
assertEquals(magic < RecordBatch.MAGIC_VALUE_V2 && compression == CompressionType.NONE ? 3 : 2, batches.size()); assertEquals(magic < RecordBatch.MAGIC_VALUE_V2 && compression.type() == CompressionType.NONE ? 3 : 2, batches.size());
for (RecordBatch batch : batches) { for (RecordBatch batch : batches) {
assertEquals(compression, batch.compressionType()); assertEquals(compression.type(), batch.compressionType());
if (magic > RecordBatch.MAGIC_VALUE_V0) { if (magic > RecordBatch.MAGIC_VALUE_V0) {
assertEquals(TimestampType.LOG_APPEND_TIME, batch.timestampType()); assertEquals(TimestampType.LOG_APPEND_TIME, batch.timestampType());
assertEquals(logAppendTime, batch.maxTimestamp()); assertEquals(logAppendTime, batch.maxTimestamp());
@ -1054,7 +1055,7 @@ public class MemoryRecordsTest {
@ParameterizedTest @ParameterizedTest
@ArgumentsSource(MemoryRecordsArgumentsProvider.class) @ArgumentsSource(MemoryRecordsArgumentsProvider.class)
public void testWithRecords(Args args) { public void testWithRecords(Args args) {
CompressionType compression = args.compression; Compression compression = args.compression;
byte magic = args.magic; byte magic = args.magic;
MemoryRecords memoryRecords = MemoryRecords.withRecords(magic, compression, MemoryRecords memoryRecords = MemoryRecords.withRecords(magic, compression,
new SimpleRecord(10L, "key1".getBytes(), "value1".getBytes())); new SimpleRecord(10L, "key1".getBytes(), "value1".getBytes()));
@ -1065,7 +1066,7 @@ public class MemoryRecordsTest {
@Test @Test
public void testUnsupportedCompress() { public void testUnsupportedCompress() {
BiFunction<Byte, CompressionType, MemoryRecords> builderBiFunction = (magic, compressionType) -> BiFunction<Byte, CompressionType, MemoryRecords> builderBiFunction = (magic, compressionType) ->
MemoryRecords.withRecords(magic, compressionType, new SimpleRecord(10L, "key1".getBytes(), "value1".getBytes())); MemoryRecords.withRecords(magic, Compression.of(compressionType).build(), new SimpleRecord(10L, "key1".getBytes(), "value1".getBytes()));
Arrays.asList(MAGIC_VALUE_V0, MAGIC_VALUE_V1).forEach(magic -> { Arrays.asList(MAGIC_VALUE_V0, MAGIC_VALUE_V1).forEach(magic -> {
Exception e = assertThrows(IllegalArgumentException.class, () -> builderBiFunction.apply(magic, CompressionType.ZSTD)); Exception e = assertThrows(IllegalArgumentException.class, () -> builderBiFunction.apply(magic, CompressionType.ZSTD));

View File

@ -16,6 +16,7 @@
*/ */
package org.apache.kafka.common.record; package org.apache.kafka.common.record;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.header.Header; import org.apache.kafka.common.header.Header;
import org.apache.kafka.common.header.internals.RecordHeader; import org.apache.kafka.common.header.internals.RecordHeader;
import org.apache.kafka.test.TestUtils; import org.apache.kafka.test.TestUtils;
@ -51,9 +52,9 @@ public class RemoteLogInputStreamTest {
private static class Args { private static class Args {
private final byte magic; private final byte magic;
private final CompressionType compression; private final Compression compression;
public Args(byte magic, CompressionType compression) { public Args(byte magic, Compression compression) {
this.magic = magic; this.magic = magic;
this.compression = compression; this.compression = compression;
} }
@ -71,7 +72,7 @@ public class RemoteLogInputStreamTest {
List<Arguments> values = new ArrayList<>(); List<Arguments> values = new ArrayList<>();
for (byte magic : asList(MAGIC_VALUE_V0, MAGIC_VALUE_V1, MAGIC_VALUE_V2)) { for (byte magic : asList(MAGIC_VALUE_V0, MAGIC_VALUE_V1, MAGIC_VALUE_V2)) {
for (CompressionType type : CompressionType.values()) { for (CompressionType type : CompressionType.values()) {
values.add(Arguments.of(new Args(magic, type))); values.add(Arguments.of(new Args(magic, Compression.of(type).build())));
} }
} }
return values.stream(); return values.stream();
@ -82,8 +83,8 @@ public class RemoteLogInputStreamTest {
@ArgumentsSource(RemoteLogInputStreamArgsProvider.class) @ArgumentsSource(RemoteLogInputStreamArgsProvider.class)
public void testSimpleBatchIteration(Args args) throws IOException { public void testSimpleBatchIteration(Args args) throws IOException {
byte magic = args.magic; byte magic = args.magic;
CompressionType compression = args.compression; Compression compression = args.compression;
if (compression == CompressionType.ZSTD && magic < MAGIC_VALUE_V2) if (compression.type() == CompressionType.ZSTD && magic < MAGIC_VALUE_V2)
return; return;
SimpleRecord firstBatchRecord = new SimpleRecord(3241324L, "a".getBytes(), "foo".getBytes()); SimpleRecord firstBatchRecord = new SimpleRecord(3241324L, "a".getBytes(), "foo".getBytes());
@ -115,11 +116,11 @@ public class RemoteLogInputStreamTest {
@ArgumentsSource(RemoteLogInputStreamArgsProvider.class) @ArgumentsSource(RemoteLogInputStreamArgsProvider.class)
public void testBatchIterationWithMultipleRecordsPerBatch(Args args) throws IOException { public void testBatchIterationWithMultipleRecordsPerBatch(Args args) throws IOException {
byte magic = args.magic; byte magic = args.magic;
CompressionType compression = args.compression; Compression compression = args.compression;
if (magic < MAGIC_VALUE_V2 && compression == CompressionType.NONE) if (magic < MAGIC_VALUE_V2 && compression.type() == CompressionType.NONE)
return; return;
if (compression == CompressionType.ZSTD && magic < MAGIC_VALUE_V2) if (compression.type() == CompressionType.ZSTD && magic < MAGIC_VALUE_V2)
return; return;
SimpleRecord[] firstBatchRecords = new SimpleRecord[]{ SimpleRecord[] firstBatchRecords = new SimpleRecord[]{
@ -159,7 +160,7 @@ public class RemoteLogInputStreamTest {
@ArgumentsSource(RemoteLogInputStreamArgsProvider.class) @ArgumentsSource(RemoteLogInputStreamArgsProvider.class)
public void testBatchIterationV2(Args args) throws IOException { public void testBatchIterationV2(Args args) throws IOException {
byte magic = args.magic; byte magic = args.magic;
CompressionType compression = args.compression; Compression compression = args.compression;
if (magic != MAGIC_VALUE_V2) if (magic != MAGIC_VALUE_V2)
return; return;
@ -213,8 +214,8 @@ public class RemoteLogInputStreamTest {
@ArgumentsSource(RemoteLogInputStreamArgsProvider.class) @ArgumentsSource(RemoteLogInputStreamArgsProvider.class)
public void testBatchIterationIncompleteBatch(Args args) throws IOException { public void testBatchIterationIncompleteBatch(Args args) throws IOException {
byte magic = args.magic; byte magic = args.magic;
CompressionType compression = args.compression; Compression compression = args.compression;
if (compression == CompressionType.ZSTD && magic < MAGIC_VALUE_V2) if (compression.type() == CompressionType.ZSTD && magic < MAGIC_VALUE_V2)
return; return;
try (FileRecords fileRecords = FileRecords.open(tempFile())) { try (FileRecords fileRecords = FileRecords.open(tempFile())) {
@ -259,9 +260,9 @@ public class RemoteLogInputStreamTest {
long maxTimestamp, long maxTimestamp,
SimpleRecord... records) { SimpleRecord... records) {
byte magic = args.magic; byte magic = args.magic;
CompressionType compression = args.compression; Compression compression = args.compression;
assertEquals(magic, batch.magic()); assertEquals(magic, batch.magic());
assertEquals(compression, batch.compressionType()); assertEquals(compression.type(), batch.compressionType());
if (magic == MAGIC_VALUE_V0) { if (magic == MAGIC_VALUE_V0) {
assertEquals(NO_TIMESTAMP_TYPE, batch.timestampType()); assertEquals(NO_TIMESTAMP_TYPE, batch.timestampType());

View File

@ -17,6 +17,7 @@
package org.apache.kafka.common.record; package org.apache.kafka.common.record;
import org.apache.kafka.common.InvalidRecordException; import org.apache.kafka.common.InvalidRecordException;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.errors.CorruptRecordException; import org.apache.kafka.common.errors.CorruptRecordException;
import org.apache.kafka.common.utils.ByteBufferOutputStream; import org.apache.kafka.common.utils.ByteBufferOutputStream;
import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.utils.Utils;
@ -47,7 +48,8 @@ public class SimpleLegacyRecordTest {
@Test @Test
public void testCompressedIterationWithEmptyRecords() throws Exception { public void testCompressedIterationWithEmptyRecords() throws Exception {
ByteBuffer emptyCompressedValue = ByteBuffer.allocate(64); ByteBuffer emptyCompressedValue = ByteBuffer.allocate(64);
OutputStream gzipOutput = CompressionType.GZIP.wrapForOutput(new ByteBufferOutputStream(emptyCompressedValue), Compression gzip = Compression.gzip().build();
OutputStream gzipOutput = gzip.wrapForOutput(new ByteBufferOutputStream(emptyCompressedValue),
RecordBatch.MAGIC_VALUE_V1); RecordBatch.MAGIC_VALUE_V1);
gzipOutput.close(); gzipOutput.close();
emptyCompressedValue.flip(); emptyCompressedValue.flip();

View File

@ -17,6 +17,7 @@
package org.apache.kafka.common.record; package org.apache.kafka.common.record;
import org.apache.kafka.common.compress.Compression;
import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
@ -71,7 +72,7 @@ public class UnalignedFileRecordsTest {
FileRecords fileRecords = FileRecords.open(tempFile()); FileRecords fileRecords = FileRecords.open(tempFile());
for (byte[] value : values) { for (byte[] value : values) {
fileRecords.append(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord(value))); fileRecords.append(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord(value)));
} }
return fileRecords; return fileRecords;

View File

@ -18,10 +18,10 @@
package org.apache.kafka.common.requests; package org.apache.kafka.common.requests;
import org.apache.kafka.common.InvalidRecordException; import org.apache.kafka.common.InvalidRecordException;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.errors.UnsupportedCompressionTypeException; import org.apache.kafka.common.errors.UnsupportedCompressionTypeException;
import org.apache.kafka.common.message.ProduceRequestData; import org.apache.kafka.common.message.ProduceRequestData;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.MemoryRecordsBuilder; import org.apache.kafka.common.record.MemoryRecordsBuilder;
import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.record.RecordBatch;
@ -48,7 +48,7 @@ public class ProduceRequestTest {
@Test @Test
public void shouldBeFlaggedAsTransactionalWhenTransactionalRecords() { public void shouldBeFlaggedAsTransactionalWhenTransactionalRecords() {
final MemoryRecords memoryRecords = MemoryRecords.withTransactionalRecords(0, CompressionType.NONE, 1L, final MemoryRecords memoryRecords = MemoryRecords.withTransactionalRecords(0, Compression.NONE, 1L,
(short) 1, 1, 1, simpleRecord); (short) 1, 1, 1, simpleRecord);
final ProduceRequest request = ProduceRequest.forCurrentMagic(new ProduceRequestData() final ProduceRequest request = ProduceRequest.forCurrentMagic(new ProduceRequestData()
@ -78,7 +78,7 @@ public class ProduceRequestTest {
@Test @Test
public void shouldBeFlaggedAsIdempotentWhenIdempotentRecords() { public void shouldBeFlaggedAsIdempotentWhenIdempotentRecords() {
final MemoryRecords memoryRecords = MemoryRecords.withIdempotentRecords(1, CompressionType.NONE, 1L, final MemoryRecords memoryRecords = MemoryRecords.withIdempotentRecords(1, Compression.NONE, 1L,
(short) 1, 1, 1, simpleRecord); (short) 1, 1, 1, simpleRecord);
final ProduceRequest request = ProduceRequest.forCurrentMagic(new ProduceRequestData() final ProduceRequest request = ProduceRequest.forCurrentMagic(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
@ -96,7 +96,7 @@ public class ProduceRequestTest {
@Test @Test
public void testBuildWithOldMessageFormat() { public void testBuildWithOldMessageFormat() {
ByteBuffer buffer = ByteBuffer.allocate(256); ByteBuffer buffer = ByteBuffer.allocate(256);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V1, CompressionType.NONE, MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V1, Compression.NONE,
TimestampType.CREATE_TIME, 0L); TimestampType.CREATE_TIME, 0L);
builder.append(10L, null, "a".getBytes()); builder.append(10L, null, "a".getBytes());
ProduceRequest.Builder requestBuilder = ProduceRequest.forMagic(RecordBatch.MAGIC_VALUE_V1, ProduceRequest.Builder requestBuilder = ProduceRequest.forMagic(RecordBatch.MAGIC_VALUE_V1,
@ -115,7 +115,7 @@ public class ProduceRequestTest {
public void testBuildWithCurrentMessageFormat() { public void testBuildWithCurrentMessageFormat() {
ByteBuffer buffer = ByteBuffer.allocate(256); ByteBuffer buffer = ByteBuffer.allocate(256);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE,
CompressionType.NONE, TimestampType.CREATE_TIME, 0L); Compression.NONE, TimestampType.CREATE_TIME, 0L);
builder.append(10L, null, "a".getBytes()); builder.append(10L, null, "a".getBytes());
ProduceRequest.Builder requestBuilder = ProduceRequest.forMagic(RecordBatch.CURRENT_MAGIC_VALUE, ProduceRequest.Builder requestBuilder = ProduceRequest.forMagic(RecordBatch.CURRENT_MAGIC_VALUE,
new ProduceRequestData() new ProduceRequestData()
@ -132,11 +132,11 @@ public class ProduceRequestTest {
@Test @Test
public void testV3AndAboveShouldContainOnlyOneRecordBatch() { public void testV3AndAboveShouldContainOnlyOneRecordBatch() {
ByteBuffer buffer = ByteBuffer.allocate(256); ByteBuffer buffer = ByteBuffer.allocate(256);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, CompressionType.NONE, TimestampType.CREATE_TIME, 0L); MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, Compression.NONE, TimestampType.CREATE_TIME, 0L);
builder.append(10L, null, "a".getBytes()); builder.append(10L, null, "a".getBytes());
builder.close(); builder.close();
builder = MemoryRecords.builder(buffer, CompressionType.NONE, TimestampType.CREATE_TIME, 1L); builder = MemoryRecords.builder(buffer, Compression.NONE, TimestampType.CREATE_TIME, 1L);
builder.append(11L, "1".getBytes(), "b".getBytes()); builder.append(11L, "1".getBytes(), "b".getBytes());
builder.append(12L, null, "c".getBytes()); builder.append(12L, null, "c".getBytes());
builder.close(); builder.close();
@ -174,7 +174,7 @@ public class ProduceRequestTest {
@Test @Test
public void testV3AndAboveCannotUseMagicV0() { public void testV3AndAboveCannotUseMagicV0() {
ByteBuffer buffer = ByteBuffer.allocate(256); ByteBuffer buffer = ByteBuffer.allocate(256);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V0, CompressionType.NONE, MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V0, Compression.NONE,
TimestampType.NO_TIMESTAMP_TYPE, 0L); TimestampType.NO_TIMESTAMP_TYPE, 0L);
builder.append(10L, null, "a".getBytes()); builder.append(10L, null, "a".getBytes());
@ -194,7 +194,7 @@ public class ProduceRequestTest {
@Test @Test
public void testV3AndAboveCannotUseMagicV1() { public void testV3AndAboveCannotUseMagicV1() {
ByteBuffer buffer = ByteBuffer.allocate(256); ByteBuffer buffer = ByteBuffer.allocate(256);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V1, CompressionType.NONE, MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V1, Compression.NONE,
TimestampType.CREATE_TIME, 0L); TimestampType.CREATE_TIME, 0L);
builder.append(10L, null, "a".getBytes()); builder.append(10L, null, "a".getBytes());
@ -214,7 +214,7 @@ public class ProduceRequestTest {
@Test @Test
public void testV6AndBelowCannotUseZStdCompression() { public void testV6AndBelowCannotUseZStdCompression() {
ByteBuffer buffer = ByteBuffer.allocate(256); ByteBuffer buffer = ByteBuffer.allocate(256);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.ZSTD, MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, Compression.zstd().build(),
TimestampType.CREATE_TIME, 0L); TimestampType.CREATE_TIME, 0L);
builder.append(10L, null, "a".getBytes()); builder.append(10L, null, "a".getBytes());
@ -245,9 +245,9 @@ public class ProduceRequestTest {
final short producerEpoch = 5; final short producerEpoch = 5;
final int sequence = 10; final int sequence = 10;
final MemoryRecords nonTxnRecords = MemoryRecords.withRecords(CompressionType.NONE, final MemoryRecords nonTxnRecords = MemoryRecords.withRecords(Compression.NONE,
new SimpleRecord("foo".getBytes())); new SimpleRecord("foo".getBytes()));
final MemoryRecords txnRecords = MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, final MemoryRecords txnRecords = MemoryRecords.withTransactionalRecords(Compression.NONE, producerId,
producerEpoch, sequence, new SimpleRecord("bar".getBytes())); producerEpoch, sequence, new SimpleRecord("bar".getBytes()));
ProduceRequest.Builder builder = ProduceRequest.forMagic(RecordBatch.CURRENT_MAGIC_VALUE, ProduceRequest.Builder builder = ProduceRequest.forMagic(RecordBatch.CURRENT_MAGIC_VALUE,
@ -271,9 +271,9 @@ public class ProduceRequestTest {
final short producerEpoch = 5; final short producerEpoch = 5;
final int sequence = 10; final int sequence = 10;
final MemoryRecords nonIdempotentRecords = MemoryRecords.withRecords(CompressionType.NONE, final MemoryRecords nonIdempotentRecords = MemoryRecords.withRecords(Compression.NONE,
new SimpleRecord("foo".getBytes())); new SimpleRecord("foo".getBytes()));
final MemoryRecords idempotentRecords = MemoryRecords.withIdempotentRecords(CompressionType.NONE, producerId, final MemoryRecords idempotentRecords = MemoryRecords.withIdempotentRecords(Compression.NONE, producerId,
producerEpoch, sequence, new SimpleRecord("bar".getBytes())); producerEpoch, sequence, new SimpleRecord("bar".getBytes()));
ProduceRequest.Builder builder = ProduceRequest.forMagic(RecordVersion.current().value, ProduceRequest.Builder builder = ProduceRequest.forMagic(RecordVersion.current().value,
@ -305,7 +305,7 @@ public class ProduceRequestTest {
.setName("topic") .setName("topic")
.setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData() .setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData()
.setIndex(1) .setIndex(1)
.setRecords(MemoryRecords.withRecords(CompressionType.NONE, simpleRecord))))) .setRecords(MemoryRecords.withRecords(Compression.NONE, simpleRecord)))))
.iterator())) .iterator()))
.setAcks((short) -1) .setAcks((short) -1)
.setTimeoutMs(10)).build(); .setTimeoutMs(10)).build();

View File

@ -29,6 +29,7 @@ import org.apache.kafka.common.acl.AclBinding;
import org.apache.kafka.common.acl.AclBindingFilter; import org.apache.kafka.common.acl.AclBindingFilter;
import org.apache.kafka.common.acl.AclOperation; import org.apache.kafka.common.acl.AclOperation;
import org.apache.kafka.common.acl.AclPermissionType; import org.apache.kafka.common.acl.AclPermissionType;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.config.ConfigResource;
import org.apache.kafka.common.errors.NotCoordinatorException; import org.apache.kafka.common.errors.NotCoordinatorException;
import org.apache.kafka.common.errors.NotEnoughReplicasException; import org.apache.kafka.common.errors.NotEnoughReplicasException;
@ -448,9 +449,9 @@ public class RequestResponseTest {
TopicPartition tp0 = new TopicPartition("test", 0); TopicPartition tp0 = new TopicPartition("test", 0);
TopicPartition tp1 = new TopicPartition("test", 1); TopicPartition tp1 = new TopicPartition("test", 1);
MemoryRecords records0 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, MemoryRecords records0 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2,
CompressionType.NONE, new SimpleRecord("woot".getBytes())); Compression.NONE, new SimpleRecord("woot".getBytes()));
MemoryRecords records1 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, MemoryRecords records1 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2,
CompressionType.NONE, new SimpleRecord("woot".getBytes()), new SimpleRecord("woot".getBytes())); Compression.NONE, new SimpleRecord("woot".getBytes()), new SimpleRecord("woot".getBytes()));
ProduceRequest request = ProduceRequest.forMagic(RecordBatch.MAGIC_VALUE_V2, ProduceRequest request = ProduceRequest.forMagic(RecordBatch.MAGIC_VALUE_V2,
new ProduceRequestData() new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(asList( .setTopicData(new ProduceRequestData.TopicProduceDataCollection(asList(
@ -1394,7 +1395,7 @@ public class RequestResponseTest {
.setEpoch(0)) .setEpoch(0))
.setPosition(234L) .setPosition(234L)
.setSize(345L) .setSize(345L)
.setUnalignedRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("blah".getBytes()))))))) .setUnalignedRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("blah".getBytes())))))))
.setThrottleTimeMs(123); .setThrottleTimeMs(123);
return new FetchSnapshotResponse(data); return new FetchSnapshotResponse(data);
} }
@ -1869,7 +1870,7 @@ public class RequestResponseTest {
LinkedHashMap<TopicIdPartition, FetchResponseData.PartitionData> responseData = new LinkedHashMap<>(); LinkedHashMap<TopicIdPartition, FetchResponseData.PartitionData> responseData = new LinkedHashMap<>();
Map<String, Uuid> topicIds = new HashMap<>(); Map<String, Uuid> topicIds = new HashMap<>();
topicIds.put("test", Uuid.randomUuid()); topicIds.put("test", Uuid.randomUuid());
MemoryRecords records = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("blah".getBytes())); MemoryRecords records = MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("blah".getBytes()));
responseData.put(new TopicIdPartition(topicIds.get("test"), new TopicPartition("test", 0)), new FetchResponseData.PartitionData() responseData.put(new TopicIdPartition(topicIds.get("test"), new TopicPartition("test", 0)), new FetchResponseData.PartitionData()
.setPartitionIndex(0) .setPartitionIndex(0)
.setHighWatermark(1000000) .setHighWatermark(1000000)
@ -1889,7 +1890,7 @@ public class RequestResponseTest {
private FetchResponse createFetchResponse(boolean includeAborted) { private FetchResponse createFetchResponse(boolean includeAborted) {
LinkedHashMap<TopicIdPartition, FetchResponseData.PartitionData> responseData = new LinkedHashMap<>(); LinkedHashMap<TopicIdPartition, FetchResponseData.PartitionData> responseData = new LinkedHashMap<>();
Uuid topicId = Uuid.randomUuid(); Uuid topicId = Uuid.randomUuid();
MemoryRecords records = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("blah".getBytes())); MemoryRecords records = MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("blah".getBytes()));
responseData.put(new TopicIdPartition(topicId, new TopicPartition("test", 0)), new FetchResponseData.PartitionData() responseData.put(new TopicIdPartition(topicId, new TopicPartition("test", 0)), new FetchResponseData.PartitionData()
.setPartitionIndex(0) .setPartitionIndex(0)
.setHighWatermark(1000000) .setHighWatermark(1000000)
@ -1919,7 +1920,7 @@ public class RequestResponseTest {
data.setErrorCode(Errors.NONE.code()) data.setErrorCode(Errors.NONE.code())
.setSessionId(123); .setSessionId(123);
} }
MemoryRecords records = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("blah".getBytes())); MemoryRecords records = MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("blah".getBytes()));
FetchResponseData.PartitionData partition = new FetchResponseData.PartitionData() FetchResponseData.PartitionData partition = new FetchResponseData.PartitionData()
.setPartitionIndex(0) .setPartitionIndex(0)
.setErrorCode(Errors.NONE.code()) .setErrorCode(Errors.NONE.code())
@ -2336,7 +2337,7 @@ public class RequestResponseTest {
private ProduceRequest createProduceRequest(short version) { private ProduceRequest createProduceRequest(short version) {
if (version < 2) { if (version < 2) {
MemoryRecords records = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("blah".getBytes())); MemoryRecords records = MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("blah".getBytes()));
ProduceRequestData data = new ProduceRequestData() ProduceRequestData data = new ProduceRequestData()
.setAcks((short) -1) .setAcks((short) -1)
.setTimeoutMs(123) .setTimeoutMs(123)
@ -2349,7 +2350,7 @@ public class RequestResponseTest {
return new ProduceRequest.Builder(version, version, data).build(version); return new ProduceRequest.Builder(version, version, data).build(version);
} }
byte magic = version == 2 ? RecordBatch.MAGIC_VALUE_V1 : RecordBatch.MAGIC_VALUE_V2; byte magic = version == 2 ? RecordBatch.MAGIC_VALUE_V1 : RecordBatch.MAGIC_VALUE_V2;
MemoryRecords records = MemoryRecords.withRecords(magic, CompressionType.NONE, new SimpleRecord("woot".getBytes())); MemoryRecords records = MemoryRecords.withRecords(magic, Compression.NONE, new SimpleRecord("woot".getBytes()));
return ProduceRequest.forMagic(magic, return ProduceRequest.forMagic(magic,
new ProduceRequestData() new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(singletonList( .setTopicData(new ProduceRequestData.TopicProduceDataCollection(singletonList(

View File

@ -32,6 +32,7 @@ import kafka.utils.CoreUtils.inLock
import kafka.utils.Implicits._ import kafka.utils.Implicits._
import kafka.utils._ import kafka.utils._
import org.apache.kafka.clients.consumer.ConsumerRecord import org.apache.kafka.clients.consumer.ConsumerRecord
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.internals.Topic
import org.apache.kafka.common.metrics.{Metrics, Sensor} import org.apache.kafka.common.metrics.{Metrics, Sensor}
import org.apache.kafka.common.metrics.stats.{Avg, Max, Meter} import org.apache.kafka.common.metrics.stats.{Avg, Max, Meter}
@ -63,7 +64,7 @@ class GroupMetadataManager(brokerId: Int,
// Visible for test. // Visible for test.
private[group] val metricsGroup: KafkaMetricsGroup = new KafkaMetricsGroup(this.getClass) private[group] val metricsGroup: KafkaMetricsGroup = new KafkaMetricsGroup(this.getClass)
private val compressionType: CompressionType = config.offsetsTopicCompressionType private val compression: Compression = Compression.of(config.offsetsTopicCompressionType).build()
private val groupMetadataCache = new Pool[String, GroupMetadata] private val groupMetadataCache = new Pool[String, GroupMetadata]
@ -258,9 +259,9 @@ class GroupMetadataManager(brokerId: Int,
val value = GroupMetadataManager.groupMetadataValue(group, groupAssignment, interBrokerProtocolVersion) val value = GroupMetadataManager.groupMetadataValue(group, groupAssignment, interBrokerProtocolVersion)
val records = { val records = {
val buffer = ByteBuffer.allocate(AbstractRecords.estimateSizeInBytes(magicValue, compressionType, val buffer = ByteBuffer.allocate(AbstractRecords.estimateSizeInBytes(magicValue, compression.`type`(),
Seq(new SimpleRecord(timestamp, key, value)).asJava)) Seq(new SimpleRecord(timestamp, key, value)).asJava))
val builder = MemoryRecords.builder(buffer, magicValue, compressionType, timestampType, 0L) val builder = MemoryRecords.builder(buffer, magicValue, compression, timestampType, 0L)
builder.append(timestamp, key, value) builder.append(timestamp, key, value)
builder.build() builder.build()
} }
@ -363,12 +364,12 @@ class GroupMetadataManager(brokerId: Int,
val value = GroupMetadataManager.offsetCommitValue(offsetAndMetadata, interBrokerProtocolVersion) val value = GroupMetadataManager.offsetCommitValue(offsetAndMetadata, interBrokerProtocolVersion)
new SimpleRecord(timestamp, key, value) new SimpleRecord(timestamp, key, value)
} }
val buffer = ByteBuffer.allocate(AbstractRecords.estimateSizeInBytes(magicValue, compressionType, records.asJava)) val buffer = ByteBuffer.allocate(AbstractRecords.estimateSizeInBytes(magicValue, compression.`type`(), records.asJava))
if (isTxnOffsetCommit && magicValue < RecordBatch.MAGIC_VALUE_V2) if (isTxnOffsetCommit && magicValue < RecordBatch.MAGIC_VALUE_V2)
throw Errors.UNSUPPORTED_FOR_MESSAGE_FORMAT.exception("Attempting to make a transaction offset commit with an invalid magic: " + magicValue) throw Errors.UNSUPPORTED_FOR_MESSAGE_FORMAT.exception("Attempting to make a transaction offset commit with an invalid magic: " + magicValue)
val builder = MemoryRecords.builder(buffer, magicValue, compressionType, timestampType, 0L, time.milliseconds(), val builder = MemoryRecords.builder(buffer, magicValue, compression, timestampType, 0L, time.milliseconds(),
producerId, producerEpoch, 0, isTxnOffsetCommit, RecordBatch.NO_PARTITION_LEADER_EPOCH) producerId, producerEpoch, 0, isTxnOffsetCommit, RecordBatch.NO_PARTITION_LEADER_EPOCH)
records.foreach(builder.append) records.foreach(builder.append)
@ -908,7 +909,7 @@ class GroupMetadataManager(brokerId: Int,
try { try {
// do not need to require acks since even if the tombstone is lost, // do not need to require acks since even if the tombstone is lost,
// it will be appended again in the next purge cycle // it will be appended again in the next purge cycle
val records = MemoryRecords.withRecords(magicValue, 0L, compressionType, timestampType, tombstones.toArray: _*) val records = MemoryRecords.withRecords(magicValue, 0L, compression, timestampType, tombstones.toArray: _*)
partition.appendRecordsToLeader(records, origin = AppendOrigin.COORDINATOR, requiredAcks = 0, partition.appendRecordsToLeader(records, origin = AppendOrigin.COORDINATOR, requiredAcks = 0,
requestLocal = requestLocal) requestLocal = requestLocal)

View File

@ -21,8 +21,9 @@ import java.nio.ByteBuffer
import java.nio.charset.StandardCharsets import java.nio.charset.StandardCharsets
import kafka.internals.generated.{TransactionLogKey, TransactionLogValue} import kafka.internals.generated.{TransactionLogKey, TransactionLogValue}
import org.apache.kafka.clients.consumer.ConsumerRecord import org.apache.kafka.clients.consumer.ConsumerRecord
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.protocol.{ByteBufferAccessor, MessageUtil} import org.apache.kafka.common.protocol.{ByteBufferAccessor, MessageUtil}
import org.apache.kafka.common.record.{CompressionType, Record, RecordBatch} import org.apache.kafka.common.record.{Record, RecordBatch}
import org.apache.kafka.common.{MessageFormatter, TopicPartition} import org.apache.kafka.common.{MessageFormatter, TopicPartition}
import scala.collection.mutable import scala.collection.mutable
@ -43,7 +44,7 @@ object TransactionLog {
// 2. compression = none // 2. compression = none
// 3. unclean leader election = disabled // 3. unclean leader election = disabled
// 4. required acks = -1 when writing // 4. required acks = -1 when writing
val EnforcedCompressionType: CompressionType = CompressionType.NONE val EnforcedCompression: Compression = Compression.NONE
val EnforcedRequiredAcks: Short = (-1).toShort val EnforcedRequiredAcks: Short = (-1).toShort
/** /**

View File

@ -165,7 +165,7 @@ class TransactionStateManager(brokerId: Int,
if (recordsBuilder == null) { if (recordsBuilder == null) {
recordsBuilder = MemoryRecords.builder( recordsBuilder = MemoryRecords.builder(
ByteBuffer.allocate(math.min(16384, maxBatchSize)), ByteBuffer.allocate(math.min(16384, maxBatchSize)),
TransactionLog.EnforcedCompressionType, TransactionLog.EnforcedCompression,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
0L, 0L,
maxBatchSize maxBatchSize
@ -621,7 +621,7 @@ class TransactionStateManager(brokerId: Int,
val valueBytes = TransactionLog.valueToBytes(newMetadata) val valueBytes = TransactionLog.valueToBytes(newMetadata)
val timestamp = time.milliseconds() val timestamp = time.milliseconds()
val records = MemoryRecords.withRecords(TransactionLog.EnforcedCompressionType, new SimpleRecord(timestamp, keyBytes, valueBytes)) val records = MemoryRecords.withRecords(TransactionLog.EnforcedCompression, new SimpleRecord(timestamp, keyBytes, valueBytes))
val topicPartition = new TopicPartition(Topic.TRANSACTION_STATE_TOPIC_NAME, partitionFor(transactionalId)) val topicPartition = new TopicPartition(Topic.TRANSACTION_STATE_TOPIC_NAME, partitionFor(transactionalId))
val recordsPerPartition = Map(topicPartition -> records) val recordsPerPartition = Map(topicPartition -> records)

View File

@ -791,7 +791,7 @@ class UnifiedLog(@volatile var logStartOffset: Long,
val offset = PrimitiveRef.ofLong(localLog.logEndOffset) val offset = PrimitiveRef.ofLong(localLog.logEndOffset)
appendInfo.setFirstOffset(offset.value) appendInfo.setFirstOffset(offset.value)
val validateAndOffsetAssignResult = try { val validateAndOffsetAssignResult = try {
val targetCompression = BrokerCompressionType.forName(config.compressionType).targetCompressionType(appendInfo.sourceCompression) val targetCompression = BrokerCompressionType.targetCompression(config.compression, appendInfo.sourceCompression())
val validator = new LogValidator(validRecords, val validator = new LogValidator(validRecords,
topicPartition, topicPartition,
time, time,

View File

@ -819,7 +819,7 @@ class KafkaApis(val requestChannel: RequestChannel,
// We will never return a logConfig when the topic is unresolved and the name is null. This is ok since we won't have any records to convert. // We will never return a logConfig when the topic is unresolved and the name is null. This is ok since we won't have any records to convert.
val logConfig = replicaManager.getLogConfig(tp.topicPartition) val logConfig = replicaManager.getLogConfig(tp.topicPartition)
if (logConfig.exists(_.compressionType == BrokerCompressionType.ZSTD.name) && versionId < 10) { if (logConfig.exists(_.compressionType == BrokerCompressionType.ZSTD) && versionId < 10) {
trace(s"Fetching messages is disabled for ZStandard compressed partition $tp. Sending unsupported version response to $clientId.") trace(s"Fetching messages is disabled for ZStandard compressed partition $tp. Sending unsupported version response to $clientId.")
FetchResponse.partitionResponse(tp, Errors.UNSUPPORTED_COMPRESSION_TYPE) FetchResponse.partitionResponse(tp, Errors.UNSUPPORTED_COMPRESSION_TYPE)
} else { } else {

View File

@ -26,6 +26,7 @@ import kafka.utils.{CoreUtils, Logging}
import kafka.utils.Implicits._ import kafka.utils.Implicits._
import org.apache.kafka.clients.CommonClientConfigs import org.apache.kafka.clients.CommonClientConfigs
import org.apache.kafka.common.Reconfigurable import org.apache.kafka.common.Reconfigurable
import org.apache.kafka.common.compress.{GzipCompression, Lz4Compression, ZstdCompression}
import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigException, ConfigResource, SaslConfigs, TopicConfig} import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigException, ConfigResource, SaslConfigs, TopicConfig}
import org.apache.kafka.common.config.ConfigDef.{ConfigKey, ValidList} import org.apache.kafka.common.config.ConfigDef.{ConfigKey, ValidList}
import org.apache.kafka.common.config.types.Password import org.apache.kafka.common.config.types.Password
@ -108,6 +109,9 @@ object KafkaConfig {
val ConnectionSetupTimeoutMaxMsProp = CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG val ConnectionSetupTimeoutMaxMsProp = CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG
val DeleteTopicEnableProp = "delete.topic.enable" val DeleteTopicEnableProp = "delete.topic.enable"
val CompressionTypeProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.COMPRESSION_TYPE_CONFIG) val CompressionTypeProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.COMPRESSION_TYPE_CONFIG)
val CompressionGzipLevelProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.COMPRESSION_GZIP_LEVEL_CONFIG)
val CompressionLz4LevelProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.COMPRESSION_LZ4_LEVEL_CONFIG)
val CompressionZstdLevelProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.COMPRESSION_ZSTD_LEVEL_CONFIG)
/************* Authorizer Configuration ***********/ /************* Authorizer Configuration ***********/
val AuthorizerClassNameProp = "authorizer.class.name" val AuthorizerClassNameProp = "authorizer.class.name"
@ -161,6 +165,9 @@ object KafkaConfig {
val CompressionTypeDoc = "Specify the final compression type for a given topic. This configuration accepts the standard compression codecs " + val CompressionTypeDoc = "Specify the final compression type for a given topic. This configuration accepts the standard compression codecs " +
"('gzip', 'snappy', 'lz4', 'zstd'). It additionally accepts 'uncompressed' which is equivalent to no compression; and " + "('gzip', 'snappy', 'lz4', 'zstd'). It additionally accepts 'uncompressed' which is equivalent to no compression; and " +
"'producer' which means retain the original compression codec set by the producer." "'producer' which means retain the original compression codec set by the producer."
val CompressionGzipLevelDoc = s"The compression level to use if $CompressionTypeProp is set to 'gzip'."
val CompressionLz4LevelDoc = s"The compression level to use if $CompressionTypeProp is set to 'lz4'."
val CompressionZstdLevelDoc = s"The compression level to use if $CompressionTypeProp is set to 'zstd'."
/************* Authorizer Configuration ***********/ /************* Authorizer Configuration ***********/
val AuthorizerClassNameDoc = s"The fully qualified name of a class that implements <code>${classOf[Authorizer].getName}</code>" + val AuthorizerClassNameDoc = s"The fully qualified name of a class that implements <code>${classOf[Authorizer].getName}</code>" +
@ -185,7 +192,6 @@ object KafkaConfig {
/** ********* Request Limit Configuration **************/ /** ********* Request Limit Configuration **************/
val MaxRequestPartitionSizeLimitDoc = "The maximum number of partitions can be served in one request." val MaxRequestPartitionSizeLimitDoc = "The maximum number of partitions can be served in one request."
/** ********* Delegation Token Configuration ****************/ /** ********* Delegation Token Configuration ****************/
val DelegationTokenSecretKeyAliasDoc = s"DEPRECATED: An alias for $DelegationTokenSecretKeyProp, which should be used instead of this config." val DelegationTokenSecretKeyAliasDoc = s"DEPRECATED: An alias for $DelegationTokenSecretKeyProp, which should be used instead of this config."
val DelegationTokenSecretKeyDoc = "Secret key to generate and verify delegation tokens. The same key must be configured across all the brokers. " + val DelegationTokenSecretKeyDoc = "Secret key to generate and verify delegation tokens. The same key must be configured across all the brokers. " +
@ -408,6 +414,9 @@ object KafkaConfig {
.define(GroupCoordinatorConfig.OFFSET_COMMIT_REQUIRED_ACKS_CONFIG, SHORT, GroupCoordinatorConfig.OFFSET_COMMIT_REQUIRED_ACKS_DEFAULT, HIGH, GroupCoordinatorConfig.OFFSET_COMMIT_REQUIRED_ACKS_DOC) .define(GroupCoordinatorConfig.OFFSET_COMMIT_REQUIRED_ACKS_CONFIG, SHORT, GroupCoordinatorConfig.OFFSET_COMMIT_REQUIRED_ACKS_DEFAULT, HIGH, GroupCoordinatorConfig.OFFSET_COMMIT_REQUIRED_ACKS_DOC)
.define(DeleteTopicEnableProp, BOOLEAN, Defaults.DELETE_TOPIC_ENABLE, HIGH, DeleteTopicEnableDoc) .define(DeleteTopicEnableProp, BOOLEAN, Defaults.DELETE_TOPIC_ENABLE, HIGH, DeleteTopicEnableDoc)
.define(CompressionTypeProp, STRING, LogConfig.DEFAULT_COMPRESSION_TYPE, in(BrokerCompressionType.names.asScala.toSeq:_*), HIGH, CompressionTypeDoc) .define(CompressionTypeProp, STRING, LogConfig.DEFAULT_COMPRESSION_TYPE, in(BrokerCompressionType.names.asScala.toSeq:_*), HIGH, CompressionTypeDoc)
.define(CompressionGzipLevelProp, INT, GzipCompression.DEFAULT_LEVEL, new GzipCompression.LevelValidator(), MEDIUM, CompressionGzipLevelDoc)
.define(CompressionLz4LevelProp, INT, Lz4Compression.DEFAULT_LEVEL, between(Lz4Compression.MIN_LEVEL, Lz4Compression.MAX_LEVEL), MEDIUM, CompressionLz4LevelDoc)
.define(CompressionZstdLevelProp, INT, ZstdCompression.DEFAULT_LEVEL, between(ZstdCompression.MIN_LEVEL, ZstdCompression.MAX_LEVEL), MEDIUM, CompressionZstdLevelDoc)
/** ********* Transaction management configuration ***********/ /** ********* Transaction management configuration ***********/
.define(TransactionStateManagerConfigs.TRANSACTIONAL_ID_EXPIRATION_MS_CONFIG, INT, TransactionStateManagerConfigs.TRANSACTIONAL_ID_EXPIRATION_MS_DEFAULT, atLeast(1), HIGH, TransactionStateManagerConfigs.TRANSACTIONAL_ID_EXPIRATION_MS_DOC) .define(TransactionStateManagerConfigs.TRANSACTIONAL_ID_EXPIRATION_MS_CONFIG, INT, TransactionStateManagerConfigs.TRANSACTIONAL_ID_EXPIRATION_MS_DEFAULT, atLeast(1), HIGH, TransactionStateManagerConfigs.TRANSACTIONAL_ID_EXPIRATION_MS_DOC)
@ -1143,6 +1152,9 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
val deleteTopicEnable = getBoolean(KafkaConfig.DeleteTopicEnableProp) val deleteTopicEnable = getBoolean(KafkaConfig.DeleteTopicEnableProp)
def compressionType = getString(KafkaConfig.CompressionTypeProp) def compressionType = getString(KafkaConfig.CompressionTypeProp)
def gzipCompressionLevel = getInt(KafkaConfig.CompressionGzipLevelProp)
def lz4CompressionLevel = getInt(KafkaConfig.CompressionLz4LevelProp)
def zstdCompressionLevel = getInt(KafkaConfig.CompressionZstdLevelProp)
/** ********* Raft Quorum Configuration *********/ /** ********* Raft Quorum Configuration *********/
val quorumVoters = getList(QuorumConfig.QUORUM_VOTERS_CONFIG) val quorumVoters = getList(QuorumConfig.QUORUM_VOTERS_CONFIG)
@ -1579,6 +1591,9 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, logCleanupPolicy) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, logCleanupPolicy)
logProps.put(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, minInSyncReplicas) logProps.put(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, minInSyncReplicas)
logProps.put(TopicConfig.COMPRESSION_TYPE_CONFIG, compressionType) logProps.put(TopicConfig.COMPRESSION_TYPE_CONFIG, compressionType)
logProps.put(TopicConfig.COMPRESSION_GZIP_LEVEL_CONFIG, gzipCompressionLevel)
logProps.put(TopicConfig.COMPRESSION_LZ4_LEVEL_CONFIG, lz4CompressionLevel)
logProps.put(TopicConfig.COMPRESSION_ZSTD_LEVEL_CONFIG, zstdCompressionLevel)
logProps.put(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, uncleanLeaderElectionEnable) logProps.put(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, uncleanLeaderElectionEnable)
logProps.put(TopicConfig.PREALLOCATE_CONFIG, logPreAllocateEnable) logProps.put(TopicConfig.PREALLOCATE_CONFIG, logPreAllocateEnable)
logProps.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, logMessageFormatVersion.version) logProps.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, logMessageFormatVersion.version)

View File

@ -27,10 +27,10 @@ import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicIdPartition;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.Uuid; import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.compress.Compression;
import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.AbstractConfig;
import org.apache.kafka.common.errors.ReplicaNotAvailableException; import org.apache.kafka.common.errors.ReplicaNotAvailableException;
import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.network.ListenerName;
import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.record.FileRecords; import org.apache.kafka.common.record.FileRecords;
import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.RecordBatch; import org.apache.kafka.common.record.RecordBatch;
@ -1210,7 +1210,7 @@ public class RemoteLogManagerTest {
private MemoryRecords records(long timestamp, private MemoryRecords records(long timestamp,
long initialOffset, long initialOffset,
int partitionLeaderEpoch) { int partitionLeaderEpoch) {
return MemoryRecords.withRecords(initialOffset, CompressionType.NONE, partitionLeaderEpoch, return MemoryRecords.withRecords(initialOffset, Compression.NONE, partitionLeaderEpoch,
new SimpleRecord(timestamp - 1, "first message".getBytes()), new SimpleRecord(timestamp - 1, "first message".getBytes()),
new SimpleRecord(timestamp + 1, "second message".getBytes()), new SimpleRecord(timestamp + 1, "second message".getBytes()),
new SimpleRecord(timestamp + 2, "third message".getBytes()) new SimpleRecord(timestamp + 2, "third message".getBytes())

View File

@ -26,6 +26,7 @@ import org.apache.kafka.clients.producer._
import org.apache.kafka.common.acl.AclOperation._ import org.apache.kafka.common.acl.AclOperation._
import org.apache.kafka.common.acl.AclPermissionType.{ALLOW, DENY} import org.apache.kafka.common.acl.AclPermissionType.{ALLOW, DENY}
import org.apache.kafka.common.acl.{AccessControlEntry, AccessControlEntryFilter, AclBindingFilter, AclOperation, AclPermissionType} import org.apache.kafka.common.acl.{AccessControlEntry, AccessControlEntryFilter, AclBindingFilter, AclOperation, AclPermissionType}
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.config.{ConfigResource, LogLevelConfig, TopicConfig} import org.apache.kafka.common.config.{ConfigResource, LogLevelConfig, TopicConfig}
import org.apache.kafka.common.errors._ import org.apache.kafka.common.errors._
import org.apache.kafka.common.internals.Topic.GROUP_METADATA_TOPIC_NAME import org.apache.kafka.common.internals.Topic.GROUP_METADATA_TOPIC_NAME
@ -42,7 +43,7 @@ import org.apache.kafka.common.message.UpdateMetadataRequestData.{UpdateMetadata
import org.apache.kafka.common.message.{AddOffsetsToTxnRequestData, AlterPartitionReassignmentsRequestData, AlterReplicaLogDirsRequestData, ControlledShutdownRequestData, CreateAclsRequestData, CreatePartitionsRequestData, CreateTopicsRequestData, DeleteAclsRequestData, DeleteGroupsRequestData, DeleteRecordsRequestData, DeleteTopicsRequestData, DescribeClusterRequestData, DescribeConfigsRequestData, DescribeGroupsRequestData, DescribeLogDirsRequestData, DescribeProducersRequestData, DescribeTransactionsRequestData, FindCoordinatorRequestData, HeartbeatRequestData, IncrementalAlterConfigsRequestData, JoinGroupRequestData, ListPartitionReassignmentsRequestData, ListTransactionsRequestData, MetadataRequestData, OffsetCommitRequestData, ProduceRequestData, SyncGroupRequestData, WriteTxnMarkersRequestData} import org.apache.kafka.common.message.{AddOffsetsToTxnRequestData, AlterPartitionReassignmentsRequestData, AlterReplicaLogDirsRequestData, ControlledShutdownRequestData, CreateAclsRequestData, CreatePartitionsRequestData, CreateTopicsRequestData, DeleteAclsRequestData, DeleteGroupsRequestData, DeleteRecordsRequestData, DeleteTopicsRequestData, DescribeClusterRequestData, DescribeConfigsRequestData, DescribeGroupsRequestData, DescribeLogDirsRequestData, DescribeProducersRequestData, DescribeTransactionsRequestData, FindCoordinatorRequestData, HeartbeatRequestData, IncrementalAlterConfigsRequestData, JoinGroupRequestData, ListPartitionReassignmentsRequestData, ListTransactionsRequestData, MetadataRequestData, OffsetCommitRequestData, ProduceRequestData, SyncGroupRequestData, WriteTxnMarkersRequestData}
import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch, SimpleRecord} import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, SimpleRecord}
import org.apache.kafka.common.requests.OffsetFetchResponse.PartitionData import org.apache.kafka.common.requests.OffsetFetchResponse.PartitionData
import org.apache.kafka.common.requests._ import org.apache.kafka.common.requests._
import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED} import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED}
@ -257,7 +258,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
.setName(tp.topic).setPartitionData(Collections.singletonList( .setName(tp.topic).setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData() new ProduceRequestData.PartitionProduceData()
.setIndex(tp.partition) .setIndex(tp.partition)
.setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes)))))) .setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("test".getBytes))))))
.iterator)) .iterator))
.setAcks(1.toShort) .setAcks(1.toShort)
.setTimeoutMs(5000)) .setTimeoutMs(5000))

View File

@ -21,11 +21,12 @@ package kafka.network
import kafka.server.BaseRequestTest import kafka.server.BaseRequestTest
import kafka.utils.TestUtils import kafka.utils.TestUtils
import org.apache.kafka.clients.admin.{Admin, AlterClientQuotasResult} import org.apache.kafka.clients.admin.{Admin, AlterClientQuotasResult}
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.message.ProduceRequestData import org.apache.kafka.common.message.ProduceRequestData
import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.quota.{ClientQuotaAlteration, ClientQuotaEntity} import org.apache.kafka.common.quota.{ClientQuotaAlteration, ClientQuotaEntity}
import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord} import org.apache.kafka.common.record.{MemoryRecords, SimpleRecord}
import org.apache.kafka.common.requests.{ProduceRequest, ProduceResponse} import org.apache.kafka.common.requests.{ProduceRequest, ProduceResponse}
import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.{KafkaException, requests} import org.apache.kafka.common.{KafkaException, requests}
@ -308,7 +309,7 @@ class DynamicConnectionQuotaTest extends BaseRequestTest {
.setName(topic) .setName(topic)
.setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData() .setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData()
.setIndex(0) .setIndex(0)
.setRecords(MemoryRecords.withRecords(CompressionType.NONE, .setRecords(MemoryRecords.withRecords(Compression.NONE,
new SimpleRecord(System.currentTimeMillis(), "key".getBytes, "value".getBytes)))))) new SimpleRecord(System.currentTimeMillis(), "key".getBytes, "value".getBytes))))))
.iterator)) .iterator))
.setAcks((-1).toShort) .setAcks((-1).toShort)

View File

@ -681,7 +681,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
val log2 = servers.head.logManager.getLog(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0)) val log2 = servers.head.logManager.getLog(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0))
.getOrElse(throw new IllegalStateException("Log not found")) .getOrElse(throw new IllegalStateException("Log not found"))
assertFalse(log2.config.delete, "Overridden clean up policy should not be updated") assertFalse(log2.config.delete, "Overridden clean up policy should not be updated")
assertEquals(BrokerCompressionType.PRODUCER.name, log2.config.compressionType) assertEquals(BrokerCompressionType.PRODUCER, log2.config.compressionType)
// Verify that we can alter subset of log configs // Verify that we can alter subset of log configs
props.clear() props.clear()

View File

@ -19,10 +19,11 @@ package kafka.raft
import kafka.log.UnifiedLog import kafka.log.UnifiedLog
import kafka.server.{KafkaConfig, KafkaRaftServer} import kafka.server.{KafkaConfig, KafkaRaftServer}
import kafka.utils.TestUtils import kafka.utils.TestUtils
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.errors.{InvalidConfigurationException, RecordTooLargeException} import org.apache.kafka.common.errors.{InvalidConfigurationException, RecordTooLargeException}
import org.apache.kafka.common.protocol import org.apache.kafka.common.protocol
import org.apache.kafka.common.protocol.{ObjectSerializationCache, Writable} import org.apache.kafka.common.protocol.{ObjectSerializationCache, Writable}
import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord} import org.apache.kafka.common.record.{MemoryRecords, SimpleRecord}
import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.utils.Utils
import org.apache.kafka.raft._ import org.apache.kafka.raft._
import org.apache.kafka.raft.internals.BatchBuilder import org.apache.kafka.raft.internals.BatchBuilder
@ -89,7 +90,7 @@ final class KafkaMetadataLogTest {
val initialOffset = log.endOffset().offset val initialOffset = log.endOffset().offset
log.appendAsLeader( log.appendAsLeader(
MemoryRecords.withRecords(initialOffset, CompressionType.NONE, currentEpoch, recordFoo), MemoryRecords.withRecords(initialOffset, Compression.NONE, currentEpoch, recordFoo),
currentEpoch currentEpoch
) )
@ -98,7 +99,7 @@ final class KafkaMetadataLogTest {
classOf[RuntimeException], classOf[RuntimeException],
() => { () => {
log.appendAsLeader( log.appendAsLeader(
MemoryRecords.withRecords(initialOffset, CompressionType.NONE, currentEpoch, recordFoo), MemoryRecords.withRecords(initialOffset, Compression.NONE, currentEpoch, recordFoo),
currentEpoch currentEpoch
) )
} }
@ -108,7 +109,7 @@ final class KafkaMetadataLogTest {
classOf[RuntimeException], classOf[RuntimeException],
() => { () => {
log.appendAsFollower( log.appendAsFollower(
MemoryRecords.withRecords(initialOffset, CompressionType.NONE, currentEpoch, recordFoo) MemoryRecords.withRecords(initialOffset, Compression.NONE, currentEpoch, recordFoo)
) )
} }
) )
@ -647,7 +648,7 @@ final class KafkaMetadataLogTest {
val batchBuilder = new BatchBuilder[Array[Byte]]( val batchBuilder = new BatchBuilder[Array[Byte]](
buffer, buffer,
new ByteArraySerde, new ByteArraySerde,
CompressionType.NONE, Compression.NONE,
0L, 0L,
mockTime.milliseconds(), mockTime.milliseconds(),
false, false,
@ -1060,7 +1061,7 @@ object KafkaMetadataLogTest {
log.appendAsLeader( log.appendAsLeader(
MemoryRecords.withRecords( MemoryRecords.withRecords(
log.endOffset().offset, log.endOffset().offset,
CompressionType.NONE, Compression.NONE,
epoch, epoch,
(0 until numberOfRecords).map(number => new SimpleRecord(number.toString.getBytes)): _* (0 until numberOfRecords).map(number => new SimpleRecord(number.toString.getBytes)): _*
), ),
@ -1071,7 +1072,7 @@ object KafkaMetadataLogTest {
def append(snapshotWriter: RawSnapshotWriter, numberOfRecords: Int): Unit = { def append(snapshotWriter: RawSnapshotWriter, numberOfRecords: Int): Unit = {
snapshotWriter.append(MemoryRecords.withRecords( snapshotWriter.append(MemoryRecords.withRecords(
0, 0,
CompressionType.NONE, Compression.NONE,
0, 0,
(0 until numberOfRecords).map(number => new SimpleRecord(number.toString.getBytes)): _* (0 until numberOfRecords).map(number => new SimpleRecord(number.toString.getBytes)): _*
)) ))

View File

@ -21,13 +21,14 @@ import kafka.cluster.BrokerEndPoint
import kafka.server.QuotaFactory.QuotaManagers import kafka.server.QuotaFactory.QuotaManagers
import kafka.server.checkpoints.LazyOffsetCheckpoints import kafka.server.checkpoints.LazyOffsetCheckpoints
import kafka.utils.{CoreUtils, Logging, TestUtils} import kafka.utils.{CoreUtils, Logging, TestUtils}
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.{Node, TopicPartition, Uuid} import org.apache.kafka.common.{Node, TopicPartition, Uuid}
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState
import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition
import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset
import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord} import org.apache.kafka.common.record.{MemoryRecords, SimpleRecord}
import org.apache.kafka.common.requests.LeaderAndIsrRequest import org.apache.kafka.common.requests.LeaderAndIsrRequest
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
import org.apache.kafka.server.common.OffsetAndEpoch import org.apache.kafka.server.common.OffsetAndEpoch
@ -265,7 +266,7 @@ class LocalLeaderEndPointTest extends Logging {
} }
private def records: MemoryRecords = { private def records: MemoryRecords = {
MemoryRecords.withRecords(CompressionType.NONE, MemoryRecords.withRecords(Compression.NONE,
new SimpleRecord("first message".getBytes()), new SimpleRecord("first message".getBytes()),
new SimpleRecord("second message".getBytes()), new SimpleRecord("second message".getBytes()),
new SimpleRecord("third message".getBytes()), new SimpleRecord("third message".getBytes()),

View File

@ -26,6 +26,7 @@ import joptsimple._
import kafka.log._ import kafka.log._
import kafka.server.BrokerTopicStats import kafka.server.BrokerTopicStats
import kafka.utils._ import kafka.utils._
import org.apache.kafka.common.compress.{Compression, GzipCompression, Lz4Compression, ZstdCompression}
import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.record._ import org.apache.kafka.common.record._
import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.utils.{Time, Utils}
@ -34,7 +35,7 @@ import org.apache.kafka.server.util.{KafkaScheduler, Scheduler}
import org.apache.kafka.server.util.CommandLineUtils import org.apache.kafka.server.util.CommandLineUtils
import org.apache.kafka.storage.internals.log.{LogConfig, LogDirFailureChannel, ProducerStateManagerConfig} import org.apache.kafka.storage.internals.log.{LogConfig, LogDirFailureChannel, ProducerStateManagerConfig}
import scala.math._ import scala.math.max
/** /**
* This test does linear writes using either a kafka log or a file and measures throughput and latency. * This test does linear writes using either a kafka log or a file and measures throughput and latency.
@ -66,29 +67,34 @@ object TestLinearWriteSpeed {
.describedAs("num_files") .describedAs("num_files")
.ofType(classOf[java.lang.Integer]) .ofType(classOf[java.lang.Integer])
.defaultsTo(1) .defaultsTo(1)
val reportingIntervalOpt = parser.accepts("reporting-interval", "The number of ms between updates.") val reportingIntervalOpt = parser.accepts("reporting-interval", "The number of ms between updates.")
.withRequiredArg .withRequiredArg
.describedAs("ms") .describedAs("ms")
.ofType(classOf[java.lang.Long]) .ofType(classOf[java.lang.Long])
.defaultsTo(1000L) .defaultsTo(1000L)
val maxThroughputOpt = parser.accepts("max-throughput-mb", "The maximum throughput.") val maxThroughputOpt = parser.accepts("max-throughput-mb", "The maximum throughput.")
.withRequiredArg .withRequiredArg
.describedAs("mb") .describedAs("mb")
.ofType(classOf[java.lang.Integer]) .ofType(classOf[java.lang.Integer])
.defaultsTo(Integer.MAX_VALUE) .defaultsTo(Integer.MAX_VALUE)
val flushIntervalOpt = parser.accepts("flush-interval", "The number of messages between flushes") val flushIntervalOpt = parser.accepts("flush-interval", "The number of messages between flushes")
.withRequiredArg() .withRequiredArg()
.describedAs("message_count") .describedAs("message_count")
.ofType(classOf[java.lang.Long]) .ofType(classOf[java.lang.Long])
.defaultsTo(Long.MaxValue) .defaultsTo(Long.MaxValue)
val compressionCodecOpt = parser.accepts("compression", "The compression codec to use") val compressionCodecOpt = parser.accepts("compression", "The compression codec to use")
.withRequiredArg .withRequiredArg
.describedAs("codec") .describedAs("codec")
.ofType(classOf[java.lang.String]) .ofType(classOf[java.lang.String])
.defaultsTo(CompressionType.NONE.name) .defaultsTo(CompressionType.NONE.name)
val mmapOpt = parser.accepts("mmap", "Do writes to memory-mapped files.") val compressionLevelOpt = parser.accepts("level", "The compression level to use")
val channelOpt = parser.accepts("channel", "Do writes to file channels.") .withRequiredArg
val logOpt = parser.accepts("log", "Do writes to kafka logs.") .describedAs("level")
.ofType(classOf[java.lang.Integer])
.defaultsTo(0)
val mmapOpt = parser.accepts("mmap", "Do writes to memory-mapped files.")
val channelOpt = parser.accepts("channel", "Do writes to file channels.")
val logOpt = parser.accepts("log", "Do writes to kafka logs.")
val options = parser.parse(args : _*) val options = parser.parse(args : _*)
@ -104,13 +110,22 @@ object TestLinearWriteSpeed {
val messageSize = options.valueOf(messageSizeOpt).intValue val messageSize = options.valueOf(messageSizeOpt).intValue
val flushInterval = options.valueOf(flushIntervalOpt).longValue val flushInterval = options.valueOf(flushIntervalOpt).longValue
val compressionType = CompressionType.forName(options.valueOf(compressionCodecOpt)) val compressionType = CompressionType.forName(options.valueOf(compressionCodecOpt))
val compressionBuilder = Compression.of(compressionType)
val compressionLevel = options.valueOf(compressionLevelOpt)
compressionType match {
case CompressionType.GZIP => compressionBuilder.asInstanceOf[GzipCompression.Builder].level(compressionLevel)
case CompressionType.LZ4 => compressionBuilder.asInstanceOf[Lz4Compression.Builder].level(compressionLevel)
case CompressionType.ZSTD => compressionBuilder.asInstanceOf[ZstdCompression.Builder].level(compressionLevel)
case _ => //Noop
}
val compression = compressionBuilder.build()
val rand = new Random val rand = new Random
rand.nextBytes(buffer.array) rand.nextBytes(buffer.array)
val numMessages = bufferSize / (messageSize + Records.LOG_OVERHEAD) val numMessages = bufferSize / (messageSize + Records.LOG_OVERHEAD)
val createTime = System.currentTimeMillis val createTime = System.currentTimeMillis
val messageSet = { val messageSet = {
val records = (0 until numMessages).map(_ => new SimpleRecord(createTime, null, new Array[Byte](messageSize))) val records = (0 until numMessages).map(_ => new SimpleRecord(createTime, null, new Array[Byte](messageSize)))
MemoryRecords.withRecords(compressionType, records: _*) MemoryRecords.withRecords(compression, records: _*)
} }
val writables = new Array[Writable](numFiles) val writables = new Array[Writable](numFiles)
@ -197,7 +212,7 @@ object TestLinearWriteSpeed {
class ChannelWritable(val file: File, val content: ByteBuffer) extends Writable { class ChannelWritable(val file: File, val content: ByteBuffer) extends Writable {
file.deleteOnExit() file.deleteOnExit()
val channel = FileChannel.open(file.toPath, StandardOpenOption.CREATE, StandardOpenOption.READ, val channel: FileChannel = FileChannel.open(file.toPath, StandardOpenOption.CREATE, StandardOpenOption.READ,
StandardOpenOption.WRITE) StandardOpenOption.WRITE)
def write(): Int = { def write(): Int = {
channel.write(content) channel.write(content)
@ -212,7 +227,7 @@ object TestLinearWriteSpeed {
class LogWritable(val dir: File, config: LogConfig, scheduler: Scheduler, val messages: MemoryRecords) extends Writable { class LogWritable(val dir: File, config: LogConfig, scheduler: Scheduler, val messages: MemoryRecords) extends Writable {
Utils.delete(dir) Utils.delete(dir)
val log = UnifiedLog( val log: UnifiedLog = UnifiedLog(
dir = dir, dir = dir,
config = config, config = config,
logStartOffset = 0L, logStartOffset = 0L,

View File

@ -47,6 +47,7 @@ import java.util.Optional
import java.util.concurrent.{CountDownLatch, Semaphore} import java.util.concurrent.{CountDownLatch, Semaphore}
import kafka.server.metadata.{KRaftMetadataCache, ZkMetadataCache} import kafka.server.metadata.{KRaftMetadataCache, ZkMetadataCache}
import org.apache.kafka.clients.ClientResponse import org.apache.kafka.clients.ClientResponse
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.replica.ClientMetadata import org.apache.kafka.common.replica.ClientMetadata
@ -159,7 +160,7 @@ class PartitionTest extends AbstractPartitionTest {
val recordArray = (1 to count).map { i => val recordArray = (1 to count).map { i =>
new SimpleRecord(s"$i".getBytes) new SimpleRecord(s"$i".getBytes)
} }
val records = MemoryRecords.withRecords(0L, CompressionType.NONE, leaderEpoch, val records = MemoryRecords.withRecords(0L, Compression.NONE, leaderEpoch,
recordArray: _*) recordArray: _*)
log.appendAsLeader(records, leaderEpoch = leaderEpoch) log.appendAsLeader(records, leaderEpoch = leaderEpoch)
} }
@ -239,11 +240,11 @@ class PartitionTest extends AbstractPartitionTest {
val leaderEpoch = 8 val leaderEpoch = 8
val log = logManager.getOrCreateLog(topicPartition, topicId = None) val log = logManager.getOrCreateLog(topicPartition, topicId = None)
log.appendAsLeader(MemoryRecords.withRecords(0L, CompressionType.NONE, 0, log.appendAsLeader(MemoryRecords.withRecords(0L, Compression.NONE, 0,
new SimpleRecord("k1".getBytes, "v1".getBytes), new SimpleRecord("k1".getBytes, "v1".getBytes),
new SimpleRecord("k2".getBytes, "v2".getBytes) new SimpleRecord("k2".getBytes, "v2".getBytes)
), leaderEpoch = 0) ), leaderEpoch = 0)
log.appendAsLeader(MemoryRecords.withRecords(0L, CompressionType.NONE, 5, log.appendAsLeader(MemoryRecords.withRecords(0L, Compression.NONE, 5,
new SimpleRecord("k3".getBytes, "v3".getBytes), new SimpleRecord("k3".getBytes, "v3".getBytes),
new SimpleRecord("k4".getBytes, "v4".getBytes) new SimpleRecord("k4".getBytes, "v4".getBytes)
), leaderEpoch = 5) ), leaderEpoch = 5)
@ -505,7 +506,7 @@ class PartitionTest extends AbstractPartitionTest {
// Write records with duplicate keys to current replica and roll at offset 6 // Write records with duplicate keys to current replica and roll at offset 6
val currentLog = partition.log.get val currentLog = partition.log.get
currentLog.appendAsLeader(MemoryRecords.withRecords(0L, CompressionType.NONE, 0, currentLog.appendAsLeader(MemoryRecords.withRecords(0L, Compression.NONE, 0,
new SimpleRecord("k1".getBytes, "v1".getBytes), new SimpleRecord("k1".getBytes, "v1".getBytes),
new SimpleRecord("k1".getBytes, "v2".getBytes), new SimpleRecord("k1".getBytes, "v2".getBytes),
new SimpleRecord("k1".getBytes, "v3".getBytes), new SimpleRecord("k1".getBytes, "v3".getBytes),
@ -514,7 +515,7 @@ class PartitionTest extends AbstractPartitionTest {
new SimpleRecord("k2".getBytes, "v6".getBytes) new SimpleRecord("k2".getBytes, "v6".getBytes)
), leaderEpoch = 0) ), leaderEpoch = 0)
currentLog.roll() currentLog.roll()
currentLog.appendAsLeader(MemoryRecords.withRecords(0L, CompressionType.NONE, 0, currentLog.appendAsLeader(MemoryRecords.withRecords(0L, Compression.NONE, 0,
new SimpleRecord("k3".getBytes, "v7".getBytes), new SimpleRecord("k3".getBytes, "v7".getBytes),
new SimpleRecord("k4".getBytes, "v8".getBytes) new SimpleRecord("k4".getBytes, "v8".getBytes)
), leaderEpoch = 0) ), leaderEpoch = 0)
@ -522,7 +523,7 @@ class PartitionTest extends AbstractPartitionTest {
// Write to the future replica as if the log had been compacted, and do not roll the segment // Write to the future replica as if the log had been compacted, and do not roll the segment
val buffer = ByteBuffer.allocate(1024) val buffer = ByteBuffer.allocate(1024)
val builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, val builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, Compression.NONE,
TimestampType.CREATE_TIME, 0L, RecordBatch.NO_TIMESTAMP, 0) TimestampType.CREATE_TIME, 0L, RecordBatch.NO_TIMESTAMP, 0)
builder.appendWithOffset(2L, new SimpleRecord("k1".getBytes, "v3".getBytes)) builder.appendWithOffset(2L, new SimpleRecord("k1".getBytes, "v3".getBytes))
builder.appendWithOffset(5L, new SimpleRecord("k2".getBytes, "v6".getBytes)) builder.appendWithOffset(5L, new SimpleRecord("k2".getBytes, "v6".getBytes))
@ -1184,7 +1185,7 @@ class PartitionTest extends AbstractPartitionTest {
def createRecords(records: Iterable[SimpleRecord], baseOffset: Long, partitionLeaderEpoch: Int = 0): MemoryRecords = { def createRecords(records: Iterable[SimpleRecord], baseOffset: Long, partitionLeaderEpoch: Int = 0): MemoryRecords = {
val buf = ByteBuffer.allocate(DefaultRecordBatch.sizeInBytes(records.asJava)) val buf = ByteBuffer.allocate(DefaultRecordBatch.sizeInBytes(records.asJava))
val builder = MemoryRecords.builder( val builder = MemoryRecords.builder(
buf, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, TimestampType.LOG_APPEND_TIME, buf, RecordBatch.CURRENT_MAGIC_VALUE, Compression.NONE, TimestampType.LOG_APPEND_TIME,
baseOffset, time.milliseconds, partitionLeaderEpoch) baseOffset, time.milliseconds, partitionLeaderEpoch)
records.foreach(builder.append) records.foreach(builder.append)
builder.build() builder.build()
@ -1197,7 +1198,7 @@ class PartitionTest extends AbstractPartitionTest {
val producerEpoch = 0.toShort val producerEpoch = 0.toShort
val isTransactional = false val isTransactional = false
val buf = ByteBuffer.allocate(DefaultRecordBatch.sizeInBytes(records.asJava)) val buf = ByteBuffer.allocate(DefaultRecordBatch.sizeInBytes(records.asJava))
val builder = MemoryRecords.builder(buf, CompressionType.NONE, baseOffset, producerId, val builder = MemoryRecords.builder(buf, Compression.NONE, baseOffset, producerId,
producerEpoch, baseSequence, isTransactional) producerEpoch, baseSequence, isTransactional)
records.foreach(builder.append) records.foreach(builder.append)
builder.build() builder.build()
@ -1210,7 +1211,7 @@ class PartitionTest extends AbstractPartitionTest {
val producerEpoch = 0.toShort val producerEpoch = 0.toShort
val isTransactional = true val isTransactional = true
val buf = ByteBuffer.allocate(DefaultRecordBatch.sizeInBytes(records.asJava)) val buf = ByteBuffer.allocate(DefaultRecordBatch.sizeInBytes(records.asJava))
val builder = MemoryRecords.builder(buf, CompressionType.NONE, baseOffset, producerId, val builder = MemoryRecords.builder(buf, Compression.NONE, baseOffset, producerId,
producerEpoch, baseSequence, isTransactional) producerEpoch, baseSequence, isTransactional)
records.foreach(builder.append) records.foreach(builder.append)
builder.build() builder.build()
@ -3180,7 +3181,7 @@ class PartitionTest extends AbstractPartitionTest {
assertEquals(Optional.of(new EpochEntry(leaderEpoch, 0L)), leaderLog.leaderEpochCache.asJava.flatMap(_.latestEntry)) assertEquals(Optional.of(new EpochEntry(leaderEpoch, 0L)), leaderLog.leaderEpochCache.asJava.flatMap(_.latestEntry))
// Write to the log to increment the log end offset. // Write to the log to increment the log end offset.
leaderLog.appendAsLeader(MemoryRecords.withRecords(0L, CompressionType.NONE, 0, leaderLog.appendAsLeader(MemoryRecords.withRecords(0L, Compression.NONE, 0,
new SimpleRecord("k1".getBytes, "v1".getBytes), new SimpleRecord("k1".getBytes, "v1".getBytes),
new SimpleRecord("k1".getBytes, "v1".getBytes) new SimpleRecord("k1".getBytes, "v1".getBytes)
), leaderEpoch = leaderEpoch) ), leaderEpoch = leaderEpoch)
@ -3671,7 +3672,7 @@ class PartitionTest extends AbstractPartitionTest {
private def seedLogData(log: UnifiedLog, numRecords: Int, leaderEpoch: Int): Unit = { private def seedLogData(log: UnifiedLog, numRecords: Int, leaderEpoch: Int): Unit = {
for (i <- 0 until numRecords) { for (i <- 0 until numRecords) {
val records = MemoryRecords.withRecords(0L, CompressionType.NONE, leaderEpoch, val records = MemoryRecords.withRecords(0L, Compression.NONE, leaderEpoch,
new SimpleRecord(s"k$i".getBytes, s"v$i".getBytes)) new SimpleRecord(s"k$i".getBytes, s"v$i".getBytes))
log.appendAsLeader(records, leaderEpoch) log.appendAsLeader(records, leaderEpoch)
} }

View File

@ -19,8 +19,9 @@ package kafka.coordinator.group
import kafka.log.UnifiedLog import kafka.log.UnifiedLog
import kafka.server.ReplicaManager import kafka.server.ReplicaManager
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.errors.NotLeaderOrFollowerException import org.apache.kafka.common.errors.NotLeaderOrFollowerException
import org.apache.kafka.common.record.{CompressionType, ControlRecordType, EndTransactionMarker, FileRecords, MemoryRecords, RecordBatch, SimpleRecord} import org.apache.kafka.common.record.{ControlRecordType, EndTransactionMarker, FileRecords, MemoryRecords, RecordBatch, SimpleRecord}
import org.apache.kafka.common.requests.TransactionResult import org.apache.kafka.common.requests.TransactionResult
import org.apache.kafka.common.utils.{MockTime, Time} import org.apache.kafka.common.utils.{MockTime, Time}
import org.apache.kafka.coordinator.group.runtime.CoordinatorLoader.UnknownRecordTypeException import org.apache.kafka.coordinator.group.runtime.CoordinatorLoader.UnknownRecordTypeException
@ -639,13 +640,13 @@ class CoordinatorLoaderImplTest {
val memoryRecords = if (producerId == RecordBatch.NO_PRODUCER_ID) { val memoryRecords = if (producerId == RecordBatch.NO_PRODUCER_ID) {
MemoryRecords.withRecords( MemoryRecords.withRecords(
startOffset, startOffset,
CompressionType.NONE, Compression.NONE,
records: _* records: _*
) )
} else { } else {
MemoryRecords.withTransactionalRecords( MemoryRecords.withTransactionalRecords(
startOffset, startOffset,
CompressionType.NONE, Compression.NONE,
producerId, producerId,
producerEpoch, producerEpoch,
0, 0,

View File

@ -18,9 +18,10 @@ package kafka.coordinator.group
import kafka.server.ReplicaManager import kafka.server.ReplicaManager
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.errors.NotLeaderOrFollowerException import org.apache.kafka.common.errors.NotLeaderOrFollowerException
import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch, SimpleRecord} import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, SimpleRecord}
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
import org.apache.kafka.coordinator.group.runtime.PartitionWriter import org.apache.kafka.coordinator.group.runtime.PartitionWriter
import org.apache.kafka.storage.internals.log.{AppendOrigin, LogConfig, VerificationGuard} import org.apache.kafka.storage.internals.log.{AppendOrigin, LogConfig, VerificationGuard}
@ -122,7 +123,7 @@ class CoordinatorPartitionWriterTest {
}) })
val batch = MemoryRecords.withRecords( val batch = MemoryRecords.withRecords(
CompressionType.NONE, Compression.NONE,
new SimpleRecord( new SimpleRecord(
0L, 0L,
"foo".getBytes(Charset.defaultCharset()), "foo".getBytes(Charset.defaultCharset()),
@ -223,7 +224,7 @@ class CoordinatorPartitionWriterTest {
}) })
val batch = MemoryRecords.withRecords( val batch = MemoryRecords.withRecords(
CompressionType.NONE, Compression.NONE,
new SimpleRecord( new SimpleRecord(
0L, 0L,
"foo".getBytes(Charset.defaultCharset()), "foo".getBytes(Charset.defaultCharset()),

View File

@ -32,6 +32,7 @@ import kafka.utils.TestUtils
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Subscription import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Subscription
import org.apache.kafka.clients.consumer.internals.ConsumerProtocol import org.apache.kafka.clients.consumer.internals.ConsumerProtocol
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid}
import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.internals.Topic
import org.apache.kafka.common.metrics.{JmxReporter, KafkaMetricsContext, Metrics => kMetrics} import org.apache.kafka.common.metrics.{JmxReporter, KafkaMetricsContext, Metrics => kMetrics}
@ -148,7 +149,7 @@ class GroupMetadataManagerTest {
) )
val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets) val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets)
val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE, offsetCommitRecords.toArray: _*) val records = MemoryRecords.withRecords(startOffset, Compression.NONE, offsetCommitRecords.toArray: _*)
expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records) expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records)
groupMetadataManager.loadGroupsAndOffsets(groupMetadataTopicPartition, groupEpoch, _ => (), 0L) groupMetadataManager.loadGroupsAndOffsets(groupMetadataTopicPartition, groupEpoch, _ => (), 0L)
@ -177,7 +178,7 @@ class GroupMetadataManagerTest {
val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets) val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets)
val groupMetadataRecord = buildEmptyGroupRecord(generation, protocolType) val groupMetadataRecord = buildEmptyGroupRecord(generation, protocolType)
val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE, val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
(offsetCommitRecords ++ Seq(groupMetadataRecord)).toArray: _*) (offsetCommitRecords ++ Seq(groupMetadataRecord)).toArray: _*)
expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records) expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records)
@ -566,7 +567,7 @@ class GroupMetadataManagerTest {
} }
private def appendConsumerOffsetCommit(buffer: ByteBuffer, baseOffset: Long, offsets: Map[TopicPartition, Long]) = { private def appendConsumerOffsetCommit(buffer: ByteBuffer, baseOffset: Long, offsets: Map[TopicPartition, Long]) = {
val builder = MemoryRecords.builder(buffer, CompressionType.NONE, TimestampType.LOG_APPEND_TIME, baseOffset) val builder = MemoryRecords.builder(buffer, Compression.NONE, TimestampType.LOG_APPEND_TIME, baseOffset)
val commitRecords = createCommittedOffsetRecords(offsets) val commitRecords = createCommittedOffsetRecords(offsets)
commitRecords.foreach(builder.append) commitRecords.foreach(builder.append)
builder.build() builder.build()
@ -575,7 +576,7 @@ class GroupMetadataManagerTest {
private def appendTransactionalOffsetCommits(buffer: ByteBuffer, producerId: Long, producerEpoch: Short, private def appendTransactionalOffsetCommits(buffer: ByteBuffer, producerId: Long, producerEpoch: Short,
baseOffset: Long, offsets: Map[TopicPartition, Long]): Int = { baseOffset: Long, offsets: Map[TopicPartition, Long]): Int = {
val builder = MemoryRecords.builder(buffer, CompressionType.NONE, baseOffset, producerId, producerEpoch, 0, true) val builder = MemoryRecords.builder(buffer, Compression.NONE, baseOffset, producerId, producerEpoch, 0, true)
val commitRecords = createCommittedOffsetRecords(offsets) val commitRecords = createCommittedOffsetRecords(offsets)
commitRecords.foreach(builder.append) commitRecords.foreach(builder.append)
builder.build() builder.build()
@ -584,7 +585,7 @@ class GroupMetadataManagerTest {
private def completeTransactionalOffsetCommit(buffer: ByteBuffer, producerId: Long, producerEpoch: Short, baseOffset: Long, private def completeTransactionalOffsetCommit(buffer: ByteBuffer, producerId: Long, producerEpoch: Short, baseOffset: Long,
isCommit: Boolean): Int = { isCommit: Boolean): Int = {
val builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE, val builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, Compression.NONE,
TimestampType.LOG_APPEND_TIME, baseOffset, time.milliseconds(), producerId, producerEpoch, 0, true, true, TimestampType.LOG_APPEND_TIME, baseOffset, time.milliseconds(), producerId, producerEpoch, 0, true, true,
RecordBatch.NO_PARTITION_LEADER_EPOCH) RecordBatch.NO_PARTITION_LEADER_EPOCH)
val controlRecordType = if (isCommit) ControlRecordType.COMMIT else ControlRecordType.ABORT val controlRecordType = if (isCommit) ControlRecordType.COMMIT else ControlRecordType.ABORT
@ -608,7 +609,7 @@ class GroupMetadataManagerTest {
val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets) val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets)
val tombstone = new SimpleRecord(GroupMetadataManager.offsetCommitKey(groupId, tombstonePartition), null) val tombstone = new SimpleRecord(GroupMetadataManager.offsetCommitKey(groupId, tombstonePartition), null)
val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE, val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
(offsetCommitRecords ++ Seq(tombstone)).toArray: _*) (offsetCommitRecords ++ Seq(tombstone)).toArray: _*)
expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records) expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records)
@ -647,7 +648,7 @@ class GroupMetadataManagerTest {
val memberId = "98098230493" val memberId = "98098230493"
val groupMetadataRecord = buildStableGroupRecordWithMember(generation, protocolType, protocol, memberId) val groupMetadataRecord = buildStableGroupRecordWithMember(generation, protocolType, protocol, memberId)
val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE, val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
(offsetCommitRecords ++ Seq(groupMetadataRecord)).toArray: _*) (offsetCommitRecords ++ Seq(groupMetadataRecord)).toArray: _*)
expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records) expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records)
@ -756,7 +757,7 @@ class GroupMetadataManagerTest {
val groupMetadataRecord = buildStableGroupRecordWithMember(generation = 15, val groupMetadataRecord = buildStableGroupRecordWithMember(generation = 15,
protocolType = "consumer", protocol = "range", memberId) protocolType = "consumer", protocol = "range", memberId)
val groupMetadataTombstone = new SimpleRecord(GroupMetadataManager.groupMetadataKey(groupId), null) val groupMetadataTombstone = new SimpleRecord(GroupMetadataManager.groupMetadataKey(groupId), null)
val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE, val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
Seq(groupMetadataRecord, groupMetadataTombstone).toArray: _*) Seq(groupMetadataRecord, groupMetadataTombstone).toArray: _*)
expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records) expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records)
@ -784,7 +785,7 @@ class GroupMetadataManagerTest {
val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets) val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets)
val groupMetadataRecord = buildStableGroupRecordWithMember(generation = 15, val groupMetadataRecord = buildStableGroupRecordWithMember(generation = 15,
protocolType = "consumer", protocol = "range", memberId, new Array[Byte](assignmentSize)) protocolType = "consumer", protocol = "range", memberId, new Array[Byte](assignmentSize))
val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE, val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
(offsetCommitRecords ++ Seq(groupMetadataRecord)).toArray: _*) (offsetCommitRecords ++ Seq(groupMetadataRecord)).toArray: _*)
expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records) expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records)
@ -845,7 +846,7 @@ class GroupMetadataManagerTest {
val memberId = "98098230493" val memberId = "98098230493"
val groupMetadataRecord = buildStableGroupRecordWithMember(generation, protocolType, protocol, memberId) val groupMetadataRecord = buildStableGroupRecordWithMember(generation, protocolType, protocol, memberId)
val groupMetadataTombstone = new SimpleRecord(GroupMetadataManager.groupMetadataKey(groupId), null) val groupMetadataTombstone = new SimpleRecord(GroupMetadataManager.groupMetadataKey(groupId), null)
val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE, val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
(Seq(groupMetadataRecord, groupMetadataTombstone) ++ offsetCommitRecords).toArray: _*) (Seq(groupMetadataRecord, groupMetadataTombstone) ++ offsetCommitRecords).toArray: _*)
expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records) expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records)
@ -879,14 +880,14 @@ class GroupMetadataManagerTest {
val segment1MemberId = "a" val segment1MemberId = "a"
val segment1Offsets = Map(tp0 -> 23L, tp1 -> 455L, tp3 -> 42L) val segment1Offsets = Map(tp0 -> 23L, tp1 -> 455L, tp3 -> 42L)
val segment1Records = MemoryRecords.withRecords(startOffset, CompressionType.NONE, val segment1Records = MemoryRecords.withRecords(startOffset, Compression.NONE,
(createCommittedOffsetRecords(segment1Offsets) ++ Seq(buildStableGroupRecordWithMember( (createCommittedOffsetRecords(segment1Offsets) ++ Seq(buildStableGroupRecordWithMember(
generation, protocolType, protocol, segment1MemberId))).toArray: _*) generation, protocolType, protocol, segment1MemberId))).toArray: _*)
val segment1End = startOffset + segment1Records.records.asScala.size val segment1End = startOffset + segment1Records.records.asScala.size
val segment2MemberId = "b" val segment2MemberId = "b"
val segment2Offsets = Map(tp0 -> 33L, tp2 -> 8992L, tp3 -> 10L) val segment2Offsets = Map(tp0 -> 33L, tp2 -> 8992L, tp3 -> 10L)
val segment2Records = MemoryRecords.withRecords(segment1End, CompressionType.NONE, val segment2Records = MemoryRecords.withRecords(segment1End, Compression.NONE,
(createCommittedOffsetRecords(segment2Offsets) ++ Seq(buildStableGroupRecordWithMember( (createCommittedOffsetRecords(segment2Offsets) ++ Seq(buildStableGroupRecordWithMember(
generation, protocolType, protocol, segment2MemberId))).toArray: _*) generation, protocolType, protocol, segment2MemberId))).toArray: _*)
val segment2End = segment1End + segment2Records.records.asScala.size val segment2End = segment1End + segment2Records.records.asScala.size
@ -2352,7 +2353,7 @@ class GroupMetadataManagerTest {
val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets, metadataVersion = metadataVersion, retentionTimeOpt = Some(100)) val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets, metadataVersion = metadataVersion, retentionTimeOpt = Some(100))
val memberId = "98098230493" val memberId = "98098230493"
val groupMetadataRecord = buildStableGroupRecordWithMember(generation, protocolType, protocol, memberId, metadataVersion = metadataVersion) val groupMetadataRecord = buildStableGroupRecordWithMember(generation, protocolType, protocol, memberId, metadataVersion = metadataVersion)
val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE, val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
(offsetCommitRecords ++ Seq(groupMetadataRecord)).toArray: _*) (offsetCommitRecords ++ Seq(groupMetadataRecord)).toArray: _*)
expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records) expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records)
@ -2391,7 +2392,7 @@ class GroupMetadataManagerTest {
val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets, retentionTimeOpt = Some(100)) val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets, retentionTimeOpt = Some(100))
val memberId = "98098230493" val memberId = "98098230493"
val groupMetadataRecord = buildStableGroupRecordWithMember(generation, protocolType, protocol, memberId) val groupMetadataRecord = buildStableGroupRecordWithMember(generation, protocolType, protocol, memberId)
val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE, val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
(offsetCommitRecords ++ Seq(groupMetadataRecord)).toArray: _*) (offsetCommitRecords ++ Seq(groupMetadataRecord)).toArray: _*)
expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records) expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records)
@ -2731,7 +2732,7 @@ class GroupMetadataManagerTest {
val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets) val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets)
val groupMetadataRecord = buildEmptyGroupRecord(generation, protocolType) val groupMetadataRecord = buildEmptyGroupRecord(generation, protocolType)
val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE, val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
(offsetCommitRecords ++ Seq(groupMetadataRecord)).toArray: _*) (offsetCommitRecords ++ Seq(groupMetadataRecord)).toArray: _*)
// Prepend empty control batch to valid records // Prepend empty control batch to valid records
@ -2948,7 +2949,7 @@ class GroupMetadataManagerTest {
val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets) val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets)
val groupMetadataRecord = buildStableGroupRecordWithMember(generation = 15, val groupMetadataRecord = buildStableGroupRecordWithMember(generation = 15,
protocolType = "consumer", protocol = "range", memberId) protocolType = "consumer", protocol = "range", memberId)
val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE, val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
(offsetCommitRecords ++ Seq(groupMetadataRecord)).toArray: _*) (offsetCommitRecords ++ Seq(groupMetadataRecord)).toArray: _*)
expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records) expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records)
@ -2995,7 +2996,7 @@ class GroupMetadataManagerTest {
val unknownRecord1 = new SimpleRecord(unknownMessage1, unknownMessage1) val unknownRecord1 = new SimpleRecord(unknownMessage1, unknownMessage1)
val unknownRecord2 = new SimpleRecord(unknownMessage2, unknownMessage2) val unknownRecord2 = new SimpleRecord(unknownMessage2, unknownMessage2)
val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE, val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
(offsetCommitRecords ++ Seq(unknownRecord1, unknownRecord2) ++ Seq(groupMetadataRecord)).toArray: _*) (offsetCommitRecords ++ Seq(unknownRecord1, unknownRecord2) ++ Seq(groupMetadataRecord)).toArray: _*)
expectGroupMetadataLoad(groupTopicPartition, startOffset, records) expectGroupMetadataLoad(groupTopicPartition, startOffset, records)

View File

@ -26,11 +26,12 @@ import kafka.log.UnifiedLog
import kafka.server.{KafkaConfig, MetadataCache, RequestLocal} import kafka.server.{KafkaConfig, MetadataCache, RequestLocal}
import kafka.utils.{Pool, TestUtils} import kafka.utils.{Pool, TestUtils}
import org.apache.kafka.clients.{ClientResponse, NetworkClient} import org.apache.kafka.clients.{ClientResponse, NetworkClient}
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.internals.Topic.TRANSACTION_STATE_TOPIC_NAME import org.apache.kafka.common.internals.Topic.TRANSACTION_STATE_TOPIC_NAME
import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.record.{CompressionType, FileRecords, MemoryRecords, RecordBatch, SimpleRecord} import org.apache.kafka.common.record.{FileRecords, MemoryRecords, RecordBatch, SimpleRecord}
import org.apache.kafka.common.requests._ import org.apache.kafka.common.requests._
import org.apache.kafka.common.utils.{LogContext, MockTime, ProducerIdAndEpoch} import org.apache.kafka.common.utils.{LogContext, MockTime, ProducerIdAndEpoch}
import org.apache.kafka.common.{Node, TopicPartition} import org.apache.kafka.common.{Node, TopicPartition}
@ -466,7 +467,7 @@ class TransactionCoordinatorConcurrencyTest extends AbstractCoordinatorConcurren
val topicPartition = new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, partitionId) val topicPartition = new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, partitionId)
val startOffset = replicaManager.getLogEndOffset(topicPartition).getOrElse(20L) val startOffset = replicaManager.getLogEndOffset(topicPartition).getOrElse(20L)
val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE, txnRecordsByPartition(partitionId).toArray: _*) val records = MemoryRecords.withRecords(startOffset, Compression.NONE, txnRecordsByPartition(partitionId).toArray: _*)
val endOffset = startOffset + records.records.asScala.size val endOffset = startOffset + records.records.asScala.size
when(logMock.logStartOffset).thenReturn(startOffset) when(logMock.logStartOffset).thenReturn(startOffset)

View File

@ -21,10 +21,11 @@ import kafka.internals.generated.TransactionLogKey
import kafka.internals.generated.TransactionLogValue import kafka.internals.generated.TransactionLogValue
import kafka.utils.TestUtils import kafka.utils.TestUtils
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.protocol.{ByteBufferAccessor, MessageUtil} import org.apache.kafka.common.protocol.{ByteBufferAccessor, MessageUtil}
import org.apache.kafka.common.protocol.types.Field.TaggedFieldsSection import org.apache.kafka.common.protocol.types.Field.TaggedFieldsSection
import org.apache.kafka.common.protocol.types.{CompactArrayOf, Field, Schema, Struct, Type} import org.apache.kafka.common.protocol.types.{CompactArrayOf, Field, Schema, Struct, Type}
import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord} import org.apache.kafka.common.record.{MemoryRecords, SimpleRecord}
import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, assertTrue} import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, assertTrue}
import org.junit.jupiter.api.Test import org.junit.jupiter.api.Test
@ -84,7 +85,7 @@ class TransactionLogTest {
new SimpleRecord(keyBytes, valueBytes) new SimpleRecord(keyBytes, valueBytes)
}.toSeq }.toSeq
val records = MemoryRecords.withRecords(0, CompressionType.NONE, txnRecords: _*) val records = MemoryRecords.withRecords(0, Compression.NONE, txnRecords: _*)
var count = 0 var count = 0
for (record <- records.records.asScala) { for (record <- records.records.asScala) {

View File

@ -28,6 +28,7 @@ import kafka.server.{ReplicaManager, RequestLocal}
import kafka.utils.{Pool, TestUtils} import kafka.utils.{Pool, TestUtils}
import kafka.zk.KafkaZkClient import kafka.zk.KafkaZkClient
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.internals.Topic.TRANSACTION_STATE_TOPIC_NAME import org.apache.kafka.common.internals.Topic.TRANSACTION_STATE_TOPIC_NAME
import org.apache.kafka.common.metrics.{JmxReporter, KafkaMetricsContext, Metrics} import org.apache.kafka.common.metrics.{JmxReporter, KafkaMetricsContext, Metrics}
import org.apache.kafka.common.protocol.{Errors, MessageUtil} import org.apache.kafka.common.protocol.{Errors, MessageUtil}
@ -166,7 +167,7 @@ class TransactionStateManagerTest {
txnMetadata1.addPartitions(Set[TopicPartition]( txnMetadata1.addPartitions(Set[TopicPartition](
new TopicPartition("topic1", 0), new TopicPartition("topic1", 0),
new TopicPartition("topic1", 1))) new TopicPartition("topic1", 1)))
val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE, val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
new SimpleRecord(txnMessageKeyBytes1, TransactionLog.valueToBytes(txnMetadata1.prepareNoTransit()))) new SimpleRecord(txnMessageKeyBytes1, TransactionLog.valueToBytes(txnMetadata1.prepareNoTransit())))
// We create a latch which is awaited while the log is loading. This ensures that the deletion // We create a latch which is awaited while the log is loading. This ensures that the deletion
@ -250,7 +251,7 @@ class TransactionStateManagerTest {
txnRecords += new SimpleRecord(txnMessageKeyBytes2, TransactionLog.valueToBytes(txnMetadata2.prepareNoTransit())) txnRecords += new SimpleRecord(txnMessageKeyBytes2, TransactionLog.valueToBytes(txnMetadata2.prepareNoTransit()))
val startOffset = 15L // it should work for any start offset val startOffset = 15L // it should work for any start offset
val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE, txnRecords.toArray: _*) val records = MemoryRecords.withRecords(startOffset, Compression.NONE, txnRecords.toArray: _*)
prepareTxnLog(topicPartition, startOffset, records) prepareTxnLog(topicPartition, startOffset, records)
@ -878,7 +879,7 @@ class TransactionStateManagerTest {
txnRecords += new SimpleRecord(txnMessageKeyBytes1, TransactionLog.valueToBytes(txnMetadata1.prepareNoTransit())) txnRecords += new SimpleRecord(txnMessageKeyBytes1, TransactionLog.valueToBytes(txnMetadata1.prepareNoTransit()))
val startOffset = 0L val startOffset = 0L
val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE, txnRecords.toArray: _*) val records = MemoryRecords.withRecords(startOffset, Compression.NONE, txnRecords.toArray: _*)
prepareTxnLog(topicPartition, 0, records) prepareTxnLog(topicPartition, 0, records)
@ -1026,7 +1027,7 @@ class TransactionStateManagerTest {
val partitionId = transactionManager.partitionFor(transactionalId1) val partitionId = transactionManager.partitionFor(transactionalId1)
val topicPartition = new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, partitionId) val topicPartition = new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, partitionId)
val expectedTombstone = new SimpleRecord(time.milliseconds(), TransactionLog.keyToBytes(transactionalId1), null) val expectedTombstone = new SimpleRecord(time.milliseconds(), TransactionLog.keyToBytes(transactionalId1), null)
val expectedRecords = MemoryRecords.withRecords(TransactionLog.EnforcedCompressionType, expectedTombstone) val expectedRecords = MemoryRecords.withRecords(TransactionLog.EnforcedCompression, expectedTombstone)
assertEquals(Set(topicPartition), appendedRecords.keySet) assertEquals(Set(topicPartition), appendedRecords.keySet)
assertEquals(Seq(expectedRecords), appendedRecords(topicPartition).toSeq) assertEquals(Seq(expectedRecords), appendedRecords(topicPartition).toSeq)
} else { } else {
@ -1041,7 +1042,7 @@ class TransactionStateManagerTest {
txnRecords += new SimpleRecord(txnMessageKeyBytes1, TransactionLog.valueToBytes(txnMetadata1.prepareNoTransit())) txnRecords += new SimpleRecord(txnMessageKeyBytes1, TransactionLog.valueToBytes(txnMetadata1.prepareNoTransit()))
val startOffset = 0L val startOffset = 0L
val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE, txnRecords.toArray: _*) val records = MemoryRecords.withRecords(startOffset, Compression.NONE, txnRecords.toArray: _*)
prepareTxnLog(topicPartition, 0, records) prepareTxnLog(topicPartition, 0, records)
@ -1148,7 +1149,7 @@ class TransactionStateManagerTest {
txnRecords += new SimpleRecord(txnMessageKeyBytes1, TransactionLog.valueToBytes(txnMetadata1.prepareNoTransit())) txnRecords += new SimpleRecord(txnMessageKeyBytes1, TransactionLog.valueToBytes(txnMetadata1.prepareNoTransit()))
val startOffset = 15L val startOffset = 15L
val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE, txnRecords.toArray: _*) val records = MemoryRecords.withRecords(startOffset, Compression.NONE, txnRecords.toArray: _*)
prepareTxnLog(topicPartition, startOffset, records) prepareTxnLog(topicPartition, startOffset, records)
transactionManager.loadTransactionsForTxnTopicPartition(partitionId, 0, (_, _, _, _) => ()) transactionManager.loadTransactionsForTxnTopicPartition(partitionId, 0, (_, _, _, _) => ())
@ -1171,7 +1172,7 @@ class TransactionStateManagerTest {
val unknownMessage = MessageUtil.toVersionPrefixedBytes(Short.MaxValue, unknownKey) val unknownMessage = MessageUtil.toVersionPrefixedBytes(Short.MaxValue, unknownKey)
val unknownRecord = new SimpleRecord(unknownMessage, unknownMessage) val unknownRecord = new SimpleRecord(unknownMessage, unknownMessage)
val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE, val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
(Seq(unknownRecord) ++ txnRecords).toArray: _*) (Seq(unknownRecord) ++ txnRecords).toArray: _*)
prepareTxnLog(topicPartition, 0, records) prepareTxnLog(topicPartition, 0, records)

View File

@ -23,8 +23,9 @@ import kafka.server.BrokerTopicStats
import kafka.utils.{Pool, TestUtils} import kafka.utils.{Pool, TestUtils}
import kafka.utils.Implicits._ import kafka.utils.Implicits._
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch} import org.apache.kafka.common.record.{MemoryRecords, RecordBatch}
import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.utils.Utils
import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
import org.apache.kafka.server.util.MockTime import org.apache.kafka.server.util.MockTime
@ -142,7 +143,7 @@ abstract class AbstractLogCleanerIntegrationTest {
def counter: Int = ctr def counter: Int = ctr
def incCounter(): Unit = ctr += 1 def incCounter(): Unit = ctr += 1
def writeDups(numKeys: Int, numDups: Int, log: UnifiedLog, codec: CompressionType, def writeDups(numKeys: Int, numDups: Int, log: UnifiedLog, codec: Compression,
startKey: Int = 0, magicValue: Byte = RecordBatch.CURRENT_MAGIC_VALUE): Seq[(Int, String, Long)] = { startKey: Int = 0, magicValue: Byte = RecordBatch.CURRENT_MAGIC_VALUE): Seq[(Int, String, Long)] = {
for (_ <- 0 until numDups; key <- startKey until (startKey + numKeys)) yield { for (_ <- 0 until numDups; key <- startKey until (startKey + numKeys)) yield {
val value = counter.toString val value = counter.toString
@ -155,7 +156,7 @@ abstract class AbstractLogCleanerIntegrationTest {
} }
} }
def createLargeSingleMessageSet(key: Int, messageFormatVersion: Byte, codec: CompressionType): (String, MemoryRecords) = { def createLargeSingleMessageSet(key: Int, messageFormatVersion: Byte, codec: Compression): (String, MemoryRecords) = {
def messageValue(length: Int): String = { def messageValue(length: Int): String = {
val random = new Random(0) val random = new Random(0)
new String(random.alphanumeric.take(length).toArray) new String(random.alphanumeric.take(length).toArray)

View File

@ -19,6 +19,7 @@ package kafka.log
import kafka.server.BrokerTopicStats import kafka.server.BrokerTopicStats
import kafka.utils._ import kafka.utils._
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch, SimpleRecord} import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch, SimpleRecord}
import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.utils.Utils
@ -50,10 +51,10 @@ class BrokerCompressionTest {
*/ */
@ParameterizedTest @ParameterizedTest
@MethodSource(Array("parameters")) @MethodSource(Array("parameters"))
def testBrokerSideCompression(messageCompression: String, brokerCompression: String): Unit = { def testBrokerSideCompression(messageCompressionType: CompressionType, brokerCompressionType: BrokerCompressionType): Unit = {
val messageCompressionType = CompressionType.forName(messageCompression) val messageCompression = Compression.of(messageCompressionType).build()
val logProps = new Properties() val logProps = new Properties()
logProps.put(TopicConfig.COMPRESSION_TYPE_CONFIG, brokerCompression) logProps.put(TopicConfig.COMPRESSION_TYPE_CONFIG, brokerCompressionType.name)
/*configure broker-side compression */ /*configure broker-side compression */
val log = UnifiedLog( val log = UnifiedLog(
dir = logDir, dir = logDir,
@ -72,7 +73,7 @@ class BrokerCompressionTest {
) )
/* append two messages */ /* append two messages */
log.appendAsLeader(MemoryRecords.withRecords(messageCompressionType, 0, log.appendAsLeader(MemoryRecords.withRecords(messageCompression, 0,
new SimpleRecord("hello".getBytes), new SimpleRecord("there".getBytes)), leaderEpoch = 0) new SimpleRecord("hello".getBytes), new SimpleRecord("there".getBytes)), leaderEpoch = 0)
def readBatch(offset: Int): RecordBatch = { def readBatch(offset: Int): RecordBatch = {
@ -83,9 +84,9 @@ class BrokerCompressionTest {
fetchInfo.records.batches.iterator.next() fetchInfo.records.batches.iterator.next()
} }
if (!brokerCompression.equals("producer")) { if (brokerCompressionType != BrokerCompressionType.PRODUCER) {
val brokerCompressionType = BrokerCompressionType.forName(brokerCompression).targetCompressionType(null) val targetCompression = BrokerCompressionType.targetCompression(log.config.compression, null)
assertEquals(brokerCompressionType, readBatch(0).compressionType, "Compression at offset 0 should produce " + brokerCompressionType) assertEquals(targetCompression.`type`(), readBatch(0).compressionType, "Compression at offset 0 should produce " + brokerCompressionType)
} }
else else
assertEquals(messageCompressionType, readBatch(0).compressionType, "Compression at offset 0 should produce " + messageCompressionType) assertEquals(messageCompressionType, readBatch(0).compressionType, "Compression at offset 0 should produce " + messageCompressionType)
@ -98,7 +99,7 @@ object BrokerCompressionTest {
java.util.Arrays.stream( java.util.Arrays.stream(
for (brokerCompression <- BrokerCompressionType.values; for (brokerCompression <- BrokerCompressionType.values;
messageCompression <- CompressionType.values messageCompression <- CompressionType.values
) yield Arguments.of(messageCompression.name, brokerCompression.name) ) yield Arguments.of(messageCompression, brokerCompression)
) )
} }
} }

View File

@ -24,9 +24,10 @@ import java.util.regex.Pattern
import java.util.Collections import java.util.Collections
import kafka.server.KafkaConfig import kafka.server.KafkaConfig
import kafka.utils.TestUtils import kafka.utils.TestUtils
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.{KafkaException, TopicPartition}
import org.apache.kafka.common.errors.KafkaStorageException import org.apache.kafka.common.errors.KafkaStorageException
import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Record, SimpleRecord} import org.apache.kafka.common.record.{MemoryRecords, Record, SimpleRecord}
import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.utils.{Time, Utils}
import org.apache.kafka.server.util.{MockTime, Scheduler} import org.apache.kafka.server.util.{MockTime, Scheduler}
import org.apache.kafka.storage.internals.log.{FetchDataInfo, LogConfig, LogDirFailureChannel, LogFileUtils, LogOffsetMetadata, LogSegment, LogSegments} import org.apache.kafka.storage.internals.log.{FetchDataInfo, LogConfig, LogDirFailureChannel, LogFileUtils, LogOffsetMetadata, LogSegment, LogSegments}
@ -99,7 +100,7 @@ class LocalLogTest {
log.append(lastOffset = initialOffset + records.size - 1, log.append(lastOffset = initialOffset + records.size - 1,
largestTimestamp = records.head.timestamp, largestTimestamp = records.head.timestamp,
shallowOffsetOfMaxTimestamp = initialOffset, shallowOffsetOfMaxTimestamp = initialOffset,
records = MemoryRecords.withRecords(initialOffset, CompressionType.NONE, 0, records.toList : _*)) records = MemoryRecords.withRecords(initialOffset, Compression.NONE, 0, records.toList : _*))
} }
private def readRecords(log: LocalLog = log, private def readRecords(log: LocalLog = log,

View File

@ -21,7 +21,8 @@ import java.io.PrintWriter
import com.yammer.metrics.core.{Gauge, MetricName} import com.yammer.metrics.core.{Gauge, MetricName}
import kafka.utils.TestUtils import kafka.utils.TestUtils
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.record.{CompressionType, RecordBatch} import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.record.RecordBatch
import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.server.metrics.KafkaYammerMetrics
import org.apache.kafka.server.util.MockTime import org.apache.kafka.server.util.MockTime
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
@ -35,7 +36,7 @@ import scala.jdk.CollectionConverters._
*/ */
class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest { class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest {
val codec: CompressionType = CompressionType.LZ4 val codec: Compression = Compression.lz4().build()
val time = new MockTime() val time = new MockTime()
val topicPartitions = Array(new TopicPartition("log", 0), new TopicPartition("log", 1), new TopicPartition("log", 2)) val topicPartitions = Array(new TopicPartition("log", 0), new TopicPartition("log", 1), new TopicPartition("log", 2))
@ -141,7 +142,7 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest {
val log = cleaner.logs.get(topicPartitions(0)) val log = cleaner.logs.get(topicPartitions(0))
val T0 = time.milliseconds val T0 = time.milliseconds
writeKeyDups(numKeys = 100, numDups = 3, log, CompressionType.NONE, timestamp = T0, startValue = 0, step = 1) writeKeyDups(numKeys = 100, numDups = 3, log, Compression.NONE, timestamp = T0, startValue = 0, step = 1)
val startSizeBlock0 = log.size val startSizeBlock0 = log.size
@ -159,7 +160,7 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest {
val T1 = time.milliseconds val T1 = time.milliseconds
// write the second block of data: all zero keys // write the second block of data: all zero keys
val appends1 = writeKeyDups(numKeys = 100, numDups = 1, log, CompressionType.NONE, timestamp = T1, startValue = 0, step = 0) val appends1 = writeKeyDups(numKeys = 100, numDups = 1, log, Compression.NONE, timestamp = T1, startValue = 0, step = 0)
// roll the active segment // roll the active segment
log.roll() log.roll()
@ -199,7 +200,7 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest {
} }
} }
private def writeKeyDups(numKeys: Int, numDups: Int, log: UnifiedLog, codec: CompressionType, timestamp: Long, private def writeKeyDups(numKeys: Int, numDups: Int, log: UnifiedLog, codec: Compression, timestamp: Long,
startValue: Int, step: Int): Seq[(Int, Int)] = { startValue: Int, step: Int): Seq[(Int, Int)] = {
var valCounter = startValue var valCounter = startValue
for (_ <- 0 until numDups; key <- 0 until numKeys) yield { for (_ <- 0 until numDups; key <- 0 until numKeys) yield {

View File

@ -19,6 +19,7 @@ package kafka.log
import kafka.utils._ import kafka.utils._
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.record.CompressionType import org.apache.kafka.common.record.CompressionType
import org.apache.kafka.server.util.MockTime import org.apache.kafka.server.util.MockTime
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
@ -45,7 +46,8 @@ class LogCleanerLagIntegrationTest extends AbstractLogCleanerIntegrationTest wit
@ParameterizedTest @ParameterizedTest
@MethodSource(Array("parameters")) @MethodSource(Array("parameters"))
def cleanerTest(codec: CompressionType): Unit = { def cleanerTest(compressionType: CompressionType): Unit = {
val codec: Compression = Compression.of(compressionType).build()
cleaner = makeCleaner(partitions = topicPartitions, cleaner = makeCleaner(partitions = topicPartitions,
backoffMs = cleanerBackOffMs, backoffMs = cleanerBackOffMs,
minCompactionLagMs = minCompactionLag, minCompactionLagMs = minCompactionLag,
@ -102,7 +104,7 @@ class LogCleanerLagIntegrationTest extends AbstractLogCleanerIntegrationTest wit
} }
} }
private def writeDups(numKeys: Int, numDups: Int, log: UnifiedLog, codec: CompressionType, timestamp: Long): Seq[(Int, Int)] = { private def writeDups(numKeys: Int, numDups: Int, log: UnifiedLog, codec: Compression, timestamp: Long): Seq[(Int, Int)] = {
for (_ <- 0 until numDups; key <- 0 until numKeys) yield { for (_ <- 0 until numDups; key <- 0 until numKeys) yield {
val count = counter val count = counter
log.appendAsLeader(TestUtils.singletonRecords(value = counter.toString.getBytes, codec = codec, log.appendAsLeader(TestUtils.singletonRecords(value = counter.toString.getBytes, codec = codec,

View File

@ -23,6 +23,7 @@ import java.util.Properties
import kafka.server.BrokerTopicStats import kafka.server.BrokerTopicStats
import kafka.utils._ import kafka.utils._
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.record._ import org.apache.kafka.common.record._
import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.utils.Utils
@ -667,10 +668,10 @@ class LogCleanerManagerTest extends Logging {
val producerId = 15L val producerId = 15L
val producerEpoch = 0.toShort val producerEpoch = 0.toShort
val sequence = 0 val sequence = 0
log.appendAsLeader(MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, producerEpoch, sequence, log.appendAsLeader(MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, producerEpoch, sequence,
new SimpleRecord(time.milliseconds(), "1".getBytes, "a".getBytes), new SimpleRecord(time.milliseconds(), "1".getBytes, "a".getBytes),
new SimpleRecord(time.milliseconds(), "2".getBytes, "b".getBytes)), leaderEpoch = 0) new SimpleRecord(time.milliseconds(), "2".getBytes, "b".getBytes)), leaderEpoch = 0)
log.appendAsLeader(MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, producerEpoch, sequence + 2, log.appendAsLeader(MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, producerEpoch, sequence + 2,
new SimpleRecord(time.milliseconds(), "3".getBytes, "c".getBytes)), leaderEpoch = 0) new SimpleRecord(time.milliseconds(), "3".getBytes, "c".getBytes)), leaderEpoch = 0)
log.roll() log.roll()
log.updateHighWatermark(3L) log.updateHighWatermark(3L)
@ -853,7 +854,7 @@ class LogCleanerManagerTest extends Logging {
new SimpleRecord(currentTimestamp, s"key-$offset".getBytes, s"value-$offset".getBytes) new SimpleRecord(currentTimestamp, s"key-$offset".getBytes, s"value-$offset".getBytes)
} }
log.appendAsLeader(MemoryRecords.withRecords(CompressionType.NONE, records:_*), leaderEpoch = 1) log.appendAsLeader(MemoryRecords.withRecords(Compression.NONE, records:_*), leaderEpoch = 1)
log.maybeIncrementHighWatermark(log.logEndOffsetMetadata) log.maybeIncrementHighWatermark(log.logEndOffsetMetadata)
} }
@ -876,6 +877,6 @@ class LogCleanerManagerTest extends Logging {
} }
private def records(key: Int, value: Int, timestamp: Long) = private def records(key: Int, value: Int, timestamp: Long) =
MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord(timestamp, key.toString.getBytes, value.toString.getBytes)) MemoryRecords.withRecords(Compression.NONE, new SimpleRecord(timestamp, key.toString.getBytes, value.toString.getBytes))
} }

View File

@ -23,6 +23,7 @@ import kafka.server.KafkaConfig
import kafka.server.checkpoints.OffsetCheckpointFile import kafka.server.checkpoints.OffsetCheckpointFile
import kafka.utils._ import kafka.utils._
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.record._ import org.apache.kafka.common.record._
import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_0_IV1, IBP_0_11_0_IV0, IBP_0_9_0} import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_0_IV1, IBP_0_11_0_IV0, IBP_0_9_0}
@ -48,7 +49,8 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati
@ParameterizedTest @ParameterizedTest
@ArgumentsSource(classOf[LogCleanerParameterizedIntegrationTest.AllCompressions]) @ArgumentsSource(classOf[LogCleanerParameterizedIntegrationTest.AllCompressions])
def cleanerTest(codec: CompressionType): Unit = { def cleanerTest(compressionType: CompressionType): Unit = {
val codec: Compression = Compression.of(compressionType).build()
val largeMessageKey = 20 val largeMessageKey = 20
val (largeMessageValue, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, RecordBatch.CURRENT_MAGIC_VALUE, codec) val (largeMessageValue, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, RecordBatch.CURRENT_MAGIC_VALUE, codec)
val maxMessageSize = largeMessageSet.sizeInBytes val maxMessageSize = largeMessageSet.sizeInBytes
@ -91,7 +93,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati
@ParameterizedTest @ParameterizedTest
@ArgumentsSource(classOf[LogCleanerParameterizedIntegrationTest.AllCompressions]) @ArgumentsSource(classOf[LogCleanerParameterizedIntegrationTest.AllCompressions])
def testCleansCombinedCompactAndDeleteTopic(codec: CompressionType): Unit = { def testCleansCombinedCompactAndDeleteTopic(compressionType: CompressionType): Unit = {
val logProps = new Properties() val logProps = new Properties()
val retentionMs: Integer = 100000 val retentionMs: Integer = 100000
logProps.put(TopicConfig.RETENTION_MS_CONFIG, retentionMs: Integer) logProps.put(TopicConfig.RETENTION_MS_CONFIG, retentionMs: Integer)
@ -101,7 +103,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati
cleaner = makeCleaner(partitions = topicPartitions.take(1), propertyOverrides = logProps, backoffMs = 100L) cleaner = makeCleaner(partitions = topicPartitions.take(1), propertyOverrides = logProps, backoffMs = 100L)
val log = cleaner.logs.get(topicPartitions(0)) val log = cleaner.logs.get(topicPartitions(0))
val messages = writeDups(numKeys = numKeys, numDups = 3, log = log, codec = codec) val messages = writeDups(numKeys = numKeys, numDups = 3, log = log, codec = Compression.of(compressionType).build())
val startSize = log.size val startSize = log.size
log.updateHighWatermark(log.logEndOffset) log.updateHighWatermark(log.logEndOffset)
@ -136,11 +138,12 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati
@nowarn("cat=deprecation") @nowarn("cat=deprecation")
@ParameterizedTest @ParameterizedTest
@ArgumentsSource(classOf[LogCleanerParameterizedIntegrationTest.ExcludeZstd]) @ArgumentsSource(classOf[LogCleanerParameterizedIntegrationTest.ExcludeZstd])
def testCleanerWithMessageFormatV0(codec: CompressionType): Unit = { def testCleanerWithMessageFormatV0(compressionType: CompressionType): Unit = {
val codec: Compression = Compression.of(compressionType).build()
val largeMessageKey = 20 val largeMessageKey = 20
val (largeMessageValue, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, RecordBatch.MAGIC_VALUE_V0, codec) val (largeMessageValue, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, RecordBatch.MAGIC_VALUE_V0, codec)
val maxMessageSize = codec match { val maxMessageSize = codec match {
case CompressionType.NONE => largeMessageSet.sizeInBytes case Compression.NONE => largeMessageSet.sizeInBytes
case _ => case _ =>
// the broker assigns absolute offsets for message format 0 which potentially causes the compressed size to // the broker assigns absolute offsets for message format 0 which potentially causes the compressed size to
// increase because the broker offsets are larger than the ones assigned by the client // increase because the broker offsets are larger than the ones assigned by the client
@ -191,6 +194,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati
@ParameterizedTest @ParameterizedTest
@ArgumentsSource(classOf[LogCleanerParameterizedIntegrationTest.ExcludeZstd]) @ArgumentsSource(classOf[LogCleanerParameterizedIntegrationTest.ExcludeZstd])
def testCleaningNestedMessagesWithV0AndV1(codec: CompressionType): Unit = { def testCleaningNestedMessagesWithV0AndV1(codec: CompressionType): Unit = {
val compression = Compression.of(codec).build()
val maxMessageSize = 192 val maxMessageSize = 192
cleaner = makeCleaner(partitions = topicPartitions, maxMessageSize = maxMessageSize, segmentSize = 256) cleaner = makeCleaner(partitions = topicPartitions, maxMessageSize = maxMessageSize, segmentSize = 256)
@ -201,15 +205,15 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati
// with compression enabled, these messages will be written as a single message containing // with compression enabled, these messages will be written as a single message containing
// all of the individual messages // all of the individual messages
var appendsV0 = writeDupsSingleMessageSet(numKeys = 2, numDups = 3, log = log, codec = codec, magicValue = RecordBatch.MAGIC_VALUE_V0) var appendsV0 = writeDupsSingleMessageSet(numKeys = 2, numDups = 3, log = log, codec = compression, magicValue = RecordBatch.MAGIC_VALUE_V0)
appendsV0 ++= writeDupsSingleMessageSet(numKeys = 2, startKey = 3, numDups = 2, log = log, codec = codec, magicValue = RecordBatch.MAGIC_VALUE_V0) appendsV0 ++= writeDupsSingleMessageSet(numKeys = 2, startKey = 3, numDups = 2, log = log, codec = compression, magicValue = RecordBatch.MAGIC_VALUE_V0)
props.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, IBP_0_10_0_IV1.version) props.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, IBP_0_10_0_IV1.version)
log.updateConfig(new LogConfig(props)) log.updateConfig(new LogConfig(props))
var appendsV1 = writeDupsSingleMessageSet(startKey = 4, numKeys = 2, numDups = 2, log = log, codec = codec, magicValue = RecordBatch.MAGIC_VALUE_V1) var appendsV1 = writeDupsSingleMessageSet(startKey = 4, numKeys = 2, numDups = 2, log = log, codec = compression, magicValue = RecordBatch.MAGIC_VALUE_V1)
appendsV1 ++= writeDupsSingleMessageSet(startKey = 4, numKeys = 2, numDups = 2, log = log, codec = codec, magicValue = RecordBatch.MAGIC_VALUE_V1) appendsV1 ++= writeDupsSingleMessageSet(startKey = 4, numKeys = 2, numDups = 2, log = log, codec = compression, magicValue = RecordBatch.MAGIC_VALUE_V1)
appendsV1 ++= writeDupsSingleMessageSet(startKey = 6, numKeys = 2, numDups = 2, log = log, codec = codec, magicValue = RecordBatch.MAGIC_VALUE_V1) appendsV1 ++= writeDupsSingleMessageSet(startKey = 6, numKeys = 2, numDups = 2, log = log, codec = compression, magicValue = RecordBatch.MAGIC_VALUE_V1)
val appends = appendsV0 ++ appendsV1 val appends = appendsV0 ++ appendsV1
@ -228,7 +232,8 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati
@ParameterizedTest @ParameterizedTest
@ArgumentsSource(classOf[LogCleanerParameterizedIntegrationTest.AllCompressions]) @ArgumentsSource(classOf[LogCleanerParameterizedIntegrationTest.AllCompressions])
def cleanerConfigUpdateTest(codec: CompressionType): Unit = { def cleanerConfigUpdateTest(compressionType: CompressionType): Unit = {
val codec: Compression = Compression.of(compressionType).build()
val largeMessageKey = 20 val largeMessageKey = 20
val (_, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, RecordBatch.CURRENT_MAGIC_VALUE, codec) val (_, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, RecordBatch.CURRENT_MAGIC_VALUE, codec)
val maxMessageSize = largeMessageSet.sizeInBytes val maxMessageSize = largeMessageSet.sizeInBytes
@ -305,7 +310,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati
} }
} }
private def writeDupsSingleMessageSet(numKeys: Int, numDups: Int, log: UnifiedLog, codec: CompressionType, private def writeDupsSingleMessageSet(numKeys: Int, numDups: Int, log: UnifiedLog, codec: Compression,
startKey: Int = 0, magicValue: Byte): Seq[(Int, String, Long)] = { startKey: Int = 0, magicValue: Byte): Seq[(Int, String, Long)] = {
val kvs = for (_ <- 0 until numDups; key <- startKey until (startKey + numKeys)) yield { val kvs = for (_ <- 0 until numDups; key <- startKey until (startKey + numKeys)) yield {
val payload = counter.toString val payload = counter.toString

View File

@ -21,6 +21,7 @@ import kafka.common._
import kafka.server.{BrokerTopicStats, KafkaConfig} import kafka.server.{BrokerTopicStats, KafkaConfig}
import kafka.utils._ import kafka.utils._
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.errors.CorruptRecordException import org.apache.kafka.common.errors.CorruptRecordException
import org.apache.kafka.common.record._ import org.apache.kafka.common.record._
@ -1116,7 +1117,7 @@ class LogCleanerTest extends Logging {
val producerId2 = 2L val producerId2 = 2L
val records = MemoryRecords.withTransactionalRecords( val records = MemoryRecords.withTransactionalRecords(
CompressionType.NONE, Compression.NONE,
producerId2, producerId2,
producerEpoch, producerEpoch,
0, 0,
@ -1993,7 +1994,7 @@ class LogCleanerTest extends Logging {
private def invalidCleanedMessage(initialOffset: Long, private def invalidCleanedMessage(initialOffset: Long,
keysAndValues: Iterable[(Int, Int)], keysAndValues: Iterable[(Int, Int)],
codec: CompressionType = CompressionType.GZIP): MemoryRecords = { compressionType: CompressionType = CompressionType.GZIP): MemoryRecords = {
// this function replicates the old versions of the cleaner which under some circumstances // this function replicates the old versions of the cleaner which under some circumstances
// would write invalid compressed message sets with the outer magic set to 1 and the inner // would write invalid compressed message sets with the outer magic set to 1 and the inner
// magic set to 0 // magic set to 0
@ -2004,6 +2005,7 @@ class LogCleanerTest extends Logging {
kv._2.toString.getBytes)) kv._2.toString.getBytes))
val buffer = ByteBuffer.allocate(math.min(math.max(records.map(_.sizeInBytes()).sum / 2, 1024), 1 << 16)) val buffer = ByteBuffer.allocate(math.min(math.max(records.map(_.sizeInBytes()).sum / 2, 1024), 1 << 16))
val codec: Compression = Compression.of(compressionType).build()
val builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V1, codec, TimestampType.CREATE_TIME, initialOffset) val builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V1, codec, TimestampType.CREATE_TIME, initialOffset)
var offset = initialOffset var offset = initialOffset
@ -2016,7 +2018,7 @@ class LogCleanerTest extends Logging {
} }
private def messageWithOffset(key: Array[Byte], value: Array[Byte], offset: Long): MemoryRecords = private def messageWithOffset(key: Array[Byte], value: Array[Byte], offset: Long): MemoryRecords =
MemoryRecords.withRecords(offset, CompressionType.NONE, 0, new SimpleRecord(key, value)) MemoryRecords.withRecords(offset, Compression.NONE, 0, new SimpleRecord(key, value))
private def messageWithOffset(key: Int, value: Int, offset: Long): MemoryRecords = private def messageWithOffset(key: Int, value: Int, offset: Long): MemoryRecords =
messageWithOffset(key.toString.getBytes, value.toString.getBytes, offset) messageWithOffset(key.toString.getBytes, value.toString.getBytes, offset)
@ -2061,7 +2063,7 @@ class LogCleanerTest extends Logging {
producerEpoch: Short = RecordBatch.NO_PRODUCER_EPOCH, producerEpoch: Short = RecordBatch.NO_PRODUCER_EPOCH,
sequence: Int = RecordBatch.NO_SEQUENCE, sequence: Int = RecordBatch.NO_SEQUENCE,
partitionLeaderEpoch: Int = RecordBatch.NO_PARTITION_LEADER_EPOCH): MemoryRecords = { partitionLeaderEpoch: Int = RecordBatch.NO_PARTITION_LEADER_EPOCH): MemoryRecords = {
MemoryRecords.withIdempotentRecords(RecordBatch.CURRENT_MAGIC_VALUE, 0L, CompressionType.NONE, producerId, producerEpoch, sequence, MemoryRecords.withIdempotentRecords(RecordBatch.CURRENT_MAGIC_VALUE, 0L, Compression.NONE, producerId, producerEpoch, sequence,
partitionLeaderEpoch, new SimpleRecord(key.toString.getBytes, value.toString.getBytes)) partitionLeaderEpoch, new SimpleRecord(key.toString.getBytes, value.toString.getBytes))
} }
@ -2097,9 +2099,9 @@ class LogCleanerTest extends Logging {
new SimpleRecord(time.milliseconds(), keyBytes, keyBytes) // the value doesn't matter since we validate offsets new SimpleRecord(time.milliseconds(), keyBytes, keyBytes) // the value doesn't matter since we validate offsets
} }
val records = if (isTransactional) val records = if (isTransactional)
MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, producerEpoch, sequence, simpleRecords.toArray: _*) MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, producerEpoch, sequence, simpleRecords.toArray: _*)
else else
MemoryRecords.withIdempotentRecords(CompressionType.NONE, producerId, producerEpoch, sequence, simpleRecords.toArray: _*) MemoryRecords.withIdempotentRecords(Compression.NONE, producerId, producerEpoch, sequence, simpleRecords.toArray: _*)
sequence += simpleRecords.size sequence += simpleRecords.size
log.appendAsLeader(records, leaderEpoch, origin) log.appendAsLeader(records, leaderEpoch, origin)
} }

View File

@ -95,6 +95,9 @@ class LogConfigTest {
case TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG => assertPropertyInvalid(name, "not_a_boolean") case TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG => assertPropertyInvalid(name, "not_a_boolean")
case TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG => assertPropertyInvalid(name, "not_a_number", "-3") case TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG => assertPropertyInvalid(name, "not_a_number", "-3")
case TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG => assertPropertyInvalid(name, "not_a_number", "-3") case TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG => assertPropertyInvalid(name, "not_a_number", "-3")
case TopicConfig.COMPRESSION_GZIP_LEVEL_CONFIG => assertPropertyInvalid(name, "not_a_number", "-2")
case TopicConfig.COMPRESSION_LZ4_LEVEL_CONFIG => assertPropertyInvalid(name, "not_a_number", "-1")
case TopicConfig.COMPRESSION_ZSTD_LEVEL_CONFIG => assertPropertyInvalid(name, "not_a_number", "-0.1")
case _ => assertPropertyInvalid(name, "not_a_number", "-1") case _ => assertPropertyInvalid(name, "not_a_number", "-1")
}) })
@ -215,7 +218,7 @@ class LogConfigTest {
values.foreach(value => { values.foreach(value => {
val props = new Properties val props = new Properties
props.setProperty(name, value.toString) props.setProperty(name, value.toString)
assertThrows(classOf[Exception], () => new LogConfig(props)) assertThrows(classOf[Exception], () => new LogConfig(props), () => s"Property $name should not allow $value")
}) })
} }

View File

@ -25,9 +25,10 @@ import kafka.server.{BrokerTopicStats, KafkaConfig}
import kafka.server.metadata.MockConfigRepository import kafka.server.metadata.MockConfigRepository
import kafka.utils.TestUtils import kafka.utils.TestUtils
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.errors.KafkaStorageException import org.apache.kafka.common.errors.KafkaStorageException
import org.apache.kafka.common.record.{CompressionType, ControlRecordType, DefaultRecordBatch, MemoryRecords, RecordBatch, RecordVersion, SimpleRecord, TimestampType} import org.apache.kafka.common.record.{ControlRecordType, DefaultRecordBatch, MemoryRecords, RecordBatch, RecordVersion, SimpleRecord, TimestampType}
import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.utils.{Time, Utils}
import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion
@ -294,7 +295,7 @@ class LogLoaderTest {
key: Array[Byte] = null, key: Array[Byte] = null,
leaderEpoch: Int, leaderEpoch: Int,
offset: Long, offset: Long,
codec: CompressionType = CompressionType.NONE, codec: Compression = Compression.NONE,
timestamp: Long = RecordBatch.NO_TIMESTAMP, timestamp: Long = RecordBatch.NO_TIMESTAMP,
magicValue: Byte = RecordBatch.CURRENT_MAGIC_VALUE): MemoryRecords = { magicValue: Byte = RecordBatch.CURRENT_MAGIC_VALUE): MemoryRecords = {
val records = Seq(new SimpleRecord(timestamp, key, value)) val records = Seq(new SimpleRecord(timestamp, key, value))
@ -1063,10 +1064,10 @@ class LogLoaderTest {
// append some messages to create some segments // append some messages to create some segments
val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024)
val log = createLog(logDir, logConfig) val log = createLog(logDir, logConfig)
val set1 = MemoryRecords.withRecords(0, CompressionType.NONE, 0, new SimpleRecord("v1".getBytes(), "k1".getBytes())) val set1 = MemoryRecords.withRecords(0, Compression.NONE, 0, new SimpleRecord("v1".getBytes(), "k1".getBytes()))
val set2 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 2, CompressionType.NONE, 0, new SimpleRecord("v3".getBytes(), "k3".getBytes())) val set2 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 2, Compression.NONE, 0, new SimpleRecord("v3".getBytes(), "k3".getBytes()))
val set3 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 3, CompressionType.NONE, 0, new SimpleRecord("v4".getBytes(), "k4".getBytes())) val set3 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 3, Compression.NONE, 0, new SimpleRecord("v4".getBytes(), "k4".getBytes()))
val set4 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 4, CompressionType.NONE, 0, new SimpleRecord("v5".getBytes(), "k5".getBytes())) val set4 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 4, Compression.NONE, 0, new SimpleRecord("v5".getBytes(), "k5".getBytes()))
//Writes into an empty log with baseOffset 0 //Writes into an empty log with baseOffset 0
log.appendAsFollower(set1) log.appendAsFollower(set1)
assertEquals(0L, log.activeSegment.baseOffset) assertEquals(0L, log.activeSegment.baseOffset)
@ -1120,14 +1121,14 @@ class LogLoaderTest {
// append some messages to create some segments // append some messages to create some segments
val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024)
val log = createLog(logDir, logConfig) val log = createLog(logDir, logConfig)
val set1 = MemoryRecords.withRecords(0, CompressionType.NONE, 0, new SimpleRecord("v1".getBytes(), "k1".getBytes())) val set1 = MemoryRecords.withRecords(0, Compression.NONE, 0, new SimpleRecord("v1".getBytes(), "k1".getBytes()))
val set2 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 2, CompressionType.GZIP, 0, val set2 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 2, Compression.gzip().build(), 0,
new SimpleRecord("v3".getBytes(), "k3".getBytes()), new SimpleRecord("v3".getBytes(), "k3".getBytes()),
new SimpleRecord("v4".getBytes(), "k4".getBytes())) new SimpleRecord("v4".getBytes(), "k4".getBytes()))
val set3 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 4, CompressionType.GZIP, 0, val set3 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 4, Compression.gzip().build(), 0,
new SimpleRecord("v5".getBytes(), "k5".getBytes()), new SimpleRecord("v5".getBytes(), "k5".getBytes()),
new SimpleRecord("v6".getBytes(), "k6".getBytes())) new SimpleRecord("v6".getBytes(), "k6".getBytes()))
val set4 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 6, CompressionType.GZIP, 0, val set4 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 6, Compression.gzip().build(), 0,
new SimpleRecord("v7".getBytes(), "k7".getBytes()), new SimpleRecord("v7".getBytes(), "k7".getBytes()),
new SimpleRecord("v8".getBytes(), "k8".getBytes())) new SimpleRecord("v8".getBytes(), "k8".getBytes()))
//Writes into an empty log with baseOffset 0 //Writes into an empty log with baseOffset 0
@ -1159,15 +1160,15 @@ class LogLoaderTest {
// append some messages to create some segments // append some messages to create some segments
val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024) val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024)
val log = createLog(logDir, logConfig) val log = createLog(logDir, logConfig)
val set1 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, 0, CompressionType.NONE, val set1 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, 0, Compression.NONE,
new SimpleRecord("v1".getBytes(), "k1".getBytes())) new SimpleRecord("v1".getBytes(), "k1".getBytes()))
val set2 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, Integer.MAX_VALUE.toLong + 2, CompressionType.GZIP, val set2 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, Integer.MAX_VALUE.toLong + 2, Compression.gzip().build(),
new SimpleRecord("v3".getBytes(), "k3".getBytes()), new SimpleRecord("v3".getBytes(), "k3".getBytes()),
new SimpleRecord("v4".getBytes(), "k4".getBytes())) new SimpleRecord("v4".getBytes(), "k4".getBytes()))
val set3 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, Integer.MAX_VALUE.toLong + 4, CompressionType.GZIP, val set3 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, Integer.MAX_VALUE.toLong + 4, Compression.gzip().build(),
new SimpleRecord("v5".getBytes(), "k5".getBytes()), new SimpleRecord("v5".getBytes(), "k5".getBytes()),
new SimpleRecord("v6".getBytes(), "k6".getBytes())) new SimpleRecord("v6".getBytes(), "k6".getBytes()))
val set4 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, Integer.MAX_VALUE.toLong + 6, CompressionType.GZIP, val set4 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, Integer.MAX_VALUE.toLong + 6, Compression.gzip().build(),
new SimpleRecord("v7".getBytes(), "k7".getBytes()), new SimpleRecord("v7".getBytes(), "k7".getBytes()),
new SimpleRecord("v8".getBytes(), "k8".getBytes())) new SimpleRecord("v8".getBytes(), "k8".getBytes()))
//Writes into an empty log with baseOffset 0 //Writes into an empty log with baseOffset 0

View File

@ -19,6 +19,7 @@ package kafka.log
import kafka.utils.TestUtils import kafka.utils.TestUtils
import kafka.utils.TestUtils.random import kafka.utils.TestUtils.random
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.record._ import org.apache.kafka.common.record._
import org.apache.kafka.common.utils.{MockTime, Time, Utils} import org.apache.kafka.common.utils.{MockTime, Time, Utils}
@ -53,7 +54,7 @@ class LogSegmentTest {
/* create a ByteBufferMessageSet for the given messages starting from the given offset */ /* create a ByteBufferMessageSet for the given messages starting from the given offset */
def records(offset: Long, records: String*): MemoryRecords = { def records(offset: Long, records: String*): MemoryRecords = {
MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, offset, CompressionType.NONE, TimestampType.CREATE_TIME, MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, offset, Compression.NONE, TimestampType.CREATE_TIME,
records.map { s => new SimpleRecord(offset * 10, s.getBytes) }: _*) records.map { s => new SimpleRecord(offset * 10, s.getBytes) }: _*)
} }
@ -347,15 +348,15 @@ class LogSegmentTest {
// append transactional records from pid1 // append transactional records from pid1
segment.append(101L, RecordBatch.NO_TIMESTAMP, segment.append(101L, RecordBatch.NO_TIMESTAMP,
100L, MemoryRecords.withTransactionalRecords(100L, CompressionType.NONE, 100L, MemoryRecords.withTransactionalRecords(100L, Compression.NONE,
pid1, producerEpoch, sequence, partitionLeaderEpoch, new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes))) pid1, producerEpoch, sequence, partitionLeaderEpoch, new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes)))
// append transactional records from pid2 // append transactional records from pid2
segment.append(103L, RecordBatch.NO_TIMESTAMP, 102L, MemoryRecords.withTransactionalRecords(102L, CompressionType.NONE, segment.append(103L, RecordBatch.NO_TIMESTAMP, 102L, MemoryRecords.withTransactionalRecords(102L, Compression.NONE,
pid2, producerEpoch, sequence, partitionLeaderEpoch, new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes))) pid2, producerEpoch, sequence, partitionLeaderEpoch, new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes)))
// append non-transactional records // append non-transactional records
segment.append(105L, RecordBatch.NO_TIMESTAMP, 104L, MemoryRecords.withRecords(104L, CompressionType.NONE, segment.append(105L, RecordBatch.NO_TIMESTAMP, 104L, MemoryRecords.withRecords(104L, Compression.NONE,
partitionLeaderEpoch, new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes))) partitionLeaderEpoch, new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes)))
// abort the transaction from pid2 (note LSO should be 100L since the txn from pid1 has not completed) // abort the transaction from pid2 (note LSO should be 100L since the txn from pid1 has not completed)
@ -413,16 +414,16 @@ class LogSegmentTest {
} }
val cache = new LeaderEpochFileCache(topicPartition, checkpoint) val cache = new LeaderEpochFileCache(topicPartition, checkpoint)
seg.append(105L, RecordBatch.NO_TIMESTAMP, 104L, MemoryRecords.withRecords(104L, CompressionType.NONE, 0, seg.append(105L, RecordBatch.NO_TIMESTAMP, 104L, MemoryRecords.withRecords(104L, Compression.NONE, 0,
new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes))) new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes)))
seg.append(107L, RecordBatch.NO_TIMESTAMP, 106L, MemoryRecords.withRecords(106L, CompressionType.NONE, 1, seg.append(107L, RecordBatch.NO_TIMESTAMP, 106L, MemoryRecords.withRecords(106L, Compression.NONE, 1,
new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes))) new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes)))
seg.append(109L, RecordBatch.NO_TIMESTAMP, 108L, MemoryRecords.withRecords(108L, CompressionType.NONE, 1, seg.append(109L, RecordBatch.NO_TIMESTAMP, 108L, MemoryRecords.withRecords(108L, Compression.NONE, 1,
new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes))) new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes)))
seg.append(111L, RecordBatch.NO_TIMESTAMP, 110, MemoryRecords.withRecords(110L, CompressionType.NONE, 2, seg.append(111L, RecordBatch.NO_TIMESTAMP, 110, MemoryRecords.withRecords(110L, Compression.NONE, 2,
new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes))) new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes)))
seg.recover(newProducerStateManager(), Optional.of(cache)) seg.recover(newProducerStateManager(), Optional.of(cache))
@ -555,7 +556,7 @@ class LogSegmentTest {
val offset = 40 val offset = 40
def records(offset: Long, record: String): MemoryRecords = def records(offset: Long, record: String): MemoryRecords =
MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, offset, CompressionType.NONE, TimestampType.CREATE_TIME, MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, offset, Compression.NONE, TimestampType.CREATE_TIME,
new SimpleRecord(offset * 1000, record.getBytes)) new SimpleRecord(offset * 1000, record.getBytes))
//Given two messages with a gap between them (e.g. mid offset compacted away) //Given two messages with a gap between them (e.g. mid offset compacted away)
@ -576,7 +577,7 @@ class LogSegmentTest {
@Test @Test
def testAppendFromFile(): Unit = { def testAppendFromFile(): Unit = {
def records(offset: Long, size: Int): MemoryRecords = def records(offset: Long, size: Int): MemoryRecords =
MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, offset, CompressionType.NONE, TimestampType.CREATE_TIME, MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, offset, Compression.NONE, TimestampType.CREATE_TIME,
new SimpleRecord(new Array[Byte](size))) new SimpleRecord(new Array[Byte](size)))
// create a log file in a separate directory to avoid conflicting with created segments // create a log file in a separate directory to avoid conflicting with created segments

View File

@ -24,7 +24,8 @@ import java.util.Properties
import kafka.server.BrokerTopicStats import kafka.server.BrokerTopicStats
import kafka.utils.TestUtils import kafka.utils.TestUtils
import org.apache.kafka.common.Uuid import org.apache.kafka.common.Uuid
import org.apache.kafka.common.record.{CompressionType, ControlRecordType, EndTransactionMarker, FileRecords, MemoryRecords, RecordBatch, SimpleRecord} import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.record.{ControlRecordType, EndTransactionMarker, FileRecords, MemoryRecords, RecordBatch, SimpleRecord}
import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.utils.{Time, Utils}
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse}
@ -156,12 +157,12 @@ object LogTestUtils {
new SimpleRecord(data, data) new SimpleRecord(data, data)
} }
segment.append(MemoryRecords.withRecords(baseOffset, CompressionType.NONE, 0, segment.append(MemoryRecords.withRecords(baseOffset, Compression.NONE, 0,
record(baseOffset))) record(baseOffset)))
segment.append(MemoryRecords.withRecords(baseOffset + 1, CompressionType.NONE, 0, segment.append(MemoryRecords.withRecords(baseOffset + 1, Compression.NONE, 0,
record(baseOffset + 1), record(baseOffset + 1),
record(baseOffset + 2))) record(baseOffset + 2)))
segment.append(MemoryRecords.withRecords(baseOffset + Int.MaxValue - 1, CompressionType.NONE, 0, segment.append(MemoryRecords.withRecords(baseOffset + Int.MaxValue - 1, Compression.NONE, 0,
record(baseOffset + Int.MaxValue - 1))) record(baseOffset + Int.MaxValue - 1)))
// Need to create the offset files explicitly to avoid triggering segment recovery to truncate segment. // Need to create the offset files explicitly to avoid triggering segment recovery to truncate segment.
Files.createFile(LogFileUtils.offsetIndexFile(logDir, baseOffset).toPath) Files.createFile(LogFileUtils.offsetIndexFile(logDir, baseOffset).toPath)
@ -265,7 +266,7 @@ object LogTestUtils {
val simpleRecords = (0 until numRecords).map { seq => val simpleRecords = (0 until numRecords).map { seq =>
new SimpleRecord(s"$seq".getBytes) new SimpleRecord(s"$seq".getBytes)
} }
val records = MemoryRecords.withRecords(CompressionType.NONE, simpleRecords: _*) val records = MemoryRecords.withRecords(Compression.NONE, simpleRecords: _*)
log.appendAsLeader(records, leaderEpoch = 0) log.appendAsLeader(records, leaderEpoch = 0)
} }
@ -288,10 +289,10 @@ object LogTestUtils {
} }
val records = if (isTransactional) { val records = if (isTransactional) {
MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, MemoryRecords.withTransactionalRecords(Compression.NONE, producerId,
producerEpoch, sequence, simpleRecords: _*) producerEpoch, sequence, simpleRecords: _*)
} else { } else {
MemoryRecords.withIdempotentRecords(CompressionType.NONE, producerId, MemoryRecords.withIdempotentRecords(Compression.NONE, producerId,
producerEpoch, sequence, simpleRecords: _*) producerEpoch, sequence, simpleRecords: _*)
} }

View File

@ -20,6 +20,7 @@ import java.nio.ByteBuffer
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import kafka.server.{BrokerTopicStats, RequestLocal} import kafka.server.{BrokerTopicStats, RequestLocal}
import kafka.utils.TestUtils.meterCount import kafka.utils.TestUtils.meterCount
import org.apache.kafka.common.compress.{Compression, GzipCompression, Lz4Compression}
import org.apache.kafka.common.errors.{InvalidTimestampException, UnsupportedCompressionTypeException, UnsupportedForMessageFormatException} import org.apache.kafka.common.errors.{InvalidTimestampException, UnsupportedCompressionTypeException, UnsupportedForMessageFormatException}
import org.apache.kafka.common.record._ import org.apache.kafka.common.record._
import org.apache.kafka.common.utils.{PrimitiveRef, Time} import org.apache.kafka.common.utils.{PrimitiveRef, Time}
@ -44,38 +45,39 @@ class LogValidatorTest {
@Test @Test
def testOnlyOneBatch(): Unit = { def testOnlyOneBatch(): Unit = {
checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V0, CompressionType.GZIP, CompressionType.GZIP) checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V0, Compression.gzip().build(), Compression.gzip().build())
checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V1, CompressionType.GZIP, CompressionType.GZIP) checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V1, Compression.gzip().build(), Compression.gzip().build())
checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V2, CompressionType.GZIP, CompressionType.GZIP) checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V2, Compression.gzip().build(), Compression.gzip().build())
checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V0, CompressionType.GZIP, CompressionType.NONE) checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V0, Compression.gzip().build(), Compression.NONE)
checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V1, CompressionType.GZIP, CompressionType.NONE) checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V1, Compression.gzip().build(), Compression.NONE)
checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V2, CompressionType.GZIP, CompressionType.NONE) checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V2, Compression.gzip().build(), Compression.NONE)
checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE, CompressionType.NONE) checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V2, Compression.NONE, Compression.NONE)
checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE, CompressionType.GZIP) checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V2, Compression.NONE, Compression.gzip().build())
} }
@Test @Test
def testAllowMultiBatch(): Unit = { def testAllowMultiBatch(): Unit = {
checkAllowMultiBatch(RecordBatch.MAGIC_VALUE_V0, CompressionType.NONE, CompressionType.NONE) checkAllowMultiBatch(RecordBatch.MAGIC_VALUE_V0, Compression.NONE, Compression.NONE)
checkAllowMultiBatch(RecordBatch.MAGIC_VALUE_V1, CompressionType.NONE, CompressionType.NONE) checkAllowMultiBatch(RecordBatch.MAGIC_VALUE_V1, Compression.NONE, Compression.NONE)
checkAllowMultiBatch(RecordBatch.MAGIC_VALUE_V0, CompressionType.NONE, CompressionType.GZIP) checkAllowMultiBatch(RecordBatch.MAGIC_VALUE_V0, Compression.NONE, Compression.gzip().build())
checkAllowMultiBatch(RecordBatch.MAGIC_VALUE_V1, CompressionType.NONE, CompressionType.GZIP) checkAllowMultiBatch(RecordBatch.MAGIC_VALUE_V1, Compression.NONE, Compression.gzip().build())
} }
@Test @Test
def testValidationOfBatchesWithNonSequentialInnerOffsets(): Unit = { def testValidationOfBatchesWithNonSequentialInnerOffsets(): Unit = {
def testMessageValidation(magicValue: Byte): Unit = { def testMessageValidation(magicValue: Byte): Unit = {
val numRecords = 20 val numRecords = 20
val invalidRecords = recordsWithNonSequentialInnerOffsets(magicValue, CompressionType.GZIP, numRecords) val compression: Compression = Compression.gzip().build()
val invalidRecords = recordsWithNonSequentialInnerOffsets(magicValue, compression, numRecords)
// Validation for v2 and above is strict for this case. For older formats, we fix invalid // Validation for v2 and above is strict for this case. For older formats, we fix invalid
// internal offsets by rewriting the batch. // internal offsets by rewriting the batch.
if (magicValue >= RecordBatch.MAGIC_VALUE_V2) { if (magicValue >= RecordBatch.MAGIC_VALUE_V2) {
assertThrows(classOf[InvalidRecordException], assertThrows(classOf[InvalidRecordException],
() => validateMessages(invalidRecords, magicValue, CompressionType.GZIP, CompressionType.GZIP) () => validateMessages(invalidRecords, magicValue, CompressionType.GZIP, compression)
) )
} else { } else {
val result = validateMessages(invalidRecords, magicValue, CompressionType.GZIP, CompressionType.GZIP) val result = validateMessages(invalidRecords, magicValue, CompressionType.GZIP, compression)
assertEquals(0 until numRecords, result.validatedRecords.records.asScala.map(_.offset)) assertEquals(0 until numRecords, result.validatedRecords.records.asScala.map(_.offset))
} }
} }
@ -87,23 +89,24 @@ class LogValidatorTest {
@Test @Test
def testMisMatchMagic(): Unit = { def testMisMatchMagic(): Unit = {
checkMismatchMagic(RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V1, CompressionType.GZIP) val compression: Compression = Compression.gzip().build()
checkMismatchMagic(RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V0, CompressionType.GZIP) checkMismatchMagic(RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V1, compression)
checkMismatchMagic(RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V0, compression)
} }
private def checkOnlyOneBatch(magic: Byte, sourceCompressionType: CompressionType, targetCompressionType: CompressionType): Unit = { private def checkOnlyOneBatch(magic: Byte, sourceCompression: Compression, targetCompression: Compression): Unit = {
assertThrows(classOf[InvalidRecordException], assertThrows(classOf[InvalidRecordException],
() => validateMessages(createTwoBatchedRecords(magic, 0L, sourceCompressionType), magic, sourceCompressionType, targetCompressionType) () => validateMessages(createTwoBatchedRecords(magic, sourceCompression), magic, sourceCompression.`type`(), targetCompression)
) )
} }
private def checkAllowMultiBatch(magic: Byte, sourceCompressionType: CompressionType, targetCompressionType: CompressionType): Unit = { private def checkAllowMultiBatch(magic: Byte, sourceCompression: Compression, targetCompression: Compression): Unit = {
validateMessages(createTwoBatchedRecords(magic, 0L, sourceCompressionType), magic, sourceCompressionType, targetCompressionType) validateMessages(createTwoBatchedRecords(magic, sourceCompression), magic, sourceCompression.`type`(), targetCompression)
} }
private def checkMismatchMagic(batchMagic: Byte, recordMagic: Byte, compressionType: CompressionType): Unit = { private def checkMismatchMagic(batchMagic: Byte, recordMagic: Byte, compression: Compression): Unit = {
assertThrows(classOf[RecordValidationException], assertThrows(classOf[RecordValidationException],
() => validateMessages(recordsWithInvalidInnerMagic(batchMagic, recordMagic, compressionType), batchMagic, compressionType, compressionType) () => validateMessages(recordsWithInvalidInnerMagic(batchMagic, recordMagic, compression), batchMagic, compression.`type`(), compression)
) )
assertEquals(metricsKeySet.count(_.getMBeanName.endsWith(s"${BrokerTopicStats.InvalidMagicNumberRecordsPerSec}")), 1) assertEquals(metricsKeySet.count(_.getMBeanName.endsWith(s"${BrokerTopicStats.InvalidMagicNumberRecordsPerSec}")), 1)
assertTrue(meterCount(s"${BrokerTopicStats.InvalidMagicNumberRecordsPerSec}") > 0) assertTrue(meterCount(s"${BrokerTopicStats.InvalidMagicNumberRecordsPerSec}") > 0)
@ -112,13 +115,13 @@ class LogValidatorTest {
private def validateMessages(records: MemoryRecords, private def validateMessages(records: MemoryRecords,
magic: Byte, magic: Byte,
sourceCompressionType: CompressionType, sourceCompressionType: CompressionType,
targetCompressionType: CompressionType): ValidationResult = { targetCompression: Compression): ValidationResult = {
val mockTime = new MockTime(0L, 0L) val mockTime = new MockTime(0L, 0L)
new LogValidator(records, new LogValidator(records,
topicPartition, topicPartition,
mockTime, mockTime,
sourceCompressionType, sourceCompressionType,
targetCompressionType, targetCompression,
false, false,
magic, magic,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
@ -150,13 +153,13 @@ class LogValidatorTest {
private def checkLogAppendTimeNonCompressed(magic: Byte): Unit = { private def checkLogAppendTimeNonCompressed(magic: Byte): Unit = {
val mockTime = new MockTime val mockTime = new MockTime
// The timestamps should be overwritten // The timestamps should be overwritten
val records = createRecords(magicValue = magic, timestamp = 1234L, codec = CompressionType.NONE) val records = createRecords(magicValue = magic, timestamp = 1234L, codec = Compression.NONE)
val offsetCounter = PrimitiveRef.ofLong(0) val offsetCounter = PrimitiveRef.ofLong(0)
val validatedResults = new LogValidator(records, val validatedResults = new LogValidator(records,
topicPartition, topicPartition,
mockTime, mockTime,
CompressionType.NONE, CompressionType.NONE,
CompressionType.NONE, Compression.NONE,
false, false,
magic, magic,
TimestampType.LOG_APPEND_TIME, TimestampType.LOG_APPEND_TIME,
@ -197,15 +200,16 @@ class LogValidatorTest {
} }
private def checkLogAppendTimeWithRecompression(targetMagic: Byte): Unit = { private def checkLogAppendTimeWithRecompression(targetMagic: Byte): Unit = {
val compression: Compression = Compression.gzip().build()
val mockTime = new MockTime val mockTime = new MockTime
// The timestamps should be overwritten // The timestamps should be overwritten
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.GZIP) val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = compression)
val validatedResults = new LogValidator( val validatedResults = new LogValidator(
records, records,
topicPartition, topicPartition,
mockTime, mockTime,
CompressionType.GZIP, CompressionType.GZIP,
CompressionType.GZIP, compression,
false, false,
targetMagic, targetMagic,
TimestampType.LOG_APPEND_TIME, TimestampType.LOG_APPEND_TIME,
@ -249,15 +253,16 @@ class LogValidatorTest {
} }
private def checkLogAppendTimeWithoutRecompression(magic: Byte): Unit = { private def checkLogAppendTimeWithoutRecompression(magic: Byte): Unit = {
val compression: Compression = Compression.gzip().build()
val mockTime = new MockTime val mockTime = new MockTime
// The timestamps should be overwritten // The timestamps should be overwritten
val records = createRecords(magicValue = magic, timestamp = 1234L, codec = CompressionType.GZIP) val records = createRecords(magicValue = magic, timestamp = 1234L, codec = compression)
val validatedResults = new LogValidator( val validatedResults = new LogValidator(
records, records,
topicPartition, topicPartition,
mockTime, mockTime,
CompressionType.GZIP, CompressionType.GZIP,
CompressionType.GZIP, compression,
false, false,
magic, magic,
TimestampType.LOG_APPEND_TIME, TimestampType.LOG_APPEND_TIME,
@ -315,7 +320,7 @@ class LogValidatorTest {
} }
private def validateRecordBatchWithCountOverrides(lastOffsetDelta: Int, count: Int): Unit = { private def validateRecordBatchWithCountOverrides(lastOffsetDelta: Int, count: Int): Unit = {
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = 1234L, codec = CompressionType.NONE) val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = 1234L, codec = Compression.NONE)
records.buffer.putInt(DefaultRecordBatch.RECORDS_COUNT_OFFSET, count) records.buffer.putInt(DefaultRecordBatch.RECORDS_COUNT_OFFSET, count)
records.buffer.putInt(DefaultRecordBatch.LAST_OFFSET_DELTA_OFFSET, lastOffsetDelta) records.buffer.putInt(DefaultRecordBatch.LAST_OFFSET_DELTA_OFFSET, lastOffsetDelta)
new LogValidator( new LogValidator(
@ -323,7 +328,7 @@ class LogValidatorTest {
topicPartition, topicPartition,
time, time,
CompressionType.GZIP, CompressionType.GZIP,
CompressionType.GZIP, Compression.gzip().build(),
false, false,
RecordBatch.MAGIC_VALUE_V2, RecordBatch.MAGIC_VALUE_V2,
TimestampType.LOG_APPEND_TIME, TimestampType.LOG_APPEND_TIME,
@ -367,7 +372,7 @@ class LogValidatorTest {
new SimpleRecord(timestampSeq(2), "beautiful".getBytes) new SimpleRecord(timestampSeq(2), "beautiful".getBytes)
) )
val records = MemoryRecords.withRecords(magic, 0L, CompressionType.NONE, TimestampType.CREATE_TIME, producerId, val records = MemoryRecords.withRecords(magic, 0L, Compression.NONE, TimestampType.CREATE_TIME, producerId,
producerEpoch, baseSequence, partitionLeaderEpoch, isTransactional, recordList: _*) producerEpoch, baseSequence, partitionLeaderEpoch, isTransactional, recordList: _*)
val offsetCounter = PrimitiveRef.ofLong(0) val offsetCounter = PrimitiveRef.ofLong(0)
@ -375,7 +380,7 @@ class LogValidatorTest {
topicPartition, topicPartition,
time, time,
CompressionType.NONE, CompressionType.NONE,
CompressionType.NONE, Compression.NONE,
false, false,
magic, magic,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
@ -452,7 +457,7 @@ class LogValidatorTest {
(RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, false, (RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, false,
RecordBatch.NO_PARTITION_LEADER_EPOCH) RecordBatch.NO_PARTITION_LEADER_EPOCH)
val records = MemoryRecords.withRecords(magic, 0L, CompressionType.NONE, TimestampType.CREATE_TIME, producerId, val records = MemoryRecords.withRecords(magic, 0L, Compression.NONE, TimestampType.CREATE_TIME, producerId,
producerEpoch, baseSequence, partitionLeaderEpoch, isTransactional, producerEpoch, baseSequence, partitionLeaderEpoch, isTransactional,
new SimpleRecord(timestampSeq(0), "hello".getBytes), new SimpleRecord(timestampSeq(0), "hello".getBytes),
new SimpleRecord(timestampSeq(1), "there".getBytes), new SimpleRecord(timestampSeq(1), "there".getBytes),
@ -465,7 +470,7 @@ class LogValidatorTest {
topicPartition, topicPartition,
time, time,
CompressionType.NONE, CompressionType.NONE,
CompressionType.GZIP, Compression.gzip().build(),
false, false,
magic, magic,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
@ -522,12 +527,13 @@ class LogValidatorTest {
} }
private def checkCreateTimeUpConversionFromV0(toMagic: Byte): Unit = { private def checkCreateTimeUpConversionFromV0(toMagic: Byte): Unit = {
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.GZIP) val compression: Compression = Compression.gzip().build()
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = compression)
val validatedResults = new LogValidator(records, val validatedResults = new LogValidator(records,
topicPartition, topicPartition,
time, time,
CompressionType.GZIP, CompressionType.GZIP,
CompressionType.GZIP, compression,
false, false,
toMagic, toMagic,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
@ -569,12 +575,13 @@ class LogValidatorTest {
@Test @Test
def testCreateTimeUpConversionV1ToV2(): Unit = { def testCreateTimeUpConversionV1ToV2(): Unit = {
val timestamp = System.currentTimeMillis() val timestamp = System.currentTimeMillis()
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, codec = CompressionType.GZIP, timestamp = timestamp) val compression: Compression = Compression.gzip().build()
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, codec = compression, timestamp = timestamp)
val validatedResults = new LogValidator(records, val validatedResults = new LogValidator(records,
topicPartition, topicPartition,
time, time,
CompressionType.GZIP, CompressionType.GZIP,
CompressionType.GZIP, compression,
false, false,
RecordBatch.MAGIC_VALUE_V2, RecordBatch.MAGIC_VALUE_V2,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
@ -631,14 +638,14 @@ class LogValidatorTest {
new SimpleRecord(timestampSeq(2), "beautiful".getBytes) new SimpleRecord(timestampSeq(2), "beautiful".getBytes)
) )
val records = MemoryRecords.withRecords(magic, 0L, CompressionType.GZIP, TimestampType.CREATE_TIME, producerId, val records = MemoryRecords.withRecords(magic, 0L, Compression.gzip().build(), TimestampType.CREATE_TIME, producerId,
producerEpoch, baseSequence, partitionLeaderEpoch, isTransactional, recordList: _*) producerEpoch, baseSequence, partitionLeaderEpoch, isTransactional, recordList: _*)
val validatedResults = new LogValidator(records, val validatedResults = new LogValidator(records,
topicPartition, topicPartition,
time, time,
CompressionType.GZIP, CompressionType.GZIP,
CompressionType.GZIP, Compression.gzip().build(),
false, false,
magic, magic,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
@ -690,13 +697,13 @@ class LogValidatorTest {
def testInvalidCreateTimeNonCompressedV1(): Unit = { def testInvalidCreateTimeNonCompressedV1(): Unit = {
val now = System.currentTimeMillis() val now = System.currentTimeMillis()
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, timestamp = now - 1001L, val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, timestamp = now - 1001L,
codec = CompressionType.NONE) codec = Compression.NONE)
assertThrows(classOf[RecordValidationException], () => new LogValidator( assertThrows(classOf[RecordValidationException], () => new LogValidator(
records, records,
topicPartition, topicPartition,
time, time,
CompressionType.NONE, CompressionType.NONE,
CompressionType.NONE, Compression.NONE,
false, false,
RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V1,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
@ -716,13 +723,13 @@ class LogValidatorTest {
def testInvalidCreateTimeNonCompressedV2(): Unit = { def testInvalidCreateTimeNonCompressedV2(): Unit = {
val now = System.currentTimeMillis() val now = System.currentTimeMillis()
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = now - 1001L, val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = now - 1001L,
codec = CompressionType.NONE) codec = Compression.NONE)
assertThrows(classOf[RecordValidationException], () => new LogValidator( assertThrows(classOf[RecordValidationException], () => new LogValidator(
records, records,
topicPartition, topicPartition,
time, time,
CompressionType.NONE, CompressionType.NONE,
CompressionType.NONE, Compression.NONE,
false, false,
RecordBatch.MAGIC_VALUE_V2, RecordBatch.MAGIC_VALUE_V2,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
@ -741,14 +748,15 @@ class LogValidatorTest {
@Test @Test
def testInvalidCreateTimeCompressedV1(): Unit = { def testInvalidCreateTimeCompressedV1(): Unit = {
val now = System.currentTimeMillis() val now = System.currentTimeMillis()
val compression: Compression = Compression.gzip().build()
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, timestamp = now - 1001L, val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, timestamp = now - 1001L,
codec = CompressionType.GZIP) codec = compression)
assertThrows(classOf[RecordValidationException], () => new LogValidator( assertThrows(classOf[RecordValidationException], () => new LogValidator(
records, records,
topicPartition, topicPartition,
time, time,
CompressionType.GZIP, CompressionType.GZIP,
CompressionType.GZIP, compression,
false, false,
RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V1,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
@ -767,14 +775,15 @@ class LogValidatorTest {
@Test @Test
def testInvalidCreateTimeCompressedV2(): Unit = { def testInvalidCreateTimeCompressedV2(): Unit = {
val now = System.currentTimeMillis() val now = System.currentTimeMillis()
val compression: Compression = Compression.gzip().build()
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = now - 1001L, val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = now - 1001L,
codec = CompressionType.GZIP) codec = compression)
assertThrows(classOf[RecordValidationException], () => new LogValidator( assertThrows(classOf[RecordValidationException], () => new LogValidator(
records, records,
topicPartition, topicPartition,
time, time,
CompressionType.GZIP, CompressionType.GZIP,
CompressionType.GZIP, compression,
false, false,
RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V1,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
@ -792,14 +801,14 @@ class LogValidatorTest {
@Test @Test
def testAbsoluteOffsetAssignmentNonCompressed(): Unit = { def testAbsoluteOffsetAssignmentNonCompressed(): Unit = {
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.NONE) val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = Compression.NONE)
val offset = 1234567 val offset = 1234567
checkOffsets(records, 0) checkOffsets(records, 0)
checkOffsets(new LogValidator(records, checkOffsets(new LogValidator(records,
topicPartition, topicPartition,
time, time,
CompressionType.NONE, CompressionType.NONE,
CompressionType.NONE, Compression.NONE,
false, false,
RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V0,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
@ -815,14 +824,15 @@ class LogValidatorTest {
@Test @Test
def testAbsoluteOffsetAssignmentCompressed(): Unit = { def testAbsoluteOffsetAssignmentCompressed(): Unit = {
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.GZIP) val compression: Compression = Compression.gzip().build()
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = compression)
val offset = 1234567 val offset = 1234567
checkOffsets(records, 0) checkOffsets(records, 0)
checkOffsets(new LogValidator(records, checkOffsets(new LogValidator(records,
topicPartition, topicPartition,
time, time,
CompressionType.GZIP, CompressionType.GZIP,
CompressionType.GZIP, compression,
false, false,
RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V0,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
@ -839,14 +849,14 @@ class LogValidatorTest {
@Test @Test
def testRelativeOffsetAssignmentNonCompressedV1(): Unit = { def testRelativeOffsetAssignmentNonCompressedV1(): Unit = {
val now = System.currentTimeMillis() val now = System.currentTimeMillis()
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, timestamp = now, codec = CompressionType.NONE) val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, timestamp = now, codec = Compression.NONE)
val offset = 1234567 val offset = 1234567
checkOffsets(records, 0) checkOffsets(records, 0)
val messageWithOffset = new LogValidator(records, val messageWithOffset = new LogValidator(records,
topicPartition, topicPartition,
time, time,
CompressionType.NONE, CompressionType.NONE,
CompressionType.NONE, Compression.NONE,
false, false,
RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V1,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
@ -864,14 +874,14 @@ class LogValidatorTest {
@Test @Test
def testRelativeOffsetAssignmentNonCompressedV2(): Unit = { def testRelativeOffsetAssignmentNonCompressedV2(): Unit = {
val now = System.currentTimeMillis() val now = System.currentTimeMillis()
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = now, codec = CompressionType.NONE) val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = now, codec = Compression.NONE)
val offset = 1234567 val offset = 1234567
checkOffsets(records, 0) checkOffsets(records, 0)
val messageWithOffset = new LogValidator(records, val messageWithOffset = new LogValidator(records,
topicPartition, topicPartition,
time, time,
CompressionType.NONE, CompressionType.NONE,
CompressionType.NONE, Compression.NONE,
false, false,
RecordBatch.MAGIC_VALUE_V2, RecordBatch.MAGIC_VALUE_V2,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
@ -889,7 +899,8 @@ class LogValidatorTest {
@Test @Test
def testRelativeOffsetAssignmentCompressedV1(): Unit = { def testRelativeOffsetAssignmentCompressedV1(): Unit = {
val now = System.currentTimeMillis() val now = System.currentTimeMillis()
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, timestamp = now, codec = CompressionType.GZIP) val compression: Compression = Compression.gzip().build()
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, timestamp = now, codec = compression)
val offset = 1234567 val offset = 1234567
checkOffsets(records, 0) checkOffsets(records, 0)
val compressedMessagesWithOffset = new LogValidator( val compressedMessagesWithOffset = new LogValidator(
@ -897,7 +908,7 @@ class LogValidatorTest {
topicPartition, topicPartition,
time, time,
CompressionType.GZIP, CompressionType.GZIP,
CompressionType.GZIP, compression,
false, false,
RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V1,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
@ -915,7 +926,8 @@ class LogValidatorTest {
@Test @Test
def testRelativeOffsetAssignmentCompressedV2(): Unit = { def testRelativeOffsetAssignmentCompressedV2(): Unit = {
val now = System.currentTimeMillis() val now = System.currentTimeMillis()
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = now, codec = CompressionType.GZIP) val compression: Compression = Compression.gzip().build()
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = now, codec = compression)
val offset = 1234567 val offset = 1234567
checkOffsets(records, 0) checkOffsets(records, 0)
val compressedMessagesWithOffset = new LogValidator( val compressedMessagesWithOffset = new LogValidator(
@ -923,7 +935,7 @@ class LogValidatorTest {
topicPartition, topicPartition,
time, time,
CompressionType.GZIP, CompressionType.GZIP,
CompressionType.GZIP, compression,
false, false,
RecordBatch.MAGIC_VALUE_V2, RecordBatch.MAGIC_VALUE_V2,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
@ -940,14 +952,14 @@ class LogValidatorTest {
@Test @Test
def testOffsetAssignmentAfterUpConversionV0ToV1NonCompressed(): Unit = { def testOffsetAssignmentAfterUpConversionV0ToV1NonCompressed(): Unit = {
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.NONE) val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = Compression.NONE)
checkOffsets(records, 0) checkOffsets(records, 0)
val offset = 1234567 val offset = 1234567
val validatedResults = new LogValidator(records, val validatedResults = new LogValidator(records,
topicPartition, topicPartition,
time, time,
CompressionType.NONE, CompressionType.NONE,
CompressionType.NONE, Compression.NONE,
false, false,
RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V1,
TimestampType.LOG_APPEND_TIME, TimestampType.LOG_APPEND_TIME,
@ -966,14 +978,14 @@ class LogValidatorTest {
@Test @Test
def testOffsetAssignmentAfterUpConversionV0ToV2NonCompressed(): Unit = { def testOffsetAssignmentAfterUpConversionV0ToV2NonCompressed(): Unit = {
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.NONE) val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = Compression.NONE)
checkOffsets(records, 0) checkOffsets(records, 0)
val offset = 1234567 val offset = 1234567
val validatedResults = new LogValidator(records, val validatedResults = new LogValidator(records,
topicPartition, topicPartition,
time, time,
CompressionType.NONE, CompressionType.NONE,
CompressionType.NONE, Compression.NONE,
false, false,
RecordBatch.MAGIC_VALUE_V2, RecordBatch.MAGIC_VALUE_V2,
TimestampType.LOG_APPEND_TIME, TimestampType.LOG_APPEND_TIME,
@ -992,14 +1004,15 @@ class LogValidatorTest {
@Test @Test
def testOffsetAssignmentAfterUpConversionV0ToV1Compressed(): Unit = { def testOffsetAssignmentAfterUpConversionV0ToV1Compressed(): Unit = {
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.GZIP) val compression: Compression = Compression.gzip().build()
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = compression)
val offset = 1234567 val offset = 1234567
checkOffsets(records, 0) checkOffsets(records, 0)
val validatedResults = new LogValidator(records, val validatedResults = new LogValidator(records,
topicPartition, topicPartition,
time, time,
CompressionType.GZIP, CompressionType.GZIP,
CompressionType.GZIP, compression,
false, false,
RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V1,
TimestampType.LOG_APPEND_TIME, TimestampType.LOG_APPEND_TIME,
@ -1018,14 +1031,15 @@ class LogValidatorTest {
@Test @Test
def testOffsetAssignmentAfterUpConversionV0ToV2Compressed(): Unit = { def testOffsetAssignmentAfterUpConversionV0ToV2Compressed(): Unit = {
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.GZIP) val compression: Compression = Compression.gzip().build()
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = compression)
val offset = 1234567 val offset = 1234567
checkOffsets(records, 0) checkOffsets(records, 0)
val validatedResults = new LogValidator(records, val validatedResults = new LogValidator(records,
topicPartition, topicPartition,
time, time,
CompressionType.GZIP, CompressionType.GZIP,
CompressionType.GZIP, compression,
false, false,
RecordBatch.MAGIC_VALUE_V2, RecordBatch.MAGIC_VALUE_V2,
TimestampType.LOG_APPEND_TIME, TimestampType.LOG_APPEND_TIME,
@ -1051,7 +1065,7 @@ class LogValidatorTest {
topicPartition, topicPartition,
time, time,
CompressionType.NONE, CompressionType.NONE,
CompressionType.NONE, Compression.NONE,
false, false,
RecordBatch.CURRENT_MAGIC_VALUE, RecordBatch.CURRENT_MAGIC_VALUE,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
@ -1074,7 +1088,7 @@ class LogValidatorTest {
topicPartition, topicPartition,
time, time,
CompressionType.NONE, CompressionType.NONE,
CompressionType.SNAPPY, Compression.snappy().build(),
false, false,
RecordBatch.CURRENT_MAGIC_VALUE, RecordBatch.CURRENT_MAGIC_VALUE,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
@ -1096,13 +1110,13 @@ class LogValidatorTest {
def testOffsetAssignmentAfterDownConversionV1ToV0NonCompressed(): Unit = { def testOffsetAssignmentAfterDownConversionV1ToV0NonCompressed(): Unit = {
val offset = 1234567 val offset = 1234567
val now = System.currentTimeMillis() val now = System.currentTimeMillis()
val records = createRecords(RecordBatch.MAGIC_VALUE_V1, now, codec = CompressionType.NONE) val records = createRecords(RecordBatch.MAGIC_VALUE_V1, now, codec = Compression.NONE)
checkOffsets(records, 0) checkOffsets(records, 0)
checkOffsets(new LogValidator(records, checkOffsets(new LogValidator(records,
topicPartition, topicPartition,
time, time,
CompressionType.NONE, CompressionType.NONE,
CompressionType.NONE, Compression.NONE,
false, false,
RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V0,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
@ -1120,13 +1134,14 @@ class LogValidatorTest {
def testOffsetAssignmentAfterDownConversionV1ToV0Compressed(): Unit = { def testOffsetAssignmentAfterDownConversionV1ToV0Compressed(): Unit = {
val offset = 1234567 val offset = 1234567
val now = System.currentTimeMillis() val now = System.currentTimeMillis()
val records = createRecords(RecordBatch.MAGIC_VALUE_V1, now, CompressionType.GZIP) val compression: Compression = Compression.gzip().build()
val records = createRecords(RecordBatch.MAGIC_VALUE_V1, now, compression)
checkOffsets(records, 0) checkOffsets(records, 0)
checkOffsets(new LogValidator(records, checkOffsets(new LogValidator(records,
topicPartition, topicPartition,
time, time,
CompressionType.GZIP, CompressionType.GZIP,
CompressionType.GZIP, compression,
false, false,
RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V0,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
@ -1142,14 +1157,14 @@ class LogValidatorTest {
@Test @Test
def testOffsetAssignmentAfterUpConversionV1ToV2NonCompressed(): Unit = { def testOffsetAssignmentAfterUpConversionV1ToV2NonCompressed(): Unit = {
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, codec = CompressionType.NONE) val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, codec = Compression.NONE)
checkOffsets(records, 0) checkOffsets(records, 0)
val offset = 1234567 val offset = 1234567
checkOffsets(new LogValidator(records, checkOffsets(new LogValidator(records,
topicPartition, topicPartition,
time, time,
CompressionType.NONE, CompressionType.NONE,
CompressionType.NONE, Compression.NONE,
false, false,
RecordBatch.MAGIC_VALUE_V2, RecordBatch.MAGIC_VALUE_V2,
TimestampType.LOG_APPEND_TIME, TimestampType.LOG_APPEND_TIME,
@ -1165,14 +1180,15 @@ class LogValidatorTest {
@Test @Test
def testOffsetAssignmentAfterUpConversionV1ToV2Compressed(): Unit = { def testOffsetAssignmentAfterUpConversionV1ToV2Compressed(): Unit = {
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, codec = CompressionType.GZIP) val compression: Compression = Compression.gzip().build()
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, codec = compression)
val offset = 1234567 val offset = 1234567
checkOffsets(records, 0) checkOffsets(records, 0)
checkOffsets(new LogValidator(records, checkOffsets(new LogValidator(records,
topicPartition, topicPartition,
time, time,
CompressionType.GZIP, CompressionType.GZIP,
CompressionType.GZIP, compression,
false, false,
RecordBatch.MAGIC_VALUE_V2, RecordBatch.MAGIC_VALUE_V2,
TimestampType.LOG_APPEND_TIME, TimestampType.LOG_APPEND_TIME,
@ -1190,13 +1206,13 @@ class LogValidatorTest {
def testOffsetAssignmentAfterDownConversionV2ToV1NonCompressed(): Unit = { def testOffsetAssignmentAfterDownConversionV2ToV1NonCompressed(): Unit = {
val offset = 1234567 val offset = 1234567
val now = System.currentTimeMillis() val now = System.currentTimeMillis()
val records = createRecords(RecordBatch.MAGIC_VALUE_V2, now, codec = CompressionType.NONE) val records = createRecords(RecordBatch.MAGIC_VALUE_V2, now, codec = Compression.NONE)
checkOffsets(records, 0) checkOffsets(records, 0)
checkOffsets(new LogValidator(records, checkOffsets(new LogValidator(records,
topicPartition, topicPartition,
time, time,
CompressionType.NONE, CompressionType.NONE,
CompressionType.NONE, Compression.NONE,
false, false,
RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V1,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
@ -1214,13 +1230,14 @@ class LogValidatorTest {
def testOffsetAssignmentAfterDownConversionV2ToV1Compressed(): Unit = { def testOffsetAssignmentAfterDownConversionV2ToV1Compressed(): Unit = {
val offset = 1234567 val offset = 1234567
val now = System.currentTimeMillis() val now = System.currentTimeMillis()
val records = createRecords(RecordBatch.MAGIC_VALUE_V2, now, CompressionType.GZIP) val compression: Compression = Compression.gzip().build()
val records = createRecords(RecordBatch.MAGIC_VALUE_V2, now, compression)
checkOffsets(records, 0) checkOffsets(records, 0)
checkOffsets(new LogValidator(records, checkOffsets(new LogValidator(records,
topicPartition, topicPartition,
time, time,
CompressionType.GZIP, CompressionType.GZIP,
CompressionType.GZIP, compression,
false, false,
RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V1,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
@ -1240,13 +1257,13 @@ class LogValidatorTest {
val producerId = 1344L val producerId = 1344L
val producerEpoch = 16.toShort val producerEpoch = 16.toShort
val sequence = 0 val sequence = 0
val records = MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, producerEpoch, sequence, val records = MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, producerEpoch, sequence,
new SimpleRecord("hello".getBytes), new SimpleRecord("there".getBytes), new SimpleRecord("beautiful".getBytes)) new SimpleRecord("hello".getBytes), new SimpleRecord("there".getBytes), new SimpleRecord("beautiful".getBytes))
assertThrows(classOf[UnsupportedForMessageFormatException], () => new LogValidator(records, assertThrows(classOf[UnsupportedForMessageFormatException], () => new LogValidator(records,
topicPartition, topicPartition,
time, time,
CompressionType.GZIP, CompressionType.GZIP,
CompressionType.GZIP, Compression.gzip().build(),
false, false,
RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V1,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
@ -1266,13 +1283,13 @@ class LogValidatorTest {
val producerId = 1344L val producerId = 1344L
val producerEpoch = 16.toShort val producerEpoch = 16.toShort
val sequence = 0 val sequence = 0
val records = MemoryRecords.withIdempotentRecords(CompressionType.NONE, producerId, producerEpoch, sequence, val records = MemoryRecords.withIdempotentRecords(Compression.NONE, producerId, producerEpoch, sequence,
new SimpleRecord("hello".getBytes), new SimpleRecord("there".getBytes), new SimpleRecord("beautiful".getBytes)) new SimpleRecord("hello".getBytes), new SimpleRecord("there".getBytes), new SimpleRecord("beautiful".getBytes))
assertThrows(classOf[UnsupportedForMessageFormatException], () => new LogValidator(records, assertThrows(classOf[UnsupportedForMessageFormatException], () => new LogValidator(records,
topicPartition, topicPartition,
time, time,
CompressionType.GZIP, CompressionType.GZIP,
CompressionType.GZIP, Compression.gzip().build(),
false, false,
RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V1,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
@ -1290,13 +1307,13 @@ class LogValidatorTest {
def testOffsetAssignmentAfterDownConversionV2ToV0NonCompressed(): Unit = { def testOffsetAssignmentAfterDownConversionV2ToV0NonCompressed(): Unit = {
val offset = 1234567 val offset = 1234567
val now = System.currentTimeMillis() val now = System.currentTimeMillis()
val records = createRecords(RecordBatch.MAGIC_VALUE_V2, now, codec = CompressionType.NONE) val records = createRecords(RecordBatch.MAGIC_VALUE_V2, now, codec = Compression.NONE)
checkOffsets(records, 0) checkOffsets(records, 0)
checkOffsets(new LogValidator(records, checkOffsets(new LogValidator(records,
topicPartition, topicPartition,
time, time,
CompressionType.NONE, CompressionType.NONE,
CompressionType.NONE, Compression.NONE,
false, false,
RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V0,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
@ -1314,13 +1331,14 @@ class LogValidatorTest {
def testOffsetAssignmentAfterDownConversionV2ToV0Compressed(): Unit = { def testOffsetAssignmentAfterDownConversionV2ToV0Compressed(): Unit = {
val offset = 1234567 val offset = 1234567
val now = System.currentTimeMillis() val now = System.currentTimeMillis()
val records = createRecords(RecordBatch.MAGIC_VALUE_V2, now, CompressionType.GZIP) val compression: Compression = Compression.gzip().build()
val records = createRecords(RecordBatch.MAGIC_VALUE_V2, now, compression)
checkOffsets(records, 0) checkOffsets(records, 0)
checkOffsets(new LogValidator(records, checkOffsets(new LogValidator(records,
topicPartition, topicPartition,
time, time,
CompressionType.GZIP, CompressionType.GZIP,
CompressionType.GZIP, compression,
false, false,
RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V0,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
@ -1342,12 +1360,12 @@ class LogValidatorTest {
topicPartition, topicPartition,
time, time,
CompressionType.GZIP, CompressionType.GZIP,
CompressionType.GZIP, Compression.gzip().build(),
false, false,
RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V0,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
5000L, 5000L,
5000L, 5000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH, RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT, AppendOrigin.CLIENT,
MetadataVersion.latestTesting MetadataVersion.latestTesting
@ -1360,18 +1378,18 @@ class LogValidatorTest {
@Test @Test
def testCompressedBatchWithoutRecordsNotAllowed(): Unit = { def testCompressedBatchWithoutRecordsNotAllowed(): Unit = {
testBatchWithoutRecordsNotAllowed(CompressionType.GZIP, CompressionType.GZIP) testBatchWithoutRecordsNotAllowed(CompressionType.GZIP, Compression.gzip().build())
} }
@Test @Test
def testZStdCompressedWithUnavailableIBPVersion(): Unit = { def testZStdCompressedWithUnavailableIBPVersion(): Unit = {
// The timestamps should be overwritten // The timestamps should be overwritten
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = 1234L, codec = CompressionType.NONE) val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = 1234L, codec = Compression.NONE)
assertThrows(classOf[UnsupportedCompressionTypeException], () => new LogValidator(records, assertThrows(classOf[UnsupportedCompressionTypeException], () => new LogValidator(records,
topicPartition, topicPartition,
time, time,
CompressionType.NONE, CompressionType.NONE,
CompressionType.ZSTD, Compression.zstd().build(),
false, false,
RecordBatch.MAGIC_VALUE_V2, RecordBatch.MAGIC_VALUE_V2,
TimestampType.LOG_APPEND_TIME, TimestampType.LOG_APPEND_TIME,
@ -1387,26 +1405,27 @@ class LogValidatorTest {
@Test @Test
def testUncompressedBatchWithoutRecordsNotAllowed(): Unit = { def testUncompressedBatchWithoutRecordsNotAllowed(): Unit = {
testBatchWithoutRecordsNotAllowed(CompressionType.NONE, CompressionType.NONE) testBatchWithoutRecordsNotAllowed(CompressionType.NONE, Compression.NONE)
} }
@Test @Test
def testRecompressedBatchWithoutRecordsNotAllowed(): Unit = { def testRecompressedBatchWithoutRecordsNotAllowed(): Unit = {
testBatchWithoutRecordsNotAllowed(CompressionType.NONE, CompressionType.GZIP) testBatchWithoutRecordsNotAllowed(CompressionType.NONE, Compression.gzip().build())
} }
@Test @Test
def testInvalidTimestampExceptionHasBatchIndex(): Unit = { def testInvalidTimestampExceptionHasBatchIndex(): Unit = {
val now = System.currentTimeMillis() val now = System.currentTimeMillis()
val compression: Compression = Compression.gzip().build()
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = now - 1001L, val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = now - 1001L,
codec = CompressionType.GZIP) codec = compression)
val e = assertThrows(classOf[RecordValidationException], val e = assertThrows(classOf[RecordValidationException],
() => new LogValidator( () => new LogValidator(
records, records,
topicPartition, topicPartition,
time, time,
CompressionType.GZIP, CompressionType.GZIP,
CompressionType.GZIP, compression,
false, false,
RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V1,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
@ -1428,9 +1447,12 @@ class LogValidatorTest {
@Test @Test
def testInvalidRecordExceptionHasBatchIndex(): Unit = { def testInvalidRecordExceptionHasBatchIndex(): Unit = {
val e = assertThrows(classOf[RecordValidationException], val e = assertThrows(classOf[RecordValidationException],
() => validateMessages(recordsWithInvalidInnerMagic( () => {
RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V1, CompressionType.GZIP), val compression: Compression = Compression.gzip().build()
RecordBatch.MAGIC_VALUE_V0, CompressionType.GZIP, CompressionType.GZIP) validateMessages(recordsWithInvalidInnerMagic(
RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V1, compression),
RecordBatch.MAGIC_VALUE_V0, CompressionType.GZIP, compression)
}
) )
assertTrue(e.invalidException.isInstanceOf[InvalidRecordException]) assertTrue(e.invalidException.isInstanceOf[InvalidRecordException])
@ -1442,12 +1464,13 @@ class LogValidatorTest {
@Test @Test
def testBatchWithInvalidRecordsAndInvalidTimestamp(): Unit = { def testBatchWithInvalidRecordsAndInvalidTimestamp(): Unit = {
val compression: Compression = Compression.gzip().build()
val records = (0 until 5).map(id => val records = (0 until 5).map(id =>
LegacyRecord.create(RecordBatch.MAGIC_VALUE_V0, 0L, null, id.toString.getBytes()) LegacyRecord.create(RecordBatch.MAGIC_VALUE_V0, 0L, null, id.toString.getBytes())
) )
val buffer = ByteBuffer.allocate(1024) val buffer = ByteBuffer.allocate(1024)
val builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V1, CompressionType.GZIP, val builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V1, compression,
TimestampType.CREATE_TIME, 0L) TimestampType.CREATE_TIME, 0L)
var offset = 0 var offset = 0
@ -1462,7 +1485,7 @@ class LogValidatorTest {
val e = assertThrows(classOf[RecordValidationException], val e = assertThrows(classOf[RecordValidationException],
() => validateMessages(invalidOffsetTimestampRecords, () => validateMessages(invalidOffsetTimestampRecords,
RecordBatch.MAGIC_VALUE_V0, CompressionType.GZIP, CompressionType.GZIP) RecordBatch.MAGIC_VALUE_V0, CompressionType.GZIP, compression)
) )
// if there is a mix of both regular InvalidRecordException and InvalidTimestampException, // if there is a mix of both regular InvalidRecordException and InvalidTimestampException,
// InvalidTimestampException takes precedence // InvalidTimestampException takes precedence
@ -1477,15 +1500,16 @@ class LogValidatorTest {
val timestampAfterMaxConfig = 1 * 60 * 60 * 1000L //1 hr val timestampAfterMaxConfig = 1 * 60 * 60 * 1000L //1 hr
val now = System.currentTimeMillis() val now = System.currentTimeMillis()
val fiveMinutesBeforeThreshold = now - timestampBeforeMaxConfig - (5 * 60 * 1000L) val fiveMinutesBeforeThreshold = now - timestampBeforeMaxConfig - (5 * 60 * 1000L)
val compression: Compression = Compression.gzip().build()
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = fiveMinutesBeforeThreshold, val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = fiveMinutesBeforeThreshold,
codec = CompressionType.GZIP) codec = compression)
val e = assertThrows(classOf[RecordValidationException], val e = assertThrows(classOf[RecordValidationException],
() => new LogValidator( () => new LogValidator(
records, records,
topicPartition, topicPartition,
time, time,
CompressionType.GZIP, CompressionType.GZIP,
CompressionType.GZIP, compression,
false, false,
RecordBatch.MAGIC_VALUE_V2, RecordBatch.MAGIC_VALUE_V2,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
@ -1511,15 +1535,16 @@ class LogValidatorTest {
val timestampAfterMaxConfig = 1 * 60 * 60 * 1000L //1 hr val timestampAfterMaxConfig = 1 * 60 * 60 * 1000L //1 hr
val now = System.currentTimeMillis() val now = System.currentTimeMillis()
val fiveMinutesAfterThreshold = now + timestampAfterMaxConfig + (5 * 60 * 1000L) val fiveMinutesAfterThreshold = now + timestampAfterMaxConfig + (5 * 60 * 1000L)
val compression: Compression = Compression.gzip().build()
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = fiveMinutesAfterThreshold, val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = fiveMinutesAfterThreshold,
codec = CompressionType.GZIP) codec = compression)
val e = assertThrows(classOf[RecordValidationException], val e = assertThrows(classOf[RecordValidationException],
() => new LogValidator( () => new LogValidator(
records, records,
topicPartition, topicPartition,
time, time,
CompressionType.GZIP, CompressionType.GZIP,
CompressionType.GZIP, compression,
false, false,
RecordBatch.MAGIC_VALUE_V2, RecordBatch.MAGIC_VALUE_V2,
TimestampType.CREATE_TIME, TimestampType.CREATE_TIME,
@ -1538,7 +1563,80 @@ class LogValidatorTest {
assertEquals(e.recordErrors.size, 3) assertEquals(e.recordErrors.size, 3)
} }
private def testBatchWithoutRecordsNotAllowed(sourceCompression: CompressionType, targetCompression: CompressionType): Unit = { @Test
def testDifferentLevelDoesNotCauseRecompression(): Unit = {
val records = List(
List.fill(256)("some").mkString("").getBytes,
List.fill(256)("data").mkString("").getBytes
)
// Records from the producer were created with gzip max level
val gzipMax: Compression = Compression.gzip().level(GzipCompression.MAX_LEVEL).build()
val recordsGzipMax = createRecords(records, RecordBatch.MAGIC_VALUE_V2, RecordBatch.NO_TIMESTAMP, gzipMax)
// The topic is configured with gzip min level
val gzipMin: Compression = Compression.gzip().level(GzipCompression.MIN_LEVEL).build()
val recordsGzipMin = createRecords(records, RecordBatch.MAGIC_VALUE_V2, RecordBatch.NO_TIMESTAMP, gzipMin)
// ensure data compressed with gzip max and min is different
assertNotEquals(recordsGzipMax, recordsGzipMin)
val validator = new LogValidator(recordsGzipMax,
topicPartition,
time,
gzipMax.`type`(),
gzipMin,
false,
RecordBatch.MAGIC_VALUE_V2,
TimestampType.CREATE_TIME,
5000L,
5000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting
)
val result = validator.validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(0L), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier
)
// ensure validated records have not been changed so they are the same as the producer records
assertEquals(recordsGzipMax, result.validatedRecords)
assertNotEquals(recordsGzipMin, result.validatedRecords)
}
@Test
def testDifferentCodecCausesRecompression(): Unit = {
val records = List(
List.fill(256)("some").mkString("").getBytes,
List.fill(256)("data").mkString("").getBytes
)
// Records from the producer were created with gzip max level
val gzipMax: Compression = Compression.gzip().level(GzipCompression.MAX_LEVEL).build()
val recordsGzipMax = createRecords(records, RecordBatch.MAGIC_VALUE_V2, RecordBatch.NO_TIMESTAMP, gzipMax)
// The topic is configured with lz4 min level
val lz4Min: Compression = Compression.lz4().level(Lz4Compression.MIN_LEVEL).build()
val recordsLz4Min = createRecords(records, RecordBatch.MAGIC_VALUE_V2, RecordBatch.NO_TIMESTAMP, lz4Min)
val validator = new LogValidator(recordsGzipMax,
topicPartition,
time,
gzipMax.`type`(),
lz4Min,
false,
RecordBatch.MAGIC_VALUE_V2,
TimestampType.CREATE_TIME,
5000L,
5000L,
RecordBatch.NO_PARTITION_LEADER_EPOCH,
AppendOrigin.CLIENT,
MetadataVersion.latestTesting
)
val result = validator.validateMessagesAndAssignOffsets(
PrimitiveRef.ofLong(0L), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier
)
// ensure validated records have been recompressed and match lz4 min level
assertEquals(recordsLz4Min, result.validatedRecords)
}
private def testBatchWithoutRecordsNotAllowed(sourceCompression: CompressionType, targetCompression: Compression): Unit = {
val offset = 1234567 val offset = 1234567
val (producerId, producerEpoch, baseSequence, isTransactional, partitionLeaderEpoch) = val (producerId, producerEpoch, baseSequence, isTransactional, partitionLeaderEpoch) =
(1324L, 10.toShort, 984, true, 40) (1324L, 10.toShort, 984, true, 40)
@ -1568,18 +1666,26 @@ class LogValidatorTest {
private def createRecords(magicValue: Byte, private def createRecords(magicValue: Byte,
timestamp: Long = RecordBatch.NO_TIMESTAMP, timestamp: Long = RecordBatch.NO_TIMESTAMP,
codec: CompressionType): MemoryRecords = { codec: Compression): MemoryRecords = {
val records = List("hello".getBytes, "there".getBytes, "beautiful".getBytes)
createRecords(records = records, magicValue = magicValue, timestamp = timestamp, codec = codec)
}
private def createRecords(records: List[Array[Byte]],
magicValue: Byte,
timestamp: Long,
codec: Compression): MemoryRecords = {
val buf = ByteBuffer.allocate(512) val buf = ByteBuffer.allocate(512)
val builder = MemoryRecords.builder(buf, magicValue, codec, TimestampType.CREATE_TIME, 0L) val builder = MemoryRecords.builder(buf, magicValue, codec, TimestampType.CREATE_TIME, 0L)
builder.appendWithOffset(0, timestamp, null, "hello".getBytes) records.indices.foreach { offset =>
builder.appendWithOffset(1, timestamp, null, "there".getBytes) builder.appendWithOffset(offset, timestamp, null, records(offset))
builder.appendWithOffset(2, timestamp, null, "beautiful".getBytes) }
builder.build() builder.build()
} }
private def createNonIncreasingOffsetRecords(magicValue: Byte, private def createNonIncreasingOffsetRecords(magicValue: Byte,
timestamp: Long = RecordBatch.NO_TIMESTAMP, timestamp: Long = RecordBatch.NO_TIMESTAMP,
codec: CompressionType = CompressionType.NONE): MemoryRecords = { codec: Compression = Compression.NONE): MemoryRecords = {
val buf = ByteBuffer.allocate(512) val buf = ByteBuffer.allocate(512)
val builder = MemoryRecords.builder(buf, magicValue, codec, TimestampType.CREATE_TIME, 0L) val builder = MemoryRecords.builder(buf, magicValue, codec, TimestampType.CREATE_TIME, 0L)
builder.appendWithOffset(0, timestamp, null, "hello".getBytes) builder.appendWithOffset(0, timestamp, null, "hello".getBytes)
@ -1588,9 +1694,7 @@ class LogValidatorTest {
builder.build() builder.build()
} }
private def createTwoBatchedRecords(magicValue: Byte, private def createTwoBatchedRecords(magicValue: Byte, codec: Compression): MemoryRecords = {
timestamp: Long,
codec: CompressionType): MemoryRecords = {
val buf = ByteBuffer.allocate(2048) val buf = ByteBuffer.allocate(2048)
var builder = MemoryRecords.builder(buf, magicValue, codec, TimestampType.CREATE_TIME, 0L) var builder = MemoryRecords.builder(buf, magicValue, codec, TimestampType.CREATE_TIME, 0L)
builder.append(10L, "1".getBytes(), "a".getBytes()) builder.append(10L, "1".getBytes(), "a".getBytes())
@ -1615,7 +1719,7 @@ class LogValidatorTest {
} }
private def recordsWithNonSequentialInnerOffsets(magicValue: Byte, private def recordsWithNonSequentialInnerOffsets(magicValue: Byte,
compression: CompressionType, compression: Compression,
numRecords: Int): MemoryRecords = { numRecords: Int): MemoryRecords = {
val records = (0 until numRecords).map { id => val records = (0 until numRecords).map { id =>
new SimpleRecord(id.toString.getBytes) new SimpleRecord(id.toString.getBytes)
@ -1633,7 +1737,7 @@ class LogValidatorTest {
private def recordsWithInvalidInnerMagic(batchMagicValue: Byte, private def recordsWithInvalidInnerMagic(batchMagicValue: Byte,
recordMagicValue: Byte, recordMagicValue: Byte,
codec: CompressionType): MemoryRecords = { codec: Compression): MemoryRecords = {
val records = (0 until 20).map(id => val records = (0 until 20).map(id =>
LegacyRecord.create(recordMagicValue, LegacyRecord.create(recordMagicValue,
RecordBatch.NO_TIMESTAMP, RecordBatch.NO_TIMESTAMP,

View File

@ -21,6 +21,7 @@ import kafka.common.{OffsetsOutOfOrderException, UnexpectedAppendOffsetException
import kafka.log.remote.RemoteLogManager import kafka.log.remote.RemoteLogManager
import kafka.server.{BrokerTopicStats, KafkaConfig} import kafka.server.{BrokerTopicStats, KafkaConfig}
import kafka.utils._ import kafka.utils._
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.{InvalidRecordException, TopicPartition, Uuid} import org.apache.kafka.common.{InvalidRecordException, TopicPartition, Uuid}
import org.apache.kafka.common.errors._ import org.apache.kafka.common.errors._
@ -200,7 +201,7 @@ class UnifiedLogTest {
)), leaderEpoch = 0) )), leaderEpoch = 0)
log.appendAsLeader(MemoryRecords.withTransactionalRecords( log.appendAsLeader(MemoryRecords.withTransactionalRecords(
CompressionType.NONE, Compression.NONE,
producerId, producerId,
producerEpoch, producerEpoch,
sequence, sequence,
@ -243,7 +244,7 @@ class UnifiedLogTest {
)), leaderEpoch = 0) )), leaderEpoch = 0)
log.appendAsLeader(MemoryRecords.withTransactionalRecords( log.appendAsLeader(MemoryRecords.withTransactionalRecords(
CompressionType.NONE, Compression.NONE,
producerId, producerId,
producerEpoch, producerEpoch,
sequence, sequence,
@ -567,7 +568,7 @@ class UnifiedLogTest {
val numSegments = log.numberOfSegments val numSegments = log.numberOfSegments
mockTime.sleep(log.config.segmentMs + 1) mockTime.sleep(log.config.segmentMs + 1)
log.appendAsLeader(MemoryRecords.withRecords(CompressionType.NONE), leaderEpoch = 0) log.appendAsLeader(MemoryRecords.withRecords(Compression.NONE), leaderEpoch = 0)
assertEquals(numSegments, log.numberOfSegments, "Appending an empty message set should not roll log even if sufficient time has passed.") assertEquals(numSegments, log.numberOfSegments, "Appending an empty message set should not roll log even if sufficient time has passed.")
} }
@ -1205,7 +1206,7 @@ class UnifiedLogTest {
val seq = 0 val seq = 0
// add some transactional records // add some transactional records
val records = MemoryRecords.withTransactionalRecords(CompressionType.NONE, pid, epoch, seq, val records = MemoryRecords.withTransactionalRecords(Compression.NONE, pid, epoch, seq,
new SimpleRecord("foo".getBytes), new SimpleRecord("foo".getBytes),
new SimpleRecord("bar".getBytes), new SimpleRecord("bar".getBytes),
new SimpleRecord("baz".getBytes)) new SimpleRecord("baz".getBytes))
@ -1328,22 +1329,22 @@ class UnifiedLogTest {
val epoch: Short = 0 val epoch: Short = 0
val buffer = ByteBuffer.allocate(512) val buffer = ByteBuffer.allocate(512)
var builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE, var builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, Compression.NONE,
TimestampType.LOG_APPEND_TIME, 0L, mockTime.milliseconds(), 1L, epoch, 0, false, 0) TimestampType.LOG_APPEND_TIME, 0L, mockTime.milliseconds(), 1L, epoch, 0, false, 0)
builder.append(new SimpleRecord("key".getBytes, "value".getBytes)) builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
builder.close() builder.close()
builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE, builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, Compression.NONE,
TimestampType.LOG_APPEND_TIME, 1L, mockTime.milliseconds(), 2L, epoch, 0, false, 0) TimestampType.LOG_APPEND_TIME, 1L, mockTime.milliseconds(), 2L, epoch, 0, false, 0)
builder.append(new SimpleRecord("key".getBytes, "value".getBytes)) builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
builder.close() builder.close()
builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE, builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, Compression.NONE,
TimestampType.LOG_APPEND_TIME, 2L, mockTime.milliseconds(), 3L, epoch, 0, false, 0) TimestampType.LOG_APPEND_TIME, 2L, mockTime.milliseconds(), 3L, epoch, 0, false, 0)
builder.append(new SimpleRecord("key".getBytes, "value".getBytes)) builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
builder.close() builder.close()
builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE, builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, Compression.NONE,
TimestampType.LOG_APPEND_TIME, 3L, mockTime.milliseconds(), 4L, epoch, 0, false, 0) TimestampType.LOG_APPEND_TIME, 3L, mockTime.milliseconds(), 4L, epoch, 0, false, 0)
builder.append(new SimpleRecord("key".getBytes, "value".getBytes)) builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
builder.close() builder.close()
@ -1378,9 +1379,9 @@ class UnifiedLogTest {
// this is a bit contrived. to trigger the duplicate case for a follower append, we have to append // this is a bit contrived. to trigger the duplicate case for a follower append, we have to append
// a batch with matching sequence numbers, but valid increasing offsets // a batch with matching sequence numbers, but valid increasing offsets
assertEquals(0L, log.logEndOffset) assertEquals(0L, log.logEndOffset)
log.appendAsFollower(MemoryRecords.withIdempotentRecords(0L, CompressionType.NONE, pid, epoch, baseSequence, log.appendAsFollower(MemoryRecords.withIdempotentRecords(0L, Compression.NONE, pid, epoch, baseSequence,
partitionLeaderEpoch, new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes))) partitionLeaderEpoch, new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes)))
log.appendAsFollower(MemoryRecords.withIdempotentRecords(2L, CompressionType.NONE, pid, epoch, baseSequence, log.appendAsFollower(MemoryRecords.withIdempotentRecords(2L, Compression.NONE, pid, epoch, baseSequence,
partitionLeaderEpoch, new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes))) partitionLeaderEpoch, new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes)))
// Ensure that even the duplicate sequences are accepted on the follower. // Ensure that even the duplicate sequences are accepted on the follower.
@ -1399,31 +1400,31 @@ class UnifiedLogTest {
val buffer = ByteBuffer.allocate(512) val buffer = ByteBuffer.allocate(512)
// pid1 seq = 0 // pid1 seq = 0
var builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, var builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, Compression.NONE,
TimestampType.LOG_APPEND_TIME, 0L, mockTime.milliseconds(), pid1, epoch, 0) TimestampType.LOG_APPEND_TIME, 0L, mockTime.milliseconds(), pid1, epoch, 0)
builder.append(new SimpleRecord("key".getBytes, "value".getBytes)) builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
builder.close() builder.close()
// pid2 seq = 0 // pid2 seq = 0
builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, Compression.NONE,
TimestampType.LOG_APPEND_TIME, 1L, mockTime.milliseconds(), pid2, epoch, 0) TimestampType.LOG_APPEND_TIME, 1L, mockTime.milliseconds(), pid2, epoch, 0)
builder.append(new SimpleRecord("key".getBytes, "value".getBytes)) builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
builder.close() builder.close()
// pid1 seq = 1 // pid1 seq = 1
builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, Compression.NONE,
TimestampType.LOG_APPEND_TIME, 2L, mockTime.milliseconds(), pid1, epoch, 1) TimestampType.LOG_APPEND_TIME, 2L, mockTime.milliseconds(), pid1, epoch, 1)
builder.append(new SimpleRecord("key".getBytes, "value".getBytes)) builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
builder.close() builder.close()
// pid2 seq = 1 // pid2 seq = 1
builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, Compression.NONE,
TimestampType.LOG_APPEND_TIME, 3L, mockTime.milliseconds(), pid2, epoch, 1) TimestampType.LOG_APPEND_TIME, 3L, mockTime.milliseconds(), pid2, epoch, 1)
builder.append(new SimpleRecord("key".getBytes, "value".getBytes)) builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
builder.close() builder.close()
// // pid1 seq = 1 (duplicate) // // pid1 seq = 1 (duplicate)
builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, Compression.NONE,
TimestampType.LOG_APPEND_TIME, 4L, mockTime.milliseconds(), pid1, epoch, 1) TimestampType.LOG_APPEND_TIME, 4L, mockTime.milliseconds(), pid1, epoch, 1)
builder.append(new SimpleRecord("key".getBytes, "value".getBytes)) builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
builder.close() builder.close()
@ -1578,7 +1579,7 @@ class UnifiedLogTest {
// now test the case that we give the offsets and use non-sequential offsets // now test the case that we give the offsets and use non-sequential offsets
for (i <- records.indices) for (i <- records.indices)
log.appendAsFollower(MemoryRecords.withRecords(messageIds(i), CompressionType.NONE, 0, records(i))) log.appendAsFollower(MemoryRecords.withRecords(messageIds(i), Compression.NONE, 0, records(i)))
for (i <- 50 until messageIds.max) { for (i <- 50 until messageIds.max) {
val idx = messageIds.indexWhere(_ >= i) val idx = messageIds.indexWhere(_ >= i)
val read = LogTestUtils.readLog(log, i, 100).records.records.iterator.next() val read = LogTestUtils.readLog(log, i, 100).records.records.iterator.next()
@ -1626,7 +1627,7 @@ class UnifiedLogTest {
// now test the case that we give the offsets and use non-sequential offsets // now test the case that we give the offsets and use non-sequential offsets
for (i <- records.indices) for (i <- records.indices)
log.appendAsFollower(MemoryRecords.withRecords(messageIds(i), CompressionType.NONE, 0, records(i))) log.appendAsFollower(MemoryRecords.withRecords(messageIds(i), Compression.NONE, 0, records(i)))
for (i <- 50 until messageIds.max) { for (i <- 50 until messageIds.max) {
val idx = messageIds.indexWhere(_ >= i) val idx = messageIds.indexWhere(_ >= i)
@ -1651,7 +1652,7 @@ class UnifiedLogTest {
// now test the case that we give the offsets and use non-sequential offsets // now test the case that we give the offsets and use non-sequential offsets
for (i <- records.indices) for (i <- records.indices)
log.appendAsFollower(MemoryRecords.withRecords(messageIds(i), CompressionType.NONE, 0, records(i))) log.appendAsFollower(MemoryRecords.withRecords(messageIds(i), Compression.NONE, 0, records(i)))
for (i <- 50 until messageIds.max) { for (i <- 50 until messageIds.max) {
assertEquals(MemoryRecords.EMPTY, LogTestUtils.readLog(log, i, maxLength = 0, minOneMessage = false).records) assertEquals(MemoryRecords.EMPTY, LogTestUtils.readLog(log, i, maxLength = 0, minOneMessage = false).records)
@ -1752,8 +1753,8 @@ class UnifiedLogTest {
val log = createLog(logDir, logConfig) val log = createLog(logDir, logConfig)
/* append 2 compressed message sets, each with two messages giving offsets 0, 1, 2, 3 */ /* append 2 compressed message sets, each with two messages giving offsets 0, 1, 2, 3 */
log.appendAsLeader(MemoryRecords.withRecords(CompressionType.GZIP, new SimpleRecord("hello".getBytes), new SimpleRecord("there".getBytes)), leaderEpoch = 0) log.appendAsLeader(MemoryRecords.withRecords(Compression.gzip().build(), new SimpleRecord("hello".getBytes), new SimpleRecord("there".getBytes)), leaderEpoch = 0)
log.appendAsLeader(MemoryRecords.withRecords(CompressionType.GZIP, new SimpleRecord("alpha".getBytes), new SimpleRecord("beta".getBytes)), leaderEpoch = 0) log.appendAsLeader(MemoryRecords.withRecords(Compression.gzip().build(), new SimpleRecord("alpha".getBytes), new SimpleRecord("beta".getBytes)), leaderEpoch = 0)
def read(offset: Int) = LogTestUtils.readLog(log, offset, 4096).records.records def read(offset: Int) = LogTestUtils.readLog(log, offset, 4096).records.records
@ -1807,7 +1808,7 @@ class UnifiedLogTest {
*/ */
@Test @Test
def testMessageSetSizeCheck(): Unit = { def testMessageSetSizeCheck(): Unit = {
val messageSet = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("You".getBytes), new SimpleRecord("bethe".getBytes)) val messageSet = MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("You".getBytes), new SimpleRecord("bethe".getBytes))
// append messages to log // append messages to log
val configSegmentSize = messageSet.sizeInBytes - 1 val configSegmentSize = messageSet.sizeInBytes - 1
val logConfig = LogTestUtils.createLogConfig(segmentBytes = configSegmentSize) val logConfig = LogTestUtils.createLogConfig(segmentBytes = configSegmentSize)
@ -1822,13 +1823,13 @@ class UnifiedLogTest {
val anotherKeyedMessage = new SimpleRecord("another key".getBytes, "this message also has a key".getBytes) val anotherKeyedMessage = new SimpleRecord("another key".getBytes, "this message also has a key".getBytes)
val unkeyedMessage = new SimpleRecord("this message does not have a key".getBytes) val unkeyedMessage = new SimpleRecord("this message does not have a key".getBytes)
val messageSetWithUnkeyedMessage = MemoryRecords.withRecords(CompressionType.NONE, unkeyedMessage, keyedMessage) val messageSetWithUnkeyedMessage = MemoryRecords.withRecords(Compression.NONE, unkeyedMessage, keyedMessage)
val messageSetWithOneUnkeyedMessage = MemoryRecords.withRecords(CompressionType.NONE, unkeyedMessage) val messageSetWithOneUnkeyedMessage = MemoryRecords.withRecords(Compression.NONE, unkeyedMessage)
val messageSetWithCompressedKeyedMessage = MemoryRecords.withRecords(CompressionType.GZIP, keyedMessage) val messageSetWithCompressedKeyedMessage = MemoryRecords.withRecords(Compression.gzip().build(), keyedMessage)
val messageSetWithCompressedUnkeyedMessage = MemoryRecords.withRecords(CompressionType.GZIP, keyedMessage, unkeyedMessage) val messageSetWithCompressedUnkeyedMessage = MemoryRecords.withRecords(Compression.gzip().build(), keyedMessage, unkeyedMessage)
val messageSetWithKeyedMessage = MemoryRecords.withRecords(CompressionType.NONE, keyedMessage) val messageSetWithKeyedMessage = MemoryRecords.withRecords(Compression.NONE, keyedMessage)
val messageSetWithKeyedMessages = MemoryRecords.withRecords(CompressionType.NONE, keyedMessage, anotherKeyedMessage) val messageSetWithKeyedMessages = MemoryRecords.withRecords(Compression.NONE, keyedMessage, anotherKeyedMessage)
val logConfig = LogTestUtils.createLogConfig(cleanupPolicy = TopicConfig.CLEANUP_POLICY_COMPACT) val logConfig = LogTestUtils.createLogConfig(cleanupPolicy = TopicConfig.CLEANUP_POLICY_COMPACT)
val log = createLog(logDir, logConfig) val log = createLog(logDir, logConfig)
@ -1872,8 +1873,8 @@ class UnifiedLogTest {
*/ */
@Test @Test
def testMessageSizeCheck(): Unit = { def testMessageSizeCheck(): Unit = {
val first = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("You".getBytes), new SimpleRecord("bethe".getBytes)) val first = MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("You".getBytes), new SimpleRecord("bethe".getBytes))
val second = MemoryRecords.withRecords(CompressionType.NONE, val second = MemoryRecords.withRecords(Compression.NONE,
new SimpleRecord("change (I need more bytes)... blah blah blah.".getBytes), new SimpleRecord("change (I need more bytes)... blah blah blah.".getBytes),
new SimpleRecord("More padding boo hoo".getBytes)) new SimpleRecord("More padding boo hoo".getBytes))
@ -1891,9 +1892,9 @@ class UnifiedLogTest {
@Test @Test
def testMessageSizeCheckInAppendAsFollower(): Unit = { def testMessageSizeCheckInAppendAsFollower(): Unit = {
val first = MemoryRecords.withRecords(0, CompressionType.NONE, 0, val first = MemoryRecords.withRecords(0, Compression.NONE, 0,
new SimpleRecord("You".getBytes), new SimpleRecord("bethe".getBytes)) new SimpleRecord("You".getBytes), new SimpleRecord("bethe".getBytes))
val second = MemoryRecords.withRecords(5, CompressionType.NONE, 0, val second = MemoryRecords.withRecords(5, Compression.NONE, 0,
new SimpleRecord("change (I need more bytes)... blah blah blah.".getBytes), new SimpleRecord("change (I need more bytes)... blah blah blah.".getBytes),
new SimpleRecord("More padding boo hoo".getBytes)) new SimpleRecord("More padding boo hoo".getBytes))
@ -1908,7 +1909,7 @@ class UnifiedLogTest {
def testLogFlushesPartitionMetadataOnAppend(): Unit = { def testLogFlushesPartitionMetadataOnAppend(): Unit = {
val logConfig = LogTestUtils.createLogConfig() val logConfig = LogTestUtils.createLogConfig()
val log = createLog(logDir, logConfig) val log = createLog(logDir, logConfig)
val record = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("simpleValue".getBytes)) val record = MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("simpleValue".getBytes))
val topicId = Uuid.randomUuid() val topicId = Uuid.randomUuid()
log.partitionMetadataFile.get.record(topicId) log.partitionMetadataFile.get.record(topicId)
@ -2000,7 +2001,7 @@ class UnifiedLogTest {
val log = createLog(logDir, logConfig) val log = createLog(logDir, logConfig)
val messages = (0 until numMessages).map { i => val messages = (0 until numMessages).map { i =>
MemoryRecords.withRecords(100 + i, CompressionType.NONE, 0, new SimpleRecord(mockTime.milliseconds + i, i.toString.getBytes())) MemoryRecords.withRecords(100 + i, Compression.NONE, 0, new SimpleRecord(mockTime.milliseconds + i, i.toString.getBytes()))
} }
messages.foreach(log.appendAsFollower) messages.foreach(log.appendAsFollower)
val timeIndexEntries = log.logSegments.asScala.foldLeft(0) { (entries, segment) => entries + segment.timeIndex.entries } val timeIndexEntries = log.logSegments.asScala.foldLeft(0) { (entries, segment) => entries + segment.timeIndex.entries }
@ -2367,7 +2368,7 @@ class UnifiedLogTest {
val appendOffsets = Seq(0L, 1L, 3L, 2L, 4L) val appendOffsets = Seq(0L, 1L, 3L, 2L, 4L)
val buffer = ByteBuffer.allocate(512) val buffer = ByteBuffer.allocate(512)
for (offset <- appendOffsets) { for (offset <- appendOffsets) {
val builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE, val builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, Compression.NONE,
TimestampType.LOG_APPEND_TIME, offset, mockTime.milliseconds(), TimestampType.LOG_APPEND_TIME, offset, mockTime.milliseconds(),
1L, 0, 0, false, 0) 1L, 0, 0, false, 0)
builder.append(new SimpleRecord("key".getBytes, "value".getBytes)) builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
@ -2385,15 +2386,16 @@ class UnifiedLogTest {
def testAppendBelowExpectedOffsetThrowsException(): Unit = { def testAppendBelowExpectedOffsetThrowsException(): Unit = {
val log = createLog(logDir, new LogConfig(new Properties)) val log = createLog(logDir, new LogConfig(new Properties))
val records = (0 until 2).map(id => new SimpleRecord(id.toString.getBytes)).toArray val records = (0 until 2).map(id => new SimpleRecord(id.toString.getBytes)).toArray
records.foreach(record => log.appendAsLeader(MemoryRecords.withRecords(CompressionType.NONE, record), leaderEpoch = 0)) records.foreach(record => log.appendAsLeader(MemoryRecords.withRecords(Compression.NONE, record), leaderEpoch = 0))
val magicVals = Seq(RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V2) val magicVals = Seq(RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V2)
val compressionTypes = Seq(CompressionType.NONE, CompressionType.LZ4) val compressionTypes = Seq(CompressionType.NONE, CompressionType.LZ4)
for (magic <- magicVals; compression <- compressionTypes) { for (magic <- magicVals; compressionType <- compressionTypes) {
val compression = Compression.of(compressionType).build()
val invalidRecord = MemoryRecords.withRecords(magic, compression, new SimpleRecord(1.toString.getBytes)) val invalidRecord = MemoryRecords.withRecords(magic, compression, new SimpleRecord(1.toString.getBytes))
assertThrows(classOf[UnexpectedAppendOffsetException], assertThrows(classOf[UnexpectedAppendOffsetException],
() => log.appendAsFollower(invalidRecord), () => log.appendAsFollower(invalidRecord),
() => s"Magic=$magic, compressionType=$compression") () => s"Magic=$magic, compressionType=$compressionType")
} }
} }
@ -2407,23 +2409,23 @@ class UnifiedLogTest {
val firstOffset = 4L val firstOffset = 4L
val magicVals = Seq(RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V2) val magicVals = Seq(RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V2)
val compressionTypes = Seq(CompressionType.NONE, CompressionType.LZ4) val compressionTypes = Seq(CompressionType.NONE, CompressionType.LZ4)
for (magic <- magicVals; compression <- compressionTypes) { for (magic <- magicVals; compressionType <- compressionTypes) {
val batch = TestUtils.records(List(new SimpleRecord("k1".getBytes, "v1".getBytes), val batch = TestUtils.records(List(new SimpleRecord("k1".getBytes, "v1".getBytes),
new SimpleRecord("k2".getBytes, "v2".getBytes), new SimpleRecord("k2".getBytes, "v2".getBytes),
new SimpleRecord("k3".getBytes, "v3".getBytes)), new SimpleRecord("k3".getBytes, "v3".getBytes)),
magicValue = magic, codec = compression, magicValue = magic, codec = Compression.of(compressionType).build(),
baseOffset = firstOffset) baseOffset = firstOffset)
val exception = assertThrows(classOf[UnexpectedAppendOffsetException], () => log.appendAsFollower(records = batch)) val exception = assertThrows(classOf[UnexpectedAppendOffsetException], () => log.appendAsFollower(records = batch))
assertEquals(firstOffset, exception.firstOffset, s"Magic=$magic, compressionType=$compression, UnexpectedAppendOffsetException#firstOffset") assertEquals(firstOffset, exception.firstOffset, s"Magic=$magic, compressionType=$compressionType, UnexpectedAppendOffsetException#firstOffset")
assertEquals(firstOffset + 2, exception.lastOffset, s"Magic=$magic, compressionType=$compression, UnexpectedAppendOffsetException#lastOffset") assertEquals(firstOffset + 2, exception.lastOffset, s"Magic=$magic, compressionType=$compressionType, UnexpectedAppendOffsetException#lastOffset")
} }
} }
@Test @Test
def testAppendWithNoTimestamp(): Unit = { def testAppendWithNoTimestamp(): Unit = {
val log = createLog(logDir, new LogConfig(new Properties)) val log = createLog(logDir, new LogConfig(new Properties))
log.appendAsLeader(MemoryRecords.withRecords(CompressionType.NONE, log.appendAsLeader(MemoryRecords.withRecords(Compression.NONE,
new SimpleRecord(RecordBatch.NO_TIMESTAMP, "key".getBytes, "value".getBytes)), leaderEpoch = 0) new SimpleRecord(RecordBatch.NO_TIMESTAMP, "key".getBytes, "value".getBytes)), leaderEpoch = 0)
} }
@ -2588,9 +2590,9 @@ class UnifiedLogTest {
// When we split the overflowed segment, only one new segment will be created. // When we split the overflowed segment, only one new segment will be created.
val overflowOffset = Int.MaxValue + 1L val overflowOffset = Int.MaxValue + 1L
val batch1 = MemoryRecords.withRecords(overflowOffset, CompressionType.NONE, 0, val batch1 = MemoryRecords.withRecords(overflowOffset, Compression.NONE, 0,
new SimpleRecord("a".getBytes)) new SimpleRecord("a".getBytes))
val batch2 = MemoryRecords.withRecords(overflowOffset + 1, CompressionType.NONE, 0, val batch2 = MemoryRecords.withRecords(overflowOffset + 1, Compression.NONE, 0,
new SimpleRecord("b".getBytes)) new SimpleRecord("b".getBytes))
testDegenerateSplitSegmentWithOverflow(segmentBaseOffset = 0L, List(batch1, batch2)) testDegenerateSplitSegmentWithOverflow(segmentBaseOffset = 0L, List(batch1, batch2))
@ -2602,7 +2604,7 @@ class UnifiedLogTest {
// the first offset of the batch is valid, but the last overflows. // the first offset of the batch is valid, but the last overflows.
val firstBatchBaseOffset = Int.MaxValue - 1 val firstBatchBaseOffset = Int.MaxValue - 1
val records = MemoryRecords.withRecords(firstBatchBaseOffset, CompressionType.NONE, 0, val records = MemoryRecords.withRecords(firstBatchBaseOffset, Compression.NONE, 0,
new SimpleRecord("a".getBytes), new SimpleRecord("a".getBytes),
new SimpleRecord("b".getBytes), new SimpleRecord("b".getBytes),
new SimpleRecord("c".getBytes)) new SimpleRecord("c".getBytes))
@ -2869,7 +2871,7 @@ class UnifiedLogTest {
//When appending messages as a leader (i.e. assignOffsets = true) //When appending messages as a leader (i.e. assignOffsets = true)
for (record <- records) for (record <- records)
log.appendAsLeader( log.appendAsLeader(
MemoryRecords.withRecords(CompressionType.NONE, record), MemoryRecords.withRecords(Compression.NONE, record),
leaderEpoch = epoch leaderEpoch = epoch
) )
@ -2887,7 +2889,7 @@ class UnifiedLogTest {
//Given each message has an offset & epoch, as msgs from leader would //Given each message has an offset & epoch, as msgs from leader would
def recordsForEpoch(i: Int): MemoryRecords = { def recordsForEpoch(i: Int): MemoryRecords = {
val recs = MemoryRecords.withRecords(messageIds(i), CompressionType.NONE, records(i)) val recs = MemoryRecords.withRecords(messageIds(i), Compression.NONE, records(i))
recs.batches.forEach{record => recs.batches.forEach{record =>
record.setPartitionLeaderEpoch(42) record.setPartitionLeaderEpoch(42)
record.setLastOffset(i) record.setLastOffset(i)
@ -3008,7 +3010,7 @@ class UnifiedLogTest {
val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024 * 5) val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024 * 5)
val log = createLog(logDir, logConfig) val log = createLog(logDir, logConfig)
val records = MemoryRecords.withRecords(CompressionType.NONE, val records = MemoryRecords.withRecords(Compression.NONE,
new SimpleRecord("foo".getBytes), new SimpleRecord("foo".getBytes),
new SimpleRecord("bar".getBytes), new SimpleRecord("bar".getBytes),
new SimpleRecord("baz".getBytes)) new SimpleRecord("baz".getBytes))
@ -3027,7 +3029,7 @@ class UnifiedLogTest {
var seq = 0 var seq = 0
// add some transactional records // add some transactional records
val records = MemoryRecords.withTransactionalRecords(CompressionType.NONE, pid, epoch, seq, val records = MemoryRecords.withTransactionalRecords(Compression.NONE, pid, epoch, seq,
new SimpleRecord("foo".getBytes), new SimpleRecord("foo".getBytes),
new SimpleRecord("bar".getBytes), new SimpleRecord("bar".getBytes),
new SimpleRecord("baz".getBytes)) new SimpleRecord("baz".getBytes))
@ -3037,7 +3039,7 @@ class UnifiedLogTest {
// add more transactional records // add more transactional records
seq += 3 seq += 3
log.appendAsLeader(MemoryRecords.withTransactionalRecords(CompressionType.NONE, pid, epoch, seq, log.appendAsLeader(MemoryRecords.withTransactionalRecords(Compression.NONE, pid, epoch, seq,
new SimpleRecord("blah".getBytes)), leaderEpoch = 0) new SimpleRecord("blah".getBytes)), leaderEpoch = 0)
// LSO should not have changed // LSO should not have changed
@ -3406,7 +3408,7 @@ class UnifiedLogTest {
val log = createLog(logDir, logConfig) val log = createLog(logDir, logConfig)
// append a few records // append a few records
appendAsFollower(log, MemoryRecords.withRecords(CompressionType.NONE, appendAsFollower(log, MemoryRecords.withRecords(Compression.NONE,
new SimpleRecord("a".getBytes), new SimpleRecord("a".getBytes),
new SimpleRecord("b".getBytes), new SimpleRecord("b".getBytes),
new SimpleRecord("c".getBytes)), 5) new SimpleRecord("c".getBytes)), 5)
@ -3436,20 +3438,20 @@ class UnifiedLogTest {
val seq2 = 0 val seq2 = 0
// add some transactional records // add some transactional records
val firstAppendInfo = log.appendAsLeader(MemoryRecords.withTransactionalRecords(CompressionType.NONE, pid1, epoch, seq1, val firstAppendInfo = log.appendAsLeader(MemoryRecords.withTransactionalRecords(Compression.NONE, pid1, epoch, seq1,
new SimpleRecord("a".getBytes), new SimpleRecord("a".getBytes),
new SimpleRecord("b".getBytes), new SimpleRecord("b".getBytes),
new SimpleRecord("c".getBytes)), leaderEpoch = 0) new SimpleRecord("c".getBytes)), leaderEpoch = 0)
assertEquals(Some(firstAppendInfo.firstOffset), log.firstUnstableOffset) assertEquals(Some(firstAppendInfo.firstOffset), log.firstUnstableOffset)
// mix in some non-transactional data // mix in some non-transactional data
log.appendAsLeader(MemoryRecords.withRecords(CompressionType.NONE, log.appendAsLeader(MemoryRecords.withRecords(Compression.NONE,
new SimpleRecord("g".getBytes), new SimpleRecord("g".getBytes),
new SimpleRecord("h".getBytes), new SimpleRecord("h".getBytes),
new SimpleRecord("i".getBytes)), leaderEpoch = 0) new SimpleRecord("i".getBytes)), leaderEpoch = 0)
// append data from a second transactional producer // append data from a second transactional producer
val secondAppendInfo = log.appendAsLeader(MemoryRecords.withTransactionalRecords(CompressionType.NONE, pid2, epoch, seq2, val secondAppendInfo = log.appendAsLeader(MemoryRecords.withTransactionalRecords(Compression.NONE, pid2, epoch, seq2,
new SimpleRecord("d".getBytes), new SimpleRecord("d".getBytes),
new SimpleRecord("e".getBytes), new SimpleRecord("e".getBytes),
new SimpleRecord("f".getBytes)), leaderEpoch = 0) new SimpleRecord("f".getBytes)), leaderEpoch = 0)
@ -3478,7 +3480,7 @@ class UnifiedLogTest {
val epoch = 5.toShort val epoch = 5.toShort
var seq = 0 var seq = 0
val records = MemoryRecords.withTransactionalRecords(CompressionType.NONE, pid, epoch, seq, val records = MemoryRecords.withTransactionalRecords(Compression.NONE, pid, epoch, seq,
new SimpleRecord("a".getBytes), new SimpleRecord("a".getBytes),
new SimpleRecord("b".getBytes), new SimpleRecord("b".getBytes),
new SimpleRecord("c".getBytes)) new SimpleRecord("c".getBytes))
@ -3491,7 +3493,7 @@ class UnifiedLogTest {
// this write should spill to the second segment // this write should spill to the second segment
seq = 3 seq = 3
log.appendAsLeader(MemoryRecords.withTransactionalRecords(CompressionType.NONE, pid, epoch, seq, log.appendAsLeader(MemoryRecords.withTransactionalRecords(Compression.NONE, pid, epoch, seq,
new SimpleRecord("d".getBytes), new SimpleRecord("d".getBytes),
new SimpleRecord("e".getBytes), new SimpleRecord("e".getBytes),
new SimpleRecord("f".getBytes)), leaderEpoch = 0) new SimpleRecord("f".getBytes)), leaderEpoch = 0)
@ -3841,7 +3843,7 @@ class UnifiedLogTest {
assertFalse(log.verificationGuard(producerId).verify(VerificationGuard.SENTINEL)) assertFalse(log.verificationGuard(producerId).verify(VerificationGuard.SENTINEL))
val idempotentRecords = MemoryRecords.withIdempotentRecords( val idempotentRecords = MemoryRecords.withIdempotentRecords(
CompressionType.NONE, Compression.NONE,
producerId, producerId,
producerEpoch, producerEpoch,
sequence, sequence,
@ -3854,7 +3856,7 @@ class UnifiedLogTest {
sequence = sequence + 2 sequence = sequence + 2
val transactionalRecords = MemoryRecords.withTransactionalRecords( val transactionalRecords = MemoryRecords.withTransactionalRecords(
CompressionType.NONE, Compression.NONE,
producerId, producerId,
producerEpoch, producerEpoch,
sequence, sequence,
@ -3939,7 +3941,7 @@ class UnifiedLogTest {
producerStateManagerConfig.setTransactionVerificationEnabled(false) producerStateManagerConfig.setTransactionVerificationEnabled(false)
val transactionalRecords = MemoryRecords.withTransactionalRecords( val transactionalRecords = MemoryRecords.withTransactionalRecords(
CompressionType.NONE, Compression.NONE,
producerId, producerId,
producerEpoch, producerEpoch,
0, 0,
@ -3965,7 +3967,7 @@ class UnifiedLogTest {
producerStateManagerConfig.setTransactionVerificationEnabled(true) producerStateManagerConfig.setTransactionVerificationEnabled(true)
val transactionalRecords = MemoryRecords.withTransactionalRecords( val transactionalRecords = MemoryRecords.withTransactionalRecords(
CompressionType.NONE, Compression.NONE,
producerId, producerId,
producerEpoch, producerEpoch,
sequence, sequence,
@ -3997,7 +3999,7 @@ class UnifiedLogTest {
assertEquals(VerificationGuard.SENTINEL, log.verificationGuard(producerId)) assertEquals(VerificationGuard.SENTINEL, log.verificationGuard(producerId))
val transactionalRecords = MemoryRecords.withTransactionalRecords( val transactionalRecords = MemoryRecords.withTransactionalRecords(
CompressionType.NONE, Compression.NONE,
producerId, producerId,
producerEpoch, producerEpoch,
sequence, sequence,
@ -4214,7 +4216,7 @@ class UnifiedLogTest {
leaderEpoch: Int = 0): (Long, Int) => Unit = { leaderEpoch: Int = 0): (Long, Int) => Unit = {
var sequence = 0 var sequence = 0
(offset: Long, numRecords: Int) => { (offset: Long, numRecords: Int) => {
val builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, TimestampType.CREATE_TIME, val builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, Compression.NONE, TimestampType.CREATE_TIME,
offset, mockTime.milliseconds(), producerId, producerEpoch, sequence, true, leaderEpoch) offset, mockTime.milliseconds(), producerId, producerEpoch, sequence, true, leaderEpoch)
for (seq <- sequence until sequence + numRecords) { for (seq <- sequence until sequence + numRecords) {
val record = new SimpleRecord(s"$seq".getBytes) val record = new SimpleRecord(s"$seq".getBytes)
@ -4238,7 +4240,7 @@ class UnifiedLogTest {
} }
private def appendNonTransactionalToBuffer(buffer: ByteBuffer, offset: Long, numRecords: Int): Unit = { private def appendNonTransactionalToBuffer(buffer: ByteBuffer, offset: Long, numRecords: Int): Unit = {
val builder = MemoryRecords.builder(buffer, CompressionType.NONE, TimestampType.CREATE_TIME, offset) val builder = MemoryRecords.builder(buffer, Compression.NONE, TimestampType.CREATE_TIME, offset)
(0 until numRecords).foreach { seq => (0 until numRecords).foreach { seq =>
builder.append(new SimpleRecord(s"$seq".getBytes)) builder.append(new SimpleRecord(s"$seq".getBytes))
} }

View File

@ -161,13 +161,14 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
assertEquals(IBP_3_0_IV1, log.config.messageFormatVersion) assertEquals(IBP_3_0_IV1, log.config.messageFormatVersion)
assertEquals(RecordVersion.V2, log.config.recordVersion) assertEquals(RecordVersion.V2, log.config.recordVersion)
val compressionType = CompressionType.LZ4.name val compressionType = CompressionType.LZ4
logProps.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, "0.11.0") logProps.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, "0.11.0")
// set compression type so that we can detect when the config change has propagated // set compression type so that we can detect when the config change has propagated
logProps.put(TopicConfig.COMPRESSION_TYPE_CONFIG, compressionType) logProps.put(TopicConfig.COMPRESSION_TYPE_CONFIG, compressionType.name)
adminZkClient.changeTopicConfig(tp.topic, logProps) adminZkClient.changeTopicConfig(tp.topic, logProps)
TestUtils.waitUntilTrue(() => TestUtils.waitUntilTrue(() =>
server.logManager.getLog(tp).get.config.compressionType == compressionType, server.logManager.getLog(tp).get.config.compression.isPresent &&
server.logManager.getLog(tp).get.config.compression.get.`type` == compressionType,
"Topic config change propagation failed") "Topic config change propagation failed")
assertEquals(IBP_3_0_IV1, log.config.messageFormatVersion) assertEquals(IBP_3_0_IV1, log.config.messageFormatVersion)
assertEquals(RecordVersion.V2, log.config.recordVersion) assertEquals(RecordVersion.V2, log.config.recordVersion)

View File

@ -21,15 +21,15 @@ import java.io.{DataInputStream, DataOutputStream}
import java.net.Socket import java.net.Socket
import java.nio.ByteBuffer import java.nio.ByteBuffer
import java.util.Collections import java.util.Collections
import kafka.integration.KafkaServerTestHarness import kafka.integration.KafkaServerTestHarness
import kafka.network.SocketServer import kafka.network.SocketServer
import kafka.utils._ import kafka.utils._
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.message.ProduceRequestData import org.apache.kafka.common.message.ProduceRequestData
import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.types.Type import org.apache.kafka.common.protocol.types.Type
import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord} import org.apache.kafka.common.record.{MemoryRecords, SimpleRecord}
import org.apache.kafka.common.requests.{ProduceResponse, ResponseHeader} import org.apache.kafka.common.requests.{ProduceResponse, ResponseHeader}
import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.utils.ByteUtils import org.apache.kafka.common.utils.ByteUtils
@ -135,7 +135,7 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness {
.setName(topicPartition.topic()).setPartitionData(Collections.singletonList( .setName(topicPartition.topic()).setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData() new ProduceRequestData.PartitionProduceData()
.setIndex(topicPartition.partition()) .setIndex(topicPartition.partition())
.setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("message".getBytes)))))) .setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("message".getBytes))))))
.iterator)) .iterator))
.setAcks(1.toShort) .setAcks(1.toShort)
.setTimeoutMs(10000) .setTimeoutMs(10000)

View File

@ -16,10 +16,10 @@
*/ */
package kafka.server package kafka.server
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid}
import org.apache.kafka.common.message.FetchResponseData import org.apache.kafka.common.message.FetchResponseData
import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.record.CompressionType
import org.apache.kafka.common.record.MemoryRecords import org.apache.kafka.common.record.MemoryRecords
import org.apache.kafka.common.record.SimpleRecord import org.apache.kafka.common.record.SimpleRecord
import org.apache.kafka.common.requests.FetchMetadata.{FINAL_EPOCH, INVALID_SESSION_ID} import org.apache.kafka.common.requests.FetchMetadata.{FINAL_EPOCH, INVALID_SESSION_ID}
@ -1854,7 +1854,7 @@ class FetchSessionTest {
.setHighWatermark(60) .setHighWatermark(60)
.setLastStableOffset(50) .setLastStableOffset(50)
.setLogStartOffset(0) .setLogStartOffset(0)
.setRecords(MemoryRecords.withRecords(CompressionType.NONE, .setRecords(MemoryRecords.withRecords(Compression.NONE,
new SimpleRecord(100, null)))) new SimpleRecord(100, null))))
respData3.put(tp3, new FetchResponseData.PartitionData() respData3.put(tp3, new FetchResponseData.PartitionData()
.setPartitionIndex(tp3.topicPartition.partition) .setPartitionIndex(tp3.topicPartition.partition)

View File

@ -20,13 +20,14 @@ package kafka.server
import kafka.cluster.BrokerEndPoint import kafka.cluster.BrokerEndPoint
import kafka.server.InitialFetchState import kafka.server.InitialFetchState
import org.apache.kafka.common.Uuid import org.apache.kafka.common.Uuid
import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch, SimpleRecord} import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, SimpleRecord}
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
object FetcherThreadTestUtils { object FetcherThreadTestUtils {
def mkBatch(baseOffset: Long, leaderEpoch: Int, records: SimpleRecord*): RecordBatch = { def mkBatch(baseOffset: Long, leaderEpoch: Int, records: SimpleRecord*): RecordBatch = {
MemoryRecords.withRecords(baseOffset, CompressionType.NONE, leaderEpoch, records: _*) MemoryRecords.withRecords(baseOffset, Compression.NONE, leaderEpoch, records: _*)
.batches.asScala.head .batches.asScala.head
} }

View File

@ -31,6 +31,7 @@ import org.apache.kafka.clients.admin.AlterConfigOp.OpType
import org.apache.kafka.clients.admin.{AlterConfigOp, ConfigEntry} import org.apache.kafka.clients.admin.{AlterConfigOp, ConfigEntry}
import org.apache.kafka.common._ import org.apache.kafka.common._
import org.apache.kafka.common.acl.AclOperation import org.apache.kafka.common.acl.AclOperation
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.config.ConfigResource
import org.apache.kafka.common.config.ConfigResource.Type.{BROKER, BROKER_LOGGER} import org.apache.kafka.common.config.ConfigResource.Type.{BROKER, BROKER_LOGGER}
import org.apache.kafka.common.errors.{ClusterAuthorizationException, UnsupportedVersionException} import org.apache.kafka.common.errors.{ClusterAuthorizationException, UnsupportedVersionException}
@ -2474,7 +2475,7 @@ class KafkaApisTest extends Logging {
.setName(tp.topic).setPartitionData(Collections.singletonList( .setName(tp.topic).setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData() new ProduceRequestData.PartitionProduceData()
.setIndex(tp.partition) .setIndex(tp.partition)
.setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes)))))) .setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("test".getBytes))))))
.iterator)) .iterator))
.setAcks(1.toShort) .setAcks(1.toShort)
.setTimeoutMs(5000)) .setTimeoutMs(5000))
@ -2536,7 +2537,7 @@ class KafkaApisTest extends Logging {
.setName(tp.topic).setPartitionData(Collections.singletonList( .setName(tp.topic).setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData() new ProduceRequestData.PartitionProduceData()
.setIndex(tp.partition) .setIndex(tp.partition)
.setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes)))))) .setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("test".getBytes))))))
.iterator)) .iterator))
.setAcks(1.toShort) .setAcks(1.toShort)
.setTimeoutMs(5000)) .setTimeoutMs(5000))
@ -2601,7 +2602,7 @@ class KafkaApisTest extends Logging {
.setName(tp.topic).setPartitionData(Collections.singletonList( .setName(tp.topic).setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData() new ProduceRequestData.PartitionProduceData()
.setIndex(tp.partition) .setIndex(tp.partition)
.setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes)))))) .setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("test".getBytes))))))
.iterator)) .iterator))
.setAcks(1.toShort) .setAcks(1.toShort)
.setTimeoutMs(5000)) .setTimeoutMs(5000))
@ -2665,7 +2666,7 @@ class KafkaApisTest extends Logging {
.setName(tp.topic).setPartitionData(Collections.singletonList( .setName(tp.topic).setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData() new ProduceRequestData.PartitionProduceData()
.setIndex(tp.partition) .setIndex(tp.partition)
.setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes)))))) .setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("test".getBytes))))))
.iterator)) .iterator))
.setAcks(1.toShort) .setAcks(1.toShort)
.setTimeoutMs(5000)) .setTimeoutMs(5000))
@ -2728,7 +2729,7 @@ class KafkaApisTest extends Logging {
.setName(tp.topic).setPartitionData(Collections.singletonList( .setName(tp.topic).setPartitionData(Collections.singletonList(
new ProduceRequestData.PartitionProduceData() new ProduceRequestData.PartitionProduceData()
.setIndex(tp.partition) .setIndex(tp.partition)
.setRecords(MemoryRecords.withTransactionalRecords(CompressionType.NONE, 0, 0, 0, new SimpleRecord("test".getBytes)))))) .setRecords(MemoryRecords.withTransactionalRecords(Compression.NONE, 0, 0, 0, new SimpleRecord("test".getBytes))))))
.iterator)) .iterator))
.setAcks(1.toShort) .setAcks(1.toShort)
.setTransactionalId(transactionalId) .setTransactionalId(transactionalId)
@ -4203,7 +4204,7 @@ class KafkaApisTest extends Logging {
any[Seq[(TopicIdPartition, FetchPartitionData)] => Unit]() any[Seq[(TopicIdPartition, FetchPartitionData)] => Unit]()
)).thenAnswer(invocation => { )).thenAnswer(invocation => {
val callback = invocation.getArgument(3).asInstanceOf[Seq[(TopicIdPartition, FetchPartitionData)] => Unit] val callback = invocation.getArgument(3).asInstanceOf[Seq[(TopicIdPartition, FetchPartitionData)] => Unit]
val records = MemoryRecords.withRecords(CompressionType.NONE, val records = MemoryRecords.withRecords(Compression.NONE,
new SimpleRecord(timestamp, "foo".getBytes(StandardCharsets.UTF_8))) new SimpleRecord(timestamp, "foo".getBytes(StandardCharsets.UTF_8)))
callback(Seq(tidp -> new FetchPartitionData(Errors.NONE, hw, 0, records, callback(Seq(tidp -> new FetchPartitionData(Errors.NONE, hw, 0, records,
Optional.empty(), OptionalLong.empty(), Optional.empty(), OptionalInt.empty(), false))) Optional.empty(), OptionalLong.empty(), Optional.empty(), OptionalInt.empty(), false)))
@ -5566,7 +5567,7 @@ class KafkaApisTest extends Logging {
val fetchFromFollower = buildRequest(new FetchRequest.Builder( val fetchFromFollower = buildRequest(new FetchRequest.Builder(
ApiKeys.FETCH.oldestVersion(), ApiKeys.FETCH.latestVersion(), 1, 1, 1000, 0, fetchDataBuilder).build()) ApiKeys.FETCH.oldestVersion(), ApiKeys.FETCH.latestVersion(), 1, 1, 1000, 0, fetchDataBuilder).build())
val records = MemoryRecords.withRecords(CompressionType.NONE, val records = MemoryRecords.withRecords(Compression.NONE,
new SimpleRecord(1000, "foo".getBytes(StandardCharsets.UTF_8))) new SimpleRecord(1000, "foo".getBytes(StandardCharsets.UTF_8)))
when(replicaManager.fetchMessages( when(replicaManager.fetchMessages(
any[FetchParams], any[FetchParams],
@ -6310,7 +6311,7 @@ class KafkaApisTest extends Logging {
.setHighWatermark(105) .setHighWatermark(105)
.setLastStableOffset(105) .setLastStableOffset(105)
.setLogStartOffset(0) .setLogStartOffset(0)
.setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord(100, raw.getBytes(StandardCharsets.UTF_8)))) .setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord(100, raw.getBytes(StandardCharsets.UTF_8))))
}.toMap.asJava) }.toMap.asJava)
data.foreach{case (tp, _) => data.foreach{case (tp, _) =>

View File

@ -30,6 +30,7 @@ import org.apache.kafka.common.metrics.Sensor
import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.record.{CompressionType, Records} import org.apache.kafka.common.record.{CompressionType, Records}
import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.compress.{GzipCompression, Lz4Compression, ZstdCompression}
import org.apache.kafka.coordinator.group.ConsumerGroupMigrationPolicy import org.apache.kafka.coordinator.group.ConsumerGroupMigrationPolicy
import org.apache.kafka.coordinator.group.Group.GroupType import org.apache.kafka.coordinator.group.Group.GroupType
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
@ -691,7 +692,6 @@ class KafkaConfigTest {
def testDefaultCompressionType(): Unit = { def testDefaultCompressionType(): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
val serverConfig = KafkaConfig.fromProps(props) val serverConfig = KafkaConfig.fromProps(props)
assertEquals(serverConfig.compressionType, "producer") assertEquals(serverConfig.compressionType, "producer")
} }
@ -700,7 +700,6 @@ class KafkaConfigTest {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
props.setProperty("compression.type", "gzip") props.setProperty("compression.type", "gzip")
val serverConfig = KafkaConfig.fromProps(props) val serverConfig = KafkaConfig.fromProps(props)
assertEquals(serverConfig.compressionType, "gzip") assertEquals(serverConfig.compressionType, "gzip")
} }
@ -711,6 +710,30 @@ class KafkaConfigTest {
assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)) assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props))
} }
@Test
def testInvalidGzipCompressionLevel(): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
props.setProperty(KafkaConfig.CompressionTypeProp, "gzip")
props.setProperty(KafkaConfig.CompressionGzipLevelProp, (GzipCompression.MAX_LEVEL + 1).toString)
assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props))
}
@Test
def testInvalidLz4CompressionLevel(): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
props.setProperty(KafkaConfig.CompressionTypeProp, "lz4")
props.setProperty(KafkaConfig.CompressionLz4LevelProp, (Lz4Compression.MAX_LEVEL + 1).toString)
assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props))
}
@Test
def testInvalidZstdCompressionLevel(): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
props.setProperty(KafkaConfig.CompressionTypeProp, "zstd")
props.setProperty(KafkaConfig.CompressionZstdLevelProp, (ZstdCompression.MAX_LEVEL + 1).toString)
assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props))
}
@Test @Test
def testInvalidInterBrokerSecurityProtocol(): Unit = { def testInvalidInterBrokerSecurityProtocol(): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
@ -917,6 +940,11 @@ class KafkaConfigTest {
case MetricConfigs.METRIC_SAMPLE_WINDOW_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "-1", "0") case MetricConfigs.METRIC_SAMPLE_WINDOW_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "-1", "0")
case MetricConfigs.METRIC_REPORTER_CLASSES_CONFIG => // ignore string case MetricConfigs.METRIC_REPORTER_CLASSES_CONFIG => // ignore string
case MetricConfigs.METRIC_RECORDING_LEVEL_CONFIG => // ignore string case MetricConfigs.METRIC_RECORDING_LEVEL_CONFIG => // ignore string
case KafkaConfig.CompressionGzipLevelProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
case KafkaConfig.CompressionLz4LevelProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
case KafkaConfig.CompressionZstdLevelProp => assertPropertyInvalid(baseProperties, name, "not_a_number", ZstdCompression.MAX_LEVEL + 1)
case KafkaConfig.RackProp => // ignore string case KafkaConfig.RackProp => // ignore string
//SSL Configs //SSL Configs
case KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG => case KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG =>
@ -1078,6 +1106,12 @@ class KafkaConfigTest {
assertDynamic(kafkaConfigProp, TopicConfig.CLEANUP_POLICY_COMPACT, () => config.logCleanupPolicy) assertDynamic(kafkaConfigProp, TopicConfig.CLEANUP_POLICY_COMPACT, () => config.logCleanupPolicy)
case TopicConfig.COMPRESSION_TYPE_CONFIG => case TopicConfig.COMPRESSION_TYPE_CONFIG =>
assertDynamic(kafkaConfigProp, "lz4", () => config.compressionType) assertDynamic(kafkaConfigProp, "lz4", () => config.compressionType)
case TopicConfig.COMPRESSION_GZIP_LEVEL_CONFIG =>
assertDynamic(kafkaConfigProp, "5", () => config.gzipCompressionLevel)
case TopicConfig.COMPRESSION_LZ4_LEVEL_CONFIG =>
assertDynamic(kafkaConfigProp, "5", () => config.lz4CompressionLevel)
case TopicConfig.COMPRESSION_ZSTD_LEVEL_CONFIG =>
assertDynamic(kafkaConfigProp, "5", () => config.zstdCompressionLevel)
case TopicConfig.SEGMENT_BYTES_CONFIG => case TopicConfig.SEGMENT_BYTES_CONFIG =>
assertDynamic(kafkaConfigProp, 10000, () => config.logSegmentBytes) assertDynamic(kafkaConfigProp, 10000, () => config.logSegmentBytes)
case TopicConfig.SEGMENT_MS_CONFIG => case TopicConfig.SEGMENT_MS_CONFIG =>

View File

@ -21,6 +21,7 @@ import java.nio.ByteBuffer
import java.util.{Collections, Properties} import java.util.{Collections, Properties}
import kafka.utils.TestUtils import kafka.utils.TestUtils
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.message.ProduceRequestData import org.apache.kafka.common.message.ProduceRequestData
import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.Errors
@ -74,10 +75,10 @@ class ProduceRequestTest extends BaseRequestTest {
assertTrue(partitionProduceResponse.recordErrors.isEmpty) assertTrue(partitionProduceResponse.recordErrors.isEmpty)
} }
sendAndCheck(MemoryRecords.withRecords(CompressionType.NONE, sendAndCheck(MemoryRecords.withRecords(Compression.NONE,
new SimpleRecord(System.currentTimeMillis(), "key".getBytes, "value".getBytes)), 0) new SimpleRecord(System.currentTimeMillis(), "key".getBytes, "value".getBytes)), 0)
sendAndCheck(MemoryRecords.withRecords(CompressionType.GZIP, sendAndCheck(MemoryRecords.withRecords(Compression.gzip().build(),
new SimpleRecord(System.currentTimeMillis(), "key1".getBytes, "value1".getBytes), new SimpleRecord(System.currentTimeMillis(), "key1".getBytes, "value1".getBytes),
new SimpleRecord(System.currentTimeMillis(), "key2".getBytes, "value2".getBytes)), 1) new SimpleRecord(System.currentTimeMillis(), "key2".getBytes, "value2".getBytes)), 1)
} }
@ -92,7 +93,7 @@ class ProduceRequestTest extends BaseRequestTest {
val partitionToLeader = TestUtils.createTopic(zkClient, topic, 1, 1, servers, topicConfig) val partitionToLeader = TestUtils.createTopic(zkClient, topic, 1, 1, servers, topicConfig)
val leader = partitionToLeader(partition) val leader = partitionToLeader(partition)
def createRecords(magicValue: Byte, timestamp: Long, codec: CompressionType): MemoryRecords = { def createRecords(magicValue: Byte, timestamp: Long, codec: Compression): MemoryRecords = {
val buf = ByteBuffer.allocate(512) val buf = ByteBuffer.allocate(512)
val builder = MemoryRecords.builder(buf, magicValue, codec, TimestampType.CREATE_TIME, 0L) val builder = MemoryRecords.builder(buf, magicValue, codec, TimestampType.CREATE_TIME, 0L)
builder.appendWithOffset(0, timestamp, null, "hello".getBytes) builder.appendWithOffset(0, timestamp, null, "hello".getBytes)
@ -101,7 +102,7 @@ class ProduceRequestTest extends BaseRequestTest {
builder.build() builder.build()
} }
val records = createRecords(RecordBatch.MAGIC_VALUE_V2, recordTimestamp, CompressionType.GZIP) val records = createRecords(RecordBatch.MAGIC_VALUE_V2, recordTimestamp, Compression.gzip().build())
val topicPartition = new TopicPartition("topic", partition) val topicPartition = new TopicPartition("topic", partition)
val produceResponse = sendProduceRequest(leader, ProduceRequest.forCurrentMagic(new ProduceRequestData() val produceResponse = sendProduceRequest(leader, ProduceRequest.forCurrentMagic(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
@ -143,7 +144,7 @@ class ProduceRequestTest extends BaseRequestTest {
val nonReplicaId = nonReplicaOpt.get.config.brokerId val nonReplicaId = nonReplicaOpt.get.config.brokerId
// Send the produce request to the non-replica // Send the produce request to the non-replica
val records = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("key".getBytes, "value".getBytes)) val records = MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("key".getBytes, "value".getBytes))
val topicPartition = new TopicPartition("topic", partition) val topicPartition = new TopicPartition("topic", partition)
val produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData() val produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData()
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList( .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
@ -177,7 +178,7 @@ class ProduceRequestTest extends BaseRequestTest {
def testCorruptLz4ProduceRequest(quorum: String): Unit = { def testCorruptLz4ProduceRequest(quorum: String): Unit = {
val (partition, leader) = createTopicAndFindPartitionWithLeader("topic") val (partition, leader) = createTopicAndFindPartitionWithLeader("topic")
val timestamp = 1000000 val timestamp = 1000000
val memoryRecords = MemoryRecords.withRecords(CompressionType.LZ4, val memoryRecords = MemoryRecords.withRecords(Compression.lz4().build(),
new SimpleRecord(timestamp, "key".getBytes, "value".getBytes)) new SimpleRecord(timestamp, "key".getBytes, "value".getBytes))
// Change the lz4 checksum value (not the kafka record crc) so that it doesn't match the contents // Change the lz4 checksum value (not the kafka record crc) so that it doesn't match the contents
val lz4ChecksumOffset = 6 val lz4ChecksumOffset = 6
@ -218,7 +219,7 @@ class ProduceRequestTest extends BaseRequestTest {
topicConfig.setProperty(TopicConfig.COMPRESSION_TYPE_CONFIG, BrokerCompressionType.ZSTD.name) topicConfig.setProperty(TopicConfig.COMPRESSION_TYPE_CONFIG, BrokerCompressionType.ZSTD.name)
val partitionToLeader = createTopic(topic, topicConfig = topicConfig) val partitionToLeader = createTopic(topic, topicConfig = topicConfig)
val leader = partitionToLeader(partition) val leader = partitionToLeader(partition)
val memoryRecords = MemoryRecords.withRecords(CompressionType.ZSTD, val memoryRecords = MemoryRecords.withRecords(Compression.zstd().build(),
new SimpleRecord(System.currentTimeMillis(), "key".getBytes, "value".getBytes)) new SimpleRecord(System.currentTimeMillis(), "key".getBytes, "value".getBytes))
val topicPartition = new TopicPartition("topic", partition) val topicPartition = new TopicPartition("topic", partition)
val partitionRecords = new ProduceRequestData() val partitionRecords = new ProduceRequestData()

View File

@ -24,6 +24,7 @@ import kafka.server.epoch.util.MockBlockingSender
import kafka.server.metadata.ZkMetadataCache import kafka.server.metadata.ZkMetadataCache
import kafka.utils.TestUtils import kafka.utils.TestUtils
import org.apache.kafka.clients.FetchSessionHandler import org.apache.kafka.clients.FetchSessionHandler
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid}
import org.apache.kafka.common.message.{FetchResponseData, UpdateMetadataRequestData} import org.apache.kafka.common.message.{FetchResponseData, UpdateMetadataRequestData}
import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition
@ -1331,7 +1332,7 @@ class ReplicaFetcherThreadTest {
val tp0 = new TopicPartition("testTopic", 0) val tp0 = new TopicPartition("testTopic", 0)
val tp1 = new TopicPartition("testTopic", 1) val tp1 = new TopicPartition("testTopic", 1)
val records = MemoryRecords.withRecords(CompressionType.NONE, val records = MemoryRecords.withRecords(Compression.NONE,
new SimpleRecord(1000, "foo".getBytes(StandardCharsets.UTF_8))) new SimpleRecord(1000, "foo".getBytes(StandardCharsets.UTF_8)))
val partitionData = new FetchResponseData.PartitionData() val partitionData = new FetchResponseData.PartitionData()
.setRecords(records) .setRecords(records)
@ -1380,7 +1381,7 @@ class ReplicaFetcherThreadTest {
when(mockBlockingSend.brokerEndPoint()).thenReturn(brokerEndPoint) when(mockBlockingSend.brokerEndPoint()).thenReturn(brokerEndPoint)
val log: UnifiedLog = mock(classOf[UnifiedLog]) val log: UnifiedLog = mock(classOf[UnifiedLog])
val records = MemoryRecords.withRecords(CompressionType.NONE, val records = MemoryRecords.withRecords(Compression.NONE,
new SimpleRecord(1000, "foo".getBytes(StandardCharsets.UTF_8))) new SimpleRecord(1000, "foo".getBytes(StandardCharsets.UTF_8)))
when(log.maybeUpdateHighWatermark(hw = 0)).thenReturn(None) when(log.maybeUpdateHighWatermark(hw = 0)).thenReturn(None)

View File

@ -22,9 +22,10 @@ import kafka.cluster.{Partition, PartitionTest}
import kafka.log.{LogManager, UnifiedLog} import kafka.log.{LogManager, UnifiedLog}
import kafka.server.QuotaFactory.QuotaManagers import kafka.server.QuotaFactory.QuotaManagers
import kafka.utils._ import kafka.utils._
import org.apache.kafka.common.compress.Compression
import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.protocol.ApiKeys
import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord} import org.apache.kafka.common.record.{MemoryRecords, SimpleRecord}
import org.apache.kafka.common.requests.FetchRequest import org.apache.kafka.common.requests.FetchRequest
import org.apache.kafka.common.requests.FetchRequest.PartitionData import org.apache.kafka.common.requests.FetchRequest.PartitionData
import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid}
@ -270,7 +271,7 @@ class ReplicaManagerQuotasTest {
minOneMessage = anyBoolean)).thenReturn( minOneMessage = anyBoolean)).thenReturn(
new FetchDataInfo( new FetchDataInfo(
new LogOffsetMetadata(0L, 0L, 0), new LogOffsetMetadata(0L, 0L, 0),
MemoryRecords.withRecords(CompressionType.NONE, record) MemoryRecords.withRecords(Compression.NONE, record)
)) ))
//if we ask for len = 0, return 0 messages //if we ask for len = 0, return 0 messages

Some files were not shown because too many files have changed in this diff Show More