mirror of https://github.com/apache/kafka.git
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:
parent
9fe3932e5c
commit
affe8da54c
|
|
@ -51,6 +51,7 @@
|
|||
<allow pkg="org.apache.kafka.clients.consumer" />
|
||||
<allow pkg="org.apache.kafka.common.annotation" />
|
||||
<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.message" />
|
||||
<allow pkg="org.apache.kafka.common.metadata" />
|
||||
|
|
|
|||
|
|
@ -173,18 +173,22 @@
|
|||
<allow pkg="org.apache.kafka.metadata" />
|
||||
<allow pkg="org.apache.kafka.common.internals" />
|
||||
</subpackage>
|
||||
<subpackage name="migration">
|
||||
<allow pkg="org.apache.kafka.controller" />
|
||||
</subpackage>
|
||||
<subpackage name="bootstrap">
|
||||
<allow pkg="org.apache.kafka.snapshot" />
|
||||
</subpackage>
|
||||
<subpackage name="fault">
|
||||
<allow pkg="org.apache.kafka.server.fault" />
|
||||
</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 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.protocol" />
|
||||
<allow pkg="org.apache.kafka.common.record" />
|
||||
|
|
|
|||
|
|
@ -76,8 +76,10 @@
|
|||
<allow pkg="com.github.luben.zstd" />
|
||||
<allow pkg="net.jpountz.lz4" />
|
||||
<allow pkg="net.jpountz.xxhash" />
|
||||
<allow pkg="org.apache.kafka.common.compress" />
|
||||
<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 name="message">
|
||||
|
|
@ -138,6 +140,7 @@
|
|||
|
||||
<subpackage name="protocol">
|
||||
<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.network" />
|
||||
<allow pkg="org.apache.kafka.common.protocol" />
|
||||
|
|
@ -166,6 +169,7 @@
|
|||
|
||||
<subpackage name="requests">
|
||||
<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.protocol" />
|
||||
<allow pkg="org.apache.kafka.common.message" />
|
||||
|
|
@ -426,6 +430,7 @@
|
|||
<allow pkg="org.apache.kafka.metadata" />
|
||||
<allow pkg="org.apache.kafka.snapshot" />
|
||||
<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.feature" />
|
||||
<allow pkg="org.apache.kafka.common.message" />
|
||||
|
|
@ -444,8 +449,9 @@
|
|||
</subpackage>
|
||||
|
||||
<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.record" />
|
||||
<allow pkg="org.apache.kafka.raft" />
|
||||
<allow pkg="org.apache.kafka.server.common" />
|
||||
<allow pkg="org.apache.kafka.test"/>
|
||||
|
|
|
|||
|
|
@ -90,7 +90,7 @@
|
|||
files="(Errors|SaslAuthenticatorTest|AgentTest|CoordinatorTest|NetworkClientTest).java"/>
|
||||
|
||||
<suppress checks="BooleanExpressionComplexity"
|
||||
files="(Utils|Topic|KafkaLZ4BlockOutputStream|AclData|JoinGroupRequest).java"/>
|
||||
files="(Utils|Topic|Lz4BlockOutputStream|AclData|JoinGroupRequest).java"/>
|
||||
|
||||
<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"/>
|
||||
|
|
|
|||
|
|
@ -41,6 +41,7 @@ import org.apache.kafka.common.MetricName;
|
|||
import org.apache.kafka.common.PartitionInfo;
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
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.errors.ApiException;
|
||||
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 Sender sender;
|
||||
private final Thread ioThread;
|
||||
private final CompressionType compressionType;
|
||||
private final Compression compression;
|
||||
private final Sensor errors;
|
||||
private final Time time;
|
||||
private final Serializer<K> keySerializer;
|
||||
|
|
@ -413,7 +414,7 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
|
|||
Arrays.asList(this.keySerializer, this.valueSerializer));
|
||||
this.maxRequestSize = config.getInt(ProducerConfig.MAX_REQUEST_SIZE_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);
|
||||
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));
|
||||
this.accumulator = new RecordAccumulator(logContext,
|
||||
batchSize,
|
||||
this.compressionType,
|
||||
compression,
|
||||
lingerMs(config),
|
||||
retryBackoffMs,
|
||||
retryBackoffMaxMs,
|
||||
|
|
@ -501,7 +502,7 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
|
|||
this.interceptors = interceptors;
|
||||
this.maxRequestSize = config.getInt(ProducerConfig.MAX_REQUEST_SIZE_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.partitionerIgnoreKeys = config.getBoolean(ProducerConfig.PARTITIONER_IGNORE_KEYS_CONFIG);
|
||||
this.apiVersions = new ApiVersions();
|
||||
|
|
@ -548,6 +549,29 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
|
|||
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) {
|
||||
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();
|
||||
|
||||
int serializedSize = AbstractRecords.estimateSizeInBytesUpperBound(apiVersions.maxUsableProduceMagic(),
|
||||
compressionType, serializedKey, serializedValue, headers);
|
||||
compression.type(), serializedKey, serializedValue, headers);
|
||||
ensureValidRecordSize(serializedSize);
|
||||
long timestamp = record.timestamp() == null ? nowMs : record.timestamp();
|
||||
|
||||
|
|
|
|||
|
|
@ -18,6 +18,9 @@ package org.apache.kafka.clients.producer;
|
|||
|
||||
import org.apache.kafka.clients.ClientDnsLookup;
|
||||
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.ConfigDef;
|
||||
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>. "
|
||||
+ "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> */
|
||||
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,
|
||||
ACKS_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(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)
|
||||
|
|
|
|||
|
|
@ -99,7 +99,7 @@ public final class ProducerBatch {
|
|||
this.retry = false;
|
||||
this.isSplitBatch = isSplitBatch;
|
||||
float compressionRatioEstimation = CompressionRatioEstimator.estimation(topicPartition.topic(),
|
||||
recordsBuilder.compressionType());
|
||||
recordsBuilder.compression().type());
|
||||
this.currentLeaderEpoch = OptionalInt.empty();
|
||||
this.attemptsWhenLeaderLastChanged = 0;
|
||||
recordsBuilder.setEstimatedCompressionRatio(compressionRatioEstimation);
|
||||
|
|
@ -146,7 +146,7 @@ public final class ProducerBatch {
|
|||
} else {
|
||||
this.recordsBuilder.append(timestamp, key, value, headers);
|
||||
this.maxRecordSize = Math.max(this.maxRecordSize, AbstractRecords.estimateSizeInBytesUpperBound(magic(),
|
||||
recordsBuilder.compressionType(), key, value, headers));
|
||||
recordsBuilder.compression().type(), key, value, headers));
|
||||
this.lastAppendTime = now;
|
||||
FutureRecordMetadata future = new FutureRecordMetadata(this.produceFuture, this.recordCount,
|
||||
timestamp,
|
||||
|
|
@ -172,7 +172,7 @@ public final class ProducerBatch {
|
|||
// No need to get the CRC.
|
||||
this.recordsBuilder.append(timestamp, key, value, headers);
|
||||
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,
|
||||
timestamp,
|
||||
key == null ? -1 : key.remaining(),
|
||||
|
|
@ -377,19 +377,19 @@ public final class ProducerBatch {
|
|||
|
||||
private ProducerBatch createBatchOffAccumulatorForRecord(Record record, int batchSize) {
|
||||
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);
|
||||
|
||||
// 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
|
||||
// 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);
|
||||
return new ProducerBatch(topicPartition, builder, this.createdMs, true);
|
||||
}
|
||||
|
||||
public boolean isCompressed() {
|
||||
return recordsBuilder.compressionType() != CompressionType.NONE;
|
||||
return recordsBuilder.compression().type() != CompressionType.NONE;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -491,7 +491,7 @@ public final class ProducerBatch {
|
|||
recordsBuilder.close();
|
||||
if (!recordsBuilder.isControlBatch()) {
|
||||
CompressionRatioEstimator.updateEstimation(topicPartition.topic(),
|
||||
recordsBuilder.compressionType(),
|
||||
recordsBuilder.compression().type(),
|
||||
(float) recordsBuilder.compressionRatio());
|
||||
}
|
||||
reopened = false;
|
||||
|
|
|
|||
|
|
@ -36,6 +36,7 @@ import org.apache.kafka.clients.CommonClientConfigs;
|
|||
import org.apache.kafka.clients.MetadataSnapshot;
|
||||
import org.apache.kafka.clients.producer.Callback;
|
||||
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.ProducerIdAndEpoch;
|
||||
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.record.AbstractRecords;
|
||||
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.MemoryRecordsBuilder;
|
||||
import org.apache.kafka.common.record.Record;
|
||||
|
|
@ -74,7 +74,7 @@ public class RecordAccumulator {
|
|||
private final AtomicInteger flushesInProgress;
|
||||
private final AtomicInteger appendsInProgress;
|
||||
private final int batchSize;
|
||||
private final CompressionType compression;
|
||||
private final Compression compression;
|
||||
private final int lingerMs;
|
||||
private final ExponentialBackoff retryBackoff;
|
||||
private final int deliveryTimeoutMs;
|
||||
|
|
@ -116,7 +116,7 @@ public class RecordAccumulator {
|
|||
*/
|
||||
public RecordAccumulator(LogContext logContext,
|
||||
int batchSize,
|
||||
CompressionType compression,
|
||||
Compression compression,
|
||||
int lingerMs,
|
||||
long retryBackoffMs,
|
||||
long retryBackoffMaxMs,
|
||||
|
|
@ -176,7 +176,7 @@ public class RecordAccumulator {
|
|||
*/
|
||||
public RecordAccumulator(LogContext logContext,
|
||||
int batchSize,
|
||||
CompressionType compression,
|
||||
Compression compression,
|
||||
int lingerMs,
|
||||
long retryBackoffMs,
|
||||
long retryBackoffMaxMs,
|
||||
|
|
@ -344,7 +344,7 @@ public class RecordAccumulator {
|
|||
|
||||
if (buffer == null) {
|
||||
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);
|
||||
// This call may block if we exhausted buffer space.
|
||||
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
|
||||
// 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.
|
||||
CompressionRatioEstimator.setEstimation(bigBatch.topicPartition.topic(), compression,
|
||||
CompressionRatioEstimator.setEstimation(bigBatch.topicPartition.topic(), compression.type(),
|
||||
Math.max(1.0f, (float) bigBatch.compressionRatio()));
|
||||
Deque<ProducerBatch> dq = bigBatch.split(this.batchSize);
|
||||
int numSplitBatches = dq.size();
|
||||
|
|
|
|||
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
@ -23,8 +23,8 @@ import net.jpountz.lz4.LZ4SafeDecompressor;
|
|||
import net.jpountz.xxhash.XXHash32;
|
||||
import net.jpountz.xxhash.XXHashFactory;
|
||||
|
||||
import org.apache.kafka.common.compress.KafkaLZ4BlockOutputStream.BD;
|
||||
import org.apache.kafka.common.compress.KafkaLZ4BlockOutputStream.FLG;
|
||||
import org.apache.kafka.common.compress.Lz4BlockOutputStream.BD;
|
||||
import org.apache.kafka.common.compress.Lz4BlockOutputStream.FLG;
|
||||
import org.apache.kafka.common.utils.BufferSupplier;
|
||||
|
||||
import java.io.IOException;
|
||||
|
|
@ -32,8 +32,8 @@ import java.io.InputStream;
|
|||
import java.nio.ByteBuffer;
|
||||
import java.nio.ByteOrder;
|
||||
|
||||
import static org.apache.kafka.common.compress.KafkaLZ4BlockOutputStream.LZ4_FRAME_INCOMPRESSIBLE_MASK;
|
||||
import static org.apache.kafka.common.compress.KafkaLZ4BlockOutputStream.MAGIC;
|
||||
import static org.apache.kafka.common.compress.Lz4BlockOutputStream.LZ4_FRAME_INCOMPRESSIBLE_MASK;
|
||||
import static org.apache.kafka.common.compress.Lz4BlockOutputStream.MAGIC;
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*/
|
||||
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 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
|
||||
* @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) {
|
||||
throw BROKEN_LZ4_EXCEPTION;
|
||||
}
|
||||
|
|
@ -33,18 +33,14 @@ import net.jpountz.xxhash.XXHashFactory;
|
|||
*
|
||||
* 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 LZ4_MAX_HEADER_LENGTH = 19;
|
||||
public static final int LZ4_FRAME_INCOMPRESSIBLE_MASK = 0x80000000;
|
||||
|
||||
public static final String CLOSED_STREAM = "The stream is already closed";
|
||||
|
||||
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 XXHash32 checksum;
|
||||
|
|
@ -64,15 +60,22 @@ public final class KafkaLZ4BlockOutputStream extends OutputStream {
|
|||
* @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
|
||||
* 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
|
||||
* every block of data
|
||||
* @param useBrokenFlagDescriptorChecksum Default: false. When true, writes an incorrect FrameDescriptor checksum
|
||||
* compatible with older kafka clients.
|
||||
* @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;
|
||||
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();
|
||||
this.useBrokenFlagDescriptorChecksum = useBrokenFlagDescriptorChecksum;
|
||||
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.
|
||||
*
|
||||
* @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
|
||||
* values will generate an exception
|
||||
* @param blockChecksum Default: false. When true, a XXHash32 checksum is computed and appended to the stream for
|
||||
* every block of data
|
||||
* @param level The compression level to use
|
||||
* @param useBrokenFlagDescriptorChecksum Default: false. When true, writes an incorrect FrameDescriptor checksum
|
||||
* compatible with older kafka clients.
|
||||
* @throws IOException
|
||||
*/
|
||||
public KafkaLZ4BlockOutputStream(OutputStream out, int blockSize, boolean blockChecksum) throws IOException {
|
||||
this(out, blockSize, blockChecksum, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new {@link OutputStream} that will compress data using the LZ4 algorithm.
|
||||
*
|
||||
* @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);
|
||||
public Lz4BlockOutputStream(OutputStream out, int level, boolean useBrokenFlagDescriptorChecksum) throws IOException {
|
||||
this(out, BLOCKSIZE_64KB, level, false, useBrokenFlagDescriptorChecksum);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -292,10 +269,6 @@ public final class KafkaLZ4BlockOutputStream extends OutputStream {
|
|||
private final int blockIndependence;
|
||||
private final int version;
|
||||
|
||||
public FLG() {
|
||||
this(false);
|
||||
}
|
||||
|
||||
public FLG(boolean blockChecksum) {
|
||||
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 reserved3;
|
||||
|
||||
public BD() {
|
||||
this(0, BLOCKSIZE_64KB, 0);
|
||||
}
|
||||
|
||||
public BD(int blockSizeValue) {
|
||||
this(0, blockSizeValue, 0);
|
||||
}
|
||||
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -14,37 +14,41 @@
|
|||
* 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.xerial.snappy.SnappyInputStream;
|
||||
import org.xerial.snappy.SnappyOutputStream;
|
||||
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
public class SnappyFactory {
|
||||
public class NoCompression implements Compression {
|
||||
|
||||
private SnappyFactory() { }
|
||||
private NoCompression() {}
|
||||
|
||||
public static OutputStream wrapForOutput(ByteBufferOutputStream buffer) {
|
||||
try {
|
||||
return new SnappyOutputStream(buffer);
|
||||
} catch (Throwable e) {
|
||||
throw new KafkaException(e);
|
||||
}
|
||||
@Override
|
||||
public CompressionType type() {
|
||||
return CompressionType.NONE;
|
||||
}
|
||||
|
||||
public static InputStream wrapForInput(ByteBuffer buffer) {
|
||||
try {
|
||||
return new SnappyInputStream(new ByteBufferInputStream(buffer));
|
||||
} catch (Throwable e) {
|
||||
throw new KafkaException(e);
|
||||
}
|
||||
@Override
|
||||
public OutputStream wrapForOutput(ByteBufferOutputStream bufferStream, byte messageVersion) {
|
||||
return bufferStream;
|
||||
}
|
||||
|
||||
@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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -173,6 +173,14 @@ public class TopicConfig {
|
|||
"accepts 'uncompressed' which is equivalent to no compression; and 'producer' which means retain the " +
|
||||
"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_DOC = "True if we should preallocate the file on disk when " +
|
||||
"creating a new log segment.";
|
||||
|
|
|
|||
|
|
@ -18,6 +18,7 @@ package org.apache.kafka.common.record;
|
|||
|
||||
import org.apache.kafka.common.InvalidRecordException;
|
||||
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.header.Header;
|
||||
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 = " +
|
||||
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);
|
||||
|
||||
long lastOffsetFromWrapper = wrapperEntry.lastOffset();
|
||||
|
|
|
|||
|
|
@ -16,158 +16,23 @@
|
|||
*/
|
||||
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
|
||||
*/
|
||||
public enum CompressionType {
|
||||
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);
|
||||
}
|
||||
},
|
||||
NONE((byte) 0, "none", 1.0f),
|
||||
|
||||
// Shipped with the JDK
|
||||
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;
|
||||
}
|
||||
},
|
||||
GZIP((byte) 1, "gzip", 1.0f),
|
||||
|
||||
// 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
|
||||
// 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
|
||||
// happens.
|
||||
|
||||
SNAPPY((byte) 2, "snappy", 1.0f) {
|
||||
@Override
|
||||
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;
|
||||
}
|
||||
|
||||
|
||||
};
|
||||
SNAPPY((byte) 2, "snappy", 1.0f),
|
||||
LZ4((byte) 3, "lz4", 1.0f),
|
||||
ZSTD((byte) 4, "zstd", 1.0f);
|
||||
|
||||
// compression type is represented by two bits in the attributes field of the record batch header, so `byte` is
|
||||
// large enough
|
||||
|
|
@ -181,34 +46,6 @@ public enum CompressionType {
|
|||
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) {
|
||||
switch (id) {
|
||||
case 0:
|
||||
|
|
|
|||
|
|
@ -18,6 +18,7 @@ package org.apache.kafka.common.record;
|
|||
|
||||
import org.apache.kafka.common.InvalidRecordException;
|
||||
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.header.Header;
|
||||
import org.apache.kafka.common.utils.BufferSupplier;
|
||||
|
|
@ -270,7 +271,7 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe
|
|||
public InputStream recordInputStream(BufferSupplier bufferSupplier) {
|
||||
final ByteBuffer buffer = this.buffer.duplicate();
|
||||
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) {
|
||||
|
|
|
|||
|
|
@ -17,6 +17,7 @@
|
|||
package org.apache.kafka.common.record;
|
||||
|
||||
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.message.KRaftVersionRecord;
|
||||
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
|
||||
* 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 {
|
||||
private static final Logger log = LoggerFactory.getLogger(MemoryRecords.class);
|
||||
|
|
@ -293,6 +294,7 @@ public class MemoryRecords extends AbstractRecords {
|
|||
ByteBufferOutputStream bufferOutputStream,
|
||||
final long deleteHorizonMs) {
|
||||
byte magic = originalBatch.magic();
|
||||
Compression compression = Compression.of(originalBatch.compressionType()).build();
|
||||
TimestampType timestampType = originalBatch.timestampType();
|
||||
long logAppendTime = timestampType == TimestampType.LOG_APPEND_TIME ?
|
||||
originalBatch.maxTimestamp() : RecordBatch.NO_TIMESTAMP;
|
||||
|
|
@ -300,7 +302,7 @@ public class MemoryRecords extends AbstractRecords {
|
|||
originalBatch.baseOffset() : retainedRecords.get(0).offset();
|
||||
|
||||
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.isControlBatch(), originalBatch.partitionLeaderEpoch(), bufferOutputStream.limit(), deleteHorizonMs);
|
||||
|
||||
|
|
@ -470,14 +472,14 @@ public class MemoryRecords extends AbstractRecords {
|
|||
}
|
||||
|
||||
public static MemoryRecordsBuilder builder(ByteBuffer buffer,
|
||||
CompressionType compressionType,
|
||||
Compression compression,
|
||||
TimestampType timestampType,
|
||||
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,
|
||||
CompressionType compressionType,
|
||||
Compression compression,
|
||||
TimestampType timestampType,
|
||||
long baseOffset,
|
||||
int maxSize) {
|
||||
|
|
@ -485,84 +487,84 @@ public class MemoryRecords extends AbstractRecords {
|
|||
if (timestampType == TimestampType.LOG_APPEND_TIME)
|
||||
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,
|
||||
false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, maxSize);
|
||||
}
|
||||
|
||||
public static MemoryRecordsBuilder idempotentBuilder(ByteBuffer buffer,
|
||||
CompressionType compressionType,
|
||||
Compression compression,
|
||||
long baseOffset,
|
||||
long producerId,
|
||||
short producerEpoch,
|
||||
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);
|
||||
}
|
||||
|
||||
public static MemoryRecordsBuilder builder(ByteBuffer buffer,
|
||||
byte magic,
|
||||
CompressionType compressionType,
|
||||
Compression compression,
|
||||
TimestampType timestampType,
|
||||
long baseOffset,
|
||||
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_PARTITION_LEADER_EPOCH);
|
||||
}
|
||||
|
||||
public static MemoryRecordsBuilder builder(ByteBuffer buffer,
|
||||
byte magic,
|
||||
CompressionType compressionType,
|
||||
Compression compression,
|
||||
TimestampType timestampType,
|
||||
long baseOffset) {
|
||||
long logAppendTime = RecordBatch.NO_TIMESTAMP;
|
||||
if (timestampType == TimestampType.LOG_APPEND_TIME)
|
||||
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_PARTITION_LEADER_EPOCH);
|
||||
}
|
||||
|
||||
public static MemoryRecordsBuilder builder(ByteBuffer buffer,
|
||||
byte magic,
|
||||
CompressionType compressionType,
|
||||
Compression compression,
|
||||
TimestampType timestampType,
|
||||
long baseOffset,
|
||||
long logAppendTime,
|
||||
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);
|
||||
}
|
||||
|
||||
public static MemoryRecordsBuilder builder(ByteBuffer buffer,
|
||||
CompressionType compressionType,
|
||||
Compression compression,
|
||||
long baseOffset,
|
||||
long producerId,
|
||||
short producerEpoch,
|
||||
int baseSequence,
|
||||
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_PARTITION_LEADER_EPOCH);
|
||||
}
|
||||
|
||||
public static MemoryRecordsBuilder builder(ByteBuffer buffer,
|
||||
byte magic,
|
||||
CompressionType compressionType,
|
||||
Compression compression,
|
||||
TimestampType timestampType,
|
||||
long baseOffset,
|
||||
long logAppendTime,
|
||||
long producerId,
|
||||
short producerEpoch,
|
||||
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);
|
||||
}
|
||||
|
||||
public static MemoryRecordsBuilder builder(ByteBuffer buffer,
|
||||
byte magic,
|
||||
CompressionType compressionType,
|
||||
Compression compression,
|
||||
TimestampType timestampType,
|
||||
long baseOffset,
|
||||
long logAppendTime,
|
||||
|
|
@ -571,13 +573,13 @@ public class MemoryRecords extends AbstractRecords {
|
|||
int baseSequence,
|
||||
boolean isTransactional,
|
||||
int partitionLeaderEpoch) {
|
||||
return builder(buffer, magic, compressionType, timestampType, baseOffset,
|
||||
return builder(buffer, magic, compression, timestampType, baseOffset,
|
||||
logAppendTime, producerId, producerEpoch, baseSequence, isTransactional, false, partitionLeaderEpoch);
|
||||
}
|
||||
|
||||
public static MemoryRecordsBuilder builder(ByteBuffer buffer,
|
||||
byte magic,
|
||||
CompressionType compressionType,
|
||||
Compression compression,
|
||||
TimestampType timestampType,
|
||||
long baseOffset,
|
||||
long logAppendTime,
|
||||
|
|
@ -587,98 +589,98 @@ public class MemoryRecords extends AbstractRecords {
|
|||
boolean isTransactional,
|
||||
boolean isControlBatch,
|
||||
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,
|
||||
buffer.remaining());
|
||||
}
|
||||
|
||||
public static MemoryRecords withRecords(CompressionType compressionType, SimpleRecord... records) {
|
||||
return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, compressionType, records);
|
||||
public static MemoryRecords withRecords(Compression compression, SimpleRecord... records) {
|
||||
return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, compression, records);
|
||||
}
|
||||
|
||||
public static MemoryRecords withRecords(CompressionType compressionType, int partitionLeaderEpoch, SimpleRecord... records) {
|
||||
return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, 0L, compressionType, TimestampType.CREATE_TIME,
|
||||
public static MemoryRecords withRecords(Compression compression, int partitionLeaderEpoch, SimpleRecord... records) {
|
||||
return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, 0L, compression, TimestampType.CREATE_TIME,
|
||||
RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE,
|
||||
partitionLeaderEpoch, false, records);
|
||||
}
|
||||
|
||||
public static MemoryRecords withRecords(byte magic, CompressionType compressionType, SimpleRecord... records) {
|
||||
return withRecords(magic, 0L, compressionType, TimestampType.CREATE_TIME, records);
|
||||
public static MemoryRecords withRecords(byte magic, Compression compression, SimpleRecord... records) {
|
||||
return withRecords(magic, 0L, compression, TimestampType.CREATE_TIME, records);
|
||||
}
|
||||
|
||||
public static MemoryRecords withRecords(long initialOffset, CompressionType compressionType, SimpleRecord... records) {
|
||||
return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, initialOffset, compressionType, TimestampType.CREATE_TIME,
|
||||
public static MemoryRecords withRecords(long initialOffset, Compression compression, SimpleRecord... records) {
|
||||
return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, initialOffset, compression, TimestampType.CREATE_TIME,
|
||||
records);
|
||||
}
|
||||
|
||||
public static MemoryRecords withRecords(byte magic, long initialOffset, CompressionType compressionType, SimpleRecord... records) {
|
||||
return withRecords(magic, initialOffset, compressionType, TimestampType.CREATE_TIME, records);
|
||||
public static MemoryRecords withRecords(byte magic, long initialOffset, Compression compression, SimpleRecord... records) {
|
||||
return withRecords(magic, initialOffset, compression, TimestampType.CREATE_TIME, records);
|
||||
}
|
||||
|
||||
public static MemoryRecords withRecords(long initialOffset, CompressionType compressionType, Integer partitionLeaderEpoch, SimpleRecord... records) {
|
||||
return withRecords(RecordBatch.CURRENT_MAGIC_VALUE, initialOffset, compressionType, TimestampType.CREATE_TIME, RecordBatch.NO_PRODUCER_ID,
|
||||
public static MemoryRecords withRecords(long initialOffset, Compression compression, Integer partitionLeaderEpoch, SimpleRecord... records) {
|
||||
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);
|
||||
}
|
||||
|
||||
public static MemoryRecords withIdempotentRecords(CompressionType compressionType, long producerId,
|
||||
public static MemoryRecords withIdempotentRecords(Compression compression, long producerId,
|
||||
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);
|
||||
}
|
||||
|
||||
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,
|
||||
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);
|
||||
}
|
||||
|
||||
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,
|
||||
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);
|
||||
}
|
||||
|
||||
public static MemoryRecords withTransactionalRecords(CompressionType compressionType, long producerId,
|
||||
public static MemoryRecords withTransactionalRecords(Compression compression, long producerId,
|
||||
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);
|
||||
}
|
||||
|
||||
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,
|
||||
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);
|
||||
}
|
||||
|
||||
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,
|
||||
SimpleRecord... records) {
|
||||
return withTransactionalRecords(RecordBatch.CURRENT_MAGIC_VALUE, initialOffset, compressionType,
|
||||
return withTransactionalRecords(RecordBatch.CURRENT_MAGIC_VALUE, initialOffset, compression,
|
||||
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) {
|
||||
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,
|
||||
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,
|
||||
int baseSequence, int partitionLeaderEpoch, boolean isTransactional,
|
||||
SimpleRecord... records) {
|
||||
if (records.length == 0)
|
||||
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);
|
||||
long logAppendTime = RecordBatch.NO_TIMESTAMP;
|
||||
if (timestampType == TimestampType.LOG_APPEND_TIME)
|
||||
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,
|
||||
partitionLeaderEpoch, sizeEstimate)) {
|
||||
for (SimpleRecord record : records)
|
||||
|
|
@ -714,7 +716,7 @@ public class MemoryRecords extends AbstractRecords {
|
|||
int partitionLeaderEpoch, long producerId, short producerEpoch,
|
||||
EndTransactionMarker marker) {
|
||||
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,
|
||||
RecordBatch.NO_SEQUENCE, isTransactional, true, partitionLeaderEpoch,
|
||||
buffer.capacity())
|
||||
|
|
@ -827,7 +829,7 @@ public class MemoryRecords extends AbstractRecords {
|
|||
return new MemoryRecordsBuilder(
|
||||
buffer,
|
||||
RecordBatch.CURRENT_MAGIC_VALUE,
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
TimestampType.CREATE_TIME,
|
||||
initialOffset,
|
||||
timestamp,
|
||||
|
|
|
|||
|
|
@ -17,6 +17,7 @@
|
|||
package org.apache.kafka.common.record;
|
||||
|
||||
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.message.KRaftVersionRecord;
|
||||
import org.apache.kafka.common.message.LeaderChangeMessage;
|
||||
|
|
@ -54,7 +55,7 @@ public class MemoryRecordsBuilder implements AutoCloseable {
|
|||
});
|
||||
|
||||
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
|
||||
// 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.
|
||||
|
|
@ -92,7 +93,7 @@ public class MemoryRecordsBuilder implements AutoCloseable {
|
|||
|
||||
public MemoryRecordsBuilder(ByteBufferOutputStream bufferStream,
|
||||
byte magic,
|
||||
CompressionType compressionType,
|
||||
Compression compression,
|
||||
TimestampType timestampType,
|
||||
long baseOffset,
|
||||
long logAppendTime,
|
||||
|
|
@ -111,7 +112,7 @@ public class MemoryRecordsBuilder implements AutoCloseable {
|
|||
throw new IllegalArgumentException("Transactional records are not supported for magic " + magic);
|
||||
if (isControlBatch)
|
||||
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);
|
||||
if (deleteHorizonMs != RecordBatch.NO_TIMESTAMP)
|
||||
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.timestampType = timestampType;
|
||||
this.compressionType = compressionType;
|
||||
this.compression = compression;
|
||||
this.baseOffset = baseOffset;
|
||||
this.logAppendTime = logAppendTime;
|
||||
this.numRecords = 0;
|
||||
|
|
@ -135,11 +136,11 @@ public class MemoryRecordsBuilder implements AutoCloseable {
|
|||
this.partitionLeaderEpoch = partitionLeaderEpoch;
|
||||
this.writeLimit = writeLimit;
|
||||
this.initialPosition = bufferStream.position();
|
||||
this.batchHeaderSizeInBytes = AbstractRecords.recordBatchHeaderSizeInBytes(magic, compressionType);
|
||||
this.batchHeaderSizeInBytes = AbstractRecords.recordBatchHeaderSizeInBytes(magic, compression.type());
|
||||
|
||||
bufferStream.position(initialPosition + batchHeaderSizeInBytes);
|
||||
this.bufferStream = bufferStream;
|
||||
this.appendStream = new DataOutputStream(compressionType.wrapForOutput(this.bufferStream, magic));
|
||||
this.appendStream = new DataOutputStream(compression.wrapForOutput(this.bufferStream, magic));
|
||||
|
||||
if (hasDeleteHorizonMs()) {
|
||||
this.baseTimestamp = deleteHorizonMs;
|
||||
|
|
@ -148,7 +149,7 @@ public class MemoryRecordsBuilder implements AutoCloseable {
|
|||
|
||||
public MemoryRecordsBuilder(ByteBufferOutputStream bufferStream,
|
||||
byte magic,
|
||||
CompressionType compressionType,
|
||||
Compression compression,
|
||||
TimestampType timestampType,
|
||||
long baseOffset,
|
||||
long logAppendTime,
|
||||
|
|
@ -159,7 +160,7 @@ public class MemoryRecordsBuilder implements AutoCloseable {
|
|||
boolean isControlBatch,
|
||||
int partitionLeaderEpoch,
|
||||
int writeLimit) {
|
||||
this(bufferStream, magic, compressionType, timestampType, baseOffset, logAppendTime, producerId,
|
||||
this(bufferStream, magic, compression, timestampType, baseOffset, logAppendTime, producerId,
|
||||
producerEpoch, baseSequence, isTransactional, isControlBatch, partitionLeaderEpoch, writeLimit,
|
||||
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
|
||||
* to fit the records appended)
|
||||
* @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 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.
|
||||
|
|
@ -186,7 +187,7 @@ public class MemoryRecordsBuilder implements AutoCloseable {
|
|||
*/
|
||||
public MemoryRecordsBuilder(ByteBuffer buffer,
|
||||
byte magic,
|
||||
CompressionType compressionType,
|
||||
Compression compression,
|
||||
TimestampType timestampType,
|
||||
long baseOffset,
|
||||
long logAppendTime,
|
||||
|
|
@ -197,7 +198,7 @@ public class MemoryRecordsBuilder implements AutoCloseable {
|
|||
boolean isControlBatch,
|
||||
int partitionLeaderEpoch,
|
||||
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,
|
||||
writeLimit);
|
||||
}
|
||||
|
|
@ -214,8 +215,8 @@ public class MemoryRecordsBuilder implements AutoCloseable {
|
|||
return actualCompressionRatio;
|
||||
}
|
||||
|
||||
public CompressionType compressionType() {
|
||||
return compressionType;
|
||||
public Compression compression() {
|
||||
return compression;
|
||||
}
|
||||
|
||||
public boolean isControlBatch() {
|
||||
|
|
@ -269,7 +270,7 @@ public class MemoryRecordsBuilder implements AutoCloseable {
|
|||
*/
|
||||
public RecordsInfo info() {
|
||||
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
|
||||
// shallowOffsetOfMaxTimestamp => case 2
|
||||
return new RecordsInfo(logAppendTime, lastOffset);
|
||||
|
|
@ -282,7 +283,7 @@ public class MemoryRecordsBuilder implements AutoCloseable {
|
|||
// shallowOffsetOfMaxTimestamp => case 1
|
||||
return new RecordsInfo(RecordBatch.NO_TIMESTAMP, -1);
|
||||
} else {
|
||||
if (compressionType != CompressionType.NONE || magic >= RecordBatch.MAGIC_VALUE_V2)
|
||||
if (compression.type() != CompressionType.NONE || magic >= RecordBatch.MAGIC_VALUE_V2)
|
||||
// maxTimestamp => case 3
|
||||
// shallowOffsetOfMaxTimestamp => case 4
|
||||
return new RecordsInfo(maxTimestamp, lastOffset);
|
||||
|
|
@ -374,7 +375,7 @@ public class MemoryRecordsBuilder implements AutoCloseable {
|
|||
} else {
|
||||
if (magic > RecordBatch.MAGIC_VALUE_V1)
|
||||
this.actualCompressionRatio = (float) writeDefaultBatchHeader() / this.uncompressedRecordsSizeInBytes;
|
||||
else if (compressionType != CompressionType.NONE)
|
||||
else if (compression.type() != CompressionType.NONE)
|
||||
this.actualCompressionRatio = (float) writeLegacyCompressedWrapperHeader() / this.uncompressedRecordsSizeInBytes;
|
||||
|
||||
ByteBuffer buffer = buffer().duplicate();
|
||||
|
|
@ -419,7 +420,7 @@ public class MemoryRecordsBuilder implements AutoCloseable {
|
|||
else
|
||||
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,
|
||||
hasDeleteHorizonMs(), partitionLeaderEpoch, numRecords);
|
||||
|
||||
|
|
@ -442,7 +443,7 @@ public class MemoryRecordsBuilder implements AutoCloseable {
|
|||
AbstractLegacyRecordBatch.writeHeader(buffer, lastOffset, wrapperSize);
|
||||
|
||||
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);
|
||||
return writtenCompressed;
|
||||
|
|
@ -778,7 +779,7 @@ public class MemoryRecordsBuilder implements AutoCloseable {
|
|||
|
||||
private long toInnerOffset(long offset) {
|
||||
// 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;
|
||||
}
|
||||
|
|
@ -817,7 +818,7 @@ public class MemoryRecordsBuilder implements AutoCloseable {
|
|||
* @return The estimated number of bytes written
|
||||
*/
|
||||
private int estimatedBytesWritten() {
|
||||
if (compressionType == CompressionType.NONE) {
|
||||
if (compression.type() == CompressionType.NONE) {
|
||||
return batchHeaderSizeInBytes + uncompressedRecordsSizeInBytes;
|
||||
} else {
|
||||
// estimate the written bytes to the underlying byte buffer based on uncompressed written bytes
|
||||
|
|
|
|||
|
|
@ -16,6 +16,7 @@
|
|||
*/
|
||||
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.utils.Time;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
|
|
@ -110,7 +111,7 @@ public class RecordsUtil {
|
|||
final TimestampType timestampType = batch.timestampType();
|
||||
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);
|
||||
for (Record record : recordBatchAndRecords.records) {
|
||||
// Down-convert this record. Ignore headers when down-converting to V0 and V1 since they are not supported
|
||||
|
|
|
|||
|
|
@ -20,6 +20,7 @@ import io.opentelemetry.proto.metrics.v1.MetricsData;
|
|||
|
||||
import org.apache.kafka.common.KafkaException;
|
||||
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.protocol.Errors;
|
||||
import org.apache.kafka.common.record.CompressionType;
|
||||
|
|
@ -190,7 +191,8 @@ public class ClientTelemetryUtils {
|
|||
|
||||
public static byte[] compress(byte[] raw, CompressionType compressionType) throws IOException {
|
||||
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.flush();
|
||||
}
|
||||
|
|
@ -201,7 +203,8 @@ public class ClientTelemetryUtils {
|
|||
|
||||
public static ByteBuffer decompress(byte[] metrics, CompressionType compressionType) {
|
||||
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)) {
|
||||
byte[] bytes = new byte[data.capacity() * 2];
|
||||
int nRead;
|
||||
|
|
|
|||
|
|
@ -34,6 +34,7 @@ import org.apache.kafka.common.Node;
|
|||
import org.apache.kafka.common.TopicIdPartition;
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
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.errors.AuthenticationException;
|
||||
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.protocol.ApiKeys;
|
||||
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.MemoryRecordsBuilder;
|
||||
import org.apache.kafka.common.record.TimestampType;
|
||||
|
|
@ -2760,7 +2760,7 @@ public class KafkaConsumerTest {
|
|||
if (fetchCount == 0) {
|
||||
records = MemoryRecords.EMPTY;
|
||||
} 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)) {
|
||||
for (int i = 0; i < fetchCount; i++)
|
||||
builder.append(0L, ("key-" + i).getBytes(), ("value-" + i).getBytes());
|
||||
|
|
|
|||
|
|
@ -21,11 +21,11 @@ import org.apache.kafka.clients.consumer.ConsumerRecord;
|
|||
import org.apache.kafka.clients.consumer.OffsetResetStrategy;
|
||||
import org.apache.kafka.common.IsolationLevel;
|
||||
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.message.FetchResponseData;
|
||||
import org.apache.kafka.common.metrics.Metrics;
|
||||
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.EndTransactionMarker;
|
||||
import org.apache.kafka.common.record.MemoryRecords;
|
||||
|
|
@ -157,7 +157,7 @@ public class CompletedFetchTest {
|
|||
@Test
|
||||
public void testCorruptedMessage() {
|
||||
// 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()) {
|
||||
builder.append(new SimpleRecord(serializer.serialize(TOPIC_NAME, UUID.randomUUID())));
|
||||
builder.append(0L, "key".getBytes(), "value".getBytes());
|
||||
|
|
@ -223,7 +223,7 @@ public class CompletedFetchTest {
|
|||
}
|
||||
|
||||
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++)
|
||||
builder.append(0L, "key".getBytes(), ("value-" + (firstMessageId + i)).getBytes());
|
||||
return builder.build();
|
||||
|
|
@ -236,7 +236,7 @@ public class CompletedFetchTest {
|
|||
|
||||
try (MemoryRecordsBuilder builder = MemoryRecords.builder(buffer,
|
||||
RecordBatch.CURRENT_MAGIC_VALUE,
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
TimestampType.CREATE_TIME,
|
||||
0,
|
||||
time.milliseconds(),
|
||||
|
|
|
|||
|
|
@ -19,13 +19,13 @@ package org.apache.kafka.clients.consumer.internals;
|
|||
import org.apache.kafka.clients.consumer.ConsumerConfig;
|
||||
import org.apache.kafka.common.KafkaException;
|
||||
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.internals.ClusterResourceListeners;
|
||||
import org.apache.kafka.common.message.FetchResponseData;
|
||||
import org.apache.kafka.common.metrics.Metrics;
|
||||
import org.apache.kafka.common.protocol.ApiKeys;
|
||||
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.MemoryRecordsBuilder;
|
||||
import org.apache.kafka.common.record.Records;
|
||||
|
|
@ -833,7 +833,7 @@ public class FetchCollectorTest {
|
|||
ByteBuffer allocate = ByteBuffer.allocate(1024);
|
||||
|
||||
try (MemoryRecordsBuilder builder = MemoryRecords.builder(allocate,
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
TimestampType.CREATE_TIME,
|
||||
0)) {
|
||||
for (int i = 0; i < recordCount; i++)
|
||||
|
|
|
|||
|
|
@ -37,6 +37,7 @@ import org.apache.kafka.common.Node;
|
|||
import org.apache.kafka.common.TopicIdPartition;
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
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.RecordTooLargeException;
|
||||
import org.apache.kafka.common.errors.SerializationException;
|
||||
|
|
@ -731,7 +732,7 @@ public class FetchRequestManagerTest {
|
|||
|
||||
ByteBuffer buffer = ByteBuffer.allocate(1024);
|
||||
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);
|
||||
builder.append(0L, "key".getBytes(), "1".getBytes());
|
||||
builder.append(0L, "key".getBytes(), "2".getBytes());
|
||||
|
|
@ -764,7 +765,7 @@ public class FetchRequestManagerTest {
|
|||
|
||||
ByteBuffer buffer = ByteBuffer.allocate(1024);
|
||||
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);
|
||||
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;
|
||||
|
||||
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);
|
||||
builder.append(0L, "key".getBytes(), Integer.toString(partitionLeaderEpoch).getBytes());
|
||||
builder.close();
|
||||
|
||||
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);
|
||||
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;
|
||||
|
||||
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);
|
||||
builder.append(0L, "key".getBytes(), null);
|
||||
builder.close();
|
||||
|
|
@ -1046,7 +1047,7 @@ public class FetchRequestManagerTest {
|
|||
|
||||
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(out,
|
||||
DefaultRecordBatch.CURRENT_MAGIC_VALUE,
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
TimestampType.CREATE_TIME,
|
||||
0L, 10L, 0L, (short) 0, 0, false, false, 0, 1024);
|
||||
builder.append(10L, "key".getBytes(), "value".getBytes());
|
||||
|
|
@ -1077,7 +1078,7 @@ public class FetchRequestManagerTest {
|
|||
public void testParseInvalidRecordBatch() {
|
||||
buildFetcher();
|
||||
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(2L, "b".getBytes(), "2".getBytes()),
|
||||
new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
|
||||
|
|
@ -1103,7 +1104,7 @@ public class FetchRequestManagerTest {
|
|||
public void testHeaders() {
|
||||
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());
|
||||
|
||||
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
|
||||
buildFetcher();
|
||||
|
||||
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE,
|
||||
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), Compression.NONE,
|
||||
TimestampType.CREATE_TIME, 0L);
|
||||
builder.appendWithOffset(15L, 0L, "key".getBytes(), "value-1".getBytes());
|
||||
builder.appendWithOffset(20L, 0L, "key".getBytes(), "value-2".getBytes());
|
||||
|
|
@ -1975,7 +1976,7 @@ public class FetchRequestManagerTest {
|
|||
assertEquals(100, (Double) partitionLag.metricValue(), EPSILON);
|
||||
|
||||
// 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);
|
||||
for (int v = 0; v < 3; v++)
|
||||
builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes());
|
||||
|
|
@ -2016,7 +2017,7 @@ public class FetchRequestManagerTest {
|
|||
assertEquals(0L, (Double) partitionLead.metricValue(), EPSILON);
|
||||
|
||||
// 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);
|
||||
for (int v = 0; v < 3; v++) {
|
||||
builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes());
|
||||
|
|
@ -2060,7 +2061,7 @@ public class FetchRequestManagerTest {
|
|||
assertEquals(50, (Double) partitionLag.metricValue(), EPSILON);
|
||||
|
||||
// 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);
|
||||
for (int v = 0; v < 3; v++)
|
||||
builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes());
|
||||
|
|
@ -2100,7 +2101,7 @@ public class FetchRequestManagerTest {
|
|||
for (TopicIdPartition tp : mkSet(tidp1, tidp2)) {
|
||||
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);
|
||||
for (int v = 0; v < 3; v++)
|
||||
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 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);
|
||||
for (int v = 0; v < 3; v++)
|
||||
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 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);
|
||||
for (int v = 0; v < 3; v++)
|
||||
builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes());
|
||||
|
|
@ -2216,7 +2217,7 @@ public class FetchRequestManagerTest {
|
|||
assertEquals(1, sendFetches());
|
||||
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);
|
||||
for (int v = 0; v < 3; v++)
|
||||
builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes());
|
||||
|
|
@ -2232,7 +2233,7 @@ public class FetchRequestManagerTest {
|
|||
.setPartitionIndex(tp1.partition())
|
||||
.setHighWatermark(100)
|
||||
.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)));
|
||||
networkClientDelegate.poll(time.timer(0));
|
||||
|
|
@ -2534,7 +2535,7 @@ public class FetchRequestManagerTest {
|
|||
public void testUpdatePositionWithLastRecordMissingFromBatch() {
|
||||
buildFetcher();
|
||||
|
||||
MemoryRecords records = MemoryRecords.withRecords(CompressionType.NONE,
|
||||
MemoryRecords records = MemoryRecords.withRecords(Compression.NONE,
|
||||
new SimpleRecord("0".getBytes(), "v".getBytes()),
|
||||
new SimpleRecord("1".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) {
|
||||
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++)
|
||||
builder.append(0L, "key".getBytes(), ("value-" + (firstMessageId + i)).getBytes());
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
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,
|
||||
RecordBatch.NO_PARTITION_LEADER_EPOCH);
|
||||
|
||||
|
|
@ -2904,7 +2905,7 @@ public class FetchRequestManagerTest {
|
|||
MemoryRecordsBuilder builder = MemoryRecords.builder(
|
||||
ByteBuffer.allocate(1024),
|
||||
RecordBatch.CURRENT_MAGIC_VALUE,
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
TimestampType.CREATE_TIME,
|
||||
0L,
|
||||
RecordBatch.NO_TIMESTAMP,
|
||||
|
|
|
|||
|
|
@ -36,6 +36,7 @@ import org.apache.kafka.common.Node;
|
|||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.TopicIdPartition;
|
||||
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.SerializationException;
|
||||
import org.apache.kafka.common.errors.TopicAuthorizationException;
|
||||
|
|
@ -680,7 +681,7 @@ public class FetcherTest {
|
|||
|
||||
ByteBuffer buffer = ByteBuffer.allocate(1024);
|
||||
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);
|
||||
builder.append(0L, "key".getBytes(), "1".getBytes());
|
||||
builder.append(0L, "key".getBytes(), "2".getBytes());
|
||||
|
|
@ -713,7 +714,7 @@ public class FetcherTest {
|
|||
|
||||
ByteBuffer buffer = ByteBuffer.allocate(1024);
|
||||
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);
|
||||
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;
|
||||
|
||||
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);
|
||||
builder.append(0L, "key".getBytes(), Integer.toString(partitionLeaderEpoch).getBytes());
|
||||
builder.close();
|
||||
|
||||
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);
|
||||
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;
|
||||
|
||||
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);
|
||||
builder.append(0L, "key".getBytes(), null);
|
||||
builder.close();
|
||||
|
|
@ -995,7 +996,7 @@ public class FetcherTest {
|
|||
|
||||
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(out,
|
||||
DefaultRecordBatch.CURRENT_MAGIC_VALUE,
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
TimestampType.CREATE_TIME,
|
||||
0L, 10L, 0L, (short) 0, 0, false, false, 0, 1024);
|
||||
builder.append(10L, "key".getBytes(), "value".getBytes());
|
||||
|
|
@ -1026,7 +1027,7 @@ public class FetcherTest {
|
|||
public void testParseInvalidRecordBatch() {
|
||||
buildFetcher();
|
||||
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(2L, "b".getBytes(), "2".getBytes()),
|
||||
new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
|
||||
|
|
@ -1052,7 +1053,7 @@ public class FetcherTest {
|
|||
public void testHeaders() {
|
||||
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());
|
||||
|
||||
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
|
||||
buildFetcher();
|
||||
|
||||
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE,
|
||||
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), Compression.NONE,
|
||||
TimestampType.CREATE_TIME, 0L);
|
||||
builder.appendWithOffset(15L, 0L, "key".getBytes(), "value-1".getBytes());
|
||||
builder.appendWithOffset(20L, 0L, "key".getBytes(), "value-2".getBytes());
|
||||
|
|
@ -1971,7 +1972,7 @@ public class FetcherTest {
|
|||
assertEquals(100, (Double) partitionLag.metricValue(), EPSILON);
|
||||
|
||||
// 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);
|
||||
for (int v = 0; v < 3; v++)
|
||||
builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes());
|
||||
|
|
@ -2012,7 +2013,7 @@ public class FetcherTest {
|
|||
assertEquals(0L, (Double) partitionLead.metricValue(), EPSILON);
|
||||
|
||||
// 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);
|
||||
for (int v = 0; v < 3; v++) {
|
||||
builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes());
|
||||
|
|
@ -2056,7 +2057,7 @@ public class FetcherTest {
|
|||
assertEquals(50, (Double) partitionLag.metricValue(), EPSILON);
|
||||
|
||||
// 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);
|
||||
for (int v = 0; v < 3; v++)
|
||||
builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes());
|
||||
|
|
@ -2096,7 +2097,7 @@ public class FetcherTest {
|
|||
for (TopicIdPartition tp : mkSet(tidp1, tidp2)) {
|
||||
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);
|
||||
for (int v = 0; v < 3; v++)
|
||||
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 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);
|
||||
for (int v = 0; v < 3; v++)
|
||||
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 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);
|
||||
for (int v = 0; v < 3; v++)
|
||||
builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes());
|
||||
|
|
@ -2212,7 +2213,7 @@ public class FetcherTest {
|
|||
assertEquals(1, sendFetches());
|
||||
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);
|
||||
for (int v = 0; v < 3; v++)
|
||||
builder.appendWithOffset(v, RecordBatch.NO_TIMESTAMP, "key".getBytes(), ("value-" + v).getBytes());
|
||||
|
|
@ -2228,7 +2229,7 @@ public class FetcherTest {
|
|||
.setPartitionIndex(tp1.partition())
|
||||
.setHighWatermark(100)
|
||||
.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)));
|
||||
consumerClient.poll(time.timer(0));
|
||||
|
|
@ -2530,7 +2531,7 @@ public class FetcherTest {
|
|||
public void testUpdatePositionWithLastRecordMissingFromBatch() {
|
||||
buildFetcher();
|
||||
|
||||
MemoryRecords records = MemoryRecords.withRecords(CompressionType.NONE,
|
||||
MemoryRecords records = MemoryRecords.withRecords(Compression.NONE,
|
||||
new SimpleRecord("0".getBytes(), "v".getBytes()),
|
||||
new SimpleRecord("1".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) {
|
||||
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++)
|
||||
builder.append(0L, "key".getBytes(), ("value-" + (firstMessageId + i)).getBytes());
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
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,
|
||||
RecordBatch.NO_PARTITION_LEADER_EPOCH);
|
||||
|
||||
|
|
@ -3108,7 +3109,7 @@ public class FetcherTest {
|
|||
MemoryRecordsBuilder builder = MemoryRecords.builder(
|
||||
ByteBuffer.allocate(1024),
|
||||
RecordBatch.CURRENT_MAGIC_VALUE,
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
TimestampType.CREATE_TIME,
|
||||
0L,
|
||||
RecordBatch.NO_TIMESTAMP,
|
||||
|
|
@ -3156,7 +3157,7 @@ public class FetcherTest {
|
|||
MemoryRecordsBuilder builder = MemoryRecords.builder(
|
||||
ByteBuffer.allocate(1024),
|
||||
RecordBatch.CURRENT_MAGIC_VALUE,
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
TimestampType.CREATE_TIME,
|
||||
0L,
|
||||
RecordBatch.NO_TIMESTAMP,
|
||||
|
|
|
|||
|
|
@ -21,6 +21,7 @@ import org.apache.kafka.clients.producer.Callback;
|
|||
import org.apache.kafka.clients.producer.RecordMetadata;
|
||||
import org.apache.kafka.common.KafkaException;
|
||||
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.internals.RecordHeader;
|
||||
import org.apache.kafka.common.record.CompressionType;
|
||||
|
|
@ -58,7 +59,7 @@ public class ProducerBatchTest {
|
|||
private final long now = 1488748346917L;
|
||||
|
||||
private final MemoryRecordsBuilder memoryRecordsBuilder = MemoryRecords.builder(ByteBuffer.allocate(512),
|
||||
CompressionType.NONE, TimestampType.CREATE_TIME, 128);
|
||||
Compression.NONE, TimestampType.CREATE_TIME, 128);
|
||||
|
||||
@Test
|
||||
public void testBatchAbort() throws Exception {
|
||||
|
|
@ -136,7 +137,7 @@ public class ProducerBatchTest {
|
|||
MemoryRecordsBuilder builder = MemoryRecords.builder(
|
||||
ByteBuffer.allocate(1024),
|
||||
MAGIC_VALUE_V2,
|
||||
compressionType,
|
||||
Compression.of(compressionType).build(),
|
||||
TimestampType.CREATE_TIME,
|
||||
0L);
|
||||
ProducerBatch batch = new ProducerBatch(new TopicPartition("topic", 1), builder, now);
|
||||
|
|
@ -176,7 +177,7 @@ public class ProducerBatchTest {
|
|||
continue;
|
||||
|
||||
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);
|
||||
while (true) {
|
||||
|
|
|
|||
|
|
@ -31,6 +31,7 @@ import org.apache.kafka.common.KafkaException;
|
|||
import org.apache.kafka.common.Node;
|
||||
import org.apache.kafka.common.PartitionInfo;
|
||||
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.metrics.Metrics;
|
||||
import org.apache.kafka.common.protocol.ApiKeys;
|
||||
|
|
@ -157,7 +158,7 @@ public class RecordAccumulatorTest {
|
|||
Collections.emptyMap(),
|
||||
cluster);
|
||||
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
|
||||
|
|
@ -225,7 +226,7 @@ public class RecordAccumulatorTest {
|
|||
int batchSize = 1025;
|
||||
|
||||
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);
|
||||
for (int i = 0; i < appends; i++) {
|
||||
// append to the first batch
|
||||
|
|
@ -262,19 +263,19 @@ public class RecordAccumulatorTest {
|
|||
|
||||
@Test
|
||||
public void testAppendLargeCompressed() throws Exception {
|
||||
testAppendLarge(CompressionType.GZIP);
|
||||
testAppendLarge(Compression.gzip().build());
|
||||
}
|
||||
|
||||
@Test
|
||||
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;
|
||||
byte[] value = new byte[2 * batchSize];
|
||||
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());
|
||||
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
|
||||
public void testAppendLargeOldMessageFormatCompressed() throws Exception {
|
||||
testAppendLargeOldMessageFormat(CompressionType.GZIP);
|
||||
testAppendLargeOldMessageFormat(Compression.gzip().build());
|
||||
}
|
||||
|
||||
@Test
|
||||
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;
|
||||
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));
|
||||
|
||||
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());
|
||||
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 {
|
||||
int lingerMs = 10;
|
||||
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);
|
||||
assertEquals(0, accum.ready(metadataCache, time.milliseconds()).readyNodes.size(), "No partitions should be ready");
|
||||
time.sleep(10);
|
||||
|
|
@ -355,7 +356,7 @@ public class RecordAccumulatorTest {
|
|||
@Test
|
||||
public void testPartialDrain() throws Exception {
|
||||
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;
|
||||
List<TopicPartition> partitions = asList(tp1, tp2);
|
||||
for (TopicPartition tp : partitions) {
|
||||
|
|
@ -375,7 +376,7 @@ public class RecordAccumulatorTest {
|
|||
final int msgs = 10000;
|
||||
final int numParts = 2;
|
||||
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<>();
|
||||
for (int i = 0; i < numThreads; i++) {
|
||||
threads.add(new Thread(() -> {
|
||||
|
|
@ -418,7 +419,7 @@ public class RecordAccumulatorTest {
|
|||
int batchSize = 1025;
|
||||
|
||||
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
|
||||
int appends = expectedNumAppends(batchSize);
|
||||
|
||||
|
|
@ -459,7 +460,7 @@ public class RecordAccumulatorTest {
|
|||
String metricGrpName = "producer-metrics";
|
||||
|
||||
final RecordAccumulator accum = new RecordAccumulator(logContext, batchSize,
|
||||
CompressionType.NONE, lingerMs, retryBackoffMs, retryBackoffMaxMs,
|
||||
Compression.NONE, lingerMs, retryBackoffMs, retryBackoffMaxMs,
|
||||
deliveryTimeoutMs, metrics, metricGrpName, time, new ApiVersions(), null,
|
||||
new BufferPool(totalSize, batchSize, metrics, time, metricGrpName));
|
||||
|
||||
|
|
@ -524,7 +525,7 @@ public class RecordAccumulatorTest {
|
|||
String metricGrpName = "producer-metrics";
|
||||
|
||||
final RecordAccumulator accum = new RecordAccumulator(logContext, batchSize,
|
||||
CompressionType.NONE, lingerMs, retryBackoffMs, retryBackoffMaxMs,
|
||||
Compression.NONE, lingerMs, retryBackoffMs, retryBackoffMaxMs,
|
||||
deliveryTimeoutMs, metrics, metricGrpName, time, new ApiVersions(), null,
|
||||
new BufferPool(totalSize, batchSize, metrics, time, metricGrpName));
|
||||
|
||||
|
|
@ -585,7 +586,7 @@ public class RecordAccumulatorTest {
|
|||
Collections.emptyMap());
|
||||
|
||||
final RecordAccumulator accum = new RecordAccumulator(logContext, batchSize,
|
||||
CompressionType.NONE, lingerMs, retryBackoffMs, retryBackoffMaxMs,
|
||||
Compression.NONE, lingerMs, retryBackoffMs, retryBackoffMaxMs,
|
||||
deliveryTimeoutMs, metrics, metricGrpName, time, new ApiVersions(), null,
|
||||
new BufferPool(totalSize, batchSize, metrics, time, metricGrpName));
|
||||
|
||||
|
|
@ -645,7 +646,7 @@ public class RecordAccumulatorTest {
|
|||
public void testFlush() throws Exception {
|
||||
int lingerMs = Integer.MAX_VALUE;
|
||||
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++) {
|
||||
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
|
||||
public void testAwaitFlushComplete() throws Exception {
|
||||
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.beginFlush();
|
||||
|
|
@ -704,7 +705,7 @@ public class RecordAccumulatorTest {
|
|||
|
||||
final AtomicInteger numExceptionReceivedInCallback = new AtomicInteger(0);
|
||||
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 {
|
||||
@Override
|
||||
public void onCompletion(RecordMetadata metadata, Exception exception) {
|
||||
|
|
@ -747,7 +748,7 @@ public class RecordAccumulatorTest {
|
|||
|
||||
final AtomicInteger numExceptionReceivedInCallback = new AtomicInteger(0);
|
||||
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();
|
||||
|
||||
class TestCallback implements RecordAccumulator.AppendCallbacks {
|
||||
|
|
@ -795,7 +796,7 @@ public class RecordAccumulatorTest {
|
|||
// test case assumes that the records do not fill the batch completely
|
||||
int batchSize = 1025;
|
||||
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
|
||||
for (Boolean mute: muteStates) {
|
||||
|
|
@ -845,7 +846,7 @@ public class RecordAccumulatorTest {
|
|||
int batchSize = 1025;
|
||||
|
||||
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);
|
||||
|
||||
// Test batches not in retry
|
||||
|
|
@ -938,7 +939,7 @@ public class RecordAccumulatorTest {
|
|||
int batchSize = 1025;
|
||||
|
||||
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);
|
||||
for (int i = 0; i < appends; i++) {
|
||||
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));
|
||||
TransactionManager transactionManager = new TransactionManager(new LogContext(), null, 0, retryBackoffMs, apiVersions);
|
||||
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));
|
||||
assertThrows(UnsupportedVersionException.class,
|
||||
() -> 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);
|
||||
RecordAccumulator accumulator = createTestRecordAccumulator(transactionManager, deliveryTimeoutMs,
|
||||
batchSize, totalSize, CompressionType.NONE, lingerMs);
|
||||
batchSize, totalSize, Compression.NONE, lingerMs);
|
||||
|
||||
ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(12345L, (short) 5);
|
||||
Mockito.when(transactionManager.producerIdAndEpoch()).thenReturn(producerIdAndEpoch);
|
||||
|
|
@ -1036,11 +1037,11 @@ public class RecordAccumulatorTest {
|
|||
@Test
|
||||
public void testSplitAndReenqueue() throws ExecutionException, InterruptedException {
|
||||
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
|
||||
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);
|
||||
|
||||
byte[] value = new byte[1024];
|
||||
|
|
@ -1091,7 +1092,7 @@ public class RecordAccumulatorTest {
|
|||
|
||||
// First set the compression ratio estimation to be good.
|
||||
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);
|
||||
assertTrue(numSplitBatches > 0, "There should be some split batches");
|
||||
// Drain all the split batches.
|
||||
|
|
@ -1115,7 +1116,7 @@ public class RecordAccumulatorTest {
|
|||
final int batchSize = 1024;
|
||||
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
|
||||
for (int goodCompRatioPercentage = 1; goodCompRatioPercentage < 100; goodCompRatioPercentage++) {
|
||||
int numSplit = 0;
|
||||
|
|
@ -1146,7 +1147,7 @@ public class RecordAccumulatorTest {
|
|||
int batchSize = 1025;
|
||||
|
||||
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);
|
||||
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
|
||||
int batchSize = 1025;
|
||||
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.
|
||||
for (Boolean mute : muteStates) {
|
||||
|
|
@ -1220,7 +1221,7 @@ public class RecordAccumulatorTest {
|
|||
|
||||
Partitioner partitioner = new DefaultPartitioner();
|
||||
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);
|
||||
|
||||
// Create first batch
|
||||
|
|
@ -1293,7 +1294,7 @@ public class RecordAccumulatorTest {
|
|||
|
||||
long totalSize = 1024 * 1024;
|
||||
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.
|
||||
final AtomicInteger partition = new AtomicInteger(RecordMetadata.UNKNOWN_PARTITION);
|
||||
|
|
@ -1366,7 +1367,7 @@ public class RecordAccumulatorTest {
|
|||
RecordAccumulator.PartitionerConfig config = new RecordAccumulator.PartitionerConfig(true, 100);
|
||||
long totalSize = 1024 * 1024;
|
||||
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,
|
||||
new BufferPool(totalSize, batchSize, metrics, time, "producer-internal-metrics"));
|
||||
|
||||
|
|
@ -1451,7 +1452,7 @@ public class RecordAccumulatorTest {
|
|||
long totalSize = 1024 * 1024;
|
||||
int batchSize = 512; // note that this is also a "sticky" limit for the partitioner
|
||||
int valSize = 32;
|
||||
RecordAccumulator accum = createTestRecordAccumulator(batchSize, totalSize, CompressionType.NONE, 10);
|
||||
RecordAccumulator accum = createTestRecordAccumulator(batchSize, totalSize, Compression.NONE, 10);
|
||||
byte[] value = new byte[valSize];
|
||||
|
||||
for (int c = 10; c-- > 0; ) {
|
||||
|
|
@ -1495,7 +1496,7 @@ public class RecordAccumulatorTest {
|
|||
long totalSize = 10 * 1024;
|
||||
String metricGrpName = "producer-metrics";
|
||||
final RecordAccumulator accum = new RecordAccumulator(logContext, batchSize,
|
||||
CompressionType.NONE, lingerMs, retryBackoffMs, retryBackoffMaxMs,
|
||||
Compression.NONE, lingerMs, retryBackoffMs, retryBackoffMaxMs,
|
||||
deliveryTimeoutMs, metrics, metricGrpName, time, new ApiVersions(), null,
|
||||
new BufferPool(totalSize, batchSize, metrics, time, metricGrpName));
|
||||
|
||||
|
|
@ -1600,8 +1601,7 @@ public class RecordAccumulatorTest {
|
|||
int batchSize = 10;
|
||||
int lingerMs = 10;
|
||||
long totalSize = 10 * 1024;
|
||||
RecordAccumulator accum = createTestRecordAccumulator(batchSize, totalSize,
|
||||
CompressionType.NONE, lingerMs);
|
||||
RecordAccumulator accum = createTestRecordAccumulator(batchSize, totalSize, Compression.NONE, lingerMs);
|
||||
|
||||
// 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);
|
||||
|
|
@ -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;
|
||||
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) {
|
||||
return createTestRecordAccumulator(null, deliveryTimeoutMs, batchSize, totalSize, type, lingerMs);
|
||||
private RecordAccumulator createTestRecordAccumulator(int deliveryTimeoutMs, int batchSize, long totalSize, Compression compression, int lingerMs) {
|
||||
return createTestRecordAccumulator(null, deliveryTimeoutMs, batchSize, totalSize, compression, lingerMs);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -1741,7 +1741,7 @@ public class RecordAccumulatorTest {
|
|||
int deliveryTimeoutMs,
|
||||
int batchSize,
|
||||
long totalSize,
|
||||
CompressionType type,
|
||||
Compression compression,
|
||||
int lingerMs
|
||||
) {
|
||||
long retryBackoffMs = 100L;
|
||||
|
|
@ -1751,7 +1751,7 @@ public class RecordAccumulatorTest {
|
|||
return new RecordAccumulator(
|
||||
logContext,
|
||||
batchSize,
|
||||
type,
|
||||
compression,
|
||||
lingerMs,
|
||||
retryBackoffMs,
|
||||
retryBackoffMaxMs,
|
||||
|
|
|
|||
|
|
@ -32,6 +32,7 @@ import org.apache.kafka.common.MetricName;
|
|||
import org.apache.kafka.common.MetricNameTemplate;
|
||||
import org.apache.kafka.common.Node;
|
||||
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.InvalidRequestException;
|
||||
import org.apache.kafka.common.errors.InvalidTxnStateException;
|
||||
|
|
@ -569,7 +570,7 @@ public class SenderTest {
|
|||
// otherwise it wouldn't update the stats.
|
||||
RecordAccumulator.PartitionerConfig config = new RecordAccumulator.PartitionerConfig(false, 42);
|
||||
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,
|
||||
new BufferPool(totalSize, batchSize, m, time, "producer-internal-metrics"));
|
||||
|
||||
|
|
@ -2417,7 +2418,7 @@ public class SenderTest {
|
|||
// Set a good compression ratio.
|
||||
CompressionRatioEstimator.setEstimation(topic, CompressionType.GZIP, 0.2f);
|
||||
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,
|
||||
new BufferPool(totalSize, batchSize, metrics, time, "producer-internal-metrics"));
|
||||
SenderMetricsRegistry senderMetrics = new SenderMetricsRegistry(m);
|
||||
|
|
@ -3200,7 +3201,7 @@ public class SenderTest {
|
|||
long retryBackoffMaxMs = 100L;
|
||||
// lingerMs is 0 to send batch as soon as any records are available on it.
|
||||
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);
|
||||
Sender sender = new Sender(logContext, client, metadata, this.accumulator, false,
|
||||
MAX_REQUEST_SIZE, ACKS_ALL,
|
||||
|
|
@ -3314,7 +3315,7 @@ public class SenderTest {
|
|||
long retryBackoffMaxMs = 100L;
|
||||
// lingerMs is 0 to send batch as soon as any records are available on it.
|
||||
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);
|
||||
Sender sender = new Sender(logContext, client, metadata, this.accumulator, false,
|
||||
MAX_REQUEST_SIZE, ACKS_ALL,
|
||||
|
|
@ -3394,7 +3395,7 @@ public class SenderTest {
|
|||
long retryBackoffMaxMs = 100L;
|
||||
// lingerMs is 0 to send batch as soon as any records are available on it.
|
||||
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);
|
||||
Sender sender = new Sender(logContext, client, metadata, this.accumulator, false,
|
||||
MAX_REQUEST_SIZE, ACKS_ALL,
|
||||
|
|
@ -3696,7 +3697,7 @@ public class SenderTest {
|
|||
this.metrics = new Metrics(metricConfig, time);
|
||||
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);
|
||||
this.senderMetricsRegistry = new SenderMetricsRegistry(this.metrics);
|
||||
this.sender = new Sender(logContext, this.client, this.metadata, this.accumulator, guaranteeOrder, MAX_REQUEST_SIZE, ACKS_ALL,
|
||||
|
|
|
|||
|
|
@ -27,6 +27,7 @@ import org.apache.kafka.clients.producer.RecordMetadata;
|
|||
import org.apache.kafka.common.KafkaException;
|
||||
import org.apache.kafka.common.Node;
|
||||
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.GroupAuthorizationException;
|
||||
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.protocol.ApiKeys;
|
||||
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.MemoryRecordsBuilder;
|
||||
import org.apache.kafka.common.record.MutableRecordBatch;
|
||||
|
|
@ -178,7 +178,7 @@ public class TransactionManagerTest {
|
|||
String metricGrpName = "producer-metrics";
|
||||
|
||||
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,
|
||||
new BufferPool(totalSize, batchSize, metrics, time, metricGrpName));
|
||||
|
||||
|
|
@ -690,7 +690,7 @@ public class TransactionManagerTest {
|
|||
final int requestTimeout = 10000;
|
||||
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,
|
||||
new BufferPool(1024 * 1024, 16 * 1024, metrics, time, ""));
|
||||
|
||||
|
|
@ -760,7 +760,7 @@ public class TransactionManagerTest {
|
|||
|
||||
private ProducerBatch batchWithValue(TopicPartition tp, String value) {
|
||||
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(64),
|
||||
CompressionType.NONE, TimestampType.CREATE_TIME, 0L);
|
||||
Compression.NONE, TimestampType.CREATE_TIME, 0L);
|
||||
long currentTimeMs = time.milliseconds();
|
||||
ProducerBatch batch = new ProducerBatch(tp, builder, currentTimeMs);
|
||||
batch.tryAppend(currentTimeMs, new byte[0], value.getBytes(), new Header[0], null, currentTimeMs);
|
||||
|
|
|
|||
|
|
@ -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));
|
||||
}
|
||||
}
|
||||
|
|
@ -18,7 +18,11 @@ package org.apache.kafka.common.compress;
|
|||
|
||||
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.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.params.ParameterizedTest;
|
||||
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.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.ByteOrder;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
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.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class KafkaLZ4Test {
|
||||
public class Lz4CompressionTest {
|
||||
|
||||
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 {
|
||||
String name;
|
||||
byte[] payload;
|
||||
|
|
@ -67,14 +140,16 @@ public class KafkaLZ4Test {
|
|||
private static class Args {
|
||||
final boolean useBrokenFlagDescriptorChecksum;
|
||||
final boolean ignoreFlagDescriptorChecksum;
|
||||
final int level;
|
||||
final byte[] payload;
|
||||
final boolean close;
|
||||
final boolean blockChecksum;
|
||||
|
||||
Args(boolean useBrokenFlagDescriptorChecksum, boolean ignoreFlagDescriptorChecksum,
|
||||
boolean blockChecksum, boolean close, Payload payload) {
|
||||
int level, boolean blockChecksum, boolean close, Payload payload) {
|
||||
this.useBrokenFlagDescriptorChecksum = useBrokenFlagDescriptorChecksum;
|
||||
this.ignoreFlagDescriptorChecksum = ignoreFlagDescriptorChecksum;
|
||||
this.level = level;
|
||||
this.blockChecksum = blockChecksum;
|
||||
this.close = close;
|
||||
this.payload = payload.payload;
|
||||
|
|
@ -84,6 +159,7 @@ public class KafkaLZ4Test {
|
|||
public String toString() {
|
||||
return "useBrokenFlagDescriptorChecksum=" + useBrokenFlagDescriptorChecksum +
|
||||
", ignoreFlagDescriptorChecksum=" + ignoreFlagDescriptorChecksum +
|
||||
", level=" + level +
|
||||
", blockChecksum=" + blockChecksum +
|
||||
", close=" + close +
|
||||
", payload=" + Arrays.toString(payload);
|
||||
|
|
@ -115,7 +191,8 @@ public class KafkaLZ4Test {
|
|||
for (boolean ignore : Arrays.asList(false, true))
|
||||
for (boolean blockChecksum : 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();
|
||||
}
|
||||
|
|
@ -126,11 +203,11 @@ public class KafkaLZ4Test {
|
|||
public void testHeaderPrematureEnd(Args args) {
|
||||
ByteBuffer buffer = ByteBuffer.allocate(2);
|
||||
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 {
|
||||
return new KafkaLZ4BlockInputStream(buffer, BufferSupplier.create(), ignoreFlagDescriptorChecksum);
|
||||
private Lz4BlockInputStream makeInputStream(ByteBuffer buffer, boolean ignoreFlagDescriptorChecksum) throws IOException {
|
||||
return new Lz4BlockInputStream(buffer, BufferSupplier.create(), ignoreFlagDescriptorChecksum);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
|
|
@ -140,7 +217,7 @@ public class KafkaLZ4Test {
|
|||
compressed[0] = 0x00;
|
||||
ByteBuffer buffer = ByteBuffer.wrap(compressed);
|
||||
IOException e = assertThrows(IOException.class, () -> makeInputStream(buffer, args.ignoreFlagDescriptorChecksum));
|
||||
assertEquals(KafkaLZ4BlockInputStream.NOT_SUPPORTED, e.getMessage());
|
||||
assertEquals(Lz4BlockInputStream.NOT_SUPPORTED, e.getMessage());
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
|
|
@ -154,7 +231,7 @@ public class KafkaLZ4Test {
|
|||
makeInputStream(buffer, args.ignoreFlagDescriptorChecksum);
|
||||
} else {
|
||||
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 {
|
||||
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);
|
||||
|
||||
int n = 100;
|
||||
|
|
@ -319,7 +396,7 @@ public class KafkaLZ4Test {
|
|||
private void testDecompression(ByteBuffer buffer, Args args) throws IOException {
|
||||
IOException error = null;
|
||||
try {
|
||||
KafkaLZ4BlockInputStream decompressed = makeInputStream(buffer, args.ignoreFlagDescriptorChecksum);
|
||||
Lz4BlockInputStream decompressed = makeInputStream(buffer, args.ignoreFlagDescriptorChecksum);
|
||||
|
||||
byte[] testPayload = new byte[args.payload.length];
|
||||
|
||||
|
|
@ -341,10 +418,10 @@ public class KafkaLZ4Test {
|
|||
assertArrayEquals(args.payload, testPayload);
|
||||
} catch (IOException e) {
|
||||
if (!args.ignoreFlagDescriptorChecksum && args.useBrokenFlagDescriptorChecksum) {
|
||||
assertEquals(KafkaLZ4BlockInputStream.DESCRIPTOR_HASH_MISMATCH, e.getMessage());
|
||||
assertEquals(Lz4BlockInputStream.DESCRIPTOR_HASH_MISMATCH, e.getMessage());
|
||||
error = e;
|
||||
} else if (!args.close) {
|
||||
assertEquals(KafkaLZ4BlockInputStream.PREMATURE_EOS, e.getMessage());
|
||||
assertEquals(Lz4BlockInputStream.PREMATURE_EOS, e.getMessage());
|
||||
error = e;
|
||||
} else {
|
||||
throw e;
|
||||
|
|
@ -356,9 +433,10 @@ public class KafkaLZ4Test {
|
|||
|
||||
private byte[] compressedBytes(Args args) throws IOException {
|
||||
ByteArrayOutputStream output = new ByteArrayOutputStream();
|
||||
KafkaLZ4BlockOutputStream lz4 = new KafkaLZ4BlockOutputStream(
|
||||
Lz4BlockOutputStream lz4 = new Lz4BlockOutputStream(
|
||||
output,
|
||||
KafkaLZ4BlockOutputStream.BLOCKSIZE_64KB,
|
||||
Lz4BlockOutputStream.BLOCKSIZE_64KB,
|
||||
args.level,
|
||||
args.blockChecksum,
|
||||
args.useBrokenFlagDescriptorChecksum
|
||||
);
|
||||
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
@ -16,9 +16,9 @@
|
|||
*/
|
||||
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.MessageUtil;
|
||||
import org.apache.kafka.common.record.CompressionType;
|
||||
import org.apache.kafka.common.record.MemoryRecords;
|
||||
import org.apache.kafka.common.record.SimpleRecord;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
|
@ -32,7 +32,7 @@ public class RecordsSerdeTest {
|
|||
|
||||
@Test
|
||||
public void testSerdeRecords() {
|
||||
MemoryRecords records = MemoryRecords.withRecords(CompressionType.NONE,
|
||||
MemoryRecords records = MemoryRecords.withRecords(Compression.NONE,
|
||||
new SimpleRecord("foo".getBytes()),
|
||||
new SimpleRecord("bar".getBytes()));
|
||||
|
||||
|
|
|
|||
|
|
@ -16,8 +16,8 @@
|
|||
*/
|
||||
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.record.CompressionType;
|
||||
import org.apache.kafka.common.record.MemoryRecords;
|
||||
import org.apache.kafka.common.record.MemoryRecordsBuilder;
|
||||
import org.apache.kafka.common.record.SimpleRecord;
|
||||
|
|
@ -155,7 +155,7 @@ public class SendBuilderTest {
|
|||
private MemoryRecords createRecords(ByteBuffer buffer, String value) {
|
||||
MemoryRecordsBuilder recordsBuilder = MemoryRecords.builder(
|
||||
buffer,
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
TimestampType.CREATE_TIME,
|
||||
0L
|
||||
);
|
||||
|
|
|
|||
|
|
@ -16,7 +16,7 @@
|
|||
*/
|
||||
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.SimpleRecord;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
|
@ -50,7 +50,7 @@ public class TypeTest {
|
|||
|
||||
@Test
|
||||
public void testRecordsSerde() {
|
||||
MemoryRecords records = MemoryRecords.withRecords(CompressionType.NONE,
|
||||
MemoryRecords records = MemoryRecords.withRecords(Compression.NONE,
|
||||
new SimpleRecord("foo".getBytes()),
|
||||
new SimpleRecord("bar".getBytes()));
|
||||
ByteBuffer buffer = ByteBuffer.allocate(Type.RECORDS.sizeOf(records));
|
||||
|
|
@ -81,7 +81,7 @@ public class TypeTest {
|
|||
|
||||
@Test
|
||||
public void testCompactRecordsSerde() {
|
||||
MemoryRecords records = MemoryRecords.withRecords(CompressionType.NONE,
|
||||
MemoryRecords records = MemoryRecords.withRecords(Compression.NONE,
|
||||
new SimpleRecord("foo".getBytes()),
|
||||
new SimpleRecord("bar".getBytes()));
|
||||
ByteBuffer buffer = ByteBuffer.allocate(Type.COMPACT_RECORDS.sizeOf(records));
|
||||
|
|
|
|||
|
|
@ -17,6 +17,7 @@
|
|||
package org.apache.kafka.common.record;
|
||||
|
||||
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.utils.Utils;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
|
@ -40,7 +41,7 @@ public class AbstractLegacyRecordBatchTest {
|
|||
};
|
||||
|
||||
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 firstOffset = lastOffset - simpleRecords.length + 1;
|
||||
|
|
@ -75,7 +76,7 @@ public class AbstractLegacyRecordBatchTest {
|
|||
};
|
||||
|
||||
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());
|
||||
batch.setLastOffset(0L);
|
||||
|
|
@ -95,7 +96,7 @@ public class AbstractLegacyRecordBatchTest {
|
|||
};
|
||||
|
||||
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());
|
||||
batch.setLastOffset(1L);
|
||||
|
|
@ -106,7 +107,7 @@ public class AbstractLegacyRecordBatchTest {
|
|||
@Test
|
||||
public void testSetNoTimestampTypeNotAllowed() {
|
||||
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(2L, "b".getBytes(), "2".getBytes()),
|
||||
new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
|
||||
|
|
@ -117,7 +118,7 @@ public class AbstractLegacyRecordBatchTest {
|
|||
@Test
|
||||
public void testSetLogAppendTimeNotAllowedV0() {
|
||||
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(2L, "b".getBytes(), "2".getBytes()),
|
||||
new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
|
||||
|
|
@ -129,7 +130,7 @@ public class AbstractLegacyRecordBatchTest {
|
|||
@Test
|
||||
public void testSetCreateTimeNotAllowedV0() {
|
||||
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(2L, "b".getBytes(), "2".getBytes()),
|
||||
new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
|
||||
|
|
@ -141,7 +142,7 @@ public class AbstractLegacyRecordBatchTest {
|
|||
@Test
|
||||
public void testSetPartitionLeaderEpochNotAllowedV0() {
|
||||
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(2L, "b".getBytes(), "2".getBytes()),
|
||||
new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
|
||||
|
|
@ -152,7 +153,7 @@ public class AbstractLegacyRecordBatchTest {
|
|||
@Test
|
||||
public void testSetPartitionLeaderEpochNotAllowedV1() {
|
||||
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(2L, "b".getBytes(), "2".getBytes()),
|
||||
new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
|
||||
|
|
@ -163,7 +164,7 @@ public class AbstractLegacyRecordBatchTest {
|
|||
@Test
|
||||
public void testSetLogAppendTimeV1() {
|
||||
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(2L, "b".getBytes(), "2".getBytes()),
|
||||
new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
|
||||
|
|
@ -188,7 +189,7 @@ public class AbstractLegacyRecordBatchTest {
|
|||
@Test
|
||||
public void testSetCreateTimeV1() {
|
||||
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(2L, "b".getBytes(), "2".getBytes()),
|
||||
new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
|
||||
|
|
@ -222,7 +223,7 @@ public class AbstractLegacyRecordBatchTest {
|
|||
// Check V0
|
||||
try {
|
||||
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());
|
||||
batch.setLastOffset(1L);
|
||||
|
|
@ -236,7 +237,7 @@ public class AbstractLegacyRecordBatchTest {
|
|||
// Check V1
|
||||
try {
|
||||
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());
|
||||
batch.setLastOffset(1L);
|
||||
|
|
|
|||
|
|
@ -16,6 +16,7 @@
|
|||
*/
|
||||
package org.apache.kafka.common.record;
|
||||
|
||||
import org.apache.kafka.common.compress.Compression;
|
||||
import org.apache.kafka.common.errors.CorruptRecordException;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
|
|
@ -33,12 +34,12 @@ public class ByteBufferLogInputStreamTest {
|
|||
@Test
|
||||
public void iteratorIgnoresIncompleteEntries() {
|
||||
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(20L, "b".getBytes(), "2".getBytes());
|
||||
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(40L, "d".getBytes(), "4".getBytes());
|
||||
builder.close();
|
||||
|
|
@ -58,14 +59,14 @@ public class ByteBufferLogInputStreamTest {
|
|||
@Test
|
||||
public void iteratorRaisesOnTooSmallRecords() {
|
||||
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(20L, "b".getBytes(), "2".getBytes());
|
||||
builder.close();
|
||||
|
||||
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(40L, "d".getBytes(), "4".getBytes());
|
||||
builder.close();
|
||||
|
|
@ -81,14 +82,14 @@ public class ByteBufferLogInputStreamTest {
|
|||
@Test
|
||||
public void iteratorRaisesOnInvalidMagic() {
|
||||
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(20L, "b".getBytes(), "2".getBytes());
|
||||
builder.close();
|
||||
|
||||
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(40L, "d".getBytes(), "4".getBytes());
|
||||
builder.close();
|
||||
|
|
@ -104,11 +105,11 @@ public class ByteBufferLogInputStreamTest {
|
|||
@Test
|
||||
public void iteratorRaisesOnTooLargeRecords() {
|
||||
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.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(40L, "d".getBytes(), "4".getBytes());
|
||||
builder.close();
|
||||
|
|
|
|||
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
|
@ -17,7 +17,8 @@
|
|||
package org.apache.kafka.common.record;
|
||||
|
||||
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.header.Header;
|
||||
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.assertTrue;
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.ArgumentMatchers.anyByte;
|
||||
import static org.mockito.ArgumentMatchers.anyInt;
|
||||
import static org.mockito.ArgumentMatchers.anyLong;
|
||||
import static org.mockito.Mockito.doReturn;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.never;
|
||||
import static org.mockito.Mockito.spy;
|
||||
import static org.mockito.Mockito.times;
|
||||
import static org.mockito.Mockito.verify;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class DefaultRecordBatchTest {
|
||||
// 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() {
|
||||
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);
|
||||
builder.appendWithOffset(1234567, 1L, "a".getBytes(), "v".getBytes());
|
||||
builder.appendWithOffset(1234568, 2L, "b".getBytes(), "v".getBytes());
|
||||
|
|
@ -134,7 +132,7 @@ public class DefaultRecordBatchTest {
|
|||
|
||||
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);
|
||||
builder.appendWithOffset(1234567, 1L, "a".getBytes(), "v".getBytes());
|
||||
builder.appendWithOffset(1234568, 2L, "b".getBytes(), "v".getBytes());
|
||||
|
|
@ -161,7 +159,7 @@ public class DefaultRecordBatchTest {
|
|||
int baseSequence = Integer.MAX_VALUE - 1;
|
||||
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);
|
||||
builder.appendWithOffset(1234567, 1L, "a".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(), "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)));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInvalidRecordSize() {
|
||||
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(2L, "b".getBytes(), "2".getBytes()),
|
||||
new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
|
||||
|
|
@ -256,7 +254,7 @@ public class DefaultRecordBatchTest {
|
|||
@Test
|
||||
public void testInvalidCrc() {
|
||||
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(2L, "b".getBytes(), "2".getBytes()),
|
||||
new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
|
||||
|
|
@ -277,7 +275,7 @@ public class DefaultRecordBatchTest {
|
|||
new SimpleRecord(3L, "c".getBytes(), "3".getBytes())
|
||||
};
|
||||
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 firstOffset = lastOffset - simpleRecords.length + 1;
|
||||
|
|
@ -300,7 +298,7 @@ public class DefaultRecordBatchTest {
|
|||
@Test
|
||||
public void testSetPartitionLeaderEpoch() {
|
||||
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(2L, "b".getBytes(), "2".getBytes()),
|
||||
new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
|
||||
|
|
@ -320,7 +318,7 @@ public class DefaultRecordBatchTest {
|
|||
@Test
|
||||
public void testSetLogAppendTime() {
|
||||
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(2L, "b".getBytes(), "2".getBytes()),
|
||||
new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
|
||||
|
|
@ -345,7 +343,7 @@ public class DefaultRecordBatchTest {
|
|||
@Test
|
||||
public void testSetNoTimestampTypeNotAllowed() {
|
||||
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(2L, "b".getBytes(), "2".getBytes()),
|
||||
new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
|
||||
|
|
@ -361,7 +359,7 @@ public class DefaultRecordBatchTest {
|
|||
|
||||
ByteBuffer buffer = ByteBuffer.allocate(128);
|
||||
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,
|
||||
buffer.remaining());
|
||||
|
||||
|
|
@ -385,8 +383,9 @@ public class DefaultRecordBatchTest {
|
|||
@ParameterizedTest
|
||||
@EnumSource(value = CompressionType.class)
|
||||
public void testStreamingIteratorConsistency(CompressionType compressionType) {
|
||||
Compression compression = Compression.of(compressionType).build();
|
||||
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(2L, "b".getBytes(), "2".getBytes()),
|
||||
new SimpleRecord(3L, "c".getBytes(), "3".getBytes()));
|
||||
|
|
@ -399,12 +398,13 @@ public class DefaultRecordBatchTest {
|
|||
@ParameterizedTest
|
||||
@EnumSource(value = CompressionType.class)
|
||||
public void testSkipKeyValueIteratorCorrectness(CompressionType compressionType) {
|
||||
Compression compression = Compression.of(compressionType).build();
|
||||
Header[] headers = {new RecordHeader("k1", "v1".getBytes()), new RecordHeader("k2", null)};
|
||||
byte[] largeRecordValue = new byte[200 * 1024]; // 200KB
|
||||
RANDOM.nextBytes(largeRecordValue);
|
||||
|
||||
MemoryRecords records = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, 0L,
|
||||
compressionType, TimestampType.CREATE_TIME,
|
||||
compression, TimestampType.CREATE_TIME,
|
||||
// one sample with small value size
|
||||
new SimpleRecord(1L, "a".getBytes(), "1".getBytes()),
|
||||
// one sample with null value
|
||||
|
|
@ -448,8 +448,9 @@ public class DefaultRecordBatchTest {
|
|||
@ParameterizedTest
|
||||
@MethodSource
|
||||
public void testBufferReuseInSkipKeyValueIterator(CompressionType compressionType, int expectedNumBufferAllocations, byte[] recordValue) {
|
||||
Compression compression = Compression.of(compressionType).build();
|
||||
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(9999L, "b".getBytes(), recordValue)
|
||||
);
|
||||
|
|
@ -500,7 +501,7 @@ public class DefaultRecordBatchTest {
|
|||
@MethodSource
|
||||
public void testZstdJniForSkipKeyValueIterator(int expectedJniCalls, byte[] recordValue) throws IOException {
|
||||
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)
|
||||
);
|
||||
|
||||
|
|
@ -508,19 +509,15 @@ public class DefaultRecordBatchTest {
|
|||
final ByteBuffer compressedBuf = records.buffer();
|
||||
// Create a RecordBatch object
|
||||
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
|
||||
ByteBuffer recordsBuffer = compressedBuf.duplicate();
|
||||
recordsBuffer.position(RECORDS_OFFSET);
|
||||
|
||||
try (final BufferSupplier bufferSupplier = BufferSupplier.create();
|
||||
final InputStream zstdStream = spy(ZstdFactory.wrapForInput(recordsBuffer, batch.magic(), bufferSupplier));
|
||||
final InputStream chunkedStream = new ChunkedBytesStream(zstdStream, bufferSupplier, 16 * 1024, false)) {
|
||||
|
||||
when(mockCompression.wrapForInput(any(ByteBuffer.class), anyByte(), any(BufferSupplier.class))).thenReturn(chunkedStream);
|
||||
|
||||
final InputStream zstdStream = spy(ZstdCompression.wrapForZstdInput(recordsBuffer, bufferSupplier));
|
||||
final InputStream chunkedStream = new ChunkedBytesStream(zstdStream, bufferSupplier, 16 * 1024, false)
|
||||
) {
|
||||
doReturn(chunkedStream).when(batch).recordInputStream(any());
|
||||
try (CloseableIterator<Record> streamingIterator = batch.skipKeyValueIterator(bufferSupplier)) {
|
||||
assertNotNull(streamingIterator);
|
||||
Utils.toList(streamingIterator);
|
||||
|
|
@ -571,7 +568,7 @@ public class DefaultRecordBatchTest {
|
|||
private static DefaultRecordBatch recordsWithInvalidRecordCount(Byte magicValue, long timestamp,
|
||||
CompressionType codec, int invalidCount) {
|
||||
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(1, timestamp, null, "there".getBytes());
|
||||
builder.appendWithOffset(2, timestamp, null, "beautiful".getBytes());
|
||||
|
|
|
|||
|
|
@ -16,6 +16,7 @@
|
|||
*/
|
||||
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.utils.Utils;
|
||||
import org.apache.kafka.test.TestUtils;
|
||||
|
|
@ -50,9 +51,9 @@ public class FileLogInputStreamTest {
|
|||
|
||||
private static class Args {
|
||||
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.compression = compression;
|
||||
}
|
||||
|
|
@ -71,7 +72,7 @@ public class FileLogInputStreamTest {
|
|||
List<Arguments> arguments = new ArrayList<>();
|
||||
for (byte magic : asList(MAGIC_VALUE_V0, MAGIC_VALUE_V1, MAGIC_VALUE_V2))
|
||||
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();
|
||||
}
|
||||
}
|
||||
|
|
@ -79,9 +80,9 @@ public class FileLogInputStreamTest {
|
|||
@ParameterizedTest
|
||||
@ArgumentsSource(FileLogInputStreamArgumentsProvider.class)
|
||||
public void testWriteTo(Args args) throws IOException {
|
||||
CompressionType compression = args.compression;
|
||||
Compression compression = args.compression;
|
||||
byte magic = args.magic;
|
||||
if (compression == CompressionType.ZSTD && magic < MAGIC_VALUE_V2)
|
||||
if (compression.type() == CompressionType.ZSTD && magic < MAGIC_VALUE_V2)
|
||||
return;
|
||||
|
||||
try (FileRecords fileRecords = FileRecords.open(tempFile())) {
|
||||
|
|
@ -110,9 +111,9 @@ public class FileLogInputStreamTest {
|
|||
@ParameterizedTest
|
||||
@ArgumentsSource(FileLogInputStreamArgumentsProvider.class)
|
||||
public void testSimpleBatchIteration(Args args) throws IOException {
|
||||
CompressionType compression = args.compression;
|
||||
Compression compression = args.compression;
|
||||
byte magic = args.magic;
|
||||
if (compression == CompressionType.ZSTD && magic < MAGIC_VALUE_V2)
|
||||
if (compression.type() == CompressionType.ZSTD && magic < MAGIC_VALUE_V2)
|
||||
return;
|
||||
|
||||
try (FileRecords fileRecords = FileRecords.open(tempFile())) {
|
||||
|
|
@ -140,12 +141,12 @@ public class FileLogInputStreamTest {
|
|||
@ParameterizedTest
|
||||
@ArgumentsSource(FileLogInputStreamArgumentsProvider.class)
|
||||
public void testBatchIterationWithMultipleRecordsPerBatch(Args args) throws IOException {
|
||||
CompressionType compression = args.compression;
|
||||
Compression compression = args.compression;
|
||||
byte magic = args.magic;
|
||||
if (magic < MAGIC_VALUE_V2 && compression == CompressionType.NONE)
|
||||
if (magic < MAGIC_VALUE_V2 && compression.type() == CompressionType.NONE)
|
||||
return;
|
||||
|
||||
if (compression == CompressionType.ZSTD && magic < MAGIC_VALUE_V2)
|
||||
if (compression.type() == CompressionType.ZSTD && magic < MAGIC_VALUE_V2)
|
||||
return;
|
||||
|
||||
try (FileRecords fileRecords = FileRecords.open(tempFile())) {
|
||||
|
|
@ -181,7 +182,7 @@ public class FileLogInputStreamTest {
|
|||
@ParameterizedTest
|
||||
@ArgumentsSource(FileLogInputStreamArgumentsProvider.class)
|
||||
public void testBatchIterationV2(Args args) throws IOException {
|
||||
CompressionType compression = args.compression;
|
||||
Compression compression = args.compression;
|
||||
byte magic = args.magic;
|
||||
if (magic != MAGIC_VALUE_V2)
|
||||
return;
|
||||
|
|
@ -229,9 +230,9 @@ public class FileLogInputStreamTest {
|
|||
@ParameterizedTest
|
||||
@ArgumentsSource(FileLogInputStreamArgumentsProvider.class)
|
||||
public void testBatchIterationIncompleteBatch(Args args) throws IOException {
|
||||
CompressionType compression = args.compression;
|
||||
Compression compression = args.compression;
|
||||
byte magic = args.magic;
|
||||
if (compression == CompressionType.ZSTD && magic < MAGIC_VALUE_V2)
|
||||
if (compression.type() == CompressionType.ZSTD && magic < MAGIC_VALUE_V2)
|
||||
return;
|
||||
|
||||
try (FileRecords fileRecords = FileRecords.open(tempFile())) {
|
||||
|
|
@ -288,10 +289,10 @@ public class FileLogInputStreamTest {
|
|||
|
||||
private void assertGenericRecordBatchData(Args args, RecordBatch batch, long baseOffset, long maxTimestamp,
|
||||
SimpleRecord... records) {
|
||||
CompressionType compression = args.compression;
|
||||
Compression compression = args.compression;
|
||||
byte magic = args.magic;
|
||||
assertEquals(magic, batch.magic());
|
||||
assertEquals(compression, batch.compressionType());
|
||||
assertEquals(compression.type(), batch.compressionType());
|
||||
|
||||
if (magic == MAGIC_VALUE_V0) {
|
||||
assertEquals(NO_TIMESTAMP_TYPE, batch.timestampType());
|
||||
|
|
|
|||
|
|
@ -18,6 +18,8 @@ package org.apache.kafka.common.record;
|
|||
|
||||
import org.apache.kafka.common.KafkaException;
|
||||
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.internals.RecordHeader;
|
||||
import org.apache.kafka.common.network.TransferableChannel;
|
||||
|
|
@ -116,7 +118,7 @@ public class FileRecordsTest {
|
|||
public void testFileSize() throws IOException {
|
||||
assertEquals(fileRecords.channel().size(), fileRecords.sizeInBytes());
|
||||
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());
|
||||
}
|
||||
}
|
||||
|
|
@ -247,7 +249,7 @@ public class FileRecordsTest {
|
|||
public void testSearch() throws IOException {
|
||||
// append a new message with a high offset
|
||||
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);
|
||||
int position = 0;
|
||||
|
|
@ -484,7 +486,7 @@ public class FileRecordsTest {
|
|||
int leaderEpoch) throws IOException {
|
||||
ByteBuffer buffer = ByteBuffer.allocate(128);
|
||||
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]));
|
||||
fileRecords.append(builder.build());
|
||||
}
|
||||
|
|
@ -497,7 +499,7 @@ public class FileRecordsTest {
|
|||
random.nextBytes(bytes);
|
||||
|
||||
// records
|
||||
CompressionType compressionType = CompressionType.GZIP;
|
||||
GzipCompression compression = Compression.gzip().build();
|
||||
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<SimpleRecord> records = asList(
|
||||
|
|
@ -508,7 +510,7 @@ public class FileRecordsTest {
|
|||
// create MemoryRecords
|
||||
ByteBuffer buffer = ByteBuffer.allocate(8000);
|
||||
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.close();
|
||||
}
|
||||
|
|
@ -518,18 +520,18 @@ public class FileRecordsTest {
|
|||
try (FileRecords fileRecords = FileRecords.open(tempFile())) {
|
||||
fileRecords.append(MemoryRecords.readableRecords(buffer));
|
||||
fileRecords.flush();
|
||||
downConvertAndVerifyRecords(records, offsets, fileRecords, compressionType, toMagic, 0L, time);
|
||||
downConvertAndVerifyRecords(records, offsets, fileRecords, compression, toMagic, 0L, time);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConversion() throws IOException {
|
||||
doTestConversion(CompressionType.NONE, RecordBatch.MAGIC_VALUE_V0);
|
||||
doTestConversion(CompressionType.GZIP, RecordBatch.MAGIC_VALUE_V0);
|
||||
doTestConversion(CompressionType.NONE, RecordBatch.MAGIC_VALUE_V1);
|
||||
doTestConversion(CompressionType.GZIP, RecordBatch.MAGIC_VALUE_V1);
|
||||
doTestConversion(CompressionType.NONE, RecordBatch.MAGIC_VALUE_V2);
|
||||
doTestConversion(CompressionType.GZIP, RecordBatch.MAGIC_VALUE_V2);
|
||||
doTestConversion(Compression.NONE, RecordBatch.MAGIC_VALUE_V0);
|
||||
doTestConversion(Compression.gzip().build(), RecordBatch.MAGIC_VALUE_V0);
|
||||
doTestConversion(Compression.NONE, RecordBatch.MAGIC_VALUE_V1);
|
||||
doTestConversion(Compression.gzip().build(), RecordBatch.MAGIC_VALUE_V1);
|
||||
doTestConversion(Compression.NONE, RecordBatch.MAGIC_VALUE_V2);
|
||||
doTestConversion(Compression.gzip().build(), RecordBatch.MAGIC_VALUE_V2);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
@ -551,7 +553,7 @@ public class FileRecordsTest {
|
|||
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);
|
||||
|
||||
Header[] headers = {new RecordHeader("headerKey1", "headerValue1".getBytes()),
|
||||
|
|
@ -572,19 +574,19 @@ public class FileRecordsTest {
|
|||
assertEquals(offsets.size(), records.size(), "incorrect test setup");
|
||||
|
||||
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);
|
||||
for (int i = 0; i < 3; i++)
|
||||
builder.appendWithOffset(offsets.get(i), records.get(i));
|
||||
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);
|
||||
for (int i = 3; i < 6; i++)
|
||||
builder.appendWithOffset(offsets.get(i), records.get(i));
|
||||
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++)
|
||||
builder.appendWithOffset(offsets.get(i), records.get(i));
|
||||
builder.close();
|
||||
|
|
@ -594,9 +596,9 @@ public class FileRecordsTest {
|
|||
try (FileRecords fileRecords = FileRecords.open(tempFile())) {
|
||||
fileRecords.append(MemoryRecords.readableRecords(buffer));
|
||||
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;
|
||||
if (toMagic == RecordBatch.MAGIC_VALUE_V0)
|
||||
firstOffset = 11L; // v1 record
|
||||
|
|
@ -607,10 +609,10 @@ public class FileRecordsTest {
|
|||
int index = filteredOffsets.indexOf(firstOffset) - 1;
|
||||
filteredRecords.remove(index);
|
||||
filteredOffsets.remove(index);
|
||||
downConvertAndVerifyRecords(filteredRecords, filteredOffsets, fileRecords, compressionType, toMagic, firstOffset, time);
|
||||
downConvertAndVerifyRecords(filteredRecords, filteredOffsets, fileRecords, compression, toMagic, firstOffset, time);
|
||||
} else {
|
||||
// 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,
|
||||
List<Long> initialOffsets,
|
||||
FileRecords fileRecords,
|
||||
CompressionType compressionType,
|
||||
Compression compression,
|
||||
byte toMagic,
|
||||
long firstOffset,
|
||||
Time time) {
|
||||
|
|
@ -632,7 +634,7 @@ public class FileRecordsTest {
|
|||
// Test the normal down-conversion path
|
||||
List<Records> convertedRecords = new ArrayList<>();
|
||||
convertedRecords.add(fileRecords.downConvert(toMagic, firstOffset, time).records());
|
||||
verifyConvertedRecords(initialRecords, initialOffsets, convertedRecords, compressionType, toMagic);
|
||||
verifyConvertedRecords(initialRecords, initialOffsets, convertedRecords, compression, toMagic);
|
||||
convertedRecords.clear();
|
||||
|
||||
// Test the lazy down-conversion path
|
||||
|
|
@ -648,7 +650,7 @@ public class FileRecordsTest {
|
|||
Iterator<ConvertedRecords<?>> it = lazyRecords.iterator(readSize);
|
||||
while (it.hasNext())
|
||||
convertedRecords.add(it.next().records());
|
||||
verifyConvertedRecords(initialRecords, initialOffsets, convertedRecords, compressionType, toMagic);
|
||||
verifyConvertedRecords(initialRecords, initialOffsets, convertedRecords, compression, toMagic);
|
||||
convertedRecords.clear();
|
||||
}
|
||||
}
|
||||
|
|
@ -656,7 +658,7 @@ public class FileRecordsTest {
|
|||
private void verifyConvertedRecords(List<SimpleRecord> initialRecords,
|
||||
List<Long> initialOffsets,
|
||||
List<Records> convertedRecordsList,
|
||||
CompressionType compressionType,
|
||||
Compression compression,
|
||||
byte magicByte) {
|
||||
int i = 0;
|
||||
|
||||
|
|
@ -667,7 +669,7 @@ public class FileRecordsTest {
|
|||
assertEquals(TimestampType.NO_TIMESTAMP_TYPE, batch.timestampType());
|
||||
else
|
||||
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) {
|
||||
assertTrue(record.hasMagic(batch.magic()), "Inner record should have magic " + magicByte);
|
||||
assertEquals(initialOffsets.get(i).longValue(), record.offset(), "Offset should not change");
|
||||
|
|
@ -710,7 +712,7 @@ public class FileRecordsTest {
|
|||
for (byte[] value : values) {
|
||||
ByteBuffer buffer = ByteBuffer.allocate(128);
|
||||
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);
|
||||
fileRecords.append(builder.build());
|
||||
}
|
||||
|
|
|
|||
|
|
@ -17,6 +17,7 @@
|
|||
package org.apache.kafka.common.record;
|
||||
|
||||
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.internals.RecordHeader;
|
||||
import org.apache.kafka.common.network.TransferableChannel;
|
||||
|
|
@ -112,19 +113,20 @@ public class LazyDownConversionRecordsTest {
|
|||
assertEquals(offsets.size(), records.size(), "incorrect test setup");
|
||||
|
||||
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);
|
||||
for (int i = 0; i < 3; i++)
|
||||
builder.appendWithOffset(offsets.get(i), records.get(i));
|
||||
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);
|
||||
for (int i = 3; i < 6; i++)
|
||||
builder.appendWithOffset(offsets.get(i), records.get(i));
|
||||
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);
|
||||
for (int i = 6; i < 10; i++)
|
||||
builder.appendWithOffset(offsets.get(i), records.get(i));
|
||||
|
|
|
|||
|
|
@ -16,6 +16,7 @@
|
|||
*/
|
||||
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.message.LeaderChangeMessage;
|
||||
import org.apache.kafka.common.message.LeaderChangeMessage.Voter;
|
||||
|
|
@ -60,12 +61,12 @@ public class MemoryRecordsBuilderTest {
|
|||
|
||||
private static class Args {
|
||||
final int bufferOffset;
|
||||
final CompressionType compressionType;
|
||||
final Compression compression;
|
||||
final byte magic;
|
||||
|
||||
public Args(int bufferOffset, CompressionType compressionType, byte magic) {
|
||||
public Args(int bufferOffset, Compression compression, byte magic) {
|
||||
this.bufferOffset = bufferOffset;
|
||||
this.compressionType = compressionType;
|
||||
this.compression = compression;
|
||||
this.magic = magic;
|
||||
}
|
||||
|
||||
|
|
@ -73,7 +74,7 @@ public class MemoryRecordsBuilderTest {
|
|||
public String toString() {
|
||||
return "magic=" + magic +
|
||||
", bufferOffset=" + bufferOffset +
|
||||
", compressionType=" + compressionType;
|
||||
", compression=" + compression;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -87,7 +88,7 @@ public class MemoryRecordsBuilderTest {
|
|||
? Collections.singletonList(RecordBatch.MAGIC_VALUE_V2)
|
||||
: asList(RecordBatch.MAGIC_VALUE_V0, MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V2);
|
||||
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();
|
||||
}
|
||||
|
|
@ -97,13 +98,13 @@ public class MemoryRecordsBuilderTest {
|
|||
|
||||
@Test
|
||||
public void testUnsupportedCompress() {
|
||||
BiFunction<Byte, CompressionType, MemoryRecordsBuilder> builderBiFunction = (magic, compressionType) ->
|
||||
new MemoryRecordsBuilder(ByteBuffer.allocate(128), magic, compressionType, TimestampType.CREATE_TIME, 0L, 0L,
|
||||
BiFunction<Byte, Compression, MemoryRecordsBuilder> builderBiFunction = (magic, compression) ->
|
||||
new MemoryRecordsBuilder(ByteBuffer.allocate(128), magic, compression, TimestampType.CREATE_TIME, 0L, 0L,
|
||||
RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE,
|
||||
false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, 128);
|
||||
|
||||
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);
|
||||
});
|
||||
}
|
||||
|
|
@ -115,7 +116,7 @@ public class MemoryRecordsBuilderTest {
|
|||
ByteBuffer buffer = allocateBuffer(128, args);
|
||||
|
||||
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,
|
||||
false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity()).build();
|
||||
|
||||
|
|
@ -131,7 +132,7 @@ public class MemoryRecordsBuilderTest {
|
|||
short epoch = 15;
|
||||
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,
|
||||
RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
|
||||
|
||||
|
|
@ -156,7 +157,7 @@ public class MemoryRecordsBuilderTest {
|
|||
short epoch = 15;
|
||||
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());
|
||||
if (args.magic < MAGIC_VALUE_V2) {
|
||||
assertThrows(IllegalArgumentException.class, supplier::get);
|
||||
|
|
@ -174,7 +175,7 @@ public class MemoryRecordsBuilderTest {
|
|||
short epoch = RecordBatch.NO_PRODUCER_EPOCH;
|
||||
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());
|
||||
|
||||
if (args.magic < MAGIC_VALUE_V2) {
|
||||
|
|
@ -193,7 +194,7 @@ public class MemoryRecordsBuilderTest {
|
|||
short epoch = 15;
|
||||
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());
|
||||
|
||||
if (args.magic < MAGIC_VALUE_V2) {
|
||||
|
|
@ -212,7 +213,7 @@ public class MemoryRecordsBuilderTest {
|
|||
short epoch = 15;
|
||||
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,
|
||||
RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
|
||||
|
||||
|
|
@ -233,7 +234,7 @@ public class MemoryRecordsBuilderTest {
|
|||
short epoch = 15;
|
||||
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());
|
||||
|
||||
if (args.magic < MAGIC_VALUE_V2) {
|
||||
|
|
@ -249,7 +250,7 @@ public class MemoryRecordsBuilderTest {
|
|||
@ArgumentsSource(MemoryRecordsBuilderArgumentsProvider.class)
|
||||
public void testWriteLeaderChangeControlBatchWithoutLeaderEpoch(Args 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,
|
||||
RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE,
|
||||
false, true, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
|
||||
|
|
@ -272,7 +273,7 @@ public class MemoryRecordsBuilderTest {
|
|||
final int leaderEpoch = 5;
|
||||
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,
|
||||
RecordBatch.NO_SEQUENCE, false, true, leaderEpoch, buffer.capacity());
|
||||
|
||||
|
|
@ -313,7 +314,7 @@ public class MemoryRecordsBuilderTest {
|
|||
} else {
|
||||
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,
|
||||
false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
|
||||
|
||||
|
|
@ -324,7 +325,7 @@ public class MemoryRecordsBuilderTest {
|
|||
}
|
||||
|
||||
MemoryRecords built = builder.build();
|
||||
if (args.compressionType == CompressionType.NONE) {
|
||||
if (args.compression.type() == CompressionType.NONE) {
|
||||
assertEquals(1.0, builder.compressionRatio(), 0.00001);
|
||||
} else {
|
||||
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) {
|
||||
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,
|
||||
false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
|
||||
|
||||
|
|
@ -355,7 +356,7 @@ public class MemoryRecordsBuilderTest {
|
|||
int bytesWrittenBeforeClose = builder.estimatedSizeInBytes();
|
||||
MemoryRecords records = builder.build();
|
||||
assertEquals(records.sizeInBytes(), builder.estimatedSizeInBytes());
|
||||
if (args.compressionType == CompressionType.NONE)
|
||||
if (args.compression.type() == CompressionType.NONE)
|
||||
assertEquals(records.sizeInBytes(), bytesWrittenBeforeClose);
|
||||
}
|
||||
|
||||
|
|
@ -367,7 +368,7 @@ public class MemoryRecordsBuilderTest {
|
|||
ByteBuffer buffer = allocateBuffer(1024, args);
|
||||
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,
|
||||
RecordBatch.NO_SEQUENCE, false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
|
||||
builder.append(0L, "a".getBytes(), "1".getBytes());
|
||||
|
|
@ -378,7 +379,7 @@ public class MemoryRecordsBuilderTest {
|
|||
MemoryRecordsBuilder.RecordsInfo info = builder.info();
|
||||
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);
|
||||
else
|
||||
assertEquals(2L, info.shallowOffsetOfMaxTimestamp);
|
||||
|
|
@ -400,7 +401,7 @@ public class MemoryRecordsBuilderTest {
|
|||
ByteBuffer buffer = allocateBuffer(1024, args);
|
||||
|
||||
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,
|
||||
false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
|
||||
builder.append(0L, "a".getBytes(), "1".getBytes());
|
||||
|
|
@ -417,7 +418,7 @@ public class MemoryRecordsBuilderTest {
|
|||
|
||||
if (magic == MAGIC_VALUE_V0)
|
||||
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);
|
||||
else
|
||||
assertEquals(2L, info.shallowOffsetOfMaxTimestamp);
|
||||
|
|
@ -439,7 +440,7 @@ public class MemoryRecordsBuilderTest {
|
|||
@ArgumentsSource(MemoryRecordsBuilderArgumentsProvider.class)
|
||||
public void testAppendedChecksumConsistency(Args args) {
|
||||
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, RecordBatch.NO_SEQUENCE, false, false,
|
||||
RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
|
||||
|
|
@ -458,7 +459,7 @@ public class MemoryRecordsBuilderTest {
|
|||
byte[] value = "bar".getBytes();
|
||||
int writeLimit = 0;
|
||||
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,
|
||||
RecordBatch.NO_SEQUENCE, false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, writeLimit);
|
||||
|
||||
|
|
@ -485,7 +486,7 @@ public class MemoryRecordsBuilderTest {
|
|||
ByteBuffer buffer = allocateBuffer(64, args);
|
||||
|
||||
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,
|
||||
false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
|
||||
builder.setEstimatedCompressionRatio(0.5f);
|
||||
|
|
@ -523,7 +524,7 @@ public class MemoryRecordsBuilderTest {
|
|||
ByteBuffer buffer = allocateBuffer(1024, args);
|
||||
|
||||
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,
|
||||
false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
|
||||
|
||||
|
|
@ -538,8 +539,9 @@ public class MemoryRecordsBuilderTest {
|
|||
@EnumSource(CompressionType.class)
|
||||
public void convertV2ToV1UsingMixedCreateAndLogAppendTime(CompressionType compressionType) {
|
||||
ByteBuffer buffer = ByteBuffer.allocate(512);
|
||||
Compression compression = Compression.of(compressionType).build();
|
||||
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.close();
|
||||
|
||||
|
|
@ -550,7 +552,7 @@ public class MemoryRecordsBuilderTest {
|
|||
|
||||
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);
|
||||
builder.append(12L, "2".getBytes(), "b".getBytes());
|
||||
builder.append(13L, "3".getBytes(), "c".getBytes());
|
||||
|
|
@ -566,16 +568,16 @@ public class MemoryRecordsBuilderTest {
|
|||
Supplier<ConvertedRecords<MemoryRecords>> convertedRecordsSupplier = () ->
|
||||
MemoryRecords.readableRecords(buffer).downConvert(MAGIC_VALUE_V1, 0, time);
|
||||
|
||||
if (compressionType != CompressionType.ZSTD) {
|
||||
if (compression.type() != CompressionType.ZSTD) {
|
||||
ConvertedRecords<MemoryRecords> convertedRecords = convertedRecordsSupplier.get();
|
||||
MemoryRecords records = convertedRecords.records();
|
||||
|
||||
// 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);
|
||||
|
||||
List<? extends RecordBatch> batches = Utils.toList(records.batches().iterator());
|
||||
if (compressionType != CompressionType.NONE) {
|
||||
if (compression.type() != CompressionType.NONE) {
|
||||
assertEquals(2, batches.size());
|
||||
assertEquals(TimestampType.LOG_APPEND_TIME, batches.get(0).timestampType());
|
||||
assertEquals(TimestampType.CREATE_TIME, batches.get(1).timestampType());
|
||||
|
|
@ -602,8 +604,9 @@ public class MemoryRecordsBuilderTest {
|
|||
public void convertToV1WithMixedV0AndV2Data(CompressionType compressionType) {
|
||||
ByteBuffer buffer = ByteBuffer.allocate(512);
|
||||
|
||||
Compression compression = Compression.of(compressionType).build();
|
||||
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) {
|
||||
assertThrows(IllegalArgumentException.class, supplier::get);
|
||||
|
|
@ -612,7 +615,7 @@ public class MemoryRecordsBuilderTest {
|
|||
builder.append(RecordBatch.NO_TIMESTAMP, "1".getBytes(), "a".getBytes());
|
||||
builder.close();
|
||||
|
||||
builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, compressionType,
|
||||
builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, compression,
|
||||
TimestampType.CREATE_TIME, 1L);
|
||||
builder.append(11L, "2".getBytes(), "b".getBytes());
|
||||
builder.append(12L, "3".getBytes(), "c".getBytes());
|
||||
|
|
@ -623,7 +626,7 @@ public class MemoryRecordsBuilderTest {
|
|||
ConvertedRecords<MemoryRecords> convertedRecords = MemoryRecords.readableRecords(buffer)
|
||||
.downConvert(MAGIC_VALUE_V1, 0, time);
|
||||
MemoryRecords records = convertedRecords.records();
|
||||
verifyRecordsProcessingStats(compressionType, convertedRecords.recordConversionStats(), 3, 2,
|
||||
verifyRecordsProcessingStats(compression, convertedRecords.recordConversionStats(), 3, 2,
|
||||
records.sizeInBytes(), buffer.limit());
|
||||
|
||||
List<? extends RecordBatch> batches = Utils.toList(records.batches().iterator());
|
||||
|
|
@ -663,7 +666,7 @@ public class MemoryRecordsBuilderTest {
|
|||
assertEquals("1", utf8(logRecords.get(0).key()));
|
||||
assertEquals("2", utf8(logRecords.get(1).key()));
|
||||
assertEquals("3", utf8(logRecords.get(2).key()));
|
||||
verifyRecordsProcessingStats(compressionType, convertedRecords.recordConversionStats(), 3, 2,
|
||||
verifyRecordsProcessingStats(compression, convertedRecords.recordConversionStats(), 3, 2,
|
||||
records.sizeInBytes(), buffer.limit());
|
||||
} else {
|
||||
assertEquals(2, batches.size());
|
||||
|
|
@ -673,7 +676,7 @@ public class MemoryRecordsBuilderTest {
|
|||
assertEquals(2, batches.get(1).baseOffset());
|
||||
assertEquals("1", utf8(logRecords.get(0).key()));
|
||||
assertEquals("3", utf8(logRecords.get(1).key()));
|
||||
verifyRecordsProcessingStats(compressionType, convertedRecords.recordConversionStats(), 3, 1,
|
||||
verifyRecordsProcessingStats(compression, convertedRecords.recordConversionStats(), 3, 1,
|
||||
records.sizeInBytes(), buffer.limit());
|
||||
}
|
||||
}
|
||||
|
|
@ -684,7 +687,7 @@ public class MemoryRecordsBuilderTest {
|
|||
public void shouldThrowIllegalStateExceptionOnBuildWhenAborted(Args 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, false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
|
||||
builder.abort();
|
||||
|
|
@ -696,7 +699,7 @@ public class MemoryRecordsBuilderTest {
|
|||
public void shouldResetBufferToInitialPositionOnAbort(Args 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,
|
||||
false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
|
||||
builder.append(0L, "a".getBytes(), "1".getBytes());
|
||||
|
|
@ -709,7 +712,7 @@ public class MemoryRecordsBuilderTest {
|
|||
public void shouldThrowIllegalStateExceptionOnCloseWhenAborted(Args 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,
|
||||
false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
|
||||
builder.abort();
|
||||
|
|
@ -721,7 +724,7 @@ public class MemoryRecordsBuilderTest {
|
|||
public void shouldThrowIllegalStateExceptionOnAppendWhenAborted(Args 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,
|
||||
false, false, RecordBatch.NO_PARTITION_LEADER_EPOCH, buffer.capacity());
|
||||
builder.abort();
|
||||
|
|
@ -743,7 +746,7 @@ public class MemoryRecordsBuilderTest {
|
|||
int iterations = 0;
|
||||
while (iterations++ < 100) {
|
||||
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,
|
||||
RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, false, false,
|
||||
RecordBatch.NO_PARTITION_LEADER_EPOCH, 0);
|
||||
|
|
@ -769,7 +772,7 @@ public class MemoryRecordsBuilderTest {
|
|||
int payloadLen = 1024 * 1024;
|
||||
ByteBuffer buffer = ByteBuffer.allocate(payloadLen * 2);
|
||||
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,
|
||||
RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, false, false,
|
||||
RecordBatch.NO_PARTITION_LEADER_EPOCH, 0, deleteHorizon);
|
||||
|
|
@ -798,13 +801,13 @@ public class MemoryRecordsBuilderTest {
|
|||
List<Arguments> values = new ArrayList<>();
|
||||
for (int bufferOffset : Arrays.asList(0, 15))
|
||||
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();
|
||||
}
|
||||
}
|
||||
|
||||
private void verifyRecordsProcessingStats(CompressionType compressionType, RecordValidationStats processingStats,
|
||||
private void verifyRecordsProcessingStats(Compression compression, RecordValidationStats processingStats,
|
||||
int numRecords, int numRecordsConverted, long finalBytes,
|
||||
long preConvertedBytes) {
|
||||
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.
|
||||
assertTrue(processingStats.conversionTimeNanos() >= 0, "Processing time not recorded: " + processingStats);
|
||||
long tempBytes = processingStats.temporaryMemoryBytes();
|
||||
if (compressionType == CompressionType.NONE) {
|
||||
if (compression.type() == CompressionType.NONE) {
|
||||
if (numRecordsConverted == 0)
|
||||
assertEquals(finalBytes, tempBytes);
|
||||
else if (numRecordsConverted == numRecords)
|
||||
|
|
|
|||
|
|
@ -17,6 +17,7 @@
|
|||
package org.apache.kafka.common.record;
|
||||
|
||||
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.header.internals.RecordHeaders;
|
||||
import org.apache.kafka.common.message.LeaderChangeMessage;
|
||||
|
|
@ -58,14 +59,14 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
|
|||
public class MemoryRecordsTest {
|
||||
|
||||
private static class Args {
|
||||
final CompressionType compression;
|
||||
final Compression compression;
|
||||
final byte magic;
|
||||
final long firstOffset;
|
||||
final long pid;
|
||||
final short epoch;
|
||||
final int firstSequence;
|
||||
|
||||
public Args(byte magic, long firstOffset, CompressionType compression) {
|
||||
public Args(byte magic, long firstOffset, Compression compression) {
|
||||
this.magic = magic;
|
||||
this.compression = compression;
|
||||
this.firstOffset = firstOffset;
|
||||
|
|
@ -84,7 +85,7 @@ public class MemoryRecordsTest {
|
|||
public String toString() {
|
||||
return "magic=" + magic +
|
||||
", firstOffset=" + firstOffset +
|
||||
", compressionType=" + compression;
|
||||
", compression=" + compression;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -98,7 +99,7 @@ public class MemoryRecordsTest {
|
|||
? Collections.singletonList(RecordBatch.MAGIC_VALUE_V2)
|
||||
: asList(RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V2);
|
||||
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();
|
||||
}
|
||||
|
|
@ -110,7 +111,7 @@ public class MemoryRecordsTest {
|
|||
List<Arguments> arguments = new ArrayList<>();
|
||||
for (long firstOffset : asList(0L, 57L))
|
||||
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();
|
||||
}
|
||||
|
|
@ -121,7 +122,7 @@ public class MemoryRecordsTest {
|
|||
@ParameterizedTest
|
||||
@ArgumentsSource(MemoryRecordsArgumentsProvider.class)
|
||||
public void testIterator(Args args) {
|
||||
CompressionType compression = args.compression;
|
||||
Compression compression = args.compression;
|
||||
byte magic = args.magic;
|
||||
long pid = args.pid;
|
||||
short epoch = args.epoch;
|
||||
|
|
@ -151,7 +152,7 @@ public class MemoryRecordsTest {
|
|||
int total = 0;
|
||||
for (RecordBatch batch : memoryRecords.batches()) {
|
||||
assertTrue(batch.isValid());
|
||||
assertEquals(compression, batch.compressionType());
|
||||
assertEquals(compression.type(), batch.compressionType());
|
||||
assertEquals(firstOffset + total, batch.baseOffset());
|
||||
|
||||
if (magic >= RecordBatch.MAGIC_VALUE_V2) {
|
||||
|
|
@ -243,10 +244,10 @@ public class MemoryRecordsTest {
|
|||
@ParameterizedTest
|
||||
@ArgumentsSource(MemoryRecordsArgumentsProvider.class)
|
||||
public void testChecksum(Args args) {
|
||||
CompressionType compression = args.compression;
|
||||
Compression compression = args.compression;
|
||||
byte magic = args.magic;
|
||||
// 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;
|
||||
|
||||
SimpleRecord[] records = {
|
||||
|
|
@ -256,17 +257,17 @@ public class MemoryRecordsTest {
|
|||
RecordBatch batch = MemoryRecords.withRecords(magic, compression, records).batches().iterator().next();
|
||||
long expectedChecksum;
|
||||
if (magic == RecordBatch.MAGIC_VALUE_V0) {
|
||||
if (compression == CompressionType.NONE)
|
||||
if (compression.type() == CompressionType.NONE)
|
||||
expectedChecksum = 1978725405L;
|
||||
else
|
||||
expectedChecksum = 66944826L;
|
||||
} else if (magic == RecordBatch.MAGIC_VALUE_V1) {
|
||||
if (compression == CompressionType.NONE)
|
||||
if (compression.type() == CompressionType.NONE)
|
||||
expectedChecksum = 109425508L;
|
||||
else
|
||||
expectedChecksum = 1407303399L;
|
||||
} else {
|
||||
if (compression == CompressionType.NONE)
|
||||
if (compression.type() == CompressionType.NONE)
|
||||
expectedChecksum = 3851219455L;
|
||||
else
|
||||
expectedChecksum = 2745969314L;
|
||||
|
|
@ -590,7 +591,7 @@ public class MemoryRecordsTest {
|
|||
@ParameterizedTest
|
||||
@ArgumentsSource(MemoryRecordsArgumentsProvider.class)
|
||||
public void testFilterToBatchDiscard(Args args) {
|
||||
CompressionType compression = args.compression;
|
||||
Compression compression = args.compression;
|
||||
byte magic = args.magic;
|
||||
|
||||
ByteBuffer buffer = ByteBuffer.allocate(2048);
|
||||
|
|
@ -635,7 +636,7 @@ public class MemoryRecordsTest {
|
|||
MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered);
|
||||
|
||||
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(0, batches.get(0).lastOffset());
|
||||
assertEquals(5, batches.get(1).lastOffset());
|
||||
|
|
@ -650,7 +651,7 @@ public class MemoryRecordsTest {
|
|||
@ArgumentsSource(MemoryRecordsArgumentsProvider.class)
|
||||
public void testFilterToAlreadyCompactedLog(Args args) {
|
||||
byte magic = args.magic;
|
||||
CompressionType compression = args.compression;
|
||||
Compression compression = args.compression;
|
||||
|
||||
ByteBuffer buffer = ByteBuffer.allocate(2048);
|
||||
|
||||
|
|
@ -699,7 +700,7 @@ public class MemoryRecordsTest {
|
|||
@ArgumentsSource(MemoryRecordsArgumentsProvider.class)
|
||||
public void testFilterToPreservesProducerInfo(Args args) {
|
||||
byte magic = args.magic;
|
||||
CompressionType compression = args.compression;
|
||||
Compression compression = args.compression;
|
||||
ByteBuffer buffer = ByteBuffer.allocate(2048);
|
||||
|
||||
// non-idempotent, non-transactional
|
||||
|
|
@ -802,7 +803,7 @@ public class MemoryRecordsTest {
|
|||
@ArgumentsSource(MemoryRecordsArgumentsProvider.class)
|
||||
public void testFilterToWithUndersizedBuffer(Args args) {
|
||||
byte magic = args.magic;
|
||||
CompressionType compression = args.compression;
|
||||
Compression compression = args.compression;
|
||||
|
||||
ByteBuffer buffer = ByteBuffer.allocate(1024);
|
||||
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME, 0L);
|
||||
|
|
@ -856,7 +857,7 @@ public class MemoryRecordsTest {
|
|||
@ArgumentsSource(MemoryRecordsArgumentsProvider.class)
|
||||
public void testFilterTo(Args args) {
|
||||
byte magic = args.magic;
|
||||
CompressionType compression = args.compression;
|
||||
Compression compression = args.compression;
|
||||
|
||||
ByteBuffer buffer = ByteBuffer.allocate(2048);
|
||||
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME, 0L);
|
||||
|
|
@ -893,7 +894,7 @@ public class MemoryRecordsTest {
|
|||
assertEquals(filtered.limit(), result.bytesRetained());
|
||||
if (magic > RecordBatch.MAGIC_VALUE_V0) {
|
||||
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());
|
||||
else
|
||||
assertEquals(5L, result.shallowOffsetOfMaxTimestamp());
|
||||
|
|
@ -906,7 +907,7 @@ public class MemoryRecordsTest {
|
|||
final List<Long> expectedStartOffsets;
|
||||
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);
|
||||
expectedStartOffsets = asList(1L, 4L, 5L, 6L);
|
||||
expectedMaxTimestamps = asList(11L, 20L, 15L, 16L);
|
||||
|
|
@ -927,7 +928,7 @@ public class MemoryRecordsTest {
|
|||
assertEquals(expectedStartOffsets.get(i).longValue(), batch.baseOffset());
|
||||
assertEquals(expectedEndOffsets.get(i).longValue(), batch.lastOffset());
|
||||
assertEquals(magic, batch.magic());
|
||||
assertEquals(compression, batch.compressionType());
|
||||
assertEquals(compression.type(), batch.compressionType());
|
||||
if (magic >= RecordBatch.MAGIC_VALUE_V1) {
|
||||
assertEquals(expectedMaxTimestamps.get(i).longValue(), batch.maxTimestamp());
|
||||
assertEquals(TimestampType.CREATE_TIME, batch.timestampType());
|
||||
|
|
@ -973,7 +974,7 @@ public class MemoryRecordsTest {
|
|||
@ArgumentsSource(MemoryRecordsArgumentsProvider.class)
|
||||
public void testFilterToPreservesLogAppendTime(Args args) {
|
||||
byte magic = args.magic;
|
||||
CompressionType compression = args.compression;
|
||||
Compression compression = args.compression;
|
||||
long pid = args.pid;
|
||||
short epoch = args.epoch;
|
||||
int firstSequence = args.firstSequence;
|
||||
|
|
@ -1008,10 +1009,10 @@ public class MemoryRecordsTest {
|
|||
MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered);
|
||||
|
||||
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) {
|
||||
assertEquals(compression, batch.compressionType());
|
||||
assertEquals(compression.type(), batch.compressionType());
|
||||
if (magic > RecordBatch.MAGIC_VALUE_V0) {
|
||||
assertEquals(TimestampType.LOG_APPEND_TIME, batch.timestampType());
|
||||
assertEquals(logAppendTime, batch.maxTimestamp());
|
||||
|
|
@ -1054,7 +1055,7 @@ public class MemoryRecordsTest {
|
|||
@ParameterizedTest
|
||||
@ArgumentsSource(MemoryRecordsArgumentsProvider.class)
|
||||
public void testWithRecords(Args args) {
|
||||
CompressionType compression = args.compression;
|
||||
Compression compression = args.compression;
|
||||
byte magic = args.magic;
|
||||
MemoryRecords memoryRecords = MemoryRecords.withRecords(magic, compression,
|
||||
new SimpleRecord(10L, "key1".getBytes(), "value1".getBytes()));
|
||||
|
|
@ -1065,7 +1066,7 @@ public class MemoryRecordsTest {
|
|||
@Test
|
||||
public void testUnsupportedCompress() {
|
||||
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 -> {
|
||||
Exception e = assertThrows(IllegalArgumentException.class, () -> builderBiFunction.apply(magic, CompressionType.ZSTD));
|
||||
|
|
|
|||
|
|
@ -16,6 +16,7 @@
|
|||
*/
|
||||
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.internals.RecordHeader;
|
||||
import org.apache.kafka.test.TestUtils;
|
||||
|
|
@ -51,9 +52,9 @@ public class RemoteLogInputStreamTest {
|
|||
|
||||
private static class Args {
|
||||
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.compression = compression;
|
||||
}
|
||||
|
|
@ -71,7 +72,7 @@ public class RemoteLogInputStreamTest {
|
|||
List<Arguments> values = new ArrayList<>();
|
||||
for (byte magic : asList(MAGIC_VALUE_V0, MAGIC_VALUE_V1, MAGIC_VALUE_V2)) {
|
||||
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();
|
||||
|
|
@ -82,8 +83,8 @@ public class RemoteLogInputStreamTest {
|
|||
@ArgumentsSource(RemoteLogInputStreamArgsProvider.class)
|
||||
public void testSimpleBatchIteration(Args args) throws IOException {
|
||||
byte magic = args.magic;
|
||||
CompressionType compression = args.compression;
|
||||
if (compression == CompressionType.ZSTD && magic < MAGIC_VALUE_V2)
|
||||
Compression compression = args.compression;
|
||||
if (compression.type() == CompressionType.ZSTD && magic < MAGIC_VALUE_V2)
|
||||
return;
|
||||
|
||||
SimpleRecord firstBatchRecord = new SimpleRecord(3241324L, "a".getBytes(), "foo".getBytes());
|
||||
|
|
@ -115,11 +116,11 @@ public class RemoteLogInputStreamTest {
|
|||
@ArgumentsSource(RemoteLogInputStreamArgsProvider.class)
|
||||
public void testBatchIterationWithMultipleRecordsPerBatch(Args args) throws IOException {
|
||||
byte magic = args.magic;
|
||||
CompressionType compression = args.compression;
|
||||
if (magic < MAGIC_VALUE_V2 && compression == CompressionType.NONE)
|
||||
Compression compression = args.compression;
|
||||
if (magic < MAGIC_VALUE_V2 && compression.type() == CompressionType.NONE)
|
||||
return;
|
||||
|
||||
if (compression == CompressionType.ZSTD && magic < MAGIC_VALUE_V2)
|
||||
if (compression.type() == CompressionType.ZSTD && magic < MAGIC_VALUE_V2)
|
||||
return;
|
||||
|
||||
SimpleRecord[] firstBatchRecords = new SimpleRecord[]{
|
||||
|
|
@ -159,7 +160,7 @@ public class RemoteLogInputStreamTest {
|
|||
@ArgumentsSource(RemoteLogInputStreamArgsProvider.class)
|
||||
public void testBatchIterationV2(Args args) throws IOException {
|
||||
byte magic = args.magic;
|
||||
CompressionType compression = args.compression;
|
||||
Compression compression = args.compression;
|
||||
if (magic != MAGIC_VALUE_V2)
|
||||
return;
|
||||
|
||||
|
|
@ -213,8 +214,8 @@ public class RemoteLogInputStreamTest {
|
|||
@ArgumentsSource(RemoteLogInputStreamArgsProvider.class)
|
||||
public void testBatchIterationIncompleteBatch(Args args) throws IOException {
|
||||
byte magic = args.magic;
|
||||
CompressionType compression = args.compression;
|
||||
if (compression == CompressionType.ZSTD && magic < MAGIC_VALUE_V2)
|
||||
Compression compression = args.compression;
|
||||
if (compression.type() == CompressionType.ZSTD && magic < MAGIC_VALUE_V2)
|
||||
return;
|
||||
|
||||
try (FileRecords fileRecords = FileRecords.open(tempFile())) {
|
||||
|
|
@ -259,9 +260,9 @@ public class RemoteLogInputStreamTest {
|
|||
long maxTimestamp,
|
||||
SimpleRecord... records) {
|
||||
byte magic = args.magic;
|
||||
CompressionType compression = args.compression;
|
||||
Compression compression = args.compression;
|
||||
assertEquals(magic, batch.magic());
|
||||
assertEquals(compression, batch.compressionType());
|
||||
assertEquals(compression.type(), batch.compressionType());
|
||||
|
||||
if (magic == MAGIC_VALUE_V0) {
|
||||
assertEquals(NO_TIMESTAMP_TYPE, batch.timestampType());
|
||||
|
|
|
|||
|
|
@ -17,6 +17,7 @@
|
|||
package org.apache.kafka.common.record;
|
||||
|
||||
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.utils.ByteBufferOutputStream;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
|
|
@ -47,7 +48,8 @@ public class SimpleLegacyRecordTest {
|
|||
@Test
|
||||
public void testCompressedIterationWithEmptyRecords() throws Exception {
|
||||
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);
|
||||
gzipOutput.close();
|
||||
emptyCompressedValue.flip();
|
||||
|
|
|
|||
|
|
@ -17,6 +17,7 @@
|
|||
package org.apache.kafka.common.record;
|
||||
|
||||
|
||||
import org.apache.kafka.common.compress.Compression;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
|
@ -71,7 +72,7 @@ public class UnalignedFileRecordsTest {
|
|||
FileRecords fileRecords = FileRecords.open(tempFile());
|
||||
|
||||
for (byte[] value : values) {
|
||||
fileRecords.append(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord(value)));
|
||||
fileRecords.append(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord(value)));
|
||||
}
|
||||
|
||||
return fileRecords;
|
||||
|
|
|
|||
|
|
@ -18,10 +18,10 @@
|
|||
package org.apache.kafka.common.requests;
|
||||
|
||||
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.message.ProduceRequestData;
|
||||
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.MemoryRecordsBuilder;
|
||||
import org.apache.kafka.common.record.RecordBatch;
|
||||
|
|
@ -48,7 +48,7 @@ public class ProduceRequestTest {
|
|||
|
||||
@Test
|
||||
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);
|
||||
|
||||
final ProduceRequest request = ProduceRequest.forCurrentMagic(new ProduceRequestData()
|
||||
|
|
@ -78,7 +78,7 @@ public class ProduceRequestTest {
|
|||
|
||||
@Test
|
||||
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);
|
||||
final ProduceRequest request = ProduceRequest.forCurrentMagic(new ProduceRequestData()
|
||||
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
|
||||
|
|
@ -96,7 +96,7 @@ public class ProduceRequestTest {
|
|||
@Test
|
||||
public void testBuildWithOldMessageFormat() {
|
||||
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);
|
||||
builder.append(10L, null, "a".getBytes());
|
||||
ProduceRequest.Builder requestBuilder = ProduceRequest.forMagic(RecordBatch.MAGIC_VALUE_V1,
|
||||
|
|
@ -115,7 +115,7 @@ public class ProduceRequestTest {
|
|||
public void testBuildWithCurrentMessageFormat() {
|
||||
ByteBuffer buffer = ByteBuffer.allocate(256);
|
||||
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());
|
||||
ProduceRequest.Builder requestBuilder = ProduceRequest.forMagic(RecordBatch.CURRENT_MAGIC_VALUE,
|
||||
new ProduceRequestData()
|
||||
|
|
@ -132,11 +132,11 @@ public class ProduceRequestTest {
|
|||
@Test
|
||||
public void testV3AndAboveShouldContainOnlyOneRecordBatch() {
|
||||
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.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(12L, null, "c".getBytes());
|
||||
builder.close();
|
||||
|
|
@ -174,7 +174,7 @@ public class ProduceRequestTest {
|
|||
@Test
|
||||
public void testV3AndAboveCannotUseMagicV0() {
|
||||
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);
|
||||
builder.append(10L, null, "a".getBytes());
|
||||
|
||||
|
|
@ -194,7 +194,7 @@ public class ProduceRequestTest {
|
|||
@Test
|
||||
public void testV3AndAboveCannotUseMagicV1() {
|
||||
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);
|
||||
builder.append(10L, null, "a".getBytes());
|
||||
|
||||
|
|
@ -214,7 +214,7 @@ public class ProduceRequestTest {
|
|||
@Test
|
||||
public void testV6AndBelowCannotUseZStdCompression() {
|
||||
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);
|
||||
builder.append(10L, null, "a".getBytes());
|
||||
|
||||
|
|
@ -245,9 +245,9 @@ public class ProduceRequestTest {
|
|||
final short producerEpoch = 5;
|
||||
final int sequence = 10;
|
||||
|
||||
final MemoryRecords nonTxnRecords = MemoryRecords.withRecords(CompressionType.NONE,
|
||||
final MemoryRecords nonTxnRecords = MemoryRecords.withRecords(Compression.NONE,
|
||||
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()));
|
||||
|
||||
ProduceRequest.Builder builder = ProduceRequest.forMagic(RecordBatch.CURRENT_MAGIC_VALUE,
|
||||
|
|
@ -271,9 +271,9 @@ public class ProduceRequestTest {
|
|||
final short producerEpoch = 5;
|
||||
final int sequence = 10;
|
||||
|
||||
final MemoryRecords nonIdempotentRecords = MemoryRecords.withRecords(CompressionType.NONE,
|
||||
final MemoryRecords nonIdempotentRecords = MemoryRecords.withRecords(Compression.NONE,
|
||||
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()));
|
||||
|
||||
ProduceRequest.Builder builder = ProduceRequest.forMagic(RecordVersion.current().value,
|
||||
|
|
@ -305,7 +305,7 @@ public class ProduceRequestTest {
|
|||
.setName("topic")
|
||||
.setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData()
|
||||
.setIndex(1)
|
||||
.setRecords(MemoryRecords.withRecords(CompressionType.NONE, simpleRecord)))))
|
||||
.setRecords(MemoryRecords.withRecords(Compression.NONE, simpleRecord)))))
|
||||
.iterator()))
|
||||
.setAcks((short) -1)
|
||||
.setTimeoutMs(10)).build();
|
||||
|
|
|
|||
|
|
@ -29,6 +29,7 @@ import org.apache.kafka.common.acl.AclBinding;
|
|||
import org.apache.kafka.common.acl.AclBindingFilter;
|
||||
import org.apache.kafka.common.acl.AclOperation;
|
||||
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.errors.NotCoordinatorException;
|
||||
import org.apache.kafka.common.errors.NotEnoughReplicasException;
|
||||
|
|
@ -448,9 +449,9 @@ public class RequestResponseTest {
|
|||
TopicPartition tp0 = new TopicPartition("test", 0);
|
||||
TopicPartition tp1 = new TopicPartition("test", 1);
|
||||
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,
|
||||
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,
|
||||
new ProduceRequestData()
|
||||
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(asList(
|
||||
|
|
@ -1394,7 +1395,7 @@ public class RequestResponseTest {
|
|||
.setEpoch(0))
|
||||
.setPosition(234L)
|
||||
.setSize(345L)
|
||||
.setUnalignedRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("blah".getBytes())))))))
|
||||
.setUnalignedRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("blah".getBytes())))))))
|
||||
.setThrottleTimeMs(123);
|
||||
return new FetchSnapshotResponse(data);
|
||||
}
|
||||
|
|
@ -1869,7 +1870,7 @@ public class RequestResponseTest {
|
|||
LinkedHashMap<TopicIdPartition, FetchResponseData.PartitionData> responseData = new LinkedHashMap<>();
|
||||
Map<String, Uuid> topicIds = new HashMap<>();
|
||||
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()
|
||||
.setPartitionIndex(0)
|
||||
.setHighWatermark(1000000)
|
||||
|
|
@ -1889,7 +1890,7 @@ public class RequestResponseTest {
|
|||
private FetchResponse createFetchResponse(boolean includeAborted) {
|
||||
LinkedHashMap<TopicIdPartition, FetchResponseData.PartitionData> responseData = new LinkedHashMap<>();
|
||||
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()
|
||||
.setPartitionIndex(0)
|
||||
.setHighWatermark(1000000)
|
||||
|
|
@ -1919,7 +1920,7 @@ public class RequestResponseTest {
|
|||
data.setErrorCode(Errors.NONE.code())
|
||||
.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()
|
||||
.setPartitionIndex(0)
|
||||
.setErrorCode(Errors.NONE.code())
|
||||
|
|
@ -2336,7 +2337,7 @@ public class RequestResponseTest {
|
|||
|
||||
private ProduceRequest createProduceRequest(short version) {
|
||||
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()
|
||||
.setAcks((short) -1)
|
||||
.setTimeoutMs(123)
|
||||
|
|
@ -2349,7 +2350,7 @@ public class RequestResponseTest {
|
|||
return new ProduceRequest.Builder(version, version, data).build(version);
|
||||
}
|
||||
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,
|
||||
new ProduceRequestData()
|
||||
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(singletonList(
|
||||
|
|
|
|||
|
|
@ -32,6 +32,7 @@ import kafka.utils.CoreUtils.inLock
|
|||
import kafka.utils.Implicits._
|
||||
import kafka.utils._
|
||||
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.metrics.{Metrics, Sensor}
|
||||
import org.apache.kafka.common.metrics.stats.{Avg, Max, Meter}
|
||||
|
|
@ -63,7 +64,7 @@ class GroupMetadataManager(brokerId: Int,
|
|||
// Visible for test.
|
||||
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]
|
||||
|
||||
|
|
@ -258,9 +259,9 @@ class GroupMetadataManager(brokerId: Int,
|
|||
val value = GroupMetadataManager.groupMetadataValue(group, groupAssignment, interBrokerProtocolVersion)
|
||||
|
||||
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))
|
||||
val builder = MemoryRecords.builder(buffer, magicValue, compressionType, timestampType, 0L)
|
||||
val builder = MemoryRecords.builder(buffer, magicValue, compression, timestampType, 0L)
|
||||
builder.append(timestamp, key, value)
|
||||
builder.build()
|
||||
}
|
||||
|
|
@ -363,12 +364,12 @@ class GroupMetadataManager(brokerId: Int,
|
|||
val value = GroupMetadataManager.offsetCommitValue(offsetAndMetadata, interBrokerProtocolVersion)
|
||||
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)
|
||||
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)
|
||||
|
||||
records.foreach(builder.append)
|
||||
|
|
@ -908,7 +909,7 @@ class GroupMetadataManager(brokerId: Int,
|
|||
try {
|
||||
// do not need to require acks since even if the tombstone is lost,
|
||||
// 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,
|
||||
requestLocal = requestLocal)
|
||||
|
||||
|
|
|
|||
|
|
@ -21,8 +21,9 @@ import java.nio.ByteBuffer
|
|||
import java.nio.charset.StandardCharsets
|
||||
import kafka.internals.generated.{TransactionLogKey, TransactionLogValue}
|
||||
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.record.{CompressionType, Record, RecordBatch}
|
||||
import org.apache.kafka.common.record.{Record, RecordBatch}
|
||||
import org.apache.kafka.common.{MessageFormatter, TopicPartition}
|
||||
|
||||
import scala.collection.mutable
|
||||
|
|
@ -43,7 +44,7 @@ object TransactionLog {
|
|||
// 2. compression = none
|
||||
// 3. unclean leader election = disabled
|
||||
// 4. required acks = -1 when writing
|
||||
val EnforcedCompressionType: CompressionType = CompressionType.NONE
|
||||
val EnforcedCompression: Compression = Compression.NONE
|
||||
val EnforcedRequiredAcks: Short = (-1).toShort
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -165,7 +165,7 @@ class TransactionStateManager(brokerId: Int,
|
|||
if (recordsBuilder == null) {
|
||||
recordsBuilder = MemoryRecords.builder(
|
||||
ByteBuffer.allocate(math.min(16384, maxBatchSize)),
|
||||
TransactionLog.EnforcedCompressionType,
|
||||
TransactionLog.EnforcedCompression,
|
||||
TimestampType.CREATE_TIME,
|
||||
0L,
|
||||
maxBatchSize
|
||||
|
|
@ -621,7 +621,7 @@ class TransactionStateManager(brokerId: Int,
|
|||
val valueBytes = TransactionLog.valueToBytes(newMetadata)
|
||||
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 recordsPerPartition = Map(topicPartition -> records)
|
||||
|
||||
|
|
|
|||
|
|
@ -791,7 +791,7 @@ class UnifiedLog(@volatile var logStartOffset: Long,
|
|||
val offset = PrimitiveRef.ofLong(localLog.logEndOffset)
|
||||
appendInfo.setFirstOffset(offset.value)
|
||||
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,
|
||||
topicPartition,
|
||||
time,
|
||||
|
|
|
|||
|
|
@ -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.
|
||||
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.")
|
||||
FetchResponse.partitionResponse(tp, Errors.UNSUPPORTED_COMPRESSION_TYPE)
|
||||
} else {
|
||||
|
|
|
|||
|
|
@ -26,6 +26,7 @@ import kafka.utils.{CoreUtils, Logging}
|
|||
import kafka.utils.Implicits._
|
||||
import org.apache.kafka.clients.CommonClientConfigs
|
||||
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.ConfigDef.{ConfigKey, ValidList}
|
||||
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 DeleteTopicEnableProp = "delete.topic.enable"
|
||||
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 ***********/
|
||||
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 " +
|
||||
"('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."
|
||||
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 ***********/
|
||||
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 **************/
|
||||
val MaxRequestPartitionSizeLimitDoc = "The maximum number of partitions can be served in one request."
|
||||
|
||||
|
||||
/** ********* Delegation Token Configuration ****************/
|
||||
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. " +
|
||||
|
|
@ -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(DeleteTopicEnableProp, BOOLEAN, Defaults.DELETE_TOPIC_ENABLE, HIGH, DeleteTopicEnableDoc)
|
||||
.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 ***********/
|
||||
.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)
|
||||
def compressionType = getString(KafkaConfig.CompressionTypeProp)
|
||||
def gzipCompressionLevel = getInt(KafkaConfig.CompressionGzipLevelProp)
|
||||
def lz4CompressionLevel = getInt(KafkaConfig.CompressionLz4LevelProp)
|
||||
def zstdCompressionLevel = getInt(KafkaConfig.CompressionZstdLevelProp)
|
||||
|
||||
/** ********* Raft Quorum Configuration *********/
|
||||
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.MIN_IN_SYNC_REPLICAS_CONFIG, minInSyncReplicas)
|
||||
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.PREALLOCATE_CONFIG, logPreAllocateEnable)
|
||||
logProps.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, logMessageFormatVersion.version)
|
||||
|
|
|
|||
|
|
@ -27,10 +27,10 @@ import org.apache.kafka.common.KafkaException;
|
|||
import org.apache.kafka.common.TopicIdPartition;
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
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.errors.ReplicaNotAvailableException;
|
||||
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.MemoryRecords;
|
||||
import org.apache.kafka.common.record.RecordBatch;
|
||||
|
|
@ -1210,7 +1210,7 @@ public class RemoteLogManagerTest {
|
|||
private MemoryRecords records(long timestamp,
|
||||
long initialOffset,
|
||||
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, "second message".getBytes()),
|
||||
new SimpleRecord(timestamp + 2, "third message".getBytes())
|
||||
|
|
|
|||
|
|
@ -26,6 +26,7 @@ import org.apache.kafka.clients.producer._
|
|||
import org.apache.kafka.common.acl.AclOperation._
|
||||
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.compress.Compression
|
||||
import org.apache.kafka.common.config.{ConfigResource, LogLevelConfig, TopicConfig}
|
||||
import org.apache.kafka.common.errors._
|
||||
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.network.ListenerName
|
||||
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._
|
||||
import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED}
|
||||
|
|
@ -257,7 +258,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
|
|||
.setName(tp.topic).setPartitionData(Collections.singletonList(
|
||||
new ProduceRequestData.PartitionProduceData()
|
||||
.setIndex(tp.partition)
|
||||
.setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes))))))
|
||||
.setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("test".getBytes))))))
|
||||
.iterator))
|
||||
.setAcks(1.toShort)
|
||||
.setTimeoutMs(5000))
|
||||
|
|
|
|||
|
|
@ -21,11 +21,12 @@ package kafka.network
|
|||
import kafka.server.BaseRequestTest
|
||||
import kafka.utils.TestUtils
|
||||
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.network.ListenerName
|
||||
import org.apache.kafka.common.protocol.Errors
|
||||
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.security.auth.SecurityProtocol
|
||||
import org.apache.kafka.common.{KafkaException, requests}
|
||||
|
|
@ -308,7 +309,7 @@ class DynamicConnectionQuotaTest extends BaseRequestTest {
|
|||
.setName(topic)
|
||||
.setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData()
|
||||
.setIndex(0)
|
||||
.setRecords(MemoryRecords.withRecords(CompressionType.NONE,
|
||||
.setRecords(MemoryRecords.withRecords(Compression.NONE,
|
||||
new SimpleRecord(System.currentTimeMillis(), "key".getBytes, "value".getBytes))))))
|
||||
.iterator))
|
||||
.setAcks((-1).toShort)
|
||||
|
|
|
|||
|
|
@ -681,7 +681,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
|
|||
val log2 = servers.head.logManager.getLog(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0))
|
||||
.getOrElse(throw new IllegalStateException("Log not found"))
|
||||
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
|
||||
props.clear()
|
||||
|
|
|
|||
|
|
@ -19,10 +19,11 @@ package kafka.raft
|
|||
import kafka.log.UnifiedLog
|
||||
import kafka.server.{KafkaConfig, KafkaRaftServer}
|
||||
import kafka.utils.TestUtils
|
||||
import org.apache.kafka.common.compress.Compression
|
||||
import org.apache.kafka.common.errors.{InvalidConfigurationException, RecordTooLargeException}
|
||||
import org.apache.kafka.common.protocol
|
||||
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.raft._
|
||||
import org.apache.kafka.raft.internals.BatchBuilder
|
||||
|
|
@ -89,7 +90,7 @@ final class KafkaMetadataLogTest {
|
|||
val initialOffset = log.endOffset().offset
|
||||
|
||||
log.appendAsLeader(
|
||||
MemoryRecords.withRecords(initialOffset, CompressionType.NONE, currentEpoch, recordFoo),
|
||||
MemoryRecords.withRecords(initialOffset, Compression.NONE, currentEpoch, recordFoo),
|
||||
currentEpoch
|
||||
)
|
||||
|
||||
|
|
@ -98,7 +99,7 @@ final class KafkaMetadataLogTest {
|
|||
classOf[RuntimeException],
|
||||
() => {
|
||||
log.appendAsLeader(
|
||||
MemoryRecords.withRecords(initialOffset, CompressionType.NONE, currentEpoch, recordFoo),
|
||||
MemoryRecords.withRecords(initialOffset, Compression.NONE, currentEpoch, recordFoo),
|
||||
currentEpoch
|
||||
)
|
||||
}
|
||||
|
|
@ -108,7 +109,7 @@ final class KafkaMetadataLogTest {
|
|||
classOf[RuntimeException],
|
||||
() => {
|
||||
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]](
|
||||
buffer,
|
||||
new ByteArraySerde,
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
0L,
|
||||
mockTime.milliseconds(),
|
||||
false,
|
||||
|
|
@ -1060,7 +1061,7 @@ object KafkaMetadataLogTest {
|
|||
log.appendAsLeader(
|
||||
MemoryRecords.withRecords(
|
||||
log.endOffset().offset,
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
epoch,
|
||||
(0 until numberOfRecords).map(number => new SimpleRecord(number.toString.getBytes)): _*
|
||||
),
|
||||
|
|
@ -1071,7 +1072,7 @@ object KafkaMetadataLogTest {
|
|||
def append(snapshotWriter: RawSnapshotWriter, numberOfRecords: Int): Unit = {
|
||||
snapshotWriter.append(MemoryRecords.withRecords(
|
||||
0,
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
0,
|
||||
(0 until numberOfRecords).map(number => new SimpleRecord(number.toString.getBytes)): _*
|
||||
))
|
||||
|
|
|
|||
|
|
@ -21,13 +21,14 @@ import kafka.cluster.BrokerEndPoint
|
|||
import kafka.server.QuotaFactory.QuotaManagers
|
||||
import kafka.server.checkpoints.LazyOffsetCheckpoints
|
||||
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.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState
|
||||
import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition
|
||||
import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset
|
||||
import org.apache.kafka.common.metrics.Metrics
|
||||
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.ProduceResponse.PartitionResponse
|
||||
import org.apache.kafka.server.common.OffsetAndEpoch
|
||||
|
|
@ -265,7 +266,7 @@ class LocalLeaderEndPointTest extends Logging {
|
|||
}
|
||||
|
||||
private def records: MemoryRecords = {
|
||||
MemoryRecords.withRecords(CompressionType.NONE,
|
||||
MemoryRecords.withRecords(Compression.NONE,
|
||||
new SimpleRecord("first message".getBytes()),
|
||||
new SimpleRecord("second message".getBytes()),
|
||||
new SimpleRecord("third message".getBytes()),
|
||||
|
|
|
|||
|
|
@ -26,6 +26,7 @@ import joptsimple._
|
|||
import kafka.log._
|
||||
import kafka.server.BrokerTopicStats
|
||||
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.record._
|
||||
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.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.
|
||||
|
|
@ -66,29 +67,34 @@ object TestLinearWriteSpeed {
|
|||
.describedAs("num_files")
|
||||
.ofType(classOf[java.lang.Integer])
|
||||
.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
|
||||
.describedAs("ms")
|
||||
.ofType(classOf[java.lang.Long])
|
||||
.defaultsTo(1000L)
|
||||
val maxThroughputOpt = parser.accepts("max-throughput-mb", "The maximum throughput.")
|
||||
val maxThroughputOpt = parser.accepts("max-throughput-mb", "The maximum throughput.")
|
||||
.withRequiredArg
|
||||
.describedAs("mb")
|
||||
.ofType(classOf[java.lang.Integer])
|
||||
.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()
|
||||
.describedAs("message_count")
|
||||
.ofType(classOf[java.lang.Long])
|
||||
.defaultsTo(Long.MaxValue)
|
||||
val compressionCodecOpt = parser.accepts("compression", "The compression codec to use")
|
||||
val compressionCodecOpt = parser.accepts("compression", "The compression codec to use")
|
||||
.withRequiredArg
|
||||
.describedAs("codec")
|
||||
.ofType(classOf[java.lang.String])
|
||||
.defaultsTo(CompressionType.NONE.name)
|
||||
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 compressionLevelOpt = parser.accepts("level", "The compression level to use")
|
||||
.withRequiredArg
|
||||
.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 : _*)
|
||||
|
||||
|
|
@ -104,13 +110,22 @@ object TestLinearWriteSpeed {
|
|||
val messageSize = options.valueOf(messageSizeOpt).intValue
|
||||
val flushInterval = options.valueOf(flushIntervalOpt).longValue
|
||||
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
|
||||
rand.nextBytes(buffer.array)
|
||||
val numMessages = bufferSize / (messageSize + Records.LOG_OVERHEAD)
|
||||
val createTime = System.currentTimeMillis
|
||||
val messageSet = {
|
||||
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)
|
||||
|
|
@ -197,7 +212,7 @@ object TestLinearWriteSpeed {
|
|||
|
||||
class ChannelWritable(val file: File, val content: ByteBuffer) extends Writable {
|
||||
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)
|
||||
def write(): Int = {
|
||||
channel.write(content)
|
||||
|
|
@ -212,7 +227,7 @@ object TestLinearWriteSpeed {
|
|||
|
||||
class LogWritable(val dir: File, config: LogConfig, scheduler: Scheduler, val messages: MemoryRecords) extends Writable {
|
||||
Utils.delete(dir)
|
||||
val log = UnifiedLog(
|
||||
val log: UnifiedLog = UnifiedLog(
|
||||
dir = dir,
|
||||
config = config,
|
||||
logStartOffset = 0L,
|
||||
|
|
|
|||
|
|
@ -47,6 +47,7 @@ import java.util.Optional
|
|||
import java.util.concurrent.{CountDownLatch, Semaphore}
|
||||
import kafka.server.metadata.{KRaftMetadataCache, ZkMetadataCache}
|
||||
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.network.ListenerName
|
||||
import org.apache.kafka.common.replica.ClientMetadata
|
||||
|
|
@ -159,7 +160,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
val recordArray = (1 to count).map { i =>
|
||||
new SimpleRecord(s"$i".getBytes)
|
||||
}
|
||||
val records = MemoryRecords.withRecords(0L, CompressionType.NONE, leaderEpoch,
|
||||
val records = MemoryRecords.withRecords(0L, Compression.NONE, leaderEpoch,
|
||||
recordArray: _*)
|
||||
log.appendAsLeader(records, leaderEpoch = leaderEpoch)
|
||||
}
|
||||
|
|
@ -239,11 +240,11 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
val leaderEpoch = 8
|
||||
|
||||
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("k2".getBytes, "v2".getBytes)
|
||||
), 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("k4".getBytes, "v4".getBytes)
|
||||
), leaderEpoch = 5)
|
||||
|
|
@ -505,7 +506,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
|
||||
// Write records with duplicate keys to current replica and roll at offset 6
|
||||
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, "v2".getBytes),
|
||||
new SimpleRecord("k1".getBytes, "v3".getBytes),
|
||||
|
|
@ -514,7 +515,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
new SimpleRecord("k2".getBytes, "v6".getBytes)
|
||||
), leaderEpoch = 0)
|
||||
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("k4".getBytes, "v8".getBytes)
|
||||
), 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
|
||||
|
||||
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)
|
||||
builder.appendWithOffset(2L, new SimpleRecord("k1".getBytes, "v3".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 = {
|
||||
val buf = ByteBuffer.allocate(DefaultRecordBatch.sizeInBytes(records.asJava))
|
||||
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)
|
||||
records.foreach(builder.append)
|
||||
builder.build()
|
||||
|
|
@ -1197,7 +1198,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
val producerEpoch = 0.toShort
|
||||
val isTransactional = false
|
||||
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)
|
||||
records.foreach(builder.append)
|
||||
builder.build()
|
||||
|
|
@ -1210,7 +1211,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
val producerEpoch = 0.toShort
|
||||
val isTransactional = true
|
||||
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)
|
||||
records.foreach(builder.append)
|
||||
builder.build()
|
||||
|
|
@ -3180,7 +3181,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
assertEquals(Optional.of(new EpochEntry(leaderEpoch, 0L)), leaderLog.leaderEpochCache.asJava.flatMap(_.latestEntry))
|
||||
|
||||
// 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)
|
||||
), leaderEpoch = leaderEpoch)
|
||||
|
|
@ -3671,7 +3672,7 @@ class PartitionTest extends AbstractPartitionTest {
|
|||
|
||||
private def seedLogData(log: UnifiedLog, numRecords: Int, leaderEpoch: Int): Unit = {
|
||||
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))
|
||||
log.appendAsLeader(records, leaderEpoch)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -19,8 +19,9 @@ package kafka.coordinator.group
|
|||
import kafka.log.UnifiedLog
|
||||
import kafka.server.ReplicaManager
|
||||
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.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.utils.{MockTime, Time}
|
||||
import org.apache.kafka.coordinator.group.runtime.CoordinatorLoader.UnknownRecordTypeException
|
||||
|
|
@ -639,13 +640,13 @@ class CoordinatorLoaderImplTest {
|
|||
val memoryRecords = if (producerId == RecordBatch.NO_PRODUCER_ID) {
|
||||
MemoryRecords.withRecords(
|
||||
startOffset,
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
records: _*
|
||||
)
|
||||
} else {
|
||||
MemoryRecords.withTransactionalRecords(
|
||||
startOffset,
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
producerId,
|
||||
producerEpoch,
|
||||
0,
|
||||
|
|
|
|||
|
|
@ -18,9 +18,10 @@ package kafka.coordinator.group
|
|||
|
||||
import kafka.server.ReplicaManager
|
||||
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.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.coordinator.group.runtime.PartitionWriter
|
||||
import org.apache.kafka.storage.internals.log.{AppendOrigin, LogConfig, VerificationGuard}
|
||||
|
|
@ -122,7 +123,7 @@ class CoordinatorPartitionWriterTest {
|
|||
})
|
||||
|
||||
val batch = MemoryRecords.withRecords(
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
new SimpleRecord(
|
||||
0L,
|
||||
"foo".getBytes(Charset.defaultCharset()),
|
||||
|
|
@ -223,7 +224,7 @@ class CoordinatorPartitionWriterTest {
|
|||
})
|
||||
|
||||
val batch = MemoryRecords.withRecords(
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
new SimpleRecord(
|
||||
0L,
|
||||
"foo".getBytes(Charset.defaultCharset()),
|
||||
|
|
|
|||
|
|
@ -32,6 +32,7 @@ import kafka.utils.TestUtils
|
|||
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor
|
||||
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Subscription
|
||||
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.internals.Topic
|
||||
import org.apache.kafka.common.metrics.{JmxReporter, KafkaMetricsContext, Metrics => kMetrics}
|
||||
|
|
@ -148,7 +149,7 @@ class GroupMetadataManagerTest {
|
|||
)
|
||||
|
||||
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)
|
||||
|
||||
groupMetadataManager.loadGroupsAndOffsets(groupMetadataTopicPartition, groupEpoch, _ => (), 0L)
|
||||
|
|
@ -177,7 +178,7 @@ class GroupMetadataManagerTest {
|
|||
|
||||
val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets)
|
||||
val groupMetadataRecord = buildEmptyGroupRecord(generation, protocolType)
|
||||
val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE,
|
||||
val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
|
||||
(offsetCommitRecords ++ Seq(groupMetadataRecord)).toArray: _*)
|
||||
|
||||
expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records)
|
||||
|
|
@ -566,7 +567,7 @@ class GroupMetadataManagerTest {
|
|||
}
|
||||
|
||||
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)
|
||||
commitRecords.foreach(builder.append)
|
||||
builder.build()
|
||||
|
|
@ -575,7 +576,7 @@ class GroupMetadataManagerTest {
|
|||
|
||||
private def appendTransactionalOffsetCommits(buffer: ByteBuffer, producerId: Long, producerEpoch: Short,
|
||||
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)
|
||||
commitRecords.foreach(builder.append)
|
||||
builder.build()
|
||||
|
|
@ -584,7 +585,7 @@ class GroupMetadataManagerTest {
|
|||
|
||||
private def completeTransactionalOffsetCommit(buffer: ByteBuffer, producerId: Long, producerEpoch: Short, baseOffset: Long,
|
||||
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,
|
||||
RecordBatch.NO_PARTITION_LEADER_EPOCH)
|
||||
val controlRecordType = if (isCommit) ControlRecordType.COMMIT else ControlRecordType.ABORT
|
||||
|
|
@ -608,7 +609,7 @@ class GroupMetadataManagerTest {
|
|||
|
||||
val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets)
|
||||
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: _*)
|
||||
|
||||
expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records)
|
||||
|
|
@ -647,7 +648,7 @@ class GroupMetadataManagerTest {
|
|||
val memberId = "98098230493"
|
||||
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: _*)
|
||||
|
||||
expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records)
|
||||
|
|
@ -756,7 +757,7 @@ class GroupMetadataManagerTest {
|
|||
val groupMetadataRecord = buildStableGroupRecordWithMember(generation = 15,
|
||||
protocolType = "consumer", protocol = "range", memberId)
|
||||
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: _*)
|
||||
|
||||
expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records)
|
||||
|
|
@ -784,7 +785,7 @@ class GroupMetadataManagerTest {
|
|||
val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets)
|
||||
val groupMetadataRecord = buildStableGroupRecordWithMember(generation = 15,
|
||||
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: _*)
|
||||
|
||||
expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records)
|
||||
|
|
@ -845,7 +846,7 @@ class GroupMetadataManagerTest {
|
|||
val memberId = "98098230493"
|
||||
val groupMetadataRecord = buildStableGroupRecordWithMember(generation, protocolType, protocol, memberId)
|
||||
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: _*)
|
||||
|
||||
expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records)
|
||||
|
|
@ -879,14 +880,14 @@ class GroupMetadataManagerTest {
|
|||
|
||||
val segment1MemberId = "a"
|
||||
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(
|
||||
generation, protocolType, protocol, segment1MemberId))).toArray: _*)
|
||||
val segment1End = startOffset + segment1Records.records.asScala.size
|
||||
|
||||
val segment2MemberId = "b"
|
||||
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(
|
||||
generation, protocolType, protocol, segment2MemberId))).toArray: _*)
|
||||
val segment2End = segment1End + segment2Records.records.asScala.size
|
||||
|
|
@ -2352,7 +2353,7 @@ class GroupMetadataManagerTest {
|
|||
val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets, metadataVersion = metadataVersion, retentionTimeOpt = Some(100))
|
||||
val memberId = "98098230493"
|
||||
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: _*)
|
||||
|
||||
expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records)
|
||||
|
|
@ -2391,7 +2392,7 @@ class GroupMetadataManagerTest {
|
|||
val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets, retentionTimeOpt = Some(100))
|
||||
val memberId = "98098230493"
|
||||
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: _*)
|
||||
|
||||
expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records)
|
||||
|
|
@ -2731,7 +2732,7 @@ class GroupMetadataManagerTest {
|
|||
|
||||
val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets)
|
||||
val groupMetadataRecord = buildEmptyGroupRecord(generation, protocolType)
|
||||
val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE,
|
||||
val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
|
||||
(offsetCommitRecords ++ Seq(groupMetadataRecord)).toArray: _*)
|
||||
|
||||
// Prepend empty control batch to valid records
|
||||
|
|
@ -2948,7 +2949,7 @@ class GroupMetadataManagerTest {
|
|||
val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets)
|
||||
val groupMetadataRecord = buildStableGroupRecordWithMember(generation = 15,
|
||||
protocolType = "consumer", protocol = "range", memberId)
|
||||
val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE,
|
||||
val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
|
||||
(offsetCommitRecords ++ Seq(groupMetadataRecord)).toArray: _*)
|
||||
|
||||
expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records)
|
||||
|
|
@ -2995,7 +2996,7 @@ class GroupMetadataManagerTest {
|
|||
val unknownRecord1 = new SimpleRecord(unknownMessage1, unknownMessage1)
|
||||
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: _*)
|
||||
|
||||
expectGroupMetadataLoad(groupTopicPartition, startOffset, records)
|
||||
|
|
|
|||
|
|
@ -26,11 +26,12 @@ import kafka.log.UnifiedLog
|
|||
import kafka.server.{KafkaConfig, MetadataCache, RequestLocal}
|
||||
import kafka.utils.{Pool, TestUtils}
|
||||
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.metrics.Metrics
|
||||
import org.apache.kafka.common.network.ListenerName
|
||||
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.utils.{LogContext, MockTime, ProducerIdAndEpoch}
|
||||
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 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
|
||||
|
||||
when(logMock.logStartOffset).thenReturn(startOffset)
|
||||
|
|
|
|||
|
|
@ -21,10 +21,11 @@ import kafka.internals.generated.TransactionLogKey
|
|||
import kafka.internals.generated.TransactionLogValue
|
||||
import kafka.utils.TestUtils
|
||||
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.types.Field.TaggedFieldsSection
|
||||
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.Test
|
||||
|
||||
|
|
@ -84,7 +85,7 @@ class TransactionLogTest {
|
|||
new SimpleRecord(keyBytes, valueBytes)
|
||||
}.toSeq
|
||||
|
||||
val records = MemoryRecords.withRecords(0, CompressionType.NONE, txnRecords: _*)
|
||||
val records = MemoryRecords.withRecords(0, Compression.NONE, txnRecords: _*)
|
||||
|
||||
var count = 0
|
||||
for (record <- records.records.asScala) {
|
||||
|
|
|
|||
|
|
@ -28,6 +28,7 @@ import kafka.server.{ReplicaManager, RequestLocal}
|
|||
import kafka.utils.{Pool, TestUtils}
|
||||
import kafka.zk.KafkaZkClient
|
||||
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.metrics.{JmxReporter, KafkaMetricsContext, Metrics}
|
||||
import org.apache.kafka.common.protocol.{Errors, MessageUtil}
|
||||
|
|
@ -166,7 +167,7 @@ class TransactionStateManagerTest {
|
|||
txnMetadata1.addPartitions(Set[TopicPartition](
|
||||
new TopicPartition("topic1", 0),
|
||||
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())))
|
||||
|
||||
// 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()))
|
||||
|
||||
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)
|
||||
|
||||
|
|
@ -878,7 +879,7 @@ class TransactionStateManagerTest {
|
|||
|
||||
txnRecords += new SimpleRecord(txnMessageKeyBytes1, TransactionLog.valueToBytes(txnMetadata1.prepareNoTransit()))
|
||||
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)
|
||||
|
||||
|
|
@ -1026,7 +1027,7 @@ class TransactionStateManagerTest {
|
|||
val partitionId = transactionManager.partitionFor(transactionalId1)
|
||||
val topicPartition = new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, partitionId)
|
||||
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(Seq(expectedRecords), appendedRecords(topicPartition).toSeq)
|
||||
} else {
|
||||
|
|
@ -1041,7 +1042,7 @@ class TransactionStateManagerTest {
|
|||
|
||||
txnRecords += new SimpleRecord(txnMessageKeyBytes1, TransactionLog.valueToBytes(txnMetadata1.prepareNoTransit()))
|
||||
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)
|
||||
|
||||
|
|
@ -1148,7 +1149,7 @@ class TransactionStateManagerTest {
|
|||
txnRecords += new SimpleRecord(txnMessageKeyBytes1, TransactionLog.valueToBytes(txnMetadata1.prepareNoTransit()))
|
||||
|
||||
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)
|
||||
transactionManager.loadTransactionsForTxnTopicPartition(partitionId, 0, (_, _, _, _) => ())
|
||||
|
|
@ -1171,7 +1172,7 @@ class TransactionStateManagerTest {
|
|||
val unknownMessage = MessageUtil.toVersionPrefixedBytes(Short.MaxValue, unknownKey)
|
||||
val unknownRecord = new SimpleRecord(unknownMessage, unknownMessage)
|
||||
|
||||
val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE,
|
||||
val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
|
||||
(Seq(unknownRecord) ++ txnRecords).toArray: _*)
|
||||
|
||||
prepareTxnLog(topicPartition, 0, records)
|
||||
|
|
|
|||
|
|
@ -23,8 +23,9 @@ import kafka.server.BrokerTopicStats
|
|||
import kafka.utils.{Pool, TestUtils}
|
||||
import kafka.utils.Implicits._
|
||||
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.record.{CompressionType, MemoryRecords, RecordBatch}
|
||||
import org.apache.kafka.common.record.{MemoryRecords, RecordBatch}
|
||||
import org.apache.kafka.common.utils.Utils
|
||||
import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
|
||||
import org.apache.kafka.server.util.MockTime
|
||||
|
|
@ -142,7 +143,7 @@ abstract class AbstractLogCleanerIntegrationTest {
|
|||
def counter: Int = ctr
|
||||
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)] = {
|
||||
for (_ <- 0 until numDups; key <- startKey until (startKey + numKeys)) yield {
|
||||
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 = {
|
||||
val random = new Random(0)
|
||||
new String(random.alphanumeric.take(length).toArray)
|
||||
|
|
|
|||
|
|
@ -19,6 +19,7 @@ package kafka.log
|
|||
|
||||
import kafka.server.BrokerTopicStats
|
||||
import kafka.utils._
|
||||
import org.apache.kafka.common.compress.Compression
|
||||
import org.apache.kafka.common.config.TopicConfig
|
||||
import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch, SimpleRecord}
|
||||
import org.apache.kafka.common.utils.Utils
|
||||
|
|
@ -50,10 +51,10 @@ class BrokerCompressionTest {
|
|||
*/
|
||||
@ParameterizedTest
|
||||
@MethodSource(Array("parameters"))
|
||||
def testBrokerSideCompression(messageCompression: String, brokerCompression: String): Unit = {
|
||||
val messageCompressionType = CompressionType.forName(messageCompression)
|
||||
def testBrokerSideCompression(messageCompressionType: CompressionType, brokerCompressionType: BrokerCompressionType): Unit = {
|
||||
val messageCompression = Compression.of(messageCompressionType).build()
|
||||
val logProps = new Properties()
|
||||
logProps.put(TopicConfig.COMPRESSION_TYPE_CONFIG, brokerCompression)
|
||||
logProps.put(TopicConfig.COMPRESSION_TYPE_CONFIG, brokerCompressionType.name)
|
||||
/*configure broker-side compression */
|
||||
val log = UnifiedLog(
|
||||
dir = logDir,
|
||||
|
|
@ -72,7 +73,7 @@ class BrokerCompressionTest {
|
|||
)
|
||||
|
||||
/* 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)
|
||||
|
||||
def readBatch(offset: Int): RecordBatch = {
|
||||
|
|
@ -83,9 +84,9 @@ class BrokerCompressionTest {
|
|||
fetchInfo.records.batches.iterator.next()
|
||||
}
|
||||
|
||||
if (!brokerCompression.equals("producer")) {
|
||||
val brokerCompressionType = BrokerCompressionType.forName(brokerCompression).targetCompressionType(null)
|
||||
assertEquals(brokerCompressionType, readBatch(0).compressionType, "Compression at offset 0 should produce " + brokerCompressionType)
|
||||
if (brokerCompressionType != BrokerCompressionType.PRODUCER) {
|
||||
val targetCompression = BrokerCompressionType.targetCompression(log.config.compression, null)
|
||||
assertEquals(targetCompression.`type`(), readBatch(0).compressionType, "Compression at offset 0 should produce " + brokerCompressionType)
|
||||
}
|
||||
else
|
||||
assertEquals(messageCompressionType, readBatch(0).compressionType, "Compression at offset 0 should produce " + messageCompressionType)
|
||||
|
|
@ -98,7 +99,7 @@ object BrokerCompressionTest {
|
|||
java.util.Arrays.stream(
|
||||
for (brokerCompression <- BrokerCompressionType.values;
|
||||
messageCompression <- CompressionType.values
|
||||
) yield Arguments.of(messageCompression.name, brokerCompression.name)
|
||||
) yield Arguments.of(messageCompression, brokerCompression)
|
||||
)
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -24,9 +24,10 @@ import java.util.regex.Pattern
|
|||
import java.util.Collections
|
||||
import kafka.server.KafkaConfig
|
||||
import kafka.utils.TestUtils
|
||||
import org.apache.kafka.common.compress.Compression
|
||||
import org.apache.kafka.common.{KafkaException, TopicPartition}
|
||||
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.server.util.{MockTime, Scheduler}
|
||||
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,
|
||||
largestTimestamp = records.head.timestamp,
|
||||
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,
|
||||
|
|
|
|||
|
|
@ -21,7 +21,8 @@ import java.io.PrintWriter
|
|||
import com.yammer.metrics.core.{Gauge, MetricName}
|
||||
import kafka.utils.TestUtils
|
||||
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.util.MockTime
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
|
|
@ -35,7 +36,7 @@ import scala.jdk.CollectionConverters._
|
|||
*/
|
||||
class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest {
|
||||
|
||||
val codec: CompressionType = CompressionType.LZ4
|
||||
val codec: Compression = Compression.lz4().build()
|
||||
|
||||
val time = new MockTime()
|
||||
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 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
|
||||
|
||||
|
|
@ -159,7 +160,7 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest {
|
|||
val T1 = time.milliseconds
|
||||
|
||||
// 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
|
||||
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)] = {
|
||||
var valCounter = startValue
|
||||
for (_ <- 0 until numDups; key <- 0 until numKeys) yield {
|
||||
|
|
|
|||
|
|
@ -19,6 +19,7 @@ package kafka.log
|
|||
|
||||
import kafka.utils._
|
||||
import org.apache.kafka.common.TopicPartition
|
||||
import org.apache.kafka.common.compress.Compression
|
||||
import org.apache.kafka.common.record.CompressionType
|
||||
import org.apache.kafka.server.util.MockTime
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
|
|
@ -45,7 +46,8 @@ class LogCleanerLagIntegrationTest extends AbstractLogCleanerIntegrationTest wit
|
|||
|
||||
@ParameterizedTest
|
||||
@MethodSource(Array("parameters"))
|
||||
def cleanerTest(codec: CompressionType): Unit = {
|
||||
def cleanerTest(compressionType: CompressionType): Unit = {
|
||||
val codec: Compression = Compression.of(compressionType).build()
|
||||
cleaner = makeCleaner(partitions = topicPartitions,
|
||||
backoffMs = cleanerBackOffMs,
|
||||
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 {
|
||||
val count = counter
|
||||
log.appendAsLeader(TestUtils.singletonRecords(value = counter.toString.getBytes, codec = codec,
|
||||
|
|
|
|||
|
|
@ -23,6 +23,7 @@ import java.util.Properties
|
|||
import kafka.server.BrokerTopicStats
|
||||
import kafka.utils._
|
||||
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.record._
|
||||
import org.apache.kafka.common.utils.Utils
|
||||
|
|
@ -667,10 +668,10 @@ class LogCleanerManagerTest extends Logging {
|
|||
val producerId = 15L
|
||||
val producerEpoch = 0.toShort
|
||||
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(), "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)
|
||||
log.roll()
|
||||
log.updateHighWatermark(3L)
|
||||
|
|
@ -853,7 +854,7 @@ class LogCleanerManagerTest extends Logging {
|
|||
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)
|
||||
}
|
||||
|
||||
|
|
@ -876,6 +877,6 @@ class LogCleanerManagerTest extends Logging {
|
|||
}
|
||||
|
||||
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))
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -23,6 +23,7 @@ import kafka.server.KafkaConfig
|
|||
import kafka.server.checkpoints.OffsetCheckpointFile
|
||||
import kafka.utils._
|
||||
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.record._
|
||||
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
|
||||
@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 (largeMessageValue, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, RecordBatch.CURRENT_MAGIC_VALUE, codec)
|
||||
val maxMessageSize = largeMessageSet.sizeInBytes
|
||||
|
|
@ -91,7 +93,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati
|
|||
|
||||
@ParameterizedTest
|
||||
@ArgumentsSource(classOf[LogCleanerParameterizedIntegrationTest.AllCompressions])
|
||||
def testCleansCombinedCompactAndDeleteTopic(codec: CompressionType): Unit = {
|
||||
def testCleansCombinedCompactAndDeleteTopic(compressionType: CompressionType): Unit = {
|
||||
val logProps = new Properties()
|
||||
val retentionMs: Integer = 100000
|
||||
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)
|
||||
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
|
||||
|
||||
log.updateHighWatermark(log.logEndOffset)
|
||||
|
|
@ -136,11 +138,12 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati
|
|||
@nowarn("cat=deprecation")
|
||||
@ParameterizedTest
|
||||
@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 (largeMessageValue, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, RecordBatch.MAGIC_VALUE_V0, codec)
|
||||
val maxMessageSize = codec match {
|
||||
case CompressionType.NONE => largeMessageSet.sizeInBytes
|
||||
case Compression.NONE => largeMessageSet.sizeInBytes
|
||||
case _ =>
|
||||
// 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
|
||||
|
|
@ -191,6 +194,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati
|
|||
@ParameterizedTest
|
||||
@ArgumentsSource(classOf[LogCleanerParameterizedIntegrationTest.ExcludeZstd])
|
||||
def testCleaningNestedMessagesWithV0AndV1(codec: CompressionType): Unit = {
|
||||
val compression = Compression.of(codec).build()
|
||||
val maxMessageSize = 192
|
||||
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
|
||||
// all of the individual messages
|
||||
var appendsV0 = writeDupsSingleMessageSet(numKeys = 2, numDups = 3, log = log, codec = codec, magicValue = RecordBatch.MAGIC_VALUE_V0)
|
||||
appendsV0 ++= writeDupsSingleMessageSet(numKeys = 2, startKey = 3, numDups = 2, 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 = compression, magicValue = RecordBatch.MAGIC_VALUE_V0)
|
||||
|
||||
props.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, IBP_0_10_0_IV1.version)
|
||||
log.updateConfig(new LogConfig(props))
|
||||
|
||||
var 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 = codec, magicValue = RecordBatch.MAGIC_VALUE_V1)
|
||||
appendsV1 ++= writeDupsSingleMessageSet(startKey = 6, 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 = compression, 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
|
||||
|
||||
|
|
@ -228,7 +232,8 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati
|
|||
|
||||
@ParameterizedTest
|
||||
@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 (_, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, RecordBatch.CURRENT_MAGIC_VALUE, codec)
|
||||
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)] = {
|
||||
val kvs = for (_ <- 0 until numDups; key <- startKey until (startKey + numKeys)) yield {
|
||||
val payload = counter.toString
|
||||
|
|
|
|||
|
|
@ -21,6 +21,7 @@ import kafka.common._
|
|||
import kafka.server.{BrokerTopicStats, KafkaConfig}
|
||||
import kafka.utils._
|
||||
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.errors.CorruptRecordException
|
||||
import org.apache.kafka.common.record._
|
||||
|
|
@ -1116,7 +1117,7 @@ class LogCleanerTest extends Logging {
|
|||
|
||||
val producerId2 = 2L
|
||||
val records = MemoryRecords.withTransactionalRecords(
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
producerId2,
|
||||
producerEpoch,
|
||||
0,
|
||||
|
|
@ -1993,7 +1994,7 @@ class LogCleanerTest extends Logging {
|
|||
|
||||
private def invalidCleanedMessage(initialOffset: Long,
|
||||
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
|
||||
// would write invalid compressed message sets with the outer magic set to 1 and the inner
|
||||
// magic set to 0
|
||||
|
|
@ -2004,6 +2005,7 @@ class LogCleanerTest extends Logging {
|
|||
kv._2.toString.getBytes))
|
||||
|
||||
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)
|
||||
|
||||
var offset = initialOffset
|
||||
|
|
@ -2016,7 +2018,7 @@ class LogCleanerTest extends Logging {
|
|||
}
|
||||
|
||||
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 =
|
||||
messageWithOffset(key.toString.getBytes, value.toString.getBytes, offset)
|
||||
|
|
@ -2061,7 +2063,7 @@ class LogCleanerTest extends Logging {
|
|||
producerEpoch: Short = RecordBatch.NO_PRODUCER_EPOCH,
|
||||
sequence: Int = RecordBatch.NO_SEQUENCE,
|
||||
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))
|
||||
}
|
||||
|
||||
|
|
@ -2097,9 +2099,9 @@ class LogCleanerTest extends Logging {
|
|||
new SimpleRecord(time.milliseconds(), keyBytes, keyBytes) // the value doesn't matter since we validate offsets
|
||||
}
|
||||
val records = if (isTransactional)
|
||||
MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, producerEpoch, sequence, simpleRecords.toArray: _*)
|
||||
MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, producerEpoch, sequence, simpleRecords.toArray: _*)
|
||||
else
|
||||
MemoryRecords.withIdempotentRecords(CompressionType.NONE, producerId, producerEpoch, sequence, simpleRecords.toArray: _*)
|
||||
MemoryRecords.withIdempotentRecords(Compression.NONE, producerId, producerEpoch, sequence, simpleRecords.toArray: _*)
|
||||
sequence += simpleRecords.size
|
||||
log.appendAsLeader(records, leaderEpoch, origin)
|
||||
}
|
||||
|
|
|
|||
|
|
@ -95,6 +95,9 @@ class LogConfigTest {
|
|||
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_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")
|
||||
})
|
||||
|
|
@ -215,7 +218,7 @@ class LogConfigTest {
|
|||
values.foreach(value => {
|
||||
val props = new Properties
|
||||
props.setProperty(name, value.toString)
|
||||
assertThrows(classOf[Exception], () => new LogConfig(props))
|
||||
assertThrows(classOf[Exception], () => new LogConfig(props), () => s"Property $name should not allow $value")
|
||||
})
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -25,9 +25,10 @@ import kafka.server.{BrokerTopicStats, KafkaConfig}
|
|||
import kafka.server.metadata.MockConfigRepository
|
||||
import kafka.utils.TestUtils
|
||||
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.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.coordinator.transaction.TransactionLogConfigs
|
||||
import org.apache.kafka.server.common.MetadataVersion
|
||||
|
|
@ -294,7 +295,7 @@ class LogLoaderTest {
|
|||
key: Array[Byte] = null,
|
||||
leaderEpoch: Int,
|
||||
offset: Long,
|
||||
codec: CompressionType = CompressionType.NONE,
|
||||
codec: Compression = Compression.NONE,
|
||||
timestamp: Long = RecordBatch.NO_TIMESTAMP,
|
||||
magicValue: Byte = RecordBatch.CURRENT_MAGIC_VALUE): MemoryRecords = {
|
||||
val records = Seq(new SimpleRecord(timestamp, key, value))
|
||||
|
|
@ -1063,10 +1064,10 @@ class LogLoaderTest {
|
|||
// append some messages to create some segments
|
||||
val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024)
|
||||
val log = createLog(logDir, logConfig)
|
||||
val set1 = MemoryRecords.withRecords(0, CompressionType.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 set3 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 3, CompressionType.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 set1 = MemoryRecords.withRecords(0, Compression.NONE, 0, new SimpleRecord("v1".getBytes(), "k1".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, Compression.NONE, 0, new SimpleRecord("v4".getBytes(), "k4".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
|
||||
log.appendAsFollower(set1)
|
||||
assertEquals(0L, log.activeSegment.baseOffset)
|
||||
|
|
@ -1120,14 +1121,14 @@ class LogLoaderTest {
|
|||
// append some messages to create some segments
|
||||
val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024)
|
||||
val log = createLog(logDir, logConfig)
|
||||
val set1 = MemoryRecords.withRecords(0, CompressionType.NONE, 0, new SimpleRecord("v1".getBytes(), "k1".getBytes()))
|
||||
val set2 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 2, CompressionType.GZIP, 0,
|
||||
val set1 = MemoryRecords.withRecords(0, Compression.NONE, 0, new SimpleRecord("v1".getBytes(), "k1".getBytes()))
|
||||
val set2 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 2, Compression.gzip().build(), 0,
|
||||
new SimpleRecord("v3".getBytes(), "k3".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("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("v8".getBytes(), "k8".getBytes()))
|
||||
//Writes into an empty log with baseOffset 0
|
||||
|
|
@ -1159,15 +1160,15 @@ class LogLoaderTest {
|
|||
// append some messages to create some segments
|
||||
val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024)
|
||||
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()))
|
||||
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("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("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("v8".getBytes(), "k8".getBytes()))
|
||||
//Writes into an empty log with baseOffset 0
|
||||
|
|
|
|||
|
|
@ -19,6 +19,7 @@ package kafka.log
|
|||
import kafka.utils.TestUtils
|
||||
import kafka.utils.TestUtils.random
|
||||
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.record._
|
||||
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 */
|
||||
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) }: _*)
|
||||
}
|
||||
|
||||
|
|
@ -347,15 +348,15 @@ class LogSegmentTest {
|
|||
|
||||
// append transactional records from pid1
|
||||
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)))
|
||||
|
||||
// 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)))
|
||||
|
||||
// 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)))
|
||||
|
||||
// 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)
|
||||
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)))
|
||||
|
||||
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)))
|
||||
|
||||
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)))
|
||||
|
||||
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)))
|
||||
|
||||
seg.recover(newProducerStateManager(), Optional.of(cache))
|
||||
|
|
@ -555,7 +556,7 @@ class LogSegmentTest {
|
|||
val offset = 40
|
||||
|
||||
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))
|
||||
|
||||
//Given two messages with a gap between them (e.g. mid offset compacted away)
|
||||
|
|
@ -576,7 +577,7 @@ class LogSegmentTest {
|
|||
@Test
|
||||
def testAppendFromFile(): Unit = {
|
||||
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)))
|
||||
|
||||
// create a log file in a separate directory to avoid conflicting with created segments
|
||||
|
|
|
|||
|
|
@ -24,7 +24,8 @@ import java.util.Properties
|
|||
import kafka.server.BrokerTopicStats
|
||||
import kafka.utils.TestUtils
|
||||
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.junit.jupiter.api.Assertions.{assertEquals, assertFalse}
|
||||
|
||||
|
|
@ -156,12 +157,12 @@ object LogTestUtils {
|
|||
new SimpleRecord(data, data)
|
||||
}
|
||||
|
||||
segment.append(MemoryRecords.withRecords(baseOffset, CompressionType.NONE, 0,
|
||||
segment.append(MemoryRecords.withRecords(baseOffset, Compression.NONE, 0,
|
||||
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 + 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)))
|
||||
// Need to create the offset files explicitly to avoid triggering segment recovery to truncate segment.
|
||||
Files.createFile(LogFileUtils.offsetIndexFile(logDir, baseOffset).toPath)
|
||||
|
|
@ -265,7 +266,7 @@ object LogTestUtils {
|
|||
val simpleRecords = (0 until numRecords).map { seq =>
|
||||
new SimpleRecord(s"$seq".getBytes)
|
||||
}
|
||||
val records = MemoryRecords.withRecords(CompressionType.NONE, simpleRecords: _*)
|
||||
val records = MemoryRecords.withRecords(Compression.NONE, simpleRecords: _*)
|
||||
log.appendAsLeader(records, leaderEpoch = 0)
|
||||
}
|
||||
|
||||
|
|
@ -288,10 +289,10 @@ object LogTestUtils {
|
|||
}
|
||||
|
||||
val records = if (isTransactional) {
|
||||
MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId,
|
||||
MemoryRecords.withTransactionalRecords(Compression.NONE, producerId,
|
||||
producerEpoch, sequence, simpleRecords: _*)
|
||||
} else {
|
||||
MemoryRecords.withIdempotentRecords(CompressionType.NONE, producerId,
|
||||
MemoryRecords.withIdempotentRecords(Compression.NONE, producerId,
|
||||
producerEpoch, sequence, simpleRecords: _*)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -20,6 +20,7 @@ import java.nio.ByteBuffer
|
|||
import java.util.concurrent.TimeUnit
|
||||
import kafka.server.{BrokerTopicStats, RequestLocal}
|
||||
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.record._
|
||||
import org.apache.kafka.common.utils.{PrimitiveRef, Time}
|
||||
|
|
@ -44,38 +45,39 @@ class LogValidatorTest {
|
|||
|
||||
@Test
|
||||
def testOnlyOneBatch(): Unit = {
|
||||
checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V0, CompressionType.GZIP, CompressionType.GZIP)
|
||||
checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V1, CompressionType.GZIP, CompressionType.GZIP)
|
||||
checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V2, CompressionType.GZIP, CompressionType.GZIP)
|
||||
checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V0, CompressionType.GZIP, CompressionType.NONE)
|
||||
checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V1, CompressionType.GZIP, CompressionType.NONE)
|
||||
checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V2, CompressionType.GZIP, CompressionType.NONE)
|
||||
checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE, CompressionType.NONE)
|
||||
checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE, CompressionType.GZIP)
|
||||
checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V0, Compression.gzip().build(), Compression.gzip().build())
|
||||
checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V1, Compression.gzip().build(), Compression.gzip().build())
|
||||
checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V2, Compression.gzip().build(), Compression.gzip().build())
|
||||
checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V0, Compression.gzip().build(), Compression.NONE)
|
||||
checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V1, Compression.gzip().build(), Compression.NONE)
|
||||
checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V2, Compression.gzip().build(), Compression.NONE)
|
||||
checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V2, Compression.NONE, Compression.NONE)
|
||||
checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V2, Compression.NONE, Compression.gzip().build())
|
||||
}
|
||||
|
||||
@Test
|
||||
def testAllowMultiBatch(): Unit = {
|
||||
checkAllowMultiBatch(RecordBatch.MAGIC_VALUE_V0, CompressionType.NONE, CompressionType.NONE)
|
||||
checkAllowMultiBatch(RecordBatch.MAGIC_VALUE_V1, CompressionType.NONE, CompressionType.NONE)
|
||||
checkAllowMultiBatch(RecordBatch.MAGIC_VALUE_V0, CompressionType.NONE, CompressionType.GZIP)
|
||||
checkAllowMultiBatch(RecordBatch.MAGIC_VALUE_V1, CompressionType.NONE, CompressionType.GZIP)
|
||||
checkAllowMultiBatch(RecordBatch.MAGIC_VALUE_V0, Compression.NONE, Compression.NONE)
|
||||
checkAllowMultiBatch(RecordBatch.MAGIC_VALUE_V1, Compression.NONE, Compression.NONE)
|
||||
checkAllowMultiBatch(RecordBatch.MAGIC_VALUE_V0, Compression.NONE, Compression.gzip().build())
|
||||
checkAllowMultiBatch(RecordBatch.MAGIC_VALUE_V1, Compression.NONE, Compression.gzip().build())
|
||||
}
|
||||
|
||||
@Test
|
||||
def testValidationOfBatchesWithNonSequentialInnerOffsets(): Unit = {
|
||||
def testMessageValidation(magicValue: Byte): Unit = {
|
||||
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
|
||||
// internal offsets by rewriting the batch.
|
||||
if (magicValue >= RecordBatch.MAGIC_VALUE_V2) {
|
||||
assertThrows(classOf[InvalidRecordException],
|
||||
() => validateMessages(invalidRecords, magicValue, CompressionType.GZIP, CompressionType.GZIP)
|
||||
() => validateMessages(invalidRecords, magicValue, CompressionType.GZIP, compression)
|
||||
)
|
||||
} 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))
|
||||
}
|
||||
}
|
||||
|
|
@ -87,23 +89,24 @@ class LogValidatorTest {
|
|||
|
||||
@Test
|
||||
def testMisMatchMagic(): Unit = {
|
||||
checkMismatchMagic(RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V1, CompressionType.GZIP)
|
||||
checkMismatchMagic(RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V0, CompressionType.GZIP)
|
||||
val compression: Compression = Compression.gzip().build()
|
||||
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],
|
||||
() => 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 = {
|
||||
validateMessages(createTwoBatchedRecords(magic, 0L, sourceCompressionType), magic, sourceCompressionType, targetCompressionType)
|
||||
private def checkAllowMultiBatch(magic: Byte, sourceCompression: Compression, targetCompression: Compression): Unit = {
|
||||
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],
|
||||
() => 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)
|
||||
assertTrue(meterCount(s"${BrokerTopicStats.InvalidMagicNumberRecordsPerSec}") > 0)
|
||||
|
|
@ -112,13 +115,13 @@ class LogValidatorTest {
|
|||
private def validateMessages(records: MemoryRecords,
|
||||
magic: Byte,
|
||||
sourceCompressionType: CompressionType,
|
||||
targetCompressionType: CompressionType): ValidationResult = {
|
||||
targetCompression: Compression): ValidationResult = {
|
||||
val mockTime = new MockTime(0L, 0L)
|
||||
new LogValidator(records,
|
||||
topicPartition,
|
||||
mockTime,
|
||||
sourceCompressionType,
|
||||
targetCompressionType,
|
||||
targetCompression,
|
||||
false,
|
||||
magic,
|
||||
TimestampType.CREATE_TIME,
|
||||
|
|
@ -150,13 +153,13 @@ class LogValidatorTest {
|
|||
private def checkLogAppendTimeNonCompressed(magic: Byte): Unit = {
|
||||
val mockTime = new MockTime
|
||||
// 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 validatedResults = new LogValidator(records,
|
||||
topicPartition,
|
||||
mockTime,
|
||||
CompressionType.NONE,
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
false,
|
||||
magic,
|
||||
TimestampType.LOG_APPEND_TIME,
|
||||
|
|
@ -197,15 +200,16 @@ class LogValidatorTest {
|
|||
}
|
||||
|
||||
private def checkLogAppendTimeWithRecompression(targetMagic: Byte): Unit = {
|
||||
val compression: Compression = Compression.gzip().build()
|
||||
val mockTime = new MockTime
|
||||
// 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(
|
||||
records,
|
||||
topicPartition,
|
||||
mockTime,
|
||||
CompressionType.GZIP,
|
||||
CompressionType.GZIP,
|
||||
compression,
|
||||
false,
|
||||
targetMagic,
|
||||
TimestampType.LOG_APPEND_TIME,
|
||||
|
|
@ -249,15 +253,16 @@ class LogValidatorTest {
|
|||
}
|
||||
|
||||
private def checkLogAppendTimeWithoutRecompression(magic: Byte): Unit = {
|
||||
val compression: Compression = Compression.gzip().build()
|
||||
val mockTime = new MockTime
|
||||
// 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(
|
||||
records,
|
||||
topicPartition,
|
||||
mockTime,
|
||||
CompressionType.GZIP,
|
||||
CompressionType.GZIP,
|
||||
compression,
|
||||
false,
|
||||
magic,
|
||||
TimestampType.LOG_APPEND_TIME,
|
||||
|
|
@ -315,7 +320,7 @@ class LogValidatorTest {
|
|||
}
|
||||
|
||||
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.LAST_OFFSET_DELTA_OFFSET, lastOffsetDelta)
|
||||
new LogValidator(
|
||||
|
|
@ -323,7 +328,7 @@ class LogValidatorTest {
|
|||
topicPartition,
|
||||
time,
|
||||
CompressionType.GZIP,
|
||||
CompressionType.GZIP,
|
||||
Compression.gzip().build(),
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V2,
|
||||
TimestampType.LOG_APPEND_TIME,
|
||||
|
|
@ -367,7 +372,7 @@ class LogValidatorTest {
|
|||
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: _*)
|
||||
|
||||
val offsetCounter = PrimitiveRef.ofLong(0)
|
||||
|
|
@ -375,7 +380,7 @@ class LogValidatorTest {
|
|||
topicPartition,
|
||||
time,
|
||||
CompressionType.NONE,
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
false,
|
||||
magic,
|
||||
TimestampType.CREATE_TIME,
|
||||
|
|
@ -452,7 +457,7 @@ class LogValidatorTest {
|
|||
(RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, false,
|
||||
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,
|
||||
new SimpleRecord(timestampSeq(0), "hello".getBytes),
|
||||
new SimpleRecord(timestampSeq(1), "there".getBytes),
|
||||
|
|
@ -465,7 +470,7 @@ class LogValidatorTest {
|
|||
topicPartition,
|
||||
time,
|
||||
CompressionType.NONE,
|
||||
CompressionType.GZIP,
|
||||
Compression.gzip().build(),
|
||||
false,
|
||||
magic,
|
||||
TimestampType.CREATE_TIME,
|
||||
|
|
@ -522,12 +527,13 @@ class LogValidatorTest {
|
|||
}
|
||||
|
||||
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,
|
||||
topicPartition,
|
||||
time,
|
||||
CompressionType.GZIP,
|
||||
CompressionType.GZIP,
|
||||
compression,
|
||||
false,
|
||||
toMagic,
|
||||
TimestampType.CREATE_TIME,
|
||||
|
|
@ -569,12 +575,13 @@ class LogValidatorTest {
|
|||
@Test
|
||||
def testCreateTimeUpConversionV1ToV2(): Unit = {
|
||||
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,
|
||||
topicPartition,
|
||||
time,
|
||||
CompressionType.GZIP,
|
||||
CompressionType.GZIP,
|
||||
compression,
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V2,
|
||||
TimestampType.CREATE_TIME,
|
||||
|
|
@ -631,14 +638,14 @@ class LogValidatorTest {
|
|||
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: _*)
|
||||
|
||||
val validatedResults = new LogValidator(records,
|
||||
topicPartition,
|
||||
time,
|
||||
CompressionType.GZIP,
|
||||
CompressionType.GZIP,
|
||||
Compression.gzip().build(),
|
||||
false,
|
||||
magic,
|
||||
TimestampType.CREATE_TIME,
|
||||
|
|
@ -690,13 +697,13 @@ class LogValidatorTest {
|
|||
def testInvalidCreateTimeNonCompressedV1(): Unit = {
|
||||
val now = System.currentTimeMillis()
|
||||
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, timestamp = now - 1001L,
|
||||
codec = CompressionType.NONE)
|
||||
codec = Compression.NONE)
|
||||
assertThrows(classOf[RecordValidationException], () => new LogValidator(
|
||||
records,
|
||||
topicPartition,
|
||||
time,
|
||||
CompressionType.NONE,
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V1,
|
||||
TimestampType.CREATE_TIME,
|
||||
|
|
@ -716,13 +723,13 @@ class LogValidatorTest {
|
|||
def testInvalidCreateTimeNonCompressedV2(): Unit = {
|
||||
val now = System.currentTimeMillis()
|
||||
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = now - 1001L,
|
||||
codec = CompressionType.NONE)
|
||||
codec = Compression.NONE)
|
||||
assertThrows(classOf[RecordValidationException], () => new LogValidator(
|
||||
records,
|
||||
topicPartition,
|
||||
time,
|
||||
CompressionType.NONE,
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V2,
|
||||
TimestampType.CREATE_TIME,
|
||||
|
|
@ -741,14 +748,15 @@ class LogValidatorTest {
|
|||
@Test
|
||||
def testInvalidCreateTimeCompressedV1(): Unit = {
|
||||
val now = System.currentTimeMillis()
|
||||
val compression: Compression = Compression.gzip().build()
|
||||
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, timestamp = now - 1001L,
|
||||
codec = CompressionType.GZIP)
|
||||
codec = compression)
|
||||
assertThrows(classOf[RecordValidationException], () => new LogValidator(
|
||||
records,
|
||||
topicPartition,
|
||||
time,
|
||||
CompressionType.GZIP,
|
||||
CompressionType.GZIP,
|
||||
compression,
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V1,
|
||||
TimestampType.CREATE_TIME,
|
||||
|
|
@ -767,14 +775,15 @@ class LogValidatorTest {
|
|||
@Test
|
||||
def testInvalidCreateTimeCompressedV2(): Unit = {
|
||||
val now = System.currentTimeMillis()
|
||||
val compression: Compression = Compression.gzip().build()
|
||||
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = now - 1001L,
|
||||
codec = CompressionType.GZIP)
|
||||
codec = compression)
|
||||
assertThrows(classOf[RecordValidationException], () => new LogValidator(
|
||||
records,
|
||||
topicPartition,
|
||||
time,
|
||||
CompressionType.GZIP,
|
||||
CompressionType.GZIP,
|
||||
compression,
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V1,
|
||||
TimestampType.CREATE_TIME,
|
||||
|
|
@ -792,14 +801,14 @@ class LogValidatorTest {
|
|||
|
||||
@Test
|
||||
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
|
||||
checkOffsets(records, 0)
|
||||
checkOffsets(new LogValidator(records,
|
||||
topicPartition,
|
||||
time,
|
||||
CompressionType.NONE,
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V0,
|
||||
TimestampType.CREATE_TIME,
|
||||
|
|
@ -815,14 +824,15 @@ class LogValidatorTest {
|
|||
|
||||
@Test
|
||||
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
|
||||
checkOffsets(records, 0)
|
||||
checkOffsets(new LogValidator(records,
|
||||
topicPartition,
|
||||
time,
|
||||
CompressionType.GZIP,
|
||||
CompressionType.GZIP,
|
||||
compression,
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V0,
|
||||
TimestampType.CREATE_TIME,
|
||||
|
|
@ -839,14 +849,14 @@ class LogValidatorTest {
|
|||
@Test
|
||||
def testRelativeOffsetAssignmentNonCompressedV1(): Unit = {
|
||||
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
|
||||
checkOffsets(records, 0)
|
||||
val messageWithOffset = new LogValidator(records,
|
||||
topicPartition,
|
||||
time,
|
||||
CompressionType.NONE,
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V1,
|
||||
TimestampType.CREATE_TIME,
|
||||
|
|
@ -864,14 +874,14 @@ class LogValidatorTest {
|
|||
@Test
|
||||
def testRelativeOffsetAssignmentNonCompressedV2(): Unit = {
|
||||
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
|
||||
checkOffsets(records, 0)
|
||||
val messageWithOffset = new LogValidator(records,
|
||||
topicPartition,
|
||||
time,
|
||||
CompressionType.NONE,
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V2,
|
||||
TimestampType.CREATE_TIME,
|
||||
|
|
@ -889,7 +899,8 @@ class LogValidatorTest {
|
|||
@Test
|
||||
def testRelativeOffsetAssignmentCompressedV1(): Unit = {
|
||||
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
|
||||
checkOffsets(records, 0)
|
||||
val compressedMessagesWithOffset = new LogValidator(
|
||||
|
|
@ -897,7 +908,7 @@ class LogValidatorTest {
|
|||
topicPartition,
|
||||
time,
|
||||
CompressionType.GZIP,
|
||||
CompressionType.GZIP,
|
||||
compression,
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V1,
|
||||
TimestampType.CREATE_TIME,
|
||||
|
|
@ -915,7 +926,8 @@ class LogValidatorTest {
|
|||
@Test
|
||||
def testRelativeOffsetAssignmentCompressedV2(): Unit = {
|
||||
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
|
||||
checkOffsets(records, 0)
|
||||
val compressedMessagesWithOffset = new LogValidator(
|
||||
|
|
@ -923,7 +935,7 @@ class LogValidatorTest {
|
|||
topicPartition,
|
||||
time,
|
||||
CompressionType.GZIP,
|
||||
CompressionType.GZIP,
|
||||
compression,
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V2,
|
||||
TimestampType.CREATE_TIME,
|
||||
|
|
@ -940,14 +952,14 @@ class LogValidatorTest {
|
|||
|
||||
@Test
|
||||
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)
|
||||
val offset = 1234567
|
||||
val validatedResults = new LogValidator(records,
|
||||
topicPartition,
|
||||
time,
|
||||
CompressionType.NONE,
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V1,
|
||||
TimestampType.LOG_APPEND_TIME,
|
||||
|
|
@ -966,14 +978,14 @@ class LogValidatorTest {
|
|||
|
||||
@Test
|
||||
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)
|
||||
val offset = 1234567
|
||||
val validatedResults = new LogValidator(records,
|
||||
topicPartition,
|
||||
time,
|
||||
CompressionType.NONE,
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V2,
|
||||
TimestampType.LOG_APPEND_TIME,
|
||||
|
|
@ -992,14 +1004,15 @@ class LogValidatorTest {
|
|||
|
||||
@Test
|
||||
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
|
||||
checkOffsets(records, 0)
|
||||
val validatedResults = new LogValidator(records,
|
||||
topicPartition,
|
||||
time,
|
||||
CompressionType.GZIP,
|
||||
CompressionType.GZIP,
|
||||
compression,
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V1,
|
||||
TimestampType.LOG_APPEND_TIME,
|
||||
|
|
@ -1018,14 +1031,15 @@ class LogValidatorTest {
|
|||
|
||||
@Test
|
||||
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
|
||||
checkOffsets(records, 0)
|
||||
val validatedResults = new LogValidator(records,
|
||||
topicPartition,
|
||||
time,
|
||||
CompressionType.GZIP,
|
||||
CompressionType.GZIP,
|
||||
compression,
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V2,
|
||||
TimestampType.LOG_APPEND_TIME,
|
||||
|
|
@ -1051,7 +1065,7 @@ class LogValidatorTest {
|
|||
topicPartition,
|
||||
time,
|
||||
CompressionType.NONE,
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
false,
|
||||
RecordBatch.CURRENT_MAGIC_VALUE,
|
||||
TimestampType.CREATE_TIME,
|
||||
|
|
@ -1074,7 +1088,7 @@ class LogValidatorTest {
|
|||
topicPartition,
|
||||
time,
|
||||
CompressionType.NONE,
|
||||
CompressionType.SNAPPY,
|
||||
Compression.snappy().build(),
|
||||
false,
|
||||
RecordBatch.CURRENT_MAGIC_VALUE,
|
||||
TimestampType.CREATE_TIME,
|
||||
|
|
@ -1096,13 +1110,13 @@ class LogValidatorTest {
|
|||
def testOffsetAssignmentAfterDownConversionV1ToV0NonCompressed(): Unit = {
|
||||
val offset = 1234567
|
||||
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(new LogValidator(records,
|
||||
topicPartition,
|
||||
time,
|
||||
CompressionType.NONE,
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V0,
|
||||
TimestampType.CREATE_TIME,
|
||||
|
|
@ -1120,13 +1134,14 @@ class LogValidatorTest {
|
|||
def testOffsetAssignmentAfterDownConversionV1ToV0Compressed(): Unit = {
|
||||
val offset = 1234567
|
||||
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(new LogValidator(records,
|
||||
topicPartition,
|
||||
time,
|
||||
CompressionType.GZIP,
|
||||
CompressionType.GZIP,
|
||||
compression,
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V0,
|
||||
TimestampType.CREATE_TIME,
|
||||
|
|
@ -1142,14 +1157,14 @@ class LogValidatorTest {
|
|||
|
||||
@Test
|
||||
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)
|
||||
val offset = 1234567
|
||||
checkOffsets(new LogValidator(records,
|
||||
topicPartition,
|
||||
time,
|
||||
CompressionType.NONE,
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V2,
|
||||
TimestampType.LOG_APPEND_TIME,
|
||||
|
|
@ -1165,14 +1180,15 @@ class LogValidatorTest {
|
|||
|
||||
@Test
|
||||
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
|
||||
checkOffsets(records, 0)
|
||||
checkOffsets(new LogValidator(records,
|
||||
topicPartition,
|
||||
time,
|
||||
CompressionType.GZIP,
|
||||
CompressionType.GZIP,
|
||||
compression,
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V2,
|
||||
TimestampType.LOG_APPEND_TIME,
|
||||
|
|
@ -1190,13 +1206,13 @@ class LogValidatorTest {
|
|||
def testOffsetAssignmentAfterDownConversionV2ToV1NonCompressed(): Unit = {
|
||||
val offset = 1234567
|
||||
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(new LogValidator(records,
|
||||
topicPartition,
|
||||
time,
|
||||
CompressionType.NONE,
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V1,
|
||||
TimestampType.CREATE_TIME,
|
||||
|
|
@ -1214,13 +1230,14 @@ class LogValidatorTest {
|
|||
def testOffsetAssignmentAfterDownConversionV2ToV1Compressed(): Unit = {
|
||||
val offset = 1234567
|
||||
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(new LogValidator(records,
|
||||
topicPartition,
|
||||
time,
|
||||
CompressionType.GZIP,
|
||||
CompressionType.GZIP,
|
||||
compression,
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V1,
|
||||
TimestampType.CREATE_TIME,
|
||||
|
|
@ -1240,13 +1257,13 @@ class LogValidatorTest {
|
|||
val producerId = 1344L
|
||||
val producerEpoch = 16.toShort
|
||||
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))
|
||||
assertThrows(classOf[UnsupportedForMessageFormatException], () => new LogValidator(records,
|
||||
topicPartition,
|
||||
time,
|
||||
CompressionType.GZIP,
|
||||
CompressionType.GZIP,
|
||||
Compression.gzip().build(),
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V1,
|
||||
TimestampType.CREATE_TIME,
|
||||
|
|
@ -1266,13 +1283,13 @@ class LogValidatorTest {
|
|||
val producerId = 1344L
|
||||
val producerEpoch = 16.toShort
|
||||
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))
|
||||
assertThrows(classOf[UnsupportedForMessageFormatException], () => new LogValidator(records,
|
||||
topicPartition,
|
||||
time,
|
||||
CompressionType.GZIP,
|
||||
CompressionType.GZIP,
|
||||
Compression.gzip().build(),
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V1,
|
||||
TimestampType.CREATE_TIME,
|
||||
|
|
@ -1290,13 +1307,13 @@ class LogValidatorTest {
|
|||
def testOffsetAssignmentAfterDownConversionV2ToV0NonCompressed(): Unit = {
|
||||
val offset = 1234567
|
||||
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(new LogValidator(records,
|
||||
topicPartition,
|
||||
time,
|
||||
CompressionType.NONE,
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V0,
|
||||
TimestampType.CREATE_TIME,
|
||||
|
|
@ -1314,13 +1331,14 @@ class LogValidatorTest {
|
|||
def testOffsetAssignmentAfterDownConversionV2ToV0Compressed(): Unit = {
|
||||
val offset = 1234567
|
||||
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(new LogValidator(records,
|
||||
topicPartition,
|
||||
time,
|
||||
CompressionType.GZIP,
|
||||
CompressionType.GZIP,
|
||||
compression,
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V0,
|
||||
TimestampType.CREATE_TIME,
|
||||
|
|
@ -1342,12 +1360,12 @@ class LogValidatorTest {
|
|||
topicPartition,
|
||||
time,
|
||||
CompressionType.GZIP,
|
||||
CompressionType.GZIP,
|
||||
Compression.gzip().build(),
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V0,
|
||||
TimestampType.CREATE_TIME,
|
||||
5000L,
|
||||
5000L,
|
||||
5000L,
|
||||
RecordBatch.NO_PARTITION_LEADER_EPOCH,
|
||||
AppendOrigin.CLIENT,
|
||||
MetadataVersion.latestTesting
|
||||
|
|
@ -1360,18 +1378,18 @@ class LogValidatorTest {
|
|||
|
||||
@Test
|
||||
def testCompressedBatchWithoutRecordsNotAllowed(): Unit = {
|
||||
testBatchWithoutRecordsNotAllowed(CompressionType.GZIP, CompressionType.GZIP)
|
||||
testBatchWithoutRecordsNotAllowed(CompressionType.GZIP, Compression.gzip().build())
|
||||
}
|
||||
|
||||
@Test
|
||||
def testZStdCompressedWithUnavailableIBPVersion(): Unit = {
|
||||
// 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,
|
||||
topicPartition,
|
||||
time,
|
||||
CompressionType.NONE,
|
||||
CompressionType.ZSTD,
|
||||
Compression.zstd().build(),
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V2,
|
||||
TimestampType.LOG_APPEND_TIME,
|
||||
|
|
@ -1387,26 +1405,27 @@ class LogValidatorTest {
|
|||
|
||||
@Test
|
||||
def testUncompressedBatchWithoutRecordsNotAllowed(): Unit = {
|
||||
testBatchWithoutRecordsNotAllowed(CompressionType.NONE, CompressionType.NONE)
|
||||
testBatchWithoutRecordsNotAllowed(CompressionType.NONE, Compression.NONE)
|
||||
}
|
||||
|
||||
@Test
|
||||
def testRecompressedBatchWithoutRecordsNotAllowed(): Unit = {
|
||||
testBatchWithoutRecordsNotAllowed(CompressionType.NONE, CompressionType.GZIP)
|
||||
testBatchWithoutRecordsNotAllowed(CompressionType.NONE, Compression.gzip().build())
|
||||
}
|
||||
|
||||
@Test
|
||||
def testInvalidTimestampExceptionHasBatchIndex(): Unit = {
|
||||
val now = System.currentTimeMillis()
|
||||
val compression: Compression = Compression.gzip().build()
|
||||
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = now - 1001L,
|
||||
codec = CompressionType.GZIP)
|
||||
codec = compression)
|
||||
val e = assertThrows(classOf[RecordValidationException],
|
||||
() => new LogValidator(
|
||||
records,
|
||||
topicPartition,
|
||||
time,
|
||||
CompressionType.GZIP,
|
||||
CompressionType.GZIP,
|
||||
compression,
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V1,
|
||||
TimestampType.CREATE_TIME,
|
||||
|
|
@ -1428,9 +1447,12 @@ class LogValidatorTest {
|
|||
@Test
|
||||
def testInvalidRecordExceptionHasBatchIndex(): Unit = {
|
||||
val e = assertThrows(classOf[RecordValidationException],
|
||||
() => validateMessages(recordsWithInvalidInnerMagic(
|
||||
RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V1, CompressionType.GZIP),
|
||||
RecordBatch.MAGIC_VALUE_V0, CompressionType.GZIP, CompressionType.GZIP)
|
||||
() => {
|
||||
val compression: Compression = Compression.gzip().build()
|
||||
validateMessages(recordsWithInvalidInnerMagic(
|
||||
RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V1, compression),
|
||||
RecordBatch.MAGIC_VALUE_V0, CompressionType.GZIP, compression)
|
||||
}
|
||||
)
|
||||
|
||||
assertTrue(e.invalidException.isInstanceOf[InvalidRecordException])
|
||||
|
|
@ -1442,12 +1464,13 @@ class LogValidatorTest {
|
|||
|
||||
@Test
|
||||
def testBatchWithInvalidRecordsAndInvalidTimestamp(): Unit = {
|
||||
val compression: Compression = Compression.gzip().build()
|
||||
val records = (0 until 5).map(id =>
|
||||
LegacyRecord.create(RecordBatch.MAGIC_VALUE_V0, 0L, null, id.toString.getBytes())
|
||||
)
|
||||
|
||||
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)
|
||||
var offset = 0
|
||||
|
||||
|
|
@ -1462,7 +1485,7 @@ class LogValidatorTest {
|
|||
|
||||
val e = assertThrows(classOf[RecordValidationException],
|
||||
() => 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,
|
||||
// InvalidTimestampException takes precedence
|
||||
|
|
@ -1477,15 +1500,16 @@ class LogValidatorTest {
|
|||
val timestampAfterMaxConfig = 1 * 60 * 60 * 1000L //1 hr
|
||||
val now = System.currentTimeMillis()
|
||||
val fiveMinutesBeforeThreshold = now - timestampBeforeMaxConfig - (5 * 60 * 1000L)
|
||||
val compression: Compression = Compression.gzip().build()
|
||||
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = fiveMinutesBeforeThreshold,
|
||||
codec = CompressionType.GZIP)
|
||||
codec = compression)
|
||||
val e = assertThrows(classOf[RecordValidationException],
|
||||
() => new LogValidator(
|
||||
records,
|
||||
topicPartition,
|
||||
time,
|
||||
CompressionType.GZIP,
|
||||
CompressionType.GZIP,
|
||||
compression,
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V2,
|
||||
TimestampType.CREATE_TIME,
|
||||
|
|
@ -1511,15 +1535,16 @@ class LogValidatorTest {
|
|||
val timestampAfterMaxConfig = 1 * 60 * 60 * 1000L //1 hr
|
||||
val now = System.currentTimeMillis()
|
||||
val fiveMinutesAfterThreshold = now + timestampAfterMaxConfig + (5 * 60 * 1000L)
|
||||
val compression: Compression = Compression.gzip().build()
|
||||
val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = fiveMinutesAfterThreshold,
|
||||
codec = CompressionType.GZIP)
|
||||
codec = compression)
|
||||
val e = assertThrows(classOf[RecordValidationException],
|
||||
() => new LogValidator(
|
||||
records,
|
||||
topicPartition,
|
||||
time,
|
||||
CompressionType.GZIP,
|
||||
CompressionType.GZIP,
|
||||
compression,
|
||||
false,
|
||||
RecordBatch.MAGIC_VALUE_V2,
|
||||
TimestampType.CREATE_TIME,
|
||||
|
|
@ -1538,7 +1563,80 @@ class LogValidatorTest {
|
|||
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 (producerId, producerEpoch, baseSequence, isTransactional, partitionLeaderEpoch) =
|
||||
(1324L, 10.toShort, 984, true, 40)
|
||||
|
|
@ -1568,18 +1666,26 @@ class LogValidatorTest {
|
|||
|
||||
private def createRecords(magicValue: Byte,
|
||||
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 builder = MemoryRecords.builder(buf, magicValue, codec, TimestampType.CREATE_TIME, 0L)
|
||||
builder.appendWithOffset(0, timestamp, null, "hello".getBytes)
|
||||
builder.appendWithOffset(1, timestamp, null, "there".getBytes)
|
||||
builder.appendWithOffset(2, timestamp, null, "beautiful".getBytes)
|
||||
records.indices.foreach { offset =>
|
||||
builder.appendWithOffset(offset, timestamp, null, records(offset))
|
||||
}
|
||||
builder.build()
|
||||
}
|
||||
|
||||
private def createNonIncreasingOffsetRecords(magicValue: Byte,
|
||||
timestamp: Long = RecordBatch.NO_TIMESTAMP,
|
||||
codec: CompressionType = CompressionType.NONE): MemoryRecords = {
|
||||
codec: Compression = Compression.NONE): MemoryRecords = {
|
||||
val buf = ByteBuffer.allocate(512)
|
||||
val builder = MemoryRecords.builder(buf, magicValue, codec, TimestampType.CREATE_TIME, 0L)
|
||||
builder.appendWithOffset(0, timestamp, null, "hello".getBytes)
|
||||
|
|
@ -1588,9 +1694,7 @@ class LogValidatorTest {
|
|||
builder.build()
|
||||
}
|
||||
|
||||
private def createTwoBatchedRecords(magicValue: Byte,
|
||||
timestamp: Long,
|
||||
codec: CompressionType): MemoryRecords = {
|
||||
private def createTwoBatchedRecords(magicValue: Byte, codec: Compression): MemoryRecords = {
|
||||
val buf = ByteBuffer.allocate(2048)
|
||||
var builder = MemoryRecords.builder(buf, magicValue, codec, TimestampType.CREATE_TIME, 0L)
|
||||
builder.append(10L, "1".getBytes(), "a".getBytes())
|
||||
|
|
@ -1615,7 +1719,7 @@ class LogValidatorTest {
|
|||
}
|
||||
|
||||
private def recordsWithNonSequentialInnerOffsets(magicValue: Byte,
|
||||
compression: CompressionType,
|
||||
compression: Compression,
|
||||
numRecords: Int): MemoryRecords = {
|
||||
val records = (0 until numRecords).map { id =>
|
||||
new SimpleRecord(id.toString.getBytes)
|
||||
|
|
@ -1633,7 +1737,7 @@ class LogValidatorTest {
|
|||
|
||||
private def recordsWithInvalidInnerMagic(batchMagicValue: Byte,
|
||||
recordMagicValue: Byte,
|
||||
codec: CompressionType): MemoryRecords = {
|
||||
codec: Compression): MemoryRecords = {
|
||||
val records = (0 until 20).map(id =>
|
||||
LegacyRecord.create(recordMagicValue,
|
||||
RecordBatch.NO_TIMESTAMP,
|
||||
|
|
|
|||
|
|
@ -21,6 +21,7 @@ import kafka.common.{OffsetsOutOfOrderException, UnexpectedAppendOffsetException
|
|||
import kafka.log.remote.RemoteLogManager
|
||||
import kafka.server.{BrokerTopicStats, KafkaConfig}
|
||||
import kafka.utils._
|
||||
import org.apache.kafka.common.compress.Compression
|
||||
import org.apache.kafka.common.config.TopicConfig
|
||||
import org.apache.kafka.common.{InvalidRecordException, TopicPartition, Uuid}
|
||||
import org.apache.kafka.common.errors._
|
||||
|
|
@ -200,7 +201,7 @@ class UnifiedLogTest {
|
|||
)), leaderEpoch = 0)
|
||||
|
||||
log.appendAsLeader(MemoryRecords.withTransactionalRecords(
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
producerId,
|
||||
producerEpoch,
|
||||
sequence,
|
||||
|
|
@ -243,7 +244,7 @@ class UnifiedLogTest {
|
|||
)), leaderEpoch = 0)
|
||||
|
||||
log.appendAsLeader(MemoryRecords.withTransactionalRecords(
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
producerId,
|
||||
producerEpoch,
|
||||
sequence,
|
||||
|
|
@ -567,7 +568,7 @@ class UnifiedLogTest {
|
|||
|
||||
val numSegments = log.numberOfSegments
|
||||
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.")
|
||||
}
|
||||
|
||||
|
|
@ -1205,7 +1206,7 @@ class UnifiedLogTest {
|
|||
val seq = 0
|
||||
|
||||
// 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("bar".getBytes),
|
||||
new SimpleRecord("baz".getBytes))
|
||||
|
|
@ -1328,22 +1329,22 @@ class UnifiedLogTest {
|
|||
val epoch: Short = 0
|
||||
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)
|
||||
builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
|
||||
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)
|
||||
builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
|
||||
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)
|
||||
builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
|
||||
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)
|
||||
builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
|
||||
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
|
||||
// a batch with matching sequence numbers, but valid increasing offsets
|
||||
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)))
|
||||
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)))
|
||||
|
||||
// Ensure that even the duplicate sequences are accepted on the follower.
|
||||
|
|
@ -1399,31 +1400,31 @@ class UnifiedLogTest {
|
|||
val buffer = ByteBuffer.allocate(512)
|
||||
|
||||
// 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)
|
||||
builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
|
||||
builder.close()
|
||||
|
||||
// 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)
|
||||
builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
|
||||
builder.close()
|
||||
|
||||
// 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)
|
||||
builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
|
||||
builder.close()
|
||||
|
||||
// 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)
|
||||
builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
|
||||
builder.close()
|
||||
|
||||
// // 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)
|
||||
builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
|
||||
builder.close()
|
||||
|
|
@ -1578,7 +1579,7 @@ class UnifiedLogTest {
|
|||
|
||||
// now test the case that we give the offsets and use non-sequential offsets
|
||||
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) {
|
||||
val idx = messageIds.indexWhere(_ >= i)
|
||||
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
|
||||
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) {
|
||||
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
|
||||
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) {
|
||||
assertEquals(MemoryRecords.EMPTY, LogTestUtils.readLog(log, i, maxLength = 0, minOneMessage = false).records)
|
||||
|
|
@ -1752,8 +1753,8 @@ class UnifiedLogTest {
|
|||
val log = createLog(logDir, logConfig)
|
||||
|
||||
/* 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(CompressionType.GZIP, new SimpleRecord("alpha".getBytes), new SimpleRecord("beta".getBytes)), leaderEpoch = 0)
|
||||
log.appendAsLeader(MemoryRecords.withRecords(Compression.gzip().build(), new SimpleRecord("hello".getBytes), new SimpleRecord("there".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
|
||||
|
||||
|
|
@ -1807,7 +1808,7 @@ class UnifiedLogTest {
|
|||
*/
|
||||
@Test
|
||||
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
|
||||
val configSegmentSize = messageSet.sizeInBytes - 1
|
||||
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 unkeyedMessage = new SimpleRecord("this message does not have a key".getBytes)
|
||||
|
||||
val messageSetWithUnkeyedMessage = MemoryRecords.withRecords(CompressionType.NONE, unkeyedMessage, keyedMessage)
|
||||
val messageSetWithOneUnkeyedMessage = MemoryRecords.withRecords(CompressionType.NONE, unkeyedMessage)
|
||||
val messageSetWithCompressedKeyedMessage = MemoryRecords.withRecords(CompressionType.GZIP, keyedMessage)
|
||||
val messageSetWithCompressedUnkeyedMessage = MemoryRecords.withRecords(CompressionType.GZIP, keyedMessage, unkeyedMessage)
|
||||
val messageSetWithUnkeyedMessage = MemoryRecords.withRecords(Compression.NONE, unkeyedMessage, keyedMessage)
|
||||
val messageSetWithOneUnkeyedMessage = MemoryRecords.withRecords(Compression.NONE, unkeyedMessage)
|
||||
val messageSetWithCompressedKeyedMessage = MemoryRecords.withRecords(Compression.gzip().build(), keyedMessage)
|
||||
val messageSetWithCompressedUnkeyedMessage = MemoryRecords.withRecords(Compression.gzip().build(), keyedMessage, unkeyedMessage)
|
||||
|
||||
val messageSetWithKeyedMessage = MemoryRecords.withRecords(CompressionType.NONE, keyedMessage)
|
||||
val messageSetWithKeyedMessages = MemoryRecords.withRecords(CompressionType.NONE, keyedMessage, anotherKeyedMessage)
|
||||
val messageSetWithKeyedMessage = MemoryRecords.withRecords(Compression.NONE, keyedMessage)
|
||||
val messageSetWithKeyedMessages = MemoryRecords.withRecords(Compression.NONE, keyedMessage, anotherKeyedMessage)
|
||||
|
||||
val logConfig = LogTestUtils.createLogConfig(cleanupPolicy = TopicConfig.CLEANUP_POLICY_COMPACT)
|
||||
val log = createLog(logDir, logConfig)
|
||||
|
|
@ -1872,8 +1873,8 @@ class UnifiedLogTest {
|
|||
*/
|
||||
@Test
|
||||
def testMessageSizeCheck(): Unit = {
|
||||
val first = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("You".getBytes), new SimpleRecord("bethe".getBytes))
|
||||
val second = MemoryRecords.withRecords(CompressionType.NONE,
|
||||
val first = MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("You".getBytes), new SimpleRecord("bethe".getBytes))
|
||||
val second = MemoryRecords.withRecords(Compression.NONE,
|
||||
new SimpleRecord("change (I need more bytes)... blah blah blah.".getBytes),
|
||||
new SimpleRecord("More padding boo hoo".getBytes))
|
||||
|
||||
|
|
@ -1891,9 +1892,9 @@ class UnifiedLogTest {
|
|||
|
||||
@Test
|
||||
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))
|
||||
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("More padding boo hoo".getBytes))
|
||||
|
||||
|
|
@ -1908,7 +1909,7 @@ class UnifiedLogTest {
|
|||
def testLogFlushesPartitionMetadataOnAppend(): Unit = {
|
||||
val logConfig = LogTestUtils.createLogConfig()
|
||||
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()
|
||||
log.partitionMetadataFile.get.record(topicId)
|
||||
|
|
@ -2000,7 +2001,7 @@ class UnifiedLogTest {
|
|||
val log = createLog(logDir, logConfig)
|
||||
|
||||
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)
|
||||
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 buffer = ByteBuffer.allocate(512)
|
||||
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(),
|
||||
1L, 0, 0, false, 0)
|
||||
builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
|
||||
|
|
@ -2385,15 +2386,16 @@ class UnifiedLogTest {
|
|||
def testAppendBelowExpectedOffsetThrowsException(): Unit = {
|
||||
val log = createLog(logDir, new LogConfig(new Properties))
|
||||
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 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))
|
||||
assertThrows(classOf[UnexpectedAppendOffsetException],
|
||||
() => log.appendAsFollower(invalidRecord),
|
||||
() => s"Magic=$magic, compressionType=$compression")
|
||||
() => s"Magic=$magic, compressionType=$compressionType")
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -2407,23 +2409,23 @@ class UnifiedLogTest {
|
|||
val firstOffset = 4L
|
||||
val magicVals = Seq(RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V2)
|
||||
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),
|
||||
new SimpleRecord("k2".getBytes, "v2".getBytes),
|
||||
new SimpleRecord("k3".getBytes, "v3".getBytes)),
|
||||
magicValue = magic, codec = compression,
|
||||
magicValue = magic, codec = Compression.of(compressionType).build(),
|
||||
baseOffset = firstOffset)
|
||||
|
||||
val exception = assertThrows(classOf[UnexpectedAppendOffsetException], () => log.appendAsFollower(records = batch))
|
||||
assertEquals(firstOffset, exception.firstOffset, s"Magic=$magic, compressionType=$compression, UnexpectedAppendOffsetException#firstOffset")
|
||||
assertEquals(firstOffset + 2, exception.lastOffset, s"Magic=$magic, compressionType=$compression, UnexpectedAppendOffsetException#lastOffset")
|
||||
assertEquals(firstOffset, exception.firstOffset, s"Magic=$magic, compressionType=$compressionType, UnexpectedAppendOffsetException#firstOffset")
|
||||
assertEquals(firstOffset + 2, exception.lastOffset, s"Magic=$magic, compressionType=$compressionType, UnexpectedAppendOffsetException#lastOffset")
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
def testAppendWithNoTimestamp(): Unit = {
|
||||
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)
|
||||
}
|
||||
|
||||
|
|
@ -2588,9 +2590,9 @@ class UnifiedLogTest {
|
|||
// When we split the overflowed segment, only one new segment will be created.
|
||||
|
||||
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))
|
||||
val batch2 = MemoryRecords.withRecords(overflowOffset + 1, CompressionType.NONE, 0,
|
||||
val batch2 = MemoryRecords.withRecords(overflowOffset + 1, Compression.NONE, 0,
|
||||
new SimpleRecord("b".getBytes))
|
||||
|
||||
testDegenerateSplitSegmentWithOverflow(segmentBaseOffset = 0L, List(batch1, batch2))
|
||||
|
|
@ -2602,7 +2604,7 @@ class UnifiedLogTest {
|
|||
// the first offset of the batch is valid, but the last overflows.
|
||||
|
||||
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("b".getBytes),
|
||||
new SimpleRecord("c".getBytes))
|
||||
|
|
@ -2869,7 +2871,7 @@ class UnifiedLogTest {
|
|||
//When appending messages as a leader (i.e. assignOffsets = true)
|
||||
for (record <- records)
|
||||
log.appendAsLeader(
|
||||
MemoryRecords.withRecords(CompressionType.NONE, record),
|
||||
MemoryRecords.withRecords(Compression.NONE, record),
|
||||
leaderEpoch = epoch
|
||||
)
|
||||
|
||||
|
|
@ -2887,7 +2889,7 @@ class UnifiedLogTest {
|
|||
|
||||
//Given each message has an offset & epoch, as msgs from leader would
|
||||
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 =>
|
||||
record.setPartitionLeaderEpoch(42)
|
||||
record.setLastOffset(i)
|
||||
|
|
@ -3008,7 +3010,7 @@ class UnifiedLogTest {
|
|||
val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024 * 5)
|
||||
val log = createLog(logDir, logConfig)
|
||||
|
||||
val records = MemoryRecords.withRecords(CompressionType.NONE,
|
||||
val records = MemoryRecords.withRecords(Compression.NONE,
|
||||
new SimpleRecord("foo".getBytes),
|
||||
new SimpleRecord("bar".getBytes),
|
||||
new SimpleRecord("baz".getBytes))
|
||||
|
|
@ -3027,7 +3029,7 @@ class UnifiedLogTest {
|
|||
var seq = 0
|
||||
|
||||
// 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("bar".getBytes),
|
||||
new SimpleRecord("baz".getBytes))
|
||||
|
|
@ -3037,7 +3039,7 @@ class UnifiedLogTest {
|
|||
|
||||
// add more transactional records
|
||||
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)
|
||||
|
||||
// LSO should not have changed
|
||||
|
|
@ -3406,7 +3408,7 @@ class UnifiedLogTest {
|
|||
val log = createLog(logDir, logConfig)
|
||||
|
||||
// append a few records
|
||||
appendAsFollower(log, MemoryRecords.withRecords(CompressionType.NONE,
|
||||
appendAsFollower(log, MemoryRecords.withRecords(Compression.NONE,
|
||||
new SimpleRecord("a".getBytes),
|
||||
new SimpleRecord("b".getBytes),
|
||||
new SimpleRecord("c".getBytes)), 5)
|
||||
|
|
@ -3436,20 +3438,20 @@ class UnifiedLogTest {
|
|||
val seq2 = 0
|
||||
|
||||
// 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("b".getBytes),
|
||||
new SimpleRecord("c".getBytes)), leaderEpoch = 0)
|
||||
assertEquals(Some(firstAppendInfo.firstOffset), log.firstUnstableOffset)
|
||||
|
||||
// mix in some non-transactional data
|
||||
log.appendAsLeader(MemoryRecords.withRecords(CompressionType.NONE,
|
||||
log.appendAsLeader(MemoryRecords.withRecords(Compression.NONE,
|
||||
new SimpleRecord("g".getBytes),
|
||||
new SimpleRecord("h".getBytes),
|
||||
new SimpleRecord("i".getBytes)), leaderEpoch = 0)
|
||||
|
||||
// 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("e".getBytes),
|
||||
new SimpleRecord("f".getBytes)), leaderEpoch = 0)
|
||||
|
|
@ -3478,7 +3480,7 @@ class UnifiedLogTest {
|
|||
val epoch = 5.toShort
|
||||
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("b".getBytes),
|
||||
new SimpleRecord("c".getBytes))
|
||||
|
|
@ -3491,7 +3493,7 @@ class UnifiedLogTest {
|
|||
|
||||
// this write should spill to the second segment
|
||||
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("e".getBytes),
|
||||
new SimpleRecord("f".getBytes)), leaderEpoch = 0)
|
||||
|
|
@ -3841,7 +3843,7 @@ class UnifiedLogTest {
|
|||
assertFalse(log.verificationGuard(producerId).verify(VerificationGuard.SENTINEL))
|
||||
|
||||
val idempotentRecords = MemoryRecords.withIdempotentRecords(
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
producerId,
|
||||
producerEpoch,
|
||||
sequence,
|
||||
|
|
@ -3854,7 +3856,7 @@ class UnifiedLogTest {
|
|||
sequence = sequence + 2
|
||||
|
||||
val transactionalRecords = MemoryRecords.withTransactionalRecords(
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
producerId,
|
||||
producerEpoch,
|
||||
sequence,
|
||||
|
|
@ -3939,7 +3941,7 @@ class UnifiedLogTest {
|
|||
producerStateManagerConfig.setTransactionVerificationEnabled(false)
|
||||
|
||||
val transactionalRecords = MemoryRecords.withTransactionalRecords(
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
producerId,
|
||||
producerEpoch,
|
||||
0,
|
||||
|
|
@ -3965,7 +3967,7 @@ class UnifiedLogTest {
|
|||
producerStateManagerConfig.setTransactionVerificationEnabled(true)
|
||||
|
||||
val transactionalRecords = MemoryRecords.withTransactionalRecords(
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
producerId,
|
||||
producerEpoch,
|
||||
sequence,
|
||||
|
|
@ -3997,7 +3999,7 @@ class UnifiedLogTest {
|
|||
assertEquals(VerificationGuard.SENTINEL, log.verificationGuard(producerId))
|
||||
|
||||
val transactionalRecords = MemoryRecords.withTransactionalRecords(
|
||||
CompressionType.NONE,
|
||||
Compression.NONE,
|
||||
producerId,
|
||||
producerEpoch,
|
||||
sequence,
|
||||
|
|
@ -4214,7 +4216,7 @@ class UnifiedLogTest {
|
|||
leaderEpoch: Int = 0): (Long, Int) => Unit = {
|
||||
var sequence = 0
|
||||
(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)
|
||||
for (seq <- sequence until sequence + numRecords) {
|
||||
val record = new SimpleRecord(s"$seq".getBytes)
|
||||
|
|
@ -4238,7 +4240,7 @@ class UnifiedLogTest {
|
|||
}
|
||||
|
||||
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 =>
|
||||
builder.append(new SimpleRecord(s"$seq".getBytes))
|
||||
}
|
||||
|
|
|
|||
|
|
@ -161,13 +161,14 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
|
|||
assertEquals(IBP_3_0_IV1, log.config.messageFormatVersion)
|
||||
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")
|
||||
// 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)
|
||||
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")
|
||||
assertEquals(IBP_3_0_IV1, log.config.messageFormatVersion)
|
||||
assertEquals(RecordVersion.V2, log.config.recordVersion)
|
||||
|
|
|
|||
|
|
@ -21,15 +21,15 @@ import java.io.{DataInputStream, DataOutputStream}
|
|||
import java.net.Socket
|
||||
import java.nio.ByteBuffer
|
||||
import java.util.Collections
|
||||
|
||||
import kafka.integration.KafkaServerTestHarness
|
||||
import kafka.network.SocketServer
|
||||
import kafka.utils._
|
||||
import org.apache.kafka.common.compress.Compression
|
||||
import org.apache.kafka.common.message.ProduceRequestData
|
||||
import org.apache.kafka.common.network.ListenerName
|
||||
import org.apache.kafka.common.protocol.types.Type
|
||||
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.security.auth.SecurityProtocol
|
||||
import org.apache.kafka.common.utils.ByteUtils
|
||||
|
|
@ -135,7 +135,7 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness {
|
|||
.setName(topicPartition.topic()).setPartitionData(Collections.singletonList(
|
||||
new ProduceRequestData.PartitionProduceData()
|
||||
.setIndex(topicPartition.partition())
|
||||
.setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("message".getBytes))))))
|
||||
.setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("message".getBytes))))))
|
||||
.iterator))
|
||||
.setAcks(1.toShort)
|
||||
.setTimeoutMs(10000)
|
||||
|
|
|
|||
|
|
@ -16,10 +16,10 @@
|
|||
*/
|
||||
package kafka.server
|
||||
|
||||
import org.apache.kafka.common.compress.Compression
|
||||
import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid}
|
||||
import org.apache.kafka.common.message.FetchResponseData
|
||||
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.SimpleRecord
|
||||
import org.apache.kafka.common.requests.FetchMetadata.{FINAL_EPOCH, INVALID_SESSION_ID}
|
||||
|
|
@ -1854,7 +1854,7 @@ class FetchSessionTest {
|
|||
.setHighWatermark(60)
|
||||
.setLastStableOffset(50)
|
||||
.setLogStartOffset(0)
|
||||
.setRecords(MemoryRecords.withRecords(CompressionType.NONE,
|
||||
.setRecords(MemoryRecords.withRecords(Compression.NONE,
|
||||
new SimpleRecord(100, null))))
|
||||
respData3.put(tp3, new FetchResponseData.PartitionData()
|
||||
.setPartitionIndex(tp3.topicPartition.partition)
|
||||
|
|
|
|||
|
|
@ -20,13 +20,14 @@ package kafka.server
|
|||
import kafka.cluster.BrokerEndPoint
|
||||
import kafka.server.InitialFetchState
|
||||
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._
|
||||
|
||||
object FetcherThreadTestUtils {
|
||||
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
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -31,6 +31,7 @@ import org.apache.kafka.clients.admin.AlterConfigOp.OpType
|
|||
import org.apache.kafka.clients.admin.{AlterConfigOp, ConfigEntry}
|
||||
import org.apache.kafka.common._
|
||||
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.Type.{BROKER, BROKER_LOGGER}
|
||||
import org.apache.kafka.common.errors.{ClusterAuthorizationException, UnsupportedVersionException}
|
||||
|
|
@ -2474,7 +2475,7 @@ class KafkaApisTest extends Logging {
|
|||
.setName(tp.topic).setPartitionData(Collections.singletonList(
|
||||
new ProduceRequestData.PartitionProduceData()
|
||||
.setIndex(tp.partition)
|
||||
.setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes))))))
|
||||
.setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("test".getBytes))))))
|
||||
.iterator))
|
||||
.setAcks(1.toShort)
|
||||
.setTimeoutMs(5000))
|
||||
|
|
@ -2536,7 +2537,7 @@ class KafkaApisTest extends Logging {
|
|||
.setName(tp.topic).setPartitionData(Collections.singletonList(
|
||||
new ProduceRequestData.PartitionProduceData()
|
||||
.setIndex(tp.partition)
|
||||
.setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes))))))
|
||||
.setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("test".getBytes))))))
|
||||
.iterator))
|
||||
.setAcks(1.toShort)
|
||||
.setTimeoutMs(5000))
|
||||
|
|
@ -2601,7 +2602,7 @@ class KafkaApisTest extends Logging {
|
|||
.setName(tp.topic).setPartitionData(Collections.singletonList(
|
||||
new ProduceRequestData.PartitionProduceData()
|
||||
.setIndex(tp.partition)
|
||||
.setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes))))))
|
||||
.setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("test".getBytes))))))
|
||||
.iterator))
|
||||
.setAcks(1.toShort)
|
||||
.setTimeoutMs(5000))
|
||||
|
|
@ -2665,7 +2666,7 @@ class KafkaApisTest extends Logging {
|
|||
.setName(tp.topic).setPartitionData(Collections.singletonList(
|
||||
new ProduceRequestData.PartitionProduceData()
|
||||
.setIndex(tp.partition)
|
||||
.setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes))))))
|
||||
.setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("test".getBytes))))))
|
||||
.iterator))
|
||||
.setAcks(1.toShort)
|
||||
.setTimeoutMs(5000))
|
||||
|
|
@ -2728,7 +2729,7 @@ class KafkaApisTest extends Logging {
|
|||
.setName(tp.topic).setPartitionData(Collections.singletonList(
|
||||
new ProduceRequestData.PartitionProduceData()
|
||||
.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))
|
||||
.setAcks(1.toShort)
|
||||
.setTransactionalId(transactionalId)
|
||||
|
|
@ -4203,7 +4204,7 @@ class KafkaApisTest extends Logging {
|
|||
any[Seq[(TopicIdPartition, FetchPartitionData)] => Unit]()
|
||||
)).thenAnswer(invocation => {
|
||||
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)))
|
||||
callback(Seq(tidp -> new FetchPartitionData(Errors.NONE, hw, 0, records,
|
||||
Optional.empty(), OptionalLong.empty(), Optional.empty(), OptionalInt.empty(), false)))
|
||||
|
|
@ -5566,7 +5567,7 @@ class KafkaApisTest extends Logging {
|
|||
val fetchFromFollower = buildRequest(new FetchRequest.Builder(
|
||||
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)))
|
||||
when(replicaManager.fetchMessages(
|
||||
any[FetchParams],
|
||||
|
|
@ -6310,7 +6311,7 @@ class KafkaApisTest extends Logging {
|
|||
.setHighWatermark(105)
|
||||
.setLastStableOffset(105)
|
||||
.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)
|
||||
|
||||
data.foreach{case (tp, _) =>
|
||||
|
|
|
|||
|
|
@ -30,6 +30,7 @@ import org.apache.kafka.common.metrics.Sensor
|
|||
import org.apache.kafka.common.network.ListenerName
|
||||
import org.apache.kafka.common.record.{CompressionType, Records}
|
||||
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.Group.GroupType
|
||||
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
|
||||
|
|
@ -691,7 +692,6 @@ class KafkaConfigTest {
|
|||
def testDefaultCompressionType(): Unit = {
|
||||
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
|
||||
val serverConfig = KafkaConfig.fromProps(props)
|
||||
|
||||
assertEquals(serverConfig.compressionType, "producer")
|
||||
}
|
||||
|
||||
|
|
@ -700,7 +700,6 @@ class KafkaConfigTest {
|
|||
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
|
||||
props.setProperty("compression.type", "gzip")
|
||||
val serverConfig = KafkaConfig.fromProps(props)
|
||||
|
||||
assertEquals(serverConfig.compressionType, "gzip")
|
||||
}
|
||||
|
||||
|
|
@ -711,6 +710,30 @@ class KafkaConfigTest {
|
|||
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
|
||||
def testInvalidInterBrokerSecurityProtocol(): Unit = {
|
||||
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_REPORTER_CLASSES_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
|
||||
//SSL Configs
|
||||
case KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG =>
|
||||
|
|
@ -1078,6 +1106,12 @@ class KafkaConfigTest {
|
|||
assertDynamic(kafkaConfigProp, TopicConfig.CLEANUP_POLICY_COMPACT, () => config.logCleanupPolicy)
|
||||
case TopicConfig.COMPRESSION_TYPE_CONFIG =>
|
||||
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 =>
|
||||
assertDynamic(kafkaConfigProp, 10000, () => config.logSegmentBytes)
|
||||
case TopicConfig.SEGMENT_MS_CONFIG =>
|
||||
|
|
|
|||
|
|
@ -21,6 +21,7 @@ import java.nio.ByteBuffer
|
|||
import java.util.{Collections, Properties}
|
||||
import kafka.utils.TestUtils
|
||||
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.message.ProduceRequestData
|
||||
import org.apache.kafka.common.protocol.Errors
|
||||
|
|
@ -74,10 +75,10 @@ class ProduceRequestTest extends BaseRequestTest {
|
|||
assertTrue(partitionProduceResponse.recordErrors.isEmpty)
|
||||
}
|
||||
|
||||
sendAndCheck(MemoryRecords.withRecords(CompressionType.NONE,
|
||||
sendAndCheck(MemoryRecords.withRecords(Compression.NONE,
|
||||
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(), "key2".getBytes, "value2".getBytes)), 1)
|
||||
}
|
||||
|
|
@ -92,7 +93,7 @@ class ProduceRequestTest extends BaseRequestTest {
|
|||
val partitionToLeader = TestUtils.createTopic(zkClient, topic, 1, 1, servers, topicConfig)
|
||||
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 builder = MemoryRecords.builder(buf, magicValue, codec, TimestampType.CREATE_TIME, 0L)
|
||||
builder.appendWithOffset(0, timestamp, null, "hello".getBytes)
|
||||
|
|
@ -101,7 +102,7 @@ class ProduceRequestTest extends BaseRequestTest {
|
|||
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 produceResponse = sendProduceRequest(leader, ProduceRequest.forCurrentMagic(new ProduceRequestData()
|
||||
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
|
||||
|
|
@ -143,7 +144,7 @@ class ProduceRequestTest extends BaseRequestTest {
|
|||
val nonReplicaId = nonReplicaOpt.get.config.brokerId
|
||||
|
||||
// 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 produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData()
|
||||
.setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
|
||||
|
|
@ -177,7 +178,7 @@ class ProduceRequestTest extends BaseRequestTest {
|
|||
def testCorruptLz4ProduceRequest(quorum: String): Unit = {
|
||||
val (partition, leader) = createTopicAndFindPartitionWithLeader("topic")
|
||||
val timestamp = 1000000
|
||||
val memoryRecords = MemoryRecords.withRecords(CompressionType.LZ4,
|
||||
val memoryRecords = MemoryRecords.withRecords(Compression.lz4().build(),
|
||||
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
|
||||
val lz4ChecksumOffset = 6
|
||||
|
|
@ -218,7 +219,7 @@ class ProduceRequestTest extends BaseRequestTest {
|
|||
topicConfig.setProperty(TopicConfig.COMPRESSION_TYPE_CONFIG, BrokerCompressionType.ZSTD.name)
|
||||
val partitionToLeader = createTopic(topic, topicConfig = topicConfig)
|
||||
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))
|
||||
val topicPartition = new TopicPartition("topic", partition)
|
||||
val partitionRecords = new ProduceRequestData()
|
||||
|
|
|
|||
|
|
@ -24,6 +24,7 @@ import kafka.server.epoch.util.MockBlockingSender
|
|||
import kafka.server.metadata.ZkMetadataCache
|
||||
import kafka.utils.TestUtils
|
||||
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.message.{FetchResponseData, UpdateMetadataRequestData}
|
||||
import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition
|
||||
|
|
@ -1331,7 +1332,7 @@ class ReplicaFetcherThreadTest {
|
|||
|
||||
val tp0 = new TopicPartition("testTopic", 0)
|
||||
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)))
|
||||
val partitionData = new FetchResponseData.PartitionData()
|
||||
.setRecords(records)
|
||||
|
|
@ -1380,7 +1381,7 @@ class ReplicaFetcherThreadTest {
|
|||
when(mockBlockingSend.brokerEndPoint()).thenReturn(brokerEndPoint)
|
||||
|
||||
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)))
|
||||
when(log.maybeUpdateHighWatermark(hw = 0)).thenReturn(None)
|
||||
|
||||
|
|
|
|||
|
|
@ -22,9 +22,10 @@ import kafka.cluster.{Partition, PartitionTest}
|
|||
import kafka.log.{LogManager, UnifiedLog}
|
||||
import kafka.server.QuotaFactory.QuotaManagers
|
||||
import kafka.utils._
|
||||
import org.apache.kafka.common.compress.Compression
|
||||
import org.apache.kafka.common.metrics.Metrics
|
||||
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.PartitionData
|
||||
import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid}
|
||||
|
|
@ -270,7 +271,7 @@ class ReplicaManagerQuotasTest {
|
|||
minOneMessage = anyBoolean)).thenReturn(
|
||||
new FetchDataInfo(
|
||||
new LogOffsetMetadata(0L, 0L, 0),
|
||||
MemoryRecords.withRecords(CompressionType.NONE, record)
|
||||
MemoryRecords.withRecords(Compression.NONE, record)
|
||||
))
|
||||
|
||||
//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
Loading…
Reference in New Issue