mirror of https://github.com/apache/kafka.git
KAFKA-4390; Replace MessageSet usage with client-side alternatives
Author: Jason Gustafson <jason@confluent.io> Reviewers: Ismael Juma <ismael@juma.me.uk>, Guozhang Wang <wangguoz@gmail.com>, Jun Rao <junrao@gmail.com> Closes #2140 from hachikuji/KAFKA4390
This commit is contained in:
parent
6626b058c7
commit
67f1e5b91b
|
|
@ -89,6 +89,7 @@
|
||||||
<allow pkg="net.jpountz" />
|
<allow pkg="net.jpountz" />
|
||||||
<allow pkg="org.apache.kafka.common.record" />
|
<allow pkg="org.apache.kafka.common.record" />
|
||||||
<allow pkg="org.apache.kafka.common.network" />
|
<allow pkg="org.apache.kafka.common.network" />
|
||||||
|
<allow pkg="org.apache.kafka.common.errors" />
|
||||||
</subpackage>
|
</subpackage>
|
||||||
|
|
||||||
<subpackage name="requests">
|
<subpackage name="requests">
|
||||||
|
|
|
||||||
|
|
@ -61,6 +61,7 @@ import java.util.Collection;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
|
import java.util.Iterator;
|
||||||
import java.util.LinkedHashMap;
|
import java.util.LinkedHashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Locale;
|
import java.util.Locale;
|
||||||
|
|
@ -686,11 +687,13 @@ public class Fetcher<K, V> {
|
||||||
}
|
}
|
||||||
|
|
||||||
List<ConsumerRecord<K, V>> parsed = new ArrayList<>();
|
List<ConsumerRecord<K, V>> parsed = new ArrayList<>();
|
||||||
for (LogEntry logEntry : partition.records) {
|
Iterator<LogEntry> deepIterator = partition.records.deepIterator();
|
||||||
|
while (deepIterator.hasNext()) {
|
||||||
|
LogEntry logEntry = deepIterator.next();
|
||||||
// Skip the messages earlier than current position.
|
// Skip the messages earlier than current position.
|
||||||
if (logEntry.offset() >= position) {
|
if (logEntry.offset() >= position) {
|
||||||
parsed.add(parseRecord(tp, logEntry));
|
parsed.add(parseRecord(tp, logEntry));
|
||||||
bytes += logEntry.size();
|
bytes += logEntry.sizeInBytes();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -198,7 +198,7 @@ public final class BufferPool {
|
||||||
* memory as free.
|
* memory as free.
|
||||||
*
|
*
|
||||||
* @param buffer The buffer to return
|
* @param buffer The buffer to return
|
||||||
* @param size The size of the buffer to mark as deallocated, note that this maybe smaller than buffer.capacity
|
* @param size The size of the buffer to mark as deallocated, note that this may be smaller than buffer.capacity
|
||||||
* since the buffer may re-allocate itself during in-place compression
|
* since the buffer may re-allocate itself during in-place compression
|
||||||
*/
|
*/
|
||||||
public void deallocate(ByteBuffer buffer, int size) {
|
public void deallocate(ByteBuffer buffer, int size) {
|
||||||
|
|
|
||||||
|
|
@ -27,8 +27,10 @@ import org.apache.kafka.common.metrics.Sensor;
|
||||||
import org.apache.kafka.common.metrics.stats.Rate;
|
import org.apache.kafka.common.metrics.stats.Rate;
|
||||||
import org.apache.kafka.common.record.CompressionType;
|
import org.apache.kafka.common.record.CompressionType;
|
||||||
import org.apache.kafka.common.record.MemoryRecords;
|
import org.apache.kafka.common.record.MemoryRecords;
|
||||||
|
import org.apache.kafka.common.record.MemoryRecordsBuilder;
|
||||||
import org.apache.kafka.common.record.Record;
|
import org.apache.kafka.common.record.Record;
|
||||||
import org.apache.kafka.common.record.Records;
|
import org.apache.kafka.common.record.Records;
|
||||||
|
import org.apache.kafka.common.record.TimestampType;
|
||||||
import org.apache.kafka.common.utils.CopyOnWriteMap;
|
import org.apache.kafka.common.utils.CopyOnWriteMap;
|
||||||
import org.apache.kafka.common.utils.Time;
|
import org.apache.kafka.common.utils.Time;
|
||||||
import org.apache.kafka.common.utils.Utils;
|
import org.apache.kafka.common.utils.Utils;
|
||||||
|
|
@ -49,7 +51,7 @@ import java.util.concurrent.ConcurrentMap;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This class acts as a queue that accumulates records into {@link org.apache.kafka.common.record.MemoryRecords}
|
* This class acts as a queue that accumulates records into {@link MemoryRecords}
|
||||||
* instances to be sent to the server.
|
* instances to be sent to the server.
|
||||||
* <p>
|
* <p>
|
||||||
* The accumulator uses a bounded amount of memory and append calls will block when that memory is exhausted, unless
|
* The accumulator uses a bounded amount of memory and append calls will block when that memory is exhausted, unless
|
||||||
|
|
@ -77,7 +79,7 @@ public final class RecordAccumulator {
|
||||||
/**
|
/**
|
||||||
* Create a new record accumulator
|
* Create a new record accumulator
|
||||||
*
|
*
|
||||||
* @param batchSize The size to use when allocating {@link org.apache.kafka.common.record.MemoryRecords} instances
|
* @param batchSize The size to use when allocating {@link MemoryRecords} instances
|
||||||
* @param totalSize The maximum memory the record accumulator can use.
|
* @param totalSize The maximum memory the record accumulator can use.
|
||||||
* @param compression The compression codec for the records
|
* @param compression The compression codec for the records
|
||||||
* @param lingerMs An artificial delay time to add before declaring a records instance that isn't full ready for
|
* @param lingerMs An artificial delay time to add before declaring a records instance that isn't full ready for
|
||||||
|
|
@ -190,13 +192,13 @@ public final class RecordAccumulator {
|
||||||
free.deallocate(buffer);
|
free.deallocate(buffer);
|
||||||
return appendResult;
|
return appendResult;
|
||||||
}
|
}
|
||||||
MemoryRecords records = MemoryRecords.emptyRecords(buffer, compression, this.batchSize);
|
MemoryRecordsBuilder recordsBuilder = MemoryRecords.builder(buffer, compression, TimestampType.CREATE_TIME, this.batchSize);
|
||||||
RecordBatch batch = new RecordBatch(tp, records, time.milliseconds());
|
RecordBatch batch = new RecordBatch(tp, recordsBuilder, time.milliseconds());
|
||||||
FutureRecordMetadata future = Utils.notNull(batch.tryAppend(timestamp, key, value, callback, time.milliseconds()));
|
FutureRecordMetadata future = Utils.notNull(batch.tryAppend(timestamp, key, value, callback, time.milliseconds()));
|
||||||
|
|
||||||
dq.addLast(batch);
|
dq.addLast(batch);
|
||||||
incomplete.add(batch);
|
incomplete.add(batch);
|
||||||
return new RecordAppendResult(future, dq.size() > 1 || batch.records.isFull(), true);
|
return new RecordAppendResult(future, dq.size() > 1 || batch.isFull(), true);
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
appendsInProgress.decrementAndGet();
|
appendsInProgress.decrementAndGet();
|
||||||
|
|
@ -212,9 +214,9 @@ public final class RecordAccumulator {
|
||||||
if (last != null) {
|
if (last != null) {
|
||||||
FutureRecordMetadata future = last.tryAppend(timestamp, key, value, callback, time.milliseconds());
|
FutureRecordMetadata future = last.tryAppend(timestamp, key, value, callback, time.milliseconds());
|
||||||
if (future == null)
|
if (future == null)
|
||||||
last.records.close();
|
last.close();
|
||||||
else
|
else
|
||||||
return new RecordAppendResult(future, deque.size() > 1 || last.records.isFull(), false);
|
return new RecordAppendResult(future, deque.size() > 1 || last.isFull(), false);
|
||||||
}
|
}
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
@ -240,7 +242,7 @@ public final class RecordAccumulator {
|
||||||
Iterator<RecordBatch> batchIterator = dq.iterator();
|
Iterator<RecordBatch> batchIterator = dq.iterator();
|
||||||
while (batchIterator.hasNext()) {
|
while (batchIterator.hasNext()) {
|
||||||
RecordBatch batch = batchIterator.next();
|
RecordBatch batch = batchIterator.next();
|
||||||
boolean isFull = batch != lastBatch || batch.records.isFull();
|
boolean isFull = batch != lastBatch || batch.isFull();
|
||||||
// check if the batch is expired
|
// check if the batch is expired
|
||||||
if (batch.maybeExpire(requestTimeout, retryBackoffMs, now, this.lingerMs, isFull)) {
|
if (batch.maybeExpire(requestTimeout, retryBackoffMs, now, this.lingerMs, isFull)) {
|
||||||
expiredBatches.add(batch);
|
expiredBatches.add(batch);
|
||||||
|
|
@ -319,7 +321,7 @@ public final class RecordAccumulator {
|
||||||
long waitedTimeMs = nowMs - batch.lastAttemptMs;
|
long waitedTimeMs = nowMs - batch.lastAttemptMs;
|
||||||
long timeToWaitMs = backingOff ? retryBackoffMs : lingerMs;
|
long timeToWaitMs = backingOff ? retryBackoffMs : lingerMs;
|
||||||
long timeLeftMs = Math.max(timeToWaitMs - waitedTimeMs, 0);
|
long timeLeftMs = Math.max(timeToWaitMs - waitedTimeMs, 0);
|
||||||
boolean full = deque.size() > 1 || batch.records.isFull();
|
boolean full = deque.size() > 1 || batch.isFull();
|
||||||
boolean expired = waitedTimeMs >= timeToWaitMs;
|
boolean expired = waitedTimeMs >= timeToWaitMs;
|
||||||
boolean sendable = full || expired || exhausted || closed || flushInProgress();
|
boolean sendable = full || expired || exhausted || closed || flushInProgress();
|
||||||
if (sendable && !backingOff) {
|
if (sendable && !backingOff) {
|
||||||
|
|
@ -389,15 +391,15 @@ public final class RecordAccumulator {
|
||||||
boolean backoff = first.attempts > 0 && first.lastAttemptMs + retryBackoffMs > now;
|
boolean backoff = first.attempts > 0 && first.lastAttemptMs + retryBackoffMs > now;
|
||||||
// Only drain the batch if it is not during backoff period.
|
// Only drain the batch if it is not during backoff period.
|
||||||
if (!backoff) {
|
if (!backoff) {
|
||||||
if (size + first.records.sizeInBytes() > maxSize && !ready.isEmpty()) {
|
if (size + first.sizeInBytes() > maxSize && !ready.isEmpty()) {
|
||||||
// there is a rare case that a single batch size is larger than the request size due
|
// there is a rare case that a single batch size is larger than the request size due
|
||||||
// to compression; in this case we will still eventually send this batch in a single
|
// to compression; in this case we will still eventually send this batch in a single
|
||||||
// request
|
// request
|
||||||
break;
|
break;
|
||||||
} else {
|
} else {
|
||||||
RecordBatch batch = deque.pollFirst();
|
RecordBatch batch = deque.pollFirst();
|
||||||
batch.records.close();
|
batch.close();
|
||||||
size += batch.records.sizeInBytes();
|
size += batch.sizeInBytes();
|
||||||
ready.add(batch);
|
ready.add(batch);
|
||||||
batch.drainedMs = now;
|
batch.drainedMs = now;
|
||||||
}
|
}
|
||||||
|
|
@ -437,7 +439,7 @@ public final class RecordAccumulator {
|
||||||
*/
|
*/
|
||||||
public void deallocate(RecordBatch batch) {
|
public void deallocate(RecordBatch batch) {
|
||||||
incomplete.remove(batch);
|
incomplete.remove(batch);
|
||||||
free.deallocate(batch.records.buffer(), batch.records.initialCapacity());
|
free.deallocate(batch.buffer(), batch.initialCapacity());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -507,7 +509,7 @@ public final class RecordAccumulator {
|
||||||
Deque<RecordBatch> dq = getDeque(batch.topicPartition);
|
Deque<RecordBatch> dq = getDeque(batch.topicPartition);
|
||||||
// Close the batch before aborting
|
// Close the batch before aborting
|
||||||
synchronized (dq) {
|
synchronized (dq) {
|
||||||
batch.records.close();
|
batch.close();
|
||||||
dq.remove(batch);
|
dq.remove(batch);
|
||||||
}
|
}
|
||||||
batch.done(-1L, Record.NO_TIMESTAMP, new IllegalStateException("Producer is closed forcefully."));
|
batch.done(-1L, Record.NO_TIMESTAMP, new IllegalStateException("Producer is closed forcefully."));
|
||||||
|
|
|
||||||
|
|
@ -12,18 +12,20 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.clients.producer.internals;
|
package org.apache.kafka.clients.producer.internals;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.List;
|
|
||||||
|
|
||||||
import org.apache.kafka.clients.producer.Callback;
|
import org.apache.kafka.clients.producer.Callback;
|
||||||
import org.apache.kafka.clients.producer.RecordMetadata;
|
import org.apache.kafka.clients.producer.RecordMetadata;
|
||||||
import org.apache.kafka.common.TopicPartition;
|
import org.apache.kafka.common.TopicPartition;
|
||||||
import org.apache.kafka.common.errors.TimeoutException;
|
import org.apache.kafka.common.errors.TimeoutException;
|
||||||
import org.apache.kafka.common.record.MemoryRecords;
|
import org.apache.kafka.common.record.MemoryRecords;
|
||||||
|
import org.apache.kafka.common.record.MemoryRecordsBuilder;
|
||||||
import org.apache.kafka.common.record.Record;
|
import org.apache.kafka.common.record.Record;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A batch of records that is or will be sent.
|
* A batch of records that is or will be sent.
|
||||||
*
|
*
|
||||||
|
|
@ -39,21 +41,21 @@ public final class RecordBatch {
|
||||||
public final long createdMs;
|
public final long createdMs;
|
||||||
public long drainedMs;
|
public long drainedMs;
|
||||||
public long lastAttemptMs;
|
public long lastAttemptMs;
|
||||||
public final MemoryRecords records;
|
|
||||||
public final TopicPartition topicPartition;
|
public final TopicPartition topicPartition;
|
||||||
public final ProduceRequestResult produceFuture;
|
public final ProduceRequestResult produceFuture;
|
||||||
public long lastAppendTime;
|
public long lastAppendTime;
|
||||||
private final List<Thunk> thunks;
|
private final List<Thunk> thunks;
|
||||||
private long offsetCounter = 0L;
|
private long offsetCounter = 0L;
|
||||||
private boolean retry;
|
private boolean retry;
|
||||||
|
private final MemoryRecordsBuilder recordsBuilder;
|
||||||
|
|
||||||
public RecordBatch(TopicPartition tp, MemoryRecords records, long now) {
|
public RecordBatch(TopicPartition tp, MemoryRecordsBuilder recordsBuilder, long now) {
|
||||||
this.createdMs = now;
|
this.createdMs = now;
|
||||||
this.lastAttemptMs = now;
|
this.lastAttemptMs = now;
|
||||||
this.records = records;
|
this.recordsBuilder = recordsBuilder;
|
||||||
this.topicPartition = tp;
|
this.topicPartition = tp;
|
||||||
this.produceFuture = new ProduceRequestResult();
|
this.produceFuture = new ProduceRequestResult();
|
||||||
this.thunks = new ArrayList<Thunk>();
|
this.thunks = new ArrayList<>();
|
||||||
this.lastAppendTime = createdMs;
|
this.lastAppendTime = createdMs;
|
||||||
this.retry = false;
|
this.retry = false;
|
||||||
}
|
}
|
||||||
|
|
@ -64,10 +66,10 @@ public final class RecordBatch {
|
||||||
* @return The RecordSend corresponding to this record or null if there isn't sufficient room.
|
* @return The RecordSend corresponding to this record or null if there isn't sufficient room.
|
||||||
*/
|
*/
|
||||||
public FutureRecordMetadata tryAppend(long timestamp, byte[] key, byte[] value, Callback callback, long now) {
|
public FutureRecordMetadata tryAppend(long timestamp, byte[] key, byte[] value, Callback callback, long now) {
|
||||||
if (!this.records.hasRoomFor(key, value)) {
|
if (!recordsBuilder.hasRoomFor(key, value)) {
|
||||||
return null;
|
return null;
|
||||||
} else {
|
} else {
|
||||||
long checksum = this.records.append(offsetCounter++, timestamp, key, value);
|
long checksum = this.recordsBuilder.append(offsetCounter++, timestamp, key, value);
|
||||||
this.maxRecordSize = Math.max(this.maxRecordSize, Record.recordSize(key, value));
|
this.maxRecordSize = Math.max(this.maxRecordSize, Record.recordSize(key, value));
|
||||||
this.lastAppendTime = now;
|
this.lastAppendTime = now;
|
||||||
FutureRecordMetadata future = new FutureRecordMetadata(this.produceFuture, this.recordCount,
|
FutureRecordMetadata future = new FutureRecordMetadata(this.produceFuture, this.recordCount,
|
||||||
|
|
@ -94,9 +96,8 @@ public final class RecordBatch {
|
||||||
baseOffset,
|
baseOffset,
|
||||||
exception);
|
exception);
|
||||||
// execute callbacks
|
// execute callbacks
|
||||||
for (int i = 0; i < this.thunks.size(); i++) {
|
for (Thunk thunk : thunks) {
|
||||||
try {
|
try {
|
||||||
Thunk thunk = this.thunks.get(i);
|
|
||||||
if (exception == null) {
|
if (exception == null) {
|
||||||
// If the timestamp returned by server is NoTimestamp, that means CreateTime is used. Otherwise LogAppendTime is used.
|
// If the timestamp returned by server is NoTimestamp, that means CreateTime is used. Otherwise LogAppendTime is used.
|
||||||
RecordMetadata metadata = new RecordMetadata(this.topicPartition, baseOffset, thunk.future.relativeOffset(),
|
RecordMetadata metadata = new RecordMetadata(this.topicPartition, baseOffset, thunk.future.relativeOffset(),
|
||||||
|
|
@ -156,7 +157,7 @@ public final class RecordBatch {
|
||||||
}
|
}
|
||||||
|
|
||||||
if (expire) {
|
if (expire) {
|
||||||
this.records.close();
|
close();
|
||||||
this.done(-1L, Record.NO_TIMESTAMP,
|
this.done(-1L, Record.NO_TIMESTAMP,
|
||||||
new TimeoutException("Expiring " + recordCount + " record(s) for " + topicPartition + " due to " + errorMessage));
|
new TimeoutException("Expiring " + recordCount + " record(s) for " + topicPartition + " due to " + errorMessage));
|
||||||
}
|
}
|
||||||
|
|
@ -177,4 +178,37 @@ public final class RecordBatch {
|
||||||
public void setRetry() {
|
public void setRetry() {
|
||||||
this.retry = true;
|
this.retry = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public MemoryRecords records() {
|
||||||
|
return recordsBuilder.build();
|
||||||
|
}
|
||||||
|
|
||||||
|
public int sizeInBytes() {
|
||||||
|
return recordsBuilder.sizeInBytes();
|
||||||
|
}
|
||||||
|
|
||||||
|
public double compressionRate() {
|
||||||
|
return recordsBuilder.compressionRate();
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean isFull() {
|
||||||
|
return recordsBuilder.isFull();
|
||||||
|
}
|
||||||
|
|
||||||
|
public void close() {
|
||||||
|
recordsBuilder.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
public ByteBuffer buffer() {
|
||||||
|
return recordsBuilder.buffer();
|
||||||
|
}
|
||||||
|
|
||||||
|
public int initialCapacity() {
|
||||||
|
return recordsBuilder.initialCapacity();
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean isWritable() {
|
||||||
|
return !recordsBuilder.isClosed();
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -345,7 +345,7 @@ public class Sender implements Runnable {
|
||||||
final Map<TopicPartition, RecordBatch> recordsByPartition = new HashMap<>(batches.size());
|
final Map<TopicPartition, RecordBatch> recordsByPartition = new HashMap<>(batches.size());
|
||||||
for (RecordBatch batch : batches) {
|
for (RecordBatch batch : batches) {
|
||||||
TopicPartition tp = batch.topicPartition;
|
TopicPartition tp = batch.topicPartition;
|
||||||
produceRecordsByPartition.put(tp, batch.records);
|
produceRecordsByPartition.put(tp, batch.records());
|
||||||
recordsByPartition.put(tp, batch);
|
recordsByPartition.put(tp, batch);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -505,17 +505,17 @@ public class Sender implements Runnable {
|
||||||
// per-topic bytes send rate
|
// per-topic bytes send rate
|
||||||
String topicByteRateName = "topic." + topic + ".bytes";
|
String topicByteRateName = "topic." + topic + ".bytes";
|
||||||
Sensor topicByteRate = Utils.notNull(this.metrics.getSensor(topicByteRateName));
|
Sensor topicByteRate = Utils.notNull(this.metrics.getSensor(topicByteRateName));
|
||||||
topicByteRate.record(batch.records.sizeInBytes());
|
topicByteRate.record(batch.sizeInBytes());
|
||||||
|
|
||||||
// per-topic compression rate
|
// per-topic compression rate
|
||||||
String topicCompressionRateName = "topic." + topic + ".compression-rate";
|
String topicCompressionRateName = "topic." + topic + ".compression-rate";
|
||||||
Sensor topicCompressionRate = Utils.notNull(this.metrics.getSensor(topicCompressionRateName));
|
Sensor topicCompressionRate = Utils.notNull(this.metrics.getSensor(topicCompressionRateName));
|
||||||
topicCompressionRate.record(batch.records.compressionRate());
|
topicCompressionRate.record(batch.compressionRate());
|
||||||
|
|
||||||
// global metrics
|
// global metrics
|
||||||
this.batchSizeSensor.record(batch.records.sizeInBytes(), now);
|
this.batchSizeSensor.record(batch.sizeInBytes(), now);
|
||||||
this.queueTimeSensor.record(batch.drainedMs - batch.createdMs, now);
|
this.queueTimeSensor.record(batch.drainedMs - batch.createdMs, now);
|
||||||
this.compressionRateSensor.record(batch.records.compressionRate());
|
this.compressionRateSensor.record(batch.compressionRate());
|
||||||
this.maxRecordSizeSensor.record(batch.maxRecordSize, now);
|
this.maxRecordSizeSensor.record(batch.maxRecordSize, now);
|
||||||
records += batch.recordCount;
|
records += batch.recordCount;
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,92 @@
|
||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p/>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p/>
|
||||||
|
* 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.utils.AbstractIterator;
|
||||||
|
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
public abstract class AbstractRecords implements Records {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean hasMatchingShallowMagic(byte magic) {
|
||||||
|
Iterator<? extends LogEntry> iterator = shallowIterator();
|
||||||
|
while (iterator.hasNext())
|
||||||
|
if (iterator.next().magic() != magic)
|
||||||
|
return false;
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Convert this message set to use the specified message format.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public Records toMessageFormat(byte toMagic) {
|
||||||
|
List<LogEntry> converted = new ArrayList<>();
|
||||||
|
Iterator<LogEntry> deepIterator = deepIterator();
|
||||||
|
while (deepIterator.hasNext()) {
|
||||||
|
LogEntry entry = deepIterator.next();
|
||||||
|
converted.add(LogEntry.create(entry.offset(), entry.record().convert(toMagic)));
|
||||||
|
}
|
||||||
|
|
||||||
|
if (converted.isEmpty()) {
|
||||||
|
// This indicates that the message is too large, which indicates that the buffer is not large
|
||||||
|
// enough to hold a full log entry. We just return all the bytes in the file message set.
|
||||||
|
// Even though the message set does not have the right format version, we expect old clients
|
||||||
|
// to raise an error to the user after reading the message size and seeing that there
|
||||||
|
// are not enough available bytes in the response to read the full message.
|
||||||
|
return this;
|
||||||
|
} else {
|
||||||
|
// We use the first message to determine the compression type for the resulting message set.
|
||||||
|
// This could result in message sets which are either larger or smaller than the original size.
|
||||||
|
// For example, it could end up larger if most messages were previously compressed, but
|
||||||
|
// it just so happens that the first one is not. There is also some risk that this can
|
||||||
|
// cause some timestamp information to be lost (e.g. if the timestamp type was changed) since
|
||||||
|
// we are essentially merging multiple message sets. However, currently this method is only
|
||||||
|
// used for down-conversion, so we've ignored the problem.
|
||||||
|
CompressionType compressionType = shallowIterator().next().record().compressionType();
|
||||||
|
return MemoryRecords.withLogEntries(compressionType, converted);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static int estimatedSize(CompressionType compressionType, Iterable<LogEntry> entries) {
|
||||||
|
int size = 0;
|
||||||
|
for (LogEntry entry : entries)
|
||||||
|
size += entry.sizeInBytes();
|
||||||
|
// NOTE: 1024 is the minimum block size for snappy encoding
|
||||||
|
return compressionType == CompressionType.NONE ? size : Math.min(Math.max(size / 2, 1024), 1 << 16);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get an iterator over the deep records.
|
||||||
|
* @return An iterator over the records
|
||||||
|
*/
|
||||||
|
public Iterator<Record> records() {
|
||||||
|
return new AbstractIterator<Record>() {
|
||||||
|
private final Iterator<? extends LogEntry> deepEntries = deepIterator();
|
||||||
|
@Override
|
||||||
|
protected Record makeNext() {
|
||||||
|
if (deepEntries.hasNext())
|
||||||
|
return deepEntries.next().record();
|
||||||
|
return allDone();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -16,34 +16,41 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.common.record;
|
package org.apache.kafka.common.record;
|
||||||
|
|
||||||
|
import java.io.DataInputStream;
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A byte buffer backed input inputStream
|
* A byte buffer backed input inputStream
|
||||||
*/
|
*/
|
||||||
public class ByteBufferInputStream extends InputStream {
|
public class ByteBufferInputStream extends DataInputStream {
|
||||||
|
|
||||||
private ByteBuffer buffer;
|
|
||||||
|
|
||||||
public ByteBufferInputStream(ByteBuffer buffer) {
|
public ByteBufferInputStream(ByteBuffer buffer) {
|
||||||
this.buffer = buffer;
|
super(new UnderlyingInputStream(buffer));
|
||||||
}
|
}
|
||||||
|
|
||||||
public int read() {
|
private static class UnderlyingInputStream extends InputStream {
|
||||||
if (!buffer.hasRemaining()) {
|
private ByteBuffer buffer;
|
||||||
return -1;
|
|
||||||
}
|
|
||||||
return buffer.get() & 0xFF;
|
|
||||||
}
|
|
||||||
|
|
||||||
public int read(byte[] bytes, int off, int len) {
|
public UnderlyingInputStream(ByteBuffer buffer) {
|
||||||
if (!buffer.hasRemaining()) {
|
this.buffer = buffer;
|
||||||
return -1;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
len = Math.min(len, buffer.remaining());
|
public int read() {
|
||||||
buffer.get(bytes, off, len);
|
if (!buffer.hasRemaining()) {
|
||||||
return len;
|
return -1;
|
||||||
|
}
|
||||||
|
return buffer.get() & 0xFF;
|
||||||
|
}
|
||||||
|
|
||||||
|
public int read(byte[] bytes, int off, int len) {
|
||||||
|
if (!buffer.hasRemaining()) {
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
|
||||||
|
len = Math.min(len, buffer.remaining());
|
||||||
|
buffer.get(bytes, off, len);
|
||||||
|
return len;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,119 @@
|
||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p/>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p/>
|
||||||
|
* 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.errors.CorruptRecordException;
|
||||||
|
import org.apache.kafka.common.utils.Utils;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
|
||||||
|
import static org.apache.kafka.common.record.Records.LOG_OVERHEAD;
|
||||||
|
import static org.apache.kafka.common.record.Records.OFFSET_OFFSET;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A byte buffer backed log input stream. This class avoids the need to copy records by returning
|
||||||
|
* slices from the underlying byte buffer.
|
||||||
|
*/
|
||||||
|
class ByteBufferLogInputStream implements LogInputStream<ByteBufferLogInputStream.ByteBufferLogEntry> {
|
||||||
|
private final ByteBuffer buffer;
|
||||||
|
private final int maxMessageSize;
|
||||||
|
|
||||||
|
ByteBufferLogInputStream(ByteBuffer buffer, int maxMessageSize) {
|
||||||
|
this.buffer = buffer;
|
||||||
|
this.maxMessageSize = maxMessageSize;
|
||||||
|
}
|
||||||
|
|
||||||
|
public ByteBufferLogEntry nextEntry() throws IOException {
|
||||||
|
int remaining = buffer.remaining();
|
||||||
|
if (remaining < LOG_OVERHEAD)
|
||||||
|
return null;
|
||||||
|
|
||||||
|
int recordSize = buffer.getInt(buffer.position() + Records.SIZE_OFFSET);
|
||||||
|
if (recordSize < Record.RECORD_OVERHEAD_V0)
|
||||||
|
throw new CorruptRecordException(String.format("Record size is less than the minimum record overhead (%d)", Record.RECORD_OVERHEAD_V0));
|
||||||
|
if (recordSize > maxMessageSize)
|
||||||
|
throw new CorruptRecordException(String.format("Record size exceeds the largest allowable message size (%d).", maxMessageSize));
|
||||||
|
|
||||||
|
int entrySize = recordSize + LOG_OVERHEAD;
|
||||||
|
if (remaining < entrySize)
|
||||||
|
return null;
|
||||||
|
|
||||||
|
ByteBuffer entrySlice = buffer.slice();
|
||||||
|
entrySlice.limit(entrySize);
|
||||||
|
buffer.position(buffer.position() + entrySize);
|
||||||
|
return new ByteBufferLogEntry(entrySlice);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class ByteBufferLogEntry extends LogEntry {
|
||||||
|
private final ByteBuffer buffer;
|
||||||
|
private final Record record;
|
||||||
|
|
||||||
|
private ByteBufferLogEntry(ByteBuffer buffer) {
|
||||||
|
this.buffer = buffer;
|
||||||
|
buffer.position(LOG_OVERHEAD);
|
||||||
|
this.record = new Record(buffer.slice());
|
||||||
|
buffer.position(OFFSET_OFFSET);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long offset() {
|
||||||
|
return buffer.getLong(OFFSET_OFFSET);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Record record() {
|
||||||
|
return record;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setOffset(long offset) {
|
||||||
|
buffer.putLong(OFFSET_OFFSET, offset);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setCreateTime(long timestamp) {
|
||||||
|
if (record.magic() == Record.MAGIC_VALUE_V0)
|
||||||
|
throw new IllegalArgumentException("Cannot set timestamp for a record with magic = 0");
|
||||||
|
|
||||||
|
long currentTimestamp = record.timestamp();
|
||||||
|
// We don't need to recompute crc if the timestamp is not updated.
|
||||||
|
if (record.timestampType() == TimestampType.CREATE_TIME && currentTimestamp == timestamp)
|
||||||
|
return;
|
||||||
|
|
||||||
|
byte attributes = record.attributes();
|
||||||
|
buffer.put(LOG_OVERHEAD + Record.ATTRIBUTES_OFFSET, TimestampType.CREATE_TIME.updateAttributes(attributes));
|
||||||
|
buffer.putLong(LOG_OVERHEAD + Record.TIMESTAMP_OFFSET, timestamp);
|
||||||
|
long crc = record.computeChecksum();
|
||||||
|
Utils.writeUnsignedInt(buffer, LOG_OVERHEAD + Record.CRC_OFFSET, crc);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setLogAppendTime(long timestamp) {
|
||||||
|
if (record.magic() == Record.MAGIC_VALUE_V0)
|
||||||
|
throw new IllegalArgumentException("Cannot set timestamp for a record with magic = 0");
|
||||||
|
|
||||||
|
byte attributes = record.attributes();
|
||||||
|
buffer.put(LOG_OVERHEAD + Record.ATTRIBUTES_OFFSET, TimestampType.LOG_APPEND_TIME.updateAttributes(attributes));
|
||||||
|
buffer.putLong(LOG_OVERHEAD + Record.TIMESTAMP_OFFSET, timestamp);
|
||||||
|
long crc = record.computeChecksum();
|
||||||
|
Utils.writeUnsignedInt(buffer, LOG_OVERHEAD + Record.CRC_OFFSET, crc);
|
||||||
|
}
|
||||||
|
|
||||||
|
public ByteBuffer buffer() {
|
||||||
|
return buffer;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -16,42 +16,54 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.common.record;
|
package org.apache.kafka.common.record;
|
||||||
|
|
||||||
|
import java.io.DataOutputStream;
|
||||||
import java.io.OutputStream;
|
import java.io.OutputStream;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A byte buffer backed output outputStream
|
* A byte buffer backed output outputStream
|
||||||
*/
|
*/
|
||||||
public class ByteBufferOutputStream extends OutputStream {
|
public class ByteBufferOutputStream extends DataOutputStream {
|
||||||
|
|
||||||
private static final float REALLOCATION_FACTOR = 1.1f;
|
private static final float REALLOCATION_FACTOR = 1.1f;
|
||||||
|
|
||||||
private ByteBuffer buffer;
|
|
||||||
|
|
||||||
public ByteBufferOutputStream(ByteBuffer buffer) {
|
public ByteBufferOutputStream(ByteBuffer buffer) {
|
||||||
this.buffer = buffer;
|
super(new UnderlyingOutputStream(buffer));
|
||||||
}
|
|
||||||
|
|
||||||
public void write(int b) {
|
|
||||||
if (buffer.remaining() < 1)
|
|
||||||
expandBuffer(buffer.capacity() + 1);
|
|
||||||
buffer.put((byte) b);
|
|
||||||
}
|
|
||||||
|
|
||||||
public void write(byte[] bytes, int off, int len) {
|
|
||||||
if (buffer.remaining() < len)
|
|
||||||
expandBuffer(buffer.capacity() + len);
|
|
||||||
buffer.put(bytes, off, len);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public ByteBuffer buffer() {
|
public ByteBuffer buffer() {
|
||||||
return buffer;
|
return ((UnderlyingOutputStream) out).buffer;
|
||||||
}
|
}
|
||||||
|
|
||||||
private void expandBuffer(int size) {
|
public static class UnderlyingOutputStream extends OutputStream {
|
||||||
int expandSize = Math.max((int) (buffer.capacity() * REALLOCATION_FACTOR), size);
|
private ByteBuffer buffer;
|
||||||
ByteBuffer temp = ByteBuffer.allocate(expandSize);
|
|
||||||
temp.put(buffer.array(), buffer.arrayOffset(), buffer.position());
|
public UnderlyingOutputStream(ByteBuffer buffer) {
|
||||||
buffer = temp;
|
this.buffer = buffer;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void write(int b) {
|
||||||
|
if (buffer.remaining() < 1)
|
||||||
|
expandBuffer(buffer.capacity() + 1);
|
||||||
|
buffer.put((byte) b);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void write(byte[] bytes, int off, int len) {
|
||||||
|
if (buffer.remaining() < len)
|
||||||
|
expandBuffer(buffer.capacity() + len);
|
||||||
|
buffer.put(bytes, off, len);
|
||||||
|
}
|
||||||
|
|
||||||
|
public ByteBuffer buffer() {
|
||||||
|
return buffer;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void expandBuffer(int size) {
|
||||||
|
int expandSize = Math.max((int) (buffer.capacity() * REALLOCATION_FACTOR), size);
|
||||||
|
ByteBuffer temp = ByteBuffer.allocate(expandSize);
|
||||||
|
temp.put(buffer.array(), buffer.arrayOffset(), buffer.position());
|
||||||
|
buffer = temp;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -26,7 +26,7 @@ public enum CompressionType {
|
||||||
public final String name;
|
public final String name;
|
||||||
public final float rate;
|
public final float rate;
|
||||||
|
|
||||||
private CompressionType(int id, String name, float rate) {
|
CompressionType(int id, String name, float rate) {
|
||||||
this.id = id;
|
this.id = id;
|
||||||
this.name = name;
|
this.name = name;
|
||||||
this.rate = rate;
|
this.rate = rate;
|
||||||
|
|
|
||||||
|
|
@ -1,332 +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 java.lang.reflect.Constructor;
|
|
||||||
import org.apache.kafka.common.KafkaException;
|
|
||||||
import org.apache.kafka.common.utils.Utils;
|
|
||||||
|
|
||||||
import java.io.InputStream;
|
|
||||||
import java.io.OutputStream;
|
|
||||||
import java.io.DataInputStream;
|
|
||||||
import java.io.DataOutputStream;
|
|
||||||
import java.io.IOException;
|
|
||||||
import java.nio.ByteBuffer;
|
|
||||||
import java.util.zip.GZIPInputStream;
|
|
||||||
import java.util.zip.GZIPOutputStream;
|
|
||||||
|
|
||||||
public class Compressor {
|
|
||||||
|
|
||||||
static private final float COMPRESSION_RATE_DAMPING_FACTOR = 0.9f;
|
|
||||||
static private final float COMPRESSION_RATE_ESTIMATION_FACTOR = 1.05f;
|
|
||||||
static private final int COMPRESSION_DEFAULT_BUFFER_SIZE = 1024;
|
|
||||||
|
|
||||||
private static final float[] TYPE_TO_RATE;
|
|
||||||
|
|
||||||
static {
|
|
||||||
int maxTypeId = -1;
|
|
||||||
for (CompressionType type : CompressionType.values())
|
|
||||||
maxTypeId = Math.max(maxTypeId, type.id);
|
|
||||||
TYPE_TO_RATE = new float[maxTypeId + 1];
|
|
||||||
for (CompressionType type : CompressionType.values()) {
|
|
||||||
TYPE_TO_RATE[type.id] = type.rate;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// dynamically load the snappy and lz4 classes to avoid runtime dependency if we are not using compression
|
|
||||||
// caching constructors to avoid invoking of Class.forName method for each batch
|
|
||||||
private static MemoizingConstructorSupplier snappyOutputStreamSupplier = new MemoizingConstructorSupplier(new ConstructorSupplier() {
|
|
||||||
@Override
|
|
||||||
public Constructor get() throws ClassNotFoundException, NoSuchMethodException {
|
|
||||||
return Class.forName("org.xerial.snappy.SnappyOutputStream")
|
|
||||||
.getConstructor(OutputStream.class, Integer.TYPE);
|
|
||||||
}
|
|
||||||
});
|
|
||||||
|
|
||||||
private static MemoizingConstructorSupplier lz4OutputStreamSupplier = new MemoizingConstructorSupplier(new ConstructorSupplier() {
|
|
||||||
@Override
|
|
||||||
public Constructor get() throws ClassNotFoundException, NoSuchMethodException {
|
|
||||||
return Class.forName("org.apache.kafka.common.record.KafkaLZ4BlockOutputStream")
|
|
||||||
.getConstructor(OutputStream.class);
|
|
||||||
}
|
|
||||||
});
|
|
||||||
|
|
||||||
private static MemoizingConstructorSupplier snappyInputStreamSupplier = new MemoizingConstructorSupplier(new ConstructorSupplier() {
|
|
||||||
@Override
|
|
||||||
public Constructor get() throws ClassNotFoundException, NoSuchMethodException {
|
|
||||||
return Class.forName("org.xerial.snappy.SnappyInputStream")
|
|
||||||
.getConstructor(InputStream.class);
|
|
||||||
}
|
|
||||||
});
|
|
||||||
|
|
||||||
private static MemoizingConstructorSupplier lz4InputStreamSupplier = new MemoizingConstructorSupplier(new ConstructorSupplier() {
|
|
||||||
@Override
|
|
||||||
public Constructor get() throws ClassNotFoundException, NoSuchMethodException {
|
|
||||||
return Class.forName("org.apache.kafka.common.record.KafkaLZ4BlockInputStream")
|
|
||||||
.getConstructor(InputStream.class, Boolean.TYPE);
|
|
||||||
}
|
|
||||||
});
|
|
||||||
|
|
||||||
private final CompressionType type;
|
|
||||||
private final DataOutputStream appendStream;
|
|
||||||
private final ByteBufferOutputStream bufferStream;
|
|
||||||
private final int initPos;
|
|
||||||
|
|
||||||
public long writtenUncompressed;
|
|
||||||
public long numRecords;
|
|
||||||
public float compressionRate;
|
|
||||||
public long maxTimestamp;
|
|
||||||
|
|
||||||
public Compressor(ByteBuffer buffer, CompressionType type) {
|
|
||||||
this.type = type;
|
|
||||||
this.initPos = buffer.position();
|
|
||||||
|
|
||||||
this.numRecords = 0;
|
|
||||||
this.writtenUncompressed = 0;
|
|
||||||
this.compressionRate = 1;
|
|
||||||
this.maxTimestamp = Record.NO_TIMESTAMP;
|
|
||||||
|
|
||||||
if (type != CompressionType.NONE) {
|
|
||||||
// for compressed records, leave space for the header and the shallow message metadata
|
|
||||||
// and move the starting position to the value payload offset
|
|
||||||
buffer.position(initPos + Records.LOG_OVERHEAD + Record.RECORD_OVERHEAD);
|
|
||||||
}
|
|
||||||
|
|
||||||
// create the stream
|
|
||||||
bufferStream = new ByteBufferOutputStream(buffer);
|
|
||||||
appendStream = wrapForOutput(bufferStream, type, COMPRESSION_DEFAULT_BUFFER_SIZE);
|
|
||||||
}
|
|
||||||
|
|
||||||
public ByteBuffer buffer() {
|
|
||||||
return bufferStream.buffer();
|
|
||||||
}
|
|
||||||
|
|
||||||
public double compressionRate() {
|
|
||||||
return compressionRate;
|
|
||||||
}
|
|
||||||
|
|
||||||
public void close() {
|
|
||||||
try {
|
|
||||||
appendStream.close();
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new KafkaException(e);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (type != CompressionType.NONE) {
|
|
||||||
ByteBuffer buffer = bufferStream.buffer();
|
|
||||||
int pos = buffer.position();
|
|
||||||
// write the header, for the end offset write as number of records - 1
|
|
||||||
buffer.position(initPos);
|
|
||||||
buffer.putLong(numRecords - 1);
|
|
||||||
buffer.putInt(pos - initPos - Records.LOG_OVERHEAD);
|
|
||||||
// write the shallow message (the crc and value size are not correct yet)
|
|
||||||
Record.write(buffer, maxTimestamp, null, null, type, 0, -1);
|
|
||||||
// compute the fill the value size
|
|
||||||
int valueSize = pos - initPos - Records.LOG_OVERHEAD - Record.RECORD_OVERHEAD;
|
|
||||||
buffer.putInt(initPos + Records.LOG_OVERHEAD + Record.KEY_OFFSET_V1, valueSize);
|
|
||||||
// compute and fill the crc at the beginning of the message
|
|
||||||
long crc = Record.computeChecksum(buffer,
|
|
||||||
initPos + Records.LOG_OVERHEAD + Record.MAGIC_OFFSET,
|
|
||||||
pos - initPos - Records.LOG_OVERHEAD - Record.MAGIC_OFFSET);
|
|
||||||
Utils.writeUnsignedInt(buffer, initPos + Records.LOG_OVERHEAD + Record.CRC_OFFSET, crc);
|
|
||||||
// reset the position
|
|
||||||
buffer.position(pos);
|
|
||||||
|
|
||||||
// update the compression ratio
|
|
||||||
this.compressionRate = (float) buffer.position() / this.writtenUncompressed;
|
|
||||||
TYPE_TO_RATE[type.id] = TYPE_TO_RATE[type.id] * COMPRESSION_RATE_DAMPING_FACTOR +
|
|
||||||
compressionRate * (1 - COMPRESSION_RATE_DAMPING_FACTOR);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// Note that for all the write operations below, IO exceptions should
|
|
||||||
// never be thrown since the underlying ByteBufferOutputStream does not throw IOException;
|
|
||||||
// therefore upon encountering this issue we just close the append stream.
|
|
||||||
|
|
||||||
public void putLong(final long value) {
|
|
||||||
try {
|
|
||||||
appendStream.writeLong(value);
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new KafkaException("I/O exception when writing to the append stream, closing", e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public void putInt(final int value) {
|
|
||||||
try {
|
|
||||||
appendStream.writeInt(value);
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new KafkaException("I/O exception when writing to the append stream, closing", e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public void put(final ByteBuffer buffer) {
|
|
||||||
try {
|
|
||||||
appendStream.write(buffer.array(), buffer.arrayOffset(), buffer.limit());
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new KafkaException("I/O exception when writing to the append stream, closing", e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public void putByte(final byte value) {
|
|
||||||
try {
|
|
||||||
appendStream.write(value);
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new KafkaException("I/O exception when writing to the append stream, closing", e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public void put(final byte[] bytes, final int offset, final int len) {
|
|
||||||
try {
|
|
||||||
appendStream.write(bytes, offset, len);
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new KafkaException("I/O exception when writing to the append stream, closing", e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @return CRC of the record
|
|
||||||
*/
|
|
||||||
public long putRecord(long timestamp, byte[] key, byte[] value, CompressionType type,
|
|
||||||
int valueOffset, int valueSize) {
|
|
||||||
// put a record as un-compressed into the underlying stream
|
|
||||||
long crc = Record.computeChecksum(timestamp, key, value, type, valueOffset, valueSize);
|
|
||||||
byte attributes = Record.computeAttributes(type);
|
|
||||||
putRecord(crc, attributes, timestamp, key, value, valueOffset, valueSize);
|
|
||||||
return crc;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Put a record as uncompressed into the underlying stream
|
|
||||||
* @return CRC of the record
|
|
||||||
*/
|
|
||||||
public long putRecord(long timestamp, byte[] key, byte[] value) {
|
|
||||||
return putRecord(timestamp, key, value, CompressionType.NONE, 0, -1);
|
|
||||||
}
|
|
||||||
|
|
||||||
private void putRecord(final long crc, final byte attributes, final long timestamp, final byte[] key, final byte[] value, final int valueOffset, final int valueSize) {
|
|
||||||
maxTimestamp = Math.max(maxTimestamp, timestamp);
|
|
||||||
Record.write(this, crc, attributes, timestamp, key, value, valueOffset, valueSize);
|
|
||||||
}
|
|
||||||
|
|
||||||
public void recordWritten(int size) {
|
|
||||||
numRecords += 1;
|
|
||||||
writtenUncompressed += size;
|
|
||||||
}
|
|
||||||
|
|
||||||
public long numRecordsWritten() {
|
|
||||||
return numRecords;
|
|
||||||
}
|
|
||||||
|
|
||||||
public long estimatedBytesWritten() {
|
|
||||||
if (type == CompressionType.NONE) {
|
|
||||||
return bufferStream.buffer().position();
|
|
||||||
} else {
|
|
||||||
// estimate the written bytes to the underlying byte buffer based on uncompressed written bytes
|
|
||||||
return (long) (writtenUncompressed * TYPE_TO_RATE[type.id] * COMPRESSION_RATE_ESTIMATION_FACTOR);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// the following two functions also need to be public since they are used in MemoryRecords.iteration
|
|
||||||
|
|
||||||
public static DataOutputStream wrapForOutput(ByteBufferOutputStream buffer, CompressionType type, int bufferSize) {
|
|
||||||
try {
|
|
||||||
switch (type) {
|
|
||||||
case NONE:
|
|
||||||
return new DataOutputStream(buffer);
|
|
||||||
case GZIP:
|
|
||||||
return new DataOutputStream(new GZIPOutputStream(buffer, bufferSize));
|
|
||||||
case SNAPPY:
|
|
||||||
try {
|
|
||||||
OutputStream stream = (OutputStream) snappyOutputStreamSupplier.get().newInstance(buffer, bufferSize);
|
|
||||||
return new DataOutputStream(stream);
|
|
||||||
} catch (Exception e) {
|
|
||||||
throw new KafkaException(e);
|
|
||||||
}
|
|
||||||
case LZ4:
|
|
||||||
try {
|
|
||||||
OutputStream stream = (OutputStream) lz4OutputStreamSupplier.get().newInstance(buffer);
|
|
||||||
return new DataOutputStream(stream);
|
|
||||||
} catch (Exception e) {
|
|
||||||
throw new KafkaException(e);
|
|
||||||
}
|
|
||||||
default:
|
|
||||||
throw new IllegalArgumentException("Unknown compression type: " + type);
|
|
||||||
}
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new KafkaException(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public static DataInputStream wrapForInput(ByteBufferInputStream buffer, CompressionType type, byte messageVersion) {
|
|
||||||
try {
|
|
||||||
switch (type) {
|
|
||||||
case NONE:
|
|
||||||
return new DataInputStream(buffer);
|
|
||||||
case GZIP:
|
|
||||||
return new DataInputStream(new GZIPInputStream(buffer));
|
|
||||||
case SNAPPY:
|
|
||||||
try {
|
|
||||||
InputStream stream = (InputStream) snappyInputStreamSupplier.get().newInstance(buffer);
|
|
||||||
return new DataInputStream(stream);
|
|
||||||
} catch (Exception e) {
|
|
||||||
throw new KafkaException(e);
|
|
||||||
}
|
|
||||||
case LZ4:
|
|
||||||
try {
|
|
||||||
InputStream stream = (InputStream) lz4InputStreamSupplier.get().newInstance(buffer,
|
|
||||||
messageVersion == Record.MAGIC_VALUE_V0);
|
|
||||||
return new DataInputStream(stream);
|
|
||||||
} catch (Exception e) {
|
|
||||||
throw new KafkaException(e);
|
|
||||||
}
|
|
||||||
default:
|
|
||||||
throw new IllegalArgumentException("Unknown compression type: " + type);
|
|
||||||
}
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new KafkaException(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private interface ConstructorSupplier {
|
|
||||||
Constructor get() throws ClassNotFoundException, NoSuchMethodException;
|
|
||||||
}
|
|
||||||
|
|
||||||
// this code is based on Guava's @see{com.google.common.base.Suppliers.MemoizingSupplier}
|
|
||||||
private static class MemoizingConstructorSupplier {
|
|
||||||
final ConstructorSupplier delegate;
|
|
||||||
transient volatile boolean initialized;
|
|
||||||
transient Constructor value;
|
|
||||||
|
|
||||||
public MemoizingConstructorSupplier(ConstructorSupplier delegate) {
|
|
||||||
this.delegate = delegate;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Constructor get() throws NoSuchMethodException, ClassNotFoundException {
|
|
||||||
if (!initialized) {
|
|
||||||
synchronized (this) {
|
|
||||||
if (!initialized) {
|
|
||||||
Constructor constructor = delegate.get();
|
|
||||||
value = constructor;
|
|
||||||
initialized = true;
|
|
||||||
return constructor;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return value;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
@ -0,0 +1,166 @@
|
||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p/>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p/>
|
||||||
|
* 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.KafkaException;
|
||||||
|
import org.apache.kafka.common.errors.CorruptRecordException;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.nio.channels.FileChannel;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A log input stream which is backed by a {@link FileChannel}.
|
||||||
|
*/
|
||||||
|
public class FileLogInputStream implements LogInputStream<FileLogInputStream.FileChannelLogEntry> {
|
||||||
|
private int position;
|
||||||
|
private final int end;
|
||||||
|
private final FileChannel channel;
|
||||||
|
private final int maxRecordSize;
|
||||||
|
private final ByteBuffer logHeaderBuffer = ByteBuffer.allocate(Records.LOG_OVERHEAD);
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a new log input stream over the FileChannel
|
||||||
|
* @param channel Underlying FileChannel
|
||||||
|
* @param maxRecordSize Maximum size of records
|
||||||
|
* @param start Position in the file channel to start from
|
||||||
|
* @param end Position in the file channel not to read past
|
||||||
|
*/
|
||||||
|
public FileLogInputStream(FileChannel channel,
|
||||||
|
int maxRecordSize,
|
||||||
|
int start,
|
||||||
|
int end) {
|
||||||
|
this.channel = channel;
|
||||||
|
this.maxRecordSize = maxRecordSize;
|
||||||
|
this.position = start;
|
||||||
|
this.end = end;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public FileChannelLogEntry nextEntry() throws IOException {
|
||||||
|
if (position + Records.LOG_OVERHEAD >= end)
|
||||||
|
return null;
|
||||||
|
|
||||||
|
logHeaderBuffer.rewind();
|
||||||
|
channel.read(logHeaderBuffer, position);
|
||||||
|
if (logHeaderBuffer.hasRemaining())
|
||||||
|
return null;
|
||||||
|
|
||||||
|
logHeaderBuffer.rewind();
|
||||||
|
long offset = logHeaderBuffer.getLong();
|
||||||
|
int size = logHeaderBuffer.getInt();
|
||||||
|
|
||||||
|
if (size < Record.RECORD_OVERHEAD_V0)
|
||||||
|
throw new CorruptRecordException(String.format("Record size is smaller than minimum record overhead (%d).", Record.RECORD_OVERHEAD_V0));
|
||||||
|
|
||||||
|
if (size > maxRecordSize)
|
||||||
|
throw new CorruptRecordException(String.format("Record size exceeds the largest allowable message size (%d).", maxRecordSize));
|
||||||
|
|
||||||
|
if (position + Records.LOG_OVERHEAD + size > end)
|
||||||
|
return null;
|
||||||
|
|
||||||
|
FileChannelLogEntry logEntry = new FileChannelLogEntry(offset, channel, position, size);
|
||||||
|
position += logEntry.sizeInBytes();
|
||||||
|
return logEntry;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Log entry backed by an underlying FileChannel. This allows iteration over the shallow log
|
||||||
|
* entries without needing to read the record data into memory until it is needed. The downside
|
||||||
|
* is that entries will generally no longer be readable when the underlying channel is closed.
|
||||||
|
*/
|
||||||
|
public static class FileChannelLogEntry extends LogEntry {
|
||||||
|
private final long offset;
|
||||||
|
private final FileChannel channel;
|
||||||
|
private final int position;
|
||||||
|
private final int recordSize;
|
||||||
|
private Record record = null;
|
||||||
|
|
||||||
|
private FileChannelLogEntry(long offset,
|
||||||
|
FileChannel channel,
|
||||||
|
int position,
|
||||||
|
int recordSize) {
|
||||||
|
this.offset = offset;
|
||||||
|
this.channel = channel;
|
||||||
|
this.position = position;
|
||||||
|
this.recordSize = recordSize;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long offset() {
|
||||||
|
return offset;
|
||||||
|
}
|
||||||
|
|
||||||
|
public int position() {
|
||||||
|
return position;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public byte magic() {
|
||||||
|
if (record != null)
|
||||||
|
return record.magic();
|
||||||
|
|
||||||
|
try {
|
||||||
|
byte[] magic = new byte[1];
|
||||||
|
ByteBuffer buf = ByteBuffer.wrap(magic);
|
||||||
|
channel.read(buf, position + Records.LOG_OVERHEAD + Record.MAGIC_OFFSET);
|
||||||
|
if (buf.hasRemaining())
|
||||||
|
throw new KafkaException("Failed to read magic byte from FileChannel " + channel);
|
||||||
|
return magic[0];
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new KafkaException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Force load the record and its data (key and value) into memory.
|
||||||
|
* @return The resulting record
|
||||||
|
* @throws IOException for any IO errors reading from the underlying file
|
||||||
|
*/
|
||||||
|
private Record loadRecord() throws IOException {
|
||||||
|
if (record != null)
|
||||||
|
return record;
|
||||||
|
|
||||||
|
ByteBuffer recordBuffer = ByteBuffer.allocate(recordSize);
|
||||||
|
channel.read(recordBuffer, position + Records.LOG_OVERHEAD);
|
||||||
|
if (recordBuffer.hasRemaining())
|
||||||
|
throw new IOException("Failed to read full record from channel " + channel);
|
||||||
|
|
||||||
|
recordBuffer.rewind();
|
||||||
|
record = new Record(recordBuffer);
|
||||||
|
return record;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Record record() {
|
||||||
|
if (record != null)
|
||||||
|
return record;
|
||||||
|
|
||||||
|
try {
|
||||||
|
return loadRecord();
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new KafkaException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int sizeInBytes() {
|
||||||
|
return Records.LOG_OVERHEAD + recordSize;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -18,22 +18,31 @@ package org.apache.kafka.common.record;
|
||||||
|
|
||||||
import org.apache.kafka.common.KafkaException;
|
import org.apache.kafka.common.KafkaException;
|
||||||
import org.apache.kafka.common.network.TransportLayer;
|
import org.apache.kafka.common.network.TransportLayer;
|
||||||
|
import org.apache.kafka.common.record.FileLogInputStream.FileChannelLogEntry;
|
||||||
|
import org.apache.kafka.common.utils.Utils;
|
||||||
|
|
||||||
|
import java.io.Closeable;
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
|
import java.io.FileInputStream;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.io.RandomAccessFile;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.nio.channels.FileChannel;
|
import java.nio.channels.FileChannel;
|
||||||
import java.nio.channels.GatheringByteChannel;
|
import java.nio.channels.GatheringByteChannel;
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* File-backed record set.
|
* A {@link Records} implementation backed by a file. An optional start and end position can be applied to this
|
||||||
|
* instance to enable slicing a range of the log records.
|
||||||
*/
|
*/
|
||||||
public class FileRecords implements Records {
|
public class FileRecords extends AbstractRecords implements Closeable {
|
||||||
private final File file;
|
private final boolean isSlice;
|
||||||
private final FileChannel channel;
|
private final FileChannel channel;
|
||||||
private final long start;
|
private final int start;
|
||||||
private final long end;
|
private final int end;
|
||||||
private final long size;
|
private volatile File file;
|
||||||
|
private final AtomicInteger size;
|
||||||
|
|
||||||
public FileRecords(File file,
|
public FileRecords(File file,
|
||||||
FileChannel channel,
|
FileChannel channel,
|
||||||
|
|
@ -44,83 +53,435 @@ public class FileRecords implements Records {
|
||||||
this.channel = channel;
|
this.channel = channel;
|
||||||
this.start = start;
|
this.start = start;
|
||||||
this.end = end;
|
this.end = end;
|
||||||
|
this.isSlice = isSlice;
|
||||||
|
this.size = new AtomicInteger();
|
||||||
|
|
||||||
if (isSlice)
|
// set the initial size of the buffer
|
||||||
this.size = end - start;
|
resize();
|
||||||
else
|
}
|
||||||
this.size = Math.min(channel.size(), end) - start;
|
|
||||||
|
public void resize() throws IOException {
|
||||||
|
if (isSlice) {
|
||||||
|
size.set(end - start);
|
||||||
|
} else {
|
||||||
|
int limit = Math.min((int) channel.size(), end);
|
||||||
|
size.set(limit - start);
|
||||||
|
|
||||||
|
// if this is not a slice, update the file pointer to the end of the file
|
||||||
|
// set the file position to the last byte in the file
|
||||||
|
channel.position(limit);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int sizeInBytes() {
|
public int sizeInBytes() {
|
||||||
return (int) size;
|
return size.get();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the underlying file.
|
||||||
|
* @return The file
|
||||||
|
*/
|
||||||
|
public File file() {
|
||||||
|
return file;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the underlying file channel.
|
||||||
|
* @return The file channel
|
||||||
|
*/
|
||||||
|
public FileChannel channel() {
|
||||||
|
return channel;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Read log entries into a given buffer.
|
||||||
|
* @param buffer The buffer to write the entries to
|
||||||
|
* @param position Position in the buffer to read from
|
||||||
|
* @return The same buffer
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public ByteBuffer readInto(ByteBuffer buffer, int position) throws IOException {
|
||||||
|
channel.read(buffer, position + this.start);
|
||||||
|
buffer.flip();
|
||||||
|
return buffer;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return a slice of records from this instance, which is a view into this set starting from the given position
|
||||||
|
* and with the given size limit.
|
||||||
|
*
|
||||||
|
* If the size is beyond the end of the file, the end will be based on the size of the file at the time of the read.
|
||||||
|
*
|
||||||
|
* If this message set is already sliced, the position will be taken relative to that slicing.
|
||||||
|
*
|
||||||
|
* @param position The start position to begin the read from
|
||||||
|
* @param size The number of bytes after the start position to include
|
||||||
|
* @return A sliced wrapper on this message set limited based on the given position and size
|
||||||
|
*/
|
||||||
|
public FileRecords read(int position, int size) throws IOException {
|
||||||
|
if (position < 0)
|
||||||
|
throw new IllegalArgumentException("Invalid position: " + position);
|
||||||
|
if (size < 0)
|
||||||
|
throw new IllegalArgumentException("Invalid size: " + size);
|
||||||
|
|
||||||
|
final int end;
|
||||||
|
if (this.start + position + size < 0)
|
||||||
|
end = sizeInBytes();
|
||||||
|
else
|
||||||
|
end = Math.min(this.start + position + size, sizeInBytes());
|
||||||
|
return new FileRecords(file, channel, this.start + position, end, true);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Append log entries to the buffer
|
||||||
|
* @param records The records to append
|
||||||
|
* @return the number of bytes written to the underlying file
|
||||||
|
*/
|
||||||
|
public int append(MemoryRecords records) throws IOException {
|
||||||
|
int written = records.writeFullyTo(channel);
|
||||||
|
size.getAndAdd(written);
|
||||||
|
return written;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Commit all written data to the physical disk
|
||||||
|
*/
|
||||||
|
public void flush() throws IOException {
|
||||||
|
channel.force(true);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Close this record set
|
||||||
|
*/
|
||||||
|
public void close() throws IOException {
|
||||||
|
flush();
|
||||||
|
trim();
|
||||||
|
channel.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Delete this message set from the filesystem
|
||||||
|
* @return True iff this message set was deleted.
|
||||||
|
*/
|
||||||
|
public boolean delete() {
|
||||||
|
Utils.closeQuietly(channel, "FileChannel");
|
||||||
|
return file.delete();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Trim file when close or roll to next file
|
||||||
|
*/
|
||||||
|
public void trim() throws IOException {
|
||||||
|
truncateTo(sizeInBytes());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Update the file reference (to be used with caution since this does not reopen the file channel)
|
||||||
|
* @param file The new file to use
|
||||||
|
*/
|
||||||
|
public void setFile(File file) {
|
||||||
|
this.file = file;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Rename the file that backs this message set
|
||||||
|
* @throws IOException if rename fails.
|
||||||
|
*/
|
||||||
|
public void renameTo(File f) throws IOException {
|
||||||
|
try {
|
||||||
|
Utils.atomicMoveWithFallback(file.toPath(), f.toPath());
|
||||||
|
} finally {
|
||||||
|
this.file = f;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Truncate this file message set to the given size in bytes. Note that this API does no checking that the
|
||||||
|
* given size falls on a valid message boundary.
|
||||||
|
* In some versions of the JDK truncating to the same size as the file message set will cause an
|
||||||
|
* update of the files mtime, so truncate is only performed if the targetSize is smaller than the
|
||||||
|
* size of the underlying FileChannel.
|
||||||
|
* It is expected that no other threads will do writes to the log when this function is called.
|
||||||
|
* @param targetSize The size to truncate to. Must be between 0 and sizeInBytes.
|
||||||
|
* @return The number of bytes truncated off
|
||||||
|
*/
|
||||||
|
public int truncateTo(int targetSize) throws IOException {
|
||||||
|
int originalSize = sizeInBytes();
|
||||||
|
if (targetSize > originalSize || targetSize < 0)
|
||||||
|
throw new KafkaException("Attempt to truncate log segment to " + targetSize + " bytes failed, " +
|
||||||
|
" size of this log segment is " + originalSize + " bytes.");
|
||||||
|
if (targetSize < (int) channel.size()) {
|
||||||
|
channel.truncate(targetSize);
|
||||||
|
channel.position(targetSize);
|
||||||
|
size.set(targetSize);
|
||||||
|
}
|
||||||
|
return originalSize - targetSize;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long writeTo(GatheringByteChannel destChannel, long offset, int length) throws IOException {
|
public long writeTo(GatheringByteChannel destChannel, long offset, int length) throws IOException {
|
||||||
long newSize = Math.min(channel.size(), end) - start;
|
long newSize = Math.min(channel.size(), end) - start;
|
||||||
if (newSize < size)
|
if (newSize < size.get())
|
||||||
throw new KafkaException(String.format("Size of FileRecords %s has been truncated during write: old size %d, new size %d", file.getAbsolutePath(), size, newSize));
|
throw new KafkaException(String.format("Size of FileRecords %s has been truncated during write: old size %d, new size %d", file.getAbsolutePath(), size, newSize));
|
||||||
|
|
||||||
if (offset > size)
|
|
||||||
throw new KafkaException(String.format("The requested offset %d is out of range. The size of this FileRecords is %d.", offset, size));
|
|
||||||
|
|
||||||
long position = start + offset;
|
long position = start + offset;
|
||||||
long count = Math.min(length, this.size - offset);
|
long count = Math.min(length, size.get());
|
||||||
|
final long bytesTransferred;
|
||||||
if (destChannel instanceof TransportLayer) {
|
if (destChannel instanceof TransportLayer) {
|
||||||
TransportLayer tl = (TransportLayer) destChannel;
|
TransportLayer tl = (TransportLayer) destChannel;
|
||||||
return tl.transferFrom(this.channel, position, count);
|
bytesTransferred = tl.transferFrom(channel, position, count);
|
||||||
} else {
|
} else {
|
||||||
return this.channel.transferTo(position, count, destChannel);
|
bytesTransferred = channel.transferTo(position, count, destChannel);
|
||||||
}
|
}
|
||||||
|
return bytesTransferred;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Search forward for the file position of the last offset that is greater than or equal to the target offset
|
||||||
|
* and return its physical position and the size of the message (including log overhead) at the returned offset. If
|
||||||
|
* no such offsets are found, return null.
|
||||||
|
*
|
||||||
|
* @param targetOffset The offset to search for.
|
||||||
|
* @param startingPosition The starting position in the file to begin searching from.
|
||||||
|
*/
|
||||||
|
public LogEntryPosition searchForOffsetWithSize(long targetOffset, int startingPosition) {
|
||||||
|
Iterator<FileChannelLogEntry> iterator = shallowIteratorFrom(Integer.MAX_VALUE, startingPosition);
|
||||||
|
while (iterator.hasNext()) {
|
||||||
|
FileChannelLogEntry entry = iterator.next();
|
||||||
|
long offset = entry.offset();
|
||||||
|
if (offset >= targetOffset)
|
||||||
|
return new LogEntryPosition(offset, entry.position(), entry.sizeInBytes());
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Search forward for the message whose timestamp is greater than or equals to the target timestamp.
|
||||||
|
*
|
||||||
|
* @param targetTimestamp The timestamp to search for.
|
||||||
|
* @param startingPosition The starting position to search.
|
||||||
|
* @return The timestamp and offset of the message found. None, if no message is found.
|
||||||
|
*/
|
||||||
|
public TimestampAndOffset searchForTimestamp(long targetTimestamp, int startingPosition) {
|
||||||
|
Iterator<FileChannelLogEntry> shallowIterator = shallowIteratorFrom(startingPosition);
|
||||||
|
while (shallowIterator.hasNext()) {
|
||||||
|
LogEntry shallowEntry = shallowIterator.next();
|
||||||
|
Record shallowRecord = shallowEntry.record();
|
||||||
|
if (shallowRecord.timestamp() >= targetTimestamp) {
|
||||||
|
// We found a message
|
||||||
|
for (LogEntry deepLogEntry : shallowEntry) {
|
||||||
|
long timestamp = deepLogEntry.record().timestamp();
|
||||||
|
if (timestamp >= targetTimestamp)
|
||||||
|
return new TimestampAndOffset(timestamp, deepLogEntry.offset());
|
||||||
|
}
|
||||||
|
throw new IllegalStateException(String.format("The message set (max timestamp = %s, max offset = %s" +
|
||||||
|
" should contain target timestamp %s, but does not.", shallowRecord.timestamp(),
|
||||||
|
shallowEntry.offset(), targetTimestamp));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the largest timestamp of the messages after a given position in this file message set.
|
||||||
|
* @param startingPosition The starting position.
|
||||||
|
* @return The largest timestamp of the messages after the given position.
|
||||||
|
*/
|
||||||
|
public TimestampAndOffset largestTimestampAfter(int startingPosition) {
|
||||||
|
long maxTimestamp = Record.NO_TIMESTAMP;
|
||||||
|
long offsetOfMaxTimestamp = -1L;
|
||||||
|
|
||||||
|
Iterator<FileChannelLogEntry> shallowIterator = shallowIteratorFrom(startingPosition);
|
||||||
|
while (shallowIterator.hasNext()) {
|
||||||
|
LogEntry shallowEntry = shallowIterator.next();
|
||||||
|
long timestamp = shallowEntry.record().timestamp();
|
||||||
|
if (timestamp > maxTimestamp) {
|
||||||
|
maxTimestamp = timestamp;
|
||||||
|
offsetOfMaxTimestamp = shallowEntry.offset();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return new TimestampAndOffset(maxTimestamp, offsetOfMaxTimestamp);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get an iterator over the shallow entries in the file. Note that the entries are
|
||||||
|
* backed by the open file channel. When the channel is closed (i.e. when this instance
|
||||||
|
* is closed), the entries will generally no longer be readable.
|
||||||
|
* @return An iterator over the shallow entries
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public Iterator<FileChannelLogEntry> shallowIterator() {
|
||||||
|
return shallowIteratorFrom(start);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get an iterator over the shallow entries, enforcing a maximum record size
|
||||||
|
* @param maxRecordSize The maximum allowable size of individual records (including compressed record sets)
|
||||||
|
* @return An iterator over the shallow entries
|
||||||
|
*/
|
||||||
|
public Iterator<FileChannelLogEntry> shallowIterator(int maxRecordSize) {
|
||||||
|
return shallowIteratorFrom(maxRecordSize, start);
|
||||||
|
}
|
||||||
|
|
||||||
|
private Iterator<FileChannelLogEntry> shallowIteratorFrom(int start) {
|
||||||
|
return shallowIteratorFrom(Integer.MAX_VALUE, start);
|
||||||
|
}
|
||||||
|
|
||||||
|
private Iterator<FileChannelLogEntry> shallowIteratorFrom(int maxRecordSize, int start) {
|
||||||
|
final int end;
|
||||||
|
if (isSlice)
|
||||||
|
end = this.end;
|
||||||
|
else
|
||||||
|
end = this.sizeInBytes();
|
||||||
|
FileLogInputStream inputStream = new FileLogInputStream(channel, maxRecordSize, start, end);
|
||||||
|
return RecordsIterator.shallowIterator(inputStream);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public RecordsIterator iterator() {
|
public Iterator<LogEntry> deepIterator() {
|
||||||
return new RecordsIterator(new FileLogInputStream(channel, start, end), false);
|
final int end;
|
||||||
|
if (isSlice)
|
||||||
|
end = this.end;
|
||||||
|
else
|
||||||
|
end = this.sizeInBytes();
|
||||||
|
FileLogInputStream inputStream = new FileLogInputStream(channel, Integer.MAX_VALUE, start, end);
|
||||||
|
return new RecordsIterator(inputStream, false, false, Integer.MAX_VALUE);
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class FileLogInputStream implements LogInputStream {
|
public static FileRecords open(File file,
|
||||||
private long position;
|
boolean mutable,
|
||||||
protected final long end;
|
boolean fileAlreadyExists,
|
||||||
protected final FileChannel channel;
|
int initFileSize,
|
||||||
private final ByteBuffer logHeaderBuffer = ByteBuffer.allocate(Records.LOG_OVERHEAD);
|
boolean preallocate) throws IOException {
|
||||||
|
FileChannel channel = openChannel(file, mutable, fileAlreadyExists, initFileSize, preallocate);
|
||||||
|
int end = (!fileAlreadyExists && preallocate) ? 0 : Integer.MAX_VALUE;
|
||||||
|
return new FileRecords(file, channel, 0, end, false);
|
||||||
|
}
|
||||||
|
|
||||||
public FileLogInputStream(FileChannel channel, long start, long end) {
|
public static FileRecords open(File file,
|
||||||
this.channel = channel;
|
boolean fileAlreadyExists,
|
||||||
this.position = start;
|
int initFileSize,
|
||||||
this.end = end;
|
boolean preallocate) throws IOException {
|
||||||
|
return open(file, true, fileAlreadyExists, initFileSize, preallocate);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static FileRecords open(File file, boolean mutable) throws IOException {
|
||||||
|
return open(file, mutable, false, 0, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static FileRecords open(File file) throws IOException {
|
||||||
|
return open(file, true);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Open a channel for the given file
|
||||||
|
* For windows NTFS and some old LINUX file system, set preallocate to true and initFileSize
|
||||||
|
* with one value (for example 512 * 1025 *1024 ) can improve the kafka produce performance.
|
||||||
|
* @param file File path
|
||||||
|
* @param mutable mutable
|
||||||
|
* @param fileAlreadyExists File already exists or not
|
||||||
|
* @param initFileSize The size used for pre allocate file, for example 512 * 1025 *1024
|
||||||
|
* @param preallocate Pre allocate file or not, gotten from configuration.
|
||||||
|
*/
|
||||||
|
private static FileChannel openChannel(File file,
|
||||||
|
boolean mutable,
|
||||||
|
boolean fileAlreadyExists,
|
||||||
|
int initFileSize,
|
||||||
|
boolean preallocate) throws IOException {
|
||||||
|
if (mutable) {
|
||||||
|
if (fileAlreadyExists) {
|
||||||
|
return new RandomAccessFile(file, "rw").getChannel();
|
||||||
|
} else {
|
||||||
|
if (preallocate) {
|
||||||
|
RandomAccessFile randomAccessFile = new RandomAccessFile(file, "rw");
|
||||||
|
randomAccessFile.setLength(initFileSize);
|
||||||
|
return randomAccessFile.getChannel();
|
||||||
|
} else {
|
||||||
|
return new RandomAccessFile(file, "rw").getChannel();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
return new FileInputStream(file).getChannel();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class LogEntryPosition {
|
||||||
|
public final long offset;
|
||||||
|
public final int position;
|
||||||
|
public final int size;
|
||||||
|
|
||||||
|
public LogEntryPosition(long offset, int position, int size) {
|
||||||
|
this.offset = offset;
|
||||||
|
this.position = position;
|
||||||
|
this.size = size;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public LogEntry nextEntry() throws IOException {
|
public boolean equals(Object o) {
|
||||||
if (position + Records.LOG_OVERHEAD >= end)
|
if (this == o) return true;
|
||||||
return null;
|
if (o == null || getClass() != o.getClass()) return false;
|
||||||
|
|
||||||
logHeaderBuffer.rewind();
|
LogEntryPosition that = (LogEntryPosition) o;
|
||||||
channel.read(logHeaderBuffer, position);
|
|
||||||
if (logHeaderBuffer.hasRemaining())
|
|
||||||
return null;
|
|
||||||
|
|
||||||
logHeaderBuffer.rewind();
|
if (offset != that.offset) return false;
|
||||||
long offset = logHeaderBuffer.getLong();
|
if (position != that.position) return false;
|
||||||
int size = logHeaderBuffer.getInt();
|
return size == that.size;
|
||||||
if (size < 0)
|
|
||||||
throw new IllegalStateException("Record with size " + size);
|
|
||||||
|
|
||||||
if (position + Records.LOG_OVERHEAD + size > end)
|
}
|
||||||
return null;
|
|
||||||
|
|
||||||
ByteBuffer recordBuffer = ByteBuffer.allocate(size);
|
@Override
|
||||||
channel.read(recordBuffer, position + Records.LOG_OVERHEAD);
|
public int hashCode() {
|
||||||
if (recordBuffer.hasRemaining())
|
int result = (int) (offset ^ (offset >>> 32));
|
||||||
return null;
|
result = 31 * result + position;
|
||||||
recordBuffer.rewind();
|
result = 31 * result + size;
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
Record record = new Record(recordBuffer);
|
@Override
|
||||||
LogEntry logEntry = new LogEntry(offset, record);
|
public String toString() {
|
||||||
position += logEntry.size();
|
return "LogEntryPosition(" +
|
||||||
return logEntry;
|
"offset=" + offset +
|
||||||
|
", position=" + position +
|
||||||
|
", size=" + size +
|
||||||
|
')';
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static class TimestampAndOffset {
|
||||||
|
public final long timestamp;
|
||||||
|
public final long offset;
|
||||||
|
|
||||||
|
public TimestampAndOffset(long timestamp, long offset) {
|
||||||
|
this.timestamp = timestamp;
|
||||||
|
this.offset = offset;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o) {
|
||||||
|
if (this == o) return true;
|
||||||
|
if (o == null || getClass() != o.getClass()) return false;
|
||||||
|
|
||||||
|
TimestampAndOffset that = (TimestampAndOffset) o;
|
||||||
|
|
||||||
|
if (timestamp != that.timestamp) return false;
|
||||||
|
return offset == that.offset;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
int result = (int) (timestamp ^ (timestamp >>> 32));
|
||||||
|
result = 31 * result + (int) (offset ^ (offset >>> 32));
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return "TimestampAndOffset(" +
|
||||||
|
"timestamp=" + timestamp +
|
||||||
|
", offset=" + offset +
|
||||||
|
')';
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -16,9 +16,9 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.common.record;
|
package org.apache.kafka.common.record;
|
||||||
|
|
||||||
import org.apache.kafka.common.KafkaException;
|
import org.apache.kafka.common.errors.CorruptRecordException;
|
||||||
|
|
||||||
public class InvalidRecordException extends KafkaException {
|
public class InvalidRecordException extends CorruptRecordException {
|
||||||
|
|
||||||
private static final long serialVersionUID = 1;
|
private static final long serialVersionUID = 1;
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -16,33 +16,156 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.common.record;
|
package org.apache.kafka.common.record;
|
||||||
|
|
||||||
|
import java.io.DataOutputStream;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.Iterator;
|
||||||
|
|
||||||
|
import static org.apache.kafka.common.record.Records.LOG_OVERHEAD;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* An offset and record pair
|
* An offset and record pair
|
||||||
*/
|
*/
|
||||||
public final class LogEntry {
|
public abstract class LogEntry implements Iterable<LogEntry> {
|
||||||
|
|
||||||
private final long offset;
|
/**
|
||||||
private final Record record;
|
* Get the offset of this entry. Note that if this entry contains a compressed
|
||||||
|
* message set, then this offset will be the last offset of the nested entries
|
||||||
|
* @return the last offset contained in this entry
|
||||||
|
*/
|
||||||
|
public abstract long offset();
|
||||||
|
|
||||||
public LogEntry(long offset, Record record) {
|
/**
|
||||||
this.offset = offset;
|
* Get the shallow record for this log entry.
|
||||||
this.record = record;
|
* @return the shallow record
|
||||||
|
*/
|
||||||
|
public abstract Record record();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the first offset of the records contained in this entry. Note that this
|
||||||
|
* generally requires deep iteration, which requires decompression, so this should
|
||||||
|
* be used with caution.
|
||||||
|
* @return The first offset contained in this entry
|
||||||
|
*/
|
||||||
|
public long firstOffset() {
|
||||||
|
return iterator().next().offset();
|
||||||
}
|
}
|
||||||
|
|
||||||
public long offset() {
|
/**
|
||||||
return this.offset;
|
* Get the offset following this entry (i.e. the last offset contained in this entry plus one).
|
||||||
|
* @return the next consecutive offset following this entry
|
||||||
|
*/
|
||||||
|
public long nextOffset() {
|
||||||
|
return offset() + 1;
|
||||||
}
|
}
|
||||||
|
|
||||||
public Record record() {
|
/**
|
||||||
return this.record;
|
* Get the message format version of this entry (i.e its magic value).
|
||||||
|
* @return the magic byte
|
||||||
|
*/
|
||||||
|
public byte magic() {
|
||||||
|
return record().magic();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
return "LogEntry(" + offset + ", " + record + ")";
|
return "LogEntry(" + offset() + ", " + record() + ")";
|
||||||
}
|
}
|
||||||
|
|
||||||
public int size() {
|
/**
|
||||||
return record.size() + Records.LOG_OVERHEAD;
|
* Get the size in bytes of this entry, including the size of the record and the log overhead.
|
||||||
|
* @return The size in bytes of this entry
|
||||||
|
*/
|
||||||
|
public int sizeInBytes() {
|
||||||
|
return record().sizeInBytes() + LOG_OVERHEAD;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Check whether this entry contains a compressed message set.
|
||||||
|
* @return true if so, false otherwise
|
||||||
|
*/
|
||||||
|
public boolean isCompressed() {
|
||||||
|
return record().compressionType() != CompressionType.NONE;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write this entry into a buffer.
|
||||||
|
* @param buffer The buffer to write the entry to
|
||||||
|
*/
|
||||||
|
public void writeTo(ByteBuffer buffer) {
|
||||||
|
writeHeader(buffer, offset(), record().sizeInBytes());
|
||||||
|
buffer.put(record().buffer().duplicate());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get an iterator for the nested entries contained within this log entry. Note that
|
||||||
|
* if the entry is not compressed, then this method will return an iterator over the
|
||||||
|
* shallow entry only (i.e. this object).
|
||||||
|
* @return An iterator over the entries contained within this log entry
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public Iterator<LogEntry> iterator() {
|
||||||
|
if (isCompressed())
|
||||||
|
return new RecordsIterator.DeepRecordsIterator(this, false, Integer.MAX_VALUE);
|
||||||
|
return Collections.singletonList(this).iterator();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean equals(Object o) {
|
||||||
|
if (this == o) return true;
|
||||||
|
if (o == null || !(o instanceof LogEntry)) return false;
|
||||||
|
|
||||||
|
LogEntry that = (LogEntry) o;
|
||||||
|
|
||||||
|
if (offset() != that.offset()) return false;
|
||||||
|
Record thisRecord = record();
|
||||||
|
Record thatRecord = that.record();
|
||||||
|
return thisRecord != null ? thisRecord.equals(thatRecord) : thatRecord == null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
long offset = offset();
|
||||||
|
Record record = record();
|
||||||
|
int result = (int) (offset ^ (offset >>> 32));
|
||||||
|
result = 31 * result + (record != null ? record.hashCode() : 0);
|
||||||
|
return result;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void writeHeader(ByteBuffer buffer, long offset, int size) {
|
||||||
|
buffer.putLong(offset);
|
||||||
|
buffer.putInt(size);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static void writeHeader(DataOutputStream out, long offset, int size) throws IOException {
|
||||||
|
out.writeLong(offset);
|
||||||
|
out.writeInt(size);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class SimpleLogEntry extends LogEntry {
|
||||||
|
private final long offset;
|
||||||
|
private final Record record;
|
||||||
|
|
||||||
|
public SimpleLogEntry(long offset, Record record) {
|
||||||
|
this.offset = offset;
|
||||||
|
this.record = record;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long offset() {
|
||||||
|
return offset;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Record record() {
|
||||||
|
return record;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
public static LogEntry create(long offset, Record record) {
|
||||||
|
return new SimpleLogEntry(offset, record);
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -20,10 +20,14 @@ import java.io.IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* An abstraction between an underlying input stream and record iterators, a LogInputStream
|
* An abstraction between an underlying input stream and record iterators, a LogInputStream
|
||||||
* returns only the shallow log entries, depending on {@link org.apache.kafka.common.record.RecordsIterator.DeepRecordsIterator}
|
* returns only the shallow log entries, depending on {@link RecordsIterator.DeepRecordsIterator}
|
||||||
* for the deep iteration.
|
* for the deep iteration. The generic typing allows for implementations which present only
|
||||||
|
* a view of the log entries, which enables more efficient iteration when the record data is
|
||||||
|
* not actually needed. See for example {@link org.apache.kafka.common.record.FileLogInputStream.FileChannelLogEntry}
|
||||||
|
* in which the record is not brought into memory until needed.
|
||||||
|
* @param <T> Type parameter of the log entry
|
||||||
*/
|
*/
|
||||||
interface LogInputStream {
|
interface LogInputStream<T extends LogEntry> {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get the next log entry from the underlying input stream.
|
* Get the next log entry from the underlying input stream.
|
||||||
|
|
@ -31,5 +35,5 @@ interface LogInputStream {
|
||||||
* @return The next log entry or null if there is none
|
* @return The next log entry or null if there is none
|
||||||
* @throws IOException for any IO errors
|
* @throws IOException for any IO errors
|
||||||
*/
|
*/
|
||||||
LogEntry nextEntry() throws IOException;
|
T nextEntry() throws IOException;
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -12,197 +12,185 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.common.record;
|
package org.apache.kafka.common.record;
|
||||||
|
|
||||||
import java.io.DataInputStream;
|
import org.apache.kafka.common.record.ByteBufferLogInputStream.ByteBufferLogEntry;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.nio.channels.GatheringByteChannel;
|
import java.nio.channels.GatheringByteChannel;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A {@link Records} implementation backed by a ByteBuffer.
|
* A {@link Records} implementation backed by a ByteBuffer. This is used only for reading or
|
||||||
|
* modifying in-place an existing buffer of log entries. To create a new buffer see {@link MemoryRecordsBuilder},
|
||||||
|
* or one of the {@link #builder(ByteBuffer, byte, CompressionType, TimestampType) builder} variants.
|
||||||
*/
|
*/
|
||||||
public class MemoryRecords implements Records {
|
public class MemoryRecords extends AbstractRecords {
|
||||||
|
|
||||||
public final static MemoryRecords EMPTY = MemoryRecords.readableRecords(ByteBuffer.allocate(0));
|
public final static MemoryRecords EMPTY = MemoryRecords.readableRecords(ByteBuffer.allocate(0));
|
||||||
|
|
||||||
private final static int WRITE_LIMIT_FOR_READABLE_ONLY = -1;
|
|
||||||
|
|
||||||
// the compressor used for appends-only
|
|
||||||
private final Compressor compressor;
|
|
||||||
|
|
||||||
// the write limit for writable buffer, which may be smaller than the buffer capacity
|
|
||||||
private final int writeLimit;
|
|
||||||
|
|
||||||
// the capacity of the initial buffer, which is only used for de-allocation of writable records
|
|
||||||
private final int initialCapacity;
|
|
||||||
|
|
||||||
// the underlying buffer used for read; while the records are still writable it is null
|
// the underlying buffer used for read; while the records are still writable it is null
|
||||||
private ByteBuffer buffer;
|
private ByteBuffer buffer;
|
||||||
|
private int validBytes = -1;
|
||||||
// indicate if the memory records is writable or not (i.e. used for appends or read-only)
|
|
||||||
private boolean writable;
|
|
||||||
|
|
||||||
// Construct a writable memory records
|
// Construct a writable memory records
|
||||||
private MemoryRecords(ByteBuffer buffer, CompressionType type, boolean writable, int writeLimit) {
|
private MemoryRecords(ByteBuffer buffer) {
|
||||||
this.writable = writable;
|
this.buffer = buffer;
|
||||||
this.writeLimit = writeLimit;
|
|
||||||
this.initialCapacity = buffer.capacity();
|
|
||||||
if (this.writable) {
|
|
||||||
this.buffer = null;
|
|
||||||
this.compressor = new Compressor(buffer, type);
|
|
||||||
} else {
|
|
||||||
this.buffer = buffer;
|
|
||||||
this.compressor = null;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public static MemoryRecords emptyRecords(ByteBuffer buffer, CompressionType type, int writeLimit) {
|
|
||||||
return new MemoryRecords(buffer, type, true, writeLimit);
|
|
||||||
}
|
|
||||||
|
|
||||||
public static MemoryRecords emptyRecords(ByteBuffer buffer, CompressionType type) {
|
|
||||||
// use the buffer capacity as the default write limit
|
|
||||||
return emptyRecords(buffer, type, buffer.capacity());
|
|
||||||
}
|
|
||||||
|
|
||||||
public static MemoryRecords readableRecords(ByteBuffer buffer) {
|
|
||||||
return new MemoryRecords(buffer, CompressionType.NONE, false, WRITE_LIMIT_FOR_READABLE_ONLY);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Append the given record and offset to the buffer
|
|
||||||
*/
|
|
||||||
public void append(long offset, Record record) {
|
|
||||||
if (!writable)
|
|
||||||
throw new IllegalStateException("Memory records is not writable");
|
|
||||||
|
|
||||||
int size = record.size();
|
|
||||||
compressor.putLong(offset);
|
|
||||||
compressor.putInt(size);
|
|
||||||
compressor.put(record.buffer());
|
|
||||||
compressor.recordWritten(size + Records.LOG_OVERHEAD);
|
|
||||||
record.buffer().rewind();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Append a new record and offset to the buffer
|
|
||||||
* @return crc of the record
|
|
||||||
*/
|
|
||||||
public long append(long offset, long timestamp, byte[] key, byte[] value) {
|
|
||||||
if (!writable)
|
|
||||||
throw new IllegalStateException("Memory records is not writable");
|
|
||||||
|
|
||||||
int size = Record.recordSize(key, value);
|
|
||||||
compressor.putLong(offset);
|
|
||||||
compressor.putInt(size);
|
|
||||||
long crc = compressor.putRecord(timestamp, key, value);
|
|
||||||
compressor.recordWritten(size + Records.LOG_OVERHEAD);
|
|
||||||
return crc;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Check if we have room for a new record containing the given key/value pair
|
|
||||||
*
|
|
||||||
* Note that the return value is based on the estimate of the bytes written to the compressor, which may not be
|
|
||||||
* accurate if compression is really used. When this happens, the following append may cause dynamic buffer
|
|
||||||
* re-allocation in the underlying byte buffer stream.
|
|
||||||
*
|
|
||||||
* There is an exceptional case when appending a single message whose size is larger than the batch size, the
|
|
||||||
* capacity will be the message size which is larger than the write limit, i.e. the batch size. In this case
|
|
||||||
* the checking should be based on the capacity of the initialized buffer rather than the write limit in order
|
|
||||||
* to accept this single record.
|
|
||||||
*/
|
|
||||||
public boolean hasRoomFor(byte[] key, byte[] value) {
|
|
||||||
if (!this.writable)
|
|
||||||
return false;
|
|
||||||
|
|
||||||
return this.compressor.numRecordsWritten() == 0 ?
|
|
||||||
this.initialCapacity >= Records.LOG_OVERHEAD + Record.recordSize(key, value) :
|
|
||||||
this.writeLimit >= this.compressor.estimatedBytesWritten() + Records.LOG_OVERHEAD + Record.recordSize(key, value);
|
|
||||||
}
|
|
||||||
|
|
||||||
public boolean isFull() {
|
|
||||||
return !this.writable || this.writeLimit <= this.compressor.estimatedBytesWritten();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Close this batch for no more appends
|
|
||||||
*/
|
|
||||||
public void close() {
|
|
||||||
if (writable) {
|
|
||||||
// close the compressor to fill-in wrapper message metadata if necessary
|
|
||||||
compressor.close();
|
|
||||||
|
|
||||||
// flip the underlying buffer to be ready for reads
|
|
||||||
buffer = compressor.buffer();
|
|
||||||
buffer.flip();
|
|
||||||
|
|
||||||
// reset the writable flag
|
|
||||||
writable = false;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The size of this record set
|
|
||||||
*/
|
|
||||||
@Override
|
@Override
|
||||||
public int sizeInBytes() {
|
public int sizeInBytes() {
|
||||||
if (writable) {
|
return buffer.limit();
|
||||||
return compressor.buffer().position();
|
|
||||||
} else {
|
|
||||||
return buffer.limit();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public long writeTo(GatheringByteChannel channel, long offset, int length) throws IOException {
|
public long writeTo(GatheringByteChannel channel, long position, int length) throws IOException {
|
||||||
ByteBuffer dup = buffer.duplicate();
|
ByteBuffer dup = buffer.duplicate();
|
||||||
int position = (int) offset;
|
int pos = (int) position;
|
||||||
dup.position(position);
|
dup.position(pos);
|
||||||
dup.limit(position + length);
|
dup.limit(pos + length);
|
||||||
return channel.write(dup);
|
return channel.write(dup);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The compression rate of this record set
|
* Write all records to the given channel (including partial records).
|
||||||
|
* @param channel The channel to write to
|
||||||
|
* @return The number of bytes written
|
||||||
|
* @throws IOException For any IO errors writing to the channel
|
||||||
*/
|
*/
|
||||||
public double compressionRate() {
|
public int writeFullyTo(GatheringByteChannel channel) throws IOException {
|
||||||
if (compressor == null)
|
buffer.mark();
|
||||||
return 1.0;
|
int written = 0;
|
||||||
else
|
while (written < sizeInBytes())
|
||||||
return compressor.compressionRate();
|
written += channel.write(buffer);
|
||||||
|
buffer.reset();
|
||||||
|
return written;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Return the capacity of the initial buffer, for writable records
|
* The total number of bytes in this message set not including any partial, trailing messages. This
|
||||||
* it may be different from the current buffer's capacity
|
* may be smaller than what is returned by {@link #sizeInBytes()}.
|
||||||
|
* @return The number of valid bytes
|
||||||
*/
|
*/
|
||||||
public int initialCapacity() {
|
public int validBytes() {
|
||||||
return this.initialCapacity;
|
if (validBytes >= 0)
|
||||||
|
return validBytes;
|
||||||
|
|
||||||
|
int bytes = 0;
|
||||||
|
Iterator<ByteBufferLogEntry> iterator = shallowIterator();
|
||||||
|
while (iterator.hasNext())
|
||||||
|
bytes += iterator.next().sizeInBytes();
|
||||||
|
|
||||||
|
this.validBytes = bytes;
|
||||||
|
return bytes;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get the byte buffer that backs this records instance for reading
|
* Filter the records into the provided ByteBuffer.
|
||||||
|
* @param filter The filter function
|
||||||
|
* @param buffer The byte buffer to write the filtered records to
|
||||||
|
* @return A FilterResult with a summary of the output (for metrics)
|
||||||
|
*/
|
||||||
|
public FilterResult filterTo(LogEntryFilter filter, ByteBuffer buffer) {
|
||||||
|
long maxTimestamp = Record.NO_TIMESTAMP;
|
||||||
|
long shallowOffsetOfMaxTimestamp = -1L;
|
||||||
|
int messagesRead = 0;
|
||||||
|
int bytesRead = 0;
|
||||||
|
int messagesRetained = 0;
|
||||||
|
int bytesRetained = 0;
|
||||||
|
|
||||||
|
Iterator<ByteBufferLogEntry> shallowIterator = shallowIterator();
|
||||||
|
while (shallowIterator.hasNext()) {
|
||||||
|
ByteBufferLogEntry shallowEntry = shallowIterator.next();
|
||||||
|
bytesRead += shallowEntry.sizeInBytes();
|
||||||
|
|
||||||
|
// We use the absolute offset to decide whether to retain the message or not (this is handled by the
|
||||||
|
// deep iterator). Because of KAFKA-4298, we have to allow for the possibility that a previous version
|
||||||
|
// corrupted the log by writing a compressed message set with a wrapper magic value not matching the magic
|
||||||
|
// of the inner messages. This will be fixed as we recopy the messages to the destination buffer.
|
||||||
|
|
||||||
|
Record shallowRecord = shallowEntry.record();
|
||||||
|
byte shallowMagic = shallowRecord.magic();
|
||||||
|
boolean writeOriginalEntry = true;
|
||||||
|
List<LogEntry> retainedEntries = new ArrayList<>();
|
||||||
|
|
||||||
|
for (LogEntry deepEntry : shallowEntry) {
|
||||||
|
Record deepRecord = deepEntry.record();
|
||||||
|
messagesRead += 1;
|
||||||
|
|
||||||
|
if (filter.shouldRetain(deepEntry)) {
|
||||||
|
// Check for log corruption due to KAFKA-4298. If we find it, make sure that we overwrite
|
||||||
|
// the corrupted entry with correct data.
|
||||||
|
if (shallowMagic != deepRecord.magic())
|
||||||
|
writeOriginalEntry = false;
|
||||||
|
|
||||||
|
retainedEntries.add(deepEntry);
|
||||||
|
} else {
|
||||||
|
writeOriginalEntry = false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (writeOriginalEntry) {
|
||||||
|
// There are no messages compacted out and no message format conversion, write the original message set back
|
||||||
|
shallowEntry.writeTo(buffer);
|
||||||
|
messagesRetained += retainedEntries.size();
|
||||||
|
bytesRetained += shallowEntry.sizeInBytes();
|
||||||
|
|
||||||
|
if (shallowRecord.timestamp() > maxTimestamp) {
|
||||||
|
maxTimestamp = shallowRecord.timestamp();
|
||||||
|
shallowOffsetOfMaxTimestamp = shallowEntry.offset();
|
||||||
|
}
|
||||||
|
} else if (!retainedEntries.isEmpty()) {
|
||||||
|
ByteBuffer slice = buffer.slice();
|
||||||
|
MemoryRecordsBuilder builder = builderWithEntries(slice, shallowRecord.timestampType(), shallowRecord.compressionType(),
|
||||||
|
shallowRecord.timestamp(), retainedEntries);
|
||||||
|
MemoryRecords records = builder.build();
|
||||||
|
buffer.position(buffer.position() + slice.position());
|
||||||
|
messagesRetained += retainedEntries.size();
|
||||||
|
bytesRetained += records.sizeInBytes();
|
||||||
|
|
||||||
|
MemoryRecordsBuilder.RecordsInfo info = builder.info();
|
||||||
|
if (info.maxTimestamp > maxTimestamp) {
|
||||||
|
maxTimestamp = info.maxTimestamp;
|
||||||
|
shallowOffsetOfMaxTimestamp = info.shallowOffsetOfMaxTimestamp;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return new FilterResult(messagesRead, bytesRead, messagesRetained, bytesRetained, maxTimestamp, shallowOffsetOfMaxTimestamp);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the byte buffer that backs this instance for reading.
|
||||||
*/
|
*/
|
||||||
public ByteBuffer buffer() {
|
public ByteBuffer buffer() {
|
||||||
if (writable)
|
|
||||||
throw new IllegalStateException("The memory records must not be writable any more before getting its underlying buffer");
|
|
||||||
|
|
||||||
return buffer.duplicate();
|
return buffer.duplicate();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterator<LogEntry> iterator() {
|
public Iterator<ByteBufferLogEntry> shallowIterator() {
|
||||||
ByteBuffer input = this.buffer.duplicate();
|
return RecordsIterator.shallowIterator(new ByteBufferLogInputStream(buffer.duplicate(), Integer.MAX_VALUE));
|
||||||
if (writable)
|
|
||||||
// flip on a duplicate buffer for reading
|
|
||||||
input.flip();
|
|
||||||
return new RecordsIterator(new ByteBufferLogInputStream(input), false);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Iterator<LogEntry> deepIterator() {
|
||||||
|
return deepIterator(false);
|
||||||
|
}
|
||||||
|
|
||||||
|
public Iterator<LogEntry> deepIterator(boolean ensureMatchingMagic) {
|
||||||
|
return deepIterator(ensureMatchingMagic, Integer.MAX_VALUE);
|
||||||
|
}
|
||||||
|
|
||||||
|
public Iterator<LogEntry> deepIterator(boolean ensureMatchingMagic, int maxMessageSize) {
|
||||||
|
return new RecordsIterator(new ByteBufferLogInputStream(buffer.duplicate(), maxMessageSize), false,
|
||||||
|
ensureMatchingMagic, maxMessageSize);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String toString() {
|
public String toString() {
|
||||||
Iterator<LogEntry> iter = iterator();
|
Iterator<LogEntry> iter = deepIterator();
|
||||||
StringBuilder builder = new StringBuilder();
|
StringBuilder builder = new StringBuilder();
|
||||||
builder.append('[');
|
builder.append('[');
|
||||||
while (iter.hasNext()) {
|
while (iter.hasNext()) {
|
||||||
|
|
@ -214,16 +202,13 @@ public class MemoryRecords implements Records {
|
||||||
builder.append("record=");
|
builder.append("record=");
|
||||||
builder.append(entry.record());
|
builder.append(entry.record());
|
||||||
builder.append(")");
|
builder.append(")");
|
||||||
|
if (iter.hasNext())
|
||||||
|
builder.append(", ");
|
||||||
}
|
}
|
||||||
builder.append(']');
|
builder.append(']');
|
||||||
return builder.toString();
|
return builder.toString();
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Visible for testing */
|
|
||||||
public boolean isWritable() {
|
|
||||||
return writable;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean equals(Object o) {
|
public boolean equals(Object o) {
|
||||||
if (this == o) return true;
|
if (this == o) return true;
|
||||||
|
|
@ -232,7 +217,6 @@ public class MemoryRecords implements Records {
|
||||||
MemoryRecords that = (MemoryRecords) o;
|
MemoryRecords that = (MemoryRecords) o;
|
||||||
|
|
||||||
return buffer.equals(that.buffer);
|
return buffer.equals(that.buffer);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
@ -240,28 +224,153 @@ public class MemoryRecords implements Records {
|
||||||
return buffer.hashCode();
|
return buffer.hashCode();
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class ByteBufferLogInputStream implements LogInputStream {
|
public interface LogEntryFilter {
|
||||||
private final DataInputStream stream;
|
boolean shouldRetain(LogEntry entry);
|
||||||
private final ByteBuffer buffer;
|
}
|
||||||
|
|
||||||
private ByteBufferLogInputStream(ByteBuffer buffer) {
|
public static class FilterResult {
|
||||||
this.stream = new DataInputStream(new ByteBufferInputStream(buffer));
|
public final int messagesRead;
|
||||||
this.buffer = buffer;
|
public final int bytesRead;
|
||||||
}
|
public final int messagesRetained;
|
||||||
|
public final int bytesRetained;
|
||||||
|
public final long maxTimestamp;
|
||||||
|
public final long shallowOffsetOfMaxTimestamp;
|
||||||
|
|
||||||
public LogEntry nextEntry() throws IOException {
|
public FilterResult(int messagesRead,
|
||||||
long offset = stream.readLong();
|
int bytesRead,
|
||||||
int size = stream.readInt();
|
int messagesRetained,
|
||||||
if (size < 0)
|
int bytesRetained,
|
||||||
throw new IllegalStateException("Record with size " + size);
|
long maxTimestamp,
|
||||||
|
long shallowOffsetOfMaxTimestamp) {
|
||||||
ByteBuffer slice = buffer.slice();
|
this.messagesRead = messagesRead;
|
||||||
int newPos = buffer.position() + size;
|
this.bytesRead = bytesRead;
|
||||||
if (newPos > buffer.limit())
|
this.messagesRetained = messagesRetained;
|
||||||
return null;
|
this.bytesRetained = bytesRetained;
|
||||||
buffer.position(newPos);
|
this.maxTimestamp = maxTimestamp;
|
||||||
slice.limit(size);
|
this.shallowOffsetOfMaxTimestamp = shallowOffsetOfMaxTimestamp;
|
||||||
return new LogEntry(offset, new Record(slice));
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static MemoryRecordsBuilder builder(ByteBuffer buffer,
|
||||||
|
CompressionType compressionType,
|
||||||
|
TimestampType timestampType,
|
||||||
|
int writeLimit) {
|
||||||
|
return new MemoryRecordsBuilder(buffer, Record.CURRENT_MAGIC_VALUE, compressionType, timestampType, 0L, System.currentTimeMillis(), writeLimit);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static MemoryRecordsBuilder builder(ByteBuffer buffer,
|
||||||
|
byte magic,
|
||||||
|
CompressionType compressionType,
|
||||||
|
TimestampType timestampType,
|
||||||
|
long baseOffset,
|
||||||
|
long logAppendTime) {
|
||||||
|
return new MemoryRecordsBuilder(buffer, magic, compressionType, timestampType, baseOffset, logAppendTime, buffer.capacity());
|
||||||
|
}
|
||||||
|
|
||||||
|
public static MemoryRecordsBuilder builder(ByteBuffer buffer,
|
||||||
|
CompressionType compressionType,
|
||||||
|
TimestampType timestampType) {
|
||||||
|
// use the buffer capacity as the default write limit
|
||||||
|
return builder(buffer, compressionType, timestampType, buffer.capacity());
|
||||||
|
}
|
||||||
|
|
||||||
|
public static MemoryRecordsBuilder builder(ByteBuffer buffer,
|
||||||
|
byte magic,
|
||||||
|
CompressionType compressionType,
|
||||||
|
TimestampType timestampType) {
|
||||||
|
return builder(buffer, magic, compressionType, timestampType, 0L);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static MemoryRecordsBuilder builder(ByteBuffer buffer,
|
||||||
|
byte magic,
|
||||||
|
CompressionType compressionType,
|
||||||
|
TimestampType timestampType,
|
||||||
|
long baseOffset) {
|
||||||
|
return builder(buffer, magic, compressionType, timestampType, baseOffset, System.currentTimeMillis());
|
||||||
|
}
|
||||||
|
|
||||||
|
public static MemoryRecords readableRecords(ByteBuffer buffer) {
|
||||||
|
return new MemoryRecords(buffer);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static MemoryRecords withLogEntries(CompressionType compressionType, List<LogEntry> entries) {
|
||||||
|
return withLogEntries(TimestampType.CREATE_TIME, compressionType, System.currentTimeMillis(), entries);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static MemoryRecords withLogEntries(LogEntry ... entries) {
|
||||||
|
return withLogEntries(CompressionType.NONE, Arrays.asList(entries));
|
||||||
|
}
|
||||||
|
|
||||||
|
public static MemoryRecords withRecords(CompressionType compressionType, long initialOffset, List<Record> records) {
|
||||||
|
return withRecords(initialOffset, TimestampType.CREATE_TIME, compressionType, System.currentTimeMillis(), records);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static MemoryRecords withRecords(Record ... records) {
|
||||||
|
return withRecords(CompressionType.NONE, 0L, Arrays.asList(records));
|
||||||
|
}
|
||||||
|
|
||||||
|
public static MemoryRecords withRecords(long initialOffset, Record ... records) {
|
||||||
|
return withRecords(CompressionType.NONE, initialOffset, Arrays.asList(records));
|
||||||
|
}
|
||||||
|
|
||||||
|
public static MemoryRecords withRecords(CompressionType compressionType, Record ... records) {
|
||||||
|
return withRecords(compressionType, 0L, Arrays.asList(records));
|
||||||
|
}
|
||||||
|
|
||||||
|
public static MemoryRecords withRecords(TimestampType timestampType, CompressionType compressionType, Record ... records) {
|
||||||
|
return withRecords(0L, timestampType, compressionType, System.currentTimeMillis(), Arrays.asList(records));
|
||||||
|
}
|
||||||
|
|
||||||
|
public static MemoryRecords withRecords(long initialOffset,
|
||||||
|
TimestampType timestampType,
|
||||||
|
CompressionType compressionType,
|
||||||
|
long logAppendTime,
|
||||||
|
List<Record> records) {
|
||||||
|
return withLogEntries(timestampType, compressionType, logAppendTime, buildLogEntries(initialOffset, records));
|
||||||
|
}
|
||||||
|
|
||||||
|
private static MemoryRecords withLogEntries(TimestampType timestampType,
|
||||||
|
CompressionType compressionType,
|
||||||
|
long logAppendTime,
|
||||||
|
List<LogEntry> entries) {
|
||||||
|
if (entries.isEmpty())
|
||||||
|
return MemoryRecords.EMPTY;
|
||||||
|
return builderWithEntries(timestampType, compressionType, logAppendTime, entries).build();
|
||||||
|
}
|
||||||
|
|
||||||
|
private static List<LogEntry> buildLogEntries(long initialOffset, List<Record> records) {
|
||||||
|
List<LogEntry> entries = new ArrayList<>();
|
||||||
|
for (Record record : records)
|
||||||
|
entries.add(LogEntry.create(initialOffset++, record));
|
||||||
|
return entries;
|
||||||
|
}
|
||||||
|
|
||||||
|
public static MemoryRecordsBuilder builderWithEntries(TimestampType timestampType,
|
||||||
|
CompressionType compressionType,
|
||||||
|
long logAppendTime,
|
||||||
|
List<LogEntry> entries) {
|
||||||
|
ByteBuffer buffer = ByteBuffer.allocate(estimatedSize(compressionType, entries));
|
||||||
|
return builderWithEntries(buffer, timestampType, compressionType, logAppendTime, entries);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static MemoryRecordsBuilder builderWithEntries(ByteBuffer buffer,
|
||||||
|
TimestampType timestampType,
|
||||||
|
CompressionType compressionType,
|
||||||
|
long logAppendTime,
|
||||||
|
List<LogEntry> entries) {
|
||||||
|
if (entries.isEmpty())
|
||||||
|
throw new IllegalArgumentException();
|
||||||
|
|
||||||
|
LogEntry firstEntry = entries.iterator().next();
|
||||||
|
long firstOffset = firstEntry.offset();
|
||||||
|
byte magic = firstEntry.record().magic();
|
||||||
|
|
||||||
|
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic, compressionType, timestampType,
|
||||||
|
firstOffset, logAppendTime);
|
||||||
|
for (LogEntry entry : entries)
|
||||||
|
builder.append(entry);
|
||||||
|
|
||||||
|
return builder;
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,461 @@
|
||||||
|
/**
|
||||||
|
* 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.KafkaException;
|
||||||
|
|
||||||
|
import java.io.DataInputStream;
|
||||||
|
import java.io.DataOutputStream;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.InputStream;
|
||||||
|
import java.io.OutputStream;
|
||||||
|
import java.lang.reflect.Constructor;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.util.zip.GZIPInputStream;
|
||||||
|
import java.util.zip.GZIPOutputStream;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This class is used to write new log data in memory, i.e. this is the write path for {@link MemoryRecords}.
|
||||||
|
* It transparently handles compression and exposes methods for appending new entries, possibly with message
|
||||||
|
* format conversion.
|
||||||
|
*/
|
||||||
|
public class MemoryRecordsBuilder {
|
||||||
|
|
||||||
|
static private final float COMPRESSION_RATE_DAMPING_FACTOR = 0.9f;
|
||||||
|
static private final float COMPRESSION_RATE_ESTIMATION_FACTOR = 1.05f;
|
||||||
|
static private final int COMPRESSION_DEFAULT_BUFFER_SIZE = 1024;
|
||||||
|
|
||||||
|
private static final float[] TYPE_TO_RATE;
|
||||||
|
|
||||||
|
static {
|
||||||
|
int maxTypeId = -1;
|
||||||
|
for (CompressionType type : CompressionType.values())
|
||||||
|
maxTypeId = Math.max(maxTypeId, type.id);
|
||||||
|
TYPE_TO_RATE = new float[maxTypeId + 1];
|
||||||
|
for (CompressionType type : CompressionType.values()) {
|
||||||
|
TYPE_TO_RATE[type.id] = type.rate;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// dynamically load the snappy and lz4 classes to avoid runtime dependency if we are not using compression
|
||||||
|
// caching constructors to avoid invoking of Class.forName method for each batch
|
||||||
|
private static MemoizingConstructorSupplier snappyOutputStreamSupplier = new MemoizingConstructorSupplier(new ConstructorSupplier() {
|
||||||
|
@Override
|
||||||
|
public Constructor get() throws ClassNotFoundException, NoSuchMethodException {
|
||||||
|
return Class.forName("org.xerial.snappy.SnappyOutputStream")
|
||||||
|
.getConstructor(OutputStream.class, Integer.TYPE);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
private static MemoizingConstructorSupplier lz4OutputStreamSupplier = new MemoizingConstructorSupplier(new ConstructorSupplier() {
|
||||||
|
@Override
|
||||||
|
public Constructor get() throws ClassNotFoundException, NoSuchMethodException {
|
||||||
|
return Class.forName("org.apache.kafka.common.record.KafkaLZ4BlockOutputStream")
|
||||||
|
.getConstructor(OutputStream.class, Boolean.TYPE);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
private static MemoizingConstructorSupplier snappyInputStreamSupplier = new MemoizingConstructorSupplier(new ConstructorSupplier() {
|
||||||
|
@Override
|
||||||
|
public Constructor get() throws ClassNotFoundException, NoSuchMethodException {
|
||||||
|
return Class.forName("org.xerial.snappy.SnappyInputStream")
|
||||||
|
.getConstructor(InputStream.class);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
private static MemoizingConstructorSupplier lz4InputStreamSupplier = new MemoizingConstructorSupplier(new ConstructorSupplier() {
|
||||||
|
@Override
|
||||||
|
public Constructor get() throws ClassNotFoundException, NoSuchMethodException {
|
||||||
|
return Class.forName("org.apache.kafka.common.record.KafkaLZ4BlockInputStream")
|
||||||
|
.getConstructor(InputStream.class, Boolean.TYPE);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
private final TimestampType timestampType;
|
||||||
|
private final CompressionType compressionType;
|
||||||
|
private final DataOutputStream appendStream;
|
||||||
|
private final ByteBufferOutputStream bufferStream;
|
||||||
|
private final byte magic;
|
||||||
|
private final int initPos;
|
||||||
|
private final long baseOffset;
|
||||||
|
private final long logAppendTime;
|
||||||
|
private final int writeLimit;
|
||||||
|
private final int initialCapacity;
|
||||||
|
|
||||||
|
private MemoryRecords builtRecords;
|
||||||
|
private long writtenUncompressed;
|
||||||
|
private long numRecords;
|
||||||
|
private float compressionRate;
|
||||||
|
private long maxTimestamp;
|
||||||
|
private long offsetOfMaxTimestamp;
|
||||||
|
private long lastOffset = -1;
|
||||||
|
|
||||||
|
public MemoryRecordsBuilder(ByteBuffer buffer,
|
||||||
|
byte magic,
|
||||||
|
CompressionType compressionType,
|
||||||
|
TimestampType timestampType,
|
||||||
|
long baseOffset,
|
||||||
|
long logAppendTime,
|
||||||
|
int writeLimit) {
|
||||||
|
this.magic = magic;
|
||||||
|
this.timestampType = timestampType;
|
||||||
|
this.compressionType = compressionType;
|
||||||
|
this.baseOffset = baseOffset;
|
||||||
|
this.logAppendTime = logAppendTime;
|
||||||
|
this.initPos = buffer.position();
|
||||||
|
this.numRecords = 0;
|
||||||
|
this.writtenUncompressed = 0;
|
||||||
|
this.compressionRate = 1;
|
||||||
|
this.maxTimestamp = Record.NO_TIMESTAMP;
|
||||||
|
this.writeLimit = writeLimit;
|
||||||
|
this.initialCapacity = buffer.capacity();
|
||||||
|
|
||||||
|
if (compressionType != CompressionType.NONE) {
|
||||||
|
// for compressed records, leave space for the header and the shallow message metadata
|
||||||
|
// and move the starting position to the value payload offset
|
||||||
|
buffer.position(initPos + Records.LOG_OVERHEAD + Record.recordOverhead(magic));
|
||||||
|
}
|
||||||
|
|
||||||
|
// create the stream
|
||||||
|
bufferStream = new ByteBufferOutputStream(buffer);
|
||||||
|
appendStream = wrapForOutput(bufferStream, compressionType, magic, COMPRESSION_DEFAULT_BUFFER_SIZE);
|
||||||
|
}
|
||||||
|
|
||||||
|
public ByteBuffer buffer() {
|
||||||
|
return bufferStream.buffer();
|
||||||
|
}
|
||||||
|
|
||||||
|
public int initialCapacity() {
|
||||||
|
return initialCapacity;
|
||||||
|
}
|
||||||
|
|
||||||
|
public double compressionRate() {
|
||||||
|
return compressionRate;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Close this builder and return the resulting buffer.
|
||||||
|
* @return The built log buffer
|
||||||
|
*/
|
||||||
|
public MemoryRecords build() {
|
||||||
|
close();
|
||||||
|
return builtRecords;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the max timestamp and its offset. If the log append time is used, then the offset will
|
||||||
|
* be either the first offset in the set if no compression is used or the last offset otherwise.
|
||||||
|
* @return The max timestamp and its offset
|
||||||
|
*/
|
||||||
|
public RecordsInfo info() {
|
||||||
|
if (timestampType == TimestampType.LOG_APPEND_TIME)
|
||||||
|
return new RecordsInfo(logAppendTime, lastOffset);
|
||||||
|
else
|
||||||
|
return new RecordsInfo(maxTimestamp, compressionType == CompressionType.NONE ? offsetOfMaxTimestamp : lastOffset);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void close() {
|
||||||
|
if (builtRecords != null)
|
||||||
|
return;
|
||||||
|
|
||||||
|
try {
|
||||||
|
appendStream.close();
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new KafkaException(e);
|
||||||
|
}
|
||||||
|
|
||||||
|
if (compressionType != CompressionType.NONE)
|
||||||
|
writerCompressedWrapperHeader();
|
||||||
|
|
||||||
|
ByteBuffer buffer = buffer().duplicate();
|
||||||
|
buffer.flip();
|
||||||
|
buffer.position(initPos);
|
||||||
|
builtRecords = MemoryRecords.readableRecords(buffer.slice());
|
||||||
|
}
|
||||||
|
|
||||||
|
private void writerCompressedWrapperHeader() {
|
||||||
|
ByteBuffer buffer = bufferStream.buffer();
|
||||||
|
int pos = buffer.position();
|
||||||
|
buffer.position(initPos);
|
||||||
|
|
||||||
|
int wrapperSize = pos - initPos - Records.LOG_OVERHEAD;
|
||||||
|
int writtenCompressed = wrapperSize - Record.recordOverhead(magic);
|
||||||
|
LogEntry.writeHeader(buffer, lastOffset, wrapperSize);
|
||||||
|
|
||||||
|
long timestamp = timestampType == TimestampType.LOG_APPEND_TIME ? logAppendTime : maxTimestamp;
|
||||||
|
Record.writeCompressedRecordHeader(buffer, magic, wrapperSize, timestamp, compressionType, timestampType);
|
||||||
|
|
||||||
|
buffer.position(pos);
|
||||||
|
|
||||||
|
// update the compression ratio
|
||||||
|
this.compressionRate = (float) writtenCompressed / this.writtenUncompressed;
|
||||||
|
TYPE_TO_RATE[compressionType.id] = TYPE_TO_RATE[compressionType.id] * COMPRESSION_RATE_DAMPING_FACTOR +
|
||||||
|
compressionRate * (1 - COMPRESSION_RATE_DAMPING_FACTOR);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Append a new record and offset to the buffer
|
||||||
|
* @param offset The absolute offset of the record in the log buffer
|
||||||
|
* @param timestamp The record timestamp
|
||||||
|
* @param key The record key
|
||||||
|
* @param value The record value
|
||||||
|
* @return crc of the record
|
||||||
|
*/
|
||||||
|
public long append(long offset, long timestamp, byte[] key, byte[] value) {
|
||||||
|
try {
|
||||||
|
if (lastOffset > 0 && offset <= lastOffset)
|
||||||
|
throw new IllegalArgumentException(String.format("Illegal offset %s following previous offset %s (Offsets must increase monotonically).", offset, lastOffset));
|
||||||
|
|
||||||
|
int size = Record.recordSize(magic, key, value);
|
||||||
|
LogEntry.writeHeader(appendStream, toInnerOffset(offset), size);
|
||||||
|
|
||||||
|
if (timestampType == TimestampType.LOG_APPEND_TIME)
|
||||||
|
timestamp = logAppendTime;
|
||||||
|
long crc = Record.write(appendStream, magic, timestamp, key, value, CompressionType.NONE, timestampType);
|
||||||
|
recordWritten(offset, timestamp, size + Records.LOG_OVERHEAD);
|
||||||
|
return crc;
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new KafkaException("I/O exception when writing to the append stream, closing", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Add the record, converting to the desired magic value if necessary.
|
||||||
|
* @param offset The offset of the record
|
||||||
|
* @param record The record to add
|
||||||
|
*/
|
||||||
|
public void convertAndAppend(long offset, Record record) {
|
||||||
|
if (magic == record.magic()) {
|
||||||
|
append(offset, record);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (lastOffset > 0 && offset <= lastOffset)
|
||||||
|
throw new IllegalArgumentException(String.format("Illegal offset %s following previous offset %s (Offsets must increase monotonically).", offset, lastOffset));
|
||||||
|
|
||||||
|
try {
|
||||||
|
int size = record.convertedSize(magic);
|
||||||
|
LogEntry.writeHeader(appendStream, toInnerOffset(offset), size);
|
||||||
|
long timestamp = timestampType == TimestampType.LOG_APPEND_TIME ? logAppendTime : record.timestamp();
|
||||||
|
record.convertTo(appendStream, magic, timestamp, timestampType);
|
||||||
|
recordWritten(offset, timestamp, size + Records.LOG_OVERHEAD);
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new KafkaException("I/O exception when writing to the append stream, closing", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Add a record without doing offset/magic validation (this should only be used in testing).
|
||||||
|
* @param offset The offset of the record
|
||||||
|
* @param record The record to add
|
||||||
|
*/
|
||||||
|
public void appendUnchecked(long offset, Record record) {
|
||||||
|
try {
|
||||||
|
int size = record.sizeInBytes();
|
||||||
|
LogEntry.writeHeader(appendStream, toInnerOffset(offset), size);
|
||||||
|
|
||||||
|
ByteBuffer buffer = record.buffer().duplicate();
|
||||||
|
appendStream.write(buffer.array(), buffer.arrayOffset(), buffer.limit());
|
||||||
|
|
||||||
|
recordWritten(offset, record.timestamp(), size + Records.LOG_OVERHEAD);
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new KafkaException("I/O exception when writing to the append stream, closing", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Append the given log entry. The entry's record must have a magic which matches the magic use to
|
||||||
|
* construct this builder and the offset must be greater than the last appended entry.
|
||||||
|
* @param entry The entry to append
|
||||||
|
*/
|
||||||
|
public void append(LogEntry entry) {
|
||||||
|
append(entry.offset(), entry.record());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Add a record with a given offset. The record must have a magic which matches the magic use to
|
||||||
|
* construct this builder and the offset must be greater than the last appended entry.
|
||||||
|
* @param offset The offset of the record
|
||||||
|
* @param record The record to add
|
||||||
|
*/
|
||||||
|
public void append(long offset, Record record) {
|
||||||
|
if (record.magic() != magic)
|
||||||
|
throw new IllegalArgumentException("Inner log entries must have matching magic values as the wrapper");
|
||||||
|
if (lastOffset > 0 && offset <= lastOffset)
|
||||||
|
throw new IllegalArgumentException(String.format("Illegal offset %s following previous offset %s (Offsets must increase monotonically).", offset, lastOffset));
|
||||||
|
appendUnchecked(offset, record);
|
||||||
|
}
|
||||||
|
|
||||||
|
private long toInnerOffset(long offset) {
|
||||||
|
// use relative offsets for compressed messages with magic v1
|
||||||
|
if (magic > 0 && compressionType != CompressionType.NONE)
|
||||||
|
return offset - baseOffset;
|
||||||
|
return offset;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void recordWritten(long offset, long timestamp, int size) {
|
||||||
|
numRecords += 1;
|
||||||
|
writtenUncompressed += size;
|
||||||
|
lastOffset = offset;
|
||||||
|
|
||||||
|
if (timestamp > maxTimestamp) {
|
||||||
|
maxTimestamp = timestamp;
|
||||||
|
offsetOfMaxTimestamp = offset;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get an estimate of the number of bytes written (based on the estimation factor hard-coded in {@link CompressionType}.
|
||||||
|
* @return The estimated number of bytes written
|
||||||
|
*/
|
||||||
|
private int estimatedBytesWritten() {
|
||||||
|
if (compressionType == CompressionType.NONE) {
|
||||||
|
return buffer().position();
|
||||||
|
} else {
|
||||||
|
// estimate the written bytes to the underlying byte buffer based on uncompressed written bytes
|
||||||
|
return (int) (writtenUncompressed * TYPE_TO_RATE[compressionType.id] * COMPRESSION_RATE_ESTIMATION_FACTOR);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Check if we have room for a new record containing the given key/value pair
|
||||||
|
*
|
||||||
|
* Note that the return value is based on the estimate of the bytes written to the compressor, which may not be
|
||||||
|
* accurate if compression is really used. When this happens, the following append may cause dynamic buffer
|
||||||
|
* re-allocation in the underlying byte buffer stream.
|
||||||
|
*
|
||||||
|
* There is an exceptional case when appending a single message whose size is larger than the batch size, the
|
||||||
|
* capacity will be the message size which is larger than the write limit, i.e. the batch size. In this case
|
||||||
|
* the checking should be based on the capacity of the initialized buffer rather than the write limit in order
|
||||||
|
* to accept this single record.
|
||||||
|
*/
|
||||||
|
public boolean hasRoomFor(byte[] key, byte[] value) {
|
||||||
|
return !isFull() && (numRecords == 0 ?
|
||||||
|
this.initialCapacity >= Records.LOG_OVERHEAD + Record.recordSize(magic, key, value) :
|
||||||
|
this.writeLimit >= estimatedBytesWritten() + Records.LOG_OVERHEAD + Record.recordSize(magic, key, value));
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean isClosed() {
|
||||||
|
return builtRecords != null;
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean isFull() {
|
||||||
|
return isClosed() || this.writeLimit <= estimatedBytesWritten();
|
||||||
|
}
|
||||||
|
|
||||||
|
public int sizeInBytes() {
|
||||||
|
return builtRecords != null ? builtRecords.sizeInBytes() : estimatedBytesWritten();
|
||||||
|
}
|
||||||
|
|
||||||
|
private static DataOutputStream wrapForOutput(ByteBufferOutputStream buffer, CompressionType type, byte messageVersion, int bufferSize) {
|
||||||
|
try {
|
||||||
|
switch (type) {
|
||||||
|
case NONE:
|
||||||
|
return buffer;
|
||||||
|
case GZIP:
|
||||||
|
return new DataOutputStream(new GZIPOutputStream(buffer, bufferSize));
|
||||||
|
case SNAPPY:
|
||||||
|
try {
|
||||||
|
OutputStream stream = (OutputStream) snappyOutputStreamSupplier.get().newInstance(buffer, bufferSize);
|
||||||
|
return new DataOutputStream(stream);
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new KafkaException(e);
|
||||||
|
}
|
||||||
|
case LZ4:
|
||||||
|
try {
|
||||||
|
OutputStream stream = (OutputStream) lz4OutputStreamSupplier.get().newInstance(buffer,
|
||||||
|
messageVersion == Record.MAGIC_VALUE_V0);
|
||||||
|
return new DataOutputStream(stream);
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new KafkaException(e);
|
||||||
|
}
|
||||||
|
default:
|
||||||
|
throw new IllegalArgumentException("Unknown compression type: " + type);
|
||||||
|
}
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new KafkaException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static DataInputStream wrapForInput(ByteBufferInputStream buffer, CompressionType type, byte messageVersion) {
|
||||||
|
try {
|
||||||
|
switch (type) {
|
||||||
|
case NONE:
|
||||||
|
return buffer;
|
||||||
|
case GZIP:
|
||||||
|
return new DataInputStream(new GZIPInputStream(buffer));
|
||||||
|
case SNAPPY:
|
||||||
|
try {
|
||||||
|
InputStream stream = (InputStream) snappyInputStreamSupplier.get().newInstance(buffer);
|
||||||
|
return new DataInputStream(stream);
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new KafkaException(e);
|
||||||
|
}
|
||||||
|
case LZ4:
|
||||||
|
try {
|
||||||
|
InputStream stream = (InputStream) lz4InputStreamSupplier.get().newInstance(buffer,
|
||||||
|
messageVersion == Record.MAGIC_VALUE_V0);
|
||||||
|
return new DataInputStream(stream);
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new KafkaException(e);
|
||||||
|
}
|
||||||
|
default:
|
||||||
|
throw new IllegalArgumentException("Unknown compression type: " + type);
|
||||||
|
}
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new KafkaException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private interface ConstructorSupplier {
|
||||||
|
Constructor get() throws ClassNotFoundException, NoSuchMethodException;
|
||||||
|
}
|
||||||
|
|
||||||
|
// this code is based on Guava's @see{com.google.common.base.Suppliers.MemoizingSupplier}
|
||||||
|
private static class MemoizingConstructorSupplier {
|
||||||
|
final ConstructorSupplier delegate;
|
||||||
|
transient volatile boolean initialized;
|
||||||
|
transient Constructor value;
|
||||||
|
|
||||||
|
public MemoizingConstructorSupplier(ConstructorSupplier delegate) {
|
||||||
|
this.delegate = delegate;
|
||||||
|
}
|
||||||
|
|
||||||
|
public Constructor get() throws NoSuchMethodException, ClassNotFoundException {
|
||||||
|
if (!initialized) {
|
||||||
|
synchronized (this) {
|
||||||
|
if (!initialized) {
|
||||||
|
value = delegate.get();
|
||||||
|
initialized = true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return value;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class RecordsInfo {
|
||||||
|
public final long maxTimestamp;
|
||||||
|
public final long shallowOffsetOfMaxTimestamp;
|
||||||
|
|
||||||
|
public RecordsInfo(long maxTimestamp,
|
||||||
|
long shallowOffsetOfMaxTimestamp) {
|
||||||
|
this.maxTimestamp = maxTimestamp;
|
||||||
|
this.shallowOffsetOfMaxTimestamp = shallowOffsetOfMaxTimestamp;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -16,11 +16,15 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.common.record;
|
package org.apache.kafka.common.record;
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import org.apache.kafka.common.KafkaException;
|
||||||
|
|
||||||
import org.apache.kafka.common.utils.Crc32;
|
import org.apache.kafka.common.utils.Crc32;
|
||||||
import org.apache.kafka.common.utils.Utils;
|
import org.apache.kafka.common.utils.Utils;
|
||||||
|
|
||||||
|
import java.io.DataOutputStream;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
|
||||||
|
import static org.apache.kafka.common.utils.Utils.wrapNullable;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A record: a serialized key and value along with the associated CRC and other fields
|
* A record: a serialized key and value along with the associated CRC and other fields
|
||||||
|
|
@ -53,7 +57,12 @@ public final class Record {
|
||||||
/**
|
/**
|
||||||
* The amount of overhead bytes in a record
|
* The amount of overhead bytes in a record
|
||||||
*/
|
*/
|
||||||
public static final int RECORD_OVERHEAD = HEADER_SIZE + TIMESTAMP_LENGTH + KEY_SIZE_LENGTH + VALUE_SIZE_LENGTH;
|
public static final int RECORD_OVERHEAD_V0 = HEADER_SIZE + KEY_SIZE_LENGTH + VALUE_SIZE_LENGTH;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The amount of overhead bytes in a record
|
||||||
|
*/
|
||||||
|
public static final int RECORD_OVERHEAD_V1 = HEADER_SIZE + TIMESTAMP_LENGTH + KEY_SIZE_LENGTH + VALUE_SIZE_LENGTH;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The "magic" values
|
* The "magic" values
|
||||||
|
|
@ -79,11 +88,6 @@ public final class Record {
|
||||||
public static final byte TIMESTAMP_TYPE_MASK = 0x08;
|
public static final byte TIMESTAMP_TYPE_MASK = 0x08;
|
||||||
public static final int TIMESTAMP_TYPE_ATTRIBUTE_OFFSET = 3;
|
public static final int TIMESTAMP_TYPE_ATTRIBUTE_OFFSET = 3;
|
||||||
|
|
||||||
/**
|
|
||||||
* Compression code for uncompressed records
|
|
||||||
*/
|
|
||||||
public static final int NO_COMPRESSION = 0;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Timestamp value for records without a timestamp
|
* Timestamp value for records without a timestamp
|
||||||
*/
|
*/
|
||||||
|
|
@ -94,155 +98,20 @@ public final class Record {
|
||||||
private final TimestampType wrapperRecordTimestampType;
|
private final TimestampType wrapperRecordTimestampType;
|
||||||
|
|
||||||
public Record(ByteBuffer buffer) {
|
public Record(ByteBuffer buffer) {
|
||||||
this.buffer = buffer;
|
this(buffer, null, null);
|
||||||
this.wrapperRecordTimestamp = null;
|
|
||||||
this.wrapperRecordTimestampType = null;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// Package private constructor for inner iteration.
|
public Record(ByteBuffer buffer, Long wrapperRecordTimestamp, TimestampType wrapperRecordTimestampType) {
|
||||||
Record(ByteBuffer buffer, Long wrapperRecordTimestamp, TimestampType wrapperRecordTimestampType) {
|
|
||||||
this.buffer = buffer;
|
this.buffer = buffer;
|
||||||
this.wrapperRecordTimestamp = wrapperRecordTimestamp;
|
this.wrapperRecordTimestamp = wrapperRecordTimestamp;
|
||||||
this.wrapperRecordTimestampType = wrapperRecordTimestampType;
|
this.wrapperRecordTimestampType = wrapperRecordTimestampType;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* A constructor to create a LogRecord. If the record's compression type is not none, then
|
|
||||||
* its value payload should be already compressed with the specified type; the constructor
|
|
||||||
* would always write the value payload as is and will not do the compression itself.
|
|
||||||
*
|
|
||||||
* @param timestamp The timestamp of the record
|
|
||||||
* @param key The key of the record (null, if none)
|
|
||||||
* @param value The record value
|
|
||||||
* @param type The compression type used on the contents of the record (if any)
|
|
||||||
* @param valueOffset The offset into the payload array used to extract payload
|
|
||||||
* @param valueSize The size of the payload to use
|
|
||||||
*/
|
|
||||||
public Record(long timestamp, byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) {
|
|
||||||
this(ByteBuffer.allocate(recordSize(key == null ? 0 : key.length,
|
|
||||||
value == null ? 0 : valueSize >= 0 ? valueSize : value.length - valueOffset)));
|
|
||||||
write(this.buffer, timestamp, key, value, type, valueOffset, valueSize);
|
|
||||||
this.buffer.rewind();
|
|
||||||
}
|
|
||||||
|
|
||||||
public Record(long timestamp, byte[] key, byte[] value, CompressionType type) {
|
|
||||||
this(timestamp, key, value, type, 0, -1);
|
|
||||||
}
|
|
||||||
|
|
||||||
public Record(long timestamp, byte[] value, CompressionType type) {
|
|
||||||
this(timestamp, null, value, type);
|
|
||||||
}
|
|
||||||
|
|
||||||
public Record(long timestamp, byte[] key, byte[] value) {
|
|
||||||
this(timestamp, key, value, CompressionType.NONE);
|
|
||||||
}
|
|
||||||
|
|
||||||
public Record(long timestamp, byte[] value) {
|
|
||||||
this(timestamp, null, value, CompressionType.NONE);
|
|
||||||
}
|
|
||||||
|
|
||||||
// Write a record to the buffer, if the record's compression type is none, then
|
|
||||||
// its value payload should be already compressed with the specified type
|
|
||||||
public static void write(ByteBuffer buffer, long timestamp, byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) {
|
|
||||||
// construct the compressor with compression type none since this function will not do any
|
|
||||||
//compression according to the input type, it will just write the record's payload as is
|
|
||||||
Compressor compressor = new Compressor(buffer, CompressionType.NONE);
|
|
||||||
try {
|
|
||||||
compressor.putRecord(timestamp, key, value, type, valueOffset, valueSize);
|
|
||||||
} finally {
|
|
||||||
compressor.close();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public static void write(Compressor compressor, long crc, byte attributes, long timestamp, byte[] key, byte[] value, int valueOffset, int valueSize) {
|
|
||||||
// write crc
|
|
||||||
compressor.putInt((int) (crc & 0xffffffffL));
|
|
||||||
// write magic value
|
|
||||||
compressor.putByte(CURRENT_MAGIC_VALUE);
|
|
||||||
// write attributes
|
|
||||||
compressor.putByte(attributes);
|
|
||||||
// write timestamp
|
|
||||||
compressor.putLong(timestamp);
|
|
||||||
// write the key
|
|
||||||
if (key == null) {
|
|
||||||
compressor.putInt(-1);
|
|
||||||
} else {
|
|
||||||
compressor.putInt(key.length);
|
|
||||||
compressor.put(key, 0, key.length);
|
|
||||||
}
|
|
||||||
// write the value
|
|
||||||
if (value == null) {
|
|
||||||
compressor.putInt(-1);
|
|
||||||
} else {
|
|
||||||
int size = valueSize >= 0 ? valueSize : (value.length - valueOffset);
|
|
||||||
compressor.putInt(size);
|
|
||||||
compressor.put(value, valueOffset, size);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public static int recordSize(byte[] key, byte[] value) {
|
|
||||||
return recordSize(key == null ? 0 : key.length, value == null ? 0 : value.length);
|
|
||||||
}
|
|
||||||
|
|
||||||
public static int recordSize(int keySize, int valueSize) {
|
|
||||||
return CRC_LENGTH + MAGIC_LENGTH + ATTRIBUTE_LENGTH + TIMESTAMP_LENGTH + KEY_SIZE_LENGTH + keySize + VALUE_SIZE_LENGTH + valueSize;
|
|
||||||
}
|
|
||||||
|
|
||||||
public ByteBuffer buffer() {
|
|
||||||
return this.buffer;
|
|
||||||
}
|
|
||||||
|
|
||||||
public static byte computeAttributes(CompressionType type) {
|
|
||||||
byte attributes = 0;
|
|
||||||
if (type.id > 0)
|
|
||||||
attributes = (byte) (attributes | (COMPRESSION_CODEC_MASK & type.id));
|
|
||||||
return attributes;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Compute the checksum of the record from the record contents
|
|
||||||
*/
|
|
||||||
public static long computeChecksum(ByteBuffer buffer, int position, int size) {
|
|
||||||
Crc32 crc = new Crc32();
|
|
||||||
crc.update(buffer.array(), buffer.arrayOffset() + position, size);
|
|
||||||
return crc.getValue();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Compute the checksum of the record from the attributes, key and value payloads
|
|
||||||
*/
|
|
||||||
public static long computeChecksum(long timestamp, byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) {
|
|
||||||
Crc32 crc = new Crc32();
|
|
||||||
crc.update(CURRENT_MAGIC_VALUE);
|
|
||||||
byte attributes = 0;
|
|
||||||
if (type.id > 0)
|
|
||||||
attributes = (byte) (attributes | (COMPRESSION_CODEC_MASK & type.id));
|
|
||||||
crc.update(attributes);
|
|
||||||
crc.updateLong(timestamp);
|
|
||||||
// update for the key
|
|
||||||
if (key == null) {
|
|
||||||
crc.updateInt(-1);
|
|
||||||
} else {
|
|
||||||
crc.updateInt(key.length);
|
|
||||||
crc.update(key, 0, key.length);
|
|
||||||
}
|
|
||||||
// update for the value
|
|
||||||
if (value == null) {
|
|
||||||
crc.updateInt(-1);
|
|
||||||
} else {
|
|
||||||
int size = valueSize >= 0 ? valueSize : (value.length - valueOffset);
|
|
||||||
crc.updateInt(size);
|
|
||||||
crc.update(value, valueOffset, size);
|
|
||||||
}
|
|
||||||
return crc.getValue();
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Compute the checksum of the record from the record contents
|
* Compute the checksum of the record from the record contents
|
||||||
*/
|
*/
|
||||||
public long computeChecksum() {
|
public long computeChecksum() {
|
||||||
return computeChecksum(buffer, MAGIC_OFFSET, buffer.limit() - MAGIC_OFFSET);
|
return Utils.computeChecksum(buffer, MAGIC_OFFSET, buffer.limit() - MAGIC_OFFSET);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -256,7 +125,15 @@ public final class Record {
|
||||||
* Returns true if the crc stored with the record matches the crc computed off the record contents
|
* Returns true if the crc stored with the record matches the crc computed off the record contents
|
||||||
*/
|
*/
|
||||||
public boolean isValid() {
|
public boolean isValid() {
|
||||||
return size() >= CRC_LENGTH && checksum() == computeChecksum();
|
return sizeInBytes() >= CRC_LENGTH && checksum() == computeChecksum();
|
||||||
|
}
|
||||||
|
|
||||||
|
public Long wrapperRecordTimestamp() {
|
||||||
|
return wrapperRecordTimestamp;
|
||||||
|
}
|
||||||
|
|
||||||
|
public TimestampType wrapperRecordTimestampType() {
|
||||||
|
return wrapperRecordTimestampType;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -264,9 +141,9 @@ public final class Record {
|
||||||
*/
|
*/
|
||||||
public void ensureValid() {
|
public void ensureValid() {
|
||||||
if (!isValid()) {
|
if (!isValid()) {
|
||||||
if (size() < CRC_LENGTH)
|
if (sizeInBytes() < CRC_LENGTH)
|
||||||
throw new InvalidRecordException("Record is corrupt (crc could not be retrieved as the record is too "
|
throw new InvalidRecordException("Record is corrupt (crc could not be retrieved as the record is too "
|
||||||
+ "small, size = " + size() + ")");
|
+ "small, size = " + sizeInBytes() + ")");
|
||||||
else
|
else
|
||||||
throw new InvalidRecordException("Record is corrupt (stored crc = " + checksum()
|
throw new InvalidRecordException("Record is corrupt (stored crc = " + checksum()
|
||||||
+ ", computed crc = " + computeChecksum() + ")");
|
+ ", computed crc = " + computeChecksum() + ")");
|
||||||
|
|
@ -274,14 +151,17 @@ public final class Record {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The complete serialized size of this record in bytes (including crc, header attributes, etc)
|
* The complete serialized size of this record in bytes (including crc, header attributes, etc), but
|
||||||
|
* excluding the log overhead (offset and record size).
|
||||||
|
* @return the size in bytes
|
||||||
*/
|
*/
|
||||||
public int size() {
|
public int sizeInBytes() {
|
||||||
return buffer.limit();
|
return buffer.limit();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The length of the key in bytes
|
* The length of the key in bytes
|
||||||
|
* @return the size in bytes of the key (0 if the key is null)
|
||||||
*/
|
*/
|
||||||
public int keySize() {
|
public int keySize() {
|
||||||
if (magic() == MAGIC_VALUE_V0)
|
if (magic() == MAGIC_VALUE_V0)
|
||||||
|
|
@ -292,6 +172,7 @@ public final class Record {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Does the record have a key?
|
* Does the record have a key?
|
||||||
|
* @return true if so, false otherwise
|
||||||
*/
|
*/
|
||||||
public boolean hasKey() {
|
public boolean hasKey() {
|
||||||
return keySize() >= 0;
|
return keySize() >= 0;
|
||||||
|
|
@ -309,13 +190,23 @@ public final class Record {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The length of the value in bytes
|
* The length of the value in bytes
|
||||||
|
* @return the size in bytes of the value (0 if the value is null)
|
||||||
*/
|
*/
|
||||||
public int valueSize() {
|
public int valueSize() {
|
||||||
return buffer.getInt(valueSizeOffset());
|
return buffer.getInt(valueSizeOffset());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The magic version of this record
|
* Check whether the value field of this record is null.
|
||||||
|
* @return true if the value is null, false otherwise
|
||||||
|
*/
|
||||||
|
public boolean hasNullValue() {
|
||||||
|
return valueSize() < 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The magic value (i.e. message format version) of this record
|
||||||
|
* @return the magic value
|
||||||
*/
|
*/
|
||||||
public byte magic() {
|
public byte magic() {
|
||||||
return buffer.get(MAGIC_OFFSET);
|
return buffer.get(MAGIC_OFFSET);
|
||||||
|
|
@ -323,6 +214,7 @@ public final class Record {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The attributes stored with this record
|
* The attributes stored with this record
|
||||||
|
* @return the attributes
|
||||||
*/
|
*/
|
||||||
public byte attributes() {
|
public byte attributes() {
|
||||||
return buffer.get(ATTRIBUTES_OFFSET);
|
return buffer.get(ATTRIBUTES_OFFSET);
|
||||||
|
|
@ -333,6 +225,8 @@ public final class Record {
|
||||||
* 1. wrapperRecordTimestampType = null and wrapperRecordTimestamp is null - Uncompressed message, timestamp is in the message.
|
* 1. wrapperRecordTimestampType = null and wrapperRecordTimestamp is null - Uncompressed message, timestamp is in the message.
|
||||||
* 2. wrapperRecordTimestampType = LOG_APPEND_TIME and WrapperRecordTimestamp is not null - Compressed message using LOG_APPEND_TIME
|
* 2. wrapperRecordTimestampType = LOG_APPEND_TIME and WrapperRecordTimestamp is not null - Compressed message using LOG_APPEND_TIME
|
||||||
* 3. wrapperRecordTimestampType = CREATE_TIME and wrapperRecordTimestamp is not null - Compressed message using CREATE_TIME
|
* 3. wrapperRecordTimestampType = CREATE_TIME and wrapperRecordTimestamp is not null - Compressed message using CREATE_TIME
|
||||||
|
*
|
||||||
|
* @return the timestamp as determined above
|
||||||
*/
|
*/
|
||||||
public long timestamp() {
|
public long timestamp() {
|
||||||
if (magic() == MAGIC_VALUE_V0)
|
if (magic() == MAGIC_VALUE_V0)
|
||||||
|
|
@ -349,6 +243,8 @@ public final class Record {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The timestamp of the message.
|
* The timestamp of the message.
|
||||||
|
* @return the timstamp type or {@link TimestampType#NO_TIMESTAMP_TYPE} if the magic is 0 or the message has
|
||||||
|
* been up-converted.
|
||||||
*/
|
*/
|
||||||
public TimestampType timestampType() {
|
public TimestampType timestampType() {
|
||||||
if (magic() == 0)
|
if (magic() == 0)
|
||||||
|
|
@ -366,36 +262,30 @@ public final class Record {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A ByteBuffer containing the value of this record
|
* A ByteBuffer containing the value of this record
|
||||||
|
* @return the value or null if the value for this record is null
|
||||||
*/
|
*/
|
||||||
public ByteBuffer value() {
|
public ByteBuffer value() {
|
||||||
return sliceDelimited(valueSizeOffset());
|
return Utils.sizeDelimited(buffer, valueSizeOffset());
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A ByteBuffer containing the message key
|
* A ByteBuffer containing the message key
|
||||||
|
* @return the buffer or null if the key for this record is null
|
||||||
*/
|
*/
|
||||||
public ByteBuffer key() {
|
public ByteBuffer key() {
|
||||||
if (magic() == MAGIC_VALUE_V0)
|
if (magic() == MAGIC_VALUE_V0)
|
||||||
return sliceDelimited(KEY_SIZE_OFFSET_V0);
|
return Utils.sizeDelimited(buffer, KEY_SIZE_OFFSET_V0);
|
||||||
else
|
else
|
||||||
return sliceDelimited(KEY_SIZE_OFFSET_V1);
|
return Utils.sizeDelimited(buffer, KEY_SIZE_OFFSET_V1);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Read a size-delimited byte buffer starting at the given offset
|
* Get the underlying buffer backing this record instance.
|
||||||
|
*
|
||||||
|
* @return the buffer
|
||||||
*/
|
*/
|
||||||
private ByteBuffer sliceDelimited(int start) {
|
public ByteBuffer buffer() {
|
||||||
int size = buffer.getInt(start);
|
return this.buffer;
|
||||||
if (size < 0) {
|
|
||||||
return null;
|
|
||||||
} else {
|
|
||||||
ByteBuffer b = buffer.duplicate();
|
|
||||||
b.position(start + 4);
|
|
||||||
b = b.slice();
|
|
||||||
b.limit(size);
|
|
||||||
b.rewind();
|
|
||||||
return b;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public String toString() {
|
public String toString() {
|
||||||
|
|
@ -434,4 +324,316 @@ public final class Record {
|
||||||
return buffer.hashCode();
|
return buffer.hashCode();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the size of this record if converted to the given format.
|
||||||
|
*
|
||||||
|
* @param toMagic The target magic version to convert to
|
||||||
|
* @return The size in bytes after conversion
|
||||||
|
*/
|
||||||
|
public int convertedSize(byte toMagic) {
|
||||||
|
return recordSize(toMagic, Math.max(0, keySize()), Math.max(0, valueSize()));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Convert this record to another message format.
|
||||||
|
*
|
||||||
|
* @param toMagic The target magic version to convert to
|
||||||
|
* @return A new record instance with a freshly allocated ByteBuffer.
|
||||||
|
*/
|
||||||
|
public Record convert(byte toMagic) {
|
||||||
|
if (toMagic == magic())
|
||||||
|
return this;
|
||||||
|
|
||||||
|
ByteBuffer buffer = ByteBuffer.allocate(convertedSize(toMagic));
|
||||||
|
TimestampType timestampType = wrapperRecordTimestampType != null ?
|
||||||
|
wrapperRecordTimestampType : TimestampType.forAttributes(attributes());
|
||||||
|
convertTo(buffer, toMagic, timestamp(), timestampType);
|
||||||
|
buffer.rewind();
|
||||||
|
return new Record(buffer);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void convertTo(ByteBuffer buffer, byte toMagic, long timestamp, TimestampType timestampType) {
|
||||||
|
if (compressionType() != CompressionType.NONE)
|
||||||
|
throw new IllegalArgumentException("Cannot use convertTo for deep conversion");
|
||||||
|
|
||||||
|
write(buffer, toMagic, timestamp, key(), value(), CompressionType.NONE, timestampType);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Convert this record to another message format and write the converted data to the provided outputs stream.
|
||||||
|
*
|
||||||
|
* @param out The output stream to write the converted data to
|
||||||
|
* @param toMagic The target magic version for conversion
|
||||||
|
* @param timestamp The timestamp to use in the converted record (for up-conversion)
|
||||||
|
* @param timestampType The timestamp type to use in the converted record (for up-conversion)
|
||||||
|
* @throws IOException for any IO errors writing the converted record.
|
||||||
|
*/
|
||||||
|
public void convertTo(DataOutputStream out, byte toMagic, long timestamp, TimestampType timestampType) throws IOException {
|
||||||
|
if (compressionType() != CompressionType.NONE)
|
||||||
|
throw new IllegalArgumentException("Cannot use convertTo for deep conversion");
|
||||||
|
|
||||||
|
write(out, toMagic, timestamp, key(), value(), CompressionType.NONE, timestampType);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create a new record instance. If the record's compression type is not none, then
|
||||||
|
* its value payload should be already compressed with the specified type; the constructor
|
||||||
|
* would always write the value payload as is and will not do the compression itself.
|
||||||
|
*
|
||||||
|
* @param magic The magic value to use
|
||||||
|
* @param timestamp The timestamp of the record
|
||||||
|
* @param key The key of the record (null, if none)
|
||||||
|
* @param value The record value
|
||||||
|
* @param compressionType The compression type used on the contents of the record (if any)
|
||||||
|
* @param timestampType The timestamp type to be used for this record
|
||||||
|
*/
|
||||||
|
public static Record create(byte magic,
|
||||||
|
long timestamp,
|
||||||
|
byte[] key,
|
||||||
|
byte[] value,
|
||||||
|
CompressionType compressionType,
|
||||||
|
TimestampType timestampType) {
|
||||||
|
int keySize = key == null ? 0 : key.length;
|
||||||
|
int valueSize = value == null ? 0 : value.length;
|
||||||
|
ByteBuffer buffer = ByteBuffer.allocate(recordSize(magic, keySize, valueSize));
|
||||||
|
write(buffer, magic, timestamp, wrapNullable(key), wrapNullable(value), compressionType, timestampType);
|
||||||
|
buffer.rewind();
|
||||||
|
return new Record(buffer);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Record create(long timestamp, byte[] key, byte[] value) {
|
||||||
|
return create(CURRENT_MAGIC_VALUE, timestamp, key, value, CompressionType.NONE, TimestampType.CREATE_TIME);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Record create(byte magic, long timestamp, byte[] key, byte[] value) {
|
||||||
|
return create(magic, timestamp, key, value, CompressionType.NONE, TimestampType.CREATE_TIME);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Record create(byte magic, TimestampType timestampType, long timestamp, byte[] key, byte[] value) {
|
||||||
|
return create(magic, timestamp, key, value, CompressionType.NONE, timestampType);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Record create(byte magic, long timestamp, byte[] value) {
|
||||||
|
return create(magic, timestamp, null, value, CompressionType.NONE, TimestampType.CREATE_TIME);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Record create(byte magic, byte[] key, byte[] value) {
|
||||||
|
return create(magic, NO_TIMESTAMP, key, value);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Record create(byte[] key, byte[] value) {
|
||||||
|
return create(NO_TIMESTAMP, key, value);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static Record create(byte[] value) {
|
||||||
|
return create(CURRENT_MAGIC_VALUE, NO_TIMESTAMP, null, value, CompressionType.NONE, TimestampType.CREATE_TIME);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the header for a compressed record set in-place (i.e. assuming the compressed record data has already
|
||||||
|
* been written at the value offset in a wrapped record). This lets you dynamically create a compressed message
|
||||||
|
* set, and then go back later and fill in its size and CRC, which saves the need for copying to another buffer.
|
||||||
|
*
|
||||||
|
* @param buffer The buffer containing the compressed record data positioned at the first offset of the
|
||||||
|
* @param magic The magic value of the record set
|
||||||
|
* @param recordSize The size of the record (including record overhead)
|
||||||
|
* @param timestamp The timestamp of the wrapper record
|
||||||
|
* @param compressionType The compression type used
|
||||||
|
* @param timestampType The timestamp type of the wrapper record
|
||||||
|
*/
|
||||||
|
public static void writeCompressedRecordHeader(ByteBuffer buffer,
|
||||||
|
byte magic,
|
||||||
|
int recordSize,
|
||||||
|
long timestamp,
|
||||||
|
CompressionType compressionType,
|
||||||
|
TimestampType timestampType) {
|
||||||
|
int recordPosition = buffer.position();
|
||||||
|
int valueSize = recordSize - recordOverhead(magic);
|
||||||
|
|
||||||
|
// write the record header with a null value (the key is always null for the wrapper)
|
||||||
|
write(buffer, magic, timestamp, null, null, compressionType, timestampType);
|
||||||
|
|
||||||
|
// now fill in the value size
|
||||||
|
buffer.putInt(recordPosition + keyOffset(magic), valueSize);
|
||||||
|
|
||||||
|
// compute and fill the crc from the beginning of the message
|
||||||
|
long crc = Utils.computeChecksum(buffer, recordPosition + MAGIC_OFFSET, recordSize - MAGIC_OFFSET);
|
||||||
|
Utils.writeUnsignedInt(buffer, recordPosition + CRC_OFFSET, crc);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void write(ByteBuffer buffer,
|
||||||
|
byte magic,
|
||||||
|
long timestamp,
|
||||||
|
ByteBuffer key,
|
||||||
|
ByteBuffer value,
|
||||||
|
CompressionType compressionType,
|
||||||
|
TimestampType timestampType) {
|
||||||
|
try {
|
||||||
|
ByteBufferOutputStream out = new ByteBufferOutputStream(buffer);
|
||||||
|
write(out, magic, timestamp, key, value, compressionType, timestampType);
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new KafkaException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write the record data with the given compression type and return the computed crc.
|
||||||
|
*
|
||||||
|
* @param out The output stream to write to
|
||||||
|
* @param magic The magic value to be used
|
||||||
|
* @param timestamp The timestamp of the record
|
||||||
|
* @param key The record key
|
||||||
|
* @param value The record value
|
||||||
|
* @param compressionType The compression type
|
||||||
|
* @param timestampType The timestamp type
|
||||||
|
* @return the computed CRC for this record.
|
||||||
|
* @throws IOException for any IO errors writing to the output stream.
|
||||||
|
*/
|
||||||
|
public static long write(DataOutputStream out,
|
||||||
|
byte magic,
|
||||||
|
long timestamp,
|
||||||
|
byte[] key,
|
||||||
|
byte[] value,
|
||||||
|
CompressionType compressionType,
|
||||||
|
TimestampType timestampType) throws IOException {
|
||||||
|
return write(out, magic, timestamp, wrapNullable(key), wrapNullable(value), compressionType, timestampType);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static long write(DataOutputStream out,
|
||||||
|
byte magic,
|
||||||
|
long timestamp,
|
||||||
|
ByteBuffer key,
|
||||||
|
ByteBuffer value,
|
||||||
|
CompressionType compressionType,
|
||||||
|
TimestampType timestampType) throws IOException {
|
||||||
|
byte attributes = computeAttributes(magic, compressionType, timestampType);
|
||||||
|
long crc = computeChecksum(magic, attributes, timestamp, key, value);
|
||||||
|
write(out, magic, crc, attributes, timestamp, key, value);
|
||||||
|
return crc;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Write a record using raw fields (without validation). This should only be used in testing.
|
||||||
|
*/
|
||||||
|
public static void write(DataOutputStream out,
|
||||||
|
byte magic,
|
||||||
|
long crc,
|
||||||
|
byte attributes,
|
||||||
|
long timestamp,
|
||||||
|
byte[] key,
|
||||||
|
byte[] value) throws IOException {
|
||||||
|
write(out, magic, crc, attributes, timestamp, wrapNullable(key), wrapNullable(value));
|
||||||
|
}
|
||||||
|
|
||||||
|
// Write a record to the buffer, if the record's compression type is none, then
|
||||||
|
// its value payload should be already compressed with the specified type
|
||||||
|
private static void write(DataOutputStream out,
|
||||||
|
byte magic,
|
||||||
|
long crc,
|
||||||
|
byte attributes,
|
||||||
|
long timestamp,
|
||||||
|
ByteBuffer key,
|
||||||
|
ByteBuffer value) throws IOException {
|
||||||
|
if (magic != MAGIC_VALUE_V0 && magic != MAGIC_VALUE_V1)
|
||||||
|
throw new IllegalArgumentException("Invalid magic value " + magic);
|
||||||
|
if (timestamp < 0 && timestamp != NO_TIMESTAMP)
|
||||||
|
throw new IllegalArgumentException("Invalid message timestamp " + timestamp);
|
||||||
|
|
||||||
|
// write crc
|
||||||
|
out.writeInt((int) (crc & 0xffffffffL));
|
||||||
|
// write magic value
|
||||||
|
out.writeByte(magic);
|
||||||
|
// write attributes
|
||||||
|
out.writeByte(attributes);
|
||||||
|
|
||||||
|
// maybe write timestamp
|
||||||
|
if (magic > 0)
|
||||||
|
out.writeLong(timestamp);
|
||||||
|
|
||||||
|
// write the key
|
||||||
|
if (key == null) {
|
||||||
|
out.writeInt(-1);
|
||||||
|
} else {
|
||||||
|
int size = key.remaining();
|
||||||
|
out.writeInt(size);
|
||||||
|
out.write(key.array(), key.arrayOffset(), size);
|
||||||
|
}
|
||||||
|
// write the value
|
||||||
|
if (value == null) {
|
||||||
|
out.writeInt(-1);
|
||||||
|
} else {
|
||||||
|
int size = value.remaining();
|
||||||
|
out.writeInt(size);
|
||||||
|
out.write(value.array(), value.arrayOffset(), size);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static int recordSize(byte[] key, byte[] value) {
|
||||||
|
return recordSize(CURRENT_MAGIC_VALUE, key, value);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static int recordSize(byte magic, byte[] key, byte[] value) {
|
||||||
|
return recordSize(magic, key == null ? 0 : key.length, value == null ? 0 : value.length);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static int recordSize(byte magic, int keySize, int valueSize) {
|
||||||
|
return recordOverhead(magic) + keySize + valueSize;
|
||||||
|
}
|
||||||
|
|
||||||
|
// visible only for testing
|
||||||
|
public static byte computeAttributes(byte magic, CompressionType type, TimestampType timestampType) {
|
||||||
|
byte attributes = 0;
|
||||||
|
if (type.id > 0)
|
||||||
|
attributes = (byte) (attributes | (COMPRESSION_CODEC_MASK & type.id));
|
||||||
|
if (magic > 0)
|
||||||
|
return timestampType.updateAttributes(attributes);
|
||||||
|
return attributes;
|
||||||
|
}
|
||||||
|
|
||||||
|
// visible only for testing
|
||||||
|
public static long computeChecksum(byte magic, byte attributes, long timestamp, byte[] key, byte[] value) {
|
||||||
|
return computeChecksum(magic, attributes, timestamp, wrapNullable(key), wrapNullable(value));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Compute the checksum of the record from the attributes, key and value payloads
|
||||||
|
*/
|
||||||
|
private static long computeChecksum(byte magic, byte attributes, long timestamp, ByteBuffer key, ByteBuffer value) {
|
||||||
|
Crc32 crc = new Crc32();
|
||||||
|
crc.update(magic);
|
||||||
|
crc.update(attributes);
|
||||||
|
if (magic > 0)
|
||||||
|
crc.updateLong(timestamp);
|
||||||
|
// update for the key
|
||||||
|
if (key == null) {
|
||||||
|
crc.updateInt(-1);
|
||||||
|
} else {
|
||||||
|
int size = key.remaining();
|
||||||
|
crc.updateInt(size);
|
||||||
|
crc.update(key.array(), key.arrayOffset(), size);
|
||||||
|
}
|
||||||
|
// update for the value
|
||||||
|
if (value == null) {
|
||||||
|
crc.updateInt(-1);
|
||||||
|
} else {
|
||||||
|
int size = value.remaining();
|
||||||
|
crc.updateInt(size);
|
||||||
|
crc.update(value.array(), value.arrayOffset(), size);
|
||||||
|
}
|
||||||
|
return crc.getValue();
|
||||||
|
}
|
||||||
|
|
||||||
|
public static int recordOverhead(byte magic) {
|
||||||
|
if (magic == 0)
|
||||||
|
return RECORD_OVERHEAD_V0;
|
||||||
|
return RECORD_OVERHEAD_V1;
|
||||||
|
}
|
||||||
|
|
||||||
|
private static int keyOffset(byte magic) {
|
||||||
|
if (magic == 0)
|
||||||
|
return KEY_OFFSET_V0;
|
||||||
|
return KEY_OFFSET_V1;
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -18,32 +18,74 @@ package org.apache.kafka.common.record;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.channels.GatheringByteChannel;
|
import java.nio.channels.GatheringByteChannel;
|
||||||
|
import java.util.Iterator;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A binary format which consists of a 4 byte size, an 8 byte offset, and the record bytes. See {@link MemoryRecords}
|
* Interface for accessing the records contained in a log. The log itself is represented as a sequence of log entries.
|
||||||
* for the in-memory representation.
|
* Each log entry consists of an 8 byte offset, a 4 byte record size, and a "shallow" {@link Record record}.
|
||||||
|
* If the entry is not compressed, then each entry will have only the shallow record contained inside it. If it is
|
||||||
|
* compressed, the entry contains "deep" records, which are packed into the value field of the shallow record. To iterate
|
||||||
|
* over the shallow records, use {@link #shallowIterator()}; for the deep records, use {@link #deepIterator()}. Note
|
||||||
|
* that the deep iterator handles both compressed and non-compressed entries: if the entry is not compressed, the
|
||||||
|
* shallow record is returned; otherwise, the shallow record is decompressed and the deep entries are returned.
|
||||||
|
* See {@link MemoryRecords} for the in-memory representation and {@link FileRecords} for the on-disk representation.
|
||||||
*/
|
*/
|
||||||
public interface Records extends Iterable<LogEntry> {
|
public interface Records {
|
||||||
|
|
||||||
int SIZE_LENGTH = 4;
|
int OFFSET_OFFSET = 0;
|
||||||
int OFFSET_LENGTH = 8;
|
int OFFSET_LENGTH = 8;
|
||||||
int LOG_OVERHEAD = SIZE_LENGTH + OFFSET_LENGTH;
|
int SIZE_OFFSET = OFFSET_OFFSET + OFFSET_LENGTH;
|
||||||
|
int SIZE_LENGTH = 4;
|
||||||
|
int LOG_OVERHEAD = SIZE_OFFSET + SIZE_LENGTH;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The size of these records in bytes
|
* The size of these records in bytes.
|
||||||
* @return The size in bytes
|
* @return The size in bytes of the records
|
||||||
*/
|
*/
|
||||||
int sizeInBytes();
|
int sizeInBytes();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Write the messages in this set to the given channel starting at the given offset byte.
|
* Write the contents of this buffer to a channel.
|
||||||
* @param channel The channel to write to
|
* @param channel The channel to write to
|
||||||
* @param position The position within this record set to begin writing from
|
* @param position The position in the buffer to write from
|
||||||
* @param length The number of bytes to write
|
* @param length The number of bytes to write
|
||||||
* @return The number of bytes written to the channel (which may be fewer than requested)
|
* @return The number of bytes written
|
||||||
* @throws IOException For any IO errors copying the
|
* @throws IOException For any IO errors
|
||||||
*/
|
*/
|
||||||
long writeTo(GatheringByteChannel channel, long position, int length) throws IOException;
|
long writeTo(GatheringByteChannel channel, long position, int length) throws IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the shallow log entries in this log buffer. Note that the signature allows subclasses
|
||||||
|
* to return a more specific log entry type. This enables optimizations such as in-place offset
|
||||||
|
* assignment (see {@link ByteBufferLogInputStream.ByteBufferLogEntry}), and partial reading of
|
||||||
|
* record data (see {@link FileLogInputStream.FileChannelLogEntry#magic()}.
|
||||||
|
* @return An iterator over the shallow entries of the log
|
||||||
|
*/
|
||||||
|
Iterator<? extends LogEntry> shallowIterator();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get the deep log entries (i.e. descend into compressed message sets). For the deep records,
|
||||||
|
* there are fewer options for optimization since the data must be decompressed before it can be
|
||||||
|
* returned. Hence there is little advantage in allowing subclasses to return a more specific type
|
||||||
|
* as we do for {@link #shallowIterator()}.
|
||||||
|
* @return An iterator over the deep entries of the log
|
||||||
|
*/
|
||||||
|
Iterator<LogEntry> deepIterator();
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Check whether all shallow entries in this buffer have a certain magic value.
|
||||||
|
* @param magic The magic value to check
|
||||||
|
* @return true if all shallow entries have a matching magic value, false otherwise
|
||||||
|
*/
|
||||||
|
boolean hasMatchingShallowMagic(byte magic);
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Convert all entries in this buffer to the format passed as a parameter. Note that this requires
|
||||||
|
* deep iteration since all of the deep records must also be converted to the desired format.
|
||||||
|
* @param toMagic The magic value to convert to
|
||||||
|
* @return A Records (which may or may not be the same instance)
|
||||||
|
*/
|
||||||
|
Records toMessageFormat(byte toMagic);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -17,6 +17,7 @@
|
||||||
package org.apache.kafka.common.record;
|
package org.apache.kafka.common.record;
|
||||||
|
|
||||||
import org.apache.kafka.common.KafkaException;
|
import org.apache.kafka.common.KafkaException;
|
||||||
|
import org.apache.kafka.common.errors.CorruptRecordException;
|
||||||
import org.apache.kafka.common.utils.AbstractIterator;
|
import org.apache.kafka.common.utils.AbstractIterator;
|
||||||
import org.apache.kafka.common.utils.Utils;
|
import org.apache.kafka.common.utils.Utils;
|
||||||
|
|
||||||
|
|
@ -25,51 +26,58 @@ import java.io.EOFException;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.util.ArrayDeque;
|
import java.util.ArrayDeque;
|
||||||
|
import java.util.Iterator;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* An iterator which handles both the shallow and deep iteration of record sets.
|
||||||
|
*/
|
||||||
public class RecordsIterator extends AbstractIterator<LogEntry> {
|
public class RecordsIterator extends AbstractIterator<LogEntry> {
|
||||||
private final LogInputStream logStream;
|
|
||||||
private final boolean shallow;
|
private final boolean shallow;
|
||||||
|
private final boolean ensureMatchingMagic;
|
||||||
|
private final int masRecordSize;
|
||||||
|
private final ShallowRecordsIterator<?> shallowIter;
|
||||||
private DeepRecordsIterator innerIter;
|
private DeepRecordsIterator innerIter;
|
||||||
|
|
||||||
public RecordsIterator(LogInputStream logStream, boolean shallow) {
|
public RecordsIterator(LogInputStream<?> logInputStream,
|
||||||
this.logStream = logStream;
|
boolean shallow,
|
||||||
|
boolean ensureMatchingMagic,
|
||||||
|
int masRecordSize) {
|
||||||
|
this.shallowIter = new ShallowRecordsIterator<>(logInputStream);
|
||||||
this.shallow = shallow;
|
this.shallow = shallow;
|
||||||
|
this.ensureMatchingMagic = ensureMatchingMagic;
|
||||||
|
this.masRecordSize = masRecordSize;
|
||||||
}
|
}
|
||||||
|
|
||||||
/*
|
/**
|
||||||
* Read the next record from the buffer.
|
* Get a shallow iterator over the given input stream.
|
||||||
*
|
* @param logInputStream The log input stream to read the entries from
|
||||||
* Note that in the compressed message set, each message value size is set as the size of the un-compressed
|
* @param <T> The type of the log entry
|
||||||
* version of the message value, so when we do de-compression allocating an array of the specified size for
|
* @return The shallow iterator.
|
||||||
* reading compressed value data is sufficient.
|
|
||||||
*/
|
*/
|
||||||
|
public static <T extends LogEntry> Iterator<T> shallowIterator(LogInputStream<T> logInputStream) {
|
||||||
|
return new ShallowRecordsIterator<>(logInputStream);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected LogEntry makeNext() {
|
protected LogEntry makeNext() {
|
||||||
if (innerDone()) {
|
if (innerDone()) {
|
||||||
try {
|
if (!shallowIter.hasNext())
|
||||||
LogEntry entry = logStream.nextEntry();
|
|
||||||
// No more record to return.
|
|
||||||
if (entry == null)
|
|
||||||
return allDone();
|
|
||||||
|
|
||||||
// decide whether to go shallow or deep iteration if it is compressed
|
|
||||||
CompressionType compressionType = entry.record().compressionType();
|
|
||||||
if (compressionType == CompressionType.NONE || shallow) {
|
|
||||||
return entry;
|
|
||||||
} else {
|
|
||||||
// init the inner iterator with the value payload of the message,
|
|
||||||
// which will de-compress the payload to a set of messages;
|
|
||||||
// since we assume nested compression is not allowed, the deep iterator
|
|
||||||
// would not try to further decompress underlying messages
|
|
||||||
// There will be at least one element in the inner iterator, so we don't
|
|
||||||
// need to call hasNext() here.
|
|
||||||
innerIter = new DeepRecordsIterator(entry);
|
|
||||||
return innerIter.next();
|
|
||||||
}
|
|
||||||
} catch (EOFException e) {
|
|
||||||
return allDone();
|
return allDone();
|
||||||
} catch (IOException e) {
|
|
||||||
throw new KafkaException(e);
|
LogEntry entry = shallowIter.next();
|
||||||
|
|
||||||
|
// decide whether to go shallow or deep iteration if it is compressed
|
||||||
|
if (shallow || !entry.isCompressed()) {
|
||||||
|
return entry;
|
||||||
|
} else {
|
||||||
|
// init the inner iterator with the value payload of the message,
|
||||||
|
// which will de-compress the payload to a set of messages;
|
||||||
|
// since we assume nested compression is not allowed, the deep iterator
|
||||||
|
// would not try to further decompress underlying messages
|
||||||
|
// There will be at least one element in the inner iterator, so we don't
|
||||||
|
// need to call hasNext() here.
|
||||||
|
innerIter = new DeepRecordsIterator(entry, ensureMatchingMagic, masRecordSize);
|
||||||
|
return innerIter.next();
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
return innerIter.next();
|
return innerIter.next();
|
||||||
|
|
@ -80,38 +88,70 @@ public class RecordsIterator extends AbstractIterator<LogEntry> {
|
||||||
return innerIter == null || !innerIter.hasNext();
|
return innerIter == null || !innerIter.hasNext();
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class DataLogInputStream implements LogInputStream {
|
private static class DataLogInputStream implements LogInputStream<LogEntry> {
|
||||||
private final DataInputStream stream;
|
private final DataInputStream stream;
|
||||||
|
protected final int maxMessageSize;
|
||||||
|
|
||||||
private DataLogInputStream(DataInputStream stream) {
|
DataLogInputStream(DataInputStream stream, int maxMessageSize) {
|
||||||
this.stream = stream;
|
this.stream = stream;
|
||||||
|
this.maxMessageSize = maxMessageSize;
|
||||||
}
|
}
|
||||||
|
|
||||||
public LogEntry nextEntry() throws IOException {
|
public LogEntry nextEntry() throws IOException {
|
||||||
long offset = stream.readLong();
|
try {
|
||||||
int size = stream.readInt();
|
long offset = stream.readLong();
|
||||||
if (size < 0)
|
int size = stream.readInt();
|
||||||
throw new IllegalStateException("Record with size " + size);
|
if (size < Record.RECORD_OVERHEAD_V0)
|
||||||
|
throw new CorruptRecordException(String.format("Record size is less than the minimum record overhead (%d)", Record.RECORD_OVERHEAD_V0));
|
||||||
|
if (size > maxMessageSize)
|
||||||
|
throw new CorruptRecordException(String.format("Record size exceeds the largest allowable message size (%d).", maxMessageSize));
|
||||||
|
|
||||||
byte[] recordBuffer = new byte[size];
|
byte[] recordBuffer = new byte[size];
|
||||||
stream.readFully(recordBuffer, 0, size);
|
stream.readFully(recordBuffer, 0, size);
|
||||||
ByteBuffer buf = ByteBuffer.wrap(recordBuffer);
|
ByteBuffer buf = ByteBuffer.wrap(recordBuffer);
|
||||||
return new LogEntry(offset, new Record(buf));
|
return LogEntry.create(offset, new Record(buf));
|
||||||
|
} catch (EOFException e) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class DeepRecordsIterator extends AbstractIterator<LogEntry> {
|
private static class ShallowRecordsIterator<T extends LogEntry> extends AbstractIterator<T> {
|
||||||
|
private final LogInputStream<T> logStream;
|
||||||
|
|
||||||
|
public ShallowRecordsIterator(LogInputStream<T> logStream) {
|
||||||
|
this.logStream = logStream;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected T makeNext() {
|
||||||
|
try {
|
||||||
|
T entry = logStream.nextEntry();
|
||||||
|
if (entry == null)
|
||||||
|
return allDone();
|
||||||
|
return entry;
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new KafkaException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static class DeepRecordsIterator extends AbstractIterator<LogEntry> {
|
||||||
private final ArrayDeque<LogEntry> logEntries;
|
private final ArrayDeque<LogEntry> logEntries;
|
||||||
private final long absoluteBaseOffset;
|
private final long absoluteBaseOffset;
|
||||||
|
private final byte wrapperMagic;
|
||||||
|
|
||||||
private DeepRecordsIterator(LogEntry entry) {
|
public DeepRecordsIterator(LogEntry wrapperEntry, boolean ensureMatchingMagic, int maxMessageSize) {
|
||||||
CompressionType compressionType = entry.record().compressionType();
|
Record wrapperRecord = wrapperEntry.record();
|
||||||
ByteBuffer buffer = entry.record().value();
|
this.wrapperMagic = wrapperRecord.magic();
|
||||||
DataInputStream stream = Compressor.wrapForInput(new ByteBufferInputStream(buffer), compressionType, entry.record().magic());
|
|
||||||
LogInputStream logStream = new DataLogInputStream(stream);
|
|
||||||
|
|
||||||
long wrapperRecordOffset = entry.offset();
|
CompressionType compressionType = wrapperRecord.compressionType();
|
||||||
long wrapperRecordTimestamp = entry.record().timestamp();
|
ByteBuffer buffer = wrapperRecord.value();
|
||||||
|
DataInputStream stream = MemoryRecordsBuilder.wrapForInput(new ByteBufferInputStream(buffer), compressionType, wrapperRecord.magic());
|
||||||
|
LogInputStream logStream = new DataLogInputStream(stream, maxMessageSize);
|
||||||
|
|
||||||
|
long wrapperRecordOffset = wrapperEntry.offset();
|
||||||
|
long wrapperRecordTimestamp = wrapperRecord.timestamp();
|
||||||
this.logEntries = new ArrayDeque<>();
|
this.logEntries = new ArrayDeque<>();
|
||||||
|
|
||||||
// If relative offset is used, we need to decompress the entire message first to compute
|
// If relative offset is used, we need to decompress the entire message first to compute
|
||||||
|
|
@ -119,22 +159,27 @@ public class RecordsIterator extends AbstractIterator<LogEntry> {
|
||||||
// do the same for message format version 0
|
// do the same for message format version 0
|
||||||
try {
|
try {
|
||||||
while (true) {
|
while (true) {
|
||||||
try {
|
LogEntry logEntry = logStream.nextEntry();
|
||||||
LogEntry logEntry = logStream.nextEntry();
|
if (logEntry == null)
|
||||||
if (entry.record().magic() > Record.MAGIC_VALUE_V0) {
|
|
||||||
Record recordWithTimestamp = new Record(
|
|
||||||
logEntry.record().buffer(),
|
|
||||||
wrapperRecordTimestamp,
|
|
||||||
entry.record().timestampType()
|
|
||||||
);
|
|
||||||
logEntry = new LogEntry(logEntry.offset(), recordWithTimestamp);
|
|
||||||
}
|
|
||||||
logEntries.add(logEntry);
|
|
||||||
} catch (EOFException e) {
|
|
||||||
break;
|
break;
|
||||||
|
|
||||||
|
Record record = logEntry.record();
|
||||||
|
byte magic = record.magic();
|
||||||
|
|
||||||
|
if (ensureMatchingMagic && magic != wrapperMagic)
|
||||||
|
throw new InvalidRecordException("Compressed message magic does not match wrapper magic");
|
||||||
|
|
||||||
|
if (magic > Record.MAGIC_VALUE_V0) {
|
||||||
|
Record recordWithTimestamp = new Record(
|
||||||
|
record.buffer(),
|
||||||
|
wrapperRecordTimestamp,
|
||||||
|
wrapperRecord.timestampType()
|
||||||
|
);
|
||||||
|
logEntry = LogEntry.create(logEntry.offset(), recordWithTimestamp);
|
||||||
}
|
}
|
||||||
|
logEntries.addLast(logEntry);
|
||||||
}
|
}
|
||||||
if (entry.record().magic() > Record.MAGIC_VALUE_V0)
|
if (wrapperMagic > Record.MAGIC_VALUE_V0)
|
||||||
this.absoluteBaseOffset = wrapperRecordOffset - logEntries.getLast().offset();
|
this.absoluteBaseOffset = wrapperRecordOffset - logEntries.getLast().offset();
|
||||||
else
|
else
|
||||||
this.absoluteBaseOffset = -1;
|
this.absoluteBaseOffset = -1;
|
||||||
|
|
@ -155,12 +200,10 @@ public class RecordsIterator extends AbstractIterator<LogEntry> {
|
||||||
// Convert offset to absolute offset if needed.
|
// Convert offset to absolute offset if needed.
|
||||||
if (absoluteBaseOffset >= 0) {
|
if (absoluteBaseOffset >= 0) {
|
||||||
long absoluteOffset = absoluteBaseOffset + entry.offset();
|
long absoluteOffset = absoluteBaseOffset + entry.offset();
|
||||||
entry = new LogEntry(absoluteOffset, entry.record());
|
entry = LogEntry.create(absoluteOffset, entry.record());
|
||||||
}
|
}
|
||||||
|
|
||||||
// decide whether to go shallow or deep iteration if it is compressed
|
if (entry.isCompressed())
|
||||||
CompressionType compression = entry.record().compressionType();
|
|
||||||
if (compression != CompressionType.NONE)
|
|
||||||
throw new InvalidRecordException("Inner messages must not be compressed");
|
throw new InvalidRecordException("Inner messages must not be compressed");
|
||||||
|
|
||||||
return entry;
|
return entry;
|
||||||
|
|
|
||||||
|
|
@ -27,6 +27,7 @@ public enum TimestampType {
|
||||||
|
|
||||||
public final int id;
|
public final int id;
|
||||||
public final String name;
|
public final String name;
|
||||||
|
|
||||||
TimestampType(int id, String name) {
|
TimestampType(int id, String name) {
|
||||||
this.id = id;
|
this.id = id;
|
||||||
this.name = name;
|
this.name = name;
|
||||||
|
|
|
||||||
|
|
@ -265,6 +265,24 @@ public class Utils {
|
||||||
return toArray(buffer, 0, buffer.limit());
|
return toArray(buffer, 0, buffer.limit());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Convert a ByteBuffer to a nullable array.
|
||||||
|
* @param buffer The buffer to convert
|
||||||
|
* @return The resulting array or null if the buffer is null
|
||||||
|
*/
|
||||||
|
public static byte[] toNullableArray(ByteBuffer buffer) {
|
||||||
|
return buffer == null ? null : toArray(buffer);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Wrap an array as a nullable ByteBuffer.
|
||||||
|
* @param array The nullable array to wrap
|
||||||
|
* @return The wrapping ByteBuffer or null if array is null
|
||||||
|
*/
|
||||||
|
public static ByteBuffer wrapNullable(byte[] array) {
|
||||||
|
return array == null ? null : ByteBuffer.wrap(array);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Read a byte array from the given offset and size in the buffer
|
* Read a byte array from the given offset and size in the buffer
|
||||||
*/
|
*/
|
||||||
|
|
@ -733,4 +751,37 @@ public class Utils {
|
||||||
public static int longHashcode(long value) {
|
public static int longHashcode(long value) {
|
||||||
return (int) (value ^ (value >>> 32));
|
return (int) (value ^ (value >>> 32));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Read a size-delimited byte buffer starting at the given offset.
|
||||||
|
* @param buffer Buffer containing the size and data
|
||||||
|
* @param start Offset in the buffer to read from
|
||||||
|
* @return A slice of the buffer containing only the delimited data (excluding the size)
|
||||||
|
*/
|
||||||
|
public static ByteBuffer sizeDelimited(ByteBuffer buffer, int start) {
|
||||||
|
int size = buffer.getInt(start);
|
||||||
|
if (size < 0) {
|
||||||
|
return null;
|
||||||
|
} else {
|
||||||
|
ByteBuffer b = buffer.duplicate();
|
||||||
|
b.position(start + 4);
|
||||||
|
b = b.slice();
|
||||||
|
b.limit(size);
|
||||||
|
b.rewind();
|
||||||
|
return b;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Compute the checksum of a range of data
|
||||||
|
* @param buffer Buffer containing the data to checksum
|
||||||
|
* @param start Offset in the buffer to read from
|
||||||
|
* @param size The number of bytes to include
|
||||||
|
*/
|
||||||
|
public static long computeChecksum(ByteBuffer buffer, int start, int size) {
|
||||||
|
Crc32 crc = new Crc32();
|
||||||
|
crc.update(buffer.array(), buffer.arrayOffset() + start, size);
|
||||||
|
return crc.getValue();
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -39,6 +39,8 @@ import org.apache.kafka.common.protocol.ApiKeys;
|
||||||
import org.apache.kafka.common.protocol.Errors;
|
import org.apache.kafka.common.protocol.Errors;
|
||||||
import org.apache.kafka.common.record.CompressionType;
|
import org.apache.kafka.common.record.CompressionType;
|
||||||
import org.apache.kafka.common.record.MemoryRecords;
|
import org.apache.kafka.common.record.MemoryRecords;
|
||||||
|
import org.apache.kafka.common.record.MemoryRecordsBuilder;
|
||||||
|
import org.apache.kafka.common.record.TimestampType;
|
||||||
import org.apache.kafka.common.requests.AbstractRequest;
|
import org.apache.kafka.common.requests.AbstractRequest;
|
||||||
import org.apache.kafka.common.requests.FetchResponse;
|
import org.apache.kafka.common.requests.FetchResponse;
|
||||||
import org.apache.kafka.common.requests.FetchResponse.PartitionData;
|
import org.apache.kafka.common.requests.FetchResponse.PartitionData;
|
||||||
|
|
@ -1323,11 +1325,10 @@ public class KafkaConsumerTest {
|
||||||
TopicPartition partition = fetchEntry.getKey();
|
TopicPartition partition = fetchEntry.getKey();
|
||||||
long fetchOffset = fetchEntry.getValue().offset;
|
long fetchOffset = fetchEntry.getValue().offset;
|
||||||
int fetchCount = fetchEntry.getValue().count;
|
int fetchCount = fetchEntry.getValue().count;
|
||||||
MemoryRecords records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE);
|
MemoryRecordsBuilder records = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME);
|
||||||
for (int i = 0; i < fetchCount; i++)
|
for (int i = 0; i < fetchCount; i++)
|
||||||
records.append(fetchOffset + i, 0L, ("key-" + i).getBytes(), ("value-" + i).getBytes());
|
records.append(fetchOffset + i, 0L, ("key-" + i).getBytes(), ("value-" + i).getBytes());
|
||||||
records.close();
|
tpResponses.put(partition, new FetchResponse.PartitionData(Errors.NONE.code(), 0, records.build()));
|
||||||
tpResponses.put(partition, new FetchResponse.PartitionData(Errors.NONE.code(), 0, records));
|
|
||||||
}
|
}
|
||||||
return new FetchResponse(tpResponses, 0);
|
return new FetchResponse(tpResponses, 0);
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -37,10 +37,12 @@ import org.apache.kafka.common.errors.TopicAuthorizationException;
|
||||||
import org.apache.kafka.common.metrics.KafkaMetric;
|
import org.apache.kafka.common.metrics.KafkaMetric;
|
||||||
import org.apache.kafka.common.metrics.Metrics;
|
import org.apache.kafka.common.metrics.Metrics;
|
||||||
import org.apache.kafka.common.protocol.Errors;
|
import org.apache.kafka.common.protocol.Errors;
|
||||||
|
import org.apache.kafka.common.record.ByteBufferOutputStream;
|
||||||
import org.apache.kafka.common.record.CompressionType;
|
import org.apache.kafka.common.record.CompressionType;
|
||||||
import org.apache.kafka.common.record.Compressor;
|
|
||||||
import org.apache.kafka.common.record.MemoryRecords;
|
import org.apache.kafka.common.record.MemoryRecords;
|
||||||
|
import org.apache.kafka.common.record.MemoryRecordsBuilder;
|
||||||
import org.apache.kafka.common.record.Record;
|
import org.apache.kafka.common.record.Record;
|
||||||
|
import org.apache.kafka.common.record.TimestampType;
|
||||||
import org.apache.kafka.common.requests.AbstractRequest;
|
import org.apache.kafka.common.requests.AbstractRequest;
|
||||||
import org.apache.kafka.common.requests.FetchRequest;
|
import org.apache.kafka.common.requests.FetchRequest;
|
||||||
import org.apache.kafka.common.requests.FetchResponse;
|
import org.apache.kafka.common.requests.FetchResponse;
|
||||||
|
|
@ -93,8 +95,8 @@ public class FetcherTest {
|
||||||
private static final double EPSILON = 0.0001;
|
private static final double EPSILON = 0.0001;
|
||||||
private ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(client, metadata, time, 100, 1000);
|
private ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(client, metadata, time, 100, 1000);
|
||||||
|
|
||||||
private MemoryRecords records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE);
|
private MemoryRecords records;
|
||||||
private MemoryRecords nextRecords = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE);
|
private MemoryRecords nextRecords;
|
||||||
private Fetcher<byte[], byte[]> fetcher = createFetcher(subscriptions, metrics);
|
private Fetcher<byte[], byte[]> fetcher = createFetcher(subscriptions, metrics);
|
||||||
private Metrics fetcherMetrics = new Metrics(time);
|
private Metrics fetcherMetrics = new Metrics(time);
|
||||||
private Fetcher<byte[], byte[]> fetcherNoAutoReset = createFetcher(subscriptionsNoAutoReset, fetcherMetrics);
|
private Fetcher<byte[], byte[]> fetcherNoAutoReset = createFetcher(subscriptionsNoAutoReset, fetcherMetrics);
|
||||||
|
|
@ -104,14 +106,16 @@ public class FetcherTest {
|
||||||
metadata.update(cluster, time.milliseconds());
|
metadata.update(cluster, time.milliseconds());
|
||||||
client.setNode(node);
|
client.setNode(node);
|
||||||
|
|
||||||
records.append(1L, 0L, "key".getBytes(), "value-1".getBytes());
|
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME);
|
||||||
records.append(2L, 0L, "key".getBytes(), "value-2".getBytes());
|
builder.append(1L, 0L, "key".getBytes(), "value-1".getBytes());
|
||||||
records.append(3L, 0L, "key".getBytes(), "value-3".getBytes());
|
builder.append(2L, 0L, "key".getBytes(), "value-2".getBytes());
|
||||||
records.close();
|
builder.append(3L, 0L, "key".getBytes(), "value-3".getBytes());
|
||||||
|
records = builder.build();
|
||||||
|
|
||||||
nextRecords.append(4L, 0L, "key".getBytes(), "value-4".getBytes());
|
builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME);
|
||||||
nextRecords.append(5L, 0L, "key".getBytes(), "value-5".getBytes());
|
builder.append(4L, 0L, "key".getBytes(), "value-4".getBytes());
|
||||||
nextRecords.close();
|
builder.append(5L, 0L, "key".getBytes(), "value-5".getBytes());
|
||||||
|
nextRecords = builder.build();
|
||||||
}
|
}
|
||||||
|
|
||||||
@After
|
@After
|
||||||
|
|
@ -129,7 +133,7 @@ public class FetcherTest {
|
||||||
assertEquals(1, fetcher.sendFetches());
|
assertEquals(1, fetcher.sendFetches());
|
||||||
assertFalse(fetcher.hasCompletedFetches());
|
assertFalse(fetcher.hasCompletedFetches());
|
||||||
|
|
||||||
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0));
|
client.prepareResponse(fetchResponse(this.records, Errors.NONE.code(), 100L, 0));
|
||||||
consumerClient.poll(0);
|
consumerClient.poll(0);
|
||||||
assertTrue(fetcher.hasCompletedFetches());
|
assertTrue(fetcher.hasCompletedFetches());
|
||||||
|
|
||||||
|
|
@ -154,7 +158,7 @@ public class FetcherTest {
|
||||||
assertEquals(1, fetcher.sendFetches());
|
assertEquals(1, fetcher.sendFetches());
|
||||||
assertFalse(fetcher.hasCompletedFetches());
|
assertFalse(fetcher.hasCompletedFetches());
|
||||||
|
|
||||||
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NOT_LEADER_FOR_PARTITION.code(), 100L, 0));
|
client.prepareResponse(fetchResponse(this.records, Errors.NOT_LEADER_FOR_PARTITION.code(), 100L, 0));
|
||||||
consumerClient.poll(0);
|
consumerClient.poll(0);
|
||||||
assertTrue(fetcher.hasCompletedFetches());
|
assertTrue(fetcher.hasCompletedFetches());
|
||||||
|
|
||||||
|
|
@ -192,7 +196,7 @@ public class FetcherTest {
|
||||||
subscriptions.assignFromUser(singleton(tp));
|
subscriptions.assignFromUser(singleton(tp));
|
||||||
subscriptions.seek(tp, 1);
|
subscriptions.seek(tp, 1);
|
||||||
|
|
||||||
client.prepareResponse(matchesOffset(tp, 1), fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0));
|
client.prepareResponse(matchesOffset(tp, 1), fetchResponse(this.records, Errors.NONE.code(), 100L, 0));
|
||||||
|
|
||||||
assertEquals(1, fetcher.sendFetches());
|
assertEquals(1, fetcher.sendFetches());
|
||||||
consumerClient.poll(0);
|
consumerClient.poll(0);
|
||||||
|
|
@ -206,29 +210,30 @@ public class FetcherTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testParseInvalidRecord() {
|
public void testParseInvalidRecord() throws Exception {
|
||||||
ByteBuffer buffer = ByteBuffer.allocate(1024);
|
ByteBuffer buffer = ByteBuffer.allocate(1024);
|
||||||
Compressor compressor = new Compressor(buffer, CompressionType.NONE);
|
ByteBufferOutputStream out = new ByteBufferOutputStream(buffer);
|
||||||
|
|
||||||
|
byte magic = Record.CURRENT_MAGIC_VALUE;
|
||||||
byte[] key = "foo".getBytes();
|
byte[] key = "foo".getBytes();
|
||||||
byte[] value = "baz".getBytes();
|
byte[] value = "baz".getBytes();
|
||||||
long offset = 0;
|
long offset = 0;
|
||||||
long timestamp = 500L;
|
long timestamp = 500L;
|
||||||
|
|
||||||
int size = Record.recordSize(key, value);
|
int size = Record.recordSize(key, value);
|
||||||
long crc = Record.computeChecksum(timestamp, key, value, CompressionType.NONE, 0, -1);
|
byte attributes = Record.computeAttributes(magic, CompressionType.NONE, TimestampType.CREATE_TIME);
|
||||||
|
long crc = Record.computeChecksum(magic, attributes, timestamp, key, value);
|
||||||
|
|
||||||
// write one valid record
|
// write one valid record
|
||||||
compressor.putLong(offset);
|
out.writeLong(offset);
|
||||||
compressor.putInt(size);
|
out.writeInt(size);
|
||||||
Record.write(compressor, crc, Record.computeAttributes(CompressionType.NONE), timestamp, key, value, 0, -1);
|
Record.write(out, magic, crc, Record.computeAttributes(magic, CompressionType.NONE, TimestampType.CREATE_TIME), timestamp, key, value);
|
||||||
|
|
||||||
// and one invalid record (note the crc)
|
// and one invalid record (note the crc)
|
||||||
compressor.putLong(offset);
|
out.writeLong(offset);
|
||||||
compressor.putInt(size);
|
out.writeInt(size);
|
||||||
Record.write(compressor, crc + 1, Record.computeAttributes(CompressionType.NONE), timestamp, key, value, 0, -1);
|
Record.write(out, magic, crc + 1, Record.computeAttributes(magic, CompressionType.NONE, TimestampType.CREATE_TIME), timestamp, key, value);
|
||||||
|
|
||||||
compressor.close();
|
|
||||||
buffer.flip();
|
buffer.flip();
|
||||||
|
|
||||||
subscriptions.assignFromUser(singleton(tp));
|
subscriptions.assignFromUser(singleton(tp));
|
||||||
|
|
@ -236,7 +241,7 @@ public class FetcherTest {
|
||||||
|
|
||||||
// normal fetch
|
// normal fetch
|
||||||
assertEquals(1, fetcher.sendFetches());
|
assertEquals(1, fetcher.sendFetches());
|
||||||
client.prepareResponse(fetchResponse(buffer, Errors.NONE.code(), 100L, 0));
|
client.prepareResponse(fetchResponse(MemoryRecords.readableRecords(buffer), Errors.NONE.code(), 100L, 0));
|
||||||
consumerClient.poll(0);
|
consumerClient.poll(0);
|
||||||
try {
|
try {
|
||||||
fetcher.fetchedRecords();
|
fetcher.fetchedRecords();
|
||||||
|
|
@ -255,8 +260,8 @@ public class FetcherTest {
|
||||||
subscriptions.assignFromUser(singleton(tp));
|
subscriptions.assignFromUser(singleton(tp));
|
||||||
subscriptions.seek(tp, 1);
|
subscriptions.seek(tp, 1);
|
||||||
|
|
||||||
client.prepareResponse(matchesOffset(tp, 1), fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0));
|
client.prepareResponse(matchesOffset(tp, 1), fetchResponse(this.records, Errors.NONE.code(), 100L, 0));
|
||||||
client.prepareResponse(matchesOffset(tp, 4), fetchResponse(this.nextRecords.buffer(), Errors.NONE.code(), 100L, 0));
|
client.prepareResponse(matchesOffset(tp, 4), fetchResponse(this.nextRecords, Errors.NONE.code(), 100L, 0));
|
||||||
|
|
||||||
assertEquals(1, fetcher.sendFetches());
|
assertEquals(1, fetcher.sendFetches());
|
||||||
consumerClient.poll(0);
|
consumerClient.poll(0);
|
||||||
|
|
@ -287,11 +292,11 @@ public class FetcherTest {
|
||||||
// if we are fetching from a compacted topic, there may be gaps in the returned records
|
// if we are fetching from a compacted topic, there may be gaps in the returned records
|
||||||
// this test verifies the fetcher updates the current fetched/consumed positions correctly for this case
|
// this test verifies the fetcher updates the current fetched/consumed positions correctly for this case
|
||||||
|
|
||||||
MemoryRecords records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE);
|
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME);
|
||||||
records.append(15L, 0L, "key".getBytes(), "value-1".getBytes());
|
builder.append(15L, 0L, "key".getBytes(), "value-1".getBytes());
|
||||||
records.append(20L, 0L, "key".getBytes(), "value-2".getBytes());
|
builder.append(20L, 0L, "key".getBytes(), "value-2".getBytes());
|
||||||
records.append(30L, 0L, "key".getBytes(), "value-3".getBytes());
|
builder.append(30L, 0L, "key".getBytes(), "value-3".getBytes());
|
||||||
records.close();
|
MemoryRecords records = builder.build();
|
||||||
|
|
||||||
List<ConsumerRecord<byte[], byte[]>> consumerRecords;
|
List<ConsumerRecord<byte[], byte[]>> consumerRecords;
|
||||||
subscriptions.assignFromUser(singleton(tp));
|
subscriptions.assignFromUser(singleton(tp));
|
||||||
|
|
@ -299,7 +304,7 @@ public class FetcherTest {
|
||||||
|
|
||||||
// normal fetch
|
// normal fetch
|
||||||
assertEquals(1, fetcher.sendFetches());
|
assertEquals(1, fetcher.sendFetches());
|
||||||
client.prepareResponse(fetchResponse(records.buffer(), Errors.NONE.code(), 100L, 0));
|
client.prepareResponse(fetchResponse(records, Errors.NONE.code(), 100L, 0));
|
||||||
consumerClient.poll(0);
|
consumerClient.poll(0);
|
||||||
consumerRecords = fetcher.fetchedRecords().get(tp);
|
consumerRecords = fetcher.fetchedRecords().get(tp);
|
||||||
assertEquals(3, consumerRecords.size());
|
assertEquals(3, consumerRecords.size());
|
||||||
|
|
@ -317,7 +322,7 @@ public class FetcherTest {
|
||||||
|
|
||||||
// resize the limit of the buffer to pretend it is only fetch-size large
|
// resize the limit of the buffer to pretend it is only fetch-size large
|
||||||
assertEquals(1, fetcher.sendFetches());
|
assertEquals(1, fetcher.sendFetches());
|
||||||
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.TOPIC_AUTHORIZATION_FAILED.code(), 100L, 0));
|
client.prepareResponse(fetchResponse(this.records, Errors.TOPIC_AUTHORIZATION_FAILED.code(), 100L, 0));
|
||||||
consumerClient.poll(0);
|
consumerClient.poll(0);
|
||||||
try {
|
try {
|
||||||
fetcher.fetchedRecords();
|
fetcher.fetchedRecords();
|
||||||
|
|
@ -337,7 +342,7 @@ public class FetcherTest {
|
||||||
|
|
||||||
// Now the rebalance happens and fetch positions are cleared
|
// Now the rebalance happens and fetch positions are cleared
|
||||||
subscriptions.assignFromSubscribed(singleton(tp));
|
subscriptions.assignFromSubscribed(singleton(tp));
|
||||||
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0));
|
client.prepareResponse(fetchResponse(this.records, Errors.NONE.code(), 100L, 0));
|
||||||
consumerClient.poll(0);
|
consumerClient.poll(0);
|
||||||
|
|
||||||
// The active fetch should be ignored since its position is no longer valid
|
// The active fetch should be ignored since its position is no longer valid
|
||||||
|
|
@ -352,7 +357,7 @@ public class FetcherTest {
|
||||||
assertEquals(1, fetcher.sendFetches());
|
assertEquals(1, fetcher.sendFetches());
|
||||||
subscriptions.pause(tp);
|
subscriptions.pause(tp);
|
||||||
|
|
||||||
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0));
|
client.prepareResponse(fetchResponse(this.records, Errors.NONE.code(), 100L, 0));
|
||||||
consumerClient.poll(0);
|
consumerClient.poll(0);
|
||||||
assertNull(fetcher.fetchedRecords().get(tp));
|
assertNull(fetcher.fetchedRecords().get(tp));
|
||||||
}
|
}
|
||||||
|
|
@ -373,7 +378,7 @@ public class FetcherTest {
|
||||||
subscriptions.seek(tp, 0);
|
subscriptions.seek(tp, 0);
|
||||||
|
|
||||||
assertEquals(1, fetcher.sendFetches());
|
assertEquals(1, fetcher.sendFetches());
|
||||||
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NOT_LEADER_FOR_PARTITION.code(), 100L, 0));
|
client.prepareResponse(fetchResponse(this.records, Errors.NOT_LEADER_FOR_PARTITION.code(), 100L, 0));
|
||||||
consumerClient.poll(0);
|
consumerClient.poll(0);
|
||||||
assertEquals(0, fetcher.fetchedRecords().size());
|
assertEquals(0, fetcher.fetchedRecords().size());
|
||||||
assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds()));
|
assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds()));
|
||||||
|
|
@ -385,7 +390,7 @@ public class FetcherTest {
|
||||||
subscriptions.seek(tp, 0);
|
subscriptions.seek(tp, 0);
|
||||||
|
|
||||||
assertEquals(1, fetcher.sendFetches());
|
assertEquals(1, fetcher.sendFetches());
|
||||||
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), 100L, 0));
|
client.prepareResponse(fetchResponse(this.records, Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), 100L, 0));
|
||||||
consumerClient.poll(0);
|
consumerClient.poll(0);
|
||||||
assertEquals(0, fetcher.fetchedRecords().size());
|
assertEquals(0, fetcher.fetchedRecords().size());
|
||||||
assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds()));
|
assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds()));
|
||||||
|
|
@ -397,7 +402,7 @@ public class FetcherTest {
|
||||||
subscriptions.seek(tp, 0);
|
subscriptions.seek(tp, 0);
|
||||||
|
|
||||||
assertEquals(1, fetcher.sendFetches());
|
assertEquals(1, fetcher.sendFetches());
|
||||||
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0));
|
client.prepareResponse(fetchResponse(this.records, Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0));
|
||||||
consumerClient.poll(0);
|
consumerClient.poll(0);
|
||||||
assertEquals(0, fetcher.fetchedRecords().size());
|
assertEquals(0, fetcher.fetchedRecords().size());
|
||||||
assertTrue(subscriptions.isOffsetResetNeeded(tp));
|
assertTrue(subscriptions.isOffsetResetNeeded(tp));
|
||||||
|
|
@ -412,7 +417,7 @@ public class FetcherTest {
|
||||||
subscriptions.seek(tp, 0);
|
subscriptions.seek(tp, 0);
|
||||||
|
|
||||||
assertEquals(1, fetcher.sendFetches());
|
assertEquals(1, fetcher.sendFetches());
|
||||||
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0));
|
client.prepareResponse(fetchResponse(this.records, Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0));
|
||||||
subscriptions.seek(tp, 1);
|
subscriptions.seek(tp, 1);
|
||||||
consumerClient.poll(0);
|
consumerClient.poll(0);
|
||||||
assertEquals(0, fetcher.fetchedRecords().size());
|
assertEquals(0, fetcher.fetchedRecords().size());
|
||||||
|
|
@ -426,7 +431,7 @@ public class FetcherTest {
|
||||||
subscriptionsNoAutoReset.seek(tp, 0);
|
subscriptionsNoAutoReset.seek(tp, 0);
|
||||||
|
|
||||||
assertTrue(fetcherNoAutoReset.sendFetches() > 0);
|
assertTrue(fetcherNoAutoReset.sendFetches() > 0);
|
||||||
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0));
|
client.prepareResponse(fetchResponse(this.records, Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0));
|
||||||
consumerClient.poll(0);
|
consumerClient.poll(0);
|
||||||
assertFalse(subscriptionsNoAutoReset.isOffsetResetNeeded(tp));
|
assertFalse(subscriptionsNoAutoReset.isOffsetResetNeeded(tp));
|
||||||
subscriptionsNoAutoReset.seek(tp, 2);
|
subscriptionsNoAutoReset.seek(tp, 2);
|
||||||
|
|
@ -439,7 +444,7 @@ public class FetcherTest {
|
||||||
subscriptionsNoAutoReset.seek(tp, 0);
|
subscriptionsNoAutoReset.seek(tp, 0);
|
||||||
|
|
||||||
fetcherNoAutoReset.sendFetches();
|
fetcherNoAutoReset.sendFetches();
|
||||||
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0));
|
client.prepareResponse(fetchResponse(this.records, Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0));
|
||||||
consumerClient.poll(0);
|
consumerClient.poll(0);
|
||||||
|
|
||||||
assertFalse(subscriptionsNoAutoReset.isOffsetResetNeeded(tp));
|
assertFalse(subscriptionsNoAutoReset.isOffsetResetNeeded(tp));
|
||||||
|
|
@ -459,7 +464,7 @@ public class FetcherTest {
|
||||||
subscriptions.seek(tp, 0);
|
subscriptions.seek(tp, 0);
|
||||||
|
|
||||||
assertEquals(1, fetcher.sendFetches());
|
assertEquals(1, fetcher.sendFetches());
|
||||||
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0), true);
|
client.prepareResponse(fetchResponse(this.records, Errors.NONE.code(), 100L, 0), true);
|
||||||
consumerClient.poll(0);
|
consumerClient.poll(0);
|
||||||
assertEquals(0, fetcher.fetchedRecords().size());
|
assertEquals(0, fetcher.fetchedRecords().size());
|
||||||
|
|
||||||
|
|
@ -611,14 +616,14 @@ public class FetcherTest {
|
||||||
// We need to make sure the message offset grows. Otherwise they will be considered as already consumed
|
// We need to make sure the message offset grows. Otherwise they will be considered as already consumed
|
||||||
// and filtered out by consumer.
|
// and filtered out by consumer.
|
||||||
if (i > 1) {
|
if (i > 1) {
|
||||||
this.records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE);
|
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME);
|
||||||
for (int v = 0; v < 3; v++) {
|
for (int v = 0; v < 3; v++) {
|
||||||
this.records.append((long) i * 3 + v, Record.NO_TIMESTAMP, "key".getBytes(), String.format("value-%d", v).getBytes());
|
builder.append((long) i * 3 + v, Record.NO_TIMESTAMP, "key".getBytes(), String.format("value-%d", v).getBytes());
|
||||||
}
|
}
|
||||||
this.records.close();
|
this.records = builder.build();
|
||||||
}
|
}
|
||||||
assertEquals(1, fetcher.sendFetches());
|
assertEquals(1, fetcher.sendFetches());
|
||||||
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 100 * i));
|
client.prepareResponse(fetchResponse(this.records, Errors.NONE.code(), 100L, 100 * i));
|
||||||
consumerClient.poll(0);
|
consumerClient.poll(0);
|
||||||
records = fetcher.fetchedRecords().get(tp);
|
records = fetcher.fetchedRecords().get(tp);
|
||||||
assertEquals(3, records.size());
|
assertEquals(3, records.size());
|
||||||
|
|
@ -722,8 +727,7 @@ public class FetcherTest {
|
||||||
return new ListOffsetResponse(allPartitionData, 1);
|
return new ListOffsetResponse(allPartitionData, 1);
|
||||||
}
|
}
|
||||||
|
|
||||||
private FetchResponse fetchResponse(ByteBuffer buffer, short error, long hw, int throttleTime) {
|
private FetchResponse fetchResponse(MemoryRecords records, short error, long hw, int throttleTime) {
|
||||||
MemoryRecords records = MemoryRecords.readableRecords(buffer);
|
|
||||||
return new FetchResponse(Collections.singletonMap(tp, new FetchResponse.PartitionData(error, hw, records)), throttleTime);
|
return new FetchResponse(Collections.singletonMap(tp, new FetchResponse.PartitionData(error, hw, records)), throttleTime);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -12,23 +12,6 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.clients.producer.internals;
|
package org.apache.kafka.clients.producer.internals;
|
||||||
|
|
||||||
import static java.util.Arrays.asList;
|
|
||||||
import static org.junit.Assert.assertEquals;
|
|
||||||
import static org.junit.Assert.assertFalse;
|
|
||||||
import static org.junit.Assert.assertTrue;
|
|
||||||
import static org.junit.Assert.fail;
|
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.Deque;
|
|
||||||
import java.util.Iterator;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Set;
|
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
|
||||||
|
|
||||||
import org.apache.kafka.clients.producer.Callback;
|
import org.apache.kafka.clients.producer.Callback;
|
||||||
import org.apache.kafka.clients.producer.RecordMetadata;
|
import org.apache.kafka.clients.producer.RecordMetadata;
|
||||||
import org.apache.kafka.common.Cluster;
|
import org.apache.kafka.common.Cluster;
|
||||||
|
|
@ -45,6 +28,23 @@ import org.apache.kafka.common.utils.Time;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.Deque;
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
|
||||||
|
import static java.util.Arrays.asList;
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertFalse;
|
||||||
|
import static org.junit.Assert.assertTrue;
|
||||||
|
import static org.junit.Assert.fail;
|
||||||
|
|
||||||
public class RecordAccumulatorTest {
|
public class RecordAccumulatorTest {
|
||||||
|
|
||||||
private String topic = "test";
|
private String topic = "test";
|
||||||
|
|
@ -84,7 +84,7 @@ public class RecordAccumulatorTest {
|
||||||
accum.append(tp1, 0L, key, value, null, maxBlockTimeMs);
|
accum.append(tp1, 0L, key, value, null, maxBlockTimeMs);
|
||||||
Deque<RecordBatch> partitionBatches = accum.batches().get(tp1);
|
Deque<RecordBatch> partitionBatches = accum.batches().get(tp1);
|
||||||
assertEquals(1, partitionBatches.size());
|
assertEquals(1, partitionBatches.size());
|
||||||
assertTrue(partitionBatches.peekFirst().records.isWritable());
|
assertTrue(partitionBatches.peekFirst().isWritable());
|
||||||
assertEquals("No partitions should be ready.", 0, accum.ready(cluster, now).readyNodes.size());
|
assertEquals("No partitions should be ready.", 0, accum.ready(cluster, now).readyNodes.size());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -93,15 +93,15 @@ public class RecordAccumulatorTest {
|
||||||
Deque<RecordBatch> partitionBatches = accum.batches().get(tp1);
|
Deque<RecordBatch> partitionBatches = accum.batches().get(tp1);
|
||||||
assertEquals(2, partitionBatches.size());
|
assertEquals(2, partitionBatches.size());
|
||||||
Iterator<RecordBatch> partitionBatchesIterator = partitionBatches.iterator();
|
Iterator<RecordBatch> partitionBatchesIterator = partitionBatches.iterator();
|
||||||
assertFalse(partitionBatchesIterator.next().records.isWritable());
|
assertFalse(partitionBatchesIterator.next().isWritable());
|
||||||
assertTrue(partitionBatchesIterator.next().records.isWritable());
|
assertTrue(partitionBatchesIterator.next().isWritable());
|
||||||
assertEquals("Our partition's leader should be ready", Collections.singleton(node1), accum.ready(cluster, time.milliseconds()).readyNodes);
|
assertEquals("Our partition's leader should be ready", Collections.singleton(node1), accum.ready(cluster, time.milliseconds()).readyNodes);
|
||||||
|
|
||||||
List<RecordBatch> batches = accum.drain(cluster, Collections.singleton(node1), Integer.MAX_VALUE, 0).get(node1.id());
|
List<RecordBatch> batches = accum.drain(cluster, Collections.singleton(node1), Integer.MAX_VALUE, 0).get(node1.id());
|
||||||
assertEquals(1, batches.size());
|
assertEquals(1, batches.size());
|
||||||
RecordBatch batch = batches.get(0);
|
RecordBatch batch = batches.get(0);
|
||||||
|
|
||||||
Iterator<LogEntry> iter = batch.records.iterator();
|
Iterator<LogEntry> iter = batch.records().deepIterator();
|
||||||
for (int i = 0; i < appends; i++) {
|
for (int i = 0; i < appends; i++) {
|
||||||
LogEntry entry = iter.next();
|
LogEntry entry = iter.next();
|
||||||
assertEquals("Keys should match", ByteBuffer.wrap(key), entry.record().key());
|
assertEquals("Keys should match", ByteBuffer.wrap(key), entry.record().key());
|
||||||
|
|
@ -130,7 +130,7 @@ public class RecordAccumulatorTest {
|
||||||
assertEquals(1, batches.size());
|
assertEquals(1, batches.size());
|
||||||
RecordBatch batch = batches.get(0);
|
RecordBatch batch = batches.get(0);
|
||||||
|
|
||||||
Iterator<LogEntry> iter = batch.records.iterator();
|
Iterator<LogEntry> iter = batch.records().deepIterator();
|
||||||
LogEntry entry = iter.next();
|
LogEntry entry = iter.next();
|
||||||
assertEquals("Keys should match", ByteBuffer.wrap(key), entry.record().key());
|
assertEquals("Keys should match", ByteBuffer.wrap(key), entry.record().key());
|
||||||
assertEquals("Values should match", ByteBuffer.wrap(value), entry.record().value());
|
assertEquals("Values should match", ByteBuffer.wrap(value), entry.record().value());
|
||||||
|
|
@ -159,7 +159,7 @@ public class RecordAccumulatorTest {
|
||||||
final int msgs = 10000;
|
final int msgs = 10000;
|
||||||
final int numParts = 2;
|
final int numParts = 2;
|
||||||
final RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, 0L, 100L, metrics, time);
|
final RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, 0L, 100L, metrics, time);
|
||||||
List<Thread> threads = new ArrayList<Thread>();
|
List<Thread> threads = new ArrayList<>();
|
||||||
for (int i = 0; i < numThreads; i++) {
|
for (int i = 0; i < numThreads; i++) {
|
||||||
threads.add(new Thread() {
|
threads.add(new Thread() {
|
||||||
public void run() {
|
public void run() {
|
||||||
|
|
@ -182,8 +182,11 @@ public class RecordAccumulatorTest {
|
||||||
List<RecordBatch> batches = accum.drain(cluster, nodes, 5 * 1024, 0).get(node1.id());
|
List<RecordBatch> batches = accum.drain(cluster, nodes, 5 * 1024, 0).get(node1.id());
|
||||||
if (batches != null) {
|
if (batches != null) {
|
||||||
for (RecordBatch batch : batches) {
|
for (RecordBatch batch : batches) {
|
||||||
for (LogEntry entry : batch.records)
|
Iterator<LogEntry> deepEntries = batch.records().deepIterator();
|
||||||
|
while (deepEntries.hasNext()) {
|
||||||
|
deepEntries.next();
|
||||||
read++;
|
read++;
|
||||||
|
}
|
||||||
accum.deallocate(batch);
|
accum.deallocate(batch);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,110 @@
|
||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p/>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p/>
|
||||||
|
* 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.junit.Test;
|
||||||
|
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.util.Iterator;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertFalse;
|
||||||
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
|
public class ByteBufferLogInputStreamTest {
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void iteratorIgnoresIncompleteEntries() {
|
||||||
|
ByteBuffer buffer = ByteBuffer.allocate(2048);
|
||||||
|
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, Record.MAGIC_VALUE_V1, CompressionType.NONE, TimestampType.CREATE_TIME, 0L);
|
||||||
|
builder.append(0L, 15L, "a".getBytes(), "1".getBytes());
|
||||||
|
builder.append(1L, 20L, "b".getBytes(), "2".getBytes());
|
||||||
|
|
||||||
|
ByteBuffer recordsBuffer = builder.build().buffer();
|
||||||
|
recordsBuffer.limit(recordsBuffer.limit() - 5);
|
||||||
|
|
||||||
|
Iterator<ByteBufferLogInputStream.ByteBufferLogEntry> iterator = MemoryRecords.readableRecords(recordsBuffer).shallowIterator();
|
||||||
|
assertTrue(iterator.hasNext());
|
||||||
|
ByteBufferLogInputStream.ByteBufferLogEntry first = iterator.next();
|
||||||
|
assertEquals(0L, first.offset());
|
||||||
|
|
||||||
|
assertFalse(iterator.hasNext());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSetCreateTimeV1() {
|
||||||
|
ByteBuffer buffer = ByteBuffer.allocate(2048);
|
||||||
|
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, Record.MAGIC_VALUE_V1, CompressionType.NONE, TimestampType.CREATE_TIME, 0L);
|
||||||
|
builder.append(0L, 15L, "a".getBytes(), "1".getBytes());
|
||||||
|
Iterator<ByteBufferLogInputStream.ByteBufferLogEntry> iterator = builder.build().shallowIterator();
|
||||||
|
|
||||||
|
assertTrue(iterator.hasNext());
|
||||||
|
ByteBufferLogInputStream.ByteBufferLogEntry entry = iterator.next();
|
||||||
|
|
||||||
|
long createTimeMs = 20L;
|
||||||
|
entry.setCreateTime(createTimeMs);
|
||||||
|
|
||||||
|
assertEquals(TimestampType.CREATE_TIME, entry.record().timestampType());
|
||||||
|
assertEquals(createTimeMs, entry.record().timestamp());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(expected = IllegalArgumentException.class)
|
||||||
|
public void testSetCreateTimeNotAllowedV0() {
|
||||||
|
ByteBuffer buffer = ByteBuffer.allocate(2048);
|
||||||
|
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, Record.MAGIC_VALUE_V0, CompressionType.NONE, TimestampType.CREATE_TIME, 0L);
|
||||||
|
builder.append(0L, 15L, "a".getBytes(), "1".getBytes());
|
||||||
|
Iterator<ByteBufferLogInputStream.ByteBufferLogEntry> iterator = builder.build().shallowIterator();
|
||||||
|
|
||||||
|
assertTrue(iterator.hasNext());
|
||||||
|
ByteBufferLogInputStream.ByteBufferLogEntry entry = iterator.next();
|
||||||
|
|
||||||
|
long createTimeMs = 20L;
|
||||||
|
entry.setCreateTime(createTimeMs);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSetLogAppendTimeV1() {
|
||||||
|
ByteBuffer buffer = ByteBuffer.allocate(2048);
|
||||||
|
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, Record.MAGIC_VALUE_V1, CompressionType.NONE, TimestampType.CREATE_TIME, 0L);
|
||||||
|
builder.append(0L, 15L, "a".getBytes(), "1".getBytes());
|
||||||
|
Iterator<ByteBufferLogInputStream.ByteBufferLogEntry> iterator = builder.build().shallowIterator();
|
||||||
|
|
||||||
|
assertTrue(iterator.hasNext());
|
||||||
|
ByteBufferLogInputStream.ByteBufferLogEntry entry = iterator.next();
|
||||||
|
|
||||||
|
long logAppendTime = 20L;
|
||||||
|
entry.setLogAppendTime(logAppendTime);
|
||||||
|
|
||||||
|
assertEquals(TimestampType.LOG_APPEND_TIME, entry.record().timestampType());
|
||||||
|
assertEquals(logAppendTime, entry.record().timestamp());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(expected = IllegalArgumentException.class)
|
||||||
|
public void testSetLogAppendTimeNotAllowedV0() {
|
||||||
|
ByteBuffer buffer = ByteBuffer.allocate(2048);
|
||||||
|
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, Record.MAGIC_VALUE_V0, CompressionType.NONE, TimestampType.CREATE_TIME, 0L);
|
||||||
|
builder.append(0L, 15L, "a".getBytes(), "1".getBytes());
|
||||||
|
Iterator<ByteBufferLogInputStream.ByteBufferLogEntry> iterator = builder.build().shallowIterator();
|
||||||
|
|
||||||
|
assertTrue(iterator.hasNext());
|
||||||
|
ByteBufferLogInputStream.ByteBufferLogEntry entry = iterator.next();
|
||||||
|
|
||||||
|
long logAppendTime = 20L;
|
||||||
|
entry.setLogAppendTime(logAppendTime);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -0,0 +1,410 @@
|
||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p/>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p/>
|
||||||
|
* 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.KafkaException;
|
||||||
|
import org.apache.kafka.test.TestUtils;
|
||||||
|
import org.easymock.EasyMock;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.io.File;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.nio.channels.FileChannel;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
import static org.apache.kafka.test.TestUtils.tempFile;
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertTrue;
|
||||||
|
import static org.junit.Assert.fail;
|
||||||
|
|
||||||
|
public class FileRecordsTest {
|
||||||
|
|
||||||
|
private Record[] records = new Record[] {
|
||||||
|
Record.create("abcd".getBytes()),
|
||||||
|
Record.create("efgh".getBytes()),
|
||||||
|
Record.create("ijkl".getBytes())
|
||||||
|
};
|
||||||
|
private FileRecords fileRecords;
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setup() throws IOException {
|
||||||
|
this.fileRecords = createFileRecords(records);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test that the cached size variable matches the actual file size as we append messages
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testFileSize() throws IOException {
|
||||||
|
assertEquals(fileRecords.channel().size(), fileRecords.sizeInBytes());
|
||||||
|
for (int i = 0; i < 20; i++) {
|
||||||
|
fileRecords.append(MemoryRecords.withRecords(Record.create("abcd".getBytes())));
|
||||||
|
assertEquals(fileRecords.channel().size(), fileRecords.sizeInBytes());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test that adding invalid bytes to the end of the log doesn't break iteration
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testIterationOverPartialAndTruncation() throws IOException {
|
||||||
|
testPartialWrite(0, fileRecords);
|
||||||
|
testPartialWrite(2, fileRecords);
|
||||||
|
testPartialWrite(4, fileRecords);
|
||||||
|
testPartialWrite(5, fileRecords);
|
||||||
|
testPartialWrite(6, fileRecords);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void testPartialWrite(int size, FileRecords fileRecords) throws IOException {
|
||||||
|
ByteBuffer buffer = ByteBuffer.allocate(size);
|
||||||
|
for (int i = 0; i < size; i++)
|
||||||
|
buffer.put((byte) 0);
|
||||||
|
|
||||||
|
buffer.rewind();
|
||||||
|
|
||||||
|
fileRecords.channel().write(buffer);
|
||||||
|
|
||||||
|
// appending those bytes should not change the contents
|
||||||
|
TestUtils.checkEquals(Arrays.asList(records).iterator(), fileRecords.records());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Iterating over the file does file reads but shouldn't change the position of the underlying FileChannel.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testIterationDoesntChangePosition() throws IOException {
|
||||||
|
long position = fileRecords.channel().position();
|
||||||
|
TestUtils.checkEquals(Arrays.asList(records).iterator(), fileRecords.records());
|
||||||
|
assertEquals(position, fileRecords.channel().position());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test a simple append and read.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testRead() throws IOException {
|
||||||
|
FileRecords read = fileRecords.read(0, fileRecords.sizeInBytes());
|
||||||
|
TestUtils.checkEquals(fileRecords.shallowIterator(), read.shallowIterator());
|
||||||
|
|
||||||
|
List<LogEntry> items = shallowEntries(read);
|
||||||
|
LogEntry second = items.get(1);
|
||||||
|
|
||||||
|
read = fileRecords.read(second.sizeInBytes(), fileRecords.sizeInBytes());
|
||||||
|
assertEquals("Try a read starting from the second message",
|
||||||
|
items.subList(1, 3), shallowEntries(read));
|
||||||
|
|
||||||
|
read = fileRecords.read(second.sizeInBytes(), second.sizeInBytes());
|
||||||
|
assertEquals("Try a read of a single message starting from the second message",
|
||||||
|
Collections.singletonList(second), shallowEntries(read));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test the MessageSet.searchFor API.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testSearch() throws IOException {
|
||||||
|
// append a new message with a high offset
|
||||||
|
Record lastMessage = Record.create("test".getBytes());
|
||||||
|
fileRecords.append(MemoryRecords.withRecords(50L, lastMessage));
|
||||||
|
|
||||||
|
List<LogEntry> entries = shallowEntries(fileRecords);
|
||||||
|
int position = 0;
|
||||||
|
|
||||||
|
int message1Size = entries.get(0).sizeInBytes();
|
||||||
|
assertEquals("Should be able to find the first message by its offset",
|
||||||
|
new FileRecords.LogEntryPosition(0L, position, message1Size),
|
||||||
|
fileRecords.searchForOffsetWithSize(0, 0));
|
||||||
|
position += message1Size;
|
||||||
|
|
||||||
|
int message2Size = entries.get(1).sizeInBytes();
|
||||||
|
assertEquals("Should be able to find second message when starting from 0",
|
||||||
|
new FileRecords.LogEntryPosition(1L, position, message2Size),
|
||||||
|
fileRecords.searchForOffsetWithSize(1, 0));
|
||||||
|
assertEquals("Should be able to find second message starting from its offset",
|
||||||
|
new FileRecords.LogEntryPosition(1L, position, message2Size),
|
||||||
|
fileRecords.searchForOffsetWithSize(1, position));
|
||||||
|
position += message2Size + entries.get(2).sizeInBytes();
|
||||||
|
|
||||||
|
int message4Size = entries.get(3).sizeInBytes();
|
||||||
|
assertEquals("Should be able to find fourth message from a non-existant offset",
|
||||||
|
new FileRecords.LogEntryPosition(50L, position, message4Size),
|
||||||
|
fileRecords.searchForOffsetWithSize(3, position));
|
||||||
|
assertEquals("Should be able to find fourth message by correct offset",
|
||||||
|
new FileRecords.LogEntryPosition(50L, position, message4Size),
|
||||||
|
fileRecords.searchForOffsetWithSize(50, position));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test that the message set iterator obeys start and end slicing
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testIteratorWithLimits() throws IOException {
|
||||||
|
LogEntry entry = shallowEntries(fileRecords).get(1);
|
||||||
|
int start = fileRecords.searchForOffsetWithSize(1, 0).position;
|
||||||
|
int size = entry.sizeInBytes();
|
||||||
|
FileRecords slice = fileRecords.read(start, size);
|
||||||
|
assertEquals(Collections.singletonList(entry), shallowEntries(slice));
|
||||||
|
FileRecords slice2 = fileRecords.read(start, size - 1);
|
||||||
|
assertEquals(Collections.emptyList(), shallowEntries(slice2));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test the truncateTo method lops off messages and appropriately updates the size
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testTruncate() throws IOException {
|
||||||
|
LogEntry entry = shallowEntries(fileRecords).get(0);
|
||||||
|
int end = fileRecords.searchForOffsetWithSize(1, 0).position;
|
||||||
|
fileRecords.truncateTo(end);
|
||||||
|
assertEquals(Collections.singletonList(entry), shallowEntries(fileRecords));
|
||||||
|
assertEquals(entry.sizeInBytes(), fileRecords.sizeInBytes());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test that truncateTo only calls truncate on the FileChannel if the size of the
|
||||||
|
* FileChannel is bigger than the target size. This is important because some JVMs
|
||||||
|
* change the mtime of the file, even if truncate should do nothing.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testTruncateNotCalledIfSizeIsSameAsTargetSize() throws IOException {
|
||||||
|
FileChannel channelMock = EasyMock.createMock(FileChannel.class);
|
||||||
|
|
||||||
|
EasyMock.expect(channelMock.size()).andReturn(42L).atLeastOnce();
|
||||||
|
EasyMock.expect(channelMock.position(42L)).andReturn(null);
|
||||||
|
EasyMock.replay(channelMock);
|
||||||
|
|
||||||
|
FileRecords fileRecords = new FileRecords(tempFile(), channelMock, 0, Integer.MAX_VALUE, false);
|
||||||
|
fileRecords.truncateTo(42);
|
||||||
|
|
||||||
|
EasyMock.verify(channelMock);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Expect a KafkaException if targetSize is bigger than the size of
|
||||||
|
* the FileRecords.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testTruncateNotCalledIfSizeIsBiggerThanTargetSize() throws IOException {
|
||||||
|
FileChannel channelMock = EasyMock.createMock(FileChannel.class);
|
||||||
|
|
||||||
|
EasyMock.expect(channelMock.size()).andReturn(42L).atLeastOnce();
|
||||||
|
EasyMock.expect(channelMock.position(42L)).andReturn(null);
|
||||||
|
EasyMock.replay(channelMock);
|
||||||
|
|
||||||
|
FileRecords fileRecords = new FileRecords(tempFile(), channelMock, 0, Integer.MAX_VALUE, false);
|
||||||
|
|
||||||
|
try {
|
||||||
|
fileRecords.truncateTo(43);
|
||||||
|
fail("Should throw KafkaException");
|
||||||
|
} catch (KafkaException e) {
|
||||||
|
// expected
|
||||||
|
}
|
||||||
|
|
||||||
|
EasyMock.verify(channelMock);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* see #testTruncateNotCalledIfSizeIsSameAsTargetSize
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testTruncateIfSizeIsDifferentToTargetSize() throws IOException {
|
||||||
|
FileChannel channelMock = EasyMock.createMock(FileChannel.class);
|
||||||
|
|
||||||
|
EasyMock.expect(channelMock.size()).andReturn(42L).atLeastOnce();
|
||||||
|
EasyMock.expect(channelMock.position(42L)).andReturn(null).once();
|
||||||
|
EasyMock.expect(channelMock.truncate(23L)).andReturn(null).once();
|
||||||
|
EasyMock.expect(channelMock.position(23L)).andReturn(null).once();
|
||||||
|
EasyMock.replay(channelMock);
|
||||||
|
|
||||||
|
FileRecords fileRecords = new FileRecords(tempFile(), channelMock, 0, Integer.MAX_VALUE, false);
|
||||||
|
fileRecords.truncateTo(23);
|
||||||
|
|
||||||
|
EasyMock.verify(channelMock);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test the new FileRecords with pre allocate as true
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testPreallocateTrue() throws IOException {
|
||||||
|
File temp = tempFile();
|
||||||
|
FileRecords fileRecords = FileRecords.open(temp, false, 512 * 1024 * 1024, true);
|
||||||
|
long position = fileRecords.channel().position();
|
||||||
|
int size = fileRecords.sizeInBytes();
|
||||||
|
assertEquals(0, position);
|
||||||
|
assertEquals(0, size);
|
||||||
|
assertEquals(512 * 1024 * 1024, temp.length());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test the new FileRecords with pre allocate as false
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testPreallocateFalse() throws IOException {
|
||||||
|
File temp = tempFile();
|
||||||
|
FileRecords set = FileRecords.open(temp, false, 512 * 1024 * 1024, false);
|
||||||
|
long position = set.channel().position();
|
||||||
|
int size = set.sizeInBytes();
|
||||||
|
assertEquals(0, position);
|
||||||
|
assertEquals(0, size);
|
||||||
|
assertEquals(0, temp.length());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test the new FileRecords with pre allocate as true and file has been clearly shut down, the file will be truncate to end of valid data.
|
||||||
|
*/
|
||||||
|
@Test
|
||||||
|
public void testPreallocateClearShutdown() throws IOException {
|
||||||
|
File temp = tempFile();
|
||||||
|
FileRecords set = FileRecords.open(temp, false, 512 * 1024 * 1024, true);
|
||||||
|
set.append(MemoryRecords.withRecords(records));
|
||||||
|
|
||||||
|
int oldPosition = (int) set.channel().position();
|
||||||
|
int oldSize = set.sizeInBytes();
|
||||||
|
assertEquals(fileRecords.sizeInBytes(), oldPosition);
|
||||||
|
assertEquals(fileRecords.sizeInBytes(), oldSize);
|
||||||
|
set.close();
|
||||||
|
|
||||||
|
File tempReopen = new File(temp.getAbsolutePath());
|
||||||
|
FileRecords setReopen = FileRecords.open(tempReopen, true, 512 * 1024 * 1024, true);
|
||||||
|
int position = (int) setReopen.channel().position();
|
||||||
|
int size = setReopen.sizeInBytes();
|
||||||
|
|
||||||
|
assertEquals(oldPosition, position);
|
||||||
|
assertEquals(oldPosition, size);
|
||||||
|
assertEquals(oldPosition, tempReopen.length());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFormatConversionWithPartialMessage() throws IOException {
|
||||||
|
LogEntry entry = shallowEntries(fileRecords).get(1);
|
||||||
|
int start = fileRecords.searchForOffsetWithSize(1, 0).position;
|
||||||
|
int size = entry.sizeInBytes();
|
||||||
|
FileRecords slice = fileRecords.read(start, size - 1);
|
||||||
|
Records messageV0 = slice.toMessageFormat(Record.MAGIC_VALUE_V0);
|
||||||
|
assertTrue("No message should be there", shallowEntries(messageV0).isEmpty());
|
||||||
|
assertEquals("There should be " + (size - 1) + " bytes", size - 1, messageV0.sizeInBytes());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testConvertNonCompressedToMagic1() throws IOException {
|
||||||
|
List<LogEntry> entries = Arrays.asList(
|
||||||
|
LogEntry.create(0L, Record.create(Record.MAGIC_VALUE_V0, Record.NO_TIMESTAMP, "k1".getBytes(), "hello".getBytes())),
|
||||||
|
LogEntry.create(2L, Record.create(Record.MAGIC_VALUE_V0, Record.NO_TIMESTAMP, "k2".getBytes(), "goodbye".getBytes())));
|
||||||
|
MemoryRecords records = MemoryRecords.withLogEntries(CompressionType.NONE, entries);
|
||||||
|
|
||||||
|
// Up conversion. In reality we only do down conversion, but up conversion should work as well.
|
||||||
|
// up conversion for non-compressed messages
|
||||||
|
try (FileRecords fileRecords = FileRecords.open(tempFile())) {
|
||||||
|
fileRecords.append(records);
|
||||||
|
fileRecords.flush();
|
||||||
|
Records convertedRecords = fileRecords.toMessageFormat(Record.MAGIC_VALUE_V1);
|
||||||
|
verifyConvertedMessageSet(entries, convertedRecords, Record.MAGIC_VALUE_V1);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testConvertCompressedToMagic1() throws IOException {
|
||||||
|
List<LogEntry> entries = Arrays.asList(
|
||||||
|
LogEntry.create(0L, Record.create(Record.MAGIC_VALUE_V0, Record.NO_TIMESTAMP, "k1".getBytes(), "hello".getBytes())),
|
||||||
|
LogEntry.create(2L, Record.create(Record.MAGIC_VALUE_V0, Record.NO_TIMESTAMP, "k2".getBytes(), "goodbye".getBytes())));
|
||||||
|
MemoryRecords records = MemoryRecords.withLogEntries(CompressionType.GZIP, entries);
|
||||||
|
|
||||||
|
// up conversion for compressed messages
|
||||||
|
try (FileRecords fileRecords = FileRecords.open(tempFile())) {
|
||||||
|
fileRecords.append(records);
|
||||||
|
fileRecords.flush();
|
||||||
|
Records convertedRecords = fileRecords.toMessageFormat(Record.MAGIC_VALUE_V1);
|
||||||
|
verifyConvertedMessageSet(entries, convertedRecords, Record.MAGIC_VALUE_V1);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testConvertNonCompressedToMagic0() throws IOException {
|
||||||
|
List<LogEntry> entries = Arrays.asList(
|
||||||
|
LogEntry.create(0L, Record.create(Record.MAGIC_VALUE_V1, 1L, "k1".getBytes(), "hello".getBytes())),
|
||||||
|
LogEntry.create(2L, Record.create(Record.MAGIC_VALUE_V1, 2L, "k2".getBytes(), "goodbye".getBytes())));
|
||||||
|
MemoryRecords records = MemoryRecords.withLogEntries(CompressionType.NONE, entries);
|
||||||
|
|
||||||
|
// down conversion for non-compressed messages
|
||||||
|
try (FileRecords fileRecords = FileRecords.open(tempFile())) {
|
||||||
|
fileRecords.append(records);
|
||||||
|
fileRecords.flush();
|
||||||
|
Records convertedRecords = fileRecords.toMessageFormat(Record.MAGIC_VALUE_V0);
|
||||||
|
verifyConvertedMessageSet(entries, convertedRecords, Record.MAGIC_VALUE_V0);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testConvertCompressedToMagic0() throws IOException {
|
||||||
|
List<LogEntry> entries = Arrays.asList(
|
||||||
|
LogEntry.create(0L, Record.create(Record.MAGIC_VALUE_V1, 1L, "k1".getBytes(), "hello".getBytes())),
|
||||||
|
LogEntry.create(2L, Record.create(Record.MAGIC_VALUE_V1, 2L, "k2".getBytes(), "goodbye".getBytes())));
|
||||||
|
MemoryRecords records = MemoryRecords.withLogEntries(CompressionType.GZIP, entries);
|
||||||
|
|
||||||
|
// down conversion for compressed messages
|
||||||
|
try (FileRecords fileRecords = FileRecords.open(tempFile())) {
|
||||||
|
fileRecords.append(records);
|
||||||
|
fileRecords.flush();
|
||||||
|
Records convertedRecords = fileRecords.toMessageFormat(Record.MAGIC_VALUE_V0);
|
||||||
|
verifyConvertedMessageSet(entries, convertedRecords, Record.MAGIC_VALUE_V0);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void verifyConvertedMessageSet(List<LogEntry> initialEntries, Records convertedRecords, byte magicByte) {
|
||||||
|
int i = 0;
|
||||||
|
for (LogEntry logEntry : deepEntries(convertedRecords)) {
|
||||||
|
assertEquals("magic byte should be " + magicByte, magicByte, logEntry.record().magic());
|
||||||
|
assertEquals("offset should not change", initialEntries.get(i).offset(), logEntry.offset());
|
||||||
|
assertEquals("key should not change", initialEntries.get(i).record().key(), logEntry.record().key());
|
||||||
|
assertEquals("payload should not change", initialEntries.get(i).record().value(), logEntry.record().value());
|
||||||
|
i += 1;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static List<LogEntry> shallowEntries(Records buffer) {
|
||||||
|
List<LogEntry> entries = new ArrayList<>();
|
||||||
|
Iterator<? extends LogEntry> iterator = buffer.shallowIterator();
|
||||||
|
while (iterator.hasNext())
|
||||||
|
entries.add(iterator.next());
|
||||||
|
return entries;
|
||||||
|
}
|
||||||
|
|
||||||
|
private static List<LogEntry> deepEntries(Records buffer) {
|
||||||
|
List<LogEntry> entries = new ArrayList<>();
|
||||||
|
Iterator<? extends LogEntry> iterator = buffer.shallowIterator();
|
||||||
|
while (iterator.hasNext()) {
|
||||||
|
for (LogEntry deepEntry : iterator.next())
|
||||||
|
entries.add(deepEntry);
|
||||||
|
}
|
||||||
|
return entries;
|
||||||
|
}
|
||||||
|
|
||||||
|
private FileRecords createFileRecords(Record ... records) throws IOException {
|
||||||
|
FileRecords fileRecords = FileRecords.open(tempFile());
|
||||||
|
fileRecords.append(MemoryRecords.withRecords(records));
|
||||||
|
fileRecords.flush();
|
||||||
|
return fileRecords;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -0,0 +1,253 @@
|
||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p/>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p/>
|
||||||
|
* 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.junit.Test;
|
||||||
|
import org.junit.runner.RunWith;
|
||||||
|
import org.junit.runners.Parameterized;
|
||||||
|
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertFalse;
|
||||||
|
|
||||||
|
@RunWith(value = Parameterized.class)
|
||||||
|
public class MemoryRecordsBuilderTest {
|
||||||
|
|
||||||
|
private final CompressionType compressionType;
|
||||||
|
private final int bufferOffset;
|
||||||
|
|
||||||
|
public MemoryRecordsBuilderTest(int bufferOffset, CompressionType compressionType) {
|
||||||
|
this.bufferOffset = bufferOffset;
|
||||||
|
this.compressionType = compressionType;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCompressionRateV0() {
|
||||||
|
ByteBuffer buffer = ByteBuffer.allocate(1024);
|
||||||
|
buffer.position(bufferOffset);
|
||||||
|
|
||||||
|
Record[] records = new Record[] {
|
||||||
|
Record.create(Record.MAGIC_VALUE_V0, 0L, "a".getBytes(), "1".getBytes()),
|
||||||
|
Record.create(Record.MAGIC_VALUE_V0, 1L, "b".getBytes(), "2".getBytes()),
|
||||||
|
Record.create(Record.MAGIC_VALUE_V0, 2L, "c".getBytes(), "3".getBytes()),
|
||||||
|
};
|
||||||
|
|
||||||
|
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, Record.MAGIC_VALUE_V0, compressionType,
|
||||||
|
TimestampType.CREATE_TIME, 0L, 0L, buffer.capacity());
|
||||||
|
|
||||||
|
int uncompressedSize = 0;
|
||||||
|
long offset = 0L;
|
||||||
|
for (Record record : records) {
|
||||||
|
uncompressedSize += record.sizeInBytes() + Records.LOG_OVERHEAD;
|
||||||
|
builder.append(offset++, record);
|
||||||
|
}
|
||||||
|
|
||||||
|
MemoryRecords built = builder.build();
|
||||||
|
if (compressionType == CompressionType.NONE) {
|
||||||
|
assertEquals(1.0, builder.compressionRate(), 0.00001);
|
||||||
|
} else {
|
||||||
|
int compressedSize = built.sizeInBytes() - Records.LOG_OVERHEAD - Record.RECORD_OVERHEAD_V0;
|
||||||
|
double computedCompressionRate = (double) compressedSize / uncompressedSize;
|
||||||
|
assertEquals(computedCompressionRate, builder.compressionRate(), 0.00001);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testCompressionRateV1() {
|
||||||
|
ByteBuffer buffer = ByteBuffer.allocate(1024);
|
||||||
|
buffer.position(bufferOffset);
|
||||||
|
|
||||||
|
Record[] records = new Record[] {
|
||||||
|
Record.create(Record.MAGIC_VALUE_V1, 0L, "a".getBytes(), "1".getBytes()),
|
||||||
|
Record.create(Record.MAGIC_VALUE_V1, 1L, "b".getBytes(), "2".getBytes()),
|
||||||
|
Record.create(Record.MAGIC_VALUE_V1, 2L, "c".getBytes(), "3".getBytes()),
|
||||||
|
};
|
||||||
|
|
||||||
|
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, Record.MAGIC_VALUE_V1, compressionType,
|
||||||
|
TimestampType.CREATE_TIME, 0L, 0L, buffer.capacity());
|
||||||
|
|
||||||
|
int uncompressedSize = 0;
|
||||||
|
long offset = 0L;
|
||||||
|
for (Record record : records) {
|
||||||
|
uncompressedSize += record.sizeInBytes() + Records.LOG_OVERHEAD;
|
||||||
|
builder.append(offset++, record);
|
||||||
|
}
|
||||||
|
|
||||||
|
MemoryRecords built = builder.build();
|
||||||
|
if (compressionType == CompressionType.NONE) {
|
||||||
|
assertEquals(1.0, builder.compressionRate(), 0.00001);
|
||||||
|
} else {
|
||||||
|
int compressedSize = built.sizeInBytes() - Records.LOG_OVERHEAD - Record.RECORD_OVERHEAD_V1;
|
||||||
|
double computedCompressionRate = (double) compressedSize / uncompressedSize;
|
||||||
|
assertEquals(computedCompressionRate, builder.compressionRate(), 0.00001);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void buildUsingLogAppendTime() {
|
||||||
|
ByteBuffer buffer = ByteBuffer.allocate(1024);
|
||||||
|
buffer.position(bufferOffset);
|
||||||
|
|
||||||
|
long logAppendTime = System.currentTimeMillis();
|
||||||
|
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, Record.MAGIC_VALUE_V1, compressionType,
|
||||||
|
TimestampType.LOG_APPEND_TIME, 0L, logAppendTime, buffer.capacity());
|
||||||
|
builder.append(0L, 0L, "a".getBytes(), "1".getBytes());
|
||||||
|
builder.append(1L, 0L, "b".getBytes(), "2".getBytes());
|
||||||
|
builder.append(2L, 0L, "c".getBytes(), "3".getBytes());
|
||||||
|
MemoryRecords records = builder.build();
|
||||||
|
|
||||||
|
MemoryRecordsBuilder.RecordsInfo info = builder.info();
|
||||||
|
assertEquals(logAppendTime, info.maxTimestamp);
|
||||||
|
|
||||||
|
assertEquals(2L, info.shallowOffsetOfMaxTimestamp);
|
||||||
|
|
||||||
|
Iterator<Record> iterator = records.records();
|
||||||
|
while (iterator.hasNext()) {
|
||||||
|
Record record = iterator.next();
|
||||||
|
assertEquals(TimestampType.LOG_APPEND_TIME, record.timestampType());
|
||||||
|
assertEquals(logAppendTime, record.timestamp());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void convertUsingLogAppendTime() {
|
||||||
|
ByteBuffer buffer = ByteBuffer.allocate(1024);
|
||||||
|
buffer.position(bufferOffset);
|
||||||
|
|
||||||
|
long logAppendTime = System.currentTimeMillis();
|
||||||
|
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, Record.MAGIC_VALUE_V1, compressionType,
|
||||||
|
TimestampType.LOG_APPEND_TIME, 0L, logAppendTime, buffer.capacity());
|
||||||
|
|
||||||
|
builder.convertAndAppend(0L, Record.create(Record.MAGIC_VALUE_V0, 0L, "a".getBytes(), "1".getBytes()));
|
||||||
|
builder.convertAndAppend(1L, Record.create(Record.MAGIC_VALUE_V0, 0L, "b".getBytes(), "2".getBytes()));
|
||||||
|
builder.convertAndAppend(2L, Record.create(Record.MAGIC_VALUE_V0, 0L, "c".getBytes(), "3".getBytes()));
|
||||||
|
MemoryRecords records = builder.build();
|
||||||
|
|
||||||
|
MemoryRecordsBuilder.RecordsInfo info = builder.info();
|
||||||
|
assertEquals(logAppendTime, info.maxTimestamp);
|
||||||
|
|
||||||
|
assertEquals(2L, info.shallowOffsetOfMaxTimestamp);
|
||||||
|
|
||||||
|
Iterator<Record> iterator = records.records();
|
||||||
|
while (iterator.hasNext()) {
|
||||||
|
Record record = iterator.next();
|
||||||
|
assertEquals(TimestampType.LOG_APPEND_TIME, record.timestampType());
|
||||||
|
assertEquals(logAppendTime, record.timestamp());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void buildUsingCreateTime() {
|
||||||
|
ByteBuffer buffer = ByteBuffer.allocate(1024);
|
||||||
|
buffer.position(bufferOffset);
|
||||||
|
|
||||||
|
long logAppendTime = System.currentTimeMillis();
|
||||||
|
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, Record.MAGIC_VALUE_V1, compressionType,
|
||||||
|
TimestampType.CREATE_TIME, 0L, logAppendTime, buffer.capacity());
|
||||||
|
builder.append(0L, 0L, "a".getBytes(), "1".getBytes());
|
||||||
|
builder.append(1L, 2L, "b".getBytes(), "2".getBytes());
|
||||||
|
builder.append(2L, 1L, "c".getBytes(), "3".getBytes());
|
||||||
|
MemoryRecords records = builder.build();
|
||||||
|
|
||||||
|
MemoryRecordsBuilder.RecordsInfo info = builder.info();
|
||||||
|
assertEquals(2L, info.maxTimestamp);
|
||||||
|
|
||||||
|
if (compressionType == CompressionType.NONE)
|
||||||
|
assertEquals(1L, info.shallowOffsetOfMaxTimestamp);
|
||||||
|
else
|
||||||
|
assertEquals(2L, info.shallowOffsetOfMaxTimestamp);
|
||||||
|
|
||||||
|
Iterator<Record> iterator = records.records();
|
||||||
|
int i = 0;
|
||||||
|
long[] expectedTimestamps = new long[] {0L, 2L, 1L};
|
||||||
|
while (iterator.hasNext()) {
|
||||||
|
Record record = iterator.next();
|
||||||
|
assertEquals(TimestampType.CREATE_TIME, record.timestampType());
|
||||||
|
assertEquals(expectedTimestamps[i++], record.timestamp());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void writePastLimit() {
|
||||||
|
ByteBuffer buffer = ByteBuffer.allocate(64);
|
||||||
|
buffer.position(bufferOffset);
|
||||||
|
|
||||||
|
long logAppendTime = System.currentTimeMillis();
|
||||||
|
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, Record.MAGIC_VALUE_V1, compressionType,
|
||||||
|
TimestampType.CREATE_TIME, 0L, logAppendTime, buffer.capacity());
|
||||||
|
builder.append(0L, 0L, "a".getBytes(), "1".getBytes());
|
||||||
|
builder.append(1L, 1L, "b".getBytes(), "2".getBytes());
|
||||||
|
|
||||||
|
assertFalse(builder.hasRoomFor("c".getBytes(), "3".getBytes()));
|
||||||
|
builder.append(2L, 2L, "c".getBytes(), "3".getBytes());
|
||||||
|
MemoryRecords records = builder.build();
|
||||||
|
|
||||||
|
MemoryRecordsBuilder.RecordsInfo info = builder.info();
|
||||||
|
assertEquals(2L, info.maxTimestamp);
|
||||||
|
assertEquals(2L, info.shallowOffsetOfMaxTimestamp);
|
||||||
|
|
||||||
|
Iterator<Record> iterator = records.records();
|
||||||
|
long i = 0L;
|
||||||
|
while (iterator.hasNext()) {
|
||||||
|
Record record = iterator.next();
|
||||||
|
assertEquals(TimestampType.CREATE_TIME, record.timestampType());
|
||||||
|
assertEquals(i++, record.timestamp());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void convertUsingCreateTime() {
|
||||||
|
ByteBuffer buffer = ByteBuffer.allocate(1024);
|
||||||
|
buffer.position(bufferOffset);
|
||||||
|
|
||||||
|
long logAppendTime = System.currentTimeMillis();
|
||||||
|
MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, Record.MAGIC_VALUE_V1, compressionType,
|
||||||
|
TimestampType.CREATE_TIME, 0L, logAppendTime, buffer.capacity());
|
||||||
|
|
||||||
|
builder.convertAndAppend(0L, Record.create(Record.MAGIC_VALUE_V0, 0L, "a".getBytes(), "1".getBytes()));
|
||||||
|
builder.convertAndAppend(0L, Record.create(Record.MAGIC_VALUE_V0, 0L, "b".getBytes(), "2".getBytes()));
|
||||||
|
builder.convertAndAppend(0L, Record.create(Record.MAGIC_VALUE_V0, 0L, "c".getBytes(), "3".getBytes()));
|
||||||
|
MemoryRecords records = builder.build();
|
||||||
|
|
||||||
|
MemoryRecordsBuilder.RecordsInfo info = builder.info();
|
||||||
|
assertEquals(Record.NO_TIMESTAMP, info.maxTimestamp);
|
||||||
|
assertEquals(0L, info.shallowOffsetOfMaxTimestamp);
|
||||||
|
|
||||||
|
Iterator<Record> iterator = records.records();
|
||||||
|
while (iterator.hasNext()) {
|
||||||
|
Record record = iterator.next();
|
||||||
|
assertEquals(TimestampType.CREATE_TIME, record.timestampType());
|
||||||
|
assertEquals(Record.NO_TIMESTAMP, record.timestamp());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Parameterized.Parameters
|
||||||
|
public static Collection<Object[]> data() {
|
||||||
|
List<Object[]> values = new ArrayList<>();
|
||||||
|
for (int bufferOffset : Arrays.asList(0, 15))
|
||||||
|
for (CompressionType compressionType : CompressionType.values())
|
||||||
|
values.add(new Object[] {bufferOffset, compressionType});
|
||||||
|
return values;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -16,53 +16,64 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.common.record;
|
package org.apache.kafka.common.record;
|
||||||
|
|
||||||
import static org.apache.kafka.common.utils.Utils.toArray;
|
import org.apache.kafka.test.TestUtils;
|
||||||
import static org.junit.Assert.assertEquals;
|
import org.junit.Test;
|
||||||
import static org.junit.Assert.assertFalse;
|
import org.junit.runner.RunWith;
|
||||||
import static org.junit.Assert.assertTrue;
|
import org.junit.runners.Parameterized;
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
import org.junit.Test;
|
import static java.util.Arrays.asList;
|
||||||
import org.junit.runner.RunWith;
|
import static org.apache.kafka.common.utils.Utils.toNullableArray;
|
||||||
import org.junit.runners.Parameterized;
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertFalse;
|
||||||
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
@RunWith(value = Parameterized.class)
|
@RunWith(value = Parameterized.class)
|
||||||
public class MemoryRecordsTest {
|
public class MemoryRecordsTest {
|
||||||
|
|
||||||
private CompressionType compression;
|
private CompressionType compression;
|
||||||
|
private byte magic;
|
||||||
|
private long firstOffset;
|
||||||
|
|
||||||
public MemoryRecordsTest(CompressionType compression) {
|
public MemoryRecordsTest(byte magic, long firstOffset, CompressionType compression) {
|
||||||
|
this.magic = magic;
|
||||||
this.compression = compression;
|
this.compression = compression;
|
||||||
|
this.firstOffset = firstOffset;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testIterator() {
|
public void testIterator() {
|
||||||
MemoryRecords recs1 = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), compression);
|
MemoryRecordsBuilder builder1 = MemoryRecords.builder(ByteBuffer.allocate(1024), magic, compression, TimestampType.CREATE_TIME, firstOffset);
|
||||||
MemoryRecords recs2 = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), compression);
|
MemoryRecordsBuilder builder2 = MemoryRecords.builder(ByteBuffer.allocate(1024), magic, compression, TimestampType.CREATE_TIME, firstOffset);
|
||||||
List<Record> list = Arrays.asList(new Record(0L, "a".getBytes(), "1".getBytes()),
|
List<Record> list = asList(
|
||||||
new Record(0L, "b".getBytes(), "2".getBytes()),
|
Record.create(magic, 1L, "a".getBytes(), "1".getBytes()),
|
||||||
new Record(0L, "c".getBytes(), "3".getBytes()));
|
Record.create(magic, 2L, "b".getBytes(), "2".getBytes()),
|
||||||
|
Record.create(magic, 3L, "c".getBytes(), "3".getBytes()),
|
||||||
|
Record.create(magic, 4L, null, "4".getBytes()),
|
||||||
|
Record.create(magic, 5L, "e".getBytes(), null),
|
||||||
|
Record.create(magic, 6L, null, null));
|
||||||
|
|
||||||
for (int i = 0; i < list.size(); i++) {
|
for (int i = 0; i < list.size(); i++) {
|
||||||
Record r = list.get(i);
|
Record r = list.get(i);
|
||||||
recs1.append(i, r);
|
builder1.append(firstOffset + i, r);
|
||||||
recs2.append(i, 0L, toArray(r.key()), toArray(r.value()));
|
builder2.append(firstOffset + i, i + 1, toNullableArray(r.key()), toNullableArray(r.value()));
|
||||||
}
|
}
|
||||||
recs1.close();
|
|
||||||
recs2.close();
|
MemoryRecords recs1 = builder1.build();
|
||||||
|
MemoryRecords recs2 = builder2.build();
|
||||||
|
|
||||||
for (int iteration = 0; iteration < 2; iteration++) {
|
for (int iteration = 0; iteration < 2; iteration++) {
|
||||||
for (MemoryRecords recs : Arrays.asList(recs1, recs2)) {
|
for (MemoryRecords recs : asList(recs1, recs2)) {
|
||||||
Iterator<LogEntry> iter = recs.iterator();
|
Iterator<LogEntry> iter = recs.deepIterator();
|
||||||
for (int i = 0; i < list.size(); i++) {
|
for (int i = 0; i < list.size(); i++) {
|
||||||
assertTrue(iter.hasNext());
|
assertTrue(iter.hasNext());
|
||||||
LogEntry entry = iter.next();
|
LogEntry entry = iter.next();
|
||||||
assertEquals((long) i, entry.offset());
|
assertEquals(firstOffset + i, entry.offset());
|
||||||
assertEquals(list.get(i), entry.record());
|
assertEquals(list.get(i), entry.record());
|
||||||
entry.record().ensureValid();
|
entry.record().ensureValid();
|
||||||
}
|
}
|
||||||
|
|
@ -73,20 +84,145 @@ public class MemoryRecordsTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testHasRoomForMethod() {
|
public void testHasRoomForMethod() {
|
||||||
MemoryRecords recs1 = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), compression);
|
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), magic, compression, TimestampType.CREATE_TIME);
|
||||||
recs1.append(0, new Record(0L, "a".getBytes(), "1".getBytes()));
|
builder.append(0, Record.create(magic, 0L, "a".getBytes(), "1".getBytes()));
|
||||||
|
|
||||||
assertTrue(recs1.hasRoomFor("b".getBytes(), "2".getBytes()));
|
assertTrue(builder.hasRoomFor("b".getBytes(), "2".getBytes()));
|
||||||
recs1.close();
|
builder.close();
|
||||||
assertFalse(recs1.hasRoomFor("b".getBytes(), "2".getBytes()));
|
assertFalse(builder.hasRoomFor("b".getBytes(), "2".getBytes()));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFilterTo() {
|
||||||
|
ByteBuffer buffer = ByteBuffer.allocate(2048);
|
||||||
|
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME);
|
||||||
|
builder.append(0L, 10L, null, "a".getBytes());
|
||||||
|
builder.close();
|
||||||
|
|
||||||
|
builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME, 1L);
|
||||||
|
builder.append(1L, 11L, "1".getBytes(), "b".getBytes());
|
||||||
|
builder.append(2L, 12L, null, "c".getBytes());
|
||||||
|
builder.close();
|
||||||
|
|
||||||
|
builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME, 3L);
|
||||||
|
builder.append(3L, 13L, null, "d".getBytes());
|
||||||
|
builder.append(4L, 20L, "4".getBytes(), "e".getBytes());
|
||||||
|
builder.append(5L, 15L, "5".getBytes(), "f".getBytes());
|
||||||
|
builder.close();
|
||||||
|
|
||||||
|
builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME, 6L);
|
||||||
|
builder.append(6L, 16L, "6".getBytes(), "g".getBytes());
|
||||||
|
builder.close();
|
||||||
|
|
||||||
|
buffer.flip();
|
||||||
|
|
||||||
|
ByteBuffer filtered = ByteBuffer.allocate(2048);
|
||||||
|
MemoryRecords.FilterResult result = MemoryRecords.readableRecords(buffer).filterTo(new RetainNonNullKeysFilter(), filtered);
|
||||||
|
|
||||||
|
filtered.flip();
|
||||||
|
|
||||||
|
assertEquals(7, result.messagesRead);
|
||||||
|
assertEquals(4, result.messagesRetained);
|
||||||
|
assertEquals(buffer.limit(), result.bytesRead);
|
||||||
|
assertEquals(filtered.limit(), result.bytesRetained);
|
||||||
|
if (magic > 0) {
|
||||||
|
assertEquals(20L, result.maxTimestamp);
|
||||||
|
if (compression == CompressionType.NONE)
|
||||||
|
assertEquals(4L, result.shallowOffsetOfMaxTimestamp);
|
||||||
|
else
|
||||||
|
assertEquals(5L, result.shallowOffsetOfMaxTimestamp);
|
||||||
|
}
|
||||||
|
|
||||||
|
MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered);
|
||||||
|
|
||||||
|
List<ByteBufferLogInputStream.ByteBufferLogEntry> shallowEntries = TestUtils.toList(filteredRecords.shallowIterator());
|
||||||
|
List<Long> expectedOffsets = compression == CompressionType.NONE ? asList(1L, 4L, 5L, 6L) : asList(1L, 5L, 6L);
|
||||||
|
assertEquals(expectedOffsets.size(), shallowEntries.size());
|
||||||
|
|
||||||
|
for (int i = 0; i < expectedOffsets.size(); i++) {
|
||||||
|
LogEntry shallowEntry = shallowEntries.get(i);
|
||||||
|
assertEquals(expectedOffsets.get(i).longValue(), shallowEntry.offset());
|
||||||
|
assertEquals(magic, shallowEntry.record().magic());
|
||||||
|
assertEquals(compression, shallowEntry.record().compressionType());
|
||||||
|
assertEquals(magic == Record.MAGIC_VALUE_V0 ? TimestampType.NO_TIMESTAMP_TYPE : TimestampType.CREATE_TIME,
|
||||||
|
shallowEntry.record().timestampType());
|
||||||
|
}
|
||||||
|
|
||||||
|
List<LogEntry> deepEntries = TestUtils.toList(filteredRecords.deepIterator());
|
||||||
|
assertEquals(4, deepEntries.size());
|
||||||
|
|
||||||
|
LogEntry first = deepEntries.get(0);
|
||||||
|
assertEquals(1L, first.offset());
|
||||||
|
assertEquals(Record.create(magic, 11L, "1".getBytes(), "b".getBytes()), first.record());
|
||||||
|
|
||||||
|
LogEntry second = deepEntries.get(1);
|
||||||
|
assertEquals(4L, second.offset());
|
||||||
|
assertEquals(Record.create(magic, 20L, "4".getBytes(), "e".getBytes()), second.record());
|
||||||
|
|
||||||
|
LogEntry third = deepEntries.get(2);
|
||||||
|
assertEquals(5L, third.offset());
|
||||||
|
assertEquals(Record.create(magic, 15L, "5".getBytes(), "f".getBytes()), third.record());
|
||||||
|
|
||||||
|
LogEntry fourth = deepEntries.get(3);
|
||||||
|
assertEquals(6L, fourth.offset());
|
||||||
|
assertEquals(Record.create(magic, 16L, "6".getBytes(), "g".getBytes()), fourth.record());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testFilterToPreservesLogAppendTime() {
|
||||||
|
long logAppendTime = System.currentTimeMillis();
|
||||||
|
|
||||||
|
ByteBuffer buffer = ByteBuffer.allocate(2048);
|
||||||
|
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic, compression,
|
||||||
|
TimestampType.LOG_APPEND_TIME, 0L, logAppendTime);
|
||||||
|
builder.append(0L, 10L, null, "a".getBytes());
|
||||||
|
builder.close();
|
||||||
|
|
||||||
|
builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.LOG_APPEND_TIME, 1L, logAppendTime);
|
||||||
|
builder.append(1L, 11L, "1".getBytes(), "b".getBytes());
|
||||||
|
builder.append(2L, 12L, null, "c".getBytes());
|
||||||
|
builder.close();
|
||||||
|
|
||||||
|
builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.LOG_APPEND_TIME, 3L, logAppendTime);
|
||||||
|
builder.append(3L, 13L, null, "d".getBytes());
|
||||||
|
builder.append(4L, 14L, "4".getBytes(), "e".getBytes());
|
||||||
|
builder.append(5L, 15L, "5".getBytes(), "f".getBytes());
|
||||||
|
builder.close();
|
||||||
|
|
||||||
|
buffer.flip();
|
||||||
|
|
||||||
|
ByteBuffer filtered = ByteBuffer.allocate(2048);
|
||||||
|
MemoryRecords.readableRecords(buffer).filterTo(new RetainNonNullKeysFilter(), filtered);
|
||||||
|
|
||||||
|
filtered.flip();
|
||||||
|
MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered);
|
||||||
|
|
||||||
|
List<ByteBufferLogInputStream.ByteBufferLogEntry> shallowEntries = TestUtils.toList(filteredRecords.shallowIterator());
|
||||||
|
assertEquals(compression == CompressionType.NONE ? 3 : 2, shallowEntries.size());
|
||||||
|
|
||||||
|
for (LogEntry shallowEntry : shallowEntries) {
|
||||||
|
assertEquals(compression, shallowEntry.record().compressionType());
|
||||||
|
if (magic > Record.MAGIC_VALUE_V0) {
|
||||||
|
assertEquals(TimestampType.LOG_APPEND_TIME, shallowEntry.record().timestampType());
|
||||||
|
assertEquals(logAppendTime, shallowEntry.record().timestamp());
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Parameterized.Parameters
|
@Parameterized.Parameters
|
||||||
public static Collection<Object[]> data() {
|
public static Collection<Object[]> data() {
|
||||||
List<Object[]> values = new ArrayList<Object[]>();
|
List<Object[]> values = new ArrayList<>();
|
||||||
for (CompressionType type: CompressionType.values())
|
for (long firstOffset : asList(0L, 57L))
|
||||||
values.add(new Object[] {type});
|
for (byte magic : asList(Record.MAGIC_VALUE_V0, Record.MAGIC_VALUE_V1))
|
||||||
|
for (CompressionType type: CompressionType.values())
|
||||||
|
values.add(new Object[] {magic, firstOffset, type});
|
||||||
return values;
|
return values;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static class RetainNonNullKeysFilter implements MemoryRecords.LogEntryFilter {
|
||||||
|
@Override
|
||||||
|
public boolean shouldRetain(LogEntry entry) {
|
||||||
|
return entry.record().hasKey();
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -35,18 +35,22 @@ import org.junit.runners.Parameterized.Parameters;
|
||||||
@RunWith(value = Parameterized.class)
|
@RunWith(value = Parameterized.class)
|
||||||
public class RecordTest {
|
public class RecordTest {
|
||||||
|
|
||||||
|
private byte magic;
|
||||||
private long timestamp;
|
private long timestamp;
|
||||||
private ByteBuffer key;
|
private ByteBuffer key;
|
||||||
private ByteBuffer value;
|
private ByteBuffer value;
|
||||||
private CompressionType compression;
|
private CompressionType compression;
|
||||||
|
private TimestampType timestampType;
|
||||||
private Record record;
|
private Record record;
|
||||||
|
|
||||||
public RecordTest(long timestamp, byte[] key, byte[] value, CompressionType compression) {
|
public RecordTest(byte magic, long timestamp, byte[] key, byte[] value, CompressionType compression) {
|
||||||
|
this.magic = magic;
|
||||||
this.timestamp = timestamp;
|
this.timestamp = timestamp;
|
||||||
|
this.timestampType = TimestampType.CREATE_TIME;
|
||||||
this.key = key == null ? null : ByteBuffer.wrap(key);
|
this.key = key == null ? null : ByteBuffer.wrap(key);
|
||||||
this.value = value == null ? null : ByteBuffer.wrap(value);
|
this.value = value == null ? null : ByteBuffer.wrap(value);
|
||||||
this.compression = compression;
|
this.compression = compression;
|
||||||
this.record = new Record(timestamp, key, value, compression);
|
this.record = Record.create(magic, timestamp, key, value, compression, timestampType);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
@ -56,22 +60,33 @@ public class RecordTest {
|
||||||
assertEquals(key, record.key());
|
assertEquals(key, record.key());
|
||||||
if (key != null)
|
if (key != null)
|
||||||
assertEquals(key.limit(), record.keySize());
|
assertEquals(key.limit(), record.keySize());
|
||||||
assertEquals(Record.CURRENT_MAGIC_VALUE, record.magic());
|
assertEquals(magic, record.magic());
|
||||||
assertEquals(value, record.value());
|
assertEquals(value, record.value());
|
||||||
if (value != null)
|
if (value != null)
|
||||||
assertEquals(value.limit(), record.valueSize());
|
assertEquals(value.limit(), record.valueSize());
|
||||||
|
if (magic > 0) {
|
||||||
|
assertEquals(timestamp, record.timestamp());
|
||||||
|
assertEquals(timestampType, record.timestampType());
|
||||||
|
} else {
|
||||||
|
assertEquals(Record.NO_TIMESTAMP, record.timestamp());
|
||||||
|
assertEquals(TimestampType.NO_TIMESTAMP_TYPE, record.timestampType());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testChecksum() {
|
public void testChecksum() {
|
||||||
assertEquals(record.checksum(), record.computeChecksum());
|
assertEquals(record.checksum(), record.computeChecksum());
|
||||||
|
|
||||||
|
byte attributes = Record.computeAttributes(magic, this.compression, TimestampType.CREATE_TIME);
|
||||||
assertEquals(record.checksum(), Record.computeChecksum(
|
assertEquals(record.checksum(), Record.computeChecksum(
|
||||||
this.timestamp,
|
magic,
|
||||||
this.key == null ? null : this.key.array(),
|
attributes,
|
||||||
this.value == null ? null : this.value.array(),
|
this.timestamp,
|
||||||
this.compression, 0, -1));
|
this.key == null ? null : this.key.array(),
|
||||||
|
this.value == null ? null : this.value.array()
|
||||||
|
));
|
||||||
assertTrue(record.isValid());
|
assertTrue(record.isValid());
|
||||||
for (int i = Record.CRC_OFFSET + Record.CRC_LENGTH; i < record.size(); i++) {
|
for (int i = Record.CRC_OFFSET + Record.CRC_LENGTH; i < record.sizeInBytes(); i++) {
|
||||||
Record copy = copyOf(record);
|
Record copy = copyOf(record);
|
||||||
copy.buffer().put(i, (byte) 69);
|
copy.buffer().put(i, (byte) 69);
|
||||||
assertFalse(copy.isValid());
|
assertFalse(copy.isValid());
|
||||||
|
|
@ -85,7 +100,7 @@ public class RecordTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
private Record copyOf(Record record) {
|
private Record copyOf(Record record) {
|
||||||
ByteBuffer buffer = ByteBuffer.allocate(record.size());
|
ByteBuffer buffer = ByteBuffer.allocate(record.sizeInBytes());
|
||||||
record.buffer().put(buffer);
|
record.buffer().put(buffer);
|
||||||
buffer.rewind();
|
buffer.rewind();
|
||||||
record.buffer().rewind();
|
record.buffer().rewind();
|
||||||
|
|
@ -101,12 +116,13 @@ public class RecordTest {
|
||||||
public static Collection<Object[]> data() {
|
public static Collection<Object[]> data() {
|
||||||
byte[] payload = new byte[1000];
|
byte[] payload = new byte[1000];
|
||||||
Arrays.fill(payload, (byte) 1);
|
Arrays.fill(payload, (byte) 1);
|
||||||
List<Object[]> values = new ArrayList<Object[]>();
|
List<Object[]> values = new ArrayList<>();
|
||||||
for (long timestamp : Arrays.asList(Record.NO_TIMESTAMP, 0L, 1L))
|
for (byte magic : Arrays.asList(Record.MAGIC_VALUE_V0, Record.MAGIC_VALUE_V1))
|
||||||
for (byte[] key : Arrays.asList(null, "".getBytes(), "key".getBytes(), payload))
|
for (long timestamp : Arrays.asList(Record.NO_TIMESTAMP, 0L, 1L))
|
||||||
for (byte[] value : Arrays.asList(null, "".getBytes(), "value".getBytes(), payload))
|
for (byte[] key : Arrays.asList(null, "".getBytes(), "key".getBytes(), payload))
|
||||||
for (CompressionType compression : CompressionType.values())
|
for (byte[] value : Arrays.asList(null, "".getBytes(), "value".getBytes(), payload))
|
||||||
values.add(new Object[] {timestamp, key, value, compression});
|
for (CompressionType compression : CompressionType.values())
|
||||||
|
values.add(new Object[] {magic, timestamp, key, value, compression});
|
||||||
return values;
|
return values;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -20,35 +20,29 @@ import org.junit.Test;
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
import static org.junit.Assert.assertFalse;
|
import static org.junit.Assert.assertFalse;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
import static org.junit.Assert.fail;
|
|
||||||
|
|
||||||
public class SimpleRecordTest {
|
public class SimpleRecordTest {
|
||||||
|
|
||||||
/* This scenario can happen if the record size field is corrupt and we end up allocating a buffer that is too small */
|
/* This scenario can happen if the record size field is corrupt and we end up allocating a buffer that is too small */
|
||||||
@Test
|
@Test(expected = InvalidRecordException.class)
|
||||||
public void testIsValidWithTooSmallBuffer() {
|
public void testIsValidWithTooSmallBuffer() {
|
||||||
ByteBuffer buffer = ByteBuffer.allocate(2);
|
ByteBuffer buffer = ByteBuffer.allocate(2);
|
||||||
Record record = new Record(buffer);
|
Record record = new Record(buffer);
|
||||||
assertFalse(record.isValid());
|
assertFalse(record.isValid());
|
||||||
try {
|
record.ensureValid();
|
||||||
record.ensureValid();
|
|
||||||
fail("InvalidRecordException should have been thrown");
|
|
||||||
} catch (InvalidRecordException e) { }
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test(expected = InvalidRecordException.class)
|
||||||
public void testIsValidWithChecksumMismatch() {
|
public void testIsValidWithChecksumMismatch() {
|
||||||
ByteBuffer buffer = ByteBuffer.allocate(4);
|
ByteBuffer buffer = ByteBuffer.allocate(4);
|
||||||
// set checksum
|
// set checksum
|
||||||
buffer.putInt(2);
|
buffer.putInt(2);
|
||||||
Record record = new Record(buffer);
|
Record record = new Record(buffer);
|
||||||
assertFalse(record.isValid());
|
assertFalse(record.isValid());
|
||||||
try {
|
record.ensureValid();
|
||||||
record.ensureValid();
|
|
||||||
fail("InvalidRecordException should have been thrown");
|
|
||||||
} catch (InvalidRecordException e) { }
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
@ -63,4 +57,40 @@ public class SimpleRecordTest {
|
||||||
record.ensureValid();
|
record.ensureValid();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testConvertFromV0ToV1() {
|
||||||
|
byte[][] keys = new byte[][] {"a".getBytes(), "".getBytes(), null, "b".getBytes()};
|
||||||
|
byte[][] values = new byte[][] {"1".getBytes(), "".getBytes(), "2".getBytes(), null};
|
||||||
|
|
||||||
|
for (int i = 0; i < keys.length; i++) {
|
||||||
|
Record record = Record.create(Record.MAGIC_VALUE_V0, Record.NO_TIMESTAMP, keys[i], values[i]);
|
||||||
|
Record converted = record.convert(Record.MAGIC_VALUE_V1);
|
||||||
|
|
||||||
|
assertEquals(Record.MAGIC_VALUE_V1, converted.magic());
|
||||||
|
assertEquals(Record.NO_TIMESTAMP, converted.timestamp());
|
||||||
|
assertEquals(record.key(), converted.key());
|
||||||
|
assertEquals(record.value(), converted.value());
|
||||||
|
assertTrue(record.isValid());
|
||||||
|
assertEquals(record.convertedSize(Record.MAGIC_VALUE_V1), converted.sizeInBytes());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testConvertFromV1ToV0() {
|
||||||
|
byte[][] keys = new byte[][] {"a".getBytes(), "".getBytes(), null, "b".getBytes()};
|
||||||
|
byte[][] values = new byte[][] {"1".getBytes(), "".getBytes(), "2".getBytes(), null};
|
||||||
|
|
||||||
|
for (int i = 0; i < keys.length; i++) {
|
||||||
|
Record record = Record.create(Record.MAGIC_VALUE_V1, System.currentTimeMillis(), keys[i], values[i]);
|
||||||
|
Record converted = record.convert(Record.MAGIC_VALUE_V0);
|
||||||
|
|
||||||
|
assertEquals(Record.MAGIC_VALUE_V0, converted.magic());
|
||||||
|
assertEquals(Record.NO_TIMESTAMP, converted.timestamp());
|
||||||
|
assertEquals(record.key(), converted.key());
|
||||||
|
assertEquals(record.value(), converted.value());
|
||||||
|
assertTrue(record.isValid());
|
||||||
|
assertEquals(record.convertedSize(Record.MAGIC_VALUE_V0), converted.sizeInBytes());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,37 @@
|
||||||
|
/**
|
||||||
|
* 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
|
||||||
|
* <p/>
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
* <p/>
|
||||||
|
* 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.junit.Test;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
|
||||||
|
public class TimestampTypeTest {
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void toAndFromAttributesCreateTime() {
|
||||||
|
byte attributes = TimestampType.CREATE_TIME.updateAttributes((byte) 0);
|
||||||
|
assertEquals(TimestampType.CREATE_TIME, TimestampType.forAttributes(attributes));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void toAndFromAttributesLogAppendTime() {
|
||||||
|
byte attributes = TimestampType.LOG_APPEND_TIME.updateAttributes((byte) 0);
|
||||||
|
assertEquals(TimestampType.LOG_APPEND_TIME, TimestampType.forAttributes(attributes));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -23,8 +23,10 @@ import org.apache.kafka.common.Node;
|
||||||
import org.apache.kafka.common.PartitionInfo;
|
import org.apache.kafka.common.PartitionInfo;
|
||||||
import org.apache.kafka.common.record.CompressionType;
|
import org.apache.kafka.common.record.CompressionType;
|
||||||
import org.apache.kafka.common.record.MemoryRecords;
|
import org.apache.kafka.common.record.MemoryRecords;
|
||||||
|
import org.apache.kafka.common.record.MemoryRecordsBuilder;
|
||||||
import org.apache.kafka.common.record.Record;
|
import org.apache.kafka.common.record.Record;
|
||||||
import org.apache.kafka.common.record.Records;
|
import org.apache.kafka.common.record.Records;
|
||||||
|
import org.apache.kafka.common.record.TimestampType;
|
||||||
import org.apache.kafka.common.utils.Utils;
|
import org.apache.kafka.common.utils.Utils;
|
||||||
|
|
||||||
import javax.xml.bind.DatatypeConverter;
|
import javax.xml.bind.DatatypeConverter;
|
||||||
|
|
@ -35,6 +37,7 @@ import java.nio.file.Files;
|
||||||
import java.nio.file.Path;
|
import java.nio.file.Path;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Properties;
|
import java.util.Properties;
|
||||||
|
|
@ -46,6 +49,7 @@ import java.util.regex.Pattern;
|
||||||
|
|
||||||
import static java.util.Arrays.asList;
|
import static java.util.Arrays.asList;
|
||||||
import static org.junit.Assert.assertEquals;
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertFalse;
|
||||||
import static org.junit.Assert.assertNotNull;
|
import static org.junit.Assert.assertNotNull;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
import static org.junit.Assert.fail;
|
import static org.junit.Assert.fail;
|
||||||
|
|
@ -185,13 +189,13 @@ public class TestUtils {
|
||||||
public static ByteBuffer partitionRecordsBuffer(final long offset, final CompressionType compressionType, final Record... records) {
|
public static ByteBuffer partitionRecordsBuffer(final long offset, final CompressionType compressionType, final Record... records) {
|
||||||
int bufferSize = 0;
|
int bufferSize = 0;
|
||||||
for (final Record record : records)
|
for (final Record record : records)
|
||||||
bufferSize += Records.LOG_OVERHEAD + record.size();
|
bufferSize += Records.LOG_OVERHEAD + record.sizeInBytes();
|
||||||
final ByteBuffer buffer = ByteBuffer.allocate(bufferSize);
|
final ByteBuffer buffer = ByteBuffer.allocate(bufferSize);
|
||||||
final MemoryRecords memoryRecords = MemoryRecords.emptyRecords(buffer, compressionType);
|
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, compressionType, TimestampType.CREATE_TIME);
|
||||||
|
long nextOffset = offset;
|
||||||
for (final Record record : records)
|
for (final Record record : records)
|
||||||
memoryRecords.append(offset, record);
|
builder.append(nextOffset++, record);
|
||||||
memoryRecords.close();
|
return builder.build().buffer();
|
||||||
return memoryRecords.buffer();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public static Properties producerConfig(final String bootstrapServers,
|
public static Properties producerConfig(final String bootstrapServers,
|
||||||
|
|
@ -309,4 +313,22 @@ public class TestUtils {
|
||||||
fail(clusterId + " cannot be converted back to UUID.");
|
fail(clusterId + " cannot be converted back to UUID.");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Throw an exception if the two iterators are of differing lengths or contain
|
||||||
|
* different messages on their Nth element
|
||||||
|
*/
|
||||||
|
public static <T> void checkEquals(Iterator<T> s1, Iterator<T> s2) {
|
||||||
|
while (s1.hasNext() && s2.hasNext())
|
||||||
|
assertEquals(s1.next(), s2.next());
|
||||||
|
assertFalse("Iterators have uneven length--first has more", s1.hasNext());
|
||||||
|
assertFalse("Iterators have uneven length--second has more", s2.hasNext());
|
||||||
|
}
|
||||||
|
|
||||||
|
public static <T> List<T> toList(Iterator<T> iterator) {
|
||||||
|
List<T> res = new ArrayList<>();
|
||||||
|
while (iterator.hasNext())
|
||||||
|
res.add(iterator.next());
|
||||||
|
return res;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -17,7 +17,7 @@
|
||||||
|
|
||||||
package kafka.api
|
package kafka.api
|
||||||
|
|
||||||
import kafka.message.Message
|
import org.apache.kafka.common.record.Record
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This class contains the different Kafka versions.
|
* This class contains the different Kafka versions.
|
||||||
|
|
@ -87,54 +87,54 @@ sealed trait ApiVersion extends Ordered[ApiVersion] {
|
||||||
// Keep the IDs in order of versions
|
// Keep the IDs in order of versions
|
||||||
case object KAFKA_0_8_0 extends ApiVersion {
|
case object KAFKA_0_8_0 extends ApiVersion {
|
||||||
val version: String = "0.8.0.X"
|
val version: String = "0.8.0.X"
|
||||||
val messageFormatVersion: Byte = Message.MagicValue_V0
|
val messageFormatVersion: Byte = Record.MAGIC_VALUE_V0
|
||||||
val id: Int = 0
|
val id: Int = 0
|
||||||
}
|
}
|
||||||
|
|
||||||
case object KAFKA_0_8_1 extends ApiVersion {
|
case object KAFKA_0_8_1 extends ApiVersion {
|
||||||
val version: String = "0.8.1.X"
|
val version: String = "0.8.1.X"
|
||||||
val messageFormatVersion: Byte = Message.MagicValue_V0
|
val messageFormatVersion: Byte = Record.MAGIC_VALUE_V0
|
||||||
val id: Int = 1
|
val id: Int = 1
|
||||||
}
|
}
|
||||||
|
|
||||||
case object KAFKA_0_8_2 extends ApiVersion {
|
case object KAFKA_0_8_2 extends ApiVersion {
|
||||||
val version: String = "0.8.2.X"
|
val version: String = "0.8.2.X"
|
||||||
val messageFormatVersion: Byte = Message.MagicValue_V0
|
val messageFormatVersion: Byte = Record.MAGIC_VALUE_V0
|
||||||
val id: Int = 2
|
val id: Int = 2
|
||||||
}
|
}
|
||||||
|
|
||||||
case object KAFKA_0_9_0 extends ApiVersion {
|
case object KAFKA_0_9_0 extends ApiVersion {
|
||||||
val version: String = "0.9.0.X"
|
val version: String = "0.9.0.X"
|
||||||
val messageFormatVersion: Byte = Message.MagicValue_V0
|
val messageFormatVersion: Byte = Record.MAGIC_VALUE_V0
|
||||||
val id: Int = 3
|
val id: Int = 3
|
||||||
}
|
}
|
||||||
|
|
||||||
case object KAFKA_0_10_0_IV0 extends ApiVersion {
|
case object KAFKA_0_10_0_IV0 extends ApiVersion {
|
||||||
val version: String = "0.10.0-IV0"
|
val version: String = "0.10.0-IV0"
|
||||||
val messageFormatVersion: Byte = Message.MagicValue_V1
|
val messageFormatVersion: Byte = Record.MAGIC_VALUE_V1
|
||||||
val id: Int = 4
|
val id: Int = 4
|
||||||
}
|
}
|
||||||
|
|
||||||
case object KAFKA_0_10_0_IV1 extends ApiVersion {
|
case object KAFKA_0_10_0_IV1 extends ApiVersion {
|
||||||
val version: String = "0.10.0-IV1"
|
val version: String = "0.10.0-IV1"
|
||||||
val messageFormatVersion: Byte = Message.MagicValue_V1
|
val messageFormatVersion: Byte = Record.MAGIC_VALUE_V1
|
||||||
val id: Int = 5
|
val id: Int = 5
|
||||||
}
|
}
|
||||||
|
|
||||||
case object KAFKA_0_10_1_IV0 extends ApiVersion {
|
case object KAFKA_0_10_1_IV0 extends ApiVersion {
|
||||||
val version: String = "0.10.1-IV0"
|
val version: String = "0.10.1-IV0"
|
||||||
val messageFormatVersion: Byte = Message.MagicValue_V1
|
val messageFormatVersion: Byte = Record.MAGIC_VALUE_V1
|
||||||
val id: Int = 6
|
val id: Int = 6
|
||||||
}
|
}
|
||||||
|
|
||||||
case object KAFKA_0_10_1_IV1 extends ApiVersion {
|
case object KAFKA_0_10_1_IV1 extends ApiVersion {
|
||||||
val version: String = "0.10.1-IV1"
|
val version: String = "0.10.1-IV1"
|
||||||
val messageFormatVersion: Byte = Message.MagicValue_V1
|
val messageFormatVersion: Byte = Record.MAGIC_VALUE_V1
|
||||||
val id: Int = 7
|
val id: Int = 7
|
||||||
}
|
}
|
||||||
|
|
||||||
case object KAFKA_0_10_1_IV2 extends ApiVersion {
|
case object KAFKA_0_10_1_IV2 extends ApiVersion {
|
||||||
val version: String = "0.10.1-IV2"
|
val version: String = "0.10.1-IV2"
|
||||||
val messageFormatVersion: Byte = Message.MagicValue_V1
|
val messageFormatVersion: Byte = Record.MAGIC_VALUE_V1
|
||||||
val id: Int = 8
|
val id: Int = 8
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -25,7 +25,6 @@ import kafka.log.LogConfig
|
||||||
import kafka.server._
|
import kafka.server._
|
||||||
import kafka.metrics.KafkaMetricsGroup
|
import kafka.metrics.KafkaMetricsGroup
|
||||||
import kafka.controller.KafkaController
|
import kafka.controller.KafkaController
|
||||||
import kafka.message.ByteBufferMessageSet
|
|
||||||
import java.io.IOException
|
import java.io.IOException
|
||||||
import java.util.concurrent.locks.ReentrantReadWriteLock
|
import java.util.concurrent.locks.ReentrantReadWriteLock
|
||||||
|
|
||||||
|
|
@ -34,6 +33,7 @@ import org.apache.kafka.common.protocol.Errors
|
||||||
|
|
||||||
import scala.collection.JavaConverters._
|
import scala.collection.JavaConverters._
|
||||||
import com.yammer.metrics.core.Gauge
|
import com.yammer.metrics.core.Gauge
|
||||||
|
import org.apache.kafka.common.record.MemoryRecords
|
||||||
import org.apache.kafka.common.requests.PartitionState
|
import org.apache.kafka.common.requests.PartitionState
|
||||||
import org.apache.kafka.common.utils.Time
|
import org.apache.kafka.common.utils.Time
|
||||||
|
|
||||||
|
|
@ -190,7 +190,7 @@ class Partition(val topic: String,
|
||||||
allReplicas.foreach(replica => getOrCreateReplica(replica))
|
allReplicas.foreach(replica => getOrCreateReplica(replica))
|
||||||
val newInSyncReplicas = partitionStateInfo.isr.asScala.map(r => getOrCreateReplica(r)).toSet
|
val newInSyncReplicas = partitionStateInfo.isr.asScala.map(r => getOrCreateReplica(r)).toSet
|
||||||
// remove assigned replicas that have been removed by the controller
|
// remove assigned replicas that have been removed by the controller
|
||||||
(assignedReplicas().map(_.brokerId) -- allReplicas).foreach(removeReplica(_))
|
(assignedReplicas().map(_.brokerId) -- allReplicas).foreach(removeReplica)
|
||||||
inSyncReplicas = newInSyncReplicas
|
inSyncReplicas = newInSyncReplicas
|
||||||
leaderEpoch = partitionStateInfo.leaderEpoch
|
leaderEpoch = partitionStateInfo.leaderEpoch
|
||||||
zkVersion = partitionStateInfo.zkVersion
|
zkVersion = partitionStateInfo.zkVersion
|
||||||
|
|
@ -440,7 +440,7 @@ class Partition(val topic: String,
|
||||||
laggingReplicas
|
laggingReplicas
|
||||||
}
|
}
|
||||||
|
|
||||||
def appendMessagesToLeader(messages: ByteBufferMessageSet, requiredAcks: Int = 0) = {
|
def appendRecordsToLeader(records: MemoryRecords, requiredAcks: Int = 0) = {
|
||||||
val (info, leaderHWIncremented) = inReadLock(leaderIsrUpdateLock) {
|
val (info, leaderHWIncremented) = inReadLock(leaderIsrUpdateLock) {
|
||||||
val leaderReplicaOpt = leaderReplicaIfLocal()
|
val leaderReplicaOpt = leaderReplicaIfLocal()
|
||||||
leaderReplicaOpt match {
|
leaderReplicaOpt match {
|
||||||
|
|
@ -455,7 +455,7 @@ class Partition(val topic: String,
|
||||||
.format(topic, partitionId, inSyncSize, minIsr))
|
.format(topic, partitionId, inSyncSize, minIsr))
|
||||||
}
|
}
|
||||||
|
|
||||||
val info = log.append(messages, assignOffsets = true)
|
val info = log.append(records, assignOffsets = true)
|
||||||
// probably unblock some follower fetch requests since log end offset has been updated
|
// probably unblock some follower fetch requests since log end offset has been updated
|
||||||
replicaManager.tryCompleteDelayedFetch(TopicPartitionOperationKey(this.topic, this.partitionId))
|
replicaManager.tryCompleteDelayedFetch(TopicPartitionOperationKey(this.topic, this.partitionId))
|
||||||
// we may need to increment high watermark since ISR could be down to 1
|
// we may need to increment high watermark since ISR could be down to 1
|
||||||
|
|
@ -480,7 +480,7 @@ class Partition(val topic: String,
|
||||||
newLeaderAndIsr, controllerEpoch, zkVersion)
|
newLeaderAndIsr, controllerEpoch, zkVersion)
|
||||||
|
|
||||||
if(updateSucceeded) {
|
if(updateSucceeded) {
|
||||||
replicaManager.recordIsrChange(new TopicAndPartition(topic, partitionId))
|
replicaManager.recordIsrChange(TopicAndPartition(topic, partitionId))
|
||||||
inSyncReplicas = newIsr
|
inSyncReplicas = newIsr
|
||||||
zkVersion = newVersion
|
zkVersion = newVersion
|
||||||
trace("ISR updated to [%s] and zkVersion updated to [%d]".format(newIsr.mkString(","), zkVersion))
|
trace("ISR updated to [%s] and zkVersion updated to [%d]".format(newIsr.mkString(","), zkVersion))
|
||||||
|
|
|
||||||
|
|
@ -17,14 +17,16 @@
|
||||||
|
|
||||||
package kafka.consumer
|
package kafka.consumer
|
||||||
|
|
||||||
import kafka.api.{OffsetRequest, Request, FetchRequestBuilder, FetchResponsePartitionData}
|
import kafka.api.{FetchRequestBuilder, FetchResponsePartitionData, OffsetRequest, Request}
|
||||||
import kafka.cluster.BrokerEndPoint
|
import kafka.cluster.BrokerEndPoint
|
||||||
import kafka.message.ByteBufferMessageSet
|
import kafka.message.ByteBufferMessageSet
|
||||||
import kafka.server.{PartitionFetchState, AbstractFetcherThread}
|
import kafka.server.{AbstractFetcherThread, PartitionFetchState}
|
||||||
import kafka.common.{ErrorMapping, TopicAndPartition}
|
import kafka.common.{ErrorMapping, TopicAndPartition}
|
||||||
|
|
||||||
import scala.collection.Map
|
import scala.collection.Map
|
||||||
import ConsumerFetcherThread._
|
import ConsumerFetcherThread._
|
||||||
import org.apache.kafka.common.TopicPartition
|
import org.apache.kafka.common.TopicPartition
|
||||||
|
import org.apache.kafka.common.record.MemoryRecords
|
||||||
|
|
||||||
class ConsumerFetcherThread(name: String,
|
class ConsumerFetcherThread(name: String,
|
||||||
val config: ConsumerConfig,
|
val config: ConsumerConfig,
|
||||||
|
|
@ -81,7 +83,7 @@ class ConsumerFetcherThread(name: String,
|
||||||
case OffsetRequest.LargestTimeString => OffsetRequest.LatestTime
|
case OffsetRequest.LargestTimeString => OffsetRequest.LatestTime
|
||||||
case _ => OffsetRequest.LatestTime
|
case _ => OffsetRequest.LatestTime
|
||||||
}
|
}
|
||||||
val topicAndPartition = new TopicAndPartition(topicPartition.topic, topicPartition.partition)
|
val topicAndPartition = TopicAndPartition(topicPartition.topic, topicPartition.partition)
|
||||||
val newOffset = simpleConsumer.earliestOrLatestOffset(topicAndPartition, startTimestamp, Request.OrdinaryConsumerId)
|
val newOffset = simpleConsumer.earliestOrLatestOffset(topicAndPartition, startTimestamp, Request.OrdinaryConsumerId)
|
||||||
val pti = partitionMap(topicPartition)
|
val pti = partitionMap(topicPartition)
|
||||||
pti.resetFetchOffset(newOffset)
|
pti.resetFetchOffset(newOffset)
|
||||||
|
|
@ -123,7 +125,7 @@ object ConsumerFetcherThread {
|
||||||
|
|
||||||
class PartitionData(val underlying: FetchResponsePartitionData) extends AbstractFetcherThread.PartitionData {
|
class PartitionData(val underlying: FetchResponsePartitionData) extends AbstractFetcherThread.PartitionData {
|
||||||
def errorCode: Short = underlying.error
|
def errorCode: Short = underlying.error
|
||||||
def toByteBufferMessageSet: ByteBufferMessageSet = underlying.messages.asInstanceOf[ByteBufferMessageSet]
|
def toRecords: MemoryRecords = underlying.messages.asInstanceOf[ByteBufferMessageSet].asRecords
|
||||||
def highWatermark: Long = underlying.hw
|
def highWatermark: Long = underlying.hw
|
||||||
def exception: Option[Throwable] =
|
def exception: Option[Throwable] =
|
||||||
if (errorCode == ErrorMapping.NoError) None else Some(ErrorMapping.exceptionFor(errorCode))
|
if (errorCode == ErrorMapping.NoError) None else Some(ErrorMapping.exceptionFor(errorCode))
|
||||||
|
|
|
||||||
|
|
@ -458,14 +458,14 @@ class GroupCoordinator(val brokerId: Int,
|
||||||
def handleFetchOffsets(groupId: String,
|
def handleFetchOffsets(groupId: String,
|
||||||
partitions: Seq[TopicPartition]): Map[TopicPartition, OffsetFetchResponse.PartitionData] = {
|
partitions: Seq[TopicPartition]): Map[TopicPartition, OffsetFetchResponse.PartitionData] = {
|
||||||
if (!isActive.get) {
|
if (!isActive.get) {
|
||||||
partitions.map { case topicPartition =>
|
partitions.map { topicPartition =>
|
||||||
(topicPartition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code))}.toMap
|
(topicPartition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code))}.toMap
|
||||||
} else if (!isCoordinatorForGroup(groupId)) {
|
} else if (!isCoordinatorForGroup(groupId)) {
|
||||||
debug("Could not fetch offsets for group %s (not group coordinator).".format(groupId))
|
debug("Could not fetch offsets for group %s (not group coordinator).".format(groupId))
|
||||||
partitions.map { case topicPartition =>
|
partitions.map { topicPartition =>
|
||||||
(topicPartition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.NOT_COORDINATOR_FOR_GROUP.code))}.toMap
|
(topicPartition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.NOT_COORDINATOR_FOR_GROUP.code))}.toMap
|
||||||
} else if (isCoordinatorLoadingInProgress(groupId)) {
|
} else if (isCoordinatorLoadingInProgress(groupId)) {
|
||||||
partitions.map { case topicPartition =>
|
partitions.map { topicPartition =>
|
||||||
(topicPartition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.GROUP_LOAD_IN_PROGRESS.code))}.toMap
|
(topicPartition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.GROUP_LOAD_IN_PROGRESS.code))}.toMap
|
||||||
} else {
|
} else {
|
||||||
// return offsets blindly regardless the current group state since the group may be using
|
// return offsets blindly regardless the current group state since the group may be using
|
||||||
|
|
|
||||||
|
|
@ -17,38 +17,31 @@
|
||||||
|
|
||||||
package kafka.coordinator
|
package kafka.coordinator
|
||||||
|
|
||||||
import org.apache.kafka.common.TopicPartition
|
|
||||||
import org.apache.kafka.common.protocol.Errors
|
|
||||||
import org.apache.kafka.common.protocol.types.{ArrayOf, Field, Schema, Struct}
|
|
||||||
import org.apache.kafka.common.protocol.types.Type.STRING
|
|
||||||
import org.apache.kafka.common.protocol.types.Type.NULLABLE_STRING
|
|
||||||
import org.apache.kafka.common.protocol.types.Type.INT32
|
|
||||||
import org.apache.kafka.common.protocol.types.Type.INT64
|
|
||||||
import org.apache.kafka.common.protocol.types.Type.BYTES
|
|
||||||
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
|
|
||||||
import org.apache.kafka.common.requests.OffsetFetchResponse
|
|
||||||
import org.apache.kafka.common.utils.Utils
|
|
||||||
import org.apache.kafka.common.utils.Time
|
|
||||||
import org.apache.kafka.clients.consumer.ConsumerRecord
|
|
||||||
import kafka.utils._
|
|
||||||
import kafka.common._
|
|
||||||
import kafka.message._
|
|
||||||
import kafka.log.FileMessageSet
|
|
||||||
import kafka.metrics.KafkaMetricsGroup
|
|
||||||
import kafka.common.TopicAndPartition
|
|
||||||
import kafka.common.MessageFormatter
|
|
||||||
import kafka.server.ReplicaManager
|
|
||||||
|
|
||||||
import scala.collection._
|
|
||||||
import java.io.PrintStream
|
import java.io.PrintStream
|
||||||
import java.nio.ByteBuffer
|
import java.nio.ByteBuffer
|
||||||
import java.util.concurrent.atomic.AtomicBoolean
|
|
||||||
import java.util.concurrent.TimeUnit
|
import java.util.concurrent.TimeUnit
|
||||||
|
import java.util.concurrent.atomic.AtomicBoolean
|
||||||
import java.util.concurrent.locks.ReentrantLock
|
import java.util.concurrent.locks.ReentrantLock
|
||||||
|
|
||||||
import com.yammer.metrics.core.Gauge
|
import com.yammer.metrics.core.Gauge
|
||||||
import kafka.api.{ApiVersion, KAFKA_0_10_1_IV0}
|
import kafka.api.{ApiVersion, KAFKA_0_10_1_IV0}
|
||||||
|
import kafka.common.{MessageFormatter, TopicAndPartition, _}
|
||||||
|
import kafka.metrics.KafkaMetricsGroup
|
||||||
|
import kafka.server.ReplicaManager
|
||||||
import kafka.utils.CoreUtils.inLock
|
import kafka.utils.CoreUtils.inLock
|
||||||
|
import kafka.utils._
|
||||||
|
import org.apache.kafka.clients.consumer.ConsumerRecord
|
||||||
|
import org.apache.kafka.common.TopicPartition
|
||||||
|
import org.apache.kafka.common.protocol.Errors
|
||||||
|
import org.apache.kafka.common.protocol.types.Type._
|
||||||
|
import org.apache.kafka.common.protocol.types.{ArrayOf, Field, Schema, Struct}
|
||||||
|
import org.apache.kafka.common.record._
|
||||||
|
import org.apache.kafka.common.requests.OffsetFetchResponse
|
||||||
|
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
|
||||||
|
import org.apache.kafka.common.utils.{Time, Utils}
|
||||||
|
|
||||||
|
import scala.collection.JavaConverters._
|
||||||
|
import scala.collection._
|
||||||
|
|
||||||
class GroupMetadataManager(val brokerId: Int,
|
class GroupMetadataManager(val brokerId: Int,
|
||||||
val interBrokerProtocolVersion: ApiVersion,
|
val interBrokerProtocolVersion: ApiVersion,
|
||||||
|
|
@ -57,6 +50,8 @@ class GroupMetadataManager(val brokerId: Int,
|
||||||
zkUtils: ZkUtils,
|
zkUtils: ZkUtils,
|
||||||
time: Time) extends Logging with KafkaMetricsGroup {
|
time: Time) extends Logging with KafkaMetricsGroup {
|
||||||
|
|
||||||
|
private val compressionType: CompressionType = CompressionType.forId(config.offsetsTopicCompressionCodec.codec)
|
||||||
|
|
||||||
private val groupMetadataCache = new Pool[String, GroupMetadata]
|
private val groupMetadataCache = new Pool[String, GroupMetadata]
|
||||||
|
|
||||||
/* lock protecting access to loading and owned partition sets */
|
/* lock protecting access to loading and owned partition sets */
|
||||||
|
|
@ -135,13 +130,11 @@ class GroupMetadataManager(val brokerId: Int,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
def prepareStoreGroup(group: GroupMetadata,
|
def prepareStoreGroup(group: GroupMetadata,
|
||||||
groupAssignment: Map[String, Array[Byte]],
|
groupAssignment: Map[String, Array[Byte]],
|
||||||
responseCallback: Errors => Unit): Option[DelayedStore] = {
|
responseCallback: Errors => Unit): Option[DelayedStore] = {
|
||||||
val magicValueAndTimestampOpt = getMessageFormatVersionAndTimestamp(partitionFor(group.groupId))
|
getMagicAndTimestamp(partitionFor(group.groupId)) match {
|
||||||
magicValueAndTimestampOpt match {
|
case Some((magicValue, timestampType, timestamp)) =>
|
||||||
case Some((magicValue, timestamp)) =>
|
|
||||||
val groupMetadataValueVersion = {
|
val groupMetadataValueVersion = {
|
||||||
if (interBrokerProtocolVersion < KAFKA_0_10_1_IV0)
|
if (interBrokerProtocolVersion < KAFKA_0_10_1_IV0)
|
||||||
0.toShort
|
0.toShort
|
||||||
|
|
@ -149,17 +142,12 @@ class GroupMetadataManager(val brokerId: Int,
|
||||||
GroupMetadataManager.CURRENT_GROUP_VALUE_SCHEMA_VERSION
|
GroupMetadataManager.CURRENT_GROUP_VALUE_SCHEMA_VERSION
|
||||||
}
|
}
|
||||||
|
|
||||||
val message = new Message(
|
val record = Record.create(magicValue, timestampType, timestamp,
|
||||||
key = GroupMetadataManager.groupMetadataKey(group.groupId),
|
GroupMetadataManager.groupMetadataKey(group.groupId),
|
||||||
bytes = GroupMetadataManager.groupMetadataValue(group, groupAssignment, version = groupMetadataValueVersion),
|
GroupMetadataManager.groupMetadataValue(group, groupAssignment, version = groupMetadataValueVersion))
|
||||||
timestamp = timestamp,
|
|
||||||
magicValue = magicValue)
|
|
||||||
|
|
||||||
val groupMetadataPartition = new TopicPartition(Topic.GroupMetadataTopicName, partitionFor(group.groupId))
|
val groupMetadataPartition = new TopicPartition(Topic.GroupMetadataTopicName, partitionFor(group.groupId))
|
||||||
|
val groupMetadataRecords = Map(groupMetadataPartition -> MemoryRecords.withRecords(timestampType, compressionType, record))
|
||||||
val groupMetadataMessageSet = Map(groupMetadataPartition ->
|
|
||||||
new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, message))
|
|
||||||
|
|
||||||
val generationId = group.generationId
|
val generationId = group.generationId
|
||||||
|
|
||||||
// set the callback function to insert the created group into cache after log append completed
|
// set the callback function to insert the created group into cache after log append completed
|
||||||
|
|
@ -212,7 +200,7 @@ class GroupMetadataManager(val brokerId: Int,
|
||||||
|
|
||||||
responseCallback(responseError)
|
responseCallback(responseError)
|
||||||
}
|
}
|
||||||
Some(DelayedStore(groupMetadataMessageSet, putCacheCallback))
|
Some(DelayedStore(groupMetadataRecords, putCacheCallback))
|
||||||
|
|
||||||
case None =>
|
case None =>
|
||||||
responseCallback(Errors.NOT_COORDINATOR_FOR_GROUP)
|
responseCallback(Errors.NOT_COORDINATOR_FOR_GROUP)
|
||||||
|
|
@ -222,11 +210,11 @@ class GroupMetadataManager(val brokerId: Int,
|
||||||
|
|
||||||
def store(delayedStore: DelayedStore) {
|
def store(delayedStore: DelayedStore) {
|
||||||
// call replica manager to append the group message
|
// call replica manager to append the group message
|
||||||
replicaManager.appendMessages(
|
replicaManager.appendRecords(
|
||||||
config.offsetCommitTimeoutMs.toLong,
|
config.offsetCommitTimeoutMs.toLong,
|
||||||
config.offsetCommitRequiredAcks,
|
config.offsetCommitRequiredAcks,
|
||||||
true, // allow appending to internal offset topic
|
true, // allow appending to internal offset topic
|
||||||
delayedStore.messageSet,
|
delayedStore.partitionRecords,
|
||||||
delayedStore.callback)
|
delayedStore.callback)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -244,22 +232,17 @@ class GroupMetadataManager(val brokerId: Int,
|
||||||
}
|
}
|
||||||
|
|
||||||
// construct the message set to append
|
// construct the message set to append
|
||||||
val magicValueAndTimestampOpt = getMessageFormatVersionAndTimestamp(partitionFor(group.groupId))
|
getMagicAndTimestamp(partitionFor(group.groupId)) match {
|
||||||
magicValueAndTimestampOpt match {
|
case Some((magicValue, timestampType, timestamp)) =>
|
||||||
case Some((magicValue, timestamp)) =>
|
val records = filteredOffsetMetadata.map { case (topicAndPartition, offsetAndMetadata) =>
|
||||||
val messages = filteredOffsetMetadata.map { case (topicAndPartition, offsetAndMetadata) =>
|
Record.create(magicValue, timestampType, timestamp,
|
||||||
new Message(
|
GroupMetadataManager.offsetCommitKey(group.groupId, topicAndPartition.topic, topicAndPartition.partition),
|
||||||
key = GroupMetadataManager.offsetCommitKey(group.groupId, topicAndPartition.topic, topicAndPartition.partition),
|
GroupMetadataManager.offsetCommitValue(offsetAndMetadata))
|
||||||
bytes = GroupMetadataManager.offsetCommitValue(offsetAndMetadata),
|
|
||||||
timestamp = timestamp,
|
|
||||||
magicValue = magicValue
|
|
||||||
)
|
|
||||||
}.toSeq
|
}.toSeq
|
||||||
|
|
||||||
val offsetTopicPartition = new TopicPartition(Topic.GroupMetadataTopicName, partitionFor(group.groupId))
|
val offsetTopicPartition = new TopicPartition(Topic.GroupMetadataTopicName, partitionFor(group.groupId))
|
||||||
|
|
||||||
val offsetsAndMetadataMessageSet = Map(offsetTopicPartition ->
|
val entries = Map(offsetTopicPartition -> MemoryRecords.withRecords(timestampType, compressionType, records:_*))
|
||||||
new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages:_*))
|
|
||||||
|
|
||||||
// set the callback function to insert offsets into cache after log append completed
|
// set the callback function to insert offsets into cache after log append completed
|
||||||
def putCacheCallback(responseStatus: Map[TopicPartition, PartitionResponse]) {
|
def putCacheCallback(responseStatus: Map[TopicPartition, PartitionResponse]) {
|
||||||
|
|
@ -330,7 +313,7 @@ class GroupMetadataManager(val brokerId: Int,
|
||||||
group.prepareOffsetCommit(offsetMetadata)
|
group.prepareOffsetCommit(offsetMetadata)
|
||||||
}
|
}
|
||||||
|
|
||||||
Some(DelayedStore(offsetsAndMetadataMessageSet, putCacheCallback))
|
Some(DelayedStore(entries, putCacheCallback))
|
||||||
|
|
||||||
case None =>
|
case None =>
|
||||||
val commitStatus = offsetMetadata.map { case (topicAndPartition, offsetAndMetadata) =>
|
val commitStatus = offsetMetadata.map { case (topicAndPartition, offsetAndMetadata) =>
|
||||||
|
|
@ -412,28 +395,30 @@ class GroupMetadataManager(val brokerId: Int,
|
||||||
|
|
||||||
while (currOffset < getHighWatermark(offsetsPartition) && !shuttingDown.get()) {
|
while (currOffset < getHighWatermark(offsetsPartition) && !shuttingDown.get()) {
|
||||||
buffer.clear()
|
buffer.clear()
|
||||||
val messages = log.read(currOffset, config.loadBufferSize, minOneMessage = true).messageSet.asInstanceOf[FileMessageSet]
|
val fileRecords = log.read(currOffset, config.loadBufferSize, minOneMessage = true).records.asInstanceOf[FileRecords]
|
||||||
messages.readInto(buffer, 0)
|
fileRecords.readInto(buffer, 0)
|
||||||
val messageSet = new ByteBufferMessageSet(buffer)
|
|
||||||
messageSet.foreach { msgAndOffset =>
|
MemoryRecords.readableRecords(buffer).deepIterator.asScala.foreach { entry =>
|
||||||
require(msgAndOffset.message.key != null, "Offset entry key should not be null")
|
val record = entry.record
|
||||||
val baseKey = GroupMetadataManager.readMessageKey(msgAndOffset.message.key)
|
|
||||||
|
require(record.hasKey, "Offset entry key should not be null")
|
||||||
|
val baseKey = GroupMetadataManager.readMessageKey(record.key)
|
||||||
|
|
||||||
if (baseKey.isInstanceOf[OffsetKey]) {
|
if (baseKey.isInstanceOf[OffsetKey]) {
|
||||||
// load offset
|
// load offset
|
||||||
val key = baseKey.key.asInstanceOf[GroupTopicPartition]
|
val key = baseKey.key.asInstanceOf[GroupTopicPartition]
|
||||||
if (msgAndOffset.message.payload == null) {
|
if (record.hasNullValue) {
|
||||||
loadedOffsets.remove(key)
|
loadedOffsets.remove(key)
|
||||||
removedOffsets.add(key)
|
removedOffsets.add(key)
|
||||||
} else {
|
} else {
|
||||||
val value = GroupMetadataManager.readOffsetMessageValue(msgAndOffset.message.payload)
|
val value = GroupMetadataManager.readOffsetMessageValue(record.value)
|
||||||
loadedOffsets.put(key, value)
|
loadedOffsets.put(key, value)
|
||||||
removedOffsets.remove(key)
|
removedOffsets.remove(key)
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
// load group metadata
|
// load group metadata
|
||||||
val groupId = baseKey.key.asInstanceOf[String]
|
val groupId = baseKey.key.asInstanceOf[String]
|
||||||
val groupMetadata = GroupMetadataManager.readGroupMessageValue(groupId, msgAndOffset.message.payload)
|
val groupMetadata = GroupMetadataManager.readGroupMessageValue(groupId, record.value)
|
||||||
if (groupMetadata != null) {
|
if (groupMetadata != null) {
|
||||||
trace(s"Loaded group metadata for group ${groupMetadata.groupId} with generation ${groupMetadata.generationId}")
|
trace(s"Loaded group metadata for group ${groupMetadata.groupId} with generation ${groupMetadata.generationId}")
|
||||||
removedGroups.remove(groupId)
|
removedGroups.remove(groupId)
|
||||||
|
|
@ -444,7 +429,7 @@ class GroupMetadataManager(val brokerId: Int,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
currOffset = msgAndOffset.nextOffset
|
currOffset = entry.nextOffset
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -467,8 +452,8 @@ class GroupMetadataManager(val brokerId: Int,
|
||||||
|
|
||||||
removedGroups.foreach { groupId =>
|
removedGroups.foreach { groupId =>
|
||||||
if (groupMetadataCache.contains(groupId))
|
if (groupMetadataCache.contains(groupId))
|
||||||
throw new IllegalStateException(s"Unexpected unload of active group ${groupId} while " +
|
throw new IllegalStateException(s"Unexpected unload of active group $groupId while " +
|
||||||
s"loading partition ${topicPartition}")
|
s"loading partition $topicPartition")
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!shuttingDown.get())
|
if (!shuttingDown.get())
|
||||||
|
|
@ -572,15 +557,15 @@ class GroupMetadataManager(val brokerId: Int,
|
||||||
}
|
}
|
||||||
|
|
||||||
val offsetsPartition = partitionFor(groupId)
|
val offsetsPartition = partitionFor(groupId)
|
||||||
getMessageFormatVersionAndTimestamp(offsetsPartition) match {
|
getMagicAndTimestamp(offsetsPartition) match {
|
||||||
case Some((magicValue, timestamp)) =>
|
case Some((magicValue, timestampType, timestamp)) =>
|
||||||
val partitionOpt = replicaManager.getPartition(Topic.GroupMetadataTopicName, offsetsPartition)
|
val partitionOpt = replicaManager.getPartition(Topic.GroupMetadataTopicName, offsetsPartition)
|
||||||
partitionOpt.foreach { partition =>
|
partitionOpt.foreach { partition =>
|
||||||
val appendPartition = TopicAndPartition(Topic.GroupMetadataTopicName, offsetsPartition)
|
val appendPartition = TopicAndPartition(Topic.GroupMetadataTopicName, offsetsPartition)
|
||||||
val tombstones = expiredOffsets.map { case (topicPartition, offsetAndMetadata) =>
|
val tombstones = expiredOffsets.map { case (topicPartition, offsetAndMetadata) =>
|
||||||
trace(s"Removing expired offset and metadata for $groupId, $topicPartition: $offsetAndMetadata")
|
trace(s"Removing expired offset and metadata for $groupId, $topicPartition: $offsetAndMetadata")
|
||||||
val commitKey = GroupMetadataManager.offsetCommitKey(groupId, topicPartition.topic, topicPartition.partition)
|
val commitKey = GroupMetadataManager.offsetCommitKey(groupId, topicPartition.topic, topicPartition.partition)
|
||||||
new Message(bytes = null, key = commitKey, timestamp = timestamp, magicValue = magicValue)
|
Record.create(magicValue, timestampType, timestamp, commitKey, null)
|
||||||
}.toBuffer
|
}.toBuffer
|
||||||
trace(s"Marked ${expiredOffsets.size} offsets in $appendPartition for deletion.")
|
trace(s"Marked ${expiredOffsets.size} offsets in $appendPartition for deletion.")
|
||||||
|
|
||||||
|
|
@ -590,8 +575,7 @@ class GroupMetadataManager(val brokerId: Int,
|
||||||
// Append the tombstone messages to the partition. It is okay if the replicas don't receive these (say,
|
// Append the tombstone messages to the partition. It is okay if the replicas don't receive these (say,
|
||||||
// if we crash or leaders move) since the new leaders will still expire the consumers with heartbeat and
|
// if we crash or leaders move) since the new leaders will still expire the consumers with heartbeat and
|
||||||
// retry removing this group.
|
// retry removing this group.
|
||||||
tombstones += new Message(bytes = null, key = GroupMetadataManager.groupMetadataKey(group.groupId),
|
tombstones += Record.create(magicValue, timestampType, timestamp, GroupMetadataManager.groupMetadataKey(group.groupId), null)
|
||||||
timestamp = timestamp, magicValue = magicValue)
|
|
||||||
trace(s"Group $groupId removed from the metadata cache and marked for deletion in $appendPartition.")
|
trace(s"Group $groupId removed from the metadata cache and marked for deletion in $appendPartition.")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -599,7 +583,7 @@ class GroupMetadataManager(val brokerId: Int,
|
||||||
try {
|
try {
|
||||||
// do not need to require acks since even if the tombstone is lost,
|
// do not need to require acks since even if the tombstone is lost,
|
||||||
// it will be appended again in the next purge cycle
|
// it will be appended again in the next purge cycle
|
||||||
partition.appendMessagesToLeader(new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, tombstones: _*))
|
partition.appendRecordsToLeader(MemoryRecords.withRecords(timestampType, compressionType, tombstones: _*))
|
||||||
offsetsRemoved += expiredOffsets.size
|
offsetsRemoved += expiredOffsets.size
|
||||||
trace(s"Successfully appended ${tombstones.size} tombstones to $appendPartition for expired offsets and/or metadata for group $groupId")
|
trace(s"Successfully appended ${tombstones.size} tombstones to $appendPartition for expired offsets and/or metadata for group $groupId")
|
||||||
} catch {
|
} catch {
|
||||||
|
|
@ -663,16 +647,11 @@ class GroupMetadataManager(val brokerId: Int,
|
||||||
* @param partition Partition of GroupMetadataTopic
|
* @param partition Partition of GroupMetadataTopic
|
||||||
* @return Option[(MessageFormatVersion, TimeStamp)] if replica is local, None otherwise
|
* @return Option[(MessageFormatVersion, TimeStamp)] if replica is local, None otherwise
|
||||||
*/
|
*/
|
||||||
private def getMessageFormatVersionAndTimestamp(partition: Int): Option[(Byte, Long)] = {
|
private def getMagicAndTimestamp(partition: Int): Option[(Byte, TimestampType, Long)] = {
|
||||||
val groupMetadataTopicAndPartition = TopicAndPartition(Topic.GroupMetadataTopicName, partition)
|
val groupMetadataTopicAndPartition = TopicAndPartition(Topic.GroupMetadataTopicName, partition)
|
||||||
replicaManager.getMessageFormatVersion(groupMetadataTopicAndPartition).map { messageFormatVersion =>
|
replicaManager.getMagicAndTimestampType(groupMetadataTopicAndPartition).map { case (messageFormatVersion, timestampType) =>
|
||||||
val timestamp = {
|
val timestamp = if (messageFormatVersion == Record.MAGIC_VALUE_V0) Record.NO_TIMESTAMP else time.milliseconds()
|
||||||
if (messageFormatVersion == Message.MagicValue_V0)
|
(messageFormatVersion, timestampType, timestamp)
|
||||||
Message.NoTimestamp
|
|
||||||
else
|
|
||||||
time.milliseconds()
|
|
||||||
}
|
|
||||||
(messageFormatVersion, timestamp)
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -964,7 +943,7 @@ object GroupMetadataManager {
|
||||||
* @return an offset-metadata object from the message
|
* @return an offset-metadata object from the message
|
||||||
*/
|
*/
|
||||||
def readOffsetMessageValue(buffer: ByteBuffer): OffsetAndMetadata = {
|
def readOffsetMessageValue(buffer: ByteBuffer): OffsetAndMetadata = {
|
||||||
if(buffer == null) { // tombstone
|
if (buffer == null) { // tombstone
|
||||||
null
|
null
|
||||||
} else {
|
} else {
|
||||||
val version = buffer.getShort
|
val version = buffer.getShort
|
||||||
|
|
@ -997,7 +976,7 @@ object GroupMetadataManager {
|
||||||
* @return a group metadata object from the message
|
* @return a group metadata object from the message
|
||||||
*/
|
*/
|
||||||
def readGroupMessageValue(groupId: String, buffer: ByteBuffer): GroupMetadata = {
|
def readGroupMessageValue(groupId: String, buffer: ByteBuffer): GroupMetadata = {
|
||||||
if(buffer == null) { // tombstone
|
if (buffer == null) { // tombstone
|
||||||
null
|
null
|
||||||
} else {
|
} else {
|
||||||
val version = buffer.getShort
|
val version = buffer.getShort
|
||||||
|
|
@ -1016,23 +995,22 @@ object GroupMetadataManager {
|
||||||
group.leaderId = value.get(LEADER_KEY).asInstanceOf[String]
|
group.leaderId = value.get(LEADER_KEY).asInstanceOf[String]
|
||||||
group.protocol = value.get(PROTOCOL_KEY).asInstanceOf[String]
|
group.protocol = value.get(PROTOCOL_KEY).asInstanceOf[String]
|
||||||
|
|
||||||
memberMetadataArray.foreach {
|
memberMetadataArray.foreach { memberMetadataObj =>
|
||||||
case memberMetadataObj =>
|
val memberMetadata = memberMetadataObj.asInstanceOf[Struct]
|
||||||
val memberMetadata = memberMetadataObj.asInstanceOf[Struct]
|
val memberId = memberMetadata.get(MEMBER_ID_KEY).asInstanceOf[String]
|
||||||
val memberId = memberMetadata.get(MEMBER_ID_KEY).asInstanceOf[String]
|
val clientId = memberMetadata.get(CLIENT_ID_KEY).asInstanceOf[String]
|
||||||
val clientId = memberMetadata.get(CLIENT_ID_KEY).asInstanceOf[String]
|
val clientHost = memberMetadata.get(CLIENT_HOST_KEY).asInstanceOf[String]
|
||||||
val clientHost = memberMetadata.get(CLIENT_HOST_KEY).asInstanceOf[String]
|
val sessionTimeout = memberMetadata.get(SESSION_TIMEOUT_KEY).asInstanceOf[Int]
|
||||||
val sessionTimeout = memberMetadata.get(SESSION_TIMEOUT_KEY).asInstanceOf[Int]
|
val rebalanceTimeout = if (version == 0) sessionTimeout else memberMetadata.get(REBALANCE_TIMEOUT_KEY).asInstanceOf[Int]
|
||||||
val rebalanceTimeout = if (version == 0) sessionTimeout else memberMetadata.get(REBALANCE_TIMEOUT_KEY).asInstanceOf[Int]
|
|
||||||
|
|
||||||
val subscription = Utils.toArray(memberMetadata.get(SUBSCRIPTION_KEY).asInstanceOf[ByteBuffer])
|
val subscription = Utils.toArray(memberMetadata.get(SUBSCRIPTION_KEY).asInstanceOf[ByteBuffer])
|
||||||
|
|
||||||
val member = new MemberMetadata(memberId, groupId, clientId, clientHost, rebalanceTimeout, sessionTimeout,
|
val member = new MemberMetadata(memberId, groupId, clientId, clientHost, rebalanceTimeout, sessionTimeout,
|
||||||
protocolType, List((group.protocol, subscription)))
|
protocolType, List((group.protocol, subscription)))
|
||||||
|
|
||||||
member.assignment = Utils.toArray(memberMetadata.get(ASSIGNMENT_KEY).asInstanceOf[ByteBuffer])
|
member.assignment = Utils.toArray(memberMetadata.get(ASSIGNMENT_KEY).asInstanceOf[ByteBuffer])
|
||||||
|
|
||||||
group.add(memberId, member)
|
group.add(memberId, member)
|
||||||
}
|
}
|
||||||
|
|
||||||
group
|
group
|
||||||
|
|
@ -1087,7 +1065,7 @@ object GroupMetadataManager {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
case class DelayedStore(messageSet: Map[TopicPartition, MessageSet],
|
case class DelayedStore(partitionRecords: Map[TopicPartition, MemoryRecords],
|
||||||
callback: Map[TopicPartition, PartitionResponse] => Unit)
|
callback: Map[TopicPartition, PartitionResponse] => Unit)
|
||||||
|
|
||||||
case class GroupTopicPartition(group: String, topicPartition: TopicPartition) {
|
case class GroupTopicPartition(group: String, topicPartition: TopicPartition) {
|
||||||
|
|
|
||||||
|
|
@ -1,445 +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 kafka.log
|
|
||||||
|
|
||||||
import java.io._
|
|
||||||
import java.nio._
|
|
||||||
import java.nio.channels._
|
|
||||||
import java.util.concurrent.atomic._
|
|
||||||
import java.util.concurrent.TimeUnit
|
|
||||||
|
|
||||||
import kafka.utils._
|
|
||||||
import kafka.message._
|
|
||||||
import kafka.common.KafkaException
|
|
||||||
import kafka.metrics.{KafkaMetricsGroup, KafkaTimer}
|
|
||||||
import org.apache.kafka.common.errors.CorruptRecordException
|
|
||||||
import org.apache.kafka.common.record.FileRecords
|
|
||||||
import org.apache.kafka.common.utils.Utils
|
|
||||||
|
|
||||||
import scala.collection.mutable.ArrayBuffer
|
|
||||||
|
|
||||||
/**
|
|
||||||
* An on-disk message set. An optional start and end position can be applied to the message set
|
|
||||||
* which will allow slicing a subset of the file.
|
|
||||||
* @param file The file name for the underlying log data
|
|
||||||
* @param channel the underlying file channel used
|
|
||||||
* @param start A lower bound on the absolute position in the file from which the message set begins
|
|
||||||
* @param end The upper bound on the absolute position in the file at which the message set ends
|
|
||||||
* @param isSlice Should the start and end parameters be used for slicing?
|
|
||||||
*/
|
|
||||||
@nonthreadsafe
|
|
||||||
class FileMessageSet private[kafka](@volatile var file: File,
|
|
||||||
private[log] val channel: FileChannel,
|
|
||||||
private[log] val start: Int,
|
|
||||||
private[log] val end: Int,
|
|
||||||
isSlice: Boolean) extends MessageSet {
|
|
||||||
/* the size of the message set in bytes */
|
|
||||||
private val _size =
|
|
||||||
if(isSlice)
|
|
||||||
new AtomicInteger(end - start) // don't check the file size if this is just a slice view
|
|
||||||
else
|
|
||||||
new AtomicInteger(math.min(channel.size.toInt, end) - start)
|
|
||||||
|
|
||||||
/* if this is not a slice, update the file pointer to the end of the file */
|
|
||||||
if (!isSlice)
|
|
||||||
/* set the file position to the last byte in the file */
|
|
||||||
channel.position(math.min(channel.size.toInt, end))
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Create a file message set with no slicing.
|
|
||||||
*/
|
|
||||||
def this(file: File, channel: FileChannel) =
|
|
||||||
this(file, channel, start = 0, end = Int.MaxValue, isSlice = false)
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Create a file message set with no slicing
|
|
||||||
*/
|
|
||||||
def this(file: File) =
|
|
||||||
this(file, FileMessageSet.openChannel(file, mutable = true))
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Create a file message set with no slicing, and with initFileSize and preallocate.
|
|
||||||
* For windows NTFS and some old LINUX file system, set preallocate to true and initFileSize
|
|
||||||
* with one value (for example 512 * 1024 *1024 ) can improve the kafka produce performance.
|
|
||||||
* If it's new file and preallocate is true, end will be set to 0. Otherwise set to Int.MaxValue.
|
|
||||||
*/
|
|
||||||
def this(file: File, fileAlreadyExists: Boolean, initFileSize: Int, preallocate: Boolean) =
|
|
||||||
this(file,
|
|
||||||
channel = FileMessageSet.openChannel(file, mutable = true, fileAlreadyExists, initFileSize, preallocate),
|
|
||||||
start = 0,
|
|
||||||
end = if (!fileAlreadyExists && preallocate) 0 else Int.MaxValue,
|
|
||||||
isSlice = false)
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Create a file message set with mutable option
|
|
||||||
*/
|
|
||||||
def this(file: File, mutable: Boolean) = this(file, FileMessageSet.openChannel(file, mutable))
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Create a slice view of the file message set that begins and ends at the given byte offsets
|
|
||||||
*/
|
|
||||||
def this(file: File, channel: FileChannel, start: Int, end: Int) =
|
|
||||||
this(file, channel, start, end, isSlice = true)
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Return a message set which is a view into this set starting from the given position and with the given size limit.
|
|
||||||
*
|
|
||||||
* If the size is beyond the end of the file, the end will be based on the size of the file at the time of the read.
|
|
||||||
*
|
|
||||||
* If this message set is already sliced, the position will be taken relative to that slicing.
|
|
||||||
*
|
|
||||||
* @param position The start position to begin the read from
|
|
||||||
* @param size The number of bytes after the start position to include
|
|
||||||
*
|
|
||||||
* @return A sliced wrapper on this message set limited based on the given position and size
|
|
||||||
*/
|
|
||||||
def read(position: Int, size: Int): FileMessageSet = {
|
|
||||||
if(position < 0)
|
|
||||||
throw new IllegalArgumentException("Invalid position: " + position)
|
|
||||||
if(size < 0)
|
|
||||||
throw new IllegalArgumentException("Invalid size: " + size)
|
|
||||||
new FileMessageSet(file,
|
|
||||||
channel,
|
|
||||||
start = this.start + position,
|
|
||||||
end = {
|
|
||||||
// Handle the integer overflow
|
|
||||||
if (this.start + position + size < 0)
|
|
||||||
sizeInBytes()
|
|
||||||
else
|
|
||||||
math.min(this.start + position + size, sizeInBytes())
|
|
||||||
})
|
|
||||||
}
|
|
||||||
|
|
||||||
override def asRecords: FileRecords = new FileRecords(file, channel, start, end, isSlice)
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Search forward for the file position of the last offset that is greater than or equal to the target offset
|
|
||||||
* and return its physical position and the size of the message (including log overhead) at the returned offset. If
|
|
||||||
* no such offsets are found, return null.
|
|
||||||
*
|
|
||||||
* @param targetOffset The offset to search for.
|
|
||||||
* @param startingPosition The starting position in the file to begin searching from.
|
|
||||||
*/
|
|
||||||
def searchForOffsetWithSize(targetOffset: Long, startingPosition: Int): (OffsetPosition, Int) = {
|
|
||||||
var position = startingPosition
|
|
||||||
val buffer = ByteBuffer.allocate(MessageSet.LogOverhead)
|
|
||||||
val size = sizeInBytes()
|
|
||||||
while(position + MessageSet.LogOverhead < size) {
|
|
||||||
buffer.rewind()
|
|
||||||
channel.read(buffer, position)
|
|
||||||
if(buffer.hasRemaining)
|
|
||||||
throw new IllegalStateException("Failed to read complete buffer for targetOffset %d startPosition %d in %s"
|
|
||||||
.format(targetOffset, startingPosition, file.getAbsolutePath))
|
|
||||||
buffer.rewind()
|
|
||||||
val offset = buffer.getLong()
|
|
||||||
val messageSize = buffer.getInt()
|
|
||||||
if (messageSize < Message.MinMessageOverhead)
|
|
||||||
throw new IllegalStateException("Invalid message size: " + messageSize)
|
|
||||||
if (offset >= targetOffset)
|
|
||||||
return (OffsetPosition(offset, position), messageSize + MessageSet.LogOverhead)
|
|
||||||
position += MessageSet.LogOverhead + messageSize
|
|
||||||
}
|
|
||||||
null
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Search forward for the message whose timestamp is greater than or equals to the target timestamp.
|
|
||||||
*
|
|
||||||
* @param targetTimestamp The timestamp to search for.
|
|
||||||
* @param startingPosition The starting position to search.
|
|
||||||
* @return The timestamp and offset of the message found. None, if no message is found.
|
|
||||||
*/
|
|
||||||
def searchForTimestamp(targetTimestamp: Long, startingPosition: Int): Option[TimestampOffset] = {
|
|
||||||
val messagesToSearch = read(startingPosition, sizeInBytes)
|
|
||||||
for (messageAndOffset <- messagesToSearch) {
|
|
||||||
val message = messageAndOffset.message
|
|
||||||
if (message.timestamp >= targetTimestamp) {
|
|
||||||
// We found a message
|
|
||||||
message.compressionCodec match {
|
|
||||||
case NoCompressionCodec =>
|
|
||||||
return Some(TimestampOffset(messageAndOffset.message.timestamp, messageAndOffset.offset))
|
|
||||||
case _ =>
|
|
||||||
// Iterate over the inner messages to get the exact offset.
|
|
||||||
for (innerMessageAndOffset <- ByteBufferMessageSet.deepIterator(messageAndOffset)) {
|
|
||||||
val timestamp = innerMessageAndOffset.message.timestamp
|
|
||||||
if (timestamp >= targetTimestamp)
|
|
||||||
return Some(TimestampOffset(innerMessageAndOffset.message.timestamp, innerMessageAndOffset.offset))
|
|
||||||
}
|
|
||||||
throw new IllegalStateException(s"The message set (max timestamp = ${message.timestamp}, max offset = ${messageAndOffset.offset}" +
|
|
||||||
s" should contain target timestamp $targetTimestamp but it does not.")
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
None
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Return the largest timestamp of the messages after a given position in this file message set.
|
|
||||||
* @param startingPosition The starting position.
|
|
||||||
* @return The largest timestamp of the messages after the given position.
|
|
||||||
*/
|
|
||||||
def largestTimestampAfter(startingPosition: Int): TimestampOffset = {
|
|
||||||
var maxTimestamp = Message.NoTimestamp
|
|
||||||
var offsetOfMaxTimestamp = -1L
|
|
||||||
val messagesToSearch = read(startingPosition, Int.MaxValue)
|
|
||||||
for (messageAndOffset <- messagesToSearch) {
|
|
||||||
if (messageAndOffset.message.timestamp > maxTimestamp) {
|
|
||||||
maxTimestamp = messageAndOffset.message.timestamp
|
|
||||||
offsetOfMaxTimestamp = messageAndOffset.offset
|
|
||||||
}
|
|
||||||
}
|
|
||||||
TimestampOffset(maxTimestamp, offsetOfMaxTimestamp)
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* This method is called before we write messages to the socket using zero-copy transfer. We need to
|
|
||||||
* make sure all the messages in the message set have the expected magic value.
|
|
||||||
*
|
|
||||||
* @param expectedMagicValue the magic value expected
|
|
||||||
* @return true if all messages have expected magic value, false otherwise
|
|
||||||
*/
|
|
||||||
override def isMagicValueInAllWrapperMessages(expectedMagicValue: Byte): Boolean = {
|
|
||||||
var location = start
|
|
||||||
val offsetAndSizeBuffer = ByteBuffer.allocate(MessageSet.LogOverhead)
|
|
||||||
val crcAndMagicByteBuffer = ByteBuffer.allocate(Message.CrcLength + Message.MagicLength)
|
|
||||||
while (location < end) {
|
|
||||||
offsetAndSizeBuffer.rewind()
|
|
||||||
channel.read(offsetAndSizeBuffer, location)
|
|
||||||
if (offsetAndSizeBuffer.hasRemaining)
|
|
||||||
return true
|
|
||||||
offsetAndSizeBuffer.rewind()
|
|
||||||
offsetAndSizeBuffer.getLong // skip offset field
|
|
||||||
val messageSize = offsetAndSizeBuffer.getInt
|
|
||||||
if (messageSize < Message.MinMessageOverhead)
|
|
||||||
throw new IllegalStateException("Invalid message size: " + messageSize)
|
|
||||||
crcAndMagicByteBuffer.rewind()
|
|
||||||
channel.read(crcAndMagicByteBuffer, location + MessageSet.LogOverhead)
|
|
||||||
if (crcAndMagicByteBuffer.get(Message.MagicOffset) != expectedMagicValue)
|
|
||||||
return false
|
|
||||||
location += (MessageSet.LogOverhead + messageSize)
|
|
||||||
}
|
|
||||||
true
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Convert this message set to use the specified message format.
|
|
||||||
*/
|
|
||||||
def toMessageFormat(toMagicValue: Byte): MessageSet = {
|
|
||||||
val offsets = new ArrayBuffer[Long]
|
|
||||||
val newMessages = new ArrayBuffer[Message]
|
|
||||||
this.foreach { messageAndOffset =>
|
|
||||||
val message = messageAndOffset.message
|
|
||||||
if (message.compressionCodec == NoCompressionCodec) {
|
|
||||||
newMessages += message.toFormatVersion(toMagicValue)
|
|
||||||
offsets += messageAndOffset.offset
|
|
||||||
} else {
|
|
||||||
// File message set only has shallow iterator. We need to do deep iteration here if needed.
|
|
||||||
val deepIter = ByteBufferMessageSet.deepIterator(messageAndOffset)
|
|
||||||
for (innerMessageAndOffset <- deepIter) {
|
|
||||||
newMessages += innerMessageAndOffset.message.toFormatVersion(toMagicValue)
|
|
||||||
offsets += innerMessageAndOffset.offset
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (sizeInBytes > 0 && newMessages.isEmpty) {
|
|
||||||
// This indicates that the message is too large. We just return all the bytes in the file message set.
|
|
||||||
this
|
|
||||||
} else {
|
|
||||||
// We use the offset seq to assign offsets so the offset of the messages does not change.
|
|
||||||
new ByteBufferMessageSet(
|
|
||||||
compressionCodec = this.headOption.map(_.message.compressionCodec).getOrElse(NoCompressionCodec),
|
|
||||||
offsetSeq = offsets,
|
|
||||||
newMessages: _*)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Get a shallow iterator over the messages in the set.
|
|
||||||
*/
|
|
||||||
override def iterator: Iterator[MessageAndOffset] = iterator(Int.MaxValue)
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Get an iterator over the messages in the set. We only do shallow iteration here.
|
|
||||||
* @param maxMessageSize A limit on allowable message size to avoid allocating unbounded memory.
|
|
||||||
* If we encounter a message larger than this we throw an InvalidMessageException.
|
|
||||||
* @return The iterator.
|
|
||||||
*/
|
|
||||||
def iterator(maxMessageSize: Int): Iterator[MessageAndOffset] = {
|
|
||||||
new IteratorTemplate[MessageAndOffset] {
|
|
||||||
var location = start
|
|
||||||
val sizeOffsetLength = 12
|
|
||||||
val sizeOffsetBuffer = ByteBuffer.allocate(sizeOffsetLength)
|
|
||||||
|
|
||||||
override def makeNext(): MessageAndOffset = {
|
|
||||||
if(location + sizeOffsetLength >= end)
|
|
||||||
return allDone()
|
|
||||||
|
|
||||||
// read the size of the item
|
|
||||||
sizeOffsetBuffer.rewind()
|
|
||||||
channel.read(sizeOffsetBuffer, location)
|
|
||||||
if(sizeOffsetBuffer.hasRemaining)
|
|
||||||
return allDone()
|
|
||||||
|
|
||||||
sizeOffsetBuffer.rewind()
|
|
||||||
val offset = sizeOffsetBuffer.getLong()
|
|
||||||
val size = sizeOffsetBuffer.getInt()
|
|
||||||
if(size < Message.MinMessageOverhead || location + sizeOffsetLength + size > end)
|
|
||||||
return allDone()
|
|
||||||
if(size > maxMessageSize)
|
|
||||||
throw new CorruptRecordException("Message size exceeds the largest allowable message size (%d).".format(maxMessageSize))
|
|
||||||
|
|
||||||
// read the item itself
|
|
||||||
val buffer = ByteBuffer.allocate(size)
|
|
||||||
channel.read(buffer, location + sizeOffsetLength)
|
|
||||||
if(buffer.hasRemaining)
|
|
||||||
return allDone()
|
|
||||||
buffer.rewind()
|
|
||||||
|
|
||||||
// increment the location and return the item
|
|
||||||
location += size + sizeOffsetLength
|
|
||||||
MessageAndOffset(new Message(buffer), offset)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The number of bytes taken up by this file set
|
|
||||||
*/
|
|
||||||
def sizeInBytes(): Int = _size.get()
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Append these messages to the message set
|
|
||||||
*/
|
|
||||||
def append(messages: ByteBufferMessageSet) {
|
|
||||||
val written = messages.writeFullyTo(channel)
|
|
||||||
_size.getAndAdd(written)
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Commit all written data to the physical disk
|
|
||||||
*/
|
|
||||||
def flush() = {
|
|
||||||
channel.force(true)
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Close this message set
|
|
||||||
*/
|
|
||||||
def close() {
|
|
||||||
flush()
|
|
||||||
trim()
|
|
||||||
channel.close()
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Trim file when close or roll to next file
|
|
||||||
*/
|
|
||||||
def trim() {
|
|
||||||
truncateTo(sizeInBytes())
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Delete this message set from the filesystem
|
|
||||||
* @return True iff this message set was deleted.
|
|
||||||
*/
|
|
||||||
def delete(): Boolean = {
|
|
||||||
CoreUtils.swallow(channel.close())
|
|
||||||
file.delete()
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Truncate this file message set to the given size in bytes. Note that this API does no checking that the
|
|
||||||
* given size falls on a valid message boundary.
|
|
||||||
* In some versions of the JDK truncating to the same size as the file message set will cause an
|
|
||||||
* update of the files mtime, so truncate is only performed if the targetSize is smaller than the
|
|
||||||
* size of the underlying FileChannel.
|
|
||||||
* It is expected that no other threads will do writes to the log when this function is called.
|
|
||||||
* @param targetSize The size to truncate to. Must be between 0 and sizeInBytes.
|
|
||||||
* @return The number of bytes truncated off
|
|
||||||
*/
|
|
||||||
def truncateTo(targetSize: Int): Int = {
|
|
||||||
val originalSize = sizeInBytes
|
|
||||||
if(targetSize > originalSize || targetSize < 0)
|
|
||||||
throw new KafkaException("Attempt to truncate log segment to " + targetSize + " bytes failed, " +
|
|
||||||
" size of this log segment is " + originalSize + " bytes.")
|
|
||||||
if (targetSize < channel.size.toInt) {
|
|
||||||
channel.truncate(targetSize)
|
|
||||||
channel.position(targetSize)
|
|
||||||
_size.set(targetSize)
|
|
||||||
}
|
|
||||||
originalSize - targetSize
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Read from the underlying file into the buffer starting at the given position
|
|
||||||
*/
|
|
||||||
def readInto(buffer: ByteBuffer, relativePosition: Int): ByteBuffer = {
|
|
||||||
channel.read(buffer, relativePosition + this.start)
|
|
||||||
buffer.flip()
|
|
||||||
buffer
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Rename the file that backs this message set
|
|
||||||
* @throws IOException if rename fails.
|
|
||||||
*/
|
|
||||||
def renameTo(f: File) {
|
|
||||||
try Utils.atomicMoveWithFallback(file.toPath, f.toPath)
|
|
||||||
finally this.file = f
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
object FileMessageSet extends Logging
|
|
||||||
{
|
|
||||||
//preserve the previous logger name after moving logger aspect from FileMessageSet to companion
|
|
||||||
override val loggerName = classOf[FileMessageSet].getName
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Open a channel for the given file
|
|
||||||
* For windows NTFS and some old LINUX file system, set preallocate to true and initFileSize
|
|
||||||
* with one value (for example 512 * 1025 *1024 ) can improve the kafka produce performance.
|
|
||||||
* @param file File path
|
|
||||||
* @param mutable mutable
|
|
||||||
* @param fileAlreadyExists File already exists or not
|
|
||||||
* @param initFileSize The size used for pre allocate file, for example 512 * 1025 *1024
|
|
||||||
* @param preallocate Pre allocate file or not, gotten from configuration.
|
|
||||||
*/
|
|
||||||
def openChannel(file: File, mutable: Boolean, fileAlreadyExists: Boolean = false, initFileSize: Int = 0, preallocate: Boolean = false): FileChannel = {
|
|
||||||
if (mutable) {
|
|
||||||
if (fileAlreadyExists)
|
|
||||||
new RandomAccessFile(file, "rw").getChannel()
|
|
||||||
else {
|
|
||||||
if (preallocate) {
|
|
||||||
val randomAccessFile = new RandomAccessFile(file, "rw")
|
|
||||||
randomAccessFile.setLength(initFileSize)
|
|
||||||
randomAccessFile.getChannel()
|
|
||||||
}
|
|
||||||
else
|
|
||||||
new RandomAccessFile(file, "rw").getChannel()
|
|
||||||
}
|
|
||||||
}
|
|
||||||
else
|
|
||||||
new FileInputStream(file).getChannel()
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
object LogFlushStats extends KafkaMetricsGroup {
|
|
||||||
val logFlushTimer = new KafkaTimer(newTimer("LogFlushRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS))
|
|
||||||
}
|
|
||||||
|
|
@ -19,7 +19,6 @@ package kafka.log
|
||||||
|
|
||||||
import kafka.api.KAFKA_0_10_0_IV0
|
import kafka.api.KAFKA_0_10_0_IV0
|
||||||
import kafka.utils._
|
import kafka.utils._
|
||||||
import kafka.message._
|
|
||||||
import kafka.common._
|
import kafka.common._
|
||||||
import kafka.metrics.KafkaMetricsGroup
|
import kafka.metrics.KafkaMetricsGroup
|
||||||
import kafka.server.{BrokerTopicStats, FetchDataInfo, LogOffsetMetadata}
|
import kafka.server.{BrokerTopicStats, FetchDataInfo, LogOffsetMetadata}
|
||||||
|
|
@ -29,16 +28,18 @@ import java.util.concurrent.atomic._
|
||||||
import java.text.NumberFormat
|
import java.text.NumberFormat
|
||||||
|
|
||||||
import org.apache.kafka.common.errors.{CorruptRecordException, OffsetOutOfRangeException, RecordBatchTooLargeException, RecordTooLargeException, UnsupportedForMessageFormatException}
|
import org.apache.kafka.common.errors.{CorruptRecordException, OffsetOutOfRangeException, RecordBatchTooLargeException, RecordTooLargeException, UnsupportedForMessageFormatException}
|
||||||
import org.apache.kafka.common.record.TimestampType
|
import org.apache.kafka.common.record._
|
||||||
import org.apache.kafka.common.requests.ListOffsetRequest
|
import org.apache.kafka.common.requests.ListOffsetRequest
|
||||||
|
|
||||||
import scala.collection.Seq
|
import scala.collection.Seq
|
||||||
import scala.collection.JavaConverters._
|
import scala.collection.JavaConverters._
|
||||||
import com.yammer.metrics.core.Gauge
|
import com.yammer.metrics.core.Gauge
|
||||||
import org.apache.kafka.common.utils.{Time, Utils}
|
import org.apache.kafka.common.utils.{Time, Utils}
|
||||||
|
import kafka.message.{BrokerCompressionCodec, CompressionCodec, NoCompressionCodec}
|
||||||
|
|
||||||
object LogAppendInfo {
|
object LogAppendInfo {
|
||||||
val UnknownLogAppendInfo = LogAppendInfo(-1, -1, Message.NoTimestamp, -1L, Message.NoTimestamp, NoCompressionCodec, NoCompressionCodec, -1, -1, offsetsMonotonic = false)
|
val UnknownLogAppendInfo = LogAppendInfo(-1, -1, Record.NO_TIMESTAMP, -1L, Record.NO_TIMESTAMP,
|
||||||
|
NoCompressionCodec, NoCompressionCodec, -1, -1, offsetsMonotonic = false)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -243,7 +244,7 @@ class Log(@volatile var dir: File,
|
||||||
val index = new OffsetIndex(indexFile, baseOffset = startOffset, maxIndexSize = config.maxIndexSize)
|
val index = new OffsetIndex(indexFile, baseOffset = startOffset, maxIndexSize = config.maxIndexSize)
|
||||||
val timeIndexFile = new File(CoreUtils.replaceSuffix(logFile.getPath, LogFileSuffix, TimeIndexFileSuffix) + SwapFileSuffix)
|
val timeIndexFile = new File(CoreUtils.replaceSuffix(logFile.getPath, LogFileSuffix, TimeIndexFileSuffix) + SwapFileSuffix)
|
||||||
val timeIndex = new TimeIndex(timeIndexFile, baseOffset = startOffset, maxIndexSize = config.maxIndexSize)
|
val timeIndex = new TimeIndex(timeIndexFile, baseOffset = startOffset, maxIndexSize = config.maxIndexSize)
|
||||||
val swapSegment = new LogSegment(new FileMessageSet(file = swapFile),
|
val swapSegment = new LogSegment(FileRecords.open(swapFile),
|
||||||
index = index,
|
index = index,
|
||||||
timeIndex = timeIndex,
|
timeIndex = timeIndex,
|
||||||
baseOffset = startOffset,
|
baseOffset = startOffset,
|
||||||
|
|
@ -338,20 +339,20 @@ class Log(@volatile var dir: File,
|
||||||
* This method will generally be responsible for assigning offsets to the messages,
|
* This method will generally be responsible for assigning offsets to the messages,
|
||||||
* however if the assignOffsets=false flag is passed we will only check that the existing offsets are valid.
|
* however if the assignOffsets=false flag is passed we will only check that the existing offsets are valid.
|
||||||
*
|
*
|
||||||
* @param messages The message set to append
|
* @param records The log records to append
|
||||||
* @param assignOffsets Should the log assign offsets to this message set or blindly apply what it is given
|
* @param assignOffsets Should the log assign offsets to this message set or blindly apply what it is given
|
||||||
* @throws KafkaStorageException If the append fails due to an I/O error.
|
* @throws KafkaStorageException If the append fails due to an I/O error.
|
||||||
* @return Information about the appended messages including the first and last offset.
|
* @return Information about the appended messages including the first and last offset.
|
||||||
*/
|
*/
|
||||||
def append(messages: ByteBufferMessageSet, assignOffsets: Boolean = true): LogAppendInfo = {
|
def append(records: MemoryRecords, assignOffsets: Boolean = true): LogAppendInfo = {
|
||||||
val appendInfo = analyzeAndValidateMessageSet(messages)
|
val appendInfo = analyzeAndValidateRecords(records)
|
||||||
|
|
||||||
// if we have any valid messages, append them to the log
|
// if we have any valid messages, append them to the log
|
||||||
if (appendInfo.shallowCount == 0)
|
if (appendInfo.shallowCount == 0)
|
||||||
return appendInfo
|
return appendInfo
|
||||||
|
|
||||||
// trim any invalid bytes or partial messages before appending it to the on-disk log
|
// trim any invalid bytes or partial messages before appending it to the on-disk log
|
||||||
var validMessages = trimInvalidBytes(messages, appendInfo)
|
var validRecords = trimInvalidBytes(records, appendInfo)
|
||||||
|
|
||||||
try {
|
try {
|
||||||
// they are valid, insert them in the log
|
// they are valid, insert them in the log
|
||||||
|
|
@ -363,20 +364,21 @@ class Log(@volatile var dir: File,
|
||||||
appendInfo.firstOffset = offset.value
|
appendInfo.firstOffset = offset.value
|
||||||
val now = time.milliseconds
|
val now = time.milliseconds
|
||||||
val validateAndOffsetAssignResult = try {
|
val validateAndOffsetAssignResult = try {
|
||||||
validMessages.validateMessagesAndAssignOffsets(offset,
|
LogValidator.validateMessagesAndAssignOffsets(validRecords,
|
||||||
now,
|
offset,
|
||||||
appendInfo.sourceCodec,
|
now,
|
||||||
appendInfo.targetCodec,
|
appendInfo.sourceCodec,
|
||||||
config.compact,
|
appendInfo.targetCodec,
|
||||||
config.messageFormatVersion.messageFormatVersion,
|
config.compact,
|
||||||
config.messageTimestampType,
|
config.messageFormatVersion.messageFormatVersion,
|
||||||
config.messageTimestampDifferenceMaxMs)
|
config.messageTimestampType,
|
||||||
|
config.messageTimestampDifferenceMaxMs)
|
||||||
} catch {
|
} catch {
|
||||||
case e: IOException => throw new KafkaException("Error in validating messages while appending to log '%s'".format(name), e)
|
case e: IOException => throw new KafkaException("Error in validating messages while appending to log '%s'".format(name), e)
|
||||||
}
|
}
|
||||||
validMessages = validateAndOffsetAssignResult.validatedMessages
|
validRecords = validateAndOffsetAssignResult.validatedRecords
|
||||||
appendInfo.maxTimestamp = validateAndOffsetAssignResult.maxTimestamp
|
appendInfo.maxTimestamp = validateAndOffsetAssignResult.maxTimestamp
|
||||||
appendInfo.offsetOfMaxTimestamp = validateAndOffsetAssignResult.offsetOfMaxTimestamp
|
appendInfo.offsetOfMaxTimestamp = validateAndOffsetAssignResult.shallowOffsetOfMaxTimestamp
|
||||||
appendInfo.lastOffset = offset.value - 1
|
appendInfo.lastOffset = offset.value - 1
|
||||||
if (config.messageTimestampType == TimestampType.LOG_APPEND_TIME)
|
if (config.messageTimestampType == TimestampType.LOG_APPEND_TIME)
|
||||||
appendInfo.logAppendTime = now
|
appendInfo.logAppendTime = now
|
||||||
|
|
@ -384,14 +386,14 @@ class Log(@volatile var dir: File,
|
||||||
// re-validate message sizes if there's a possibility that they have changed (due to re-compression or message
|
// re-validate message sizes if there's a possibility that they have changed (due to re-compression or message
|
||||||
// format conversion)
|
// format conversion)
|
||||||
if (validateAndOffsetAssignResult.messageSizeMaybeChanged) {
|
if (validateAndOffsetAssignResult.messageSizeMaybeChanged) {
|
||||||
for (messageAndOffset <- validMessages.shallowIterator) {
|
for (logEntry <- validRecords.shallowIterator.asScala) {
|
||||||
if (MessageSet.entrySize(messageAndOffset.message) > config.maxMessageSize) {
|
if (logEntry.sizeInBytes > config.maxMessageSize) {
|
||||||
// we record the original message set size instead of the trimmed size
|
// we record the original message set size instead of the trimmed size
|
||||||
// to be consistent with pre-compression bytesRejectedRate recording
|
// to be consistent with pre-compression bytesRejectedRate recording
|
||||||
BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesRejectedRate.mark(messages.sizeInBytes)
|
BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesRejectedRate.mark(records.sizeInBytes)
|
||||||
BrokerTopicStats.getBrokerAllTopicsStats.bytesRejectedRate.mark(messages.sizeInBytes)
|
BrokerTopicStats.getBrokerAllTopicsStats.bytesRejectedRate.mark(records.sizeInBytes)
|
||||||
throw new RecordTooLargeException("Message size is %d bytes which exceeds the maximum configured message size of %d."
|
throw new RecordTooLargeException("Message size is %d bytes which exceeds the maximum configured message size of %d."
|
||||||
.format(MessageSet.entrySize(messageAndOffset.message), config.maxMessageSize))
|
.format(logEntry.sizeInBytes, config.maxMessageSize))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -399,28 +401,27 @@ class Log(@volatile var dir: File,
|
||||||
} else {
|
} else {
|
||||||
// we are taking the offsets we are given
|
// we are taking the offsets we are given
|
||||||
if (!appendInfo.offsetsMonotonic || appendInfo.firstOffset < nextOffsetMetadata.messageOffset)
|
if (!appendInfo.offsetsMonotonic || appendInfo.firstOffset < nextOffsetMetadata.messageOffset)
|
||||||
throw new IllegalArgumentException("Out of order offsets found in " + messages)
|
throw new IllegalArgumentException("Out of order offsets found in " + records.deepIterator.asScala.map(_.offset))
|
||||||
}
|
}
|
||||||
|
|
||||||
// check messages set size may be exceed config.segmentSize
|
// check messages set size may be exceed config.segmentSize
|
||||||
if (validMessages.sizeInBytes > config.segmentSize) {
|
if (validRecords.sizeInBytes > config.segmentSize) {
|
||||||
throw new RecordBatchTooLargeException("Message set size is %d bytes which exceeds the maximum configured segment size of %d."
|
throw new RecordBatchTooLargeException("Message set size is %d bytes which exceeds the maximum configured segment size of %d."
|
||||||
.format(validMessages.sizeInBytes, config.segmentSize))
|
.format(validRecords.sizeInBytes, config.segmentSize))
|
||||||
}
|
}
|
||||||
|
|
||||||
// maybe roll the log if this segment is full
|
// maybe roll the log if this segment is full
|
||||||
val segment = maybeRoll(messagesSize = validMessages.sizeInBytes,
|
val segment = maybeRoll(messagesSize = validRecords.sizeInBytes, maxTimestampInMessages = appendInfo.maxTimestamp)
|
||||||
maxTimestampInMessages = appendInfo.maxTimestamp)
|
|
||||||
|
|
||||||
// now append to the log
|
// now append to the log
|
||||||
segment.append(firstOffset = appendInfo.firstOffset, largestTimestamp = appendInfo.maxTimestamp,
|
segment.append(firstOffset = appendInfo.firstOffset, largestTimestamp = appendInfo.maxTimestamp,
|
||||||
offsetOfLargestTimestamp = appendInfo.offsetOfMaxTimestamp, messages = validMessages)
|
shallowOffsetOfMaxTimestamp = appendInfo.offsetOfMaxTimestamp, records = validRecords)
|
||||||
|
|
||||||
// increment the log end offset
|
// increment the log end offset
|
||||||
updateLogEndOffset(appendInfo.lastOffset + 1)
|
updateLogEndOffset(appendInfo.lastOffset + 1)
|
||||||
|
|
||||||
trace("Appended message set to log %s with first offset: %d, next offset: %d, and messages: %s"
|
trace("Appended message set to log %s with first offset: %d, next offset: %d, and messages: %s"
|
||||||
.format(this.name, appendInfo.firstOffset, nextOffsetMetadata.messageOffset, validMessages))
|
.format(this.name, appendInfo.firstOffset, nextOffsetMetadata.messageOffset, validRecords))
|
||||||
|
|
||||||
if (unflushedMessages >= config.flushInterval)
|
if (unflushedMessages >= config.flushInterval)
|
||||||
flush()
|
flush()
|
||||||
|
|
@ -449,73 +450,74 @@ class Log(@volatile var dir: File,
|
||||||
* <li> Whether any compression codec is used (if many are used, then the last one is given)
|
* <li> Whether any compression codec is used (if many are used, then the last one is given)
|
||||||
* </ol>
|
* </ol>
|
||||||
*/
|
*/
|
||||||
private def analyzeAndValidateMessageSet(messages: ByteBufferMessageSet): LogAppendInfo = {
|
private def analyzeAndValidateRecords(records: MemoryRecords): LogAppendInfo = {
|
||||||
var shallowMessageCount = 0
|
var shallowMessageCount = 0
|
||||||
var validBytesCount = 0
|
var validBytesCount = 0
|
||||||
var firstOffset, lastOffset = -1L
|
var firstOffset, lastOffset = -1L
|
||||||
var sourceCodec: CompressionCodec = NoCompressionCodec
|
var sourceCodec: CompressionCodec = NoCompressionCodec
|
||||||
var monotonic = true
|
var monotonic = true
|
||||||
var maxTimestamp = Message.NoTimestamp
|
var maxTimestamp = Record.NO_TIMESTAMP
|
||||||
var offsetOfMaxTimestamp = -1L
|
var offsetOfMaxTimestamp = -1L
|
||||||
for(messageAndOffset <- messages.shallowIterator) {
|
for (entry <- records.shallowIterator.asScala) {
|
||||||
// update the first offset if on the first message
|
// update the first offset if on the first message
|
||||||
if(firstOffset < 0)
|
if(firstOffset < 0)
|
||||||
firstOffset = messageAndOffset.offset
|
firstOffset = entry.offset
|
||||||
// check that offsets are monotonically increasing
|
// check that offsets are monotonically increasing
|
||||||
if(lastOffset >= messageAndOffset.offset)
|
if(lastOffset >= entry.offset)
|
||||||
monotonic = false
|
monotonic = false
|
||||||
// update the last offset seen
|
// update the last offset seen
|
||||||
lastOffset = messageAndOffset.offset
|
lastOffset = entry.offset
|
||||||
|
|
||||||
val m = messageAndOffset.message
|
val record = entry.record
|
||||||
|
|
||||||
// Check if the message sizes are valid.
|
// Check if the message sizes are valid.
|
||||||
val messageSize = MessageSet.entrySize(m)
|
val messageSize = entry.sizeInBytes
|
||||||
if(messageSize > config.maxMessageSize) {
|
if(messageSize > config.maxMessageSize) {
|
||||||
BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesRejectedRate.mark(messages.sizeInBytes)
|
BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesRejectedRate.mark(records.sizeInBytes)
|
||||||
BrokerTopicStats.getBrokerAllTopicsStats.bytesRejectedRate.mark(messages.sizeInBytes)
|
BrokerTopicStats.getBrokerAllTopicsStats.bytesRejectedRate.mark(records.sizeInBytes)
|
||||||
throw new RecordTooLargeException("Message size is %d bytes which exceeds the maximum configured message size of %d."
|
throw new RecordTooLargeException("Message size is %d bytes which exceeds the maximum configured message size of %d."
|
||||||
.format(messageSize, config.maxMessageSize))
|
.format(messageSize, config.maxMessageSize))
|
||||||
}
|
}
|
||||||
|
|
||||||
// check the validity of the message by checking CRC
|
// check the validity of the message by checking CRC
|
||||||
m.ensureValid()
|
record.ensureValid()
|
||||||
if (m.timestamp > maxTimestamp) {
|
if (record.timestamp > maxTimestamp) {
|
||||||
maxTimestamp = m.timestamp
|
maxTimestamp = record.timestamp
|
||||||
offsetOfMaxTimestamp = lastOffset
|
offsetOfMaxTimestamp = lastOffset
|
||||||
}
|
}
|
||||||
shallowMessageCount += 1
|
shallowMessageCount += 1
|
||||||
validBytesCount += messageSize
|
validBytesCount += messageSize
|
||||||
|
|
||||||
val messageCodec = m.compressionCodec
|
val messageCodec = CompressionCodec.getCompressionCodec(record.compressionType.id)
|
||||||
if(messageCodec != NoCompressionCodec)
|
if (messageCodec != NoCompressionCodec)
|
||||||
sourceCodec = messageCodec
|
sourceCodec = messageCodec
|
||||||
}
|
}
|
||||||
|
|
||||||
// Apply broker-side compression if any
|
// Apply broker-side compression if any
|
||||||
val targetCodec = BrokerCompressionCodec.getTargetCompressionCodec(config.compressionType, sourceCodec)
|
val targetCodec = BrokerCompressionCodec.getTargetCompressionCodec(config.compressionType, sourceCodec)
|
||||||
|
|
||||||
LogAppendInfo(firstOffset, lastOffset, maxTimestamp, offsetOfMaxTimestamp, Message.NoTimestamp, sourceCodec, targetCodec, shallowMessageCount, validBytesCount, monotonic)
|
LogAppendInfo(firstOffset, lastOffset, maxTimestamp, offsetOfMaxTimestamp, Record.NO_TIMESTAMP, sourceCodec,
|
||||||
|
targetCodec, shallowMessageCount, validBytesCount, monotonic)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Trim any invalid bytes from the end of this message set (if there are any)
|
* Trim any invalid bytes from the end of this message set (if there are any)
|
||||||
*
|
*
|
||||||
* @param messages The message set to trim
|
* @param records The records to trim
|
||||||
* @param info The general information of the message set
|
* @param info The general information of the message set
|
||||||
* @return A trimmed message set. This may be the same as what was passed in or it may not.
|
* @return A trimmed message set. This may be the same as what was passed in or it may not.
|
||||||
*/
|
*/
|
||||||
private def trimInvalidBytes(messages: ByteBufferMessageSet, info: LogAppendInfo): ByteBufferMessageSet = {
|
private def trimInvalidBytes(records: MemoryRecords, info: LogAppendInfo): MemoryRecords = {
|
||||||
val messageSetValidBytes = info.validBytes
|
val validBytes = info.validBytes
|
||||||
if(messageSetValidBytes < 0)
|
if (validBytes < 0)
|
||||||
throw new CorruptRecordException("Illegal length of message set " + messageSetValidBytes + " Message set cannot be appended to log. Possible causes are corrupted produce requests")
|
throw new CorruptRecordException("Illegal length of message set " + validBytes + " Message set cannot be appended to log. Possible causes are corrupted produce requests")
|
||||||
if(messageSetValidBytes == messages.sizeInBytes) {
|
if (validBytes == records.sizeInBytes) {
|
||||||
messages
|
records
|
||||||
} else {
|
} else {
|
||||||
// trim invalid bytes
|
// trim invalid bytes
|
||||||
val validByteBuffer = messages.buffer.duplicate()
|
val validByteBuffer = records.buffer.duplicate()
|
||||||
validByteBuffer.limit(messageSetValidBytes)
|
validByteBuffer.limit(validBytes)
|
||||||
new ByteBufferMessageSet(validByteBuffer)
|
MemoryRecords.readableRecords(validByteBuffer)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -538,7 +540,7 @@ class Log(@volatile var dir: File,
|
||||||
val currentNextOffsetMetadata = nextOffsetMetadata
|
val currentNextOffsetMetadata = nextOffsetMetadata
|
||||||
val next = currentNextOffsetMetadata.messageOffset
|
val next = currentNextOffsetMetadata.messageOffset
|
||||||
if(startOffset == next)
|
if(startOffset == next)
|
||||||
return FetchDataInfo(currentNextOffsetMetadata, MessageSet.Empty)
|
return FetchDataInfo(currentNextOffsetMetadata, MemoryRecords.EMPTY)
|
||||||
|
|
||||||
var entry = segments.floorEntry(startOffset)
|
var entry = segments.floorEntry(startOffset)
|
||||||
|
|
||||||
|
|
@ -578,7 +580,7 @@ class Log(@volatile var dir: File,
|
||||||
// okay we are beyond the end of the last segment with no data fetched although the start offset is in range,
|
// okay we are beyond the end of the last segment with no data fetched although the start offset is in range,
|
||||||
// this can happen when all messages with offset larger than start offsets have been deleted.
|
// this can happen when all messages with offset larger than start offsets have been deleted.
|
||||||
// In this case, we will return the empty set with log end offset metadata
|
// In this case, we will return the empty set with log end offset metadata
|
||||||
FetchDataInfo(nextOffsetMetadata, MessageSet.Empty)
|
FetchDataInfo(nextOffsetMetadata, MemoryRecords.EMPTY)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -610,9 +612,9 @@ class Log(@volatile var dir: File,
|
||||||
val segmentsCopy = logSegments.toBuffer
|
val segmentsCopy = logSegments.toBuffer
|
||||||
// For the earliest and latest, we do not need to return the timestamp.
|
// For the earliest and latest, we do not need to return the timestamp.
|
||||||
if (targetTimestamp == ListOffsetRequest.EARLIEST_TIMESTAMP)
|
if (targetTimestamp == ListOffsetRequest.EARLIEST_TIMESTAMP)
|
||||||
return Some(TimestampOffset(Message.NoTimestamp, segmentsCopy.head.baseOffset))
|
return Some(TimestampOffset(Record.NO_TIMESTAMP, segmentsCopy.head.baseOffset))
|
||||||
else if (targetTimestamp == ListOffsetRequest.LATEST_TIMESTAMP)
|
else if (targetTimestamp == ListOffsetRequest.LATEST_TIMESTAMP)
|
||||||
return Some(TimestampOffset(Message.NoTimestamp, logEndOffset))
|
return Some(TimestampOffset(Record.NO_TIMESTAMP, logEndOffset))
|
||||||
|
|
||||||
val targetSeg = {
|
val targetSeg = {
|
||||||
// Get all the segments whose largest timestamp is smaller than target timestamp
|
// Get all the segments whose largest timestamp is smaller than target timestamp
|
||||||
|
|
@ -656,7 +658,7 @@ class Log(@volatile var dir: File,
|
||||||
if (segments.size == numToDelete)
|
if (segments.size == numToDelete)
|
||||||
roll()
|
roll()
|
||||||
// remove the segments for lookups
|
// remove the segments for lookups
|
||||||
deletable.foreach(deleteSegment(_))
|
deletable.foreach(deleteSegment)
|
||||||
}
|
}
|
||||||
numToDelete
|
numToDelete
|
||||||
}
|
}
|
||||||
|
|
@ -865,7 +867,7 @@ class Log(@volatile var dir: File,
|
||||||
truncateFullyAndStartAt(targetOffset)
|
truncateFullyAndStartAt(targetOffset)
|
||||||
} else {
|
} else {
|
||||||
val deletable = logSegments.filter(segment => segment.baseOffset > targetOffset)
|
val deletable = logSegments.filter(segment => segment.baseOffset > targetOffset)
|
||||||
deletable.foreach(deleteSegment(_))
|
deletable.foreach(deleteSegment)
|
||||||
activeSegment.truncateTo(targetOffset)
|
activeSegment.truncateTo(targetOffset)
|
||||||
updateLogEndOffset(targetOffset)
|
updateLogEndOffset(targetOffset)
|
||||||
this.recoveryPoint = math.min(targetOffset, this.recoveryPoint)
|
this.recoveryPoint = math.min(targetOffset, this.recoveryPoint)
|
||||||
|
|
@ -882,7 +884,7 @@ class Log(@volatile var dir: File,
|
||||||
debug("Truncate and start log '" + name + "' to " + newOffset)
|
debug("Truncate and start log '" + name + "' to " + newOffset)
|
||||||
lock synchronized {
|
lock synchronized {
|
||||||
val segmentsToDelete = logSegments.toList
|
val segmentsToDelete = logSegments.toList
|
||||||
segmentsToDelete.foreach(deleteSegment(_))
|
segmentsToDelete.foreach(deleteSegment)
|
||||||
addSegment(new LogSegment(dir,
|
addSegment(new LogSegment(dir,
|
||||||
newOffset,
|
newOffset,
|
||||||
indexIntervalBytes = config.indexInterval,
|
indexIntervalBytes = config.indexInterval,
|
||||||
|
|
|
||||||
|
|
@ -17,20 +17,21 @@
|
||||||
|
|
||||||
package kafka.log
|
package kafka.log
|
||||||
|
|
||||||
import java.io.{DataOutputStream, File}
|
import java.io.File
|
||||||
import java.nio._
|
import java.nio._
|
||||||
import java.util.Date
|
import java.util.Date
|
||||||
import java.util.concurrent.{CountDownLatch, TimeUnit}
|
import java.util.concurrent.{CountDownLatch, TimeUnit}
|
||||||
|
|
||||||
import com.yammer.metrics.core.Gauge
|
import com.yammer.metrics.core.Gauge
|
||||||
import kafka.common._
|
import kafka.common._
|
||||||
import kafka.message._
|
|
||||||
import kafka.metrics.KafkaMetricsGroup
|
import kafka.metrics.KafkaMetricsGroup
|
||||||
import kafka.utils._
|
import kafka.utils._
|
||||||
|
import org.apache.kafka.common.record.{FileRecords, LogEntry, MemoryRecords}
|
||||||
import org.apache.kafka.common.utils.Time
|
import org.apache.kafka.common.utils.Time
|
||||||
|
import MemoryRecords.LogEntryFilter
|
||||||
|
|
||||||
import scala.Iterable
|
|
||||||
import scala.collection._
|
import scala.collection._
|
||||||
|
import JavaConverters._
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The cleaner is responsible for removing obsolete records from logs which have the dedupe retention strategy.
|
* The cleaner is responsible for removing obsolete records from logs which have the dedupe retention strategy.
|
||||||
|
|
@ -390,10 +391,10 @@ private[log] class Cleaner(val id: Int,
|
||||||
val timeIndexFile = new File(segments.head.timeIndex.file.getPath + Log.CleanedFileSuffix)
|
val timeIndexFile = new File(segments.head.timeIndex.file.getPath + Log.CleanedFileSuffix)
|
||||||
indexFile.delete()
|
indexFile.delete()
|
||||||
timeIndexFile.delete()
|
timeIndexFile.delete()
|
||||||
val messages = new FileMessageSet(logFile, fileAlreadyExists = false, initFileSize = log.initFileSize(), preallocate = log.config.preallocate)
|
val records = FileRecords.open(logFile, false, log.initFileSize(), log.config.preallocate)
|
||||||
val index = new OffsetIndex(indexFile, segments.head.baseOffset, segments.head.index.maxIndexSize)
|
val index = new OffsetIndex(indexFile, segments.head.baseOffset, segments.head.index.maxIndexSize)
|
||||||
val timeIndex = new TimeIndex(timeIndexFile, segments.head.baseOffset, segments.head.timeIndex.maxIndexSize)
|
val timeIndex = new TimeIndex(timeIndexFile, segments.head.baseOffset, segments.head.timeIndex.maxIndexSize)
|
||||||
val cleaned = new LogSegment(messages, index, timeIndex, segments.head.baseOffset, segments.head.indexIntervalBytes, log.config.randomSegmentJitter, time)
|
val cleaned = new LogSegment(records, index, timeIndex, segments.head.baseOffset, segments.head.indexIntervalBytes, log.config.randomSegmentJitter, time)
|
||||||
|
|
||||||
try {
|
try {
|
||||||
// clean segments into the new destination segment
|
// clean segments into the new destination segment
|
||||||
|
|
@ -449,8 +450,12 @@ private[log] class Cleaner(val id: Int,
|
||||||
retainDeletes: Boolean,
|
retainDeletes: Boolean,
|
||||||
maxLogMessageSize: Int,
|
maxLogMessageSize: Int,
|
||||||
stats: CleanerStats) {
|
stats: CleanerStats) {
|
||||||
def shouldRetain(messageAndOffset: MessageAndOffset): Boolean =
|
def shouldRetainEntry(logEntry: LogEntry): Boolean =
|
||||||
shouldRetainMessage(source, map, retainDeletes, messageAndOffset, stats)
|
shouldRetainMessage(source, map, retainDeletes, logEntry, stats)
|
||||||
|
|
||||||
|
class LogCleanerFilter extends LogEntryFilter {
|
||||||
|
def shouldRetain(logEntry: LogEntry): Boolean = shouldRetainEntry(logEntry)
|
||||||
|
}
|
||||||
|
|
||||||
var position = 0
|
var position = 0
|
||||||
while (position < source.log.sizeInBytes) {
|
while (position < source.log.sizeInBytes) {
|
||||||
|
|
@ -460,10 +465,9 @@ private[log] class Cleaner(val id: Int,
|
||||||
writeBuffer.clear()
|
writeBuffer.clear()
|
||||||
|
|
||||||
source.log.readInto(readBuffer, position)
|
source.log.readInto(readBuffer, position)
|
||||||
val messages = new ByteBufferMessageSet(readBuffer)
|
val records = MemoryRecords.readableRecords(readBuffer)
|
||||||
throttler.maybeThrottle(messages.sizeInBytes)
|
throttler.maybeThrottle(records.sizeInBytes)
|
||||||
val result = messages.filterInto(writeBuffer, shouldRetain)
|
val result = records.filterTo(new LogCleanerFilter, writeBuffer)
|
||||||
|
|
||||||
stats.readMessages(result.messagesRead, result.bytesRead)
|
stats.readMessages(result.messagesRead, result.bytesRead)
|
||||||
stats.recopyMessages(result.messagesRetained, result.bytesRetained)
|
stats.recopyMessages(result.messagesRetained, result.bytesRetained)
|
||||||
|
|
||||||
|
|
@ -472,9 +476,10 @@ private[log] class Cleaner(val id: Int,
|
||||||
// if any messages are to be retained, write them out
|
// if any messages are to be retained, write them out
|
||||||
if (writeBuffer.position > 0) {
|
if (writeBuffer.position > 0) {
|
||||||
writeBuffer.flip()
|
writeBuffer.flip()
|
||||||
val retained = new ByteBufferMessageSet(writeBuffer)
|
|
||||||
dest.append(firstOffset = retained.head.offset, largestTimestamp = result.maxTimestamp,
|
val retained = MemoryRecords.readableRecords(writeBuffer)
|
||||||
offsetOfLargestTimestamp = result.offsetOfMaxTimestamp, messages = retained)
|
dest.append(firstOffset = retained.deepIterator().next().offset, largestTimestamp = result.maxTimestamp,
|
||||||
|
shallowOffsetOfMaxTimestamp = result.shallowOffsetOfMaxTimestamp, records = retained)
|
||||||
throttler.maybeThrottle(writeBuffer.limit)
|
throttler.maybeThrottle(writeBuffer.limit)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -488,21 +493,22 @@ private[log] class Cleaner(val id: Int,
|
||||||
private def shouldRetainMessage(source: kafka.log.LogSegment,
|
private def shouldRetainMessage(source: kafka.log.LogSegment,
|
||||||
map: kafka.log.OffsetMap,
|
map: kafka.log.OffsetMap,
|
||||||
retainDeletes: Boolean,
|
retainDeletes: Boolean,
|
||||||
entry: kafka.message.MessageAndOffset,
|
entry: LogEntry,
|
||||||
stats: CleanerStats): Boolean = {
|
stats: CleanerStats): Boolean = {
|
||||||
val pastLatestOffset = entry.offset > map.latestOffset
|
val pastLatestOffset = entry.offset > map.latestOffset
|
||||||
if (pastLatestOffset)
|
if (pastLatestOffset)
|
||||||
return true
|
return true
|
||||||
|
|
||||||
val key = entry.message.key
|
|
||||||
if (key != null) {
|
if (entry.record.hasKey) {
|
||||||
|
val key = entry.record.key
|
||||||
val foundOffset = map.get(key)
|
val foundOffset = map.get(key)
|
||||||
/* two cases in which we can get rid of a message:
|
/* two cases in which we can get rid of a message:
|
||||||
* 1) if there exists a message with the same key but higher offset
|
* 1) if there exists a message with the same key but higher offset
|
||||||
* 2) if the message is a delete "tombstone" marker and enough time has passed
|
* 2) if the message is a delete "tombstone" marker and enough time has passed
|
||||||
*/
|
*/
|
||||||
val redundant = foundOffset >= 0 && entry.offset < foundOffset
|
val redundant = foundOffset >= 0 && entry.offset < foundOffset
|
||||||
val obsoleteDelete = !retainDeletes && entry.message.isNull
|
val obsoleteDelete = !retainDeletes && entry.record.hasNullValue
|
||||||
!redundant && !obsoleteDelete
|
!redundant && !obsoleteDelete
|
||||||
} else {
|
} else {
|
||||||
stats.invalidMessage()
|
stats.invalidMessage()
|
||||||
|
|
@ -620,12 +626,12 @@ private[log] class Cleaner(val id: Int,
|
||||||
checkDone(topicAndPartition)
|
checkDone(topicAndPartition)
|
||||||
readBuffer.clear()
|
readBuffer.clear()
|
||||||
segment.log.readInto(readBuffer, position)
|
segment.log.readInto(readBuffer, position)
|
||||||
val messages = new ByteBufferMessageSet(readBuffer)
|
val records = MemoryRecords.readableRecords(readBuffer)
|
||||||
throttler.maybeThrottle(messages.sizeInBytes)
|
throttler.maybeThrottle(records.sizeInBytes)
|
||||||
|
|
||||||
val startPosition = position
|
val startPosition = position
|
||||||
for (entry <- messages) {
|
for (entry <- records.deepIterator.asScala) {
|
||||||
val message = entry.message
|
val message = entry.record
|
||||||
if (message.hasKey && entry.offset >= start) {
|
if (message.hasKey && entry.offset >= start) {
|
||||||
if (map.size < maxDesiredMapSize)
|
if (map.size < maxDesiredMapSize)
|
||||||
map.put(message.key, entry.offset)
|
map.put(message.key, entry.offset)
|
||||||
|
|
@ -634,8 +640,9 @@ private[log] class Cleaner(val id: Int,
|
||||||
}
|
}
|
||||||
stats.indexMessagesRead(1)
|
stats.indexMessagesRead(1)
|
||||||
}
|
}
|
||||||
position += messages.validBytes
|
val bytesRead = records.validBytes
|
||||||
stats.indexBytesRead(messages.validBytes)
|
position += bytesRead
|
||||||
|
stats.indexBytesRead(bytesRead)
|
||||||
|
|
||||||
// if we didn't read even one complete message, our read buffer may be too small
|
// if we didn't read even one complete message, our read buffer may be too small
|
||||||
if(position == startPosition)
|
if(position == startPosition)
|
||||||
|
|
|
||||||
|
|
@ -440,7 +440,7 @@ class LogManager(val logDirs: Array[File],
|
||||||
removedLog.dir = renamedDir
|
removedLog.dir = renamedDir
|
||||||
// change the file pointers for log and index file
|
// change the file pointers for log and index file
|
||||||
for (logSegment <- removedLog.logSegments) {
|
for (logSegment <- removedLog.logSegments) {
|
||||||
logSegment.log.file = new File(renamedDir, logSegment.log.file.getName)
|
logSegment.log.setFile(new File(renamedDir, logSegment.log.file.getName))
|
||||||
logSegment.index.file = new File(renamedDir, logSegment.index.file.getName)
|
logSegment.index.file = new File(renamedDir, logSegment.index.file.getName)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -16,15 +16,20 @@
|
||||||
*/
|
*/
|
||||||
package kafka.log
|
package kafka.log
|
||||||
|
|
||||||
import kafka.message._
|
import java.io.{File, IOException}
|
||||||
|
import java.util.concurrent.TimeUnit
|
||||||
|
|
||||||
import kafka.common._
|
import kafka.common._
|
||||||
import kafka.utils._
|
import kafka.metrics.{KafkaMetricsGroup, KafkaTimer}
|
||||||
import kafka.server.{FetchDataInfo, LogOffsetMetadata}
|
import kafka.server.{FetchDataInfo, LogOffsetMetadata}
|
||||||
|
import kafka.utils._
|
||||||
import org.apache.kafka.common.errors.CorruptRecordException
|
import org.apache.kafka.common.errors.CorruptRecordException
|
||||||
|
import org.apache.kafka.common.record.FileRecords.LogEntryPosition
|
||||||
|
import org.apache.kafka.common.record._
|
||||||
import org.apache.kafka.common.utils.Time
|
import org.apache.kafka.common.utils.Time
|
||||||
|
|
||||||
|
import scala.collection.JavaConverters._
|
||||||
import scala.math._
|
import scala.math._
|
||||||
import java.io.{File, IOException}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A segment of the log. Each segment has two components: a log and an index. The log is a FileMessageSet containing
|
* A segment of the log. Each segment has two components: a log and an index. The log is a FileMessageSet containing
|
||||||
|
|
@ -42,7 +47,7 @@ import java.io.{File, IOException}
|
||||||
* @param time The time instance
|
* @param time The time instance
|
||||||
*/
|
*/
|
||||||
@nonthreadsafe
|
@nonthreadsafe
|
||||||
class LogSegment(val log: FileMessageSet,
|
class LogSegment(val log: FileRecords,
|
||||||
val index: OffsetIndex,
|
val index: OffsetIndex,
|
||||||
val timeIndex: TimeIndex,
|
val timeIndex: TimeIndex,
|
||||||
val baseOffset: Long,
|
val baseOffset: Long,
|
||||||
|
|
@ -63,7 +68,7 @@ class LogSegment(val log: FileMessageSet,
|
||||||
@volatile private var offsetOfMaxTimestamp = timeIndex.lastEntry.offset
|
@volatile private var offsetOfMaxTimestamp = timeIndex.lastEntry.offset
|
||||||
|
|
||||||
def this(dir: File, startOffset: Long, indexIntervalBytes: Int, maxIndexSize: Int, rollJitterMs: Long, time: Time, fileAlreadyExists: Boolean = false, initFileSize: Int = 0, preallocate: Boolean = false) =
|
def this(dir: File, startOffset: Long, indexIntervalBytes: Int, maxIndexSize: Int, rollJitterMs: Long, time: Time, fileAlreadyExists: Boolean = false, initFileSize: Int = 0, preallocate: Boolean = false) =
|
||||||
this(new FileMessageSet(file = Log.logFilename(dir, startOffset), fileAlreadyExists = fileAlreadyExists, initFileSize = initFileSize, preallocate = preallocate),
|
this(FileRecords.open(Log.logFilename(dir, startOffset), fileAlreadyExists, initFileSize, preallocate),
|
||||||
new OffsetIndex(Log.indexFilename(dir, startOffset), baseOffset = startOffset, maxIndexSize = maxIndexSize),
|
new OffsetIndex(Log.indexFilename(dir, startOffset), baseOffset = startOffset, maxIndexSize = maxIndexSize),
|
||||||
new TimeIndex(Log.timeIndexFilename(dir, startOffset), baseOffset = startOffset, maxIndexSize = maxIndexSize),
|
new TimeIndex(Log.timeIndexFilename(dir, startOffset), baseOffset = startOffset, maxIndexSize = maxIndexSize),
|
||||||
startOffset,
|
startOffset,
|
||||||
|
|
@ -82,23 +87,25 @@ class LogSegment(val log: FileMessageSet,
|
||||||
*
|
*
|
||||||
* @param firstOffset The first offset in the message set.
|
* @param firstOffset The first offset in the message set.
|
||||||
* @param largestTimestamp The largest timestamp in the message set.
|
* @param largestTimestamp The largest timestamp in the message set.
|
||||||
* @param offsetOfLargestTimestamp The offset of the message that has the largest timestamp in the messages to append.
|
* @param shallowOffsetOfMaxTimestamp The offset of the message that has the largest timestamp in the messages to append.
|
||||||
* @param messages The messages to append.
|
* @param records The log entries to append.
|
||||||
*/
|
*/
|
||||||
@nonthreadsafe
|
@nonthreadsafe
|
||||||
def append(firstOffset: Long, largestTimestamp: Long, offsetOfLargestTimestamp: Long, messages: ByteBufferMessageSet) {
|
def append(firstOffset: Long, largestTimestamp: Long, shallowOffsetOfMaxTimestamp: Long, records: MemoryRecords) {
|
||||||
if (messages.sizeInBytes > 0) {
|
if (records.sizeInBytes > 0) {
|
||||||
trace("Inserting %d bytes at offset %d at position %d with largest timestamp %d at offset %d"
|
trace("Inserting %d bytes at offset %d at position %d with largest timestamp %d at shallow offset %d"
|
||||||
.format(messages.sizeInBytes, firstOffset, log.sizeInBytes(), largestTimestamp, offsetOfLargestTimestamp))
|
.format(records.sizeInBytes, firstOffset, log.sizeInBytes(), largestTimestamp, shallowOffsetOfMaxTimestamp))
|
||||||
val physicalPosition = log.sizeInBytes()
|
val physicalPosition = log.sizeInBytes()
|
||||||
if (physicalPosition == 0)
|
if (physicalPosition == 0)
|
||||||
rollingBasedTimestamp = Some(largestTimestamp)
|
rollingBasedTimestamp = Some(largestTimestamp)
|
||||||
// append the messages
|
// append the messages
|
||||||
log.append(messages)
|
val appendedBytes = log.append(records)
|
||||||
|
trace(s"Appended $appendedBytes to ${log.file()} at offset $firstOffset")
|
||||||
|
|
||||||
// Update the in memory max timestamp and corresponding offset.
|
// Update the in memory max timestamp and corresponding offset.
|
||||||
if (largestTimestamp > maxTimestampSoFar) {
|
if (largestTimestamp > maxTimestampSoFar) {
|
||||||
maxTimestampSoFar = largestTimestamp
|
maxTimestampSoFar = largestTimestamp
|
||||||
offsetOfMaxTimestamp = offsetOfLargestTimestamp
|
offsetOfMaxTimestamp = shallowOffsetOfMaxTimestamp
|
||||||
}
|
}
|
||||||
// append an entry to the index (if needed)
|
// append an entry to the index (if needed)
|
||||||
if(bytesSinceLastIndexEntry > indexIntervalBytes) {
|
if(bytesSinceLastIndexEntry > indexIntervalBytes) {
|
||||||
|
|
@ -106,7 +113,7 @@ class LogSegment(val log: FileMessageSet,
|
||||||
timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestamp)
|
timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestamp)
|
||||||
bytesSinceLastIndexEntry = 0
|
bytesSinceLastIndexEntry = 0
|
||||||
}
|
}
|
||||||
bytesSinceLastIndexEntry += messages.sizeInBytes
|
bytesSinceLastIndexEntry += records.sizeInBytes
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -123,7 +130,7 @@ class LogSegment(val log: FileMessageSet,
|
||||||
* message or null if no message meets this criteria.
|
* message or null if no message meets this criteria.
|
||||||
*/
|
*/
|
||||||
@threadsafe
|
@threadsafe
|
||||||
private[log] def translateOffset(offset: Long, startingFilePosition: Int = 0): (OffsetPosition, Int) = {
|
private[log] def translateOffset(offset: Long, startingFilePosition: Int = 0): LogEntryPosition = {
|
||||||
val mapping = index.lookup(offset)
|
val mapping = index.lookup(offset)
|
||||||
log.searchForOffsetWithSize(offset, max(mapping.position, startingFilePosition))
|
log.searchForOffsetWithSize(offset, max(mapping.position, startingFilePosition))
|
||||||
}
|
}
|
||||||
|
|
@ -154,40 +161,40 @@ class LogSegment(val log: FileMessageSet,
|
||||||
if (startOffsetAndSize == null)
|
if (startOffsetAndSize == null)
|
||||||
return null
|
return null
|
||||||
|
|
||||||
val (startPosition, messageSetSize) = startOffsetAndSize
|
val startPosition = startOffsetAndSize.position.toInt
|
||||||
val offsetMetadata = new LogOffsetMetadata(startOffset, this.baseOffset, startPosition.position)
|
val offsetMetadata = new LogOffsetMetadata(startOffset, this.baseOffset, startPosition)
|
||||||
|
|
||||||
val adjustedMaxSize =
|
val adjustedMaxSize =
|
||||||
if (minOneMessage) math.max(maxSize, messageSetSize)
|
if (minOneMessage) math.max(maxSize, startOffsetAndSize.size)
|
||||||
else maxSize
|
else maxSize
|
||||||
|
|
||||||
// return a log segment but with zero size in the case below
|
// return a log segment but with zero size in the case below
|
||||||
if (adjustedMaxSize == 0)
|
if (adjustedMaxSize == 0)
|
||||||
return FetchDataInfo(offsetMetadata, MessageSet.Empty)
|
return FetchDataInfo(offsetMetadata, MemoryRecords.EMPTY)
|
||||||
|
|
||||||
// calculate the length of the message set to read based on whether or not they gave us a maxOffset
|
// calculate the length of the message set to read based on whether or not they gave us a maxOffset
|
||||||
val length = maxOffset match {
|
val length = maxOffset match {
|
||||||
case None =>
|
case None =>
|
||||||
// no max offset, just read until the max position
|
// no max offset, just read until the max position
|
||||||
min((maxPosition - startPosition.position).toInt, adjustedMaxSize)
|
min((maxPosition - startPosition).toInt, adjustedMaxSize)
|
||||||
case Some(offset) =>
|
case Some(offset) =>
|
||||||
// there is a max offset, translate it to a file position and use that to calculate the max read size;
|
// there is a max offset, translate it to a file position and use that to calculate the max read size;
|
||||||
// when the leader of a partition changes, it's possible for the new leader's high watermark to be less than the
|
// when the leader of a partition changes, it's possible for the new leader's high watermark to be less than the
|
||||||
// true high watermark in the previous leader for a short window. In this window, if a consumer fetches on an
|
// true high watermark in the previous leader for a short window. In this window, if a consumer fetches on an
|
||||||
// offset between new leader's high watermark and the log end offset, we want to return an empty response.
|
// offset between new leader's high watermark and the log end offset, we want to return an empty response.
|
||||||
if (offset < startOffset)
|
if (offset < startOffset)
|
||||||
return FetchDataInfo(offsetMetadata, MessageSet.Empty, firstMessageSetIncomplete = false)
|
return FetchDataInfo(offsetMetadata, MemoryRecords.EMPTY, firstEntryIncomplete = false)
|
||||||
val mapping = translateOffset(offset, startPosition.position)
|
val mapping = translateOffset(offset, startPosition)
|
||||||
val endPosition =
|
val endPosition =
|
||||||
if (mapping == null)
|
if (mapping == null)
|
||||||
logSize // the max offset is off the end of the log, use the end of the file
|
logSize // the max offset is off the end of the log, use the end of the file
|
||||||
else
|
else
|
||||||
mapping._1.position
|
mapping.position
|
||||||
min(min(maxPosition, endPosition) - startPosition.position, adjustedMaxSize).toInt
|
min(min(maxPosition, endPosition) - startPosition, adjustedMaxSize).toInt
|
||||||
}
|
}
|
||||||
|
|
||||||
FetchDataInfo(offsetMetadata, log.read(startPosition.position, length),
|
FetchDataInfo(offsetMetadata, log.read(startPosition, length),
|
||||||
firstMessageSetIncomplete = adjustedMaxSize < messageSetSize)
|
firstEntryIncomplete = adjustedMaxSize < startOffsetAndSize.size)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -205,16 +212,16 @@ class LogSegment(val log: FileMessageSet,
|
||||||
timeIndex.resize(timeIndex.maxIndexSize)
|
timeIndex.resize(timeIndex.maxIndexSize)
|
||||||
var validBytes = 0
|
var validBytes = 0
|
||||||
var lastIndexEntry = 0
|
var lastIndexEntry = 0
|
||||||
val iter = log.iterator(maxMessageSize)
|
val iter = log.shallowIterator(maxMessageSize)
|
||||||
maxTimestampSoFar = Message.NoTimestamp
|
maxTimestampSoFar = Record.NO_TIMESTAMP
|
||||||
try {
|
try {
|
||||||
while(iter.hasNext) {
|
for (entry <- iter.asScala) {
|
||||||
val entry = iter.next
|
val record = entry.record
|
||||||
entry.message.ensureValid()
|
record.ensureValid()
|
||||||
|
|
||||||
// The max timestamp should have been put in the outer message, so we don't need to iterate over the inner messages.
|
// The max timestamp should have been put in the outer message, so we don't need to iterate over the inner messages.
|
||||||
if (entry.message.timestamp > maxTimestampSoFar) {
|
if (record.timestamp > maxTimestampSoFar) {
|
||||||
maxTimestampSoFar = entry.message.timestamp
|
maxTimestampSoFar = record.timestamp
|
||||||
offsetOfMaxTimestamp = entry.offset
|
offsetOfMaxTimestamp = entry.offset
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -225,11 +232,12 @@ class LogSegment(val log: FileMessageSet,
|
||||||
timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestamp)
|
timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestamp)
|
||||||
lastIndexEntry = validBytes
|
lastIndexEntry = validBytes
|
||||||
}
|
}
|
||||||
validBytes += MessageSet.entrySize(entry.message)
|
validBytes += entry.sizeInBytes()
|
||||||
}
|
}
|
||||||
} catch {
|
} catch {
|
||||||
case e: CorruptRecordException =>
|
case e: CorruptRecordException =>
|
||||||
logger.warn("Found invalid messages in log segment %s at byte offset %d: %s.".format(log.file.getAbsolutePath, validBytes, e.getMessage))
|
logger.warn("Found invalid messages in log segment %s at byte offset %d: %s."
|
||||||
|
.format(log.file.getAbsolutePath, validBytes, e.getMessage))
|
||||||
}
|
}
|
||||||
val truncated = log.sizeInBytes - validBytes
|
val truncated = log.sizeInBytes - validBytes
|
||||||
log.truncateTo(validBytes)
|
log.truncateTo(validBytes)
|
||||||
|
|
@ -276,8 +284,7 @@ class LogSegment(val log: FileMessageSet,
|
||||||
// after truncation, reset and allocate more space for the (new currently active) index
|
// after truncation, reset and allocate more space for the (new currently active) index
|
||||||
index.resize(index.maxIndexSize)
|
index.resize(index.maxIndexSize)
|
||||||
timeIndex.resize(timeIndex.maxIndexSize)
|
timeIndex.resize(timeIndex.maxIndexSize)
|
||||||
val (offsetPosition, _) = mapping
|
val bytesTruncated = log.truncateTo(mapping.position.toInt)
|
||||||
val bytesTruncated = log.truncateTo(offsetPosition.position)
|
|
||||||
if(log.sizeInBytes == 0) {
|
if(log.sizeInBytes == 0) {
|
||||||
created = time.milliseconds
|
created = time.milliseconds
|
||||||
rollingBasedTimestamp = None
|
rollingBasedTimestamp = None
|
||||||
|
|
@ -296,10 +303,10 @@ class LogSegment(val log: FileMessageSet,
|
||||||
@threadsafe
|
@threadsafe
|
||||||
def nextOffset(): Long = {
|
def nextOffset(): Long = {
|
||||||
val ms = read(index.lastOffset, None, log.sizeInBytes)
|
val ms = read(index.lastOffset, None, log.sizeInBytes)
|
||||||
if(ms == null) {
|
if (ms == null) {
|
||||||
baseOffset
|
baseOffset
|
||||||
} else {
|
} else {
|
||||||
ms.messageSet.lastOption match {
|
ms.records.shallowIterator.asScala.toSeq.lastOption match {
|
||||||
case None => baseOffset
|
case None => baseOffset
|
||||||
case Some(last) => last.nextOffset
|
case Some(last) => last.nextOffset
|
||||||
}
|
}
|
||||||
|
|
@ -360,9 +367,9 @@ class LogSegment(val log: FileMessageSet,
|
||||||
def timeWaitedForRoll(now: Long, messageTimestamp: Long) : Long = {
|
def timeWaitedForRoll(now: Long, messageTimestamp: Long) : Long = {
|
||||||
// Load the timestamp of the first message into memory
|
// Load the timestamp of the first message into memory
|
||||||
if (rollingBasedTimestamp.isEmpty) {
|
if (rollingBasedTimestamp.isEmpty) {
|
||||||
val iter = log.iterator
|
val iter = log.shallowIterator
|
||||||
if (iter.hasNext)
|
if (iter.hasNext)
|
||||||
rollingBasedTimestamp = Some(iter.next.message.timestamp)
|
rollingBasedTimestamp = Some(iter.next.record.timestamp)
|
||||||
}
|
}
|
||||||
rollingBasedTimestamp match {
|
rollingBasedTimestamp match {
|
||||||
case Some(t) if t >= 0 => messageTimestamp - t
|
case Some(t) if t >= 0 => messageTimestamp - t
|
||||||
|
|
@ -394,8 +401,11 @@ class LogSegment(val log: FileMessageSet,
|
||||||
// Get the index entry with a timestamp less than or equal to the target timestamp
|
// Get the index entry with a timestamp less than or equal to the target timestamp
|
||||||
val timestampOffset = timeIndex.lookup(timestamp)
|
val timestampOffset = timeIndex.lookup(timestamp)
|
||||||
val position = index.lookup(timestampOffset.offset).position
|
val position = index.lookup(timestampOffset.offset).position
|
||||||
|
|
||||||
// Search the timestamp
|
// Search the timestamp
|
||||||
log.searchForTimestamp(timestamp, position)
|
Option(log.searchForTimestamp(timestamp, position)).map { timestampAndOffset =>
|
||||||
|
TimestampOffset(timestampAndOffset.timestamp, timestampAndOffset.offset)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -444,3 +454,7 @@ class LogSegment(val log: FileMessageSet,
|
||||||
timeIndex.file.setLastModified(ms)
|
timeIndex.file.setLastModified(ms)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
object LogFlushStats extends KafkaMetricsGroup {
|
||||||
|
val logFlushTimer = new KafkaTimer(newTimer("LogFlushRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS))
|
||||||
|
}
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,239 @@
|
||||||
|
/**
|
||||||
|
* 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 kafka.log
|
||||||
|
|
||||||
|
import java.nio.ByteBuffer
|
||||||
|
|
||||||
|
import kafka.common.LongRef
|
||||||
|
import kafka.message.{CompressionCodec, InvalidMessageException, NoCompressionCodec}
|
||||||
|
import org.apache.kafka.common.errors.InvalidTimestampException
|
||||||
|
import org.apache.kafka.common.record._
|
||||||
|
|
||||||
|
import scala.collection.mutable
|
||||||
|
import scala.collection.JavaConverters._
|
||||||
|
|
||||||
|
private[kafka] object LogValidator {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Update the offsets for this message set and do further validation on messages including:
|
||||||
|
* 1. Messages for compacted topics must have keys
|
||||||
|
* 2. When magic value = 1, inner messages of a compressed message set must have monotonically increasing offsets
|
||||||
|
* starting from 0.
|
||||||
|
* 3. When magic value = 1, validate and maybe overwrite timestamps of messages.
|
||||||
|
*
|
||||||
|
* This method will convert the messages in the following scenarios:
|
||||||
|
* A. Magic value of a message = 0 and messageFormatVersion is 1
|
||||||
|
* B. Magic value of a message = 1 and messageFormatVersion is 0
|
||||||
|
*
|
||||||
|
* If no format conversion or value overwriting is required for messages, this method will perform in-place
|
||||||
|
* operations and avoid re-compression.
|
||||||
|
*
|
||||||
|
* Returns a ValidationAndOffsetAssignResult containing the validated message set, maximum timestamp, the offset
|
||||||
|
* of the shallow message with the max timestamp and a boolean indicating whether the message sizes may have changed.
|
||||||
|
*/
|
||||||
|
private[kafka] def validateMessagesAndAssignOffsets(records: MemoryRecords,
|
||||||
|
offsetCounter: LongRef,
|
||||||
|
now: Long,
|
||||||
|
sourceCodec: CompressionCodec,
|
||||||
|
targetCodec: CompressionCodec,
|
||||||
|
compactedTopic: Boolean = false,
|
||||||
|
messageFormatVersion: Byte = Record.CURRENT_MAGIC_VALUE,
|
||||||
|
messageTimestampType: TimestampType,
|
||||||
|
messageTimestampDiffMaxMs: Long): ValidationAndOffsetAssignResult = {
|
||||||
|
if (sourceCodec == NoCompressionCodec && targetCodec == NoCompressionCodec) {
|
||||||
|
// check the magic value
|
||||||
|
if (!records.hasMatchingShallowMagic(messageFormatVersion))
|
||||||
|
convertAndAssignOffsetsNonCompressed(records, offsetCounter, compactedTopic, now, messageTimestampType,
|
||||||
|
messageTimestampDiffMaxMs, messageFormatVersion)
|
||||||
|
else
|
||||||
|
// Do in-place validation, offset assignment and maybe set timestamp
|
||||||
|
assignOffsetsNonCompressed(records, offsetCounter, now, compactedTopic, messageTimestampType,
|
||||||
|
messageTimestampDiffMaxMs)
|
||||||
|
} else {
|
||||||
|
// Deal with compressed messages
|
||||||
|
// We cannot do in place assignment in one of the following situations:
|
||||||
|
// 1. Source and target compression codec are different
|
||||||
|
// 2. When magic value to use is 0 because offsets need to be overwritten
|
||||||
|
// 3. When magic value to use is above 0, but some fields of inner messages need to be overwritten.
|
||||||
|
// 4. Message format conversion is needed.
|
||||||
|
|
||||||
|
// No in place assignment situation 1 and 2
|
||||||
|
var inPlaceAssignment = sourceCodec == targetCodec && messageFormatVersion > Record.MAGIC_VALUE_V0
|
||||||
|
|
||||||
|
var maxTimestamp = Record.NO_TIMESTAMP
|
||||||
|
val expectedInnerOffset = new LongRef(0)
|
||||||
|
val validatedRecords = new mutable.ArrayBuffer[Record]
|
||||||
|
|
||||||
|
records.deepIterator(true).asScala.foreach { logEntry =>
|
||||||
|
val record = logEntry.record
|
||||||
|
validateKey(record, compactedTopic)
|
||||||
|
|
||||||
|
if (record.magic > Record.MAGIC_VALUE_V0 && messageFormatVersion > Record.MAGIC_VALUE_V0) {
|
||||||
|
// No in place assignment situation 3
|
||||||
|
// Validate the timestamp
|
||||||
|
validateTimestamp(record, now, messageTimestampType, messageTimestampDiffMaxMs)
|
||||||
|
// Check if we need to overwrite offset
|
||||||
|
if (logEntry.offset != expectedInnerOffset.getAndIncrement())
|
||||||
|
inPlaceAssignment = false
|
||||||
|
if (record.timestamp > maxTimestamp)
|
||||||
|
maxTimestamp = record.timestamp
|
||||||
|
}
|
||||||
|
|
||||||
|
if (sourceCodec != NoCompressionCodec && logEntry.isCompressed)
|
||||||
|
throw new InvalidMessageException("Compressed outer record should not have an inner record with a " +
|
||||||
|
s"compression attribute set: $record")
|
||||||
|
|
||||||
|
// No in place assignment situation 4
|
||||||
|
if (record.magic != messageFormatVersion)
|
||||||
|
inPlaceAssignment = false
|
||||||
|
|
||||||
|
validatedRecords += record.convert(messageFormatVersion)
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!inPlaceAssignment) {
|
||||||
|
val entries = validatedRecords.map(record => LogEntry.create(offsetCounter.getAndIncrement(), record))
|
||||||
|
val builder = MemoryRecords.builderWithEntries(messageTimestampType, CompressionType.forId(targetCodec.codec),
|
||||||
|
now, entries.asJava)
|
||||||
|
builder.close()
|
||||||
|
val info = builder.info
|
||||||
|
|
||||||
|
ValidationAndOffsetAssignResult(
|
||||||
|
validatedRecords = builder.build(),
|
||||||
|
maxTimestamp = info.maxTimestamp,
|
||||||
|
shallowOffsetOfMaxTimestamp = info.shallowOffsetOfMaxTimestamp,
|
||||||
|
messageSizeMaybeChanged = true)
|
||||||
|
} else {
|
||||||
|
// ensure the inner messages are valid
|
||||||
|
validatedRecords.foreach(_.ensureValid)
|
||||||
|
|
||||||
|
// we can update the wrapper message only and write the compressed payload as is
|
||||||
|
val entry = records.shallowIterator.next()
|
||||||
|
val offset = offsetCounter.addAndGet(validatedRecords.size) - 1
|
||||||
|
entry.setOffset(offset)
|
||||||
|
if (messageTimestampType == TimestampType.CREATE_TIME)
|
||||||
|
entry.setCreateTime(maxTimestamp)
|
||||||
|
else if (messageTimestampType == TimestampType.LOG_APPEND_TIME)
|
||||||
|
entry.setLogAppendTime(now)
|
||||||
|
|
||||||
|
ValidationAndOffsetAssignResult(validatedRecords = records,
|
||||||
|
maxTimestamp = if (messageTimestampType == TimestampType.LOG_APPEND_TIME) now else maxTimestamp,
|
||||||
|
shallowOffsetOfMaxTimestamp = offset,
|
||||||
|
messageSizeMaybeChanged = false)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private def convertAndAssignOffsetsNonCompressed(records: MemoryRecords,
|
||||||
|
offsetCounter: LongRef,
|
||||||
|
compactedTopic: Boolean,
|
||||||
|
now: Long,
|
||||||
|
timestampType: TimestampType,
|
||||||
|
messageTimestampDiffMaxMs: Long,
|
||||||
|
toMagicValue: Byte): ValidationAndOffsetAssignResult = {
|
||||||
|
val sizeInBytesAfterConversion = records.shallowIterator.asScala.map { logEntry =>
|
||||||
|
logEntry.record.convertedSize(toMagicValue)
|
||||||
|
}.sum
|
||||||
|
|
||||||
|
val newBuffer = ByteBuffer.allocate(sizeInBytesAfterConversion)
|
||||||
|
val builder = MemoryRecords.builder(newBuffer, toMagicValue, CompressionType.NONE, timestampType,
|
||||||
|
offsetCounter.value, now)
|
||||||
|
|
||||||
|
records.shallowIterator.asScala.foreach { logEntry =>
|
||||||
|
val record = logEntry.record
|
||||||
|
validateKey(record, compactedTopic)
|
||||||
|
validateTimestamp(record, now, timestampType, messageTimestampDiffMaxMs)
|
||||||
|
builder.convertAndAppend(offsetCounter.getAndIncrement(), record)
|
||||||
|
}
|
||||||
|
|
||||||
|
builder.close()
|
||||||
|
val info = builder.info
|
||||||
|
|
||||||
|
ValidationAndOffsetAssignResult(
|
||||||
|
validatedRecords = builder.build(),
|
||||||
|
maxTimestamp = info.maxTimestamp,
|
||||||
|
shallowOffsetOfMaxTimestamp = info.shallowOffsetOfMaxTimestamp,
|
||||||
|
messageSizeMaybeChanged = true)
|
||||||
|
}
|
||||||
|
|
||||||
|
private def assignOffsetsNonCompressed(records: MemoryRecords,
|
||||||
|
offsetCounter: LongRef,
|
||||||
|
now: Long,
|
||||||
|
compactedTopic: Boolean,
|
||||||
|
timestampType: TimestampType,
|
||||||
|
timestampDiffMaxMs: Long): ValidationAndOffsetAssignResult = {
|
||||||
|
var maxTimestamp = Record.NO_TIMESTAMP
|
||||||
|
var offsetOfMaxTimestamp = -1L
|
||||||
|
val firstOffset = offsetCounter.value
|
||||||
|
|
||||||
|
for (entry <- records.shallowIterator.asScala) {
|
||||||
|
val record = entry.record
|
||||||
|
validateKey(record, compactedTopic)
|
||||||
|
|
||||||
|
val offset = offsetCounter.getAndIncrement()
|
||||||
|
entry.setOffset(offset)
|
||||||
|
|
||||||
|
if (record.magic > Record.MAGIC_VALUE_V0) {
|
||||||
|
validateTimestamp(record, now, timestampType, timestampDiffMaxMs)
|
||||||
|
|
||||||
|
if (timestampType == TimestampType.LOG_APPEND_TIME)
|
||||||
|
entry.setLogAppendTime(now)
|
||||||
|
else if (record.timestamp > maxTimestamp) {
|
||||||
|
maxTimestamp = record.timestamp
|
||||||
|
offsetOfMaxTimestamp = offset
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (timestampType == TimestampType.LOG_APPEND_TIME) {
|
||||||
|
maxTimestamp = now
|
||||||
|
offsetOfMaxTimestamp = firstOffset
|
||||||
|
}
|
||||||
|
|
||||||
|
ValidationAndOffsetAssignResult(
|
||||||
|
validatedRecords = records,
|
||||||
|
maxTimestamp = maxTimestamp,
|
||||||
|
shallowOffsetOfMaxTimestamp = offsetOfMaxTimestamp,
|
||||||
|
messageSizeMaybeChanged = false)
|
||||||
|
}
|
||||||
|
|
||||||
|
private def validateKey(record: Record, compactedTopic: Boolean) {
|
||||||
|
if (compactedTopic && !record.hasKey)
|
||||||
|
throw new InvalidMessageException("Compacted topic cannot accept message without key.")
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This method validates the timestamps of a message.
|
||||||
|
* If the message is using create time, this method checks if it is within acceptable range.
|
||||||
|
*/
|
||||||
|
private def validateTimestamp(record: Record,
|
||||||
|
now: Long,
|
||||||
|
timestampType: TimestampType,
|
||||||
|
timestampDiffMaxMs: Long) {
|
||||||
|
if (timestampType == TimestampType.CREATE_TIME && math.abs(record.timestamp - now) > timestampDiffMaxMs)
|
||||||
|
throw new InvalidTimestampException(s"Timestamp ${record.timestamp} of message is out of range. " +
|
||||||
|
s"The timestamp should be within [${now - timestampDiffMaxMs}, ${now + timestampDiffMaxMs}")
|
||||||
|
if (record.timestampType == TimestampType.LOG_APPEND_TIME)
|
||||||
|
throw new InvalidTimestampException(s"Invalid timestamp type in message $record. Producer should not set " +
|
||||||
|
s"timestamp type to LogAppendTime.")
|
||||||
|
}
|
||||||
|
|
||||||
|
case class ValidationAndOffsetAssignResult(validatedRecords: MemoryRecords,
|
||||||
|
maxTimestamp: Long,
|
||||||
|
shallowOffsetOfMaxTimestamp: Long,
|
||||||
|
messageSizeMaybeChanged: Boolean)
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -21,9 +21,9 @@ import java.io.File
|
||||||
import java.nio.ByteBuffer
|
import java.nio.ByteBuffer
|
||||||
|
|
||||||
import kafka.common.InvalidOffsetException
|
import kafka.common.InvalidOffsetException
|
||||||
import kafka.message.Message
|
|
||||||
import kafka.utils.CoreUtils._
|
import kafka.utils.CoreUtils._
|
||||||
import kafka.utils.Logging
|
import kafka.utils.Logging
|
||||||
|
import org.apache.kafka.common.record.Record
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* An index that maps from the timestamp to the logical offsets of the messages in a segment. This index might be
|
* An index that maps from the timestamp to the logical offsets of the messages in a segment. This index might be
|
||||||
|
|
@ -69,7 +69,7 @@ class TimeIndex(file: File,
|
||||||
def lastEntry: TimestampOffset = {
|
def lastEntry: TimestampOffset = {
|
||||||
inLock(lock) {
|
inLock(lock) {
|
||||||
_entries match {
|
_entries match {
|
||||||
case 0 => TimestampOffset(Message.NoTimestamp, baseOffset)
|
case 0 => TimestampOffset(Record.NO_TIMESTAMP, baseOffset)
|
||||||
case s => parseEntry(mmap, s - 1).asInstanceOf[TimestampOffset]
|
case s => parseEntry(mmap, s - 1).asInstanceOf[TimestampOffset]
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -145,7 +145,7 @@ class TimeIndex(file: File,
|
||||||
val idx = mmap.duplicate
|
val idx = mmap.duplicate
|
||||||
val slot = indexSlotFor(idx, targetTimestamp, IndexSearchType.KEY)
|
val slot = indexSlotFor(idx, targetTimestamp, IndexSearchType.KEY)
|
||||||
if (slot == -1)
|
if (slot == -1)
|
||||||
TimestampOffset(Message.NoTimestamp, baseOffset)
|
TimestampOffset(Record.NO_TIMESTAMP, baseOffset)
|
||||||
else {
|
else {
|
||||||
val entry = parseEntry(idx, slot).asInstanceOf[TimestampOffset]
|
val entry = parseEntry(idx, slot).asInstanceOf[TimestampOffset]
|
||||||
TimestampOffset(entry.timestamp, entry.offset)
|
TimestampOffset(entry.timestamp, entry.offset)
|
||||||
|
|
|
||||||
|
|
@ -17,20 +17,13 @@
|
||||||
|
|
||||||
package kafka.message
|
package kafka.message
|
||||||
|
|
||||||
import kafka.utils.{CoreUtils, IteratorTemplate, Logging}
|
|
||||||
import kafka.common.{KafkaException, LongRef}
|
|
||||||
import java.nio.ByteBuffer
|
import java.nio.ByteBuffer
|
||||||
import java.nio.channels._
|
|
||||||
import java.io._
|
|
||||||
import java.util.ArrayDeque
|
|
||||||
|
|
||||||
import kafka.message.ByteBufferMessageSet.FilterResult
|
import kafka.common.LongRef
|
||||||
import org.apache.kafka.common.errors.InvalidTimestampException
|
import kafka.utils.Logging
|
||||||
import org.apache.kafka.common.record.{MemoryRecords, TimestampType}
|
import org.apache.kafka.common.record._
|
||||||
import org.apache.kafka.common.utils.Utils
|
|
||||||
|
|
||||||
import scala.collection.mutable
|
import scala.collection.JavaConverters._
|
||||||
import scala.collection.mutable.ArrayBuffer
|
|
||||||
|
|
||||||
object ByteBufferMessageSet {
|
object ByteBufferMessageSet {
|
||||||
|
|
||||||
|
|
@ -41,204 +34,19 @@ object ByteBufferMessageSet {
|
||||||
messages: Message*): ByteBuffer = {
|
messages: Message*): ByteBuffer = {
|
||||||
if (messages.isEmpty)
|
if (messages.isEmpty)
|
||||||
MessageSet.Empty.buffer
|
MessageSet.Empty.buffer
|
||||||
else if (compressionCodec == NoCompressionCodec) {
|
else {
|
||||||
val buffer = ByteBuffer.allocate(MessageSet.messageSetSize(messages))
|
|
||||||
for (message <- messages) writeMessage(buffer, message, offsetAssigner.nextAbsoluteOffset())
|
|
||||||
buffer.rewind()
|
|
||||||
buffer
|
|
||||||
} else {
|
|
||||||
val magicAndTimestamp = wrapperMessageTimestamp match {
|
val magicAndTimestamp = wrapperMessageTimestamp match {
|
||||||
case Some(ts) => MagicAndTimestamp(messages.head.magic, ts)
|
case Some(ts) => MagicAndTimestamp(messages.head.magic, ts)
|
||||||
case None => MessageSet.magicAndLargestTimestamp(messages)
|
case None => MessageSet.magicAndLargestTimestamp(messages)
|
||||||
}
|
}
|
||||||
val (messageWriter, lastOffset) = writeCompressedMessages(compressionCodec, offsetAssigner, magicAndTimestamp,
|
|
||||||
timestampType, messages)
|
val entries = messages.map(message => LogEntry.create(offsetAssigner.nextAbsoluteOffset(), message.asRecord))
|
||||||
val buffer = ByteBuffer.allocate(messageWriter.size + MessageSet.LogOverhead)
|
val builder = MemoryRecords.builderWithEntries(timestampType, CompressionType.forId(compressionCodec.codec),
|
||||||
writeMessage(buffer, messageWriter, lastOffset)
|
magicAndTimestamp.timestamp, entries.asJava)
|
||||||
buffer.rewind()
|
builder.build().buffer
|
||||||
buffer
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Deep iterator that decompresses the message sets and adjusts timestamp and offset if needed. */
|
|
||||||
def deepIterator(wrapperMessageAndOffset: MessageAndOffset, ensureMatchingMagic: Boolean = false): Iterator[MessageAndOffset] = {
|
|
||||||
|
|
||||||
import Message._
|
|
||||||
|
|
||||||
new IteratorTemplate[MessageAndOffset] {
|
|
||||||
|
|
||||||
val MessageAndOffset(wrapperMessage, wrapperMessageOffset) = wrapperMessageAndOffset
|
|
||||||
|
|
||||||
if (wrapperMessage.payload == null)
|
|
||||||
throw new KafkaException(s"Message payload is null: $wrapperMessage")
|
|
||||||
|
|
||||||
val wrapperMessageTimestampOpt: Option[Long] =
|
|
||||||
if (wrapperMessage.magic > MagicValue_V0) Some(wrapperMessage.timestamp) else None
|
|
||||||
val wrapperMessageTimestampTypeOpt: Option[TimestampType] =
|
|
||||||
if (wrapperMessage.magic > MagicValue_V0) Some(wrapperMessage.timestampType) else None
|
|
||||||
|
|
||||||
var lastInnerOffset = -1L
|
|
||||||
|
|
||||||
val messageAndOffsets = {
|
|
||||||
val inputStream = new ByteBufferBackedInputStream(wrapperMessage.payload)
|
|
||||||
val compressed = try {
|
|
||||||
new DataInputStream(CompressionFactory(wrapperMessage.compressionCodec, wrapperMessage.magic, inputStream))
|
|
||||||
} catch {
|
|
||||||
case ioe: IOException =>
|
|
||||||
throw new InvalidMessageException(s"Failed to instantiate input stream compressed with ${wrapperMessage.compressionCodec}", ioe)
|
|
||||||
}
|
|
||||||
|
|
||||||
val innerMessageAndOffsets = new ArrayDeque[MessageAndOffset]()
|
|
||||||
try {
|
|
||||||
while (true)
|
|
||||||
innerMessageAndOffsets.add(readMessageFromStream(compressed))
|
|
||||||
} catch {
|
|
||||||
case _: EOFException =>
|
|
||||||
// we don't do anything at all here, because the finally
|
|
||||||
// will close the compressed input stream, and we simply
|
|
||||||
// want to return the innerMessageAndOffsets
|
|
||||||
case ioe: IOException =>
|
|
||||||
throw new InvalidMessageException(s"Error while reading message from stream compressed with ${wrapperMessage.compressionCodec}", ioe)
|
|
||||||
} finally {
|
|
||||||
CoreUtils.swallow(compressed.close())
|
|
||||||
}
|
|
||||||
|
|
||||||
innerMessageAndOffsets
|
|
||||||
}
|
|
||||||
|
|
||||||
private def readMessageFromStream(compressed: DataInputStream): MessageAndOffset = {
|
|
||||||
val innerOffset = compressed.readLong()
|
|
||||||
val recordSize = compressed.readInt()
|
|
||||||
|
|
||||||
if (recordSize < MinMessageOverhead)
|
|
||||||
throw new InvalidMessageException(s"Message found with corrupt size `$recordSize` in deep iterator")
|
|
||||||
|
|
||||||
// read the record into an intermediate record buffer (i.e. extra copy needed)
|
|
||||||
val bufferArray = new Array[Byte](recordSize)
|
|
||||||
compressed.readFully(bufferArray, 0, recordSize)
|
|
||||||
val buffer = ByteBuffer.wrap(bufferArray)
|
|
||||||
|
|
||||||
// Override the timestamp if necessary
|
|
||||||
val newMessage = new Message(buffer, wrapperMessageTimestampOpt, wrapperMessageTimestampTypeOpt)
|
|
||||||
|
|
||||||
// Due to KAFKA-4298, it is possible for the inner and outer magic values to differ. We ignore
|
|
||||||
// this and depend on the outer message in order to decide how to compute the respective offsets
|
|
||||||
// for the inner messages
|
|
||||||
if (ensureMatchingMagic && newMessage.magic != wrapperMessage.magic)
|
|
||||||
throw new InvalidMessageException(s"Compressed message has magic value ${wrapperMessage.magic} " +
|
|
||||||
s"but inner message has magic value ${newMessage.magic}")
|
|
||||||
|
|
||||||
lastInnerOffset = innerOffset
|
|
||||||
MessageAndOffset(newMessage, innerOffset)
|
|
||||||
}
|
|
||||||
|
|
||||||
override def makeNext(): MessageAndOffset = {
|
|
||||||
messageAndOffsets.pollFirst() match {
|
|
||||||
case null => allDone()
|
|
||||||
case nextMessage@ MessageAndOffset(message, offset) =>
|
|
||||||
if (wrapperMessage.magic > MagicValue_V0) {
|
|
||||||
val relativeOffset = offset - lastInnerOffset
|
|
||||||
val absoluteOffset = wrapperMessageOffset + relativeOffset
|
|
||||||
MessageAndOffset(message, absoluteOffset)
|
|
||||||
} else {
|
|
||||||
nextMessage
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private def writeCompressedMessages(codec: CompressionCodec,
|
|
||||||
offsetAssigner: OffsetAssigner,
|
|
||||||
magicAndTimestamp: MagicAndTimestamp,
|
|
||||||
timestampType: TimestampType,
|
|
||||||
messages: Seq[Message]): (MessageWriter, Long) = {
|
|
||||||
require(codec != NoCompressionCodec, s"compressionCodec must not be $NoCompressionCodec")
|
|
||||||
require(messages.nonEmpty, "cannot write empty compressed message set")
|
|
||||||
|
|
||||||
var offset = -1L
|
|
||||||
val magic = magicAndTimestamp.magic
|
|
||||||
val messageWriter = new MessageWriter(math.min(math.max(MessageSet.messageSetSize(messages) / 2, 1024), 1 << 16))
|
|
||||||
messageWriter.write(
|
|
||||||
codec = codec,
|
|
||||||
timestamp = magicAndTimestamp.timestamp,
|
|
||||||
timestampType = timestampType,
|
|
||||||
magicValue = magic) { outputStream =>
|
|
||||||
|
|
||||||
val output = new DataOutputStream(CompressionFactory(codec, magic, outputStream))
|
|
||||||
try {
|
|
||||||
for (message <- messages) {
|
|
||||||
offset = offsetAssigner.nextAbsoluteOffset()
|
|
||||||
|
|
||||||
if (message.magic != magicAndTimestamp.magic)
|
|
||||||
throw new IllegalArgumentException("Messages in the message set must have same magic value")
|
|
||||||
|
|
||||||
// Use inner offset if magic value is greater than 0
|
|
||||||
val innerOffset = if (magicAndTimestamp.magic > Message.MagicValue_V0)
|
|
||||||
offsetAssigner.toInnerOffset(offset)
|
|
||||||
else
|
|
||||||
offset
|
|
||||||
|
|
||||||
output.writeLong(innerOffset)
|
|
||||||
output.writeInt(message.size)
|
|
||||||
output.write(message.buffer.array, message.buffer.arrayOffset, message.buffer.limit)
|
|
||||||
}
|
|
||||||
} finally {
|
|
||||||
output.close()
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
(messageWriter, offset)
|
|
||||||
}
|
|
||||||
|
|
||||||
private[kafka] def writeCompressedMessages(buffer: ByteBuffer,
|
|
||||||
codec: CompressionCodec,
|
|
||||||
messageAndOffsets: Seq[MessageAndOffset]): Int = {
|
|
||||||
require(codec != NoCompressionCodec, s"compressionCodec must not be $NoCompressionCodec")
|
|
||||||
|
|
||||||
if (messageAndOffsets.isEmpty)
|
|
||||||
0
|
|
||||||
else {
|
|
||||||
val messages = messageAndOffsets.map(_.message)
|
|
||||||
val magicAndTimestamp = MessageSet.magicAndLargestTimestamp(messages)
|
|
||||||
|
|
||||||
// ensure that we use the magic from the first message in the set when writing the wrapper
|
|
||||||
// message in order to fix message sets corrupted by KAFKA-4298
|
|
||||||
val magic = magicAndTimestamp.magic
|
|
||||||
|
|
||||||
val firstMessageAndOffset = messageAndOffsets.head
|
|
||||||
val firstAbsoluteOffset = firstMessageAndOffset.offset
|
|
||||||
val offsetAssigner = OffsetAssigner(firstAbsoluteOffset, magic, messageAndOffsets)
|
|
||||||
val timestampType = firstMessageAndOffset.message.timestampType
|
|
||||||
|
|
||||||
val (messageWriter, lastOffset) = writeCompressedMessages(codec, offsetAssigner, magicAndTimestamp,
|
|
||||||
timestampType, messages)
|
|
||||||
|
|
||||||
writeMessage(buffer, messageWriter, lastOffset)
|
|
||||||
messageWriter.size + MessageSet.LogOverhead
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private[kafka] def writeMessage(buffer: ByteBuffer, message: Message, offset: Long) {
|
|
||||||
buffer.putLong(offset)
|
|
||||||
buffer.putInt(message.size)
|
|
||||||
buffer.put(message.buffer)
|
|
||||||
message.buffer.rewind()
|
|
||||||
}
|
|
||||||
|
|
||||||
private[kafka] def writeMessage(buffer: ByteBuffer, messageWriter: MessageWriter, offset: Long) {
|
|
||||||
buffer.putLong(offset)
|
|
||||||
buffer.putInt(messageWriter.size)
|
|
||||||
messageWriter.writeTo(buffer)
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
case class FilterResult(messagesRead: Int,
|
|
||||||
bytesRead: Int,
|
|
||||||
messagesRetained: Int,
|
|
||||||
bytesRetained: Int,
|
|
||||||
maxTimestamp: Long,
|
|
||||||
offsetOfMaxTimestamp: Long)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private object OffsetAssigner {
|
private object OffsetAssigner {
|
||||||
|
|
@ -246,9 +54,6 @@ private object OffsetAssigner {
|
||||||
def apply(offsetCounter: LongRef, size: Int): OffsetAssigner =
|
def apply(offsetCounter: LongRef, size: Int): OffsetAssigner =
|
||||||
new OffsetAssigner(offsetCounter.value to offsetCounter.addAndGet(size))
|
new OffsetAssigner(offsetCounter.value to offsetCounter.addAndGet(size))
|
||||||
|
|
||||||
def apply(baseOffset: Long, magic: Byte, messageAndOffsets: Seq[MessageAndOffset]): OffsetAssigner =
|
|
||||||
new OffsetAssigner(messageAndOffsets.map(_.offset))
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private class OffsetAssigner(offsets: Seq[Long]) {
|
private class OffsetAssigner(offsets: Seq[Long]) {
|
||||||
|
|
@ -322,7 +127,6 @@ private class OffsetAssigner(offsets: Seq[Long]) {
|
||||||
*
|
*
|
||||||
*/
|
*/
|
||||||
class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Logging {
|
class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Logging {
|
||||||
private var shallowValidByteCount = -1
|
|
||||||
|
|
||||||
private[kafka] def this(compressionCodec: CompressionCodec,
|
private[kafka] def this(compressionCodec: CompressionCodec,
|
||||||
offsetCounter: LongRef,
|
offsetCounter: LongRef,
|
||||||
|
|
@ -354,33 +158,6 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi
|
||||||
|
|
||||||
override def asRecords: MemoryRecords = MemoryRecords.readableRecords(buffer.duplicate())
|
override def asRecords: MemoryRecords = MemoryRecords.readableRecords(buffer.duplicate())
|
||||||
|
|
||||||
private def shallowValidBytes: Int = {
|
|
||||||
if (shallowValidByteCount < 0) {
|
|
||||||
this.shallowValidByteCount = this.internalIterator(isShallow = true).map { messageAndOffset =>
|
|
||||||
MessageSet.entrySize(messageAndOffset.message)
|
|
||||||
}.sum
|
|
||||||
}
|
|
||||||
shallowValidByteCount
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Write the messages in this set to the given channel */
|
|
||||||
def writeFullyTo(channel: GatheringByteChannel): Int = {
|
|
||||||
buffer.mark()
|
|
||||||
var written = 0
|
|
||||||
while (written < sizeInBytes)
|
|
||||||
written += channel.write(buffer)
|
|
||||||
buffer.reset()
|
|
||||||
written
|
|
||||||
}
|
|
||||||
|
|
||||||
override def isMagicValueInAllWrapperMessages(expectedMagicValue: Byte): Boolean = {
|
|
||||||
for (messageAndOffset <- shallowIterator) {
|
|
||||||
if (messageAndOffset.message.magic != expectedMagicValue)
|
|
||||||
return false
|
|
||||||
}
|
|
||||||
true
|
|
||||||
}
|
|
||||||
|
|
||||||
/** default iterator that iterates over decompressed messages */
|
/** default iterator that iterates over decompressed messages */
|
||||||
override def iterator: Iterator[MessageAndOffset] = internalIterator()
|
override def iterator: Iterator[MessageAndOffset] = internalIterator()
|
||||||
|
|
||||||
|
|
@ -388,365 +165,12 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi
|
||||||
def shallowIterator: Iterator[MessageAndOffset] = internalIterator(isShallow = true)
|
def shallowIterator: Iterator[MessageAndOffset] = internalIterator(isShallow = true)
|
||||||
|
|
||||||
/** When flag isShallow is set to be true, we do a shallow iteration: just traverse the first level of messages. **/
|
/** When flag isShallow is set to be true, we do a shallow iteration: just traverse the first level of messages. **/
|
||||||
private def internalIterator(isShallow: Boolean = false, ensureMatchingMagic: Boolean = false): Iterator[MessageAndOffset] = {
|
private def internalIterator(isShallow: Boolean = false): Iterator[MessageAndOffset] = {
|
||||||
new IteratorTemplate[MessageAndOffset] {
|
val entries = if (isShallow)
|
||||||
var topIter = buffer.slice()
|
asRecords.shallowIterator
|
||||||
var innerIter: Iterator[MessageAndOffset] = null
|
else
|
||||||
|
asRecords.deepIterator
|
||||||
def innerDone(): Boolean = (innerIter == null || !innerIter.hasNext)
|
entries.asScala.map(MessageAndOffset.fromLogEntry)
|
||||||
|
|
||||||
def makeNextOuter: MessageAndOffset = {
|
|
||||||
// if there isn't at least an offset and size, we are done
|
|
||||||
if (topIter.remaining < 12)
|
|
||||||
return allDone()
|
|
||||||
val offset = topIter.getLong()
|
|
||||||
val size = topIter.getInt()
|
|
||||||
if(size < Message.MinMessageOverhead)
|
|
||||||
throw new InvalidMessageException("Message found with corrupt size (" + size + ") in shallow iterator")
|
|
||||||
|
|
||||||
// we have an incomplete message
|
|
||||||
if(topIter.remaining < size)
|
|
||||||
return allDone()
|
|
||||||
|
|
||||||
// read the current message and check correctness
|
|
||||||
val message = topIter.slice()
|
|
||||||
message.limit(size)
|
|
||||||
topIter.position(topIter.position + size)
|
|
||||||
val newMessage = new Message(message)
|
|
||||||
if(isShallow) {
|
|
||||||
MessageAndOffset(newMessage, offset)
|
|
||||||
} else {
|
|
||||||
newMessage.compressionCodec match {
|
|
||||||
case NoCompressionCodec =>
|
|
||||||
innerIter = null
|
|
||||||
MessageAndOffset(newMessage, offset)
|
|
||||||
case _ =>
|
|
||||||
innerIter = ByteBufferMessageSet.deepIterator(MessageAndOffset(newMessage, offset), ensureMatchingMagic)
|
|
||||||
if(!innerIter.hasNext)
|
|
||||||
innerIter = null
|
|
||||||
makeNext()
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
override def makeNext(): MessageAndOffset = {
|
|
||||||
if(isShallow){
|
|
||||||
makeNextOuter
|
|
||||||
} else {
|
|
||||||
if(innerDone())
|
|
||||||
makeNextOuter
|
|
||||||
else
|
|
||||||
innerIter.next()
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
def filterInto(buffer: ByteBuffer,
|
|
||||||
filter: MessageAndOffset => Boolean): FilterResult = {
|
|
||||||
var maxTimestamp = Message.NoTimestamp
|
|
||||||
var offsetOfMaxTimestamp = -1L
|
|
||||||
var messagesRead = 0
|
|
||||||
var bytesRead = 0
|
|
||||||
var messagesRetained = 0
|
|
||||||
var bytesRetained = 0
|
|
||||||
|
|
||||||
for (shallowMessageAndOffset <- shallowIterator) {
|
|
||||||
val shallowMessage = shallowMessageAndOffset.message
|
|
||||||
val shallowOffset = shallowMessageAndOffset.offset
|
|
||||||
val size = MessageSet.entrySize(shallowMessageAndOffset.message)
|
|
||||||
|
|
||||||
messagesRead += 1
|
|
||||||
bytesRead += size
|
|
||||||
|
|
||||||
if (shallowMessageAndOffset.message.compressionCodec == NoCompressionCodec) {
|
|
||||||
if (filter(shallowMessageAndOffset)) {
|
|
||||||
ByteBufferMessageSet.writeMessage(buffer, shallowMessage, shallowOffset)
|
|
||||||
messagesRetained += 1
|
|
||||||
bytesRetained += size
|
|
||||||
|
|
||||||
if (shallowMessage.timestamp > maxTimestamp) {
|
|
||||||
maxTimestamp = shallowMessage.timestamp
|
|
||||||
offsetOfMaxTimestamp = shallowOffset
|
|
||||||
}
|
|
||||||
}
|
|
||||||
messagesRead += 1
|
|
||||||
} else {
|
|
||||||
// We use the absolute offset to decide whether to retain the message or not (this is handled by the
|
|
||||||
// deep iterator). Because of KAFKA-4298, we have to allow for the possibility that a previous version
|
|
||||||
// corrupted the log by writing a compressed message set with a wrapper magic value not matching the magic
|
|
||||||
// of the inner messages. This will be fixed as we recopy the messages to the destination segment.
|
|
||||||
|
|
||||||
var writeOriginalMessageSet = true
|
|
||||||
val retainedMessages = ArrayBuffer[MessageAndOffset]()
|
|
||||||
val shallowMagic = shallowMessage.magic
|
|
||||||
|
|
||||||
for (deepMessageAndOffset <- ByteBufferMessageSet.deepIterator(shallowMessageAndOffset)) {
|
|
||||||
messagesRead += 1
|
|
||||||
if (filter(deepMessageAndOffset)) {
|
|
||||||
// Check for log corruption due to KAFKA-4298. If we find it, make sure that we overwrite
|
|
||||||
// the corrupted entry with correct data.
|
|
||||||
if (shallowMagic != deepMessageAndOffset.message.magic)
|
|
||||||
writeOriginalMessageSet = false
|
|
||||||
|
|
||||||
retainedMessages += deepMessageAndOffset
|
|
||||||
// We need the max timestamp and last offset for time index
|
|
||||||
if (deepMessageAndOffset.message.timestamp > maxTimestamp)
|
|
||||||
maxTimestamp = deepMessageAndOffset.message.timestamp
|
|
||||||
}
|
|
||||||
else writeOriginalMessageSet = false
|
|
||||||
}
|
|
||||||
offsetOfMaxTimestamp = if (retainedMessages.nonEmpty) retainedMessages.last.offset else -1L
|
|
||||||
// There are no messages compacted out and no message format conversion, write the original message set back
|
|
||||||
if (writeOriginalMessageSet)
|
|
||||||
ByteBufferMessageSet.writeMessage(buffer, shallowMessage, shallowOffset)
|
|
||||||
else if (retainedMessages.nonEmpty) {
|
|
||||||
val compressedSize = ByteBufferMessageSet.writeCompressedMessages(buffer, shallowMessage.compressionCodec, retainedMessages)
|
|
||||||
messagesRetained += 1
|
|
||||||
bytesRetained += compressedSize
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
FilterResult(messagesRead, bytesRead, messagesRetained, bytesRetained, maxTimestamp, offsetOfMaxTimestamp)
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Update the offsets for this message set and do further validation on messages including:
|
|
||||||
* 1. Messages for compacted topics must have keys
|
|
||||||
* 2. When magic value = 1, inner messages of a compressed message set must have monotonically increasing offsets
|
|
||||||
* starting from 0.
|
|
||||||
* 3. When magic value = 1, validate and maybe overwrite timestamps of messages.
|
|
||||||
*
|
|
||||||
* This method will convert the messages in the following scenarios:
|
|
||||||
* A. Magic value of a message = 0 and messageFormatVersion is 1
|
|
||||||
* B. Magic value of a message = 1 and messageFormatVersion is 0
|
|
||||||
*
|
|
||||||
* If no format conversion or value overwriting is required for messages, this method will perform in-place
|
|
||||||
* operations and avoid re-compression.
|
|
||||||
*
|
|
||||||
* Returns a ValidationAndOffsetAssignResult containing the validated message set, maximum timestamp, the offset
|
|
||||||
* of the shallow message with the max timestamp and a boolean indicating whether the message sizes may have changed.
|
|
||||||
*/
|
|
||||||
private[kafka] def validateMessagesAndAssignOffsets(offsetCounter: LongRef,
|
|
||||||
now: Long,
|
|
||||||
sourceCodec: CompressionCodec,
|
|
||||||
targetCodec: CompressionCodec,
|
|
||||||
compactedTopic: Boolean = false,
|
|
||||||
messageFormatVersion: Byte = Message.CurrentMagicValue,
|
|
||||||
messageTimestampType: TimestampType,
|
|
||||||
messageTimestampDiffMaxMs: Long): ValidationAndOffsetAssignResult = {
|
|
||||||
if (sourceCodec == NoCompressionCodec && targetCodec == NoCompressionCodec) {
|
|
||||||
// check the magic value
|
|
||||||
if (!isMagicValueInAllWrapperMessages(messageFormatVersion))
|
|
||||||
convertNonCompressedMessages(offsetCounter, compactedTopic, now, messageTimestampType, messageTimestampDiffMaxMs,
|
|
||||||
messageFormatVersion)
|
|
||||||
else
|
|
||||||
// Do in-place validation, offset assignment and maybe set timestamp
|
|
||||||
validateNonCompressedMessagesAndAssignOffsetInPlace(offsetCounter, now, compactedTopic, messageTimestampType,
|
|
||||||
messageTimestampDiffMaxMs)
|
|
||||||
} else {
|
|
||||||
// Deal with compressed messages
|
|
||||||
// We cannot do in place assignment in one of the following situations:
|
|
||||||
// 1. Source and target compression codec are different
|
|
||||||
// 2. When magic value to use is 0 because offsets need to be overwritten
|
|
||||||
// 3. When magic value to use is above 0, but some fields of inner messages need to be overwritten.
|
|
||||||
// 4. Message format conversion is needed.
|
|
||||||
|
|
||||||
// No in place assignment situation 1 and 2
|
|
||||||
var inPlaceAssignment = sourceCodec == targetCodec && messageFormatVersion > Message.MagicValue_V0
|
|
||||||
|
|
||||||
var maxTimestamp = Message.NoTimestamp
|
|
||||||
var offsetOfMaxTimestamp = -1L
|
|
||||||
val expectedInnerOffset = new LongRef(0)
|
|
||||||
val validatedMessages = new mutable.ArrayBuffer[Message]
|
|
||||||
|
|
||||||
this.internalIterator(isShallow = false, ensureMatchingMagic = true).foreach { messageAndOffset =>
|
|
||||||
val message = messageAndOffset.message
|
|
||||||
validateMessageKey(message, compactedTopic)
|
|
||||||
|
|
||||||
if (message.magic > Message.MagicValue_V0 && messageFormatVersion > Message.MagicValue_V0) {
|
|
||||||
// No in place assignment situation 3
|
|
||||||
// Validate the timestamp
|
|
||||||
validateTimestamp(message, now, messageTimestampType, messageTimestampDiffMaxMs)
|
|
||||||
// Check if we need to overwrite offset
|
|
||||||
if (messageAndOffset.offset != expectedInnerOffset.getAndIncrement())
|
|
||||||
inPlaceAssignment = false
|
|
||||||
if (message.timestamp > maxTimestamp) {
|
|
||||||
maxTimestamp = message.timestamp
|
|
||||||
offsetOfMaxTimestamp = offsetCounter.value + expectedInnerOffset.value - 1
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
if (sourceCodec != NoCompressionCodec && message.compressionCodec != NoCompressionCodec)
|
|
||||||
throw new InvalidMessageException("Compressed outer message should not have an inner message with a " +
|
|
||||||
s"compression attribute set: $message")
|
|
||||||
|
|
||||||
// No in place assignment situation 4
|
|
||||||
if (message.magic != messageFormatVersion)
|
|
||||||
inPlaceAssignment = false
|
|
||||||
|
|
||||||
validatedMessages += message.toFormatVersion(messageFormatVersion)
|
|
||||||
}
|
|
||||||
|
|
||||||
if (!inPlaceAssignment) {
|
|
||||||
// Cannot do in place assignment.
|
|
||||||
val (largestTimestampOfMessageSet, offsetOfMaxTimestampInMessageSet) = {
|
|
||||||
if (messageFormatVersion == Message.MagicValue_V0)
|
|
||||||
(Some(Message.NoTimestamp), -1L)
|
|
||||||
else if (messageTimestampType == TimestampType.CREATE_TIME)
|
|
||||||
(Some(maxTimestamp), {if (targetCodec == NoCompressionCodec) offsetOfMaxTimestamp else offsetCounter.value + validatedMessages.length - 1})
|
|
||||||
else // Log append time
|
|
||||||
(Some(now), {if (targetCodec == NoCompressionCodec) offsetCounter.value else offsetCounter.value + validatedMessages.length - 1})
|
|
||||||
}
|
|
||||||
|
|
||||||
ValidationAndOffsetAssignResult(validatedMessages = new ByteBufferMessageSet(compressionCodec = targetCodec,
|
|
||||||
offsetCounter = offsetCounter,
|
|
||||||
wrapperMessageTimestamp = largestTimestampOfMessageSet,
|
|
||||||
timestampType = messageTimestampType,
|
|
||||||
messages = validatedMessages: _*),
|
|
||||||
maxTimestamp = largestTimestampOfMessageSet.get,
|
|
||||||
offsetOfMaxTimestamp = offsetOfMaxTimestampInMessageSet,
|
|
||||||
messageSizeMaybeChanged = true)
|
|
||||||
} else {
|
|
||||||
// Do not do re-compression but simply update the offset, timestamp and attributes field of the wrapper message.
|
|
||||||
buffer.putLong(0, offsetCounter.addAndGet(validatedMessages.size) - 1)
|
|
||||||
// validate the messages
|
|
||||||
validatedMessages.foreach(_.ensureValid())
|
|
||||||
|
|
||||||
var crcUpdateNeeded = true
|
|
||||||
val timestampOffset = MessageSet.LogOverhead + Message.TimestampOffset
|
|
||||||
val attributeOffset = MessageSet.LogOverhead + Message.AttributesOffset
|
|
||||||
val timestamp = buffer.getLong(timestampOffset)
|
|
||||||
val attributes = buffer.get(attributeOffset)
|
|
||||||
buffer.putLong(timestampOffset, maxTimestamp)
|
|
||||||
if (messageTimestampType == TimestampType.CREATE_TIME && timestamp == maxTimestamp)
|
|
||||||
// We don't need to recompute crc if the timestamp is not updated.
|
|
||||||
crcUpdateNeeded = false
|
|
||||||
else if (messageTimestampType == TimestampType.LOG_APPEND_TIME) {
|
|
||||||
// Set timestamp type and timestamp
|
|
||||||
buffer.putLong(timestampOffset, now)
|
|
||||||
buffer.put(attributeOffset, messageTimestampType.updateAttributes(attributes))
|
|
||||||
}
|
|
||||||
|
|
||||||
if (crcUpdateNeeded) {
|
|
||||||
// need to recompute the crc value
|
|
||||||
buffer.position(MessageSet.LogOverhead)
|
|
||||||
val wrapperMessage = new Message(buffer.slice())
|
|
||||||
Utils.writeUnsignedInt(buffer, MessageSet.LogOverhead + Message.CrcOffset, wrapperMessage.computeChecksum)
|
|
||||||
}
|
|
||||||
buffer.rewind()
|
|
||||||
// For compressed messages,
|
|
||||||
ValidationAndOffsetAssignResult(validatedMessages = this,
|
|
||||||
maxTimestamp = buffer.getLong(timestampOffset),
|
|
||||||
offsetOfMaxTimestamp = buffer.getLong(0),
|
|
||||||
messageSizeMaybeChanged = false)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// We create this method to avoid a memory copy. It reads from the original message set and directly
|
|
||||||
// writes the converted messages into new message set buffer. Hence we don't need to allocate memory for each
|
|
||||||
// individual message during message format conversion.
|
|
||||||
private def convertNonCompressedMessages(offsetCounter: LongRef,
|
|
||||||
compactedTopic: Boolean,
|
|
||||||
now: Long,
|
|
||||||
timestampType: TimestampType,
|
|
||||||
messageTimestampDiffMaxMs: Long,
|
|
||||||
toMagicValue: Byte): ValidationAndOffsetAssignResult = {
|
|
||||||
val sizeInBytesAfterConversion = shallowValidBytes + this.internalIterator(isShallow = true).map { messageAndOffset =>
|
|
||||||
Message.headerSizeDiff(messageAndOffset.message.magic, toMagicValue)
|
|
||||||
}.sum
|
|
||||||
val newBuffer = ByteBuffer.allocate(sizeInBytesAfterConversion)
|
|
||||||
var newMessagePosition = 0
|
|
||||||
var maxTimestamp = Message.NoTimestamp
|
|
||||||
var offsetOfMaxTimestamp = -1L
|
|
||||||
this.internalIterator(isShallow = true).foreach { case MessageAndOffset(message, _) =>
|
|
||||||
validateMessageKey(message, compactedTopic)
|
|
||||||
validateTimestamp(message, now, timestampType, messageTimestampDiffMaxMs)
|
|
||||||
newBuffer.position(newMessagePosition)
|
|
||||||
newBuffer.putLong(offsetCounter.getAndIncrement())
|
|
||||||
val newMessageSize = message.size + Message.headerSizeDiff(message.magic, toMagicValue)
|
|
||||||
newBuffer.putInt(newMessageSize)
|
|
||||||
val newMessageBuffer = newBuffer.slice()
|
|
||||||
newMessageBuffer.limit(newMessageSize)
|
|
||||||
message.convertToBuffer(toMagicValue, newMessageBuffer, now, timestampType)
|
|
||||||
if (toMagicValue > Message.MagicValue_V0) {
|
|
||||||
val timestamp = newMessageBuffer.getLong(Message.TimestampOffset)
|
|
||||||
if (maxTimestamp < timestamp) {
|
|
||||||
maxTimestamp = timestamp
|
|
||||||
offsetOfMaxTimestamp = offsetCounter.value - 1
|
|
||||||
}
|
|
||||||
}
|
|
||||||
newMessagePosition += MessageSet.LogOverhead + newMessageSize
|
|
||||||
}
|
|
||||||
newBuffer.rewind()
|
|
||||||
new ValidationAndOffsetAssignResult(validatedMessages = new ByteBufferMessageSet(newBuffer),
|
|
||||||
maxTimestamp = maxTimestamp,
|
|
||||||
offsetOfMaxTimestamp = offsetOfMaxTimestamp,
|
|
||||||
messageSizeMaybeChanged = true)
|
|
||||||
}
|
|
||||||
|
|
||||||
private def validateNonCompressedMessagesAndAssignOffsetInPlace(offsetCounter: LongRef,
|
|
||||||
now: Long,
|
|
||||||
compactedTopic: Boolean,
|
|
||||||
timestampType: TimestampType,
|
|
||||||
timestampDiffMaxMs: Long): ValidationAndOffsetAssignResult = {
|
|
||||||
// do in-place validation and offset assignment
|
|
||||||
var messagePosition = 0
|
|
||||||
var maxTimestamp = Message.NoTimestamp
|
|
||||||
var offsetOfMaxTimestamp = -1L
|
|
||||||
buffer.mark()
|
|
||||||
while (messagePosition < sizeInBytes - MessageSet.LogOverhead) {
|
|
||||||
buffer.position(messagePosition)
|
|
||||||
buffer.putLong(offsetCounter.getAndIncrement())
|
|
||||||
val messageSize = buffer.getInt()
|
|
||||||
val messageBuffer = buffer.slice()
|
|
||||||
messageBuffer.limit(messageSize)
|
|
||||||
val message = new Message(messageBuffer)
|
|
||||||
validateMessageKey(message, compactedTopic)
|
|
||||||
if (message.magic > Message.MagicValue_V0) {
|
|
||||||
validateTimestamp(message, now, timestampType, timestampDiffMaxMs)
|
|
||||||
if (timestampType == TimestampType.LOG_APPEND_TIME) {
|
|
||||||
message.buffer.putLong(Message.TimestampOffset, now)
|
|
||||||
message.buffer.put(Message.AttributesOffset, timestampType.updateAttributes(message.attributes))
|
|
||||||
Utils.writeUnsignedInt(message.buffer, Message.CrcOffset, message.computeChecksum)
|
|
||||||
}
|
|
||||||
if (message.timestamp > maxTimestamp) {
|
|
||||||
maxTimestamp = message.timestamp
|
|
||||||
offsetOfMaxTimestamp = offsetCounter.value - 1
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
messagePosition += MessageSet.LogOverhead + messageSize
|
|
||||||
}
|
|
||||||
buffer.reset()
|
|
||||||
ValidationAndOffsetAssignResult(validatedMessages = this,
|
|
||||||
maxTimestamp = maxTimestamp,
|
|
||||||
offsetOfMaxTimestamp = offsetOfMaxTimestamp,
|
|
||||||
messageSizeMaybeChanged = false)
|
|
||||||
}
|
|
||||||
|
|
||||||
private def validateMessageKey(message: Message, compactedTopic: Boolean) {
|
|
||||||
if (compactedTopic && !message.hasKey)
|
|
||||||
throw new InvalidMessageException("Compacted topic cannot accept message without key.")
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* This method validates the timestamps of a message.
|
|
||||||
* If the message is using create time, this method checks if it is within acceptable range.
|
|
||||||
*/
|
|
||||||
private def validateTimestamp(message: Message,
|
|
||||||
now: Long,
|
|
||||||
timestampType: TimestampType,
|
|
||||||
timestampDiffMaxMs: Long) {
|
|
||||||
if (timestampType == TimestampType.CREATE_TIME && math.abs(message.timestamp - now) > timestampDiffMaxMs)
|
|
||||||
throw new InvalidTimestampException(s"Timestamp ${message.timestamp} of message is out of range. " +
|
|
||||||
s"The timestamp should be within [${now - timestampDiffMaxMs}, ${now + timestampDiffMaxMs}")
|
|
||||||
if (message.timestampType == TimestampType.LOG_APPEND_TIME)
|
|
||||||
throw new InvalidTimestampException(s"Invalid timestamp type in message $message. Producer should not set " +
|
|
||||||
s"timestamp type to LogAppendTime.")
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -757,7 +181,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi
|
||||||
/**
|
/**
|
||||||
* The total number of bytes in this message set not including any partial, trailing messages
|
* The total number of bytes in this message set not including any partial, trailing messages
|
||||||
*/
|
*/
|
||||||
def validBytes: Int = shallowValidBytes
|
def validBytes: Int = asRecords.validBytes
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Two message sets are equal if their respective byte buffers are equal
|
* Two message sets are equal if their respective byte buffers are equal
|
||||||
|
|
@ -773,8 +197,3 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi
|
||||||
override def hashCode: Int = buffer.hashCode
|
override def hashCode: Int = buffer.hashCode
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
case class ValidationAndOffsetAssignResult(validatedMessages: ByteBufferMessageSet,
|
|
||||||
maxTimestamp: Long,
|
|
||||||
offsetOfMaxTimestamp: Long,
|
|
||||||
messageSizeMaybeChanged: Boolean)
|
|
||||||
|
|
|
||||||
|
|
@ -19,7 +19,7 @@ package kafka.message
|
||||||
|
|
||||||
import java.nio._
|
import java.nio._
|
||||||
|
|
||||||
import org.apache.kafka.common.record.TimestampType
|
import org.apache.kafka.common.record.{Record, TimestampType}
|
||||||
|
|
||||||
import scala.math._
|
import scala.math._
|
||||||
import kafka.utils._
|
import kafka.utils._
|
||||||
|
|
@ -98,6 +98,11 @@ object Message {
|
||||||
MessageHeaderSizeMap(toMagicValue) - MessageHeaderSizeMap(fromMagicValue)
|
MessageHeaderSizeMap(toMagicValue) - MessageHeaderSizeMap(fromMagicValue)
|
||||||
|
|
||||||
|
|
||||||
|
def fromRecord(record: Record): Message = {
|
||||||
|
val wrapperTimestamp: Option[Long] = if (record.wrapperRecordTimestamp() == null) None else Some(record.wrapperRecordTimestamp())
|
||||||
|
val wrapperTimestampType = Option(record.wrapperRecordTimestampType())
|
||||||
|
new Message(record.buffer, wrapperTimestamp, wrapperTimestampType)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -134,6 +139,15 @@ class Message(val buffer: ByteBuffer,
|
||||||
|
|
||||||
import kafka.message.Message._
|
import kafka.message.Message._
|
||||||
|
|
||||||
|
private[message] def asRecord: Record = {
|
||||||
|
wrapperMessageTimestamp match {
|
||||||
|
case None => new Record(buffer)
|
||||||
|
case Some(timestamp) =>
|
||||||
|
val timestampType = wrapperMessageTimestampType.orNull
|
||||||
|
new Record(buffer, timestamp, timestampType)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A constructor to create a Message
|
* A constructor to create a Message
|
||||||
* @param bytes The payload of the message
|
* @param bytes The payload of the message
|
||||||
|
|
@ -327,52 +341,6 @@ class Message(val buffer: ByteBuffer,
|
||||||
*/
|
*/
|
||||||
def key: ByteBuffer = sliceDelimited(keySizeOffset)
|
def key: ByteBuffer = sliceDelimited(keySizeOffset)
|
||||||
|
|
||||||
/**
|
|
||||||
* convert the message to specified format
|
|
||||||
*/
|
|
||||||
def toFormatVersion(toMagicValue: Byte): Message = {
|
|
||||||
if (magic == toMagicValue)
|
|
||||||
this
|
|
||||||
else {
|
|
||||||
val byteBuffer = ByteBuffer.allocate(size + Message.headerSizeDiff(magic, toMagicValue))
|
|
||||||
// Copy bytes from old messages to new message
|
|
||||||
convertToBuffer(toMagicValue, byteBuffer, Message.NoTimestamp)
|
|
||||||
new Message(byteBuffer)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
def convertToBuffer(toMagicValue: Byte,
|
|
||||||
byteBuffer: ByteBuffer,
|
|
||||||
now: Long,
|
|
||||||
timestampType: TimestampType = wrapperMessageTimestampType.getOrElse(TimestampType.forAttributes(attributes))) {
|
|
||||||
if (byteBuffer.remaining() < size + headerSizeDiff(magic, toMagicValue))
|
|
||||||
throw new IndexOutOfBoundsException("The byte buffer does not have enough capacity to hold new message format " +
|
|
||||||
s"version $toMagicValue")
|
|
||||||
if (toMagicValue == Message.MagicValue_V1) {
|
|
||||||
// Up-conversion, reserve CRC and update magic byte
|
|
||||||
byteBuffer.position(Message.MagicOffset)
|
|
||||||
byteBuffer.put(Message.MagicValue_V1)
|
|
||||||
byteBuffer.put(timestampType.updateAttributes(attributes))
|
|
||||||
// Up-conversion, insert the timestamp field
|
|
||||||
if (timestampType == TimestampType.LOG_APPEND_TIME)
|
|
||||||
byteBuffer.putLong(now)
|
|
||||||
else
|
|
||||||
byteBuffer.putLong(Message.NoTimestamp)
|
|
||||||
byteBuffer.put(buffer.array(), buffer.arrayOffset() + Message.KeySizeOffset_V0, size - Message.KeySizeOffset_V0)
|
|
||||||
} else {
|
|
||||||
// Down-conversion, reserve CRC and update magic byte
|
|
||||||
byteBuffer.position(Message.MagicOffset)
|
|
||||||
byteBuffer.put(Message.MagicValue_V0)
|
|
||||||
byteBuffer.put(TimestampType.CREATE_TIME.updateAttributes(attributes))
|
|
||||||
// Down-conversion, skip the timestamp field
|
|
||||||
byteBuffer.put(buffer.array(), buffer.arrayOffset() + Message.KeySizeOffset_V1, size - Message.KeySizeOffset_V1)
|
|
||||||
}
|
|
||||||
// update crc value
|
|
||||||
val newMessage = new Message(byteBuffer)
|
|
||||||
Utils.writeUnsignedInt(byteBuffer, Message.CrcOffset, newMessage.computeChecksum)
|
|
||||||
byteBuffer.rewind()
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Read a size-delimited byte buffer starting at the given offset
|
* Read a size-delimited byte buffer starting at the given offset
|
||||||
*/
|
*/
|
||||||
|
|
@ -399,7 +367,7 @@ class Message(val buffer: ByteBuffer,
|
||||||
if (timestamp < 0 && timestamp != NoTimestamp)
|
if (timestamp < 0 && timestamp != NoTimestamp)
|
||||||
throw new IllegalArgumentException(s"Invalid message timestamp $timestamp")
|
throw new IllegalArgumentException(s"Invalid message timestamp $timestamp")
|
||||||
if (magic == MagicValue_V0 && timestamp != NoTimestamp)
|
if (magic == MagicValue_V0 && timestamp != NoTimestamp)
|
||||||
throw new IllegalArgumentException(s"Invalid timestamp $timestamp. Timestamp must be ${NoTimestamp} when magic = ${MagicValue_V0}")
|
throw new IllegalArgumentException(s"Invalid timestamp $timestamp. Timestamp must be $NoTimestamp when magic = $MagicValue_V0")
|
||||||
}
|
}
|
||||||
|
|
||||||
override def toString: String = {
|
override def toString: String = {
|
||||||
|
|
|
||||||
|
|
@ -17,6 +17,13 @@
|
||||||
|
|
||||||
package kafka.message
|
package kafka.message
|
||||||
|
|
||||||
|
import org.apache.kafka.common.record.LogEntry
|
||||||
|
|
||||||
|
object MessageAndOffset {
|
||||||
|
def fromLogEntry(logEntry : LogEntry): MessageAndOffset = {
|
||||||
|
MessageAndOffset(Message.fromRecord(logEntry.record), logEntry.offset)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
case class MessageAndOffset(message: Message, offset: Long) {
|
case class MessageAndOffset(message: Message, offset: Long) {
|
||||||
|
|
||||||
|
|
@ -28,9 +35,10 @@ case class MessageAndOffset(message: Message, offset: Long) {
|
||||||
/**
|
/**
|
||||||
* We need to decompress the message, if required, to get the offset of the first uncompressed message.
|
* We need to decompress the message, if required, to get the offset of the first uncompressed message.
|
||||||
*/
|
*/
|
||||||
def firstOffset: Long = message.compressionCodec match {
|
def firstOffset: Long = toLogEntry.firstOffset
|
||||||
case NoCompressionCodec => offset
|
|
||||||
case _ => ByteBufferMessageSet.deepIterator(this).next().offset
|
def toLogEntry: LogEntry = {
|
||||||
|
LogEntry.create(offset, message.asRecord)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -18,7 +18,6 @@
|
||||||
package kafka.message
|
package kafka.message
|
||||||
|
|
||||||
import java.nio._
|
import java.nio._
|
||||||
import java.nio.channels._
|
|
||||||
|
|
||||||
import org.apache.kafka.common.record.Records
|
import org.apache.kafka.common.record.Records
|
||||||
|
|
||||||
|
|
@ -72,11 +71,6 @@ case class MagicAndTimestamp(magic: Byte, timestamp: Long)
|
||||||
*/
|
*/
|
||||||
abstract class MessageSet extends Iterable[MessageAndOffset] {
|
abstract class MessageSet extends Iterable[MessageAndOffset] {
|
||||||
|
|
||||||
/**
|
|
||||||
* Check if all the wrapper messages in the message set have the expected magic value
|
|
||||||
*/
|
|
||||||
def isMagicValueInAllWrapperMessages(expectedMagicValue: Byte): Boolean
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Provides an iterator over the message/offset pairs in this set
|
* Provides an iterator over the message/offset pairs in this set
|
||||||
*/
|
*/
|
||||||
|
|
@ -99,7 +93,7 @@ abstract class MessageSet extends Iterable[MessageAndOffset] {
|
||||||
override def toString: String = {
|
override def toString: String = {
|
||||||
val builder = new StringBuilder()
|
val builder = new StringBuilder()
|
||||||
builder.append(getClass.getSimpleName + "(")
|
builder.append(getClass.getSimpleName + "(")
|
||||||
val iter = this.iterator
|
val iter = this.asRecords.shallowIterator()
|
||||||
var i = 0
|
var i = 0
|
||||||
while(iter.hasNext && i < 100) {
|
while(iter.hasNext && i < 100) {
|
||||||
val message = iter.next
|
val message = iter.next
|
||||||
|
|
|
||||||
|
|
@ -21,7 +21,6 @@ import java.util.concurrent.locks.ReentrantLock
|
||||||
|
|
||||||
import kafka.cluster.BrokerEndPoint
|
import kafka.cluster.BrokerEndPoint
|
||||||
import kafka.consumer.PartitionTopicInfo
|
import kafka.consumer.PartitionTopicInfo
|
||||||
import kafka.message.ByteBufferMessageSet
|
|
||||||
import kafka.utils.{DelayedItem, Pool, ShutdownableThread}
|
import kafka.utils.{DelayedItem, Pool, ShutdownableThread}
|
||||||
import kafka.common.{ClientIdAndBroker, KafkaException}
|
import kafka.common.{ClientIdAndBroker, KafkaException}
|
||||||
import kafka.metrics.KafkaMetricsGroup
|
import kafka.metrics.KafkaMetricsGroup
|
||||||
|
|
@ -38,6 +37,7 @@ import java.util.concurrent.atomic.AtomicLong
|
||||||
import com.yammer.metrics.core.Gauge
|
import com.yammer.metrics.core.Gauge
|
||||||
import org.apache.kafka.common.TopicPartition
|
import org.apache.kafka.common.TopicPartition
|
||||||
import org.apache.kafka.common.internals.PartitionStates
|
import org.apache.kafka.common.internals.PartitionStates
|
||||||
|
import org.apache.kafka.common.record.MemoryRecords
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Abstract class for fetching data from multiple partitions from the same broker.
|
* Abstract class for fetching data from multiple partitions from the same broker.
|
||||||
|
|
@ -144,15 +144,15 @@ abstract class AbstractFetcherThread(name: String,
|
||||||
Errors.forCode(partitionData.errorCode) match {
|
Errors.forCode(partitionData.errorCode) match {
|
||||||
case Errors.NONE =>
|
case Errors.NONE =>
|
||||||
try {
|
try {
|
||||||
val messages = partitionData.toByteBufferMessageSet
|
val records = partitionData.toRecords
|
||||||
val newOffset = messages.shallowIterator.toSeq.lastOption.map(_.nextOffset).getOrElse(
|
val newOffset = records.shallowIterator.asScala.toSeq.lastOption.map(_.nextOffset).getOrElse(
|
||||||
currentPartitionFetchState.offset)
|
currentPartitionFetchState.offset)
|
||||||
|
|
||||||
fetcherLagStats.getAndMaybePut(topic, partitionId).lag = Math.max(0L, partitionData.highWatermark - newOffset)
|
fetcherLagStats.getAndMaybePut(topic, partitionId).lag = Math.max(0L, partitionData.highWatermark - newOffset)
|
||||||
// Once we hand off the partition data to the subclass, we can't mess with it any more in this thread
|
// Once we hand off the partition data to the subclass, we can't mess with it any more in this thread
|
||||||
processPartitionData(topicPartition, currentPartitionFetchState.offset, partitionData)
|
processPartitionData(topicPartition, currentPartitionFetchState.offset, partitionData)
|
||||||
|
|
||||||
val validBytes = messages.validBytes
|
val validBytes = records.validBytes
|
||||||
if (validBytes > 0) {
|
if (validBytes > 0) {
|
||||||
// Update partitionStates only if there is no exception during processPartitionData
|
// Update partitionStates only if there is no exception during processPartitionData
|
||||||
partitionStates.updateAndMoveToEnd(topicPartition, new PartitionFetchState(newOffset))
|
partitionStates.updateAndMoveToEnd(topicPartition, new PartitionFetchState(newOffset))
|
||||||
|
|
@ -260,7 +260,7 @@ object AbstractFetcherThread {
|
||||||
trait PartitionData {
|
trait PartitionData {
|
||||||
def errorCode: Short
|
def errorCode: Short
|
||||||
def exception: Option[Throwable]
|
def exception: Option[Throwable]
|
||||||
def toByteBufferMessageSet: ByteBufferMessageSet
|
def toRecords: MemoryRecords
|
||||||
def highWatermark: Long
|
def highWatermark: Long
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -19,7 +19,6 @@ package kafka.server
|
||||||
|
|
||||||
import java.util.concurrent.TimeUnit
|
import java.util.concurrent.TimeUnit
|
||||||
|
|
||||||
import kafka.api.FetchResponsePartitionData
|
|
||||||
import kafka.common.TopicAndPartition
|
import kafka.common.TopicAndPartition
|
||||||
import kafka.metrics.KafkaMetricsGroup
|
import kafka.metrics.KafkaMetricsGroup
|
||||||
import org.apache.kafka.common.TopicPartition
|
import org.apache.kafka.common.TopicPartition
|
||||||
|
|
@ -59,7 +58,7 @@ class DelayedFetch(delayMs: Long,
|
||||||
fetchMetadata: FetchMetadata,
|
fetchMetadata: FetchMetadata,
|
||||||
replicaManager: ReplicaManager,
|
replicaManager: ReplicaManager,
|
||||||
quota: ReplicaQuota,
|
quota: ReplicaQuota,
|
||||||
responseCallback: Seq[(TopicAndPartition, FetchResponsePartitionData)] => Unit)
|
responseCallback: Seq[(TopicAndPartition, FetchPartitionData)] => Unit)
|
||||||
extends DelayedOperation(delayMs) {
|
extends DelayedOperation(delayMs) {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -152,7 +151,7 @@ class DelayedFetch(delayMs: Long,
|
||||||
)
|
)
|
||||||
|
|
||||||
val fetchPartitionData = logReadResults.map { case (tp, result) =>
|
val fetchPartitionData = logReadResults.map { case (tp, result) =>
|
||||||
tp -> FetchResponsePartitionData(result.errorCode, result.hw, result.info.messageSet)
|
tp -> FetchPartitionData(result.errorCode, result.hw, result.info.records)
|
||||||
}
|
}
|
||||||
|
|
||||||
responseCallback(fetchPartitionData)
|
responseCallback(fetchPartitionData)
|
||||||
|
|
|
||||||
|
|
@ -17,7 +17,8 @@
|
||||||
|
|
||||||
package kafka.server
|
package kafka.server
|
||||||
|
|
||||||
import kafka.message.MessageSet
|
import org.apache.kafka.common.record.Records
|
||||||
|
|
||||||
case class FetchDataInfo(fetchOffsetMetadata: LogOffsetMetadata, messageSet: MessageSet,
|
case class FetchDataInfo(fetchOffsetMetadata: LogOffsetMetadata,
|
||||||
firstMessageSetIncomplete: Boolean = false)
|
records: Records,
|
||||||
|
firstEntryIncomplete: Boolean = false)
|
||||||
|
|
|
||||||
|
|
@ -23,14 +23,13 @@ import java.util.{Collections, Properties}
|
||||||
import java.util
|
import java.util
|
||||||
|
|
||||||
import kafka.admin.{AdminUtils, RackAwareMode}
|
import kafka.admin.{AdminUtils, RackAwareMode}
|
||||||
import kafka.api.{ControlledShutdownRequest, ControlledShutdownResponse, FetchResponsePartitionData}
|
import kafka.api.{ControlledShutdownRequest, ControlledShutdownResponse}
|
||||||
import kafka.cluster.Partition
|
import kafka.cluster.Partition
|
||||||
import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota}
|
import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota}
|
||||||
import kafka.common._
|
import kafka.common._
|
||||||
import kafka.controller.KafkaController
|
import kafka.controller.KafkaController
|
||||||
import kafka.coordinator.{GroupCoordinator, JoinGroupResult}
|
import kafka.coordinator.{GroupCoordinator, JoinGroupResult}
|
||||||
import kafka.log._
|
import kafka.log._
|
||||||
import kafka.message.{ByteBufferMessageSet, Message}
|
|
||||||
import kafka.network._
|
import kafka.network._
|
||||||
import kafka.network.RequestChannel.{Response, Session}
|
import kafka.network.RequestChannel.{Response, Session}
|
||||||
import kafka.security.auth
|
import kafka.security.auth
|
||||||
|
|
@ -39,7 +38,7 @@ import kafka.utils.{Logging, ZKGroupTopicDirs, ZkUtils}
|
||||||
import org.apache.kafka.common.errors.{ClusterAuthorizationException, NotLeaderForPartitionException, TopicExistsException, UnknownTopicOrPartitionException, UnsupportedForMessageFormatException}
|
import org.apache.kafka.common.errors.{ClusterAuthorizationException, NotLeaderForPartitionException, TopicExistsException, UnknownTopicOrPartitionException, UnsupportedForMessageFormatException}
|
||||||
import org.apache.kafka.common.metrics.Metrics
|
import org.apache.kafka.common.metrics.Metrics
|
||||||
import org.apache.kafka.common.protocol.{ApiKeys, Errors, Protocol, SecurityProtocol}
|
import org.apache.kafka.common.protocol.{ApiKeys, Errors, Protocol, SecurityProtocol}
|
||||||
import org.apache.kafka.common.record.MemoryRecords
|
import org.apache.kafka.common.record.{MemoryRecords, Record}
|
||||||
import org.apache.kafka.common.requests._
|
import org.apache.kafka.common.requests._
|
||||||
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
|
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
|
||||||
import org.apache.kafka.common.utils.{Time, Utils}
|
import org.apache.kafka.common.utils.{Time, Utils}
|
||||||
|
|
@ -359,9 +358,9 @@ class KafkaApis(val requestChannel: RequestChannel,
|
||||||
|
|
||||||
val mergedResponseStatus = responseStatus ++
|
val mergedResponseStatus = responseStatus ++
|
||||||
unauthorizedForWriteRequestInfo.mapValues(_ =>
|
unauthorizedForWriteRequestInfo.mapValues(_ =>
|
||||||
new PartitionResponse(Errors.TOPIC_AUTHORIZATION_FAILED.code, -1, Message.NoTimestamp)) ++
|
new PartitionResponse(Errors.TOPIC_AUTHORIZATION_FAILED.code, -1, Record.NO_TIMESTAMP)) ++
|
||||||
nonExistingOrUnauthorizedForDescribeTopics.mapValues(_ =>
|
nonExistingOrUnauthorizedForDescribeTopics.mapValues(_ =>
|
||||||
new PartitionResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION.code, -1, Message.NoTimestamp))
|
new PartitionResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION.code, -1, Record.NO_TIMESTAMP))
|
||||||
|
|
||||||
var errorInResponse = false
|
var errorInResponse = false
|
||||||
|
|
||||||
|
|
@ -422,17 +421,12 @@ class KafkaApis(val requestChannel: RequestChannel,
|
||||||
else {
|
else {
|
||||||
val internalTopicsAllowed = request.header.clientId == AdminUtils.AdminClientId
|
val internalTopicsAllowed = request.header.clientId == AdminUtils.AdminClientId
|
||||||
|
|
||||||
// Convert Records to ByteBufferMessageSet
|
|
||||||
val authorizedMessagesPerPartition = authorizedRequestInfo.map {
|
|
||||||
case (topicPartition, records) => (topicPartition, new ByteBufferMessageSet(records.buffer))
|
|
||||||
}
|
|
||||||
|
|
||||||
// call the replica manager to append messages to the replicas
|
// call the replica manager to append messages to the replicas
|
||||||
replicaManager.appendMessages(
|
replicaManager.appendRecords(
|
||||||
produceRequest.timeout.toLong,
|
produceRequest.timeout.toLong,
|
||||||
produceRequest.acks,
|
produceRequest.acks,
|
||||||
internalTopicsAllowed,
|
internalTopicsAllowed,
|
||||||
authorizedMessagesPerPartition,
|
authorizedRequestInfo,
|
||||||
sendResponseCallback)
|
sendResponseCallback)
|
||||||
|
|
||||||
// if the request is put into the purgatory, it will have a held reference
|
// if the request is put into the purgatory, it will have a held reference
|
||||||
|
|
@ -467,7 +461,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
||||||
}
|
}
|
||||||
|
|
||||||
// the callback for sending a fetch response
|
// the callback for sending a fetch response
|
||||||
def sendResponseCallback(responsePartitionData: Seq[(TopicAndPartition, FetchResponsePartitionData)]) {
|
def sendResponseCallback(responsePartitionData: Seq[(TopicAndPartition, FetchPartitionData)]) {
|
||||||
val convertedPartitionData = {
|
val convertedPartitionData = {
|
||||||
// Need to down-convert message when consumer only takes magic value 0.
|
// Need to down-convert message when consumer only takes magic value 0.
|
||||||
responsePartitionData.map { case (tp, data) =>
|
responsePartitionData.map { case (tp, data) =>
|
||||||
|
|
@ -480,18 +474,18 @@ class KafkaApis(val requestChannel: RequestChannel,
|
||||||
// Please note that if the message format is changed from a higher version back to lower version this
|
// Please note that if the message format is changed from a higher version back to lower version this
|
||||||
// test might break because some messages in new message format can be delivered to consumers before 0.10.0.0
|
// test might break because some messages in new message format can be delivered to consumers before 0.10.0.0
|
||||||
// without format down conversion.
|
// without format down conversion.
|
||||||
val convertedData = if (versionId <= 1 && replicaManager.getMessageFormatVersion(tp).exists(_ > Message.MagicValue_V0) &&
|
val convertedData = if (versionId <= 1 && replicaManager.getMagicAndTimestampType(tp).exists(_._1 > Record.MAGIC_VALUE_V0) &&
|
||||||
!data.messages.isMagicValueInAllWrapperMessages(Message.MagicValue_V0)) {
|
!data.records.hasMatchingShallowMagic(Record.MAGIC_VALUE_V0)) {
|
||||||
trace(s"Down converting message to V0 for fetch request from $clientId")
|
trace(s"Down converting message to V0 for fetch request from $clientId")
|
||||||
new FetchResponsePartitionData(data.error, data.hw, data.messages.asInstanceOf[FileMessageSet].toMessageFormat(Message.MagicValue_V0))
|
FetchPartitionData(data.error, data.hw, data.records.toMessageFormat(Record.MAGIC_VALUE_V0))
|
||||||
} else data
|
} else data
|
||||||
|
|
||||||
val records = convertedData.messages.asRecords
|
new TopicPartition(tp.topic, tp.partition) -> new FetchResponse.PartitionData(convertedData.error, convertedData.hw, convertedData.records)
|
||||||
new TopicPartition(tp.topic, tp.partition) -> new FetchResponse.PartitionData(convertedData.error, convertedData.hw, records)
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
val mergedPartitionData = convertedPartitionData ++ unauthorizedForReadPartitionData ++ nonExistingOrUnauthorizedForDescribePartitionData
|
val mergedPartitionData = convertedPartitionData ++ unauthorizedForReadPartitionData ++ nonExistingOrUnauthorizedForDescribePartitionData
|
||||||
|
|
||||||
val fetchedPartitionData = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData]()
|
val fetchedPartitionData = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData]()
|
||||||
|
|
||||||
mergedPartitionData.foreach { case (topicPartition, data) =>
|
mergedPartitionData.foreach { case (topicPartition, data) =>
|
||||||
|
|
@ -660,7 +654,7 @@ class KafkaApis(val requestChannel: RequestChannel,
|
||||||
|
|
||||||
val found = {
|
val found = {
|
||||||
if (fromConsumer && timestamp == ListOffsetRequest.LATEST_TIMESTAMP)
|
if (fromConsumer && timestamp == ListOffsetRequest.LATEST_TIMESTAMP)
|
||||||
TimestampOffset(Message.NoTimestamp, localReplica.highWatermark.messageOffset)
|
TimestampOffset(Record.NO_TIMESTAMP, localReplica.highWatermark.messageOffset)
|
||||||
else {
|
else {
|
||||||
def allowed(timestampOffset: TimestampOffset): Boolean =
|
def allowed(timestampOffset: TimestampOffset): Boolean =
|
||||||
!fromConsumer || timestampOffset.offset <= localReplica.highWatermark.messageOffset
|
!fromConsumer || timestampOffset.offset <= localReplica.highWatermark.messageOffset
|
||||||
|
|
|
||||||
|
|
@ -23,7 +23,6 @@ import java.util
|
||||||
import kafka.admin.AdminUtils
|
import kafka.admin.AdminUtils
|
||||||
import kafka.cluster.BrokerEndPoint
|
import kafka.cluster.BrokerEndPoint
|
||||||
import kafka.log.LogConfig
|
import kafka.log.LogConfig
|
||||||
import kafka.message.ByteBufferMessageSet
|
|
||||||
import kafka.api.{KAFKA_0_10_0_IV0, KAFKA_0_10_1_IV1, KAFKA_0_10_1_IV2, KAFKA_0_9_0}
|
import kafka.api.{KAFKA_0_10_0_IV0, KAFKA_0_10_1_IV1, KAFKA_0_10_1_IV2, KAFKA_0_9_0}
|
||||||
import kafka.common.{KafkaStorageException, TopicAndPartition}
|
import kafka.common.{KafkaStorageException, TopicAndPartition}
|
||||||
import ReplicaFetcherThread._
|
import ReplicaFetcherThread._
|
||||||
|
|
@ -119,19 +118,19 @@ class ReplicaFetcherThread(name: String,
|
||||||
val topic = topicPartition.topic
|
val topic = topicPartition.topic
|
||||||
val partitionId = topicPartition.partition
|
val partitionId = topicPartition.partition
|
||||||
val replica = replicaMgr.getReplica(topic, partitionId).get
|
val replica = replicaMgr.getReplica(topic, partitionId).get
|
||||||
val messageSet = partitionData.toByteBufferMessageSet
|
val records = partitionData.toRecords
|
||||||
|
|
||||||
maybeWarnIfMessageOversized(messageSet, topicPartition)
|
maybeWarnIfOversizedRecords(records, topicPartition)
|
||||||
|
|
||||||
if (fetchOffset != replica.logEndOffset.messageOffset)
|
if (fetchOffset != replica.logEndOffset.messageOffset)
|
||||||
throw new RuntimeException("Offset mismatch for partition %s: fetched offset = %d, log end offset = %d.".format(topicPartition, fetchOffset, replica.logEndOffset.messageOffset))
|
throw new RuntimeException("Offset mismatch for partition %s: fetched offset = %d, log end offset = %d.".format(topicPartition, fetchOffset, replica.logEndOffset.messageOffset))
|
||||||
if (logger.isTraceEnabled)
|
if (logger.isTraceEnabled)
|
||||||
trace("Follower %d has replica log end offset %d for partition %s. Received %d messages and leader hw %d"
|
trace("Follower %d has replica log end offset %d for partition %s. Received %d messages and leader hw %d"
|
||||||
.format(replica.brokerId, replica.logEndOffset.messageOffset, topicPartition, messageSet.sizeInBytes, partitionData.highWatermark))
|
.format(replica.brokerId, replica.logEndOffset.messageOffset, topicPartition, records.sizeInBytes, partitionData.highWatermark))
|
||||||
replica.log.get.append(messageSet, assignOffsets = false)
|
replica.log.get.append(records, assignOffsets = false)
|
||||||
if (logger.isTraceEnabled)
|
if (logger.isTraceEnabled)
|
||||||
trace("Follower %d has replica log end offset %d after appending %d bytes of messages for partition %s"
|
trace("Follower %d has replica log end offset %d after appending %d bytes of messages for partition %s"
|
||||||
.format(replica.brokerId, replica.logEndOffset.messageOffset, messageSet.sizeInBytes, topicPartition))
|
.format(replica.brokerId, replica.logEndOffset.messageOffset, records.sizeInBytes, topicPartition))
|
||||||
val followerHighWatermark = replica.logEndOffset.messageOffset.min(partitionData.highWatermark)
|
val followerHighWatermark = replica.logEndOffset.messageOffset.min(partitionData.highWatermark)
|
||||||
// for the follower replica, we do not need to keep
|
// for the follower replica, we do not need to keep
|
||||||
// its segment base offset the physical position,
|
// its segment base offset the physical position,
|
||||||
|
|
@ -141,7 +140,7 @@ class ReplicaFetcherThread(name: String,
|
||||||
trace("Follower %d set replica high watermark for partition [%s,%d] to %s"
|
trace("Follower %d set replica high watermark for partition [%s,%d] to %s"
|
||||||
.format(replica.brokerId, topic, partitionId, followerHighWatermark))
|
.format(replica.brokerId, topic, partitionId, followerHighWatermark))
|
||||||
if (quota.isThrottled(TopicAndPartition(topic, partitionId)))
|
if (quota.isThrottled(TopicAndPartition(topic, partitionId)))
|
||||||
quota.record(messageSet.sizeInBytes)
|
quota.record(records.sizeInBytes)
|
||||||
} catch {
|
} catch {
|
||||||
case e: KafkaStorageException =>
|
case e: KafkaStorageException =>
|
||||||
fatal(s"Disk error while replicating data for $topicPartition", e)
|
fatal(s"Disk error while replicating data for $topicPartition", e)
|
||||||
|
|
@ -149,9 +148,9 @@ class ReplicaFetcherThread(name: String,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
def maybeWarnIfMessageOversized(messageSet: ByteBufferMessageSet, topicPartition: TopicPartition): Unit = {
|
def maybeWarnIfOversizedRecords(records: MemoryRecords, topicPartition: TopicPartition): Unit = {
|
||||||
// oversized messages don't cause replication to fail from fetch request version 3 (KIP-74)
|
// oversized messages don't cause replication to fail from fetch request version 3 (KIP-74)
|
||||||
if (fetchRequestVersion <= 2 && messageSet.sizeInBytes > 0 && messageSet.validBytes <= 0)
|
if (fetchRequestVersion <= 2 && records.sizeInBytes > 0 && records.validBytes <= 0)
|
||||||
error(s"Replication is failing due to a message that is greater than replica.fetch.max.bytes for partition $topicPartition. " +
|
error(s"Replication is failing due to a message that is greater than replica.fetch.max.bytes for partition $topicPartition. " +
|
||||||
"This generally occurs when the max.message.bytes has been overridden to exceed this value and a suitably large " +
|
"This generally occurs when the max.message.bytes has been overridden to exceed this value and a suitably large " +
|
||||||
"message has also been sent. To fix this problem increase replica.fetch.max.bytes in your broker config to be " +
|
"message has also been sent. To fix this problem increase replica.fetch.max.bytes in your broker config to be " +
|
||||||
|
|
@ -323,9 +322,8 @@ object ReplicaFetcherThread {
|
||||||
|
|
||||||
def errorCode: Short = underlying.errorCode
|
def errorCode: Short = underlying.errorCode
|
||||||
|
|
||||||
def toByteBufferMessageSet: ByteBufferMessageSet = {
|
def toRecords: MemoryRecords = {
|
||||||
val buffer = underlying.records.asInstanceOf[MemoryRecords].buffer
|
underlying.records.asInstanceOf[MemoryRecords]
|
||||||
new ByteBufferMessageSet(buffer)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
def highWatermark: Long = underlying.highWatermark
|
def highWatermark: Long = underlying.highWatermark
|
||||||
|
|
|
||||||
|
|
@ -26,7 +26,7 @@ import kafka.cluster.{Partition, Replica}
|
||||||
import kafka.common._
|
import kafka.common._
|
||||||
import kafka.controller.KafkaController
|
import kafka.controller.KafkaController
|
||||||
import kafka.log.{LogAppendInfo, LogManager}
|
import kafka.log.{LogAppendInfo, LogManager}
|
||||||
import kafka.message.{ByteBufferMessageSet, InvalidMessageException, Message, MessageSet}
|
import kafka.message.InvalidMessageException
|
||||||
import kafka.metrics.KafkaMetricsGroup
|
import kafka.metrics.KafkaMetricsGroup
|
||||||
import kafka.server.QuotaFactory.UnboundedQuota
|
import kafka.server.QuotaFactory.UnboundedQuota
|
||||||
import kafka.utils._
|
import kafka.utils._
|
||||||
|
|
@ -34,6 +34,7 @@ import org.apache.kafka.common.errors.{ControllerMovedException, CorruptRecordEx
|
||||||
import org.apache.kafka.common.TopicPartition
|
import org.apache.kafka.common.TopicPartition
|
||||||
import org.apache.kafka.common.metrics.Metrics
|
import org.apache.kafka.common.metrics.Metrics
|
||||||
import org.apache.kafka.common.protocol.Errors
|
import org.apache.kafka.common.protocol.Errors
|
||||||
|
import org.apache.kafka.common.record._
|
||||||
import org.apache.kafka.common.requests.{LeaderAndIsrRequest, PartitionState, StopReplicaRequest, UpdateMetadataRequest}
|
import org.apache.kafka.common.requests.{LeaderAndIsrRequest, PartitionState, StopReplicaRequest, UpdateMetadataRequest}
|
||||||
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
|
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
|
||||||
import org.apache.kafka.common.utils.Time
|
import org.apache.kafka.common.utils.Time
|
||||||
|
|
@ -78,9 +79,11 @@ case class LogReadResult(info: FetchDataInfo,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
case class FetchPartitionData(error: Short = Errors.NONE.code, hw: Long = -1L, records: Records)
|
||||||
|
|
||||||
object LogReadResult {
|
object LogReadResult {
|
||||||
val UnknownLogReadResult = LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata,
|
val UnknownLogReadResult = LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata,
|
||||||
MessageSet.Empty),
|
MemoryRecords.EMPTY),
|
||||||
-1L,
|
-1L,
|
||||||
-1,
|
-1,
|
||||||
false)
|
false)
|
||||||
|
|
@ -276,11 +279,7 @@ class ReplicaManager(val config: KafkaConfig,
|
||||||
}
|
}
|
||||||
|
|
||||||
def getPartition(topic: String, partitionId: Int): Option[Partition] = {
|
def getPartition(topic: String, partitionId: Int): Option[Partition] = {
|
||||||
val partition = allPartitions.get((topic, partitionId))
|
Option(allPartitions.get((topic, partitionId)))
|
||||||
if (partition == null)
|
|
||||||
None
|
|
||||||
else
|
|
||||||
Some(partition)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
def getReplicaOrException(topic: String, partition: Int): Replica = {
|
def getReplicaOrException(topic: String, partition: Int): Replica = {
|
||||||
|
|
@ -318,15 +317,15 @@ class ReplicaManager(val config: KafkaConfig,
|
||||||
* Append messages to leader replicas of the partition, and wait for them to be replicated to other replicas;
|
* Append messages to leader replicas of the partition, and wait for them to be replicated to other replicas;
|
||||||
* the callback function will be triggered either when timeout or the required acks are satisfied
|
* the callback function will be triggered either when timeout or the required acks are satisfied
|
||||||
*/
|
*/
|
||||||
def appendMessages(timeout: Long,
|
def appendRecords(timeout: Long,
|
||||||
requiredAcks: Short,
|
requiredAcks: Short,
|
||||||
internalTopicsAllowed: Boolean,
|
internalTopicsAllowed: Boolean,
|
||||||
messagesPerPartition: Map[TopicPartition, MessageSet],
|
entriesPerPartition: Map[TopicPartition, MemoryRecords],
|
||||||
responseCallback: Map[TopicPartition, PartitionResponse] => Unit) {
|
responseCallback: Map[TopicPartition, PartitionResponse] => Unit) {
|
||||||
|
|
||||||
if (isValidRequiredAcks(requiredAcks)) {
|
if (isValidRequiredAcks(requiredAcks)) {
|
||||||
val sTime = time.milliseconds
|
val sTime = time.milliseconds
|
||||||
val localProduceResults = appendToLocalLog(internalTopicsAllowed, messagesPerPartition, requiredAcks)
|
val localProduceResults = appendToLocalLog(internalTopicsAllowed, entriesPerPartition, requiredAcks)
|
||||||
debug("Produce to local log in %d ms".format(time.milliseconds - sTime))
|
debug("Produce to local log in %d ms".format(time.milliseconds - sTime))
|
||||||
|
|
||||||
val produceStatus = localProduceResults.map { case (topicPartition, result) =>
|
val produceStatus = localProduceResults.map { case (topicPartition, result) =>
|
||||||
|
|
@ -336,13 +335,13 @@ class ReplicaManager(val config: KafkaConfig,
|
||||||
new PartitionResponse(result.errorCode, result.info.firstOffset, result.info.logAppendTime)) // response status
|
new PartitionResponse(result.errorCode, result.info.firstOffset, result.info.logAppendTime)) // response status
|
||||||
}
|
}
|
||||||
|
|
||||||
if (delayedRequestRequired(requiredAcks, messagesPerPartition, localProduceResults)) {
|
if (delayedRequestRequired(requiredAcks, entriesPerPartition, localProduceResults)) {
|
||||||
// create delayed produce operation
|
// create delayed produce operation
|
||||||
val produceMetadata = ProduceMetadata(requiredAcks, produceStatus)
|
val produceMetadata = ProduceMetadata(requiredAcks, produceStatus)
|
||||||
val delayedProduce = new DelayedProduce(timeout, produceMetadata, this, responseCallback)
|
val delayedProduce = new DelayedProduce(timeout, produceMetadata, this, responseCallback)
|
||||||
|
|
||||||
// create a list of (topic, partition) pairs to use as keys for this delayed produce operation
|
// create a list of (topic, partition) pairs to use as keys for this delayed produce operation
|
||||||
val producerRequestKeys = messagesPerPartition.keys.map(new TopicPartitionOperationKey(_)).toSeq
|
val producerRequestKeys = entriesPerPartition.keys.map(new TopicPartitionOperationKey(_)).toSeq
|
||||||
|
|
||||||
// try to complete the request immediately, otherwise put it into the purgatory
|
// try to complete the request immediately, otherwise put it into the purgatory
|
||||||
// this is because while the delayed produce operation is being created, new
|
// this is because while the delayed produce operation is being created, new
|
||||||
|
|
@ -357,9 +356,9 @@ class ReplicaManager(val config: KafkaConfig,
|
||||||
} else {
|
} else {
|
||||||
// If required.acks is outside accepted range, something is wrong with the client
|
// If required.acks is outside accepted range, something is wrong with the client
|
||||||
// Just return an error and don't handle the request at all
|
// Just return an error and don't handle the request at all
|
||||||
val responseStatus = messagesPerPartition.map { case (topicAndPartition, _) =>
|
val responseStatus = entriesPerPartition.map { case (topicAndPartition, _) =>
|
||||||
topicAndPartition -> new PartitionResponse(Errors.INVALID_REQUIRED_ACKS.code,
|
topicAndPartition -> new PartitionResponse(Errors.INVALID_REQUIRED_ACKS.code,
|
||||||
LogAppendInfo.UnknownLogAppendInfo.firstOffset, Message.NoTimestamp)
|
LogAppendInfo.UnknownLogAppendInfo.firstOffset, Record.NO_TIMESTAMP)
|
||||||
}
|
}
|
||||||
responseCallback(responseStatus)
|
responseCallback(responseStatus)
|
||||||
}
|
}
|
||||||
|
|
@ -370,11 +369,12 @@ class ReplicaManager(val config: KafkaConfig,
|
||||||
// 1. required acks = -1
|
// 1. required acks = -1
|
||||||
// 2. there is data to append
|
// 2. there is data to append
|
||||||
// 3. at least one partition append was successful (fewer errors than partitions)
|
// 3. at least one partition append was successful (fewer errors than partitions)
|
||||||
private def delayedRequestRequired(requiredAcks: Short, messagesPerPartition: Map[TopicPartition, MessageSet],
|
private def delayedRequestRequired(requiredAcks: Short,
|
||||||
localProduceResults: Map[TopicPartition, LogAppendResult]): Boolean = {
|
entriesPerPartition: Map[TopicPartition, MemoryRecords],
|
||||||
|
localProduceResults: Map[TopicPartition, LogAppendResult]): Boolean = {
|
||||||
requiredAcks == -1 &&
|
requiredAcks == -1 &&
|
||||||
messagesPerPartition.nonEmpty &&
|
entriesPerPartition.nonEmpty &&
|
||||||
localProduceResults.values.count(_.error.isDefined) < messagesPerPartition.size
|
localProduceResults.values.count(_.error.isDefined) < entriesPerPartition.size
|
||||||
}
|
}
|
||||||
|
|
||||||
private def isValidRequiredAcks(requiredAcks: Short): Boolean = {
|
private def isValidRequiredAcks(requiredAcks: Short): Boolean = {
|
||||||
|
|
@ -385,10 +385,10 @@ class ReplicaManager(val config: KafkaConfig,
|
||||||
* Append the messages to the local replica logs
|
* Append the messages to the local replica logs
|
||||||
*/
|
*/
|
||||||
private def appendToLocalLog(internalTopicsAllowed: Boolean,
|
private def appendToLocalLog(internalTopicsAllowed: Boolean,
|
||||||
messagesPerPartition: Map[TopicPartition, MessageSet],
|
entriesPerPartition: Map[TopicPartition, MemoryRecords],
|
||||||
requiredAcks: Short): Map[TopicPartition, LogAppendResult] = {
|
requiredAcks: Short): Map[TopicPartition, LogAppendResult] = {
|
||||||
trace("Append [%s] to local log ".format(messagesPerPartition))
|
trace("Append [%s] to local log ".format(entriesPerPartition))
|
||||||
messagesPerPartition.map { case (topicPartition, messages) =>
|
entriesPerPartition.map { case (topicPartition, records) =>
|
||||||
BrokerTopicStats.getBrokerTopicStats(topicPartition.topic).totalProduceRequestRate.mark()
|
BrokerTopicStats.getBrokerTopicStats(topicPartition.topic).totalProduceRequestRate.mark()
|
||||||
BrokerTopicStats.getBrokerAllTopicsStats().totalProduceRequestRate.mark()
|
BrokerTopicStats.getBrokerAllTopicsStats().totalProduceRequestRate.mark()
|
||||||
|
|
||||||
|
|
@ -402,7 +402,7 @@ class ReplicaManager(val config: KafkaConfig,
|
||||||
val partitionOpt = getPartition(topicPartition.topic, topicPartition.partition)
|
val partitionOpt = getPartition(topicPartition.topic, topicPartition.partition)
|
||||||
val info = partitionOpt match {
|
val info = partitionOpt match {
|
||||||
case Some(partition) =>
|
case Some(partition) =>
|
||||||
partition.appendMessagesToLeader(messages.asInstanceOf[ByteBufferMessageSet], requiredAcks)
|
partition.appendRecordsToLeader(records, requiredAcks)
|
||||||
case None => throw new UnknownTopicOrPartitionException("Partition %s doesn't exist on %d"
|
case None => throw new UnknownTopicOrPartitionException("Partition %s doesn't exist on %d"
|
||||||
.format(topicPartition, localBrokerId))
|
.format(topicPartition, localBrokerId))
|
||||||
}
|
}
|
||||||
|
|
@ -414,13 +414,13 @@ class ReplicaManager(val config: KafkaConfig,
|
||||||
info.lastOffset - info.firstOffset + 1
|
info.lastOffset - info.firstOffset + 1
|
||||||
|
|
||||||
// update stats for successfully appended bytes and messages as bytesInRate and messageInRate
|
// update stats for successfully appended bytes and messages as bytesInRate and messageInRate
|
||||||
BrokerTopicStats.getBrokerTopicStats(topicPartition.topic).bytesInRate.mark(messages.sizeInBytes)
|
BrokerTopicStats.getBrokerTopicStats(topicPartition.topic).bytesInRate.mark(records.sizeInBytes)
|
||||||
BrokerTopicStats.getBrokerAllTopicsStats.bytesInRate.mark(messages.sizeInBytes)
|
BrokerTopicStats.getBrokerAllTopicsStats.bytesInRate.mark(records.sizeInBytes)
|
||||||
BrokerTopicStats.getBrokerTopicStats(topicPartition.topic).messagesInRate.mark(numAppendedMessages)
|
BrokerTopicStats.getBrokerTopicStats(topicPartition.topic).messagesInRate.mark(numAppendedMessages)
|
||||||
BrokerTopicStats.getBrokerAllTopicsStats.messagesInRate.mark(numAppendedMessages)
|
BrokerTopicStats.getBrokerAllTopicsStats.messagesInRate.mark(numAppendedMessages)
|
||||||
|
|
||||||
trace("%d bytes written to log %s-%d beginning at offset %d and ending at offset %d"
|
trace("%d bytes written to log %s-%d beginning at offset %d and ending at offset %d"
|
||||||
.format(messages.sizeInBytes, topicPartition.topic, topicPartition.partition, info.firstOffset, info.lastOffset))
|
.format(records.sizeInBytes, topicPartition.topic, topicPartition.partition, info.firstOffset, info.lastOffset))
|
||||||
(topicPartition, LogAppendResult(info))
|
(topicPartition, LogAppendResult(info))
|
||||||
} catch {
|
} catch {
|
||||||
// NOTE: Failed produce requests metric is not incremented for known exceptions
|
// NOTE: Failed produce requests metric is not incremented for known exceptions
|
||||||
|
|
@ -434,6 +434,7 @@ class ReplicaManager(val config: KafkaConfig,
|
||||||
_: RecordTooLargeException |
|
_: RecordTooLargeException |
|
||||||
_: RecordBatchTooLargeException |
|
_: RecordBatchTooLargeException |
|
||||||
_: CorruptRecordException |
|
_: CorruptRecordException |
|
||||||
|
_: InvalidRecordException |
|
||||||
_: InvalidMessageException |
|
_: InvalidMessageException |
|
||||||
_: InvalidTimestampException) =>
|
_: InvalidTimestampException) =>
|
||||||
(topicPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(e)))
|
(topicPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(e)))
|
||||||
|
|
@ -458,7 +459,7 @@ class ReplicaManager(val config: KafkaConfig,
|
||||||
hardMaxBytesLimit: Boolean,
|
hardMaxBytesLimit: Boolean,
|
||||||
fetchInfos: Seq[(TopicPartition, PartitionData)],
|
fetchInfos: Seq[(TopicPartition, PartitionData)],
|
||||||
quota: ReplicaQuota = UnboundedQuota,
|
quota: ReplicaQuota = UnboundedQuota,
|
||||||
responseCallback: Seq[(TopicAndPartition, FetchResponsePartitionData)] => Unit) {
|
responseCallback: Seq[(TopicAndPartition, FetchPartitionData)] => Unit) {
|
||||||
val isFromFollower = replicaId >= 0
|
val isFromFollower = replicaId >= 0
|
||||||
val fetchOnlyFromLeader: Boolean = replicaId != Request.DebuggingConsumerId
|
val fetchOnlyFromLeader: Boolean = replicaId != Request.DebuggingConsumerId
|
||||||
val fetchOnlyCommitted: Boolean = ! Request.isValidBrokerId(replicaId)
|
val fetchOnlyCommitted: Boolean = ! Request.isValidBrokerId(replicaId)
|
||||||
|
|
@ -480,7 +481,7 @@ class ReplicaManager(val config: KafkaConfig,
|
||||||
|
|
||||||
// check if this fetch request can be satisfied right away
|
// check if this fetch request can be satisfied right away
|
||||||
val logReadResultValues = logReadResults.map { case (_, v) => v }
|
val logReadResultValues = logReadResults.map { case (_, v) => v }
|
||||||
val bytesReadable = logReadResultValues.map(_.info.messageSet.sizeInBytes).sum
|
val bytesReadable = logReadResultValues.map(_.info.records.sizeInBytes).sum
|
||||||
val errorReadingData = logReadResultValues.foldLeft(false) ((errorIncurred, readResult) =>
|
val errorReadingData = logReadResultValues.foldLeft(false) ((errorIncurred, readResult) =>
|
||||||
errorIncurred || (readResult.errorCode != Errors.NONE.code))
|
errorIncurred || (readResult.errorCode != Errors.NONE.code))
|
||||||
|
|
||||||
|
|
@ -490,7 +491,7 @@ class ReplicaManager(val config: KafkaConfig,
|
||||||
// 4) some error happens while reading data
|
// 4) some error happens while reading data
|
||||||
if (timeout <= 0 || fetchInfos.isEmpty || bytesReadable >= fetchMinBytes || errorReadingData) {
|
if (timeout <= 0 || fetchInfos.isEmpty || bytesReadable >= fetchMinBytes || errorReadingData) {
|
||||||
val fetchPartitionData = logReadResults.map { case (tp, result) =>
|
val fetchPartitionData = logReadResults.map { case (tp, result) =>
|
||||||
tp -> FetchResponsePartitionData(result.errorCode, result.hw, result.info.messageSet)
|
tp -> FetchPartitionData(result.errorCode, result.hw, result.info.records)
|
||||||
}
|
}
|
||||||
responseCallback(fetchPartitionData)
|
responseCallback(fetchPartitionData)
|
||||||
} else {
|
} else {
|
||||||
|
|
@ -568,16 +569,16 @@ class ReplicaManager(val config: KafkaConfig,
|
||||||
|
|
||||||
// If the partition is being throttled, simply return an empty set.
|
// If the partition is being throttled, simply return an empty set.
|
||||||
if (shouldLeaderThrottle(quota, TopicAndPartition(tp.topic, tp.partition), replicaId))
|
if (shouldLeaderThrottle(quota, TopicAndPartition(tp.topic, tp.partition), replicaId))
|
||||||
FetchDataInfo(fetch.fetchOffsetMetadata, MessageSet.Empty)
|
FetchDataInfo(fetch.fetchOffsetMetadata, MemoryRecords.EMPTY)
|
||||||
// For FetchRequest version 3, we replace incomplete message sets with an empty one as consumers can make
|
// For FetchRequest version 3, we replace incomplete message sets with an empty one as consumers can make
|
||||||
// progress in such cases and don't need to report a `RecordTooLargeException`
|
// progress in such cases and don't need to report a `RecordTooLargeException`
|
||||||
else if (!hardMaxBytesLimit && fetch.firstMessageSetIncomplete)
|
else if (!hardMaxBytesLimit && fetch.firstEntryIncomplete)
|
||||||
FetchDataInfo(fetch.fetchOffsetMetadata, MessageSet.Empty)
|
FetchDataInfo(fetch.fetchOffsetMetadata, MemoryRecords.EMPTY)
|
||||||
else fetch
|
else fetch
|
||||||
|
|
||||||
case None =>
|
case None =>
|
||||||
error(s"Leader for partition $tp does not have a local log")
|
error(s"Leader for partition $tp does not have a local log")
|
||||||
FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty)
|
FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MemoryRecords.EMPTY)
|
||||||
}
|
}
|
||||||
|
|
||||||
val readToEndOfLog = initialLogEndOffset.messageOffset - logReadInfo.fetchOffsetMetadata.messageOffset <= 0
|
val readToEndOfLog = initialLogEndOffset.messageOffset - logReadInfo.fetchOffsetMetadata.messageOffset <= 0
|
||||||
|
|
@ -590,12 +591,14 @@ class ReplicaManager(val config: KafkaConfig,
|
||||||
_: NotLeaderForPartitionException |
|
_: NotLeaderForPartitionException |
|
||||||
_: ReplicaNotAvailableException |
|
_: ReplicaNotAvailableException |
|
||||||
_: OffsetOutOfRangeException) =>
|
_: OffsetOutOfRangeException) =>
|
||||||
LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, partitionFetchSize, false, Some(e))
|
LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MemoryRecords.EMPTY), -1L,
|
||||||
|
partitionFetchSize, false, Some(e))
|
||||||
case e: Throwable =>
|
case e: Throwable =>
|
||||||
BrokerTopicStats.getBrokerTopicStats(topic).failedFetchRequestRate.mark()
|
BrokerTopicStats.getBrokerTopicStats(topic).failedFetchRequestRate.mark()
|
||||||
BrokerTopicStats.getBrokerAllTopicsStats().failedFetchRequestRate.mark()
|
BrokerTopicStats.getBrokerAllTopicsStats().failedFetchRequestRate.mark()
|
||||||
error(s"Error processing fetch operation on partition ${tp}, offset $offset", e)
|
error(s"Error processing fetch operation on partition $tp, offset $offset", e)
|
||||||
LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, partitionFetchSize, false, Some(e))
|
LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MemoryRecords.EMPTY), -1L,
|
||||||
|
partitionFetchSize, false, Some(e))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -604,7 +607,7 @@ class ReplicaManager(val config: KafkaConfig,
|
||||||
var minOneMessage = !hardMaxBytesLimit
|
var minOneMessage = !hardMaxBytesLimit
|
||||||
readPartitionInfo.foreach { case (tp, fetchInfo) =>
|
readPartitionInfo.foreach { case (tp, fetchInfo) =>
|
||||||
val readResult = read(tp, fetchInfo, limitBytes, minOneMessage)
|
val readResult = read(tp, fetchInfo, limitBytes, minOneMessage)
|
||||||
val messageSetSize = readResult.info.messageSet.sizeInBytes
|
val messageSetSize = readResult.info.records.sizeInBytes
|
||||||
// Once we read from a non-empty partition, we stop ignoring request and partition level size limits
|
// Once we read from a non-empty partition, we stop ignoring request and partition level size limits
|
||||||
if (messageSetSize > 0)
|
if (messageSetSize > 0)
|
||||||
minOneMessage = false
|
minOneMessage = false
|
||||||
|
|
@ -625,9 +628,9 @@ class ReplicaManager(val config: KafkaConfig,
|
||||||
quota.isThrottled(topicPartition) && quota.isQuotaExceeded && !isReplicaInSync
|
quota.isThrottled(topicPartition) && quota.isQuotaExceeded && !isReplicaInSync
|
||||||
}
|
}
|
||||||
|
|
||||||
def getMessageFormatVersion(topicAndPartition: TopicAndPartition): Option[Byte] =
|
def getMagicAndTimestampType(topicAndPartition: TopicAndPartition): Option[(Byte, TimestampType)] =
|
||||||
getReplica(topicAndPartition.topic, topicAndPartition.partition).flatMap { replica =>
|
getReplica(topicAndPartition.topic, topicAndPartition.partition).flatMap { replica =>
|
||||||
replica.log.map(_.config.messageFormatVersion.messageFormatVersion)
|
replica.log.map(log => (log.config.messageFormatVersion.messageFormatVersion, log.config.messageTimestampType))
|
||||||
}
|
}
|
||||||
|
|
||||||
def maybeUpdateMetadataCache(correlationId: Int, updateMetadataRequest: UpdateMetadataRequest, metadataCache: MetadataCache) {
|
def maybeUpdateMetadataCache(correlationId: Int, updateMetadataRequest: UpdateMetadataRequest, metadataCache: MetadataCache) {
|
||||||
|
|
|
||||||
|
|
@ -23,15 +23,16 @@ import java.nio.ByteBuffer
|
||||||
import joptsimple.OptionParser
|
import joptsimple.OptionParser
|
||||||
import kafka.coordinator.{GroupMetadataKey, GroupMetadataManager, OffsetKey}
|
import kafka.coordinator.{GroupMetadataKey, GroupMetadataManager, OffsetKey}
|
||||||
import kafka.log._
|
import kafka.log._
|
||||||
import kafka.message._
|
|
||||||
import kafka.serializer.Decoder
|
import kafka.serializer.Decoder
|
||||||
import kafka.utils._
|
import kafka.utils._
|
||||||
import org.apache.kafka.clients.consumer.internals.ConsumerProtocol
|
import org.apache.kafka.clients.consumer.internals.ConsumerProtocol
|
||||||
import org.apache.kafka.common.KafkaException
|
import org.apache.kafka.common.KafkaException
|
||||||
|
import org.apache.kafka.common.record.{CompressionType, FileRecords, LogEntry, Record}
|
||||||
import org.apache.kafka.common.utils.Utils
|
import org.apache.kafka.common.utils.Utils
|
||||||
|
|
||||||
import scala.collection.mutable
|
import scala.collection.mutable
|
||||||
import scala.collection.mutable.ArrayBuffer
|
import scala.collection.mutable.ArrayBuffer
|
||||||
|
import scala.collection.JavaConverters._
|
||||||
|
|
||||||
object DumpLogSegments {
|
object DumpLogSegments {
|
||||||
|
|
||||||
|
|
@ -69,7 +70,7 @@ object DumpLogSegments {
|
||||||
|
|
||||||
CommandLineUtils.checkRequiredArgs(parser, options, filesOpt)
|
CommandLineUtils.checkRequiredArgs(parser, options, filesOpt)
|
||||||
|
|
||||||
val printDataLog = (options.has(printOpt) || options.has(offsetsOpt) || options.has(valueDecoderOpt) || options.has(keyDecoderOpt))
|
val printDataLog = options.has(printOpt) || options.has(offsetsOpt) || options.has(valueDecoderOpt) || options.has(keyDecoderOpt)
|
||||||
val verifyOnly = options.has(verifyOpt)
|
val verifyOnly = options.has(verifyOpt)
|
||||||
val indexSanityOnly = options.has(indexSanityOpt)
|
val indexSanityOnly = options.has(indexSanityOpt)
|
||||||
|
|
||||||
|
|
@ -132,7 +133,7 @@ object DumpLogSegments {
|
||||||
maxMessageSize: Int) {
|
maxMessageSize: Int) {
|
||||||
val startOffset = file.getName().split("\\.")(0).toLong
|
val startOffset = file.getName().split("\\.")(0).toLong
|
||||||
val logFile = new File(file.getAbsoluteFile.getParent, file.getName.split("\\.")(0) + Log.LogFileSuffix)
|
val logFile = new File(file.getAbsoluteFile.getParent, file.getName.split("\\.")(0) + Log.LogFileSuffix)
|
||||||
val messageSet = new FileMessageSet(logFile, false)
|
val fileRecords = FileRecords.open(logFile, false)
|
||||||
val index = new OffsetIndex(file, baseOffset = startOffset)
|
val index = new OffsetIndex(file, baseOffset = startOffset)
|
||||||
|
|
||||||
//Check that index passes sanityCheck, this is the check that determines if indexes will be rebuilt on startup or not.
|
//Check that index passes sanityCheck, this is the check that determines if indexes will be rebuilt on startup or not.
|
||||||
|
|
@ -144,11 +145,11 @@ object DumpLogSegments {
|
||||||
|
|
||||||
for(i <- 0 until index.entries) {
|
for(i <- 0 until index.entries) {
|
||||||
val entry = index.entry(i)
|
val entry = index.entry(i)
|
||||||
val partialFileMessageSet: FileMessageSet = messageSet.read(entry.position, maxMessageSize)
|
val slice = fileRecords.read(entry.position, maxMessageSize)
|
||||||
val messageAndOffset = getIterator(partialFileMessageSet.head, isDeepIteration = true).next()
|
val logEntry = getIterator(slice.shallowIterator.next, isDeepIteration = true).next()
|
||||||
if(messageAndOffset.offset != entry.offset + index.baseOffset) {
|
if (logEntry.offset != entry.offset + index.baseOffset) {
|
||||||
var misMatchesSeq = misMatchesForIndexFilesMap.getOrElse(file.getAbsolutePath, List[(Long, Long)]())
|
var misMatchesSeq = misMatchesForIndexFilesMap.getOrElse(file.getAbsolutePath, List[(Long, Long)]())
|
||||||
misMatchesSeq ::=(entry.offset + index.baseOffset, messageAndOffset.offset)
|
misMatchesSeq ::=(entry.offset + index.baseOffset, logEntry.offset)
|
||||||
misMatchesForIndexFilesMap.put(file.getAbsolutePath, misMatchesSeq)
|
misMatchesForIndexFilesMap.put(file.getAbsolutePath, misMatchesSeq)
|
||||||
}
|
}
|
||||||
// since it is a sparse file, in the event of a crash there may be many zero entries, stop if we see one
|
// since it is a sparse file, in the event of a crash there may be many zero entries, stop if we see one
|
||||||
|
|
@ -164,9 +165,9 @@ object DumpLogSegments {
|
||||||
verifyOnly: Boolean,
|
verifyOnly: Boolean,
|
||||||
timeIndexDumpErrors: TimeIndexDumpErrors,
|
timeIndexDumpErrors: TimeIndexDumpErrors,
|
||||||
maxMessageSize: Int) {
|
maxMessageSize: Int) {
|
||||||
val startOffset = file.getName().split("\\.")(0).toLong
|
val startOffset = file.getName.split("\\.")(0).toLong
|
||||||
val logFile = new File(file.getAbsoluteFile.getParent, file.getName.split("\\.")(0) + Log.LogFileSuffix)
|
val logFile = new File(file.getAbsoluteFile.getParent, file.getName.split("\\.")(0) + Log.LogFileSuffix)
|
||||||
val messageSet = new FileMessageSet(logFile, false)
|
val fileRecords = FileRecords.open(logFile, false)
|
||||||
val indexFile = new File(file.getAbsoluteFile.getParent, file.getName.split("\\.")(0) + Log.IndexFileSuffix)
|
val indexFile = new File(file.getAbsoluteFile.getParent, file.getName.split("\\.")(0) + Log.IndexFileSuffix)
|
||||||
val index = new OffsetIndex(indexFile, baseOffset = startOffset)
|
val index = new OffsetIndex(indexFile, baseOffset = startOffset)
|
||||||
val timeIndex = new TimeIndex(file, baseOffset = startOffset)
|
val timeIndex = new TimeIndex(file, baseOffset = startOffset)
|
||||||
|
|
@ -178,26 +179,26 @@ object DumpLogSegments {
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
|
|
||||||
var prevTimestamp = Message.NoTimestamp
|
var prevTimestamp = Record.NO_TIMESTAMP
|
||||||
for(i <- 0 until timeIndex.entries) {
|
for(i <- 0 until timeIndex.entries) {
|
||||||
val entry = timeIndex.entry(i)
|
val entry = timeIndex.entry(i)
|
||||||
val position = index.lookup(entry.offset + timeIndex.baseOffset).position
|
val position = index.lookup(entry.offset + timeIndex.baseOffset).position
|
||||||
val partialFileMessageSet: FileMessageSet = messageSet.read(position, Int.MaxValue)
|
val partialFileRecords = fileRecords.read(position, Int.MaxValue)
|
||||||
val shallowIter = partialFileMessageSet.iterator
|
val shallowEntries = partialFileRecords.shallowIterator.asScala
|
||||||
var maxTimestamp = Message.NoTimestamp
|
var maxTimestamp = Record.NO_TIMESTAMP
|
||||||
// We first find the message by offset then check if the timestamp is correct.
|
// We first find the message by offset then check if the timestamp is correct.
|
||||||
val wrapperMessageOpt = shallowIter.find(_.offset >= entry.offset + timeIndex.baseOffset)
|
val maybeLogEntry = shallowEntries.find(_.offset >= entry.offset + timeIndex.baseOffset)
|
||||||
wrapperMessageOpt match {
|
maybeLogEntry match {
|
||||||
case None =>
|
case None =>
|
||||||
timeIndexDumpErrors.recordShallowOffsetNotFound(file, entry.offset + timeIndex.baseOffset,
|
timeIndexDumpErrors.recordShallowOffsetNotFound(file, entry.offset + timeIndex.baseOffset,
|
||||||
-1.toLong)
|
-1.toLong)
|
||||||
case Some(wrapperMessage) if wrapperMessage.offset != entry.offset + timeIndex.baseOffset =>
|
case Some(logEntry) if logEntry.offset != entry.offset + timeIndex.baseOffset =>
|
||||||
timeIndexDumpErrors.recordShallowOffsetNotFound(file, entry.offset + timeIndex.baseOffset,
|
timeIndexDumpErrors.recordShallowOffsetNotFound(file, entry.offset + timeIndex.baseOffset,
|
||||||
wrapperMessage.offset)
|
logEntry.offset)
|
||||||
case Some(wrapperMessage) =>
|
case Some(shallowLogEntry) =>
|
||||||
val deepIter = getIterator(wrapperMessage, isDeepIteration = true)
|
val deepIter = getIterator(shallowLogEntry, isDeepIteration = true)
|
||||||
for (messageAndOffset <- deepIter)
|
for (deepLogEntry <- deepIter)
|
||||||
maxTimestamp = math.max(maxTimestamp, messageAndOffset.message.timestamp)
|
maxTimestamp = math.max(maxTimestamp, deepLogEntry.record.timestamp)
|
||||||
|
|
||||||
if (maxTimestamp != entry.timestamp)
|
if (maxTimestamp != entry.timestamp)
|
||||||
timeIndexDumpErrors.recordMismatchTimeIndex(file, entry.timestamp, maxTimestamp)
|
timeIndexDumpErrors.recordMismatchTimeIndex(file, entry.timestamp, maxTimestamp)
|
||||||
|
|
@ -216,20 +217,20 @@ object DumpLogSegments {
|
||||||
}
|
}
|
||||||
|
|
||||||
private trait MessageParser[K, V] {
|
private trait MessageParser[K, V] {
|
||||||
def parse(message: Message): (Option[K], Option[V])
|
def parse(record: Record): (Option[K], Option[V])
|
||||||
}
|
}
|
||||||
|
|
||||||
private class DecoderMessageParser[K, V](keyDecoder: Decoder[K], valueDecoder: Decoder[V]) extends MessageParser[K, V] {
|
private class DecoderMessageParser[K, V](keyDecoder: Decoder[K], valueDecoder: Decoder[V]) extends MessageParser[K, V] {
|
||||||
override def parse(message: Message): (Option[K], Option[V]) = {
|
override def parse(record: Record): (Option[K], Option[V]) = {
|
||||||
if (message.isNull) {
|
if (record.hasNullValue) {
|
||||||
(None, None)
|
(None, None)
|
||||||
} else {
|
} else {
|
||||||
val key = if (message.hasKey)
|
val key = if (record.hasKey)
|
||||||
Some(keyDecoder.fromBytes(Utils.readBytes(message.key)))
|
Some(keyDecoder.fromBytes(Utils.readBytes(record.key)))
|
||||||
else
|
else
|
||||||
None
|
None
|
||||||
|
|
||||||
val payload = Some(valueDecoder.fromBytes(Utils.readBytes(message.payload)))
|
val payload = Some(valueDecoder.fromBytes(Utils.readBytes(record.value)))
|
||||||
|
|
||||||
(key, payload)
|
(key, payload)
|
||||||
}
|
}
|
||||||
|
|
@ -249,7 +250,7 @@ object DumpLogSegments {
|
||||||
val topicPartition = offsetKey.key.topicPartition
|
val topicPartition = offsetKey.key.topicPartition
|
||||||
val offset = GroupMetadataManager.readOffsetMessageValue(payload)
|
val offset = GroupMetadataManager.readOffsetMessageValue(payload)
|
||||||
|
|
||||||
val keyString = s"offset::${group}:${topicPartition.topic}:${topicPartition.partition}"
|
val keyString = s"offset::$group:${topicPartition.topic}:${topicPartition.partition}"
|
||||||
val valueString = if (offset.metadata.isEmpty)
|
val valueString = if (offset.metadata.isEmpty)
|
||||||
String.valueOf(offset.offset)
|
String.valueOf(offset.offset)
|
||||||
else
|
else
|
||||||
|
|
@ -271,27 +272,27 @@ object DumpLogSegments {
|
||||||
if (userData.isEmpty)
|
if (userData.isEmpty)
|
||||||
s"${member.memberId}=${partitionAssignment.partitions()}"
|
s"${member.memberId}=${partitionAssignment.partitions()}"
|
||||||
else
|
else
|
||||||
s"${member.memberId}=${partitionAssignment.partitions()}:${userData}"
|
s"${member.memberId}=${partitionAssignment.partitions()}:$userData"
|
||||||
} else {
|
} else {
|
||||||
s"${member.memberId}=${hex(member.assignment)}"
|
s"${member.memberId}=${hex(member.assignment)}"
|
||||||
}
|
}
|
||||||
}.mkString("{", ",", "}")
|
}.mkString("{", ",", "}")
|
||||||
|
|
||||||
val keyString = s"metadata::${groupId}"
|
val keyString = s"metadata::$groupId"
|
||||||
val valueString = s"${protocolType}:${group.protocol}:${group.generationId}:${assignment}"
|
val valueString = s"$protocolType:${group.protocol}:${group.generationId}:$assignment"
|
||||||
|
|
||||||
(Some(keyString), Some(valueString))
|
(Some(keyString), Some(valueString))
|
||||||
}
|
}
|
||||||
|
|
||||||
override def parse(message: Message): (Option[String], Option[String]) = {
|
override def parse(record: Record): (Option[String], Option[String]) = {
|
||||||
if (message.isNull)
|
if (record.hasNullValue)
|
||||||
(None, None)
|
(None, None)
|
||||||
else if (!message.hasKey) {
|
else if (!record.hasKey) {
|
||||||
throw new KafkaException("Failed to decode message using offset topic decoder (message had a missing key)")
|
throw new KafkaException("Failed to decode message using offset topic decoder (message had a missing key)")
|
||||||
} else {
|
} else {
|
||||||
GroupMetadataManager.readMessageKey(message.key) match {
|
GroupMetadataManager.readMessageKey(record.key) match {
|
||||||
case offsetKey: OffsetKey => parseOffsets(offsetKey, message.payload)
|
case offsetKey: OffsetKey => parseOffsets(offsetKey, record.value)
|
||||||
case groupMetadataKey: GroupMetadataKey => parseGroupMetadata(groupMetadataKey, message.payload)
|
case groupMetadataKey: GroupMetadataKey => parseGroupMetadata(groupMetadataKey, record.value)
|
||||||
case _ => throw new KafkaException("Failed to decode message using offset topic decoder (message had an invalid key)")
|
case _ => throw new KafkaException("Failed to decode message using offset topic decoder (message had an invalid key)")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -307,70 +308,51 @@ object DumpLogSegments {
|
||||||
parser: MessageParser[_, _]) {
|
parser: MessageParser[_, _]) {
|
||||||
val startOffset = file.getName().split("\\.")(0).toLong
|
val startOffset = file.getName().split("\\.")(0).toLong
|
||||||
println("Starting offset: " + startOffset)
|
println("Starting offset: " + startOffset)
|
||||||
val messageSet = new FileMessageSet(file, false)
|
val messageSet = FileRecords.open(file, false)
|
||||||
var validBytes = 0L
|
var validBytes = 0L
|
||||||
var lastOffset = -1l
|
var lastOffset = -1l
|
||||||
val shallowIterator = messageSet.iterator(maxMessageSize)
|
val shallowIterator = messageSet.shallowIterator(maxMessageSize).asScala
|
||||||
for(shallowMessageAndOffset <- shallowIterator) { // this only does shallow iteration
|
for (shallowLogEntry <- shallowIterator) { // this only does shallow iteration
|
||||||
val itr = getIterator(shallowMessageAndOffset, isDeepIteration)
|
val itr = getIterator(shallowLogEntry, isDeepIteration)
|
||||||
for (messageAndOffset <- itr) {
|
for (deepLogEntry <- itr) {
|
||||||
val msg = messageAndOffset.message
|
val record = deepLogEntry.record()
|
||||||
|
|
||||||
if(lastOffset == -1)
|
if(lastOffset == -1)
|
||||||
lastOffset = messageAndOffset.offset
|
lastOffset = deepLogEntry.offset
|
||||||
// If we are iterating uncompressed messages, offsets must be consecutive
|
// If we are iterating uncompressed messages, offsets must be consecutive
|
||||||
else if (msg.compressionCodec == NoCompressionCodec && messageAndOffset.offset != lastOffset +1) {
|
else if (record.compressionType == CompressionType.NONE && deepLogEntry.offset != lastOffset +1) {
|
||||||
var nonConsecutivePairsSeq = nonConsecutivePairsForLogFilesMap.getOrElse(file.getAbsolutePath, List[(Long, Long)]())
|
var nonConsecutivePairsSeq = nonConsecutivePairsForLogFilesMap.getOrElse(file.getAbsolutePath, List[(Long, Long)]())
|
||||||
nonConsecutivePairsSeq ::=(lastOffset, messageAndOffset.offset)
|
nonConsecutivePairsSeq ::=(lastOffset, deepLogEntry.offset)
|
||||||
nonConsecutivePairsForLogFilesMap.put(file.getAbsolutePath, nonConsecutivePairsSeq)
|
nonConsecutivePairsForLogFilesMap.put(file.getAbsolutePath, nonConsecutivePairsSeq)
|
||||||
}
|
}
|
||||||
lastOffset = messageAndOffset.offset
|
lastOffset = deepLogEntry.offset
|
||||||
|
|
||||||
print("offset: " + messageAndOffset.offset + " position: " + validBytes +
|
print("offset: " + deepLogEntry.offset + " position: " + validBytes +
|
||||||
" " + msg.timestampType + ": " + msg.timestamp + " isvalid: " + msg.isValid +
|
" " + record.timestampType + ": " + record.timestamp + " isvalid: " + record.isValid +
|
||||||
" payloadsize: " + msg.payloadSize + " magic: " + msg.magic +
|
" payloadsize: " + record.valueSize + " magic: " + record.magic +
|
||||||
" compresscodec: " + msg.compressionCodec + " crc: " + msg.checksum)
|
" compresscodec: " + record.compressionType + " crc: " + record.checksum)
|
||||||
if(msg.hasKey)
|
if (record.hasKey)
|
||||||
print(" keysize: " + msg.keySize)
|
print(" keysize: " + record.keySize)
|
||||||
if(printContents) {
|
if (printContents) {
|
||||||
val (key, payload) = parser.parse(msg)
|
val (key, payload) = parser.parse(record)
|
||||||
key.map(key => print(s" key: ${key}"))
|
key.foreach(key => print(s" key: $key"))
|
||||||
payload.map(payload => print(s" payload: ${payload}"))
|
payload.foreach(payload => print(s" payload: $payload"))
|
||||||
}
|
}
|
||||||
println()
|
println()
|
||||||
}
|
}
|
||||||
validBytes += MessageSet.entrySize(shallowMessageAndOffset.message)
|
|
||||||
|
validBytes += shallowLogEntry.sizeInBytes
|
||||||
}
|
}
|
||||||
val trailingBytes = messageSet.sizeInBytes - validBytes
|
val trailingBytes = messageSet.sizeInBytes - validBytes
|
||||||
if(trailingBytes > 0)
|
if(trailingBytes > 0)
|
||||||
println("Found %d invalid bytes at the end of %s".format(trailingBytes, file.getName))
|
println("Found %d invalid bytes at the end of %s".format(trailingBytes, file.getName))
|
||||||
}
|
}
|
||||||
|
|
||||||
private def getIterator(messageAndOffset: MessageAndOffset, isDeepIteration: Boolean) = {
|
private def getIterator(logEntry: LogEntry, isDeepIteration: Boolean): Iterator[LogEntry] = {
|
||||||
if (isDeepIteration) {
|
if (isDeepIteration)
|
||||||
val message = messageAndOffset.message
|
logEntry.iterator.asScala
|
||||||
message.compressionCodec match {
|
else
|
||||||
case NoCompressionCodec =>
|
Iterator(logEntry)
|
||||||
getSingleMessageIterator(messageAndOffset)
|
|
||||||
case _ =>
|
|
||||||
ByteBufferMessageSet.deepIterator(messageAndOffset)
|
|
||||||
}
|
|
||||||
} else
|
|
||||||
getSingleMessageIterator(messageAndOffset)
|
|
||||||
}
|
|
||||||
|
|
||||||
private def getSingleMessageIterator(messageAndOffset: MessageAndOffset) = {
|
|
||||||
new IteratorTemplate[MessageAndOffset] {
|
|
||||||
var messageIterated = false
|
|
||||||
|
|
||||||
override def makeNext(): MessageAndOffset = {
|
|
||||||
if (!messageIterated) {
|
|
||||||
messageIterated = true
|
|
||||||
messageAndOffset
|
|
||||||
} else
|
|
||||||
allDone()
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
class TimeIndexDumpErrors {
|
class TimeIndexDumpErrors {
|
||||||
|
|
|
||||||
|
|
@ -17,25 +17,26 @@
|
||||||
|
|
||||||
package kafka.tools
|
package kafka.tools
|
||||||
|
|
||||||
import joptsimple.OptionParser
|
|
||||||
import kafka.cluster.BrokerEndPoint
|
|
||||||
import kafka.message.{ByteBufferMessageSet, MessageAndOffset, MessageSet}
|
|
||||||
import java.util.concurrent.CountDownLatch
|
|
||||||
import java.util.concurrent.atomic.AtomicReference
|
|
||||||
|
|
||||||
import kafka.client.ClientUtils
|
|
||||||
import java.util.regex.{Pattern, PatternSyntaxException}
|
|
||||||
|
|
||||||
import kafka.api._
|
|
||||||
import java.text.SimpleDateFormat
|
import java.text.SimpleDateFormat
|
||||||
import java.util.Date
|
import java.util.Date
|
||||||
|
import java.util.concurrent.CountDownLatch
|
||||||
|
import java.util.concurrent.atomic.AtomicReference
|
||||||
|
import java.util.regex.{Pattern, PatternSyntaxException}
|
||||||
|
|
||||||
|
import joptsimple.OptionParser
|
||||||
|
import kafka.api._
|
||||||
|
import kafka.client.ClientUtils
|
||||||
|
import kafka.cluster.BrokerEndPoint
|
||||||
import kafka.common.TopicAndPartition
|
import kafka.common.TopicAndPartition
|
||||||
import kafka.utils._
|
|
||||||
import kafka.consumer.{ConsumerConfig, SimpleConsumer, Whitelist}
|
import kafka.consumer.{ConsumerConfig, SimpleConsumer, Whitelist}
|
||||||
|
import kafka.message.{ByteBufferMessageSet, MessageSet}
|
||||||
|
import kafka.utils._
|
||||||
import org.apache.kafka.common.protocol.Errors
|
import org.apache.kafka.common.protocol.Errors
|
||||||
import org.apache.kafka.common.utils.Time
|
import org.apache.kafka.common.utils.Time
|
||||||
|
|
||||||
|
import scala.collection.JavaConverters._
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* For verifying the consistency among replicas.
|
* For verifying the consistency among replicas.
|
||||||
*
|
*
|
||||||
|
|
@ -149,15 +150,15 @@ object ReplicaVerificationTool extends Logging {
|
||||||
debug("Selected topic partitions: " + topicPartitionReplicaList)
|
debug("Selected topic partitions: " + topicPartitionReplicaList)
|
||||||
val topicAndPartitionsPerBroker: Map[Int, Seq[TopicAndPartition]] = topicPartitionReplicaList.groupBy(_.replicaId)
|
val topicAndPartitionsPerBroker: Map[Int, Seq[TopicAndPartition]] = topicPartitionReplicaList.groupBy(_.replicaId)
|
||||||
.map { case (brokerId, partitions) =>
|
.map { case (brokerId, partitions) =>
|
||||||
brokerId -> partitions.map { case partition => new TopicAndPartition(partition.topic, partition.partitionId) } }
|
brokerId -> partitions.map { partition => TopicAndPartition(partition.topic, partition.partitionId) } }
|
||||||
debug("Topic partitions per broker: " + topicAndPartitionsPerBroker)
|
debug("Topic partitions per broker: " + topicAndPartitionsPerBroker)
|
||||||
val expectedReplicasPerTopicAndPartition: Map[TopicAndPartition, Int] =
|
val expectedReplicasPerTopicAndPartition: Map[TopicAndPartition, Int] =
|
||||||
topicPartitionReplicaList.groupBy(replica => new TopicAndPartition(replica.topic, replica.partitionId))
|
topicPartitionReplicaList.groupBy(replica => TopicAndPartition(replica.topic, replica.partitionId))
|
||||||
.map { case (topicAndPartition, replicaSet) => topicAndPartition -> replicaSet.size }
|
.map { case (topicAndPartition, replicaSet) => topicAndPartition -> replicaSet.size }
|
||||||
debug("Expected replicas per topic partition: " + expectedReplicasPerTopicAndPartition)
|
debug("Expected replicas per topic partition: " + expectedReplicasPerTopicAndPartition)
|
||||||
val leadersPerBroker: Map[Int, Seq[TopicAndPartition]] = filteredTopicMetadata.flatMap { topicMetadataResponse =>
|
val leadersPerBroker: Map[Int, Seq[TopicAndPartition]] = filteredTopicMetadata.flatMap { topicMetadataResponse =>
|
||||||
topicMetadataResponse.partitionsMetadata.map { partitionMetadata =>
|
topicMetadataResponse.partitionsMetadata.map { partitionMetadata =>
|
||||||
(new TopicAndPartition(topicMetadataResponse.topic, partitionMetadata.partitionId), partitionMetadata.leader.get.id)
|
(TopicAndPartition(topicMetadataResponse.topic, partitionMetadata.partitionId), partitionMetadata.leader.get.id)
|
||||||
}
|
}
|
||||||
}.groupBy(_._2).mapValues(topicAndPartitionAndLeaderIds => topicAndPartitionAndLeaderIds.map { case (topicAndPartition, _) =>
|
}.groupBy(_._2).mapValues(topicAndPartitionAndLeaderIds => topicAndPartitionAndLeaderIds.map { case (topicAndPartition, _) =>
|
||||||
topicAndPartition
|
topicAndPartition
|
||||||
|
|
@ -200,8 +201,6 @@ object ReplicaVerificationTool extends Logging {
|
||||||
|
|
||||||
private case class TopicPartitionReplica(topic: String, partitionId: Int, replicaId: Int)
|
private case class TopicPartitionReplica(topic: String, partitionId: Int, replicaId: Int)
|
||||||
|
|
||||||
private case class ReplicaAndMessageIterator(replicaId: Int, iterator: Iterator[MessageAndOffset])
|
|
||||||
|
|
||||||
private case class MessageInfo(replicaId: Int, offset: Long, nextOffset: Long, checksum: Long)
|
private case class MessageInfo(replicaId: Int, offset: Long, nextOffset: Long, checksum: Long)
|
||||||
|
|
||||||
private class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPartition, Int],
|
private class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPartition, Int],
|
||||||
|
|
@ -276,41 +275,42 @@ private class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPa
|
||||||
assert(fetchResponsePerReplica.size == expectedReplicasPerTopicAndPartition(topicAndPartition),
|
assert(fetchResponsePerReplica.size == expectedReplicasPerTopicAndPartition(topicAndPartition),
|
||||||
"fetched " + fetchResponsePerReplica.size + " replicas for " + topicAndPartition + ", but expected "
|
"fetched " + fetchResponsePerReplica.size + " replicas for " + topicAndPartition + ", but expected "
|
||||||
+ expectedReplicasPerTopicAndPartition(topicAndPartition) + " replicas")
|
+ expectedReplicasPerTopicAndPartition(topicAndPartition) + " replicas")
|
||||||
val messageIteratorMap = fetchResponsePerReplica.map {
|
val logEntryIteratorMap = fetchResponsePerReplica.map {
|
||||||
case(replicaId, fetchResponse) =>
|
case(replicaId, fetchResponse) =>
|
||||||
replicaId -> fetchResponse.messages.asInstanceOf[ByteBufferMessageSet].shallowIterator}
|
replicaId -> fetchResponse.messages.asInstanceOf[ByteBufferMessageSet].asRecords.shallowIterator.asScala
|
||||||
|
}
|
||||||
val maxHw = fetchResponsePerReplica.values.map(_.hw).max
|
val maxHw = fetchResponsePerReplica.values.map(_.hw).max
|
||||||
|
|
||||||
// Iterate one message at a time from every replica, until high watermark is reached.
|
// Iterate one message at a time from every replica, until high watermark is reached.
|
||||||
var isMessageInAllReplicas = true
|
var isMessageInAllReplicas = true
|
||||||
while (isMessageInAllReplicas) {
|
while (isMessageInAllReplicas) {
|
||||||
var messageInfoFromFirstReplicaOpt: Option[MessageInfo] = None
|
var messageInfoFromFirstReplicaOpt: Option[MessageInfo] = None
|
||||||
for ( (replicaId, messageIterator) <- messageIteratorMap) {
|
for ( (replicaId, logEntries) <- logEntryIteratorMap) {
|
||||||
try {
|
try {
|
||||||
if (messageIterator.hasNext) {
|
if (logEntries.hasNext) {
|
||||||
val messageAndOffset = messageIterator.next()
|
val logEntry = logEntries.next()
|
||||||
|
|
||||||
// only verify up to the high watermark
|
// only verify up to the high watermark
|
||||||
if (messageAndOffset.offset >= fetchResponsePerReplica.get(replicaId).hw)
|
if (logEntry.offset >= fetchResponsePerReplica.get(replicaId).hw)
|
||||||
isMessageInAllReplicas = false
|
isMessageInAllReplicas = false
|
||||||
else {
|
else {
|
||||||
messageInfoFromFirstReplicaOpt match {
|
messageInfoFromFirstReplicaOpt match {
|
||||||
case None =>
|
case None =>
|
||||||
messageInfoFromFirstReplicaOpt = Some(
|
messageInfoFromFirstReplicaOpt = Some(
|
||||||
MessageInfo(replicaId, messageAndOffset.offset,messageAndOffset.nextOffset, messageAndOffset.message.checksum))
|
MessageInfo(replicaId, logEntry.offset,logEntry.nextOffset, logEntry.record.checksum))
|
||||||
case Some(messageInfoFromFirstReplica) =>
|
case Some(messageInfoFromFirstReplica) =>
|
||||||
if (messageInfoFromFirstReplica.offset != messageAndOffset.offset) {
|
if (messageInfoFromFirstReplica.offset != logEntry.offset) {
|
||||||
println(ReplicaVerificationTool.getCurrentTimeString + ": partition " + topicAndPartition
|
println(ReplicaVerificationTool.getCurrentTimeString + ": partition " + topicAndPartition
|
||||||
+ ": replica " + messageInfoFromFirstReplica.replicaId + "'s offset "
|
+ ": replica " + messageInfoFromFirstReplica.replicaId + "'s offset "
|
||||||
+ messageInfoFromFirstReplica.offset + " doesn't match replica "
|
+ messageInfoFromFirstReplica.offset + " doesn't match replica "
|
||||||
+ replicaId + "'s offset " + messageAndOffset.offset)
|
+ replicaId + "'s offset " + logEntry.offset)
|
||||||
System.exit(1)
|
System.exit(1)
|
||||||
}
|
}
|
||||||
if (messageInfoFromFirstReplica.checksum != messageAndOffset.message.checksum)
|
if (messageInfoFromFirstReplica.checksum != logEntry.record.checksum)
|
||||||
println(ReplicaVerificationTool.getCurrentTimeString + ": partition "
|
println(ReplicaVerificationTool.getCurrentTimeString + ": partition "
|
||||||
+ topicAndPartition + " has unmatched checksum at offset " + messageAndOffset.offset + "; replica "
|
+ topicAndPartition + " has unmatched checksum at offset " + logEntry.offset + "; replica "
|
||||||
+ messageInfoFromFirstReplica.replicaId + "'s checksum " + messageInfoFromFirstReplica.checksum
|
+ messageInfoFromFirstReplica.replicaId + "'s checksum " + messageInfoFromFirstReplica.checksum
|
||||||
+ "; replica " + replicaId + "'s checksum " + messageAndOffset.message.checksum)
|
+ "; replica " + replicaId + "'s checksum " + logEntry.record.checksum)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} else
|
} else
|
||||||
|
|
|
||||||
|
|
@ -50,11 +50,13 @@ class GroupCoordinatorIntegrationTest extends KafkaServerTestHarness {
|
||||||
def getGroupMetadataLogOpt: Option[Log] =
|
def getGroupMetadataLogOpt: Option[Log] =
|
||||||
logManager.getLog(TopicAndPartition(Topic.GroupMetadataTopicName, 0))
|
logManager.getLog(TopicAndPartition(Topic.GroupMetadataTopicName, 0))
|
||||||
|
|
||||||
TestUtils.waitUntilTrue(() => getGroupMetadataLogOpt.exists(_.logSegments.exists(_.log.nonEmpty)),
|
TestUtils.waitUntilTrue(() => getGroupMetadataLogOpt.exists(_.logSegments.exists(_.log.shallowIterator.asScala.nonEmpty)),
|
||||||
"Commit message not appended in time")
|
"Commit message not appended in time")
|
||||||
|
|
||||||
val logSegments = getGroupMetadataLogOpt.get.logSegments
|
val logSegments = getGroupMetadataLogOpt.get.logSegments
|
||||||
val incorrectCompressionCodecs = logSegments.flatMap(_.log.map(_.message.compressionCodec)).filter(_ != offsetsTopicCompressionCodec)
|
val incorrectCompressionCodecs = logSegments
|
||||||
|
.flatMap(_.log.shallowIterator.asScala.map(_.record.compressionType.id))
|
||||||
|
.filter(_ != offsetsTopicCompressionCodec.codec)
|
||||||
assertEquals("Incorrect compression codecs should be empty", Seq.empty, incorrectCompressionCodecs)
|
assertEquals("Incorrect compression codecs should be empty", Seq.empty, incorrectCompressionCodecs)
|
||||||
|
|
||||||
consumer.close()
|
consumer.close()
|
||||||
|
|
|
||||||
|
|
@ -21,12 +21,15 @@ import joptsimple.OptionParser
|
||||||
import java.util.Properties
|
import java.util.Properties
|
||||||
import java.util.Random
|
import java.util.Random
|
||||||
import java.io._
|
import java.io._
|
||||||
|
|
||||||
import kafka.consumer._
|
import kafka.consumer._
|
||||||
import kafka.serializer._
|
import kafka.serializer._
|
||||||
import kafka.utils._
|
import kafka.utils._
|
||||||
import kafka.log.FileMessageSet
|
|
||||||
import kafka.log.Log
|
import kafka.log.Log
|
||||||
import org.apache.kafka.clients.producer.{ProducerRecord, KafkaProducer, ProducerConfig}
|
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord}
|
||||||
|
import org.apache.kafka.common.record.FileRecords
|
||||||
|
|
||||||
|
import scala.collection.JavaConverters._
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This is a torture test that runs against an existing broker. Here is how it works:
|
* This is a torture test that runs against an existing broker. Here is how it works:
|
||||||
|
|
@ -135,15 +138,15 @@ object TestLogCleaning {
|
||||||
|
|
||||||
def dumpLog(dir: File) {
|
def dumpLog(dir: File) {
|
||||||
require(dir.exists, "Non-existent directory: " + dir.getAbsolutePath)
|
require(dir.exists, "Non-existent directory: " + dir.getAbsolutePath)
|
||||||
for(file <- dir.list.sorted; if file.endsWith(Log.LogFileSuffix)) {
|
for (file <- dir.list.sorted; if file.endsWith(Log.LogFileSuffix)) {
|
||||||
val ms = new FileMessageSet(new File(dir, file))
|
val fileRecords = FileRecords.open(new File(dir, file))
|
||||||
for(entry <- ms) {
|
for (entry <- fileRecords.shallowIterator.asScala) {
|
||||||
val key = TestUtils.readString(entry.message.key)
|
val key = TestUtils.readString(entry.record.key)
|
||||||
val content =
|
val content =
|
||||||
if(entry.message.isNull)
|
if(entry.record.hasNullValue)
|
||||||
null
|
null
|
||||||
else
|
else
|
||||||
TestUtils.readString(entry.message.payload)
|
TestUtils.readString(entry.record.value)
|
||||||
println("offset = %s, key = %s, content = %s".format(entry.offset, key, content))
|
println("offset = %s, key = %s, content = %s".format(entry.offset, key, content))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -20,10 +20,10 @@ package kafka
|
||||||
import java.util.Properties
|
import java.util.Properties
|
||||||
import java.util.concurrent.atomic._
|
import java.util.concurrent.atomic._
|
||||||
|
|
||||||
import kafka.message._
|
|
||||||
import kafka.log._
|
import kafka.log._
|
||||||
import kafka.utils._
|
import kafka.utils._
|
||||||
import org.apache.kafka.clients.consumer.OffsetOutOfRangeException
|
import org.apache.kafka.clients.consumer.OffsetOutOfRangeException
|
||||||
|
import org.apache.kafka.common.record.FileRecords
|
||||||
import org.apache.kafka.common.utils.Utils
|
import org.apache.kafka.common.utils.Utils
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -36,13 +36,13 @@ object StressTestLog {
|
||||||
def main(args: Array[String]) {
|
def main(args: Array[String]) {
|
||||||
val dir = TestUtils.randomPartitionLogDir(TestUtils.tempDir())
|
val dir = TestUtils.randomPartitionLogDir(TestUtils.tempDir())
|
||||||
val time = new MockTime
|
val time = new MockTime
|
||||||
val logProprties = new Properties()
|
val logProperties = new Properties()
|
||||||
logProprties.put(LogConfig.SegmentBytesProp, 64*1024*1024: java.lang.Integer)
|
logProperties.put(LogConfig.SegmentBytesProp, 64*1024*1024: java.lang.Integer)
|
||||||
logProprties.put(LogConfig.MaxMessageBytesProp, Int.MaxValue: java.lang.Integer)
|
logProperties.put(LogConfig.MaxMessageBytesProp, Int.MaxValue: java.lang.Integer)
|
||||||
logProprties.put(LogConfig.SegmentIndexBytesProp, 1024*1024: java.lang.Integer)
|
logProperties.put(LogConfig.SegmentIndexBytesProp, 1024*1024: java.lang.Integer)
|
||||||
|
|
||||||
val log = new Log(dir = dir,
|
val log = new Log(dir = dir,
|
||||||
config = LogConfig(logProprties),
|
config = LogConfig(logProperties),
|
||||||
recoveryPoint = 0L,
|
recoveryPoint = 0L,
|
||||||
scheduler = time.scheduler,
|
scheduler = time.scheduler,
|
||||||
time = time)
|
time = time)
|
||||||
|
|
@ -84,7 +84,7 @@ object StressTestLog {
|
||||||
class WriterThread(val log: Log) extends WorkerThread {
|
class WriterThread(val log: Log) extends WorkerThread {
|
||||||
@volatile var offset = 0
|
@volatile var offset = 0
|
||||||
override def work() {
|
override def work() {
|
||||||
val logAppendInfo = log.append(TestUtils.singleMessageSet(offset.toString.getBytes))
|
val logAppendInfo = log.append(TestUtils.singletonRecords(offset.toString.getBytes))
|
||||||
require(logAppendInfo.firstOffset == offset && logAppendInfo.lastOffset == offset)
|
require(logAppendInfo.firstOffset == offset && logAppendInfo.lastOffset == offset)
|
||||||
offset += 1
|
offset += 1
|
||||||
if(offset % 1000 == 0)
|
if(offset % 1000 == 0)
|
||||||
|
|
@ -96,11 +96,11 @@ object StressTestLog {
|
||||||
@volatile var offset = 0
|
@volatile var offset = 0
|
||||||
override def work() {
|
override def work() {
|
||||||
try {
|
try {
|
||||||
log.read(offset, 1024, Some(offset+1)).messageSet match {
|
log.read(offset, 1024, Some(offset+1)).records match {
|
||||||
case read: FileMessageSet if read.sizeInBytes > 0 => {
|
case read: FileRecords if read.sizeInBytes > 0 => {
|
||||||
val first = read.head
|
val first = read.shallowIterator.next()
|
||||||
require(first.offset == offset, "We should either read nothing or the message we asked for.")
|
require(first.offset == offset, "We should either read nothing or the message we asked for.")
|
||||||
require(MessageSet.entrySize(first.message) == read.sizeInBytes, "Expected %d but got %d.".format(MessageSet.entrySize(first.message), read.sizeInBytes))
|
require(first.sizeInBytes == read.sizeInBytes, "Expected %d but got %d.".format(first.sizeInBytes, read.sizeInBytes))
|
||||||
offset += 1
|
offset += 1
|
||||||
}
|
}
|
||||||
case _ =>
|
case _ =>
|
||||||
|
|
|
||||||
|
|
@ -22,13 +22,14 @@ import java.nio._
|
||||||
import java.nio.channels._
|
import java.nio.channels._
|
||||||
import java.util.{Properties, Random}
|
import java.util.{Properties, Random}
|
||||||
|
|
||||||
|
import joptsimple._
|
||||||
import kafka.log._
|
import kafka.log._
|
||||||
import kafka.utils._
|
|
||||||
import kafka.message._
|
import kafka.message._
|
||||||
|
import kafka.utils._
|
||||||
|
import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Record}
|
||||||
|
import org.apache.kafka.common.utils.{Time, Utils}
|
||||||
|
|
||||||
import scala.math._
|
import scala.math._
|
||||||
import joptsimple._
|
|
||||||
import org.apache.kafka.common.utils.{Time, Utils}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This test does linear writes using either a kafka log or a file and measures throughput and latency.
|
* This test does linear writes using either a kafka log or a file and measures throughput and latency.
|
||||||
|
|
@ -64,7 +65,7 @@ object TestLinearWriteSpeed {
|
||||||
.withRequiredArg
|
.withRequiredArg
|
||||||
.describedAs("ms")
|
.describedAs("ms")
|
||||||
.ofType(classOf[java.lang.Long])
|
.ofType(classOf[java.lang.Long])
|
||||||
.defaultsTo(1000)
|
.defaultsTo(1000L)
|
||||||
val maxThroughputOpt = parser.accepts("max-throughput-mb", "The maximum throughput.")
|
val maxThroughputOpt = parser.accepts("max-throughput-mb", "The maximum throughput.")
|
||||||
.withRequiredArg
|
.withRequiredArg
|
||||||
.describedAs("mb")
|
.describedAs("mb")
|
||||||
|
|
@ -81,7 +82,7 @@ object TestLinearWriteSpeed {
|
||||||
.ofType(classOf[java.lang.String])
|
.ofType(classOf[java.lang.String])
|
||||||
.defaultsTo(NoCompressionCodec.name)
|
.defaultsTo(NoCompressionCodec.name)
|
||||||
val mmapOpt = parser.accepts("mmap", "Do writes to memory-mapped files.")
|
val mmapOpt = parser.accepts("mmap", "Do writes to memory-mapped files.")
|
||||||
val channelOpt = parser.accepts("channel", "Do writes to file channesl.")
|
val channelOpt = parser.accepts("channel", "Do writes to file channels.")
|
||||||
val logOpt = parser.accepts("log", "Do writes to kafka logs.")
|
val logOpt = parser.accepts("log", "Do writes to kafka logs.")
|
||||||
|
|
||||||
val options = parser.parse(args : _*)
|
val options = parser.parse(args : _*)
|
||||||
|
|
@ -101,9 +102,9 @@ object TestLinearWriteSpeed {
|
||||||
val rand = new Random
|
val rand = new Random
|
||||||
rand.nextBytes(buffer.array)
|
rand.nextBytes(buffer.array)
|
||||||
val numMessages = bufferSize / (messageSize + MessageSet.LogOverhead)
|
val numMessages = bufferSize / (messageSize + MessageSet.LogOverhead)
|
||||||
val messageSet = new ByteBufferMessageSet(compressionCodec = compressionCodec,
|
val messageSet = MemoryRecords.withRecords(CompressionType.forId(compressionCodec.codec),
|
||||||
messages = (0 until numMessages).map(_ => new Message(new Array[Byte](messageSize))): _*)
|
(0 until numMessages).map(_ => Record.create(new Array[Byte](messageSize))): _*)
|
||||||
|
|
||||||
val writables = new Array[Writable](numFiles)
|
val writables = new Array[Writable](numFiles)
|
||||||
val scheduler = new KafkaScheduler(1)
|
val scheduler = new KafkaScheduler(1)
|
||||||
scheduler.startup()
|
scheduler.startup()
|
||||||
|
|
@ -199,7 +200,7 @@ object TestLinearWriteSpeed {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
class LogWritable(val dir: File, config: LogConfig, scheduler: Scheduler, val messages: ByteBufferMessageSet) extends Writable {
|
class LogWritable(val dir: File, config: LogConfig, scheduler: Scheduler, val messages: MemoryRecords) extends Writable {
|
||||||
Utils.delete(dir)
|
Utils.delete(dir)
|
||||||
val log = new Log(dir, config, 0L, scheduler, Time.SYSTEM)
|
val log = new Log(dir, config, 0L, scheduler, Time.SYSTEM)
|
||||||
def write(): Int = {
|
def write(): Int = {
|
||||||
|
|
|
||||||
|
|
@ -301,7 +301,7 @@ class DeleteTopicTest extends ZooKeeperTestHarness {
|
||||||
var counter = 0
|
var counter = 0
|
||||||
for (_ <- 0 until numDups; key <- 0 until numKeys) yield {
|
for (_ <- 0 until numDups; key <- 0 until numKeys) yield {
|
||||||
val count = counter
|
val count = counter
|
||||||
log.append(TestUtils.singleMessageSet(payload = counter.toString.getBytes, key = key.toString.getBytes), assignOffsets = true)
|
log.append(TestUtils.singletonRecords(value = counter.toString.getBytes, key = key.toString.getBytes), assignOffsets = true)
|
||||||
counter += 1
|
counter += 1
|
||||||
(key, count)
|
(key, count)
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -18,20 +18,20 @@
|
||||||
package kafka.coordinator
|
package kafka.coordinator
|
||||||
|
|
||||||
import kafka.utils.timer.MockTimer
|
import kafka.utils.timer.MockTimer
|
||||||
import org.apache.kafka.common.record.Record
|
import org.apache.kafka.common.record.{MemoryRecords, Record, TimestampType}
|
||||||
import org.junit.Assert._
|
import org.junit.Assert._
|
||||||
import kafka.common.{OffsetAndMetadata, Topic}
|
import kafka.common.{OffsetAndMetadata, Topic}
|
||||||
import kafka.message.{Message, MessageSet}
|
import kafka.server.{DelayedOperationPurgatory, KafkaConfig, ReplicaManager}
|
||||||
import kafka.server.{DelayedOperationPurgatory, ReplicaManager, KafkaConfig}
|
|
||||||
import kafka.utils._
|
import kafka.utils._
|
||||||
import org.apache.kafka.common.TopicPartition
|
import org.apache.kafka.common.TopicPartition
|
||||||
import org.apache.kafka.common.protocol.Errors
|
import org.apache.kafka.common.protocol.Errors
|
||||||
import org.apache.kafka.common.requests.{OffsetCommitRequest, JoinGroupRequest}
|
import org.apache.kafka.common.requests.{JoinGroupRequest, OffsetCommitRequest}
|
||||||
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
|
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
|
||||||
import org.easymock.{Capture, IAnswer, EasyMock}
|
import org.easymock.{Capture, EasyMock, IAnswer}
|
||||||
import org.junit.{After, Before, Test}
|
import org.junit.{After, Before, Test}
|
||||||
import org.scalatest.junit.JUnitSuite
|
import org.scalatest.junit.JUnitSuite
|
||||||
import java.util.concurrent.TimeUnit
|
import java.util.concurrent.TimeUnit
|
||||||
|
|
||||||
import scala.collection._
|
import scala.collection._
|
||||||
import scala.concurrent.duration.Duration
|
import scala.concurrent.duration.Duration
|
||||||
import scala.concurrent.{Await, Future, Promise}
|
import scala.concurrent.{Await, Future, Promise}
|
||||||
|
|
@ -305,7 +305,8 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
|
||||||
|
|
||||||
EasyMock.reset(replicaManager)
|
EasyMock.reset(replicaManager)
|
||||||
EasyMock.expect(replicaManager.getPartition(Topic.GroupMetadataTopicName, groupPartitionId)).andReturn(None)
|
EasyMock.expect(replicaManager.getPartition(Topic.GroupMetadataTopicName, groupPartitionId)).andReturn(None)
|
||||||
EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andReturn(Some(Message.MagicValue_V1)).anyTimes()
|
EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject()))
|
||||||
|
.andReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME)).anyTimes()
|
||||||
EasyMock.replay(replicaManager)
|
EasyMock.replay(replicaManager)
|
||||||
|
|
||||||
timer.advanceClock(DefaultSessionTimeout + 100)
|
timer.advanceClock(DefaultSessionTimeout + 100)
|
||||||
|
|
@ -988,17 +989,18 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
|
||||||
|
|
||||||
val capturedArgument: Capture[Map[TopicPartition, PartitionResponse] => Unit] = EasyMock.newCapture()
|
val capturedArgument: Capture[Map[TopicPartition, PartitionResponse] => Unit] = EasyMock.newCapture()
|
||||||
|
|
||||||
EasyMock.expect(replicaManager.appendMessages(EasyMock.anyLong(),
|
EasyMock.expect(replicaManager.appendRecords(EasyMock.anyLong(),
|
||||||
EasyMock.anyShort(),
|
EasyMock.anyShort(),
|
||||||
EasyMock.anyBoolean(),
|
EasyMock.anyBoolean(),
|
||||||
EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MessageSet]],
|
EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MemoryRecords]],
|
||||||
EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] {
|
EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] {
|
||||||
override def answer = capturedArgument.getValue.apply(
|
override def answer = capturedArgument.getValue.apply(
|
||||||
Map(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId) ->
|
Map(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId) ->
|
||||||
new PartitionResponse(Errors.NONE.code, 0L, Record.NO_TIMESTAMP)
|
new PartitionResponse(Errors.NONE.code, 0L, Record.NO_TIMESTAMP)
|
||||||
)
|
)
|
||||||
)})
|
)})
|
||||||
EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andReturn(Some(Message.MagicValue_V1)).anyTimes()
|
EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject()))
|
||||||
|
.andReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME)).anyTimes()
|
||||||
EasyMock.replay(replicaManager)
|
EasyMock.replay(replicaManager)
|
||||||
|
|
||||||
groupCoordinator.handleSyncGroup(groupId, generation, leaderId, assignment, responseCallback)
|
groupCoordinator.handleSyncGroup(groupId, generation, leaderId, assignment, responseCallback)
|
||||||
|
|
@ -1069,17 +1071,18 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
|
||||||
|
|
||||||
val capturedArgument: Capture[Map[TopicPartition, PartitionResponse] => Unit] = EasyMock.newCapture()
|
val capturedArgument: Capture[Map[TopicPartition, PartitionResponse] => Unit] = EasyMock.newCapture()
|
||||||
|
|
||||||
EasyMock.expect(replicaManager.appendMessages(EasyMock.anyLong(),
|
EasyMock.expect(replicaManager.appendRecords(EasyMock.anyLong(),
|
||||||
EasyMock.anyShort(),
|
EasyMock.anyShort(),
|
||||||
EasyMock.anyBoolean(),
|
EasyMock.anyBoolean(),
|
||||||
EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MessageSet]],
|
EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MemoryRecords]],
|
||||||
EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] {
|
EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] {
|
||||||
override def answer = capturedArgument.getValue.apply(
|
override def answer = capturedArgument.getValue.apply(
|
||||||
Map(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId) ->
|
Map(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId) ->
|
||||||
new PartitionResponse(Errors.NONE.code, 0L, Record.NO_TIMESTAMP)
|
new PartitionResponse(Errors.NONE.code, 0L, Record.NO_TIMESTAMP)
|
||||||
)
|
)
|
||||||
)})
|
)})
|
||||||
EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andReturn(Some(Message.MagicValue_V1)).anyTimes()
|
EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject()))
|
||||||
|
.andReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME)).anyTimes()
|
||||||
EasyMock.replay(replicaManager)
|
EasyMock.replay(replicaManager)
|
||||||
|
|
||||||
groupCoordinator.handleCommitOffsets(groupId, consumerId, generationId, offsets, responseCallback)
|
groupCoordinator.handleCommitOffsets(groupId, consumerId, generationId, offsets, responseCallback)
|
||||||
|
|
@ -1090,7 +1093,8 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
|
||||||
val (responseFuture, responseCallback) = setupHeartbeatCallback
|
val (responseFuture, responseCallback) = setupHeartbeatCallback
|
||||||
|
|
||||||
EasyMock.expect(replicaManager.getPartition(Topic.GroupMetadataTopicName, groupPartitionId)).andReturn(None)
|
EasyMock.expect(replicaManager.getPartition(Topic.GroupMetadataTopicName, groupPartitionId)).andReturn(None)
|
||||||
EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andReturn(Some(Message.MagicValue_V1)).anyTimes()
|
EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject()))
|
||||||
|
.andReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME)).anyTimes()
|
||||||
EasyMock.replay(replicaManager)
|
EasyMock.replay(replicaManager)
|
||||||
|
|
||||||
groupCoordinator.handleLeaveGroup(groupId, consumerId, responseCallback)
|
groupCoordinator.handleLeaveGroup(groupId, consumerId, responseCallback)
|
||||||
|
|
|
||||||
|
|
@ -21,12 +21,11 @@ import kafka.api.ApiVersion
|
||||||
import kafka.cluster.Partition
|
import kafka.cluster.Partition
|
||||||
import kafka.common.{OffsetAndMetadata, Topic}
|
import kafka.common.{OffsetAndMetadata, Topic}
|
||||||
import kafka.log.LogAppendInfo
|
import kafka.log.LogAppendInfo
|
||||||
import kafka.message.{ByteBufferMessageSet, Message, MessageSet}
|
|
||||||
import kafka.server.{KafkaConfig, ReplicaManager}
|
import kafka.server.{KafkaConfig, ReplicaManager}
|
||||||
import kafka.utils.{KafkaScheduler, MockTime, TestUtils, ZkUtils}
|
import kafka.utils.{KafkaScheduler, MockTime, TestUtils, ZkUtils}
|
||||||
import org.apache.kafka.common.TopicPartition
|
import org.apache.kafka.common.TopicPartition
|
||||||
import org.apache.kafka.common.protocol.Errors
|
import org.apache.kafka.common.protocol.Errors
|
||||||
import org.apache.kafka.common.record.Record
|
import org.apache.kafka.common.record.{MemoryRecords, Record, TimestampType}
|
||||||
import org.apache.kafka.common.requests.OffsetFetchResponse
|
import org.apache.kafka.common.requests.OffsetFetchResponse
|
||||||
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
|
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
|
||||||
import org.easymock.{Capture, EasyMock, IAnswer}
|
import org.easymock.{Capture, EasyMock, IAnswer}
|
||||||
|
|
@ -34,6 +33,7 @@ import org.junit.{After, Before, Test}
|
||||||
import org.junit.Assert._
|
import org.junit.Assert._
|
||||||
|
|
||||||
import scala.collection._
|
import scala.collection._
|
||||||
|
import JavaConverters._
|
||||||
|
|
||||||
class GroupMetadataManagerTest {
|
class GroupMetadataManagerTest {
|
||||||
|
|
||||||
|
|
@ -50,7 +50,6 @@ class GroupMetadataManagerTest {
|
||||||
val rebalanceTimeout = 60000
|
val rebalanceTimeout = 60000
|
||||||
val sessionTimeout = 10000
|
val sessionTimeout = 10000
|
||||||
|
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
def setUp() {
|
def setUp() {
|
||||||
val config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(nodeId = 0, zkConnect = ""))
|
val config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(nodeId = 0, zkConnect = ""))
|
||||||
|
|
@ -176,7 +175,7 @@ class GroupMetadataManagerTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
def testStoreNonEmptyGroupWhenCoordinatorHasMoved() {
|
def testStoreNonEmptyGroupWhenCoordinatorHasMoved() {
|
||||||
EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andReturn(None)
|
EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject())).andReturn(None)
|
||||||
val memberId = "memberId"
|
val memberId = "memberId"
|
||||||
val clientId = "clientId"
|
val clientId = "clientId"
|
||||||
val clientHost = "localhost"
|
val clientHost = "localhost"
|
||||||
|
|
@ -245,7 +244,7 @@ class GroupMetadataManagerTest {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
def testCommitOffsetWhenCoordinatorHasMoved() {
|
def testCommitOffsetWhenCoordinatorHasMoved() {
|
||||||
EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andReturn(None)
|
EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject())).andReturn(None)
|
||||||
val memberId = ""
|
val memberId = ""
|
||||||
val generationId = -1
|
val generationId = -1
|
||||||
val topicPartition = new TopicPartition("foo", 0)
|
val topicPartition = new TopicPartition("foo", 0)
|
||||||
|
|
@ -363,7 +362,7 @@ class GroupMetadataManagerTest {
|
||||||
time.sleep(2)
|
time.sleep(2)
|
||||||
|
|
||||||
EasyMock.reset(partition)
|
EasyMock.reset(partition)
|
||||||
EasyMock.expect(partition.appendMessagesToLeader(EasyMock.anyObject(classOf[ByteBufferMessageSet]), EasyMock.anyInt()))
|
EasyMock.expect(partition.appendRecordsToLeader(EasyMock.anyObject(classOf[MemoryRecords]), EasyMock.anyInt()))
|
||||||
.andReturn(LogAppendInfo.UnknownLogAppendInfo)
|
.andReturn(LogAppendInfo.UnknownLogAppendInfo)
|
||||||
EasyMock.replay(partition)
|
EasyMock.replay(partition)
|
||||||
|
|
||||||
|
|
@ -391,24 +390,74 @@ class GroupMetadataManagerTest {
|
||||||
|
|
||||||
// expect the group metadata tombstone
|
// expect the group metadata tombstone
|
||||||
EasyMock.reset(partition)
|
EasyMock.reset(partition)
|
||||||
val messageSetCapture: Capture[ByteBufferMessageSet] = EasyMock.newCapture()
|
val recordsCapture: Capture[MemoryRecords] = EasyMock.newCapture()
|
||||||
|
|
||||||
EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andStubReturn(Some(Message.MagicValue_V1))
|
EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject()))
|
||||||
|
.andStubReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME))
|
||||||
EasyMock.expect(replicaManager.getPartition(Topic.GroupMetadataTopicName, groupPartitionId)).andStubReturn(Some(partition))
|
EasyMock.expect(replicaManager.getPartition(Topic.GroupMetadataTopicName, groupPartitionId)).andStubReturn(Some(partition))
|
||||||
EasyMock.expect(partition.appendMessagesToLeader(EasyMock.capture(messageSetCapture), EasyMock.anyInt()))
|
EasyMock.expect(partition.appendRecordsToLeader(EasyMock.capture(recordsCapture), EasyMock.anyInt()))
|
||||||
.andReturn(LogAppendInfo.UnknownLogAppendInfo)
|
.andReturn(LogAppendInfo.UnknownLogAppendInfo)
|
||||||
EasyMock.replay(replicaManager, partition)
|
EasyMock.replay(replicaManager, partition)
|
||||||
|
|
||||||
groupMetadataManager.cleanupGroupMetadata()
|
groupMetadataManager.cleanupGroupMetadata()
|
||||||
|
|
||||||
assertTrue(messageSetCapture.hasCaptured)
|
assertTrue(recordsCapture.hasCaptured)
|
||||||
|
|
||||||
val messageSet = messageSetCapture.getValue
|
val records = recordsCapture.getValue.records.asScala.toList
|
||||||
assertEquals(1, messageSet.size)
|
assertEquals(1, records.size)
|
||||||
|
|
||||||
val metadataTombstone = messageSet.head.message
|
val metadataTombstone = records.head
|
||||||
assertTrue(metadataTombstone.hasKey)
|
assertTrue(metadataTombstone.hasKey)
|
||||||
assertTrue(metadataTombstone.isNull)
|
assertTrue(metadataTombstone.hasNullValue)
|
||||||
|
assertEquals(Record.MAGIC_VALUE_V1, metadataTombstone.magic)
|
||||||
|
assertEquals(TimestampType.CREATE_TIME, metadataTombstone.timestampType)
|
||||||
|
assertTrue(metadataTombstone.timestamp > 0)
|
||||||
|
|
||||||
|
val groupKey = GroupMetadataManager.readMessageKey(metadataTombstone.key).asInstanceOf[GroupMetadataKey]
|
||||||
|
assertEquals(groupId, groupKey.key)
|
||||||
|
|
||||||
|
// the full group should be gone since all offsets were removed
|
||||||
|
assertEquals(None, groupMetadataManager.getGroup(groupId))
|
||||||
|
val cachedOffsets = groupMetadataManager.getOffsets(groupId, Seq(topicPartition1, topicPartition2))
|
||||||
|
assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), cachedOffsets.get(topicPartition1).map(_.offset))
|
||||||
|
assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), cachedOffsets.get(topicPartition2).map(_.offset))
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
def testGroupMetadataRemovalWithLogAppendTime() {
|
||||||
|
val topicPartition1 = new TopicPartition("foo", 0)
|
||||||
|
val topicPartition2 = new TopicPartition("foo", 1)
|
||||||
|
|
||||||
|
groupMetadataManager.addPartitionOwnership(groupPartitionId)
|
||||||
|
|
||||||
|
val group = new GroupMetadata(groupId)
|
||||||
|
groupMetadataManager.addGroup(group)
|
||||||
|
group.generationId = 5
|
||||||
|
|
||||||
|
// expect the group metadata tombstone
|
||||||
|
EasyMock.reset(partition)
|
||||||
|
val recordsCapture: Capture[MemoryRecords] = EasyMock.newCapture()
|
||||||
|
|
||||||
|
EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject()))
|
||||||
|
.andStubReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.LOG_APPEND_TIME))
|
||||||
|
EasyMock.expect(replicaManager.getPartition(Topic.GroupMetadataTopicName, groupPartitionId)).andStubReturn(Some(partition))
|
||||||
|
EasyMock.expect(partition.appendRecordsToLeader(EasyMock.capture(recordsCapture), EasyMock.anyInt()))
|
||||||
|
.andReturn(LogAppendInfo.UnknownLogAppendInfo)
|
||||||
|
EasyMock.replay(replicaManager, partition)
|
||||||
|
|
||||||
|
groupMetadataManager.cleanupGroupMetadata()
|
||||||
|
|
||||||
|
assertTrue(recordsCapture.hasCaptured)
|
||||||
|
|
||||||
|
val records = recordsCapture.getValue.records.asScala.toList
|
||||||
|
assertEquals(1, records.size)
|
||||||
|
|
||||||
|
val metadataTombstone = records.head
|
||||||
|
assertTrue(metadataTombstone.hasKey)
|
||||||
|
assertTrue(metadataTombstone.hasNullValue)
|
||||||
|
assertEquals(Record.MAGIC_VALUE_V1, metadataTombstone.magic)
|
||||||
|
assertEquals(TimestampType.LOG_APPEND_TIME, metadataTombstone.timestampType)
|
||||||
|
assertTrue(metadataTombstone.timestamp > 0)
|
||||||
|
|
||||||
val groupKey = GroupMetadataManager.readMessageKey(metadataTombstone.key).asInstanceOf[GroupMetadataKey]
|
val groupKey = GroupMetadataManager.readMessageKey(metadataTombstone.key).asInstanceOf[GroupMetadataKey]
|
||||||
assertEquals(groupId, groupKey.key)
|
assertEquals(groupId, groupKey.key)
|
||||||
|
|
@ -463,22 +512,22 @@ class GroupMetadataManagerTest {
|
||||||
|
|
||||||
// expect the offset tombstone
|
// expect the offset tombstone
|
||||||
EasyMock.reset(partition)
|
EasyMock.reset(partition)
|
||||||
val messageSetCapture: Capture[ByteBufferMessageSet] = EasyMock.newCapture()
|
val recordsCapture: Capture[MemoryRecords] = EasyMock.newCapture()
|
||||||
|
|
||||||
EasyMock.expect(partition.appendMessagesToLeader(EasyMock.capture(messageSetCapture), EasyMock.anyInt()))
|
EasyMock.expect(partition.appendRecordsToLeader(EasyMock.capture(recordsCapture), EasyMock.anyInt()))
|
||||||
.andReturn(LogAppendInfo.UnknownLogAppendInfo)
|
.andReturn(LogAppendInfo.UnknownLogAppendInfo)
|
||||||
EasyMock.replay(partition)
|
EasyMock.replay(partition)
|
||||||
|
|
||||||
groupMetadataManager.cleanupGroupMetadata()
|
groupMetadataManager.cleanupGroupMetadata()
|
||||||
|
|
||||||
assertTrue(messageSetCapture.hasCaptured)
|
assertTrue(recordsCapture.hasCaptured)
|
||||||
|
|
||||||
// verify the tombstones are correct and only for the expired offsets
|
// verify the tombstones are correct and only for the expired offsets
|
||||||
val messageSet = messageSetCapture.getValue
|
val records = recordsCapture.getValue.records.asScala.toList
|
||||||
assertEquals(2, messageSet.size)
|
assertEquals(2, records.size)
|
||||||
messageSet.map(_.message).foreach { message =>
|
records.foreach { message =>
|
||||||
assertTrue(message.hasKey)
|
assertTrue(message.hasKey)
|
||||||
assertTrue(message.isNull)
|
assertTrue(message.hasNullValue)
|
||||||
val offsetKey = GroupMetadataManager.readMessageKey(message.key).asInstanceOf[OffsetKey]
|
val offsetKey = GroupMetadataManager.readMessageKey(message.key).asInstanceOf[OffsetKey]
|
||||||
assertEquals(groupId, offsetKey.key.group)
|
assertEquals(groupId, offsetKey.key.group)
|
||||||
assertEquals("foo", offsetKey.key.topicPartition.topic)
|
assertEquals("foo", offsetKey.key.topicPartition.topic)
|
||||||
|
|
@ -539,7 +588,7 @@ class GroupMetadataManagerTest {
|
||||||
|
|
||||||
// expect the offset tombstone
|
// expect the offset tombstone
|
||||||
EasyMock.reset(partition)
|
EasyMock.reset(partition)
|
||||||
EasyMock.expect(partition.appendMessagesToLeader(EasyMock.anyObject(classOf[ByteBufferMessageSet]), EasyMock.anyInt()))
|
EasyMock.expect(partition.appendRecordsToLeader(EasyMock.anyObject(classOf[MemoryRecords]), EasyMock.anyInt()))
|
||||||
.andReturn(LogAppendInfo.UnknownLogAppendInfo)
|
.andReturn(LogAppendInfo.UnknownLogAppendInfo)
|
||||||
EasyMock.replay(partition)
|
EasyMock.replay(partition)
|
||||||
|
|
||||||
|
|
@ -557,17 +606,18 @@ class GroupMetadataManagerTest {
|
||||||
|
|
||||||
private def expectAppendMessage(error: Errors) {
|
private def expectAppendMessage(error: Errors) {
|
||||||
val capturedArgument: Capture[Map[TopicPartition, PartitionResponse] => Unit] = EasyMock.newCapture()
|
val capturedArgument: Capture[Map[TopicPartition, PartitionResponse] => Unit] = EasyMock.newCapture()
|
||||||
EasyMock.expect(replicaManager.appendMessages(EasyMock.anyLong(),
|
EasyMock.expect(replicaManager.appendRecords(EasyMock.anyLong(),
|
||||||
EasyMock.anyShort(),
|
EasyMock.anyShort(),
|
||||||
EasyMock.anyBoolean(),
|
EasyMock.anyBoolean(),
|
||||||
EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MessageSet]],
|
EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MemoryRecords]],
|
||||||
EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] {
|
EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] {
|
||||||
override def answer = capturedArgument.getValue.apply(
|
override def answer = capturedArgument.getValue.apply(
|
||||||
Map(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId) ->
|
Map(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId) ->
|
||||||
new PartitionResponse(error.code, 0L, Record.NO_TIMESTAMP)
|
new PartitionResponse(error.code, 0L, Record.NO_TIMESTAMP)
|
||||||
)
|
)
|
||||||
)})
|
)})
|
||||||
EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andStubReturn(Some(Message.MagicValue_V1))
|
EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject()))
|
||||||
|
.andStubReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME))
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -25,9 +25,10 @@ import org.junit.Assert._
|
||||||
import org.junit.runner.RunWith
|
import org.junit.runner.RunWith
|
||||||
import org.junit.runners.Parameterized
|
import org.junit.runners.Parameterized
|
||||||
import org.junit.runners.Parameterized.Parameters
|
import org.junit.runners.Parameterized.Parameters
|
||||||
import org.apache.kafka.common.record.CompressionType
|
import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Record}
|
||||||
import org.apache.kafka.common.utils.Utils
|
import org.apache.kafka.common.utils.Utils
|
||||||
import java.util.{Collection, Properties}
|
import java.util.{Collection, Properties}
|
||||||
|
|
||||||
import scala.collection.JavaConverters._
|
import scala.collection.JavaConverters._
|
||||||
|
|
||||||
@RunWith(value = classOf[Parameterized])
|
@RunWith(value = classOf[Parameterized])
|
||||||
|
|
@ -50,22 +51,22 @@ class BrokerCompressionTest(messageCompression: String, brokerCompression: Strin
|
||||||
def testBrokerSideCompression() {
|
def testBrokerSideCompression() {
|
||||||
val messageCompressionCode = CompressionCodec.getCompressionCodec(messageCompression)
|
val messageCompressionCode = CompressionCodec.getCompressionCodec(messageCompression)
|
||||||
val logProps = new Properties()
|
val logProps = new Properties()
|
||||||
logProps.put(LogConfig.CompressionTypeProp,brokerCompression)
|
logProps.put(LogConfig.CompressionTypeProp, brokerCompression)
|
||||||
/*configure broker-side compression */
|
/*configure broker-side compression */
|
||||||
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
|
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
|
||||||
|
|
||||||
/* append two messages */
|
/* append two messages */
|
||||||
log.append(new ByteBufferMessageSet(messageCompressionCode, new Message("hello".getBytes), new Message("there".getBytes)))
|
log.append(MemoryRecords.withRecords(CompressionType.forId(messageCompressionCode.codec),
|
||||||
|
Record.create("hello".getBytes), Record.create("there".getBytes)))
|
||||||
|
|
||||||
def readMessage(offset: Int) = log.read(offset, 4096).messageSet.head.message
|
def readMessage(offset: Int) = log.read(offset, 4096).records.shallowIterator.next().record
|
||||||
|
|
||||||
if (!brokerCompression.equals("producer")) {
|
if (!brokerCompression.equals("producer")) {
|
||||||
val brokerCompressionCode = BrokerCompressionCodec.getCompressionCodec(brokerCompression)
|
val brokerCompressionCode = BrokerCompressionCodec.getCompressionCodec(brokerCompression)
|
||||||
assertEquals("Compression at offset 0 should produce " + brokerCompressionCode.name, brokerCompressionCode, readMessage(0).compressionCodec)
|
assertEquals("Compression at offset 0 should produce " + brokerCompressionCode.name, brokerCompressionCode.codec, readMessage(0).compressionType.id)
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
assertEquals("Compression at offset 0 should produce " + messageCompressionCode.name, messageCompressionCode, readMessage(0).compressionCodec)
|
assertEquals("Compression at offset 0 should produce " + messageCompressionCode.name, messageCompressionCode.codec, readMessage(0).compressionType.id)
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -1,354 +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 kafka.log
|
|
||||||
|
|
||||||
import java.io._
|
|
||||||
import java.nio._
|
|
||||||
import java.nio.channels._
|
|
||||||
|
|
||||||
import kafka.common.LongRef
|
|
||||||
import org.junit.Assert._
|
|
||||||
import kafka.utils.TestUtils._
|
|
||||||
import kafka.message._
|
|
||||||
import kafka.common.KafkaException
|
|
||||||
import org.easymock.EasyMock
|
|
||||||
import org.junit.Test
|
|
||||||
|
|
||||||
class FileMessageSetTest extends BaseMessageSetTestCases {
|
|
||||||
|
|
||||||
val messageSet = createMessageSet(messages)
|
|
||||||
|
|
||||||
def createMessageSet(messages: Seq[Message]): FileMessageSet = {
|
|
||||||
val set = new FileMessageSet(tempFile())
|
|
||||||
set.append(new ByteBufferMessageSet(NoCompressionCodec, messages: _*))
|
|
||||||
set.flush()
|
|
||||||
set
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Test that the cached size variable matches the actual file size as we append messages
|
|
||||||
*/
|
|
||||||
@Test
|
|
||||||
def testFileSize() {
|
|
||||||
assertEquals(messageSet.channel.size, messageSet.sizeInBytes)
|
|
||||||
for (_ <- 0 until 20) {
|
|
||||||
messageSet.append(singleMessageSet("abcd".getBytes))
|
|
||||||
assertEquals(messageSet.channel.size, messageSet.sizeInBytes)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Test that adding invalid bytes to the end of the log doesn't break iteration
|
|
||||||
*/
|
|
||||||
@Test
|
|
||||||
def testIterationOverPartialAndTruncation() {
|
|
||||||
testPartialWrite(0, messageSet)
|
|
||||||
testPartialWrite(2, messageSet)
|
|
||||||
testPartialWrite(4, messageSet)
|
|
||||||
testPartialWrite(5, messageSet)
|
|
||||||
testPartialWrite(6, messageSet)
|
|
||||||
}
|
|
||||||
|
|
||||||
def testPartialWrite(size: Int, messageSet: FileMessageSet) {
|
|
||||||
val buffer = ByteBuffer.allocate(size)
|
|
||||||
for (_ <- 0 until size)
|
|
||||||
buffer.put(0: Byte)
|
|
||||||
buffer.rewind()
|
|
||||||
messageSet.channel.write(buffer)
|
|
||||||
// appending those bytes should not change the contents
|
|
||||||
checkEquals(messages.iterator, messageSet.map(m => m.message).iterator)
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Iterating over the file does file reads but shouldn't change the position of the underlying FileChannel.
|
|
||||||
*/
|
|
||||||
@Test
|
|
||||||
def testIterationDoesntChangePosition() {
|
|
||||||
val position = messageSet.channel.position
|
|
||||||
checkEquals(messages.iterator, messageSet.map(m => m.message).iterator)
|
|
||||||
assertEquals(position, messageSet.channel.position)
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Test a simple append and read.
|
|
||||||
*/
|
|
||||||
@Test
|
|
||||||
def testRead() {
|
|
||||||
var read = messageSet.read(0, messageSet.sizeInBytes)
|
|
||||||
checkEquals(messageSet.iterator, read.iterator)
|
|
||||||
val items = read.iterator.toList
|
|
||||||
val sec = items.tail.head
|
|
||||||
read = messageSet.read(position = MessageSet.entrySize(sec.message), size = messageSet.sizeInBytes)
|
|
||||||
assertEquals("Try a read starting from the second message", items.tail, read.toList)
|
|
||||||
read = messageSet.read(MessageSet.entrySize(sec.message), MessageSet.entrySize(sec.message))
|
|
||||||
assertEquals("Try a read of a single message starting from the second message", List(items.tail.head), read.toList)
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Test the MessageSet.searchFor API.
|
|
||||||
*/
|
|
||||||
@Test
|
|
||||||
def testSearch() {
|
|
||||||
// append a new message with a high offset
|
|
||||||
val lastMessage = new Message("test".getBytes)
|
|
||||||
messageSet.append(new ByteBufferMessageSet(NoCompressionCodec, new LongRef(50), lastMessage))
|
|
||||||
val messages = messageSet.toSeq
|
|
||||||
var position = 0
|
|
||||||
val message1Size = MessageSet.entrySize(messages.head.message)
|
|
||||||
assertEquals("Should be able to find the first message by its offset",
|
|
||||||
(OffsetPosition(0L, position), message1Size),
|
|
||||||
messageSet.searchForOffsetWithSize(0, 0))
|
|
||||||
position += message1Size
|
|
||||||
val message2Size = MessageSet.entrySize(messages(1).message)
|
|
||||||
assertEquals("Should be able to find second message when starting from 0",
|
|
||||||
(OffsetPosition(1L, position), message2Size),
|
|
||||||
messageSet.searchForOffsetWithSize(1, 0))
|
|
||||||
assertEquals("Should be able to find second message starting from its offset",
|
|
||||||
(OffsetPosition(1L, position), message2Size),
|
|
||||||
messageSet.searchForOffsetWithSize(1, position))
|
|
||||||
position += message2Size + MessageSet.entrySize(messages(2).message)
|
|
||||||
val message4Size = MessageSet.entrySize(messages(3).message)
|
|
||||||
assertEquals("Should be able to find fourth message from a non-existant offset",
|
|
||||||
(OffsetPosition(50L, position), message4Size),
|
|
||||||
messageSet.searchForOffsetWithSize(3, position))
|
|
||||||
assertEquals("Should be able to find fourth message by correct offset",
|
|
||||||
(OffsetPosition(50L, position), message4Size),
|
|
||||||
messageSet.searchForOffsetWithSize(50, position))
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Test that the message set iterator obeys start and end slicing
|
|
||||||
*/
|
|
||||||
@Test
|
|
||||||
def testIteratorWithLimits() {
|
|
||||||
val message = messageSet.toList(1)
|
|
||||||
val start = messageSet.searchForOffsetWithSize(1, 0)._1.position
|
|
||||||
val size = message.message.size + 12
|
|
||||||
val slice = messageSet.read(start, size)
|
|
||||||
assertEquals(List(message), slice.toList)
|
|
||||||
val slice2 = messageSet.read(start, size - 1)
|
|
||||||
assertEquals(List(), slice2.toList)
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Test the truncateTo method lops off messages and appropriately updates the size
|
|
||||||
*/
|
|
||||||
@Test
|
|
||||||
def testTruncate() {
|
|
||||||
val message = messageSet.toList.head
|
|
||||||
val end = messageSet.searchForOffsetWithSize(1, 0)._1.position
|
|
||||||
messageSet.truncateTo(end)
|
|
||||||
assertEquals(List(message), messageSet.toList)
|
|
||||||
assertEquals(MessageSet.entrySize(message.message), messageSet.sizeInBytes)
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Test that truncateTo only calls truncate on the FileChannel if the size of the
|
|
||||||
* FileChannel is bigger than the target size. This is important because some JVMs
|
|
||||||
* change the mtime of the file, even if truncate should do nothing.
|
|
||||||
*/
|
|
||||||
@Test
|
|
||||||
def testTruncateNotCalledIfSizeIsSameAsTargetSize() {
|
|
||||||
val channelMock = EasyMock.createMock(classOf[FileChannel])
|
|
||||||
|
|
||||||
EasyMock.expect(channelMock.size).andReturn(42L).atLeastOnce()
|
|
||||||
EasyMock.expect(channelMock.position(42L)).andReturn(null)
|
|
||||||
EasyMock.replay(channelMock)
|
|
||||||
|
|
||||||
val msgSet = new FileMessageSet(tempFile(), channelMock)
|
|
||||||
msgSet.truncateTo(42)
|
|
||||||
|
|
||||||
EasyMock.verify(channelMock)
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Expect a KafkaException if targetSize is bigger than the size of
|
|
||||||
* the FileMessageSet.
|
|
||||||
*/
|
|
||||||
@Test
|
|
||||||
def testTruncateNotCalledIfSizeIsBiggerThanTargetSize() {
|
|
||||||
val channelMock = EasyMock.createMock(classOf[FileChannel])
|
|
||||||
|
|
||||||
EasyMock.expect(channelMock.size).andReturn(42L).atLeastOnce()
|
|
||||||
EasyMock.expect(channelMock.position(42L)).andReturn(null)
|
|
||||||
EasyMock.replay(channelMock)
|
|
||||||
|
|
||||||
val msgSet = new FileMessageSet(tempFile(), channelMock)
|
|
||||||
|
|
||||||
try {
|
|
||||||
msgSet.truncateTo(43)
|
|
||||||
fail("Should throw KafkaException")
|
|
||||||
} catch {
|
|
||||||
case _: KafkaException => // expected
|
|
||||||
}
|
|
||||||
|
|
||||||
EasyMock.verify(channelMock)
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* see #testTruncateNotCalledIfSizeIsSameAsTargetSize
|
|
||||||
*/
|
|
||||||
@Test
|
|
||||||
def testTruncateIfSizeIsDifferentToTargetSize() {
|
|
||||||
val channelMock = EasyMock.createMock(classOf[FileChannel])
|
|
||||||
|
|
||||||
EasyMock.expect(channelMock.size).andReturn(42L).atLeastOnce()
|
|
||||||
EasyMock.expect(channelMock.position(42L)).andReturn(null).once()
|
|
||||||
EasyMock.expect(channelMock.truncate(23L)).andReturn(null).once()
|
|
||||||
EasyMock.expect(channelMock.position(23L)).andReturn(null).once()
|
|
||||||
EasyMock.replay(channelMock)
|
|
||||||
|
|
||||||
val msgSet = new FileMessageSet(tempFile(), channelMock)
|
|
||||||
msgSet.truncateTo(23)
|
|
||||||
|
|
||||||
EasyMock.verify(channelMock)
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Test the new FileMessageSet with pre allocate as true
|
|
||||||
*/
|
|
||||||
@Test
|
|
||||||
def testPreallocateTrue() {
|
|
||||||
val temp = tempFile()
|
|
||||||
val set = new FileMessageSet(temp, false, 512 *1024 *1024, true)
|
|
||||||
val position = set.channel.position
|
|
||||||
val size = set.sizeInBytes()
|
|
||||||
assertEquals(0, position)
|
|
||||||
assertEquals(0, size)
|
|
||||||
assertEquals(512 *1024 *1024, temp.length)
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Test the new FileMessageSet with pre allocate as false
|
|
||||||
*/
|
|
||||||
@Test
|
|
||||||
def testPreallocateFalse() {
|
|
||||||
val temp = tempFile()
|
|
||||||
val set = new FileMessageSet(temp, false, 512 *1024 *1024, false)
|
|
||||||
val position = set.channel.position
|
|
||||||
val size = set.sizeInBytes()
|
|
||||||
assertEquals(0, position)
|
|
||||||
assertEquals(0, size)
|
|
||||||
assertEquals(0, temp.length)
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Test the new FileMessageSet with pre allocate as true and file has been clearly shut down, the file will be truncate to end of valid data.
|
|
||||||
*/
|
|
||||||
@Test
|
|
||||||
def testPreallocateClearShutdown() {
|
|
||||||
val temp = tempFile()
|
|
||||||
val set = new FileMessageSet(temp, false, 512 *1024 *1024, true)
|
|
||||||
set.append(new ByteBufferMessageSet(NoCompressionCodec, messages: _*))
|
|
||||||
val oldposition = set.channel.position
|
|
||||||
val oldsize = set.sizeInBytes()
|
|
||||||
assertEquals(messageSet.sizeInBytes, oldposition)
|
|
||||||
assertEquals(messageSet.sizeInBytes, oldsize)
|
|
||||||
set.close()
|
|
||||||
|
|
||||||
val tempReopen = new File(temp.getAbsolutePath())
|
|
||||||
val setReopen = new FileMessageSet(tempReopen, true, 512 *1024 *1024, true)
|
|
||||||
val position = setReopen.channel.position
|
|
||||||
val size = setReopen.sizeInBytes()
|
|
||||||
|
|
||||||
assertEquals(oldposition, position)
|
|
||||||
assertEquals(oldposition, size)
|
|
||||||
assertEquals(oldposition, tempReopen.length)
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
def testFormatConversionWithPartialMessage() {
|
|
||||||
val message = messageSet.toList(1)
|
|
||||||
val start = messageSet.searchForOffsetWithSize(1, 0)._1.position
|
|
||||||
val size = message.message.size + 12
|
|
||||||
val slice = messageSet.read(start, size - 1)
|
|
||||||
val messageV0 = slice.toMessageFormat(Message.MagicValue_V0)
|
|
||||||
assertEquals("No message should be there", 0, messageV0.size)
|
|
||||||
assertEquals(s"There should be ${size - 1} bytes", size - 1, messageV0.sizeInBytes)
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
def testMessageFormatConversion() {
|
|
||||||
|
|
||||||
// Prepare messages.
|
|
||||||
val offsets = Seq(0L, 2L)
|
|
||||||
val messagesV0 = Seq(new Message("hello".getBytes, "k1".getBytes, Message.NoTimestamp, Message.MagicValue_V0),
|
|
||||||
new Message("goodbye".getBytes, "k2".getBytes, Message.NoTimestamp, Message.MagicValue_V0))
|
|
||||||
val messageSetV0 = new ByteBufferMessageSet(
|
|
||||||
compressionCodec = NoCompressionCodec,
|
|
||||||
offsetSeq = offsets,
|
|
||||||
messages = messagesV0:_*)
|
|
||||||
val compressedMessageSetV0 = new ByteBufferMessageSet(
|
|
||||||
compressionCodec = DefaultCompressionCodec,
|
|
||||||
offsetSeq = offsets,
|
|
||||||
messages = messagesV0:_*)
|
|
||||||
|
|
||||||
val messagesV1 = Seq(new Message("hello".getBytes, "k1".getBytes, 1L, Message.MagicValue_V1),
|
|
||||||
new Message("goodbye".getBytes, "k2".getBytes, 2L, Message.MagicValue_V1))
|
|
||||||
val messageSetV1 = new ByteBufferMessageSet(
|
|
||||||
compressionCodec = NoCompressionCodec,
|
|
||||||
offsetSeq = offsets,
|
|
||||||
messages = messagesV1:_*)
|
|
||||||
val compressedMessageSetV1 = new ByteBufferMessageSet(
|
|
||||||
compressionCodec = DefaultCompressionCodec,
|
|
||||||
offsetSeq = offsets,
|
|
||||||
messages = messagesV1:_*)
|
|
||||||
|
|
||||||
// Down conversion
|
|
||||||
// down conversion for non-compressed messages
|
|
||||||
var fileMessageSet = new FileMessageSet(tempFile())
|
|
||||||
fileMessageSet.append(messageSetV1)
|
|
||||||
fileMessageSet.flush()
|
|
||||||
var convertedMessageSet = fileMessageSet.toMessageFormat(Message.MagicValue_V0)
|
|
||||||
verifyConvertedMessageSet(convertedMessageSet, Message.MagicValue_V0)
|
|
||||||
|
|
||||||
// down conversion for compressed messages
|
|
||||||
fileMessageSet = new FileMessageSet(tempFile())
|
|
||||||
fileMessageSet.append(compressedMessageSetV1)
|
|
||||||
fileMessageSet.flush()
|
|
||||||
convertedMessageSet = fileMessageSet.toMessageFormat(Message.MagicValue_V0)
|
|
||||||
verifyConvertedMessageSet(convertedMessageSet, Message.MagicValue_V0)
|
|
||||||
|
|
||||||
// Up conversion. In reality we only do down conversion, but up conversion should work as well.
|
|
||||||
// up conversion for non-compressed messages
|
|
||||||
fileMessageSet = new FileMessageSet(tempFile())
|
|
||||||
fileMessageSet.append(messageSetV0)
|
|
||||||
fileMessageSet.flush()
|
|
||||||
convertedMessageSet = fileMessageSet.toMessageFormat(Message.MagicValue_V1)
|
|
||||||
verifyConvertedMessageSet(convertedMessageSet, Message.MagicValue_V1)
|
|
||||||
|
|
||||||
// up conversion for compressed messages
|
|
||||||
fileMessageSet = new FileMessageSet(tempFile())
|
|
||||||
fileMessageSet.append(compressedMessageSetV0)
|
|
||||||
fileMessageSet.flush()
|
|
||||||
convertedMessageSet = fileMessageSet.toMessageFormat(Message.MagicValue_V1)
|
|
||||||
verifyConvertedMessageSet(convertedMessageSet, Message.MagicValue_V1)
|
|
||||||
|
|
||||||
def verifyConvertedMessageSet(convertedMessageSet: MessageSet, magicByte: Byte) {
|
|
||||||
var i = 0
|
|
||||||
for (messageAndOffset <- convertedMessageSet) {
|
|
||||||
assertEquals("magic byte should be 1", magicByte, messageAndOffset.message.magic)
|
|
||||||
assertEquals("offset should not change", offsets(i), messageAndOffset.offset)
|
|
||||||
assertEquals("key should not change", messagesV0(i).key, messageAndOffset.message.key)
|
|
||||||
assertEquals("payload should not change", messagesV0(i).payload, messageAndOffset.message.payload)
|
|
||||||
i += 1
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
@ -22,10 +22,9 @@ import java.util.Properties
|
||||||
|
|
||||||
import kafka.api.{KAFKA_0_10_0_IV1, KAFKA_0_9_0}
|
import kafka.api.{KAFKA_0_10_0_IV1, KAFKA_0_9_0}
|
||||||
import kafka.common.TopicAndPartition
|
import kafka.common.TopicAndPartition
|
||||||
import kafka.message._
|
|
||||||
import kafka.server.OffsetCheckpoint
|
import kafka.server.OffsetCheckpoint
|
||||||
import kafka.utils._
|
import kafka.utils._
|
||||||
import org.apache.kafka.common.record.CompressionType
|
import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Record}
|
||||||
import org.apache.kafka.common.utils.Utils
|
import org.apache.kafka.common.utils.Utils
|
||||||
import org.junit.Assert._
|
import org.junit.Assert._
|
||||||
import org.junit._
|
import org.junit._
|
||||||
|
|
@ -43,7 +42,7 @@ import scala.util.Random
|
||||||
@RunWith(value = classOf[Parameterized])
|
@RunWith(value = classOf[Parameterized])
|
||||||
class LogCleanerIntegrationTest(compressionCodec: String) {
|
class LogCleanerIntegrationTest(compressionCodec: String) {
|
||||||
|
|
||||||
val codec = CompressionCodec.getCompressionCodec(compressionCodec)
|
val codec = CompressionType.forName(compressionCodec)
|
||||||
val time = new MockTime()
|
val time = new MockTime()
|
||||||
val segmentSize = 256
|
val segmentSize = 256
|
||||||
val deleteDelay = 1000
|
val deleteDelay = 1000
|
||||||
|
|
@ -56,7 +55,7 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
|
||||||
@Test
|
@Test
|
||||||
def cleanerTest() {
|
def cleanerTest() {
|
||||||
val largeMessageKey = 20
|
val largeMessageKey = 20
|
||||||
val (largeMessageValue, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, Message.MagicValue_V1)
|
val (largeMessageValue, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, Record.MAGIC_VALUE_V1)
|
||||||
val maxMessageSize = largeMessageSet.sizeInBytes
|
val maxMessageSize = largeMessageSet.sizeInBytes
|
||||||
|
|
||||||
cleaner = makeCleaner(parts = 3, maxMessageSize = maxMessageSize)
|
cleaner = makeCleaner(parts = 3, maxMessageSize = maxMessageSize)
|
||||||
|
|
@ -133,13 +132,13 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
|
||||||
}
|
}
|
||||||
|
|
||||||
// returns (value, ByteBufferMessageSet)
|
// returns (value, ByteBufferMessageSet)
|
||||||
private def createLargeSingleMessageSet(key: Int, messageFormatVersion: Byte): (String, ByteBufferMessageSet) = {
|
private def createLargeSingleMessageSet(key: Int, messageFormatVersion: Byte): (String, MemoryRecords) = {
|
||||||
def messageValue(length: Int): String = {
|
def messageValue(length: Int): String = {
|
||||||
val random = new Random(0)
|
val random = new Random(0)
|
||||||
new String(random.alphanumeric.take(length).toArray)
|
new String(random.alphanumeric.take(length).toArray)
|
||||||
}
|
}
|
||||||
val value = messageValue(128)
|
val value = messageValue(128)
|
||||||
val messageSet = TestUtils.singleMessageSet(payload = value.getBytes, codec = codec, key = key.toString.getBytes,
|
val messageSet = TestUtils.singletonRecords(value = value.getBytes, codec = codec, key = key.toString.getBytes,
|
||||||
magicValue = messageFormatVersion)
|
magicValue = messageFormatVersion)
|
||||||
(value, messageSet)
|
(value, messageSet)
|
||||||
}
|
}
|
||||||
|
|
@ -147,9 +146,9 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
|
||||||
@Test
|
@Test
|
||||||
def testCleanerWithMessageFormatV0(): Unit = {
|
def testCleanerWithMessageFormatV0(): Unit = {
|
||||||
val largeMessageKey = 20
|
val largeMessageKey = 20
|
||||||
val (largeMessageValue, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, Message.MagicValue_V0)
|
val (largeMessageValue, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, Record.MAGIC_VALUE_V0)
|
||||||
val maxMessageSize = codec match {
|
val maxMessageSize = codec match {
|
||||||
case NoCompressionCodec => largeMessageSet.sizeInBytes
|
case CompressionType.NONE => largeMessageSet.sizeInBytes
|
||||||
case _ =>
|
case _ =>
|
||||||
// the broker assigns absolute offsets for message format 0 which potentially causes the compressed size to
|
// the broker assigns absolute offsets for message format 0 which potentially causes the compressed size to
|
||||||
// increase because the broker offsets are larger than the ones assigned by the client
|
// increase because the broker offsets are larger than the ones assigned by the client
|
||||||
|
|
@ -165,7 +164,7 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
|
||||||
props.put(LogConfig.MessageFormatVersionProp, KAFKA_0_9_0.version)
|
props.put(LogConfig.MessageFormatVersionProp, KAFKA_0_9_0.version)
|
||||||
log.config = new LogConfig(props)
|
log.config = new LogConfig(props)
|
||||||
|
|
||||||
val appends = writeDups(numKeys = 100, numDups = 3, log = log, codec = codec, magicValue = Message.MagicValue_V0)
|
val appends = writeDups(numKeys = 100, numDups = 3, log = log, codec = codec, magicValue = Record.MAGIC_VALUE_V0)
|
||||||
val startSize = log.size
|
val startSize = log.size
|
||||||
cleaner.startup()
|
cleaner.startup()
|
||||||
|
|
||||||
|
|
@ -177,14 +176,14 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
|
||||||
checkLogAfterAppendingDups(log, startSize, appends)
|
checkLogAfterAppendingDups(log, startSize, appends)
|
||||||
|
|
||||||
val appends2: Seq[(Int, String, Long)] = {
|
val appends2: Seq[(Int, String, Long)] = {
|
||||||
val dupsV0 = writeDups(numKeys = 40, numDups = 3, log = log, codec = codec, magicValue = Message.MagicValue_V0)
|
val dupsV0 = writeDups(numKeys = 40, numDups = 3, log = log, codec = codec, magicValue = Record.MAGIC_VALUE_V0)
|
||||||
val appendInfo = log.append(largeMessageSet, assignOffsets = true)
|
val appendInfo = log.append(largeMessageSet, assignOffsets = true)
|
||||||
val largeMessageOffset = appendInfo.firstOffset
|
val largeMessageOffset = appendInfo.firstOffset
|
||||||
|
|
||||||
// also add some messages with version 1 to check that we handle mixed format versions correctly
|
// also add some messages with version 1 to check that we handle mixed format versions correctly
|
||||||
props.put(LogConfig.MessageFormatVersionProp, KAFKA_0_10_0_IV1.version)
|
props.put(LogConfig.MessageFormatVersionProp, KAFKA_0_10_0_IV1.version)
|
||||||
log.config = new LogConfig(props)
|
log.config = new LogConfig(props)
|
||||||
val dupsV1 = writeDups(startKey = 30, numKeys = 40, numDups = 3, log = log, codec = codec, magicValue = Message.MagicValue_V1)
|
val dupsV1 = writeDups(startKey = 30, numKeys = 40, numDups = 3, log = log, codec = codec, magicValue = Record.MAGIC_VALUE_V1)
|
||||||
appends ++ dupsV0 ++ Seq((largeMessageKey, largeMessageValue, largeMessageOffset)) ++ dupsV1
|
appends ++ dupsV0 ++ Seq((largeMessageKey, largeMessageValue, largeMessageOffset)) ++ dupsV1
|
||||||
}
|
}
|
||||||
val firstDirty2 = log.activeSegment.baseOffset
|
val firstDirty2 = log.activeSegment.baseOffset
|
||||||
|
|
@ -205,15 +204,15 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
|
||||||
|
|
||||||
// with compression enabled, these messages will be written as a single message containing
|
// with compression enabled, these messages will be written as a single message containing
|
||||||
// all of the individual messages
|
// all of the individual messages
|
||||||
var appendsV0 = writeDupsSingleMessageSet(numKeys = 2, numDups = 3, log = log, codec = codec, magicValue = Message.MagicValue_V0)
|
var appendsV0 = writeDupsSingleMessageSet(numKeys = 2, numDups = 3, log = log, codec = codec, magicValue = Record.MAGIC_VALUE_V0)
|
||||||
appendsV0 ++= writeDupsSingleMessageSet(numKeys = 2, startKey = 3, numDups = 2, log = log, codec = codec, magicValue = Message.MagicValue_V0)
|
appendsV0 ++= writeDupsSingleMessageSet(numKeys = 2, startKey = 3, numDups = 2, log = log, codec = codec, magicValue = Record.MAGIC_VALUE_V0)
|
||||||
|
|
||||||
props.put(LogConfig.MessageFormatVersionProp, KAFKA_0_10_0_IV1.version)
|
props.put(LogConfig.MessageFormatVersionProp, KAFKA_0_10_0_IV1.version)
|
||||||
log.config = new LogConfig(props)
|
log.config = new LogConfig(props)
|
||||||
|
|
||||||
var appendsV1 = writeDupsSingleMessageSet(startKey = 4, numKeys = 2, numDups = 2, log = log, codec = codec, magicValue = Message.MagicValue_V1)
|
var appendsV1 = writeDupsSingleMessageSet(startKey = 4, numKeys = 2, numDups = 2, log = log, codec = codec, magicValue = Record.MAGIC_VALUE_V1)
|
||||||
appendsV1 ++= writeDupsSingleMessageSet(startKey = 4, numKeys = 2, numDups = 2, log = log, codec = codec, magicValue = Message.MagicValue_V1)
|
appendsV1 ++= writeDupsSingleMessageSet(startKey = 4, numKeys = 2, numDups = 2, log = log, codec = codec, magicValue = Record.MAGIC_VALUE_V1)
|
||||||
appendsV1 ++= writeDupsSingleMessageSet(startKey = 6, numKeys = 2, numDups = 2, log = log, codec = codec, magicValue = Message.MagicValue_V1)
|
appendsV1 ++= writeDupsSingleMessageSet(startKey = 6, numKeys = 2, numDups = 2, log = log, codec = codec, magicValue = Record.MAGIC_VALUE_V1)
|
||||||
|
|
||||||
val appends = appendsV0 ++ appendsV1
|
val appends = appendsV0 ++ appendsV1
|
||||||
|
|
||||||
|
|
@ -250,32 +249,27 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
|
||||||
}
|
}
|
||||||
|
|
||||||
private def readFromLog(log: Log): Iterable[(Int, String, Long)] = {
|
private def readFromLog(log: Log): Iterable[(Int, String, Long)] = {
|
||||||
|
import JavaConverters._
|
||||||
def messageIterator(entry: MessageAndOffset): Iterator[MessageAndOffset] =
|
for (segment <- log.logSegments; deepLogEntry <- segment.log.deepIterator.asScala) yield {
|
||||||
// create single message iterator or deep iterator depending on compression codec
|
val key = TestUtils.readString(deepLogEntry.record.key).toInt
|
||||||
if (entry.message.compressionCodec == NoCompressionCodec) Iterator(entry)
|
val value = TestUtils.readString(deepLogEntry.record.value)
|
||||||
else ByteBufferMessageSet.deepIterator(entry)
|
(key, value, deepLogEntry.offset)
|
||||||
|
|
||||||
for (segment <- log.logSegments; entry <- segment.log; messageAndOffset <- messageIterator(entry)) yield {
|
|
||||||
val key = TestUtils.readString(messageAndOffset.message.key).toInt
|
|
||||||
val value = TestUtils.readString(messageAndOffset.message.payload)
|
|
||||||
(key, value, messageAndOffset.offset)
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private def writeDups(numKeys: Int, numDups: Int, log: Log, codec: CompressionCodec,
|
private def writeDups(numKeys: Int, numDups: Int, log: Log, codec: CompressionType,
|
||||||
startKey: Int = 0, magicValue: Byte = Message.CurrentMagicValue): Seq[(Int, String, Long)] = {
|
startKey: Int = 0, magicValue: Byte = Record.CURRENT_MAGIC_VALUE): Seq[(Int, String, Long)] = {
|
||||||
for(_ <- 0 until numDups; key <- startKey until (startKey + numKeys)) yield {
|
for(_ <- 0 until numDups; key <- startKey until (startKey + numKeys)) yield {
|
||||||
val payload = counter.toString
|
val value = counter.toString
|
||||||
val appendInfo = log.append(TestUtils.singleMessageSet(payload = payload.toString.getBytes, codec = codec,
|
val appendInfo = log.append(TestUtils.singletonRecords(value = value.toString.getBytes, codec = codec,
|
||||||
key = key.toString.getBytes, magicValue = magicValue), assignOffsets = true)
|
key = key.toString.getBytes, magicValue = magicValue), assignOffsets = true)
|
||||||
counter += 1
|
counter += 1
|
||||||
(key, payload, appendInfo.firstOffset)
|
(key, value, appendInfo.firstOffset)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private def writeDupsSingleMessageSet(numKeys: Int, numDups: Int, log: Log, codec: CompressionCodec,
|
private def writeDupsSingleMessageSet(numKeys: Int, numDups: Int, log: Log, codec: CompressionType,
|
||||||
startKey: Int = 0, magicValue: Byte = Message.CurrentMagicValue): Seq[(Int, String, Long)] = {
|
startKey: Int = 0, magicValue: Byte = Record.CURRENT_MAGIC_VALUE): Seq[(Int, String, Long)] = {
|
||||||
val kvs = for (_ <- 0 until numDups; key <- startKey until (startKey + numKeys)) yield {
|
val kvs = for (_ <- 0 until numDups; key <- startKey until (startKey + numKeys)) yield {
|
||||||
val payload = counter.toString
|
val payload = counter.toString
|
||||||
counter += 1
|
counter += 1
|
||||||
|
|
@ -283,11 +277,11 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
|
||||||
}
|
}
|
||||||
|
|
||||||
val messages = kvs.map { case (key, payload) =>
|
val messages = kvs.map { case (key, payload) =>
|
||||||
new Message(payload.toString.getBytes, key.toString.getBytes, Message.NoTimestamp, magicValue)
|
Record.create(magicValue, key.toString.getBytes, payload.toString.getBytes)
|
||||||
}
|
}
|
||||||
|
|
||||||
val messageSet = new ByteBufferMessageSet(compressionCodec = codec, messages: _*)
|
val records = MemoryRecords.withRecords(codec, messages: _*)
|
||||||
val appendInfo = log.append(messageSet, assignOffsets = true)
|
val appendInfo = log.append(records, assignOffsets = true)
|
||||||
val offsets = appendInfo.firstOffset to appendInfo.lastOffset
|
val offsets = appendInfo.firstOffset to appendInfo.lastOffset
|
||||||
|
|
||||||
kvs.zip(offsets).map { case (kv, offset) => (kv._1, kv._2, offset) }
|
kvs.zip(offsets).map { case (kv, offset) => (kv._1, kv._2, offset) }
|
||||||
|
|
|
||||||
|
|
@ -21,7 +21,6 @@ import java.io.File
|
||||||
import java.util.Properties
|
import java.util.Properties
|
||||||
|
|
||||||
import kafka.common.TopicAndPartition
|
import kafka.common.TopicAndPartition
|
||||||
import kafka.message._
|
|
||||||
import kafka.utils._
|
import kafka.utils._
|
||||||
import org.apache.kafka.common.record.CompressionType
|
import org.apache.kafka.common.record.CompressionType
|
||||||
import org.apache.kafka.common.utils.Utils
|
import org.apache.kafka.common.utils.Utils
|
||||||
|
|
@ -33,7 +32,6 @@ import org.junit.runners.Parameterized.Parameters
|
||||||
|
|
||||||
import scala.collection._
|
import scala.collection._
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This is an integration test that tests the fully integrated log cleaner
|
* This is an integration test that tests the fully integrated log cleaner
|
||||||
*/
|
*/
|
||||||
|
|
@ -52,7 +50,7 @@ class LogCleanerLagIntegrationTest(compressionCodecName: String) extends Logging
|
||||||
val logDir = TestUtils.tempDir()
|
val logDir = TestUtils.tempDir()
|
||||||
var counter = 0
|
var counter = 0
|
||||||
val topics = Array(TopicAndPartition("log", 0), TopicAndPartition("log", 1), TopicAndPartition("log", 2))
|
val topics = Array(TopicAndPartition("log", 0), TopicAndPartition("log", 1), TopicAndPartition("log", 2))
|
||||||
val compressionCodec = CompressionCodec.getCompressionCodec(compressionCodecName)
|
val compressionCodec = CompressionType.forName(compressionCodecName)
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
def cleanerTest(): Unit = {
|
def cleanerTest(): Unit = {
|
||||||
|
|
@ -96,7 +94,7 @@ class LogCleanerLagIntegrationTest(compressionCodecName: String) extends Logging
|
||||||
|
|
||||||
val compactedSize = log.logSegments(0L, activeSegAtT0.baseOffset).map(_.size).sum
|
val compactedSize = log.logSegments(0L, activeSegAtT0.baseOffset).map(_.size).sum
|
||||||
debug(s"after cleaning the compacted size up to active segment at T0: $compactedSize")
|
debug(s"after cleaning the compacted size up to active segment at T0: $compactedSize")
|
||||||
val lastCleaned = cleaner.cleanerManager.allCleanerCheckpoints.get(TopicAndPartition("log", 0)).get
|
val lastCleaned = cleaner.cleanerManager.allCleanerCheckpoints(TopicAndPartition("log", 0))
|
||||||
assertTrue(s"log cleaner should have processed up to offset $firstBlock1SegmentBaseOffset, but lastCleaned=$lastCleaned", lastCleaned >= firstBlock1SegmentBaseOffset)
|
assertTrue(s"log cleaner should have processed up to offset $firstBlock1SegmentBaseOffset, but lastCleaned=$lastCleaned", lastCleaned >= firstBlock1SegmentBaseOffset)
|
||||||
assertTrue(s"log should have been compacted: size up to offset of active segment at T0=$sizeUpToActiveSegmentAtT0 compacted size=$compactedSize",
|
assertTrue(s"log should have been compacted: size up to offset of active segment at T0=$sizeUpToActiveSegmentAtT0 compacted size=$compactedSize",
|
||||||
sizeUpToActiveSegmentAtT0 > compactedSize)
|
sizeUpToActiveSegmentAtT0 > compactedSize)
|
||||||
|
|
@ -106,23 +104,19 @@ class LogCleanerLagIntegrationTest(compressionCodecName: String) extends Logging
|
||||||
}
|
}
|
||||||
|
|
||||||
private def readFromLog(log: Log): Iterable[(Int, Int)] = {
|
private def readFromLog(log: Log): Iterable[(Int, Int)] = {
|
||||||
for (segment <- log.logSegments; entry <- segment.log; messageAndOffset <- {
|
import JavaConverters._
|
||||||
// create single message iterator or deep iterator depending on compression codec
|
|
||||||
if (entry.message.compressionCodec == NoCompressionCodec)
|
for (segment <- log.logSegments; logEntry <- segment.log.deepIterator.asScala) yield {
|
||||||
Stream.cons(entry, Stream.empty).iterator
|
val key = TestUtils.readString(logEntry.record.key).toInt
|
||||||
else
|
val value = TestUtils.readString(logEntry.record.value).toInt
|
||||||
ByteBufferMessageSet.deepIterator(entry)
|
|
||||||
}) yield {
|
|
||||||
val key = TestUtils.readString(messageAndOffset.message.key).toInt
|
|
||||||
val value = TestUtils.readString(messageAndOffset.message.payload).toInt
|
|
||||||
key -> value
|
key -> value
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private def writeDups(numKeys: Int, numDups: Int, log: Log, codec: CompressionCodec, timestamp: Long): Seq[(Int, Int)] = {
|
private def writeDups(numKeys: Int, numDups: Int, log: Log, codec: CompressionType, timestamp: Long): Seq[(Int, Int)] = {
|
||||||
for (_ <- 0 until numDups; key <- 0 until numKeys) yield {
|
for (_ <- 0 until numDups; key <- 0 until numKeys) yield {
|
||||||
val count = counter
|
val count = counter
|
||||||
log.append(TestUtils.singleMessageSet(payload = counter.toString.getBytes, codec = codec, key = key.toString.getBytes, timestamp = timestamp), assignOffsets = true)
|
log.append(TestUtils.singletonRecords(value = counter.toString.getBytes, codec = codec, key = key.toString.getBytes, timestamp = timestamp), assignOffsets = true)
|
||||||
counter += 1
|
counter += 1
|
||||||
(key, count)
|
(key, count)
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -21,8 +21,8 @@ import java.io.File
|
||||||
import java.util.Properties
|
import java.util.Properties
|
||||||
|
|
||||||
import kafka.common._
|
import kafka.common._
|
||||||
import kafka.message._
|
|
||||||
import kafka.utils._
|
import kafka.utils._
|
||||||
|
import org.apache.kafka.common.record.{MemoryRecords, Record}
|
||||||
import org.apache.kafka.common.utils.Utils
|
import org.apache.kafka.common.utils.Utils
|
||||||
import org.junit.Assert._
|
import org.junit.Assert._
|
||||||
import org.junit.{After, Test}
|
import org.junit.{After, Test}
|
||||||
|
|
@ -54,8 +54,8 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
def testLogsWithSegmentsToDeleteShouldNotConsiderCleanupPolicyDeleteLogs(): Unit = {
|
def testLogsWithSegmentsToDeleteShouldNotConsiderCleanupPolicyDeleteLogs(): Unit = {
|
||||||
val messageSet = TestUtils.singleMessageSet("test".getBytes)
|
val records = TestUtils.singletonRecords("test".getBytes)
|
||||||
val log: Log = createLog(messageSet.sizeInBytes * 5, LogConfig.Delete)
|
val log: Log = createLog(records.sizeInBytes * 5, LogConfig.Delete)
|
||||||
val cleanerManager: LogCleanerManager = createCleanerManager(log)
|
val cleanerManager: LogCleanerManager = createCleanerManager(log)
|
||||||
|
|
||||||
val readyToDelete = cleanerManager.deletableLogs().size
|
val readyToDelete = cleanerManager.deletableLogs().size
|
||||||
|
|
@ -67,8 +67,8 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
def testLogsWithSegmentsToDeleteShouldConsiderCleanupPolicyCompactDeleteLogs(): Unit = {
|
def testLogsWithSegmentsToDeleteShouldConsiderCleanupPolicyCompactDeleteLogs(): Unit = {
|
||||||
val messageSet = TestUtils.singleMessageSet("test".getBytes, key="test".getBytes)
|
val records = TestUtils.singletonRecords("test".getBytes, key="test".getBytes)
|
||||||
val log: Log = createLog(messageSet.sizeInBytes * 5, LogConfig.Compact + "," + LogConfig.Delete)
|
val log: Log = createLog(records.sizeInBytes * 5, LogConfig.Compact + "," + LogConfig.Delete)
|
||||||
val cleanerManager: LogCleanerManager = createCleanerManager(log)
|
val cleanerManager: LogCleanerManager = createCleanerManager(log)
|
||||||
|
|
||||||
val readyToDelete = cleanerManager.deletableLogs().size
|
val readyToDelete = cleanerManager.deletableLogs().size
|
||||||
|
|
@ -81,8 +81,8 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
def testLogsWithSegmentsToDeleteShouldNotConsiderCleanupPolicyCompactLogs(): Unit = {
|
def testLogsWithSegmentsToDeleteShouldNotConsiderCleanupPolicyCompactLogs(): Unit = {
|
||||||
val messageSet = TestUtils.singleMessageSet("test".getBytes, key="test".getBytes)
|
val records = TestUtils.singletonRecords("test".getBytes, key="test".getBytes)
|
||||||
val log: Log = createLog(messageSet.sizeInBytes * 5, LogConfig.Compact)
|
val log: Log = createLog(records.sizeInBytes * 5, LogConfig.Compact)
|
||||||
val cleanerManager: LogCleanerManager = createCleanerManager(log)
|
val cleanerManager: LogCleanerManager = createCleanerManager(log)
|
||||||
|
|
||||||
val readyToDelete = cleanerManager.deletableLogs().size
|
val readyToDelete = cleanerManager.deletableLogs().size
|
||||||
|
|
@ -100,7 +100,7 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
|
||||||
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
|
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
|
||||||
|
|
||||||
while(log.numberOfSegments < 8)
|
while(log.numberOfSegments < 8)
|
||||||
log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt, timestamp = time.milliseconds))
|
log.append(logEntries(log.logEndOffset.toInt, log.logEndOffset.toInt, timestamp = time.milliseconds))
|
||||||
|
|
||||||
val topicAndPartition = TopicAndPartition("log", 0)
|
val topicAndPartition = TopicAndPartition("log", 0)
|
||||||
val lastClean = Map(topicAndPartition-> 0L)
|
val lastClean = Map(topicAndPartition-> 0L)
|
||||||
|
|
@ -123,7 +123,7 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
|
||||||
|
|
||||||
val t0 = time.milliseconds
|
val t0 = time.milliseconds
|
||||||
while(log.numberOfSegments < 4)
|
while(log.numberOfSegments < 4)
|
||||||
log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt, timestamp = t0))
|
log.append(logEntries(log.logEndOffset.toInt, log.logEndOffset.toInt, timestamp = t0))
|
||||||
|
|
||||||
val activeSegAtT0 = log.activeSegment
|
val activeSegAtT0 = log.activeSegment
|
||||||
|
|
||||||
|
|
@ -131,7 +131,7 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
|
||||||
val t1 = time.milliseconds
|
val t1 = time.milliseconds
|
||||||
|
|
||||||
while (log.numberOfSegments < 8)
|
while (log.numberOfSegments < 8)
|
||||||
log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt, timestamp = t1))
|
log.append(logEntries(log.logEndOffset.toInt, log.logEndOffset.toInt, timestamp = t1))
|
||||||
|
|
||||||
val topicAndPartition = TopicAndPartition("log", 0)
|
val topicAndPartition = TopicAndPartition("log", 0)
|
||||||
val lastClean = Map(topicAndPartition-> 0L)
|
val lastClean = Map(topicAndPartition-> 0L)
|
||||||
|
|
@ -155,7 +155,7 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
|
||||||
|
|
||||||
val t0 = time.milliseconds
|
val t0 = time.milliseconds
|
||||||
while (log.numberOfSegments < 8)
|
while (log.numberOfSegments < 8)
|
||||||
log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt, timestamp = t0))
|
log.append(logEntries(log.logEndOffset.toInt, log.logEndOffset.toInt, timestamp = t0))
|
||||||
|
|
||||||
time.sleep(compactionLag + 1)
|
time.sleep(compactionLag + 1)
|
||||||
|
|
||||||
|
|
@ -192,10 +192,7 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
|
||||||
private def makeLog(dir: File = logDir, config: LogConfig = logConfig) =
|
private def makeLog(dir: File = logDir, config: LogConfig = logConfig) =
|
||||||
new Log(dir = dir, config = config, recoveryPoint = 0L, scheduler = time.scheduler, time = time)
|
new Log(dir = dir, config = config, recoveryPoint = 0L, scheduler = time.scheduler, time = time)
|
||||||
|
|
||||||
private def message(key: Int, value: Int, timestamp: Long) =
|
private def logEntries(key: Int, value: Int, timestamp: Long) =
|
||||||
new ByteBufferMessageSet(new Message(key = key.toString.getBytes,
|
MemoryRecords.withRecords(Record.create(timestamp, key.toString.getBytes, value.toString.getBytes))
|
||||||
bytes = value.toString.getBytes,
|
|
||||||
timestamp = timestamp,
|
|
||||||
magicValue = Message.MagicValue_V1))
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -17,21 +17,21 @@
|
||||||
|
|
||||||
package kafka.log
|
package kafka.log
|
||||||
|
|
||||||
import java.io.{DataOutputStream, File}
|
import java.io.File
|
||||||
import java.nio._
|
import java.nio._
|
||||||
import java.nio.file.Paths
|
import java.nio.file.Paths
|
||||||
import java.util.Properties
|
import java.util.Properties
|
||||||
|
|
||||||
import kafka.common._
|
import kafka.common._
|
||||||
import kafka.message._
|
|
||||||
import kafka.utils._
|
import kafka.utils._
|
||||||
import org.apache.kafka.common.record.{MemoryRecords, TimestampType}
|
import org.apache.kafka.common.record._
|
||||||
import org.apache.kafka.common.utils.Utils
|
import org.apache.kafka.common.utils.Utils
|
||||||
import org.junit.Assert._
|
import org.junit.Assert._
|
||||||
import org.junit.{After, Test}
|
import org.junit.{After, Test}
|
||||||
import org.scalatest.junit.JUnitSuite
|
import org.scalatest.junit.JUnitSuite
|
||||||
|
|
||||||
import scala.collection._
|
import scala.collection._
|
||||||
|
import JavaConverters._
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Unit tests for the log cleaning logic
|
* Unit tests for the log cleaning logic
|
||||||
|
|
@ -66,7 +66,7 @@ class LogCleanerTest extends JUnitSuite {
|
||||||
|
|
||||||
// append messages to the log until we have four segments
|
// append messages to the log until we have four segments
|
||||||
while(log.numberOfSegments < 4)
|
while(log.numberOfSegments < 4)
|
||||||
log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt))
|
log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
|
||||||
val keysFound = keysInLog(log)
|
val keysFound = keysInLog(log)
|
||||||
assertEquals(0L until log.logEndOffset, keysFound)
|
assertEquals(0L until log.logEndOffset, keysFound)
|
||||||
|
|
||||||
|
|
@ -100,7 +100,7 @@ class LogCleanerTest extends JUnitSuite {
|
||||||
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
|
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
|
||||||
|
|
||||||
while(log.numberOfSegments < 2)
|
while(log.numberOfSegments < 2)
|
||||||
log.append(message(log.logEndOffset.toInt, Array.fill(largeMessageSize)(0: Byte)))
|
log.append(record(log.logEndOffset.toInt, Array.fill(largeMessageSize)(0: Byte)))
|
||||||
val keysFound = keysInLog(log)
|
val keysFound = keysInLog(log)
|
||||||
assertEquals(0L until log.logEndOffset, keysFound)
|
assertEquals(0L until log.logEndOffset, keysFound)
|
||||||
|
|
||||||
|
|
@ -123,23 +123,23 @@ class LogCleanerTest extends JUnitSuite {
|
||||||
logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer)
|
logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer)
|
||||||
|
|
||||||
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
|
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
|
||||||
|
|
||||||
// append messages with the keys 0 through N
|
// append messages with the keys 0 through N
|
||||||
while(log.numberOfSegments < 2)
|
while(log.numberOfSegments < 2)
|
||||||
log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt))
|
log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
|
||||||
|
|
||||||
// delete all even keys between 0 and N
|
// delete all even keys between 0 and N
|
||||||
val leo = log.logEndOffset
|
val leo = log.logEndOffset
|
||||||
for(key <- 0 until leo.toInt by 2)
|
for(key <- 0 until leo.toInt by 2)
|
||||||
log.append(deleteMessage(key))
|
log.append(tombstoneRecord(key))
|
||||||
|
|
||||||
// append some new unique keys to pad out to a new active segment
|
// append some new unique keys to pad out to a new active segment
|
||||||
while(log.numberOfSegments < 4)
|
while(log.numberOfSegments < 4)
|
||||||
log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt))
|
log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
|
||||||
|
|
||||||
cleaner.clean(LogToClean(TopicAndPartition("test", 0), log, 0, log.activeSegment.baseOffset))
|
cleaner.clean(LogToClean(TopicAndPartition("test", 0), log, 0, log.activeSegment.baseOffset))
|
||||||
val keys = keysInLog(log).toSet
|
val keys = keysInLog(log).toSet
|
||||||
assertTrue("None of the keys we deleted should still exist.",
|
assertTrue("None of the keys we deleted should still exist.",
|
||||||
(0 until leo.toInt by 2).forall(!keys.contains(_)))
|
(0 until leo.toInt by 2).forall(!keys.contains(_)))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -151,11 +151,11 @@ class LogCleanerTest extends JUnitSuite {
|
||||||
|
|
||||||
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
|
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
|
||||||
|
|
||||||
log.append(message(0,0)) // offset 0
|
log.append(record(0,0)) // offset 0
|
||||||
log.append(message(1,1)) // offset 1
|
log.append(record(1,1)) // offset 1
|
||||||
log.append(message(0,0)) // offset 2
|
log.append(record(0,0)) // offset 2
|
||||||
log.append(message(1,1)) // offset 3
|
log.append(record(1,1)) // offset 3
|
||||||
log.append(message(0,0)) // offset 4
|
log.append(record(0,0)) // offset 4
|
||||||
// roll the segment, so we can clean the messages already appended
|
// roll the segment, so we can clean the messages already appended
|
||||||
log.roll()
|
log.roll()
|
||||||
|
|
||||||
|
|
@ -180,11 +180,11 @@ class LogCleanerTest extends JUnitSuite {
|
||||||
|
|
||||||
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
|
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
|
||||||
|
|
||||||
log.append(message(0,0)) // offset 0
|
log.append(record(0,0)) // offset 0
|
||||||
log.append(message(1,1)) // offset 1
|
log.append(record(1,1)) // offset 1
|
||||||
log.append(message(0,0)) // offset 2
|
log.append(record(0,0)) // offset 2
|
||||||
log.append(message(1,1)) // offset 3
|
log.append(record(1,1)) // offset 3
|
||||||
log.append(message(0,0)) // offset 4
|
log.append(record(0,0)) // offset 4
|
||||||
// roll the segment, so we can clean the messages already appended
|
// roll the segment, so we can clean the messages already appended
|
||||||
log.roll()
|
log.roll()
|
||||||
|
|
||||||
|
|
@ -218,18 +218,18 @@ class LogCleanerTest extends JUnitSuite {
|
||||||
|
|
||||||
// append messages with the keys 0 through N-1, values equal offset
|
// append messages with the keys 0 through N-1, values equal offset
|
||||||
while(log.numberOfSegments <= numCleanableSegments)
|
while(log.numberOfSegments <= numCleanableSegments)
|
||||||
log.append(message(log.logEndOffset.toInt % N, log.logEndOffset.toInt))
|
log.append(record(log.logEndOffset.toInt % N, log.logEndOffset.toInt))
|
||||||
|
|
||||||
// at this point one message past the cleanable segments has been added
|
// at this point one message past the cleanable segments has been added
|
||||||
// the entire segment containing the first uncleanable offset should not be cleaned.
|
// the entire segment containing the first uncleanable offset should not be cleaned.
|
||||||
val firstUncleanableOffset = log.logEndOffset + 1 // +1 so it is past the baseOffset
|
val firstUncleanableOffset = log.logEndOffset + 1 // +1 so it is past the baseOffset
|
||||||
|
|
||||||
while(log.numberOfSegments < numTotalSegments - 1)
|
while(log.numberOfSegments < numTotalSegments - 1)
|
||||||
log.append(message(log.logEndOffset.toInt % N, log.logEndOffset.toInt))
|
log.append(record(log.logEndOffset.toInt % N, log.logEndOffset.toInt))
|
||||||
|
|
||||||
// the last (active) segment has just one message
|
// the last (active) segment has just one message
|
||||||
|
|
||||||
def distinctValuesBySegment = log.logSegments.map(s => s.log.map(m => TestUtils.readString(m.message.payload)).toSet.size).toSeq
|
def distinctValuesBySegment = log.logSegments.map(s => s.log.shallowIterator.asScala.map(m => TestUtils.readString(m.record.value)).toSet.size).toSeq
|
||||||
|
|
||||||
val disctinctValuesBySegmentBeforeClean = distinctValuesBySegment
|
val disctinctValuesBySegmentBeforeClean = distinctValuesBySegment
|
||||||
assertTrue("Test is not effective unless each segment contains duplicates. Increase segment size or decrease number of keys.",
|
assertTrue("Test is not effective unless each segment contains duplicates. Increase segment size or decrease number of keys.",
|
||||||
|
|
@ -253,7 +253,7 @@ class LogCleanerTest extends JUnitSuite {
|
||||||
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
|
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
|
||||||
|
|
||||||
// create 6 segments with only one message in each segment
|
// create 6 segments with only one message in each segment
|
||||||
val messageSet = TestUtils.singleMessageSet(payload = Array.fill[Byte](50)(0), key = 1.toString.getBytes)
|
val messageSet = TestUtils.singletonRecords(value = Array.fill[Byte](50)(0), key = 1.toString.getBytes)
|
||||||
for (_ <- 0 until 6)
|
for (_ <- 0 until 6)
|
||||||
log.append(messageSet, assignOffsets = true)
|
log.append(messageSet, assignOffsets = true)
|
||||||
|
|
||||||
|
|
@ -271,7 +271,7 @@ class LogCleanerTest extends JUnitSuite {
|
||||||
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
|
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
|
||||||
|
|
||||||
// create 6 segments with only one message in each segment
|
// create 6 segments with only one message in each segment
|
||||||
val messageSet = TestUtils.singleMessageSet(payload = Array.fill[Byte](50)(0), key = 1.toString.getBytes)
|
val messageSet = TestUtils.singletonRecords(value = Array.fill[Byte](50)(0), key = 1.toString.getBytes)
|
||||||
for (_ <- 0 until 6)
|
for (_ <- 0 until 6)
|
||||||
log.append(messageSet, assignOffsets = true)
|
log.append(messageSet, assignOffsets = true)
|
||||||
|
|
||||||
|
|
@ -305,14 +305,14 @@ class LogCleanerTest extends JUnitSuite {
|
||||||
|
|
||||||
// append unkeyed messages
|
// append unkeyed messages
|
||||||
while(log.numberOfSegments < 2)
|
while(log.numberOfSegments < 2)
|
||||||
log.append(unkeyedMessage(log.logEndOffset.toInt))
|
log.append(unkeyedRecord(log.logEndOffset.toInt))
|
||||||
val numInvalidMessages = unkeyedMessageCountInLog(log)
|
val numInvalidMessages = unkeyedMessageCountInLog(log)
|
||||||
|
|
||||||
val sizeWithUnkeyedMessages = log.size
|
val sizeWithUnkeyedMessages = log.size
|
||||||
|
|
||||||
// append keyed messages
|
// append keyed messages
|
||||||
while(log.numberOfSegments < 3)
|
while(log.numberOfSegments < 3)
|
||||||
log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt))
|
log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
|
||||||
|
|
||||||
val expectedSizeAfterCleaning = log.size - sizeWithUnkeyedMessages
|
val expectedSizeAfterCleaning = log.size - sizeWithUnkeyedMessages
|
||||||
val (_, stats) = cleaner.clean(LogToClean(TopicAndPartition("test", 0), log, 0, log.activeSegment.baseOffset))
|
val (_, stats) = cleaner.clean(LogToClean(TopicAndPartition("test", 0), log, 0, log.activeSegment.baseOffset))
|
||||||
|
|
@ -321,17 +321,17 @@ class LogCleanerTest extends JUnitSuite {
|
||||||
assertEquals("Log should only contain keyed messages after cleaning.", expectedSizeAfterCleaning, log.size)
|
assertEquals("Log should only contain keyed messages after cleaning.", expectedSizeAfterCleaning, log.size)
|
||||||
assertEquals("Cleaner should have seen %d invalid messages.", numInvalidMessages, stats.invalidMessagesRead)
|
assertEquals("Cleaner should have seen %d invalid messages.", numInvalidMessages, stats.invalidMessagesRead)
|
||||||
}
|
}
|
||||||
|
|
||||||
/* extract all the keys from a log */
|
/* extract all the keys from a log */
|
||||||
def keysInLog(log: Log): Iterable[Int] =
|
def keysInLog(log: Log): Iterable[Int] =
|
||||||
log.logSegments.flatMap(s => s.log.filter(!_.message.isNull).filter(_.message.hasKey).map(m => TestUtils.readString(m.message.key).toInt))
|
log.logSegments.flatMap(s => s.log.shallowIterator.asScala.filter(!_.record.hasNullValue).filter(_.record.hasKey).map(m => TestUtils.readString(m.record.key).toInt))
|
||||||
|
|
||||||
/* extract all the offsets from a log */
|
/* extract all the offsets from a log */
|
||||||
def offsetsInLog(log: Log): Iterable[Long] =
|
def offsetsInLog(log: Log): Iterable[Long] =
|
||||||
log.logSegments.flatMap(s => s.log.filter(!_.message.isNull).filter(_.message.hasKey).map(m => m.offset))
|
log.logSegments.flatMap(s => s.log.shallowIterator.asScala.filter(!_.record.hasNullValue).filter(_.record.hasKey).map(m => m.offset))
|
||||||
|
|
||||||
def unkeyedMessageCountInLog(log: Log) =
|
def unkeyedMessageCountInLog(log: Log) =
|
||||||
log.logSegments.map(s => s.log.filter(!_.message.isNull).count(m => !m.message.hasKey)).sum
|
log.logSegments.map(s => s.log.shallowIterator.asScala.filter(!_.record.hasNullValue).count(m => !m.record.hasKey)).sum
|
||||||
|
|
||||||
def abortCheckDone(topicAndPartition: TopicAndPartition): Unit = {
|
def abortCheckDone(topicAndPartition: TopicAndPartition): Unit = {
|
||||||
throw new LogCleaningAbortedException()
|
throw new LogCleaningAbortedException()
|
||||||
|
|
@ -350,7 +350,7 @@ class LogCleanerTest extends JUnitSuite {
|
||||||
|
|
||||||
// append messages to the log until we have four segments
|
// append messages to the log until we have four segments
|
||||||
while(log.numberOfSegments < 4)
|
while(log.numberOfSegments < 4)
|
||||||
log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt))
|
log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
|
||||||
|
|
||||||
val keys = keysInLog(log)
|
val keys = keysInLog(log)
|
||||||
val map = new FakeOffsetMap(Int.MaxValue)
|
val map = new FakeOffsetMap(Int.MaxValue)
|
||||||
|
|
@ -371,20 +371,20 @@ class LogCleanerTest extends JUnitSuite {
|
||||||
logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer)
|
logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer)
|
||||||
|
|
||||||
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
|
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
|
||||||
|
|
||||||
// append some messages to the log
|
// append some messages to the log
|
||||||
var i = 0
|
var i = 0
|
||||||
while(log.numberOfSegments < 10) {
|
while(log.numberOfSegments < 10) {
|
||||||
log.append(TestUtils.singleMessageSet(payload = "hello".getBytes, key = "hello".getBytes))
|
log.append(TestUtils.singletonRecords(value = "hello".getBytes, key = "hello".getBytes))
|
||||||
i += 1
|
i += 1
|
||||||
}
|
}
|
||||||
|
|
||||||
// grouping by very large values should result in a single group with all the segments in it
|
// grouping by very large values should result in a single group with all the segments in it
|
||||||
var groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue)
|
var groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue)
|
||||||
assertEquals(1, groups.size)
|
assertEquals(1, groups.size)
|
||||||
assertEquals(log.numberOfSegments, groups.head.size)
|
assertEquals(log.numberOfSegments, groups.head.size)
|
||||||
checkSegmentOrder(groups)
|
checkSegmentOrder(groups)
|
||||||
|
|
||||||
// grouping by very small values should result in all groups having one entry
|
// grouping by very small values should result in all groups having one entry
|
||||||
groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = 1, maxIndexSize = Int.MaxValue)
|
groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = 1, maxIndexSize = Int.MaxValue)
|
||||||
assertEquals(log.numberOfSegments, groups.size)
|
assertEquals(log.numberOfSegments, groups.size)
|
||||||
|
|
@ -396,20 +396,20 @@ class LogCleanerTest extends JUnitSuite {
|
||||||
checkSegmentOrder(groups)
|
checkSegmentOrder(groups)
|
||||||
|
|
||||||
val groupSize = 3
|
val groupSize = 3
|
||||||
|
|
||||||
// check grouping by log size
|
// check grouping by log size
|
||||||
val logSize = log.logSegments.take(groupSize).map(_.size).sum.toInt + 1
|
val logSize = log.logSegments.take(groupSize).map(_.size).sum.toInt + 1
|
||||||
groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = logSize, maxIndexSize = Int.MaxValue)
|
groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = logSize, maxIndexSize = Int.MaxValue)
|
||||||
checkSegmentOrder(groups)
|
checkSegmentOrder(groups)
|
||||||
assertTrue("All but the last group should be the target size.", groups.dropRight(1).forall(_.size == groupSize))
|
assertTrue("All but the last group should be the target size.", groups.dropRight(1).forall(_.size == groupSize))
|
||||||
|
|
||||||
// check grouping by index size
|
// check grouping by index size
|
||||||
val indexSize = log.logSegments.take(groupSize).map(_.index.sizeInBytes).sum + 1
|
val indexSize = log.logSegments.take(groupSize).map(_.index.sizeInBytes).sum + 1
|
||||||
groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = indexSize)
|
groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = indexSize)
|
||||||
checkSegmentOrder(groups)
|
checkSegmentOrder(groups)
|
||||||
assertTrue("All but the last group should be the target size.", groups.dropRight(1).forall(_.size == groupSize))
|
assertTrue("All but the last group should be the target size.", groups.dropRight(1).forall(_.size == groupSize))
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Validate the logic for grouping log segments together for cleaning when only a small number of
|
* Validate the logic for grouping log segments together for cleaning when only a small number of
|
||||||
* messages are retained, but the range of offsets is greater than Int.MaxValue. A group should not
|
* messages are retained, but the range of offsets is greater than Int.MaxValue. A group should not
|
||||||
|
|
@ -425,47 +425,45 @@ class LogCleanerTest extends JUnitSuite {
|
||||||
logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer)
|
logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer)
|
||||||
|
|
||||||
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
|
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
|
||||||
|
|
||||||
// fill up first segment
|
// fill up first segment
|
||||||
while (log.numberOfSegments == 1)
|
while (log.numberOfSegments == 1)
|
||||||
log.append(TestUtils.singleMessageSet(payload = "hello".getBytes, key = "hello".getBytes))
|
log.append(TestUtils.singletonRecords(value = "hello".getBytes, key = "hello".getBytes))
|
||||||
|
|
||||||
// forward offset and append message to next segment at offset Int.MaxValue
|
// forward offset and append message to next segment at offset Int.MaxValue
|
||||||
val messageSet = new ByteBufferMessageSet(NoCompressionCodec, new LongRef(Int.MaxValue - 1),
|
val records = MemoryRecords.withLogEntries(LogEntry.create(Int.MaxValue - 1, Record.create("hello".getBytes, "hello".getBytes)))
|
||||||
new Message("hello".getBytes, "hello".getBytes, Message.NoTimestamp, Message.MagicValue_V1))
|
log.append(records, assignOffsets = false)
|
||||||
log.append(messageSet, assignOffsets = false)
|
log.append(TestUtils.singletonRecords(value = "hello".getBytes, key = "hello".getBytes))
|
||||||
log.append(TestUtils.singleMessageSet(payload = "hello".getBytes, key = "hello".getBytes))
|
|
||||||
assertEquals(Int.MaxValue, log.activeSegment.index.lastOffset)
|
assertEquals(Int.MaxValue, log.activeSegment.index.lastOffset)
|
||||||
|
|
||||||
// grouping should result in a single group with maximum relative offset of Int.MaxValue
|
// grouping should result in a single group with maximum relative offset of Int.MaxValue
|
||||||
var groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue)
|
var groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue)
|
||||||
assertEquals(1, groups.size)
|
assertEquals(1, groups.size)
|
||||||
|
|
||||||
// append another message, making last offset of second segment > Int.MaxValue
|
// append another message, making last offset of second segment > Int.MaxValue
|
||||||
log.append(TestUtils.singleMessageSet(payload = "hello".getBytes, key = "hello".getBytes))
|
log.append(TestUtils.singletonRecords(value = "hello".getBytes, key = "hello".getBytes))
|
||||||
|
|
||||||
// grouping should not group the two segments to ensure that maximum relative offset in each group <= Int.MaxValue
|
// grouping should not group the two segments to ensure that maximum relative offset in each group <= Int.MaxValue
|
||||||
groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue)
|
groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue)
|
||||||
assertEquals(2, groups.size)
|
assertEquals(2, groups.size)
|
||||||
checkSegmentOrder(groups)
|
checkSegmentOrder(groups)
|
||||||
|
|
||||||
// append more messages, creating new segments, further grouping should still occur
|
// append more messages, creating new segments, further grouping should still occur
|
||||||
while (log.numberOfSegments < 4)
|
while (log.numberOfSegments < 4)
|
||||||
log.append(TestUtils.singleMessageSet(payload = "hello".getBytes, key = "hello".getBytes))
|
log.append(TestUtils.singletonRecords(value = "hello".getBytes, key = "hello".getBytes))
|
||||||
|
|
||||||
groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue)
|
groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue)
|
||||||
assertEquals(log.numberOfSegments - 1, groups.size)
|
assertEquals(log.numberOfSegments - 1, groups.size)
|
||||||
for (group <- groups)
|
for (group <- groups)
|
||||||
assertTrue("Relative offset greater than Int.MaxValue", group.last.index.lastOffset - group.head.index.baseOffset <= Int.MaxValue)
|
assertTrue("Relative offset greater than Int.MaxValue", group.last.index.lastOffset - group.head.index.baseOffset <= Int.MaxValue)
|
||||||
checkSegmentOrder(groups)
|
checkSegmentOrder(groups)
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private def checkSegmentOrder(groups: Seq[Seq[LogSegment]]): Unit = {
|
private def checkSegmentOrder(groups: Seq[Seq[LogSegment]]): Unit = {
|
||||||
val offsets = groups.flatMap(_.map(_.baseOffset))
|
val offsets = groups.flatMap(_.map(_.baseOffset))
|
||||||
assertEquals("Offsets should be in increasing order.", offsets.sorted, offsets)
|
assertEquals("Offsets should be in increasing order.", offsets.sorted, offsets)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test building an offset map off the log
|
* Test building an offset map off the log
|
||||||
*/
|
*/
|
||||||
|
|
@ -496,8 +494,7 @@ class LogCleanerTest extends JUnitSuite {
|
||||||
checkRange(map, segments(1).baseOffset.toInt, segments(3).baseOffset.toInt)
|
checkRange(map, segments(1).baseOffset.toInt, segments(3).baseOffset.toInt)
|
||||||
checkRange(map, segments(3).baseOffset.toInt, log.logEndOffset.toInt)
|
checkRange(map, segments(3).baseOffset.toInt, log.logEndOffset.toInt)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Tests recovery if broker crashes at the following stages during the cleaning sequence
|
* Tests recovery if broker crashes at the following stages during the cleaning sequence
|
||||||
* <ol>
|
* <ol>
|
||||||
|
|
@ -516,8 +513,8 @@ class LogCleanerTest extends JUnitSuite {
|
||||||
logProps.put(LogConfig.FileDeleteDelayMsProp, 10: java.lang.Integer)
|
logProps.put(LogConfig.FileDeleteDelayMsProp, 10: java.lang.Integer)
|
||||||
|
|
||||||
val config = LogConfig.fromProps(logConfig.originals, logProps)
|
val config = LogConfig.fromProps(logConfig.originals, logProps)
|
||||||
|
|
||||||
def recoverAndCheck(config: LogConfig, expectedKeys : Iterable[Int]) : Log = {
|
def recoverAndCheck(config: LogConfig, expectedKeys : Iterable[Int]) : Log = {
|
||||||
// Recover log file and check that after recovery, keys are as expected
|
// Recover log file and check that after recovery, keys are as expected
|
||||||
// and all temporary files have been deleted
|
// and all temporary files have been deleted
|
||||||
val recoveredLog = makeLog(config = config)
|
val recoveredLog = makeLog(config = config)
|
||||||
|
|
@ -530,25 +527,25 @@ class LogCleanerTest extends JUnitSuite {
|
||||||
assertEquals(expectedKeys, keysInLog(recoveredLog))
|
assertEquals(expectedKeys, keysInLog(recoveredLog))
|
||||||
recoveredLog
|
recoveredLog
|
||||||
}
|
}
|
||||||
|
|
||||||
// create a log and append some messages
|
// create a log and append some messages
|
||||||
var log = makeLog(config = config)
|
var log = makeLog(config = config)
|
||||||
var messageCount = 0
|
var messageCount = 0
|
||||||
while(log.numberOfSegments < 10) {
|
while(log.numberOfSegments < 10) {
|
||||||
log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt))
|
log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
|
||||||
messageCount += 1
|
messageCount += 1
|
||||||
}
|
}
|
||||||
val allKeys = keysInLog(log)
|
val allKeys = keysInLog(log)
|
||||||
|
|
||||||
// pretend we have odd-numbered keys
|
// pretend we have odd-numbered keys
|
||||||
val offsetMap = new FakeOffsetMap(Int.MaxValue)
|
val offsetMap = new FakeOffsetMap(Int.MaxValue)
|
||||||
for (k <- 1 until messageCount by 2)
|
for (k <- 1 until messageCount by 2)
|
||||||
offsetMap.put(key(k), Long.MaxValue)
|
offsetMap.put(key(k), Long.MaxValue)
|
||||||
|
|
||||||
// clean the log
|
// clean the log
|
||||||
cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats())
|
cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats())
|
||||||
var cleanedKeys = keysInLog(log)
|
var cleanedKeys = keysInLog(log)
|
||||||
|
|
||||||
// 1) Simulate recovery just after .cleaned file is created, before rename to .swap
|
// 1) Simulate recovery just after .cleaned file is created, before rename to .swap
|
||||||
// On recovery, clean operation is aborted. All messages should be present in the log
|
// On recovery, clean operation is aborted. All messages should be present in the log
|
||||||
log.logSegments.head.changeFileSuffixes("", Log.CleanedFileSuffix)
|
log.logSegments.head.changeFileSuffixes("", Log.CleanedFileSuffix)
|
||||||
|
|
@ -556,44 +553,44 @@ class LogCleanerTest extends JUnitSuite {
|
||||||
Utils.atomicMoveWithFallback(file.toPath, Paths.get(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, "")))
|
Utils.atomicMoveWithFallback(file.toPath, Paths.get(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, "")))
|
||||||
}
|
}
|
||||||
log = recoverAndCheck(config, allKeys)
|
log = recoverAndCheck(config, allKeys)
|
||||||
|
|
||||||
// clean again
|
// clean again
|
||||||
cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats())
|
cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats())
|
||||||
cleanedKeys = keysInLog(log)
|
cleanedKeys = keysInLog(log)
|
||||||
|
|
||||||
// 2) Simulate recovery just after swap file is created, before old segment files are
|
// 2) Simulate recovery just after swap file is created, before old segment files are
|
||||||
// renamed to .deleted. Clean operation is resumed during recovery.
|
// renamed to .deleted. Clean operation is resumed during recovery.
|
||||||
log.logSegments.head.changeFileSuffixes("", Log.SwapFileSuffix)
|
log.logSegments.head.changeFileSuffixes("", Log.SwapFileSuffix)
|
||||||
for (file <- dir.listFiles if file.getName.endsWith(Log.DeletedFileSuffix)) {
|
for (file <- dir.listFiles if file.getName.endsWith(Log.DeletedFileSuffix)) {
|
||||||
Utils.atomicMoveWithFallback(file.toPath, Paths.get(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, "")))
|
Utils.atomicMoveWithFallback(file.toPath, Paths.get(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, "")))
|
||||||
}
|
}
|
||||||
log = recoverAndCheck(config, cleanedKeys)
|
log = recoverAndCheck(config, cleanedKeys)
|
||||||
|
|
||||||
// add some more messages and clean the log again
|
// add some more messages and clean the log again
|
||||||
while(log.numberOfSegments < 10) {
|
while(log.numberOfSegments < 10) {
|
||||||
log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt))
|
log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
|
||||||
messageCount += 1
|
messageCount += 1
|
||||||
}
|
}
|
||||||
for (k <- 1 until messageCount by 2)
|
for (k <- 1 until messageCount by 2)
|
||||||
offsetMap.put(key(k), Long.MaxValue)
|
offsetMap.put(key(k), Long.MaxValue)
|
||||||
cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats())
|
cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats())
|
||||||
cleanedKeys = keysInLog(log)
|
cleanedKeys = keysInLog(log)
|
||||||
|
|
||||||
// 3) Simulate recovery after swap file is created and old segments files are renamed
|
// 3) Simulate recovery after swap file is created and old segments files are renamed
|
||||||
// to .deleted. Clean operation is resumed during recovery.
|
// to .deleted. Clean operation is resumed during recovery.
|
||||||
log.logSegments.head.changeFileSuffixes("", Log.SwapFileSuffix)
|
log.logSegments.head.changeFileSuffixes("", Log.SwapFileSuffix)
|
||||||
log = recoverAndCheck(config, cleanedKeys)
|
log = recoverAndCheck(config, cleanedKeys)
|
||||||
|
|
||||||
// add some more messages and clean the log again
|
// add some more messages and clean the log again
|
||||||
while(log.numberOfSegments < 10) {
|
while(log.numberOfSegments < 10) {
|
||||||
log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt))
|
log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
|
||||||
messageCount += 1
|
messageCount += 1
|
||||||
}
|
}
|
||||||
for (k <- 1 until messageCount by 2)
|
for (k <- 1 until messageCount by 2)
|
||||||
offsetMap.put(key(k), Long.MaxValue)
|
offsetMap.put(key(k), Long.MaxValue)
|
||||||
cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats())
|
cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats())
|
||||||
cleanedKeys = keysInLog(log)
|
cleanedKeys = keysInLog(log)
|
||||||
|
|
||||||
// 4) Simulate recovery after swap is complete, but async deletion
|
// 4) Simulate recovery after swap is complete, but async deletion
|
||||||
// is not yet complete. Clean operation is resumed during recovery.
|
// is not yet complete. Clean operation is resumed during recovery.
|
||||||
recoverAndCheck(config, cleanedKeys)
|
recoverAndCheck(config, cleanedKeys)
|
||||||
|
|
@ -631,11 +628,11 @@ class LogCleanerTest extends JUnitSuite {
|
||||||
val log = makeLog()
|
val log = makeLog()
|
||||||
val cleaner = makeCleaner(2)
|
val cleaner = makeCleaner(2)
|
||||||
|
|
||||||
log.append(message(0,0))
|
log.append(record(0,0))
|
||||||
log.append(message(1,1))
|
log.append(record(1,1))
|
||||||
log.append(message(2,2))
|
log.append(record(2,2))
|
||||||
log.append(message(3,3))
|
log.append(record(3,3))
|
||||||
log.append(message(4,4))
|
log.append(record(4,4))
|
||||||
log.roll()
|
log.roll()
|
||||||
|
|
||||||
val stats = new CleanerStats()
|
val stats = new CleanerStats()
|
||||||
|
|
@ -653,7 +650,7 @@ class LogCleanerTest extends JUnitSuite {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
def testCleanCorruptMessageSet() {
|
def testCleanCorruptMessageSet() {
|
||||||
val codec = SnappyCompressionCodec
|
val codec = CompressionType.GZIP
|
||||||
|
|
||||||
val logProps = new Properties()
|
val logProps = new Properties()
|
||||||
logProps.put(LogConfig.CompressionTypeProp, codec.name)
|
logProps.put(LogConfig.CompressionTypeProp, codec.name)
|
||||||
|
|
@ -682,10 +679,10 @@ class LogCleanerTest extends JUnitSuite {
|
||||||
|
|
||||||
cleaner.clean(LogToClean(TopicAndPartition("test", 0), log, 0, log.activeSegment.baseOffset))
|
cleaner.clean(LogToClean(TopicAndPartition("test", 0), log, 0, log.activeSegment.baseOffset))
|
||||||
|
|
||||||
for (segment <- log.logSegments; shallowMessage <- segment.log.iterator; deepMessage <- ByteBufferMessageSet.deepIterator(shallowMessage)) {
|
for (segment <- log.logSegments; shallowLogEntry <- segment.log.shallowIterator.asScala; deepLogEntry <- shallowLogEntry.asScala) {
|
||||||
assertEquals(shallowMessage.message.magic, deepMessage.message.magic)
|
assertEquals(shallowLogEntry.record.magic, deepLogEntry.record.magic)
|
||||||
val value = TestUtils.readString(deepMessage.message.payload).toLong
|
val value = TestUtils.readString(deepLogEntry.record.value).toLong
|
||||||
assertEquals(deepMessage.offset, value)
|
assertEquals(deepLogEntry.offset, value)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -704,7 +701,7 @@ class LogCleanerTest extends JUnitSuite {
|
||||||
val corruptedMessage = invalidCleanedMessage(offset, set)
|
val corruptedMessage = invalidCleanedMessage(offset, set)
|
||||||
val records = MemoryRecords.readableRecords(corruptedMessage.buffer)
|
val records = MemoryRecords.readableRecords(corruptedMessage.buffer)
|
||||||
|
|
||||||
for (logEntry <- records.iterator.asScala) {
|
for (logEntry <- records.deepIterator.asScala) {
|
||||||
val offset = logEntry.offset
|
val offset = logEntry.offset
|
||||||
val value = TestUtils.readString(logEntry.record.value).toLong
|
val value = TestUtils.readString(logEntry.record.value).toLong
|
||||||
assertEquals(offset, value)
|
assertEquals(offset, value)
|
||||||
|
|
@ -718,94 +715,64 @@ class LogCleanerTest extends JUnitSuite {
|
||||||
|
|
||||||
private def invalidCleanedMessage(initialOffset: Long,
|
private def invalidCleanedMessage(initialOffset: Long,
|
||||||
keysAndValues: Iterable[(Int, Int)],
|
keysAndValues: Iterable[(Int, Int)],
|
||||||
codec: CompressionCodec = SnappyCompressionCodec): ByteBufferMessageSet = {
|
codec: CompressionType = CompressionType.GZIP): MemoryRecords = {
|
||||||
// this function replicates the old versions of the cleaner which under some circumstances
|
// this function replicates the old versions of the cleaner which under some circumstances
|
||||||
// would write invalid compressed message sets with the outer magic set to 1 and the inner
|
// would write invalid compressed message sets with the outer magic set to 1 and the inner
|
||||||
// magic set to 0
|
// magic set to 0
|
||||||
|
val records = keysAndValues.map(kv =>
|
||||||
|
Record.create(Record.MAGIC_VALUE_V0,
|
||||||
|
Record.NO_TIMESTAMP,
|
||||||
|
kv._1.toString.getBytes,
|
||||||
|
kv._2.toString.getBytes))
|
||||||
|
|
||||||
val messages = keysAndValues.map(kv =>
|
val buffer = ByteBuffer.allocate(math.min(math.max(records.map(_.sizeInBytes()).sum / 2, 1024), 1 << 16))
|
||||||
new Message(key = kv._1.toString.getBytes,
|
val builder = MemoryRecords.builder(buffer, Record.MAGIC_VALUE_V1, codec, TimestampType.CREATE_TIME)
|
||||||
bytes = kv._2.toString.getBytes,
|
|
||||||
timestamp = Message.NoTimestamp,
|
|
||||||
magicValue = Message.MagicValue_V0))
|
|
||||||
|
|
||||||
val messageWriter = new MessageWriter(math.min(math.max(MessageSet.messageSetSize(messages) / 2, 1024), 1 << 16))
|
var offset = initialOffset
|
||||||
var lastOffset = initialOffset
|
records.foreach { record =>
|
||||||
|
builder.appendUnchecked(offset, record)
|
||||||
messageWriter.write(
|
offset += 1
|
||||||
codec = codec,
|
|
||||||
timestamp = Message.NoTimestamp,
|
|
||||||
timestampType = TimestampType.CREATE_TIME,
|
|
||||||
magicValue = Message.MagicValue_V1) { outputStream =>
|
|
||||||
|
|
||||||
val output = new DataOutputStream(CompressionFactory(codec, Message.MagicValue_V1, outputStream))
|
|
||||||
try {
|
|
||||||
for (message <- messages) {
|
|
||||||
val innerOffset = lastOffset - initialOffset
|
|
||||||
output.writeLong(innerOffset)
|
|
||||||
output.writeInt(message.size)
|
|
||||||
output.write(message.buffer.array, message.buffer.arrayOffset, message.buffer.limit)
|
|
||||||
lastOffset += 1
|
|
||||||
}
|
|
||||||
} finally {
|
|
||||||
output.close()
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
val buffer = ByteBuffer.allocate(messageWriter.size + MessageSet.LogOverhead)
|
|
||||||
ByteBufferMessageSet.writeMessage(buffer, messageWriter, lastOffset - 1)
|
|
||||||
buffer.rewind()
|
|
||||||
|
|
||||||
new ByteBufferMessageSet(buffer)
|
builder.build()
|
||||||
}
|
}
|
||||||
|
|
||||||
private def messageWithOffset(key: Int, value: Int, offset: Long) =
|
private def messageWithOffset(key: Int, value: Int, offset: Long) =
|
||||||
new ByteBufferMessageSet(NoCompressionCodec, Seq(offset),
|
MemoryRecords.withLogEntries(LogEntry.create(offset, Record.create(key.toString.getBytes, value.toString.getBytes)))
|
||||||
new Message(key = key.toString.getBytes,
|
|
||||||
bytes = value.toString.getBytes,
|
|
||||||
timestamp = Message.NoTimestamp,
|
|
||||||
magicValue = Message.MagicValue_V1))
|
|
||||||
|
|
||||||
|
|
||||||
def makeLog(dir: File = dir, config: LogConfig = logConfig) =
|
def makeLog(dir: File = dir, config: LogConfig = logConfig) =
|
||||||
new Log(dir = dir, config = config, recoveryPoint = 0L, scheduler = time.scheduler, time = time)
|
new Log(dir = dir, config = config, recoveryPoint = 0L, scheduler = time.scheduler, time = time)
|
||||||
|
|
||||||
def noOpCheckDone(topicAndPartition: TopicAndPartition) { /* do nothing */ }
|
def noOpCheckDone(topicAndPartition: TopicAndPartition) { /* do nothing */ }
|
||||||
|
|
||||||
def makeCleaner(capacity: Int, checkDone: (TopicAndPartition) => Unit = noOpCheckDone, maxMessageSize: Int = 64*1024) =
|
def makeCleaner(capacity: Int, checkDone: (TopicAndPartition) => Unit = noOpCheckDone, maxMessageSize: Int = 64*1024) =
|
||||||
new Cleaner(id = 0,
|
new Cleaner(id = 0,
|
||||||
offsetMap = new FakeOffsetMap(capacity),
|
offsetMap = new FakeOffsetMap(capacity),
|
||||||
ioBufferSize = maxMessageSize,
|
ioBufferSize = maxMessageSize,
|
||||||
maxIoBufferSize = maxMessageSize,
|
maxIoBufferSize = maxMessageSize,
|
||||||
dupBufferLoadFactor = 0.75,
|
dupBufferLoadFactor = 0.75,
|
||||||
throttler = throttler,
|
throttler = throttler,
|
||||||
time = time,
|
time = time,
|
||||||
checkDone = checkDone )
|
checkDone = checkDone )
|
||||||
|
|
||||||
def writeToLog(log: Log, seq: Iterable[(Int, Int)]): Iterable[Long] = {
|
def writeToLog(log: Log, seq: Iterable[(Int, Int)]): Iterable[Long] = {
|
||||||
for((key, value) <- seq)
|
for((key, value) <- seq)
|
||||||
yield log.append(message(key, value)).firstOffset
|
yield log.append(record(key, value)).firstOffset
|
||||||
}
|
}
|
||||||
|
|
||||||
def key(id: Int) = ByteBuffer.wrap(id.toString.getBytes)
|
def key(id: Int) = ByteBuffer.wrap(id.toString.getBytes)
|
||||||
|
|
||||||
def message(key: Int, value: Int): ByteBufferMessageSet =
|
|
||||||
message(key, value.toString.getBytes)
|
|
||||||
|
|
||||||
def message(key: Int, value: Array[Byte]) =
|
def record(key: Int, value: Int): MemoryRecords =
|
||||||
new ByteBufferMessageSet(new Message(key = key.toString.getBytes,
|
record(key, value.toString.getBytes)
|
||||||
bytes = value,
|
|
||||||
timestamp = Message.NoTimestamp,
|
|
||||||
magicValue = Message.MagicValue_V1))
|
|
||||||
|
|
||||||
def unkeyedMessage(value: Int) =
|
def record(key: Int, value: Array[Byte]) =
|
||||||
new ByteBufferMessageSet(new Message(bytes = value.toString.getBytes))
|
MemoryRecords.withRecords(Record.create(key.toString.getBytes, value))
|
||||||
|
|
||||||
|
def unkeyedRecord(value: Int) =
|
||||||
|
MemoryRecords.withRecords(Record.create(value.toString.getBytes))
|
||||||
|
|
||||||
|
def tombstoneRecord(key: Int) = record(key, null)
|
||||||
|
|
||||||
def deleteMessage(key: Int) =
|
|
||||||
new ByteBufferMessageSet(new Message(key = key.toString.getBytes,
|
|
||||||
bytes = null,
|
|
||||||
timestamp = Message.NoTimestamp,
|
|
||||||
magicValue = Message.MagicValue_V1))
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
class FakeOffsetMap(val slots: Int) extends OffsetMap {
|
class FakeOffsetMap(val slots: Int) extends OffsetMap {
|
||||||
|
|
|
||||||
|
|
@ -67,7 +67,7 @@ class LogManagerTest {
|
||||||
val log = logManager.createLog(TopicAndPartition(name, 0), logConfig)
|
val log = logManager.createLog(TopicAndPartition(name, 0), logConfig)
|
||||||
val logFile = new File(logDir, name + "-0")
|
val logFile = new File(logDir, name + "-0")
|
||||||
assertTrue(logFile.exists)
|
assertTrue(logFile.exists)
|
||||||
log.append(TestUtils.singleMessageSet("test".getBytes()))
|
log.append(TestUtils.singletonRecords("test".getBytes()))
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -89,7 +89,7 @@ class LogManagerTest {
|
||||||
val log = logManager.createLog(TopicAndPartition(name, 0), logConfig)
|
val log = logManager.createLog(TopicAndPartition(name, 0), logConfig)
|
||||||
var offset = 0L
|
var offset = 0L
|
||||||
for(_ <- 0 until 200) {
|
for(_ <- 0 until 200) {
|
||||||
val set = TestUtils.singleMessageSet("test".getBytes())
|
val set = TestUtils.singletonRecords("test".getBytes())
|
||||||
val info = log.append(set)
|
val info = log.append(set)
|
||||||
offset = info.lastOffset
|
offset = info.lastOffset
|
||||||
}
|
}
|
||||||
|
|
@ -101,7 +101,7 @@ class LogManagerTest {
|
||||||
assertEquals("Now there should only be only one segment in the index.", 1, log.numberOfSegments)
|
assertEquals("Now there should only be only one segment in the index.", 1, log.numberOfSegments)
|
||||||
time.sleep(log.config.fileDeleteDelayMs + 1)
|
time.sleep(log.config.fileDeleteDelayMs + 1)
|
||||||
assertEquals("Files should have been deleted", log.numberOfSegments * 3, log.dir.list.length)
|
assertEquals("Files should have been deleted", log.numberOfSegments * 3, log.dir.list.length)
|
||||||
assertEquals("Should get empty fetch off new log.", 0, log.read(offset+1, 1024).messageSet.sizeInBytes)
|
assertEquals("Should get empty fetch off new log.", 0, log.read(offset+1, 1024).records.sizeInBytes)
|
||||||
|
|
||||||
try {
|
try {
|
||||||
log.read(0, 1024)
|
log.read(0, 1024)
|
||||||
|
|
@ -110,7 +110,7 @@ class LogManagerTest {
|
||||||
case _: OffsetOutOfRangeException => // This is good.
|
case _: OffsetOutOfRangeException => // This is good.
|
||||||
}
|
}
|
||||||
// log should still be appendable
|
// log should still be appendable
|
||||||
log.append(TestUtils.singleMessageSet("test".getBytes()))
|
log.append(TestUtils.singletonRecords("test".getBytes()))
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -118,7 +118,7 @@ class LogManagerTest {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
def testCleanupSegmentsToMaintainSize() {
|
def testCleanupSegmentsToMaintainSize() {
|
||||||
val setSize = TestUtils.singleMessageSet("test".getBytes()).sizeInBytes
|
val setSize = TestUtils.singletonRecords("test".getBytes()).sizeInBytes
|
||||||
logManager.shutdown()
|
logManager.shutdown()
|
||||||
val logProps = new Properties()
|
val logProps = new Properties()
|
||||||
logProps.put(LogConfig.SegmentBytesProp, 10 * setSize: java.lang.Integer)
|
logProps.put(LogConfig.SegmentBytesProp, 10 * setSize: java.lang.Integer)
|
||||||
|
|
@ -135,7 +135,7 @@ class LogManagerTest {
|
||||||
// add a bunch of messages that should be larger than the retentionSize
|
// add a bunch of messages that should be larger than the retentionSize
|
||||||
val numMessages = 200
|
val numMessages = 200
|
||||||
for (_ <- 0 until numMessages) {
|
for (_ <- 0 until numMessages) {
|
||||||
val set = TestUtils.singleMessageSet("test".getBytes())
|
val set = TestUtils.singletonRecords("test".getBytes())
|
||||||
val info = log.append(set)
|
val info = log.append(set)
|
||||||
offset = info.firstOffset
|
offset = info.firstOffset
|
||||||
}
|
}
|
||||||
|
|
@ -147,7 +147,7 @@ class LogManagerTest {
|
||||||
assertEquals("Now there should be exactly 6 segments", 6, log.numberOfSegments)
|
assertEquals("Now there should be exactly 6 segments", 6, log.numberOfSegments)
|
||||||
time.sleep(log.config.fileDeleteDelayMs + 1)
|
time.sleep(log.config.fileDeleteDelayMs + 1)
|
||||||
assertEquals("Files should have been deleted", log.numberOfSegments * 3, log.dir.list.length)
|
assertEquals("Files should have been deleted", log.numberOfSegments * 3, log.dir.list.length)
|
||||||
assertEquals("Should get empty fetch off new log.", 0, log.read(offset + 1, 1024).messageSet.sizeInBytes)
|
assertEquals("Should get empty fetch off new log.", 0, log.read(offset + 1, 1024).records.sizeInBytes)
|
||||||
try {
|
try {
|
||||||
log.read(0, 1024)
|
log.read(0, 1024)
|
||||||
fail("Should get exception from fetching earlier.")
|
fail("Should get exception from fetching earlier.")
|
||||||
|
|
@ -155,7 +155,7 @@ class LogManagerTest {
|
||||||
case _: OffsetOutOfRangeException => // This is good.
|
case _: OffsetOutOfRangeException => // This is good.
|
||||||
}
|
}
|
||||||
// log should still be appendable
|
// log should still be appendable
|
||||||
log.append(TestUtils.singleMessageSet("test".getBytes()))
|
log.append(TestUtils.singletonRecords("test".getBytes()))
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -169,7 +169,7 @@ class LogManagerTest {
|
||||||
val log = logManager.createLog(TopicAndPartition(name, 0), LogConfig.fromProps(logConfig.originals, logProps))
|
val log = logManager.createLog(TopicAndPartition(name, 0), LogConfig.fromProps(logConfig.originals, logProps))
|
||||||
var offset = 0L
|
var offset = 0L
|
||||||
for (_ <- 0 until 200) {
|
for (_ <- 0 until 200) {
|
||||||
val set = TestUtils.singleMessageSet("test".getBytes(), key="test".getBytes())
|
val set = TestUtils.singletonRecords("test".getBytes(), key="test".getBytes())
|
||||||
val info = log.append(set)
|
val info = log.append(set)
|
||||||
offset = info.lastOffset
|
offset = info.lastOffset
|
||||||
}
|
}
|
||||||
|
|
@ -198,7 +198,7 @@ class LogManagerTest {
|
||||||
val log = logManager.createLog(TopicAndPartition(name, 0), config)
|
val log = logManager.createLog(TopicAndPartition(name, 0), config)
|
||||||
val lastFlush = log.lastFlushTime
|
val lastFlush = log.lastFlushTime
|
||||||
for (_ <- 0 until 200) {
|
for (_ <- 0 until 200) {
|
||||||
val set = TestUtils.singleMessageSet("test".getBytes())
|
val set = TestUtils.singletonRecords("test".getBytes())
|
||||||
log.append(set)
|
log.append(set)
|
||||||
}
|
}
|
||||||
time.sleep(logManager.InitialTaskDelayMs)
|
time.sleep(logManager.InitialTaskDelayMs)
|
||||||
|
|
@ -280,7 +280,7 @@ class LogManagerTest {
|
||||||
val logs = topicAndPartitions.map(this.logManager.createLog(_, logConfig))
|
val logs = topicAndPartitions.map(this.logManager.createLog(_, logConfig))
|
||||||
logs.foreach(log => {
|
logs.foreach(log => {
|
||||||
for (_ <- 0 until 50)
|
for (_ <- 0 until 50)
|
||||||
log.append(TestUtils.singleMessageSet("test".getBytes()))
|
log.append(TestUtils.singletonRecords("test".getBytes()))
|
||||||
|
|
||||||
log.flush()
|
log.flush()
|
||||||
})
|
})
|
||||||
|
|
|
||||||
|
|
@ -16,15 +16,13 @@
|
||||||
*/
|
*/
|
||||||
package kafka.log
|
package kafka.log
|
||||||
|
|
||||||
import org.junit.Assert._
|
|
||||||
import java.util.concurrent.atomic._
|
|
||||||
|
|
||||||
import kafka.common.LongRef
|
|
||||||
import org.junit.{After, Test}
|
|
||||||
import kafka.utils.TestUtils
|
import kafka.utils.TestUtils
|
||||||
import kafka.message._
|
import org.apache.kafka.common.record.{FileRecords, MemoryRecords, Record}
|
||||||
import org.apache.kafka.common.utils.Time
|
import org.apache.kafka.common.utils.Time
|
||||||
|
import org.junit.Assert._
|
||||||
|
import org.junit.{After, Test}
|
||||||
|
|
||||||
|
import scala.collection.JavaConverters._
|
||||||
import scala.collection._
|
import scala.collection._
|
||||||
|
|
||||||
class LogSegmentTest {
|
class LogSegmentTest {
|
||||||
|
|
@ -34,7 +32,7 @@ class LogSegmentTest {
|
||||||
/* create a segment with the given base offset */
|
/* create a segment with the given base offset */
|
||||||
def createSegment(offset: Long, indexIntervalBytes: Int = 10): LogSegment = {
|
def createSegment(offset: Long, indexIntervalBytes: Int = 10): LogSegment = {
|
||||||
val msFile = TestUtils.tempFile()
|
val msFile = TestUtils.tempFile()
|
||||||
val ms = new FileMessageSet(msFile)
|
val ms = FileRecords.open(msFile)
|
||||||
val idxFile = TestUtils.tempFile()
|
val idxFile = TestUtils.tempFile()
|
||||||
val timeIdxFile = TestUtils.tempFile()
|
val timeIdxFile = TestUtils.tempFile()
|
||||||
idxFile.delete()
|
idxFile.delete()
|
||||||
|
|
@ -47,12 +45,10 @@ class LogSegmentTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
/* create a ByteBufferMessageSet for the given messages starting from the given offset */
|
/* create a ByteBufferMessageSet for the given messages starting from the given offset */
|
||||||
def messages(offset: Long, messages: String*): ByteBufferMessageSet = {
|
def records(offset: Long, records: String*): MemoryRecords = {
|
||||||
new ByteBufferMessageSet(compressionCodec = NoCompressionCodec,
|
MemoryRecords.withRecords(offset, records.map(s => Record.create(Record.MAGIC_VALUE_V1, offset * 10, s.getBytes)):_*)
|
||||||
offsetCounter = new LongRef(offset),
|
|
||||||
messages = messages.map(s => new Message(s.getBytes, offset * 10, Message.MagicValue_V1)):_*)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@After
|
@After
|
||||||
def teardown() {
|
def teardown() {
|
||||||
for(seg <- segments) {
|
for(seg <- segments) {
|
||||||
|
|
@ -60,7 +56,7 @@ class LogSegmentTest {
|
||||||
seg.log.delete()
|
seg.log.delete()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A read on an empty log segment should return null
|
* A read on an empty log segment should return null
|
||||||
*/
|
*/
|
||||||
|
|
@ -70,7 +66,7 @@ class LogSegmentTest {
|
||||||
val read = seg.read(startOffset = 40, maxSize = 300, maxOffset = None)
|
val read = seg.read(startOffset = 40, maxSize = 300, maxOffset = None)
|
||||||
assertNull("Read beyond the last offset in the segment should be null", read)
|
assertNull("Read beyond the last offset in the segment should be null", read)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Reading from before the first offset in the segment should return messages
|
* Reading from before the first offset in the segment should return messages
|
||||||
* beginning with the first message in the segment
|
* beginning with the first message in the segment
|
||||||
|
|
@ -78,12 +74,12 @@ class LogSegmentTest {
|
||||||
@Test
|
@Test
|
||||||
def testReadBeforeFirstOffset() {
|
def testReadBeforeFirstOffset() {
|
||||||
val seg = createSegment(40)
|
val seg = createSegment(40)
|
||||||
val ms = messages(50, "hello", "there", "little", "bee")
|
val ms = records(50, "hello", "there", "little", "bee")
|
||||||
seg.append(50, Message.NoTimestamp, -1L, ms)
|
seg.append(50, Record.NO_TIMESTAMP, -1L, ms)
|
||||||
val read = seg.read(startOffset = 41, maxSize = 300, maxOffset = None).messageSet
|
val read = seg.read(startOffset = 41, maxSize = 300, maxOffset = None).records
|
||||||
assertEquals(ms.toList, read.toList)
|
assertEquals(ms.deepIterator.asScala.toList, read.deepIterator.asScala.toList)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* If we set the startOffset and maxOffset for the read to be the same value
|
* If we set the startOffset and maxOffset for the read to be the same value
|
||||||
* we should get only the first message in the log
|
* we should get only the first message in the log
|
||||||
|
|
@ -92,28 +88,28 @@ class LogSegmentTest {
|
||||||
def testMaxOffset() {
|
def testMaxOffset() {
|
||||||
val baseOffset = 50
|
val baseOffset = 50
|
||||||
val seg = createSegment(baseOffset)
|
val seg = createSegment(baseOffset)
|
||||||
val ms = messages(baseOffset, "hello", "there", "beautiful")
|
val ms = records(baseOffset, "hello", "there", "beautiful")
|
||||||
seg.append(baseOffset, Message.NoTimestamp, -1L, ms)
|
seg.append(baseOffset, Record.NO_TIMESTAMP, -1L, ms)
|
||||||
def validate(offset: Long) =
|
def validate(offset: Long) =
|
||||||
assertEquals(ms.filter(_.offset == offset).toList,
|
assertEquals(ms.deepIterator.asScala.filter(_.offset == offset).toList,
|
||||||
seg.read(startOffset = offset, maxSize = 1024, maxOffset = Some(offset+1)).messageSet.toList)
|
seg.read(startOffset = offset, maxSize = 1024, maxOffset = Some(offset+1)).records.deepIterator.asScala.toList)
|
||||||
validate(50)
|
validate(50)
|
||||||
validate(51)
|
validate(51)
|
||||||
validate(52)
|
validate(52)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* If we read from an offset beyond the last offset in the segment we should get null
|
* If we read from an offset beyond the last offset in the segment we should get null
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
def testReadAfterLast() {
|
def testReadAfterLast() {
|
||||||
val seg = createSegment(40)
|
val seg = createSegment(40)
|
||||||
val ms = messages(50, "hello", "there")
|
val ms = records(50, "hello", "there")
|
||||||
seg.append(50, Message.NoTimestamp, -1L, ms)
|
seg.append(50, Record.NO_TIMESTAMP, -1L, ms)
|
||||||
val read = seg.read(startOffset = 52, maxSize = 200, maxOffset = None)
|
val read = seg.read(startOffset = 52, maxSize = 200, maxOffset = None)
|
||||||
assertNull("Read beyond the last offset in the segment should give null", read)
|
assertNull("Read beyond the last offset in the segment should give null", read)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* If we read from an offset which doesn't exist we should get a message set beginning
|
* If we read from an offset which doesn't exist we should get a message set beginning
|
||||||
* with the least offset greater than the given startOffset.
|
* with the least offset greater than the given startOffset.
|
||||||
|
|
@ -121,14 +117,14 @@ class LogSegmentTest {
|
||||||
@Test
|
@Test
|
||||||
def testReadFromGap() {
|
def testReadFromGap() {
|
||||||
val seg = createSegment(40)
|
val seg = createSegment(40)
|
||||||
val ms = messages(50, "hello", "there")
|
val ms = records(50, "hello", "there")
|
||||||
seg.append(50, Message.NoTimestamp, -1L, ms)
|
seg.append(50, Record.NO_TIMESTAMP, -1L, ms)
|
||||||
val ms2 = messages(60, "alpha", "beta")
|
val ms2 = records(60, "alpha", "beta")
|
||||||
seg.append(60, Message.NoTimestamp, -1L, ms2)
|
seg.append(60, Record.NO_TIMESTAMP, -1L, ms2)
|
||||||
val read = seg.read(startOffset = 55, maxSize = 200, maxOffset = None)
|
val read = seg.read(startOffset = 55, maxSize = 200, maxOffset = None)
|
||||||
assertEquals(ms2.toList, read.messageSet.toList)
|
assertEquals(ms2.deepIterator.asScala.toList, read.records.deepIterator.asScala.toList)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* In a loop append two messages then truncate off the second of those messages and check that we can read
|
* In a loop append two messages then truncate off the second of those messages and check that we can read
|
||||||
* the first but not the second message.
|
* the first but not the second message.
|
||||||
|
|
@ -138,18 +134,18 @@ class LogSegmentTest {
|
||||||
val seg = createSegment(40)
|
val seg = createSegment(40)
|
||||||
var offset = 40
|
var offset = 40
|
||||||
for (_ <- 0 until 30) {
|
for (_ <- 0 until 30) {
|
||||||
val ms1 = messages(offset, "hello")
|
val ms1 = records(offset, "hello")
|
||||||
seg.append(offset, Message.NoTimestamp, -1L, ms1)
|
seg.append(offset, Record.NO_TIMESTAMP, -1L, ms1)
|
||||||
val ms2 = messages(offset + 1, "hello")
|
val ms2 = records(offset + 1, "hello")
|
||||||
seg.append(offset + 1, Message.NoTimestamp, -1L, ms2)
|
seg.append(offset + 1, Record.NO_TIMESTAMP, -1L, ms2)
|
||||||
// check that we can read back both messages
|
// check that we can read back both messages
|
||||||
val read = seg.read(offset, None, 10000)
|
val read = seg.read(offset, None, 10000)
|
||||||
assertEquals(List(ms1.head, ms2.head), read.messageSet.toList)
|
assertEquals(List(ms1.deepIterator.next(), ms2.deepIterator.next()), read.records.deepIterator.asScala.toList)
|
||||||
// now truncate off the last message
|
// now truncate off the last message
|
||||||
seg.truncateTo(offset + 1)
|
seg.truncateTo(offset + 1)
|
||||||
val read2 = seg.read(offset, None, 10000)
|
val read2 = seg.read(offset, None, 10000)
|
||||||
assertEquals(1, read2.messageSet.size)
|
assertEquals(1, read2.records.deepIterator.asScala.size)
|
||||||
assertEquals(ms1.head, read2.messageSet.head)
|
assertEquals(ms1.deepIterator.next(), read2.records.deepIterator.next())
|
||||||
offset += 1
|
offset += 1
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -157,10 +153,10 @@ class LogSegmentTest {
|
||||||
@Test
|
@Test
|
||||||
def testReloadLargestTimestampAfterTruncation() {
|
def testReloadLargestTimestampAfterTruncation() {
|
||||||
val numMessages = 30
|
val numMessages = 30
|
||||||
val seg = createSegment(40, 2 * messages(0, "hello").sizeInBytes - 1)
|
val seg = createSegment(40, 2 * records(0, "hello").sizeInBytes - 1)
|
||||||
var offset = 40
|
var offset = 40
|
||||||
for (_ <- 0 until numMessages) {
|
for (_ <- 0 until numMessages) {
|
||||||
seg.append(offset, offset, offset, messages(offset, "hello"))
|
seg.append(offset, offset, offset, records(offset, "hello"))
|
||||||
offset += 1
|
offset += 1
|
||||||
}
|
}
|
||||||
val expectedNumEntries = numMessages / 2 - 1
|
val expectedNumEntries = numMessages / 2 - 1
|
||||||
|
|
@ -179,10 +175,10 @@ class LogSegmentTest {
|
||||||
def testTruncateFull() {
|
def testTruncateFull() {
|
||||||
// test the case where we fully truncate the log
|
// test the case where we fully truncate the log
|
||||||
val seg = createSegment(40)
|
val seg = createSegment(40)
|
||||||
seg.append(40, Message.NoTimestamp, -1L, messages(40, "hello", "there"))
|
seg.append(40, Record.NO_TIMESTAMP, -1L, records(40, "hello", "there"))
|
||||||
seg.truncateTo(0)
|
seg.truncateTo(0)
|
||||||
assertNull("Segment should be empty.", seg.read(0, None, 1024))
|
assertNull("Segment should be empty.", seg.read(0, None, 1024))
|
||||||
seg.append(40, Message.NoTimestamp, -1L, messages(40, "hello", "there"))
|
seg.append(40, Record.NO_TIMESTAMP, -1L, records(40, "hello", "there"))
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -190,11 +186,11 @@ class LogSegmentTest {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
def testFindOffsetByTimestamp() {
|
def testFindOffsetByTimestamp() {
|
||||||
val messageSize = messages(0, s"msg00").sizeInBytes
|
val messageSize = records(0, s"msg00").sizeInBytes
|
||||||
val seg = createSegment(40, messageSize * 2 - 1)
|
val seg = createSegment(40, messageSize * 2 - 1)
|
||||||
// Produce some messages
|
// Produce some messages
|
||||||
for (i <- 40 until 50)
|
for (i <- 40 until 50)
|
||||||
seg.append(i, i * 10, i, messages(i, s"msg$i"))
|
seg.append(i, i * 10, i, records(i, s"msg$i"))
|
||||||
|
|
||||||
assertEquals(490, seg.largestTimestamp)
|
assertEquals(490, seg.largestTimestamp)
|
||||||
// Search for an indexed timestamp
|
// Search for an indexed timestamp
|
||||||
|
|
@ -218,10 +214,10 @@ class LogSegmentTest {
|
||||||
def testNextOffsetCalculation() {
|
def testNextOffsetCalculation() {
|
||||||
val seg = createSegment(40)
|
val seg = createSegment(40)
|
||||||
assertEquals(40, seg.nextOffset)
|
assertEquals(40, seg.nextOffset)
|
||||||
seg.append(50, Message.NoTimestamp, -1L, messages(50, "hello", "there", "you"))
|
seg.append(50, Record.NO_TIMESTAMP, -1L, records(50, "hello", "there", "you"))
|
||||||
assertEquals(53, seg.nextOffset())
|
assertEquals(53, seg.nextOffset())
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Test that we can change the file suffixes for the log and index files
|
* Test that we can change the file suffixes for the log and index files
|
||||||
*/
|
*/
|
||||||
|
|
@ -236,7 +232,7 @@ class LogSegmentTest {
|
||||||
assertTrue(seg.log.file.exists)
|
assertTrue(seg.log.file.exists)
|
||||||
assertTrue(seg.index.file.exists)
|
assertTrue(seg.index.file.exists)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create a segment with some data and an index. Then corrupt the index,
|
* Create a segment with some data and an index. Then corrupt the index,
|
||||||
* and recover the segment, the entries should all be readable.
|
* and recover the segment, the entries should all be readable.
|
||||||
|
|
@ -245,12 +241,12 @@ class LogSegmentTest {
|
||||||
def testRecoveryFixesCorruptIndex() {
|
def testRecoveryFixesCorruptIndex() {
|
||||||
val seg = createSegment(0)
|
val seg = createSegment(0)
|
||||||
for(i <- 0 until 100)
|
for(i <- 0 until 100)
|
||||||
seg.append(i, Message.NoTimestamp, -1L, messages(i, i.toString))
|
seg.append(i, Record.NO_TIMESTAMP, -1L, records(i, i.toString))
|
||||||
val indexFile = seg.index.file
|
val indexFile = seg.index.file
|
||||||
TestUtils.writeNonsenseToFile(indexFile, 5, indexFile.length.toInt)
|
TestUtils.writeNonsenseToFile(indexFile, 5, indexFile.length.toInt)
|
||||||
seg.recover(64*1024)
|
seg.recover(64*1024)
|
||||||
for(i <- 0 until 100)
|
for(i <- 0 until 100)
|
||||||
assertEquals(i, seg.read(i, Some(i + 1), 1024).messageSet.head.offset)
|
assertEquals(i, seg.read(i, Some(i + 1), 1024).records.deepIterator.next().offset)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -261,7 +257,7 @@ class LogSegmentTest {
|
||||||
def testRecoveryFixesCorruptTimeIndex() {
|
def testRecoveryFixesCorruptTimeIndex() {
|
||||||
val seg = createSegment(0)
|
val seg = createSegment(0)
|
||||||
for(i <- 0 until 100)
|
for(i <- 0 until 100)
|
||||||
seg.append(i, i * 10, i, messages(i, i.toString))
|
seg.append(i, i * 10, i, records(i, i.toString))
|
||||||
val timeIndexFile = seg.timeIndex.file
|
val timeIndexFile = seg.timeIndex.file
|
||||||
TestUtils.writeNonsenseToFile(timeIndexFile, 5, timeIndexFile.length.toInt)
|
TestUtils.writeNonsenseToFile(timeIndexFile, 5, timeIndexFile.length.toInt)
|
||||||
seg.recover(64*1024)
|
seg.recover(64*1024)
|
||||||
|
|
@ -271,7 +267,7 @@ class LogSegmentTest {
|
||||||
assertEquals(i + 1, seg.findOffsetByTimestamp(i * 10 + 1).get.offset)
|
assertEquals(i + 1, seg.findOffsetByTimestamp(i * 10 + 1).get.offset)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Randomly corrupt a log a number of times and attempt recovery.
|
* Randomly corrupt a log a number of times and attempt recovery.
|
||||||
*/
|
*/
|
||||||
|
|
@ -281,13 +277,15 @@ class LogSegmentTest {
|
||||||
for (_ <- 0 until 10) {
|
for (_ <- 0 until 10) {
|
||||||
val seg = createSegment(0)
|
val seg = createSegment(0)
|
||||||
for(i <- 0 until messagesAppended)
|
for(i <- 0 until messagesAppended)
|
||||||
seg.append(i, Message.NoTimestamp, -1L, messages(i, i.toString))
|
seg.append(i, Record.NO_TIMESTAMP, -1L, records(i, i.toString))
|
||||||
val offsetToBeginCorruption = TestUtils.random.nextInt(messagesAppended)
|
val offsetToBeginCorruption = TestUtils.random.nextInt(messagesAppended)
|
||||||
// start corrupting somewhere in the middle of the chosen record all the way to the end
|
// start corrupting somewhere in the middle of the chosen record all the way to the end
|
||||||
val position = seg.log.searchForOffsetWithSize(offsetToBeginCorruption, 0)._1.position + TestUtils.random.nextInt(15)
|
|
||||||
TestUtils.writeNonsenseToFile(seg.log.file, position, seg.log.file.length.toInt - position)
|
val recordPosition = seg.log.searchForOffsetWithSize(offsetToBeginCorruption, 0)
|
||||||
|
val position = recordPosition.position + TestUtils.random.nextInt(15)
|
||||||
|
TestUtils.writeNonsenseToFile(seg.log.file, position, (seg.log.file.length - position).toInt)
|
||||||
seg.recover(64*1024)
|
seg.recover(64*1024)
|
||||||
assertEquals("Should have truncated off bad messages.", (0 until offsetToBeginCorruption).toList, seg.log.map(_.offset).toList)
|
assertEquals("Should have truncated off bad messages.", (0 until offsetToBeginCorruption).toList, seg.log.shallowIterator.asScala.map(_.offset).toList)
|
||||||
seg.delete()
|
seg.delete()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -304,12 +302,12 @@ class LogSegmentTest {
|
||||||
@Test
|
@Test
|
||||||
def testCreateWithInitFileSizeAppendMessage() {
|
def testCreateWithInitFileSizeAppendMessage() {
|
||||||
val seg = createSegment(40, false, 512*1024*1024, true)
|
val seg = createSegment(40, false, 512*1024*1024, true)
|
||||||
val ms = messages(50, "hello", "there")
|
val ms = records(50, "hello", "there")
|
||||||
seg.append(50, Message.NoTimestamp, -1L, ms)
|
seg.append(50, Record.NO_TIMESTAMP, -1L, ms)
|
||||||
val ms2 = messages(60, "alpha", "beta")
|
val ms2 = records(60, "alpha", "beta")
|
||||||
seg.append(60, Message.NoTimestamp, -1L, ms2)
|
seg.append(60, Record.NO_TIMESTAMP, -1L, ms2)
|
||||||
val read = seg.read(startOffset = 55, maxSize = 200, maxOffset = None)
|
val read = seg.read(startOffset = 55, maxSize = 200, maxOffset = None)
|
||||||
assertEquals(ms2.toList, read.messageSet.toList)
|
assertEquals(ms2.deepIterator.asScala.toList, read.records.deepIterator.asScala.toList)
|
||||||
}
|
}
|
||||||
|
|
||||||
/* create a segment with pre allocate and clearly shut down*/
|
/* create a segment with pre allocate and clearly shut down*/
|
||||||
|
|
@ -318,12 +316,12 @@ class LogSegmentTest {
|
||||||
val tempDir = TestUtils.tempDir()
|
val tempDir = TestUtils.tempDir()
|
||||||
val seg = new LogSegment(tempDir, 40, 10, 1000, 0, Time.SYSTEM, false, 512*1024*1024, true)
|
val seg = new LogSegment(tempDir, 40, 10, 1000, 0, Time.SYSTEM, false, 512*1024*1024, true)
|
||||||
|
|
||||||
val ms = messages(50, "hello", "there")
|
val ms = records(50, "hello", "there")
|
||||||
seg.append(50, Message.NoTimestamp, -1L, ms)
|
seg.append(50, Record.NO_TIMESTAMP, -1L, ms)
|
||||||
val ms2 = messages(60, "alpha", "beta")
|
val ms2 = records(60, "alpha", "beta")
|
||||||
seg.append(60, Message.NoTimestamp, -1L, ms2)
|
seg.append(60, Record.NO_TIMESTAMP, -1L, ms2)
|
||||||
val read = seg.read(startOffset = 55, maxSize = 200, maxOffset = None)
|
val read = seg.read(startOffset = 55, maxSize = 200, maxOffset = None)
|
||||||
assertEquals(ms2.toList, read.messageSet.toList)
|
assertEquals(ms2.deepIterator.asScala.toList, read.records.deepIterator.asScala.toList)
|
||||||
val oldSize = seg.log.sizeInBytes()
|
val oldSize = seg.log.sizeInBytes()
|
||||||
val oldPosition = seg.log.channel.position
|
val oldPosition = seg.log.channel.position
|
||||||
val oldFileSize = seg.log.file.length
|
val oldFileSize = seg.log.file.length
|
||||||
|
|
@ -336,7 +334,7 @@ class LogSegmentTest {
|
||||||
segments += segReopen
|
segments += segReopen
|
||||||
|
|
||||||
val readAgain = segReopen.read(startOffset = 55, maxSize = 200, maxOffset = None)
|
val readAgain = segReopen.read(startOffset = 55, maxSize = 200, maxOffset = None)
|
||||||
assertEquals(ms2.toList, readAgain.messageSet.toList)
|
assertEquals(ms2.deepIterator.asScala.toList, readAgain.records.deepIterator.asScala.toList)
|
||||||
val size = segReopen.log.sizeInBytes()
|
val size = segReopen.log.sizeInBytes()
|
||||||
val position = segReopen.log.channel.position
|
val position = segReopen.log.channel.position
|
||||||
val fileSize = segReopen.log.file.length
|
val fileSize = segReopen.log.file.length
|
||||||
|
|
|
||||||
|
|
@ -22,15 +22,16 @@ import java.util.Properties
|
||||||
|
|
||||||
import org.apache.kafka.common.errors.{CorruptRecordException, OffsetOutOfRangeException, RecordBatchTooLargeException, RecordTooLargeException}
|
import org.apache.kafka.common.errors.{CorruptRecordException, OffsetOutOfRangeException, RecordBatchTooLargeException, RecordTooLargeException}
|
||||||
import kafka.api.ApiVersion
|
import kafka.api.ApiVersion
|
||||||
import kafka.common.LongRef
|
|
||||||
import org.junit.Assert._
|
import org.junit.Assert._
|
||||||
import org.scalatest.junit.JUnitSuite
|
import org.scalatest.junit.JUnitSuite
|
||||||
import org.junit.{After, Before, Test}
|
import org.junit.{After, Before, Test}
|
||||||
import kafka.message._
|
|
||||||
import kafka.utils._
|
import kafka.utils._
|
||||||
import kafka.server.KafkaConfig
|
import kafka.server.KafkaConfig
|
||||||
|
import org.apache.kafka.common.record._
|
||||||
import org.apache.kafka.common.utils.Utils
|
import org.apache.kafka.common.utils.Utils
|
||||||
|
|
||||||
|
import scala.collection.JavaConverters._
|
||||||
|
|
||||||
class LogTest extends JUnitSuite {
|
class LogTest extends JUnitSuite {
|
||||||
|
|
||||||
val tmpDir = TestUtils.tempDir()
|
val tmpDir = TestUtils.tempDir()
|
||||||
|
|
@ -63,7 +64,7 @@ class LogTest extends JUnitSuite {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
def testTimeBasedLogRoll() {
|
def testTimeBasedLogRoll() {
|
||||||
val set = TestUtils.singleMessageSet("test".getBytes)
|
val set = TestUtils.singletonRecords("test".getBytes)
|
||||||
|
|
||||||
val logProps = new Properties()
|
val logProps = new Properties()
|
||||||
logProps.put(LogConfig.SegmentMsProp, (1 * 60 * 60L): java.lang.Long)
|
logProps.put(LogConfig.SegmentMsProp, (1 * 60 * 60L): java.lang.Long)
|
||||||
|
|
@ -91,7 +92,7 @@ class LogTest extends JUnitSuite {
|
||||||
|
|
||||||
// Append a message with timestamp to a segment whose first messgae do not have a timestamp.
|
// Append a message with timestamp to a segment whose first messgae do not have a timestamp.
|
||||||
val setWithTimestamp =
|
val setWithTimestamp =
|
||||||
TestUtils.singleMessageSet(payload = "test".getBytes, timestamp = time.milliseconds + log.config.segmentMs + 1)
|
TestUtils.singletonRecords(value = "test".getBytes, timestamp = time.milliseconds + log.config.segmentMs + 1)
|
||||||
log.append(setWithTimestamp)
|
log.append(setWithTimestamp)
|
||||||
assertEquals("Segment should not have been rolled out because the log rolling should be based on wall clock.", 4, log.numberOfSegments)
|
assertEquals("Segment should not have been rolled out because the log rolling should be based on wall clock.", 4, log.numberOfSegments)
|
||||||
|
|
||||||
|
|
@ -105,14 +106,14 @@ class LogTest extends JUnitSuite {
|
||||||
log.append(setWithTimestamp)
|
log.append(setWithTimestamp)
|
||||||
assertEquals("Log should not roll because the roll should depend on timestamp of the first message.", 5, log.numberOfSegments)
|
assertEquals("Log should not roll because the roll should depend on timestamp of the first message.", 5, log.numberOfSegments)
|
||||||
|
|
||||||
val setWithExpiredTimestamp = TestUtils.singleMessageSet(payload = "test".getBytes, timestamp = time.milliseconds)
|
val setWithExpiredTimestamp = TestUtils.singletonRecords(value = "test".getBytes, timestamp = time.milliseconds)
|
||||||
log.append(setWithExpiredTimestamp)
|
log.append(setWithExpiredTimestamp)
|
||||||
assertEquals("Log should roll because the timestamp in the message should make the log segment expire.", 6, log.numberOfSegments)
|
assertEquals("Log should roll because the timestamp in the message should make the log segment expire.", 6, log.numberOfSegments)
|
||||||
|
|
||||||
val numSegments = log.numberOfSegments
|
val numSegments = log.numberOfSegments
|
||||||
time.sleep(log.config.segmentMs + 1)
|
time.sleep(log.config.segmentMs + 1)
|
||||||
log.append(new ByteBufferMessageSet())
|
log.append(MemoryRecords.withLogEntries())
|
||||||
assertEquals("Appending an empty message set should not roll log even if succient time has passed.", numSegments, log.numberOfSegments)
|
assertEquals("Appending an empty message set should not roll log even if sufficient time has passed.", numSegments, log.numberOfSegments)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -121,7 +122,7 @@ class LogTest extends JUnitSuite {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
def testTimeBasedLogRollJitter() {
|
def testTimeBasedLogRollJitter() {
|
||||||
val set = TestUtils.singleMessageSet("test".getBytes)
|
val set = TestUtils.singletonRecords("test".getBytes)
|
||||||
val maxJitter = 20 * 60L
|
val maxJitter = 20 * 60L
|
||||||
|
|
||||||
val logProps = new Properties()
|
val logProps = new Properties()
|
||||||
|
|
@ -149,7 +150,7 @@ class LogTest extends JUnitSuite {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
def testSizeBasedLogRoll() {
|
def testSizeBasedLogRoll() {
|
||||||
val set = TestUtils.singleMessageSet("test".getBytes)
|
val set = TestUtils.singletonRecords("test".getBytes)
|
||||||
val setSize = set.sizeInBytes
|
val setSize = set.sizeInBytes
|
||||||
val msgPerSeg = 10
|
val msgPerSeg = 10
|
||||||
val segmentSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages
|
val segmentSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages
|
||||||
|
|
@ -176,7 +177,7 @@ class LogTest extends JUnitSuite {
|
||||||
def testLoadEmptyLog() {
|
def testLoadEmptyLog() {
|
||||||
createEmptyLogs(logDir, 0)
|
createEmptyLogs(logDir, 0)
|
||||||
val log = new Log(logDir, logConfig, recoveryPoint = 0L, time.scheduler, time = time)
|
val log = new Log(logDir, logConfig, recoveryPoint = 0L, time.scheduler, time = time)
|
||||||
log.append(TestUtils.singleMessageSet("test".getBytes))
|
log.append(TestUtils.singletonRecords("test".getBytes))
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -189,16 +190,17 @@ class LogTest extends JUnitSuite {
|
||||||
// We use need to use magic value 1 here because the test is message size sensitive.
|
// We use need to use magic value 1 here because the test is message size sensitive.
|
||||||
logProps.put(LogConfig.MessageFormatVersionProp, ApiVersion.latestVersion.toString)
|
logProps.put(LogConfig.MessageFormatVersionProp, ApiVersion.latestVersion.toString)
|
||||||
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
|
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
|
||||||
val messages = (0 until 100 by 2).map(id => new Message(id.toString.getBytes)).toArray
|
val records = (0 until 100 by 2).map(id => Record.create(id.toString.getBytes)).toArray
|
||||||
|
|
||||||
for(i <- 0 until messages.length)
|
for(i <- records.indices)
|
||||||
log.append(new ByteBufferMessageSet(NoCompressionCodec, messages = messages(i)))
|
log.append(MemoryRecords.withRecords(records(i)))
|
||||||
for(i <- 0 until messages.length) {
|
|
||||||
val read = log.read(i, 100, Some(i+1)).messageSet.head
|
for(i <- records.indices) {
|
||||||
|
val read = log.read(i, 100, Some(i+1)).records.shallowIterator.next()
|
||||||
assertEquals("Offset read should match order appended.", i, read.offset)
|
assertEquals("Offset read should match order appended.", i, read.offset)
|
||||||
assertEquals("Message should match appended.", messages(i), read.message)
|
assertEquals("Message should match appended.", records(i), read.record)
|
||||||
}
|
}
|
||||||
assertEquals("Reading beyond the last message returns nothing.", 0, log.read(messages.length, 100, None).messageSet.size)
|
assertEquals("Reading beyond the last message returns nothing.", 0, log.read(records.length, 100, None).records.shallowIterator.asScala.size)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -211,16 +213,16 @@ class LogTest extends JUnitSuite {
|
||||||
logProps.put(LogConfig.SegmentBytesProp, 71: java.lang.Integer)
|
logProps.put(LogConfig.SegmentBytesProp, 71: java.lang.Integer)
|
||||||
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
|
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
|
||||||
val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray
|
val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray
|
||||||
val messages = messageIds.map(id => new Message(id.toString.getBytes))
|
val records = messageIds.map(id => Record.create(id.toString.getBytes))
|
||||||
|
|
||||||
// now test the case that we give the offsets and use non-sequential offsets
|
// now test the case that we give the offsets and use non-sequential offsets
|
||||||
for(i <- 0 until messages.length)
|
for(i <- records.indices)
|
||||||
log.append(new ByteBufferMessageSet(NoCompressionCodec, new LongRef(messageIds(i)), messages = messages(i)), assignOffsets = false)
|
log.append(MemoryRecords.withLogEntries(LogEntry.create(messageIds(i), records(i))), assignOffsets = false)
|
||||||
for(i <- 50 until messageIds.max) {
|
for(i <- 50 until messageIds.max) {
|
||||||
val idx = messageIds.indexWhere(_ >= i)
|
val idx = messageIds.indexWhere(_ >= i)
|
||||||
val read = log.read(i, 100, None).messageSet.head
|
val read = log.read(i, 100, None).records.shallowIterator.next()
|
||||||
assertEquals("Offset read should match message id.", messageIds(idx), read.offset)
|
assertEquals("Offset read should match message id.", messageIds(idx), read.offset)
|
||||||
assertEquals("Message should match appended.", messages(idx), read.message)
|
assertEquals("Message should match appended.", records(idx), read.record)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -238,12 +240,12 @@ class LogTest extends JUnitSuite {
|
||||||
|
|
||||||
// keep appending until we have two segments with only a single message in the second segment
|
// keep appending until we have two segments with only a single message in the second segment
|
||||||
while(log.numberOfSegments == 1)
|
while(log.numberOfSegments == 1)
|
||||||
log.append(new ByteBufferMessageSet(NoCompressionCodec, messages = new Message("42".getBytes)))
|
log.append(MemoryRecords.withRecords(Record.create("42".getBytes)))
|
||||||
|
|
||||||
// now manually truncate off all but one message from the first segment to create a gap in the messages
|
// now manually truncate off all but one message from the first segment to create a gap in the messages
|
||||||
log.logSegments.head.truncateTo(1)
|
log.logSegments.head.truncateTo(1)
|
||||||
|
|
||||||
assertEquals("A read should now return the last message in the log", log.logEndOffset - 1, log.read(1, 200, None).messageSet.head.offset)
|
assertEquals("A read should now return the last message in the log", log.logEndOffset - 1, log.read(1, 200, None).records.shallowIterator.next().offset)
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
@ -252,12 +254,11 @@ class LogTest extends JUnitSuite {
|
||||||
logProps.put(LogConfig.SegmentBytesProp, 71: java.lang.Integer)
|
logProps.put(LogConfig.SegmentBytesProp, 71: java.lang.Integer)
|
||||||
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
|
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
|
||||||
val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray
|
val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray
|
||||||
val messages = messageIds.map(id => new Message(id.toString.getBytes))
|
val records = messageIds.map(id => Record.create(id.toString.getBytes))
|
||||||
|
|
||||||
// now test the case that we give the offsets and use non-sequential offsets
|
// now test the case that we give the offsets and use non-sequential offsets
|
||||||
for (i <- 0 until messages.length)
|
for (i <- records.indices)
|
||||||
log.append(new ByteBufferMessageSet(NoCompressionCodec, new LongRef(messageIds(i)), messages = messages(i)),
|
log.append(MemoryRecords.withLogEntries(LogEntry.create(messageIds(i), records(i))), assignOffsets = false)
|
||||||
assignOffsets = false)
|
|
||||||
|
|
||||||
for (i <- 50 until messageIds.max) {
|
for (i <- 50 until messageIds.max) {
|
||||||
val idx = messageIds.indexWhere(_ >= i)
|
val idx = messageIds.indexWhere(_ >= i)
|
||||||
|
|
@ -265,13 +266,13 @@ class LogTest extends JUnitSuite {
|
||||||
log.read(i, 1, minOneMessage = true),
|
log.read(i, 1, minOneMessage = true),
|
||||||
log.read(i, 100, minOneMessage = true),
|
log.read(i, 100, minOneMessage = true),
|
||||||
log.read(i, 100, Some(10000), minOneMessage = true)
|
log.read(i, 100, Some(10000), minOneMessage = true)
|
||||||
).map(_.messageSet.head)
|
).map(_.records.shallowIterator.next())
|
||||||
reads.foreach { read =>
|
reads.foreach { read =>
|
||||||
assertEquals("Offset read should match message id.", messageIds(idx), read.offset)
|
assertEquals("Offset read should match message id.", messageIds(idx), read.offset)
|
||||||
assertEquals("Message should match appended.", messages(idx), read.message)
|
assertEquals("Message should match appended.", records(idx), read.record)
|
||||||
}
|
}
|
||||||
|
|
||||||
assertEquals(Seq.empty, log.read(i, 1, Some(1), minOneMessage = true).messageSet.toIndexedSeq)
|
assertEquals(Seq.empty, log.read(i, 1, Some(1), minOneMessage = true).records.shallowIterator.asScala.toIndexedSeq)
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
@ -282,15 +283,14 @@ class LogTest extends JUnitSuite {
|
||||||
logProps.put(LogConfig.SegmentBytesProp, 71: java.lang.Integer)
|
logProps.put(LogConfig.SegmentBytesProp, 71: java.lang.Integer)
|
||||||
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
|
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
|
||||||
val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray
|
val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray
|
||||||
val messages = messageIds.map(id => new Message(id.toString.getBytes))
|
val records = messageIds.map(id => Record.create(id.toString.getBytes))
|
||||||
|
|
||||||
// now test the case that we give the offsets and use non-sequential offsets
|
// now test the case that we give the offsets and use non-sequential offsets
|
||||||
for (i <- 0 until messages.length)
|
for (i <- records.indices)
|
||||||
log.append(new ByteBufferMessageSet(NoCompressionCodec, new LongRef(messageIds(i)), messages = messages(i)),
|
log.append(MemoryRecords.withLogEntries(LogEntry.create(messageIds(i), records(i))), assignOffsets = false)
|
||||||
assignOffsets = false)
|
|
||||||
|
|
||||||
for (i <- 50 until messageIds.max) {
|
for (i <- 50 until messageIds.max) {
|
||||||
assertEquals(MessageSet.Empty, log.read(i, 0).messageSet)
|
assertEquals(MemoryRecords.EMPTY, log.read(i, 0).records)
|
||||||
|
|
||||||
// we return an incomplete message instead of an empty one for the case below
|
// we return an incomplete message instead of an empty one for the case below
|
||||||
// we use this mechanism to tell consumers of the fetch request version 2 and below that the message size is
|
// we use this mechanism to tell consumers of the fetch request version 2 and below that the message size is
|
||||||
|
|
@ -298,9 +298,9 @@ class LogTest extends JUnitSuite {
|
||||||
// in fetch request version 3, we no longer need this as we return oversized messages from the first non-empty
|
// in fetch request version 3, we no longer need this as we return oversized messages from the first non-empty
|
||||||
// partition
|
// partition
|
||||||
val fetchInfo = log.read(i, 1)
|
val fetchInfo = log.read(i, 1)
|
||||||
assertTrue(fetchInfo.firstMessageSetIncomplete)
|
assertTrue(fetchInfo.firstEntryIncomplete)
|
||||||
assertTrue(fetchInfo.messageSet.isInstanceOf[FileMessageSet])
|
assertTrue(fetchInfo.records.isInstanceOf[FileRecords])
|
||||||
assertEquals(1, fetchInfo.messageSet.sizeInBytes)
|
assertEquals(1, fetchInfo.records.sizeInBytes)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -318,9 +318,9 @@ class LogTest extends JUnitSuite {
|
||||||
// set up replica log starting with offset 1024 and with one message (at offset 1024)
|
// set up replica log starting with offset 1024 and with one message (at offset 1024)
|
||||||
logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer)
|
logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer)
|
||||||
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
|
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
|
||||||
log.append(new ByteBufferMessageSet(NoCompressionCodec, messages = new Message("42".getBytes)))
|
log.append(MemoryRecords.withRecords(Record.create("42".getBytes)))
|
||||||
|
|
||||||
assertEquals("Reading at the log end offset should produce 0 byte read.", 0, log.read(1025, 1000).messageSet.sizeInBytes)
|
assertEquals("Reading at the log end offset should produce 0 byte read.", 0, log.read(1025, 1000).records.sizeInBytes)
|
||||||
|
|
||||||
try {
|
try {
|
||||||
log.read(0, 1000)
|
log.read(0, 1000)
|
||||||
|
|
@ -336,7 +336,7 @@ class LogTest extends JUnitSuite {
|
||||||
case _: OffsetOutOfRangeException => // This is good.
|
case _: OffsetOutOfRangeException => // This is good.
|
||||||
}
|
}
|
||||||
|
|
||||||
assertEquals("Reading from below the specified maxOffset should produce 0 byte read.", 0, log.read(1025, 1000, Some(1024)).messageSet.sizeInBytes)
|
assertEquals("Reading from below the specified maxOffset should produce 0 byte read.", 0, log.read(1025, 1000, Some(1024)).records.sizeInBytes)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -350,21 +350,22 @@ class LogTest extends JUnitSuite {
|
||||||
logProps.put(LogConfig.SegmentBytesProp, 100: java.lang.Integer)
|
logProps.put(LogConfig.SegmentBytesProp, 100: java.lang.Integer)
|
||||||
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
|
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
|
||||||
val numMessages = 100
|
val numMessages = 100
|
||||||
val messageSets = (0 until numMessages).map(i => TestUtils.singleMessageSet(i.toString.getBytes))
|
val messageSets = (0 until numMessages).map(i => TestUtils.singletonRecords(i.toString.getBytes))
|
||||||
messageSets.foreach(log.append(_))
|
messageSets.foreach(log.append(_))
|
||||||
log.flush
|
log.flush
|
||||||
|
|
||||||
/* do successive reads to ensure all our messages are there */
|
/* do successive reads to ensure all our messages are there */
|
||||||
var offset = 0L
|
var offset = 0L
|
||||||
for(i <- 0 until numMessages) {
|
for(i <- 0 until numMessages) {
|
||||||
val messages = log.read(offset, 1024*1024).messageSet
|
val messages = log.read(offset, 1024*1024).records.shallowIterator
|
||||||
assertEquals("Offsets not equal", offset, messages.head.offset)
|
val head = messages.next()
|
||||||
assertEquals("Messages not equal at offset " + offset, messageSets(i).head.message,
|
assertEquals("Offsets not equal", offset, head.offset)
|
||||||
messages.head.message.toFormatVersion(messageSets(i).head.message.magic))
|
assertEquals("Messages not equal at offset " + offset, messageSets(i).shallowIterator.next().record,
|
||||||
offset = messages.head.offset + 1
|
head.record.convert(messageSets(i).shallowIterator.next().record.magic))
|
||||||
|
offset = head.offset + 1
|
||||||
}
|
}
|
||||||
val lastRead = log.read(startOffset = numMessages, maxLength = 1024*1024, maxOffset = Some(numMessages + 1)).messageSet
|
val lastRead = log.read(startOffset = numMessages, maxLength = 1024*1024, maxOffset = Some(numMessages + 1)).records
|
||||||
assertEquals("Should be no more messages", 0, lastRead.size)
|
assertEquals("Should be no more messages", 0, lastRead.shallowIterator.asScala.size)
|
||||||
|
|
||||||
// check that rolling the log forced a flushed the log--the flush is asyn so retry in case of failure
|
// check that rolling the log forced a flushed the log--the flush is asyn so retry in case of failure
|
||||||
TestUtils.retry(1000L){
|
TestUtils.retry(1000L){
|
||||||
|
|
@ -383,10 +384,10 @@ class LogTest extends JUnitSuite {
|
||||||
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
|
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
|
||||||
|
|
||||||
/* append 2 compressed message sets, each with two messages giving offsets 0, 1, 2, 3 */
|
/* append 2 compressed message sets, each with two messages giving offsets 0, 1, 2, 3 */
|
||||||
log.append(new ByteBufferMessageSet(DefaultCompressionCodec, new Message("hello".getBytes), new Message("there".getBytes)))
|
log.append(MemoryRecords.withRecords(CompressionType.GZIP, Record.create("hello".getBytes), Record.create("there".getBytes)))
|
||||||
log.append(new ByteBufferMessageSet(DefaultCompressionCodec, new Message("alpha".getBytes), new Message("beta".getBytes)))
|
log.append(MemoryRecords.withRecords(CompressionType.GZIP, Record.create("alpha".getBytes), Record.create("beta".getBytes)))
|
||||||
|
|
||||||
def read(offset: Int) = ByteBufferMessageSet.deepIterator(log.read(offset, 4096).messageSet.head)
|
def read(offset: Int) = log.read(offset, 4096).records.deepIterator
|
||||||
|
|
||||||
/* we should always get the first message in the compressed set when reading any offset in the set */
|
/* we should always get the first message in the compressed set when reading any offset in the set */
|
||||||
assertEquals("Read at offset 0 should produce 0", 0, read(0).next().offset)
|
assertEquals("Read at offset 0 should produce 0", 0, read(0).next().offset)
|
||||||
|
|
@ -408,7 +409,7 @@ class LogTest extends JUnitSuite {
|
||||||
logProps.put(LogConfig.RetentionMsProp, 0: java.lang.Integer)
|
logProps.put(LogConfig.RetentionMsProp, 0: java.lang.Integer)
|
||||||
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
|
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
|
||||||
for(i <- 0 until messagesToAppend)
|
for(i <- 0 until messagesToAppend)
|
||||||
log.append(TestUtils.singleMessageSet(payload = i.toString.getBytes, timestamp = time.milliseconds - 10))
|
log.append(TestUtils.singletonRecords(value = i.toString.getBytes, timestamp = time.milliseconds - 10))
|
||||||
|
|
||||||
val currOffset = log.logEndOffset
|
val currOffset = log.logEndOffset
|
||||||
assertEquals(currOffset, messagesToAppend)
|
assertEquals(currOffset, messagesToAppend)
|
||||||
|
|
@ -422,7 +423,7 @@ class LogTest extends JUnitSuite {
|
||||||
assertEquals("Still no change in the logEndOffset", currOffset, log.logEndOffset)
|
assertEquals("Still no change in the logEndOffset", currOffset, log.logEndOffset)
|
||||||
assertEquals("Should still be able to append and should get the logEndOffset assigned to the new append",
|
assertEquals("Should still be able to append and should get the logEndOffset assigned to the new append",
|
||||||
currOffset,
|
currOffset,
|
||||||
log.append(TestUtils.singleMessageSet("hello".getBytes)).firstOffset)
|
log.append(TestUtils.singletonRecords("hello".getBytes)).firstOffset)
|
||||||
|
|
||||||
// cleanup the log
|
// cleanup the log
|
||||||
log.delete()
|
log.delete()
|
||||||
|
|
@ -435,7 +436,7 @@ class LogTest extends JUnitSuite {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
def testMessageSetSizeCheck() {
|
def testMessageSetSizeCheck() {
|
||||||
val messageSet = new ByteBufferMessageSet(NoCompressionCodec, new Message ("You".getBytes), new Message("bethe".getBytes))
|
val messageSet = MemoryRecords.withRecords(Record.create("You".getBytes), Record.create("bethe".getBytes))
|
||||||
// append messages to log
|
// append messages to log
|
||||||
val configSegmentSize = messageSet.sizeInBytes - 1
|
val configSegmentSize = messageSet.sizeInBytes - 1
|
||||||
val logProps = new Properties()
|
val logProps = new Properties()
|
||||||
|
|
@ -454,17 +455,17 @@ class LogTest extends JUnitSuite {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
def testCompactedTopicConstraints() {
|
def testCompactedTopicConstraints() {
|
||||||
val keyedMessage = new Message(bytes = "this message has a key".getBytes, key = "and here it is".getBytes, Message.NoTimestamp, Message.CurrentMagicValue)
|
val keyedMessage = Record.create(Record.CURRENT_MAGIC_VALUE, Record.NO_TIMESTAMP, "and here it is".getBytes, "this message has a key".getBytes)
|
||||||
val anotherKeyedMessage = new Message(bytes = "this message also has a key".getBytes, key ="another key".getBytes, Message.NoTimestamp, Message.CurrentMagicValue)
|
val anotherKeyedMessage = Record.create(Record.CURRENT_MAGIC_VALUE, Record.NO_TIMESTAMP, "another key".getBytes, "this message also has a key".getBytes)
|
||||||
val unkeyedMessage = new Message(bytes = "this message does not have a key".getBytes)
|
val unkeyedMessage = Record.create("this message does not have a key".getBytes)
|
||||||
|
|
||||||
val messageSetWithUnkeyedMessage = new ByteBufferMessageSet(NoCompressionCodec, unkeyedMessage, keyedMessage)
|
val messageSetWithUnkeyedMessage = MemoryRecords.withRecords(CompressionType.NONE, unkeyedMessage, keyedMessage)
|
||||||
val messageSetWithOneUnkeyedMessage = new ByteBufferMessageSet(NoCompressionCodec, unkeyedMessage)
|
val messageSetWithOneUnkeyedMessage = MemoryRecords.withRecords(CompressionType.NONE, unkeyedMessage)
|
||||||
val messageSetWithCompressedKeyedMessage = new ByteBufferMessageSet(GZIPCompressionCodec, keyedMessage)
|
val messageSetWithCompressedKeyedMessage = MemoryRecords.withRecords(CompressionType.GZIP, keyedMessage)
|
||||||
val messageSetWithCompressedUnkeyedMessage = new ByteBufferMessageSet(GZIPCompressionCodec, keyedMessage, unkeyedMessage)
|
val messageSetWithCompressedUnkeyedMessage = MemoryRecords.withRecords(CompressionType.GZIP, keyedMessage, unkeyedMessage)
|
||||||
|
|
||||||
val messageSetWithKeyedMessage = new ByteBufferMessageSet(NoCompressionCodec, keyedMessage)
|
val messageSetWithKeyedMessage = MemoryRecords.withRecords(CompressionType.NONE, keyedMessage)
|
||||||
val messageSetWithKeyedMessages = new ByteBufferMessageSet(NoCompressionCodec, keyedMessage, anotherKeyedMessage)
|
val messageSetWithKeyedMessages = MemoryRecords.withRecords(CompressionType.NONE, keyedMessage, anotherKeyedMessage)
|
||||||
|
|
||||||
val logProps = new Properties()
|
val logProps = new Properties()
|
||||||
logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Compact)
|
logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Compact)
|
||||||
|
|
@ -502,8 +503,8 @@ class LogTest extends JUnitSuite {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
def testMessageSizeCheck() {
|
def testMessageSizeCheck() {
|
||||||
val first = new ByteBufferMessageSet(NoCompressionCodec, new Message ("You".getBytes), new Message("bethe".getBytes))
|
val first = MemoryRecords.withRecords(CompressionType.NONE, Record.create("You".getBytes), Record.create("bethe".getBytes))
|
||||||
val second = new ByteBufferMessageSet(NoCompressionCodec, new Message("change (I need more bytes)".getBytes))
|
val second = MemoryRecords.withRecords(CompressionType.NONE, Record.create("change (I need more bytes)".getBytes))
|
||||||
|
|
||||||
// append messages to log
|
// append messages to log
|
||||||
val maxMessageSize = second.sizeInBytes - 1
|
val maxMessageSize = second.sizeInBytes - 1
|
||||||
|
|
@ -537,7 +538,7 @@ class LogTest extends JUnitSuite {
|
||||||
val config = LogConfig(logProps)
|
val config = LogConfig(logProps)
|
||||||
var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time)
|
var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time)
|
||||||
for(i <- 0 until numMessages)
|
for(i <- 0 until numMessages)
|
||||||
log.append(TestUtils.singleMessageSet(payload = TestUtils.randomBytes(messageSize),
|
log.append(TestUtils.singletonRecords(value = TestUtils.randomBytes(messageSize),
|
||||||
timestamp = time.milliseconds + i * 10))
|
timestamp = time.milliseconds + i * 10))
|
||||||
assertEquals("After appending %d messages to an empty log, the log end offset should be %d".format(numMessages, numMessages), numMessages, log.logEndOffset)
|
assertEquals("After appending %d messages to an empty log, the log end offset should be %d".format(numMessages, numMessages), numMessages, log.logEndOffset)
|
||||||
val lastIndexOffset = log.activeSegment.index.lastOffset
|
val lastIndexOffset = log.activeSegment.index.lastOffset
|
||||||
|
|
@ -585,7 +586,7 @@ class LogTest extends JUnitSuite {
|
||||||
val log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time)
|
val log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time)
|
||||||
|
|
||||||
val messages = (0 until numMessages).map { i =>
|
val messages = (0 until numMessages).map { i =>
|
||||||
new ByteBufferMessageSet(NoCompressionCodec, new LongRef(100 + i), new Message(i.toString.getBytes(), time.milliseconds + i, Message.MagicValue_V1))
|
MemoryRecords.withLogEntries(LogEntry.create(100 + i, Record.create(Record.MAGIC_VALUE_V1, time.milliseconds + i, i.toString.getBytes())))
|
||||||
}
|
}
|
||||||
messages.foreach(log.append(_, assignOffsets = false))
|
messages.foreach(log.append(_, assignOffsets = false))
|
||||||
val timeIndexEntries = log.logSegments.foldLeft(0) { (entries, segment) => entries + segment.timeIndex.entries }
|
val timeIndexEntries = log.logSegments.foldLeft(0) { (entries, segment) => entries + segment.timeIndex.entries }
|
||||||
|
|
@ -608,7 +609,7 @@ class LogTest extends JUnitSuite {
|
||||||
val config = LogConfig(logProps)
|
val config = LogConfig(logProps)
|
||||||
var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time)
|
var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time)
|
||||||
for(i <- 0 until numMessages)
|
for(i <- 0 until numMessages)
|
||||||
log.append(TestUtils.singleMessageSet(payload = TestUtils.randomBytes(10), timestamp = time.milliseconds + i * 10))
|
log.append(TestUtils.singletonRecords(value = TestUtils.randomBytes(10), timestamp = time.milliseconds + i * 10))
|
||||||
val indexFiles = log.logSegments.map(_.index.file)
|
val indexFiles = log.logSegments.map(_.index.file)
|
||||||
val timeIndexFiles = log.logSegments.map(_.timeIndex.file)
|
val timeIndexFiles = log.logSegments.map(_.timeIndex.file)
|
||||||
log.close()
|
log.close()
|
||||||
|
|
@ -623,7 +624,7 @@ class LogTest extends JUnitSuite {
|
||||||
assertTrue("The index should have been rebuilt", log.logSegments.head.index.entries > 0)
|
assertTrue("The index should have been rebuilt", log.logSegments.head.index.entries > 0)
|
||||||
assertTrue("The time index should have been rebuilt", log.logSegments.head.timeIndex.entries > 0)
|
assertTrue("The time index should have been rebuilt", log.logSegments.head.timeIndex.entries > 0)
|
||||||
for(i <- 0 until numMessages) {
|
for(i <- 0 until numMessages) {
|
||||||
assertEquals(i, log.read(i, 100, None).messageSet.head.offset)
|
assertEquals(i, log.read(i, 100, None).records.shallowIterator.next().offset)
|
||||||
if (i == 0)
|
if (i == 0)
|
||||||
assertEquals(log.logSegments.head.baseOffset, log.fetchOffsetsByTimestamp(time.milliseconds + i * 10).get.offset)
|
assertEquals(log.logSegments.head.baseOffset, log.fetchOffsetsByTimestamp(time.milliseconds + i * 10).get.offset)
|
||||||
else
|
else
|
||||||
|
|
@ -647,7 +648,7 @@ class LogTest extends JUnitSuite {
|
||||||
val config = LogConfig(logProps)
|
val config = LogConfig(logProps)
|
||||||
var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time)
|
var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time)
|
||||||
for(i <- 0 until numMessages)
|
for(i <- 0 until numMessages)
|
||||||
log.append(TestUtils.singleMessageSet(payload = TestUtils.randomBytes(10), timestamp = time.milliseconds + i * 10))
|
log.append(TestUtils.singletonRecords(value = TestUtils.randomBytes(10), timestamp = time.milliseconds + i * 10))
|
||||||
val timeIndexFiles = log.logSegments.map(_.timeIndex.file)
|
val timeIndexFiles = log.logSegments.map(_.timeIndex.file)
|
||||||
log.close()
|
log.close()
|
||||||
|
|
||||||
|
|
@ -676,7 +677,7 @@ class LogTest extends JUnitSuite {
|
||||||
val config = LogConfig(logProps)
|
val config = LogConfig(logProps)
|
||||||
var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time)
|
var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time)
|
||||||
for(i <- 0 until numMessages)
|
for(i <- 0 until numMessages)
|
||||||
log.append(TestUtils.singleMessageSet(payload = TestUtils.randomBytes(10), timestamp = time.milliseconds + i * 10))
|
log.append(TestUtils.singletonRecords(value = TestUtils.randomBytes(10), timestamp = time.milliseconds + i * 10))
|
||||||
val indexFiles = log.logSegments.map(_.index.file)
|
val indexFiles = log.logSegments.map(_.index.file)
|
||||||
val timeIndexFiles = log.logSegments.map(_.timeIndex.file)
|
val timeIndexFiles = log.logSegments.map(_.timeIndex.file)
|
||||||
log.close()
|
log.close()
|
||||||
|
|
@ -699,7 +700,7 @@ class LogTest extends JUnitSuite {
|
||||||
log = new Log(logDir, config, recoveryPoint = 200L, time.scheduler, time)
|
log = new Log(logDir, config, recoveryPoint = 200L, time.scheduler, time)
|
||||||
assertEquals("Should have %d messages when log is reopened".format(numMessages), numMessages, log.logEndOffset)
|
assertEquals("Should have %d messages when log is reopened".format(numMessages), numMessages, log.logEndOffset)
|
||||||
for(i <- 0 until numMessages) {
|
for(i <- 0 until numMessages) {
|
||||||
assertEquals(i, log.read(i, 100, None).messageSet.head.offset)
|
assertEquals(i, log.read(i, 100, None).records.shallowIterator.next().offset)
|
||||||
if (i == 0)
|
if (i == 0)
|
||||||
assertEquals(log.logSegments.head.baseOffset, log.fetchOffsetsByTimestamp(time.milliseconds + i * 10).get.offset)
|
assertEquals(log.logSegments.head.baseOffset, log.fetchOffsetsByTimestamp(time.milliseconds + i * 10).get.offset)
|
||||||
else
|
else
|
||||||
|
|
@ -713,7 +714,7 @@ class LogTest extends JUnitSuite {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
def testTruncateTo() {
|
def testTruncateTo() {
|
||||||
val set = TestUtils.singleMessageSet("test".getBytes)
|
val set = TestUtils.singletonRecords("test".getBytes)
|
||||||
val setSize = set.sizeInBytes
|
val setSize = set.sizeInBytes
|
||||||
val msgPerSeg = 10
|
val msgPerSeg = 10
|
||||||
val segmentSize = msgPerSeg * setSize // each segment will be 10 messages
|
val segmentSize = msgPerSeg * setSize // each segment will be 10 messages
|
||||||
|
|
@ -770,7 +771,7 @@ class LogTest extends JUnitSuite {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
def testIndexResizingAtTruncation() {
|
def testIndexResizingAtTruncation() {
|
||||||
val setSize = TestUtils.singleMessageSet(payload = "test".getBytes).sizeInBytes
|
val setSize = TestUtils.singletonRecords(value = "test".getBytes).sizeInBytes
|
||||||
val msgPerSeg = 10
|
val msgPerSeg = 10
|
||||||
val segmentSize = msgPerSeg * setSize // each segment will be 10 messages
|
val segmentSize = msgPerSeg * setSize // each segment will be 10 messages
|
||||||
val logProps = new Properties()
|
val logProps = new Properties()
|
||||||
|
|
@ -781,12 +782,12 @@ class LogTest extends JUnitSuite {
|
||||||
assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
|
assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
|
||||||
|
|
||||||
for (i<- 1 to msgPerSeg)
|
for (i<- 1 to msgPerSeg)
|
||||||
log.append(TestUtils.singleMessageSet(payload = "test".getBytes, timestamp = time.milliseconds + i))
|
log.append(TestUtils.singletonRecords(value = "test".getBytes, timestamp = time.milliseconds + i))
|
||||||
assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
|
assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
|
||||||
|
|
||||||
time.sleep(msgPerSeg)
|
time.sleep(msgPerSeg)
|
||||||
for (i<- 1 to msgPerSeg)
|
for (i<- 1 to msgPerSeg)
|
||||||
log.append(TestUtils.singleMessageSet(payload = "test".getBytes, timestamp = time.milliseconds + i))
|
log.append(TestUtils.singletonRecords(value = "test".getBytes, timestamp = time.milliseconds + i))
|
||||||
assertEquals("There should be exactly 2 segment.", 2, log.numberOfSegments)
|
assertEquals("There should be exactly 2 segment.", 2, log.numberOfSegments)
|
||||||
val expectedEntries = msgPerSeg - 1
|
val expectedEntries = msgPerSeg - 1
|
||||||
|
|
||||||
|
|
@ -800,7 +801,7 @@ class LogTest extends JUnitSuite {
|
||||||
|
|
||||||
time.sleep(msgPerSeg)
|
time.sleep(msgPerSeg)
|
||||||
for (i<- 1 to msgPerSeg)
|
for (i<- 1 to msgPerSeg)
|
||||||
log.append(TestUtils.singleMessageSet(payload = "test".getBytes, timestamp = time.milliseconds + i))
|
log.append(TestUtils.singletonRecords(value = "test".getBytes, timestamp = time.milliseconds + i))
|
||||||
assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
|
assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -814,7 +815,7 @@ class LogTest extends JUnitSuite {
|
||||||
val bogusIndex2 = Log.indexFilename(logDir, 5)
|
val bogusIndex2 = Log.indexFilename(logDir, 5)
|
||||||
val bogusTimeIndex2 = Log.timeIndexFilename(logDir, 5)
|
val bogusTimeIndex2 = Log.timeIndexFilename(logDir, 5)
|
||||||
|
|
||||||
val set = TestUtils.singleMessageSet("test".getBytes)
|
val set = TestUtils.singletonRecords("test".getBytes)
|
||||||
val logProps = new Properties()
|
val logProps = new Properties()
|
||||||
logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer)
|
logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer)
|
||||||
logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer)
|
logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer)
|
||||||
|
|
@ -842,7 +843,7 @@ class LogTest extends JUnitSuite {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
def testReopenThenTruncate() {
|
def testReopenThenTruncate() {
|
||||||
val set = TestUtils.singleMessageSet("test".getBytes)
|
val set = TestUtils.singletonRecords("test".getBytes)
|
||||||
val logProps = new Properties()
|
val logProps = new Properties()
|
||||||
logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer)
|
logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer)
|
||||||
logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer)
|
logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer)
|
||||||
|
|
@ -875,7 +876,7 @@ class LogTest extends JUnitSuite {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
def testAsyncDelete() {
|
def testAsyncDelete() {
|
||||||
val set = TestUtils.singleMessageSet("test".getBytes)
|
val set = TestUtils.singletonRecords("test".getBytes)
|
||||||
val asyncDeleteMs = 1000
|
val asyncDeleteMs = 1000
|
||||||
val logProps = new Properties()
|
val logProps = new Properties()
|
||||||
logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer)
|
logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer)
|
||||||
|
|
@ -921,7 +922,7 @@ class LogTest extends JUnitSuite {
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
def testOpenDeletesObsoleteFiles() {
|
def testOpenDeletesObsoleteFiles() {
|
||||||
val set = TestUtils.singleMessageSet("test".getBytes)
|
val set = TestUtils.singletonRecords("test".getBytes)
|
||||||
val logProps = new Properties()
|
val logProps = new Properties()
|
||||||
logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer)
|
logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer)
|
||||||
logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer)
|
logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer)
|
||||||
|
|
@ -957,10 +958,10 @@ class LogTest extends JUnitSuite {
|
||||||
recoveryPoint = 0L,
|
recoveryPoint = 0L,
|
||||||
time.scheduler,
|
time.scheduler,
|
||||||
time)
|
time)
|
||||||
log.append(new ByteBufferMessageSet(new Message(bytes = null)))
|
log.append(MemoryRecords.withRecords(Record.create(null)))
|
||||||
val messageSet = log.read(0, 4096, None).messageSet
|
val head = log.read(0, 4096, None).records.shallowIterator().next()
|
||||||
assertEquals(0, messageSet.head.offset)
|
assertEquals(0, head.offset)
|
||||||
assertTrue("Message payload should be null.", messageSet.head.message.isNull)
|
assertTrue("Message payload should be null.", head.record.hasNullValue)
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(expected = classOf[IllegalArgumentException])
|
@Test(expected = classOf[IllegalArgumentException])
|
||||||
|
|
@ -970,9 +971,9 @@ class LogTest extends JUnitSuite {
|
||||||
recoveryPoint = 0L,
|
recoveryPoint = 0L,
|
||||||
time.scheduler,
|
time.scheduler,
|
||||||
time)
|
time)
|
||||||
val messages = (0 until 2).map(id => new Message(id.toString.getBytes)).toArray
|
val messages = (0 until 2).map(id => Record.create(id.toString.getBytes)).toArray
|
||||||
messages.foreach(message => log.append(new ByteBufferMessageSet(message)))
|
messages.foreach(record => log.append(MemoryRecords.withRecords(record)))
|
||||||
val invalidMessage = new ByteBufferMessageSet(new Message(1.toString.getBytes))
|
val invalidMessage = MemoryRecords.withRecords(Record.create(1.toString.getBytes))
|
||||||
log.append(invalidMessage, assignOffsets = false)
|
log.append(invalidMessage, assignOffsets = false)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -984,7 +985,7 @@ class LogTest extends JUnitSuite {
|
||||||
logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer)
|
logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer)
|
||||||
logProps.put(LogConfig.MaxMessageBytesProp, 64*1024: java.lang.Integer)
|
logProps.put(LogConfig.MaxMessageBytesProp, 64*1024: java.lang.Integer)
|
||||||
val config = LogConfig(logProps)
|
val config = LogConfig(logProps)
|
||||||
val set = TestUtils.singleMessageSet("test".getBytes)
|
val set = TestUtils.singletonRecords("test".getBytes)
|
||||||
val recoveryPoint = 50L
|
val recoveryPoint = 50L
|
||||||
for (_ <- 0 until 50) {
|
for (_ <- 0 until 50) {
|
||||||
// create a log and write some messages to it
|
// create a log and write some messages to it
|
||||||
|
|
@ -997,7 +998,7 @@ class LogTest extends JUnitSuite {
|
||||||
val numMessages = 50 + TestUtils.random.nextInt(50)
|
val numMessages = 50 + TestUtils.random.nextInt(50)
|
||||||
for (_ <- 0 until numMessages)
|
for (_ <- 0 until numMessages)
|
||||||
log.append(set)
|
log.append(set)
|
||||||
val messages = log.logSegments.flatMap(_.log.iterator.toList)
|
val messages = log.logSegments.flatMap(_.log.deepIterator.asScala.toList)
|
||||||
log.close()
|
log.close()
|
||||||
|
|
||||||
// corrupt index and log by appending random bytes
|
// corrupt index and log by appending random bytes
|
||||||
|
|
@ -1007,7 +1008,8 @@ class LogTest extends JUnitSuite {
|
||||||
// attempt recovery
|
// attempt recovery
|
||||||
log = new Log(logDir, config, recoveryPoint, time.scheduler, time)
|
log = new Log(logDir, config, recoveryPoint, time.scheduler, time)
|
||||||
assertEquals(numMessages, log.logEndOffset)
|
assertEquals(numMessages, log.logEndOffset)
|
||||||
assertEquals("Messages in the log after recovery should be the same.", messages, log.logSegments.flatMap(_.log.iterator.toList))
|
assertEquals("Messages in the log after recovery should be the same.", messages,
|
||||||
|
log.logSegments.flatMap(_.log.deepIterator.asScala.toList))
|
||||||
Utils.delete(logDir)
|
Utils.delete(logDir)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -1020,7 +1022,7 @@ class LogTest extends JUnitSuite {
|
||||||
logProps.put(LogConfig.MaxMessageBytesProp, 64*1024: java.lang.Integer)
|
logProps.put(LogConfig.MaxMessageBytesProp, 64*1024: java.lang.Integer)
|
||||||
logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer)
|
logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer)
|
||||||
val config = LogConfig(logProps)
|
val config = LogConfig(logProps)
|
||||||
val set = TestUtils.singleMessageSet("test".getBytes)
|
val set = TestUtils.singletonRecords("test".getBytes)
|
||||||
val parentLogDir = logDir.getParentFile
|
val parentLogDir = logDir.getParentFile
|
||||||
assertTrue("Data directory %s must exist", parentLogDir.isDirectory)
|
assertTrue("Data directory %s must exist", parentLogDir.isDirectory)
|
||||||
val cleanShutdownFile = new File(parentLogDir, Log.CleanShutdownFile)
|
val cleanShutdownFile = new File(parentLogDir, Log.CleanShutdownFile)
|
||||||
|
|
@ -1121,7 +1123,7 @@ class LogTest extends JUnitSuite {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
def testDeleteOldSegmentsMethod() {
|
def testDeleteOldSegmentsMethod() {
|
||||||
val set = TestUtils.singleMessageSet("test".getBytes)
|
val set = TestUtils.singletonRecords("test".getBytes)
|
||||||
val logProps = new Properties()
|
val logProps = new Properties()
|
||||||
logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer)
|
logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer)
|
||||||
logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer)
|
logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer)
|
||||||
|
|
@ -1154,7 +1156,7 @@ class LogTest extends JUnitSuite {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
def shouldDeleteSizeBasedSegments() {
|
def shouldDeleteSizeBasedSegments() {
|
||||||
val set = TestUtils.singleMessageSet("test".getBytes)
|
val set = TestUtils.singletonRecords("test".getBytes)
|
||||||
val log = createLog(set.sizeInBytes, retentionBytes = set.sizeInBytes * 10)
|
val log = createLog(set.sizeInBytes, retentionBytes = set.sizeInBytes * 10)
|
||||||
|
|
||||||
// append some messages to create some segments
|
// append some messages to create some segments
|
||||||
|
|
@ -1167,7 +1169,7 @@ class LogTest extends JUnitSuite {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
def shouldNotDeleteSizeBasedSegmentsWhenUnderRetentionSize() {
|
def shouldNotDeleteSizeBasedSegmentsWhenUnderRetentionSize() {
|
||||||
val set = TestUtils.singleMessageSet("test".getBytes)
|
val set = TestUtils.singletonRecords("test".getBytes)
|
||||||
val log = createLog(set.sizeInBytes, retentionBytes = set.sizeInBytes * 15)
|
val log = createLog(set.sizeInBytes, retentionBytes = set.sizeInBytes * 15)
|
||||||
|
|
||||||
// append some messages to create some segments
|
// append some messages to create some segments
|
||||||
|
|
@ -1180,7 +1182,7 @@ class LogTest extends JUnitSuite {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
def shouldDeleteTimeBasedSegmentsReadyToBeDeleted() {
|
def shouldDeleteTimeBasedSegmentsReadyToBeDeleted() {
|
||||||
val set = TestUtils.singleMessageSet("test".getBytes, timestamp = 10)
|
val set = TestUtils.singletonRecords("test".getBytes, timestamp = 10)
|
||||||
val log = createLog(set.sizeInBytes, retentionMs = 10000)
|
val log = createLog(set.sizeInBytes, retentionMs = 10000)
|
||||||
|
|
||||||
// append some messages to create some segments
|
// append some messages to create some segments
|
||||||
|
|
@ -1193,7 +1195,7 @@ class LogTest extends JUnitSuite {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
def shouldNotDeleteTimeBasedSegmentsWhenNoneReadyToBeDeleted() {
|
def shouldNotDeleteTimeBasedSegmentsWhenNoneReadyToBeDeleted() {
|
||||||
val set = TestUtils.singleMessageSet("test".getBytes, timestamp = time.milliseconds)
|
val set = TestUtils.singletonRecords("test".getBytes, timestamp = time.milliseconds)
|
||||||
val log = createLog(set.sizeInBytes, retentionMs = 10000000)
|
val log = createLog(set.sizeInBytes, retentionMs = 10000000)
|
||||||
|
|
||||||
// append some messages to create some segments
|
// append some messages to create some segments
|
||||||
|
|
@ -1206,7 +1208,7 @@ class LogTest extends JUnitSuite {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
def shouldNotDeleteSegmentsWhenPolicyDoesNotIncludeDelete() {
|
def shouldNotDeleteSegmentsWhenPolicyDoesNotIncludeDelete() {
|
||||||
val set = TestUtils.singleMessageSet("test".getBytes, key = "test".getBytes(), timestamp = 10L)
|
val set = TestUtils.singletonRecords("test".getBytes, key = "test".getBytes(), timestamp = 10L)
|
||||||
val log = createLog(set.sizeInBytes,
|
val log = createLog(set.sizeInBytes,
|
||||||
retentionMs = 10000,
|
retentionMs = 10000,
|
||||||
cleanupPolicy = "compact")
|
cleanupPolicy = "compact")
|
||||||
|
|
@ -1225,7 +1227,7 @@ class LogTest extends JUnitSuite {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
def shouldDeleteSegmentsReadyToBeDeletedWhenCleanupPolicyIsCompactAndDelete() {
|
def shouldDeleteSegmentsReadyToBeDeletedWhenCleanupPolicyIsCompactAndDelete() {
|
||||||
val set = TestUtils.singleMessageSet("test".getBytes, key = "test".getBytes,timestamp = 10L)
|
val set = TestUtils.singletonRecords("test".getBytes, key = "test".getBytes,timestamp = 10L)
|
||||||
val log = createLog(set.sizeInBytes,
|
val log = createLog(set.sizeInBytes,
|
||||||
retentionMs = 10000,
|
retentionMs = 10000,
|
||||||
cleanupPolicy = "compact,delete")
|
cleanupPolicy = "compact,delete")
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,395 @@
|
||||||
|
/**
|
||||||
|
* 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 kafka.log
|
||||||
|
|
||||||
|
import java.nio.ByteBuffer
|
||||||
|
|
||||||
|
import kafka.common.LongRef
|
||||||
|
import kafka.message._
|
||||||
|
import org.apache.kafka.common.errors.InvalidTimestampException
|
||||||
|
import org.apache.kafka.common.record._
|
||||||
|
import org.junit.Assert._
|
||||||
|
import org.junit.Test
|
||||||
|
import org.scalatest.junit.JUnitSuite
|
||||||
|
|
||||||
|
import scala.collection.JavaConverters._
|
||||||
|
|
||||||
|
class LogValidatorTest extends JUnitSuite {
|
||||||
|
|
||||||
|
@Test
|
||||||
|
def testLogAppendTimeNonCompressed() {
|
||||||
|
val now = System.currentTimeMillis()
|
||||||
|
// The timestamps should be overwritten
|
||||||
|
val records = createRecords(magicValue = Record.MAGIC_VALUE_V1, timestamp = 0L, codec = CompressionType.NONE)
|
||||||
|
val validatedResults = LogValidator.validateMessagesAndAssignOffsets(records,
|
||||||
|
offsetCounter = new LongRef(0),
|
||||||
|
now = now,
|
||||||
|
sourceCodec = NoCompressionCodec,
|
||||||
|
targetCodec = NoCompressionCodec,
|
||||||
|
messageFormatVersion = Record.MAGIC_VALUE_V1,
|
||||||
|
messageTimestampType = TimestampType.LOG_APPEND_TIME,
|
||||||
|
messageTimestampDiffMaxMs = 1000L)
|
||||||
|
val validatedRecords = validatedResults.validatedRecords
|
||||||
|
assertEquals("number of messages should not change", records.deepIterator.asScala.size, validatedRecords.deepIterator.asScala.size)
|
||||||
|
validatedRecords.deepIterator.asScala.foreach(logEntry => validateLogAppendTime(now, logEntry.record))
|
||||||
|
assertEquals(s"Max timestamp should be $now", now, validatedResults.maxTimestamp)
|
||||||
|
assertEquals(s"The offset of max timestamp should be 0", 0, validatedResults.shallowOffsetOfMaxTimestamp)
|
||||||
|
assertFalse("Message size should not have been changed", validatedResults.messageSizeMaybeChanged)
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
def testLogAppendTimeWithRecompression() {
|
||||||
|
val now = System.currentTimeMillis()
|
||||||
|
// The timestamps should be overwritten
|
||||||
|
val records = createRecords(magicValue = Record.MAGIC_VALUE_V0, codec = CompressionType.GZIP)
|
||||||
|
val validatedResults = LogValidator.validateMessagesAndAssignOffsets(
|
||||||
|
records,
|
||||||
|
offsetCounter = new LongRef(0),
|
||||||
|
now = now,
|
||||||
|
sourceCodec = DefaultCompressionCodec,
|
||||||
|
targetCodec = DefaultCompressionCodec,
|
||||||
|
messageFormatVersion = Record.MAGIC_VALUE_V1,
|
||||||
|
messageTimestampType = TimestampType.LOG_APPEND_TIME,
|
||||||
|
messageTimestampDiffMaxMs = 1000L)
|
||||||
|
val validatedRecords = validatedResults.validatedRecords
|
||||||
|
|
||||||
|
assertEquals("number of messages should not change", records.deepIterator.asScala.size, validatedRecords.deepIterator.asScala.size)
|
||||||
|
validatedRecords.deepIterator.asScala.foreach(logEntry => validateLogAppendTime(now, logEntry.record))
|
||||||
|
assertTrue("MessageSet should still valid", validatedRecords.shallowIterator.next().record.isValid)
|
||||||
|
assertEquals(s"Max timestamp should be $now", now, validatedResults.maxTimestamp)
|
||||||
|
assertEquals(s"The offset of max timestamp should be ${records.deepIterator.asScala.size - 1}",
|
||||||
|
records.deepIterator.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestamp)
|
||||||
|
assertTrue("Message size may have been changed", validatedResults.messageSizeMaybeChanged)
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
def testLogAppendTimeWithoutRecompression() {
|
||||||
|
val now = System.currentTimeMillis()
|
||||||
|
// The timestamps should be overwritten
|
||||||
|
val records = createRecords(magicValue = Record.MAGIC_VALUE_V1,
|
||||||
|
timestamp = 0L, codec = CompressionType.GZIP)
|
||||||
|
val validatedResults = LogValidator.validateMessagesAndAssignOffsets(
|
||||||
|
records,
|
||||||
|
offsetCounter = new LongRef(0),
|
||||||
|
now = now,
|
||||||
|
sourceCodec = DefaultCompressionCodec,
|
||||||
|
targetCodec = DefaultCompressionCodec,
|
||||||
|
messageFormatVersion = Record.MAGIC_VALUE_V1,
|
||||||
|
messageTimestampType = TimestampType.LOG_APPEND_TIME,
|
||||||
|
messageTimestampDiffMaxMs = 1000L)
|
||||||
|
val validatedRecords = validatedResults.validatedRecords
|
||||||
|
|
||||||
|
assertEquals("number of messages should not change", records.deepIterator.asScala.size,
|
||||||
|
validatedRecords.deepIterator.asScala.size)
|
||||||
|
validatedRecords.deepIterator.asScala.foreach(logEntry => validateLogAppendTime(now, logEntry.record))
|
||||||
|
assertTrue("MessageSet should still valid", validatedRecords.shallowIterator.next().record.isValid)
|
||||||
|
assertEquals(s"Max timestamp should be $now", now, validatedResults.maxTimestamp)
|
||||||
|
assertEquals(s"The offset of max timestamp should be ${records.deepIterator.asScala.size - 1}",
|
||||||
|
records.deepIterator.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestamp)
|
||||||
|
assertFalse("Message size should not have been changed", validatedResults.messageSizeMaybeChanged)
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
def testCreateTimeNonCompressed() {
|
||||||
|
val now = System.currentTimeMillis()
|
||||||
|
val timestampSeq = Seq(now - 1, now + 1, now)
|
||||||
|
val records =
|
||||||
|
MemoryRecords.withRecords(CompressionType.NONE,
|
||||||
|
Record.create(Record.MAGIC_VALUE_V1, timestampSeq(0), "hello".getBytes),
|
||||||
|
Record.create(Record.MAGIC_VALUE_V1, timestampSeq(1), "there".getBytes),
|
||||||
|
Record.create(Record.MAGIC_VALUE_V1, timestampSeq(2), "beautiful".getBytes))
|
||||||
|
|
||||||
|
val validatingResults = LogValidator.validateMessagesAndAssignOffsets(records,
|
||||||
|
offsetCounter = new LongRef(0),
|
||||||
|
now = System.currentTimeMillis(),
|
||||||
|
sourceCodec = NoCompressionCodec,
|
||||||
|
targetCodec = NoCompressionCodec,
|
||||||
|
messageFormatVersion = Record.MAGIC_VALUE_V1,
|
||||||
|
messageTimestampType = TimestampType.CREATE_TIME,
|
||||||
|
messageTimestampDiffMaxMs = 1000L)
|
||||||
|
val validatedRecords = validatingResults.validatedRecords
|
||||||
|
|
||||||
|
var i = 0
|
||||||
|
for (logEntry <- validatedRecords.deepIterator.asScala) {
|
||||||
|
logEntry.record.ensureValid()
|
||||||
|
assertEquals(logEntry.record.timestamp, timestampSeq(i))
|
||||||
|
assertEquals(logEntry.record.timestampType, TimestampType.CREATE_TIME)
|
||||||
|
i += 1
|
||||||
|
}
|
||||||
|
assertEquals(s"Max timestamp should be ${now + 1}", now + 1, validatingResults.maxTimestamp)
|
||||||
|
assertEquals(s"Offset of max timestamp should be 1", 1, validatingResults.shallowOffsetOfMaxTimestamp)
|
||||||
|
assertFalse("Message size should not have been changed", validatingResults.messageSizeMaybeChanged)
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
def testCreateTimeCompressed() {
|
||||||
|
val now = System.currentTimeMillis()
|
||||||
|
val timestampSeq = Seq(now - 1, now + 1, now)
|
||||||
|
val records =
|
||||||
|
MemoryRecords.withRecords(CompressionType.GZIP,
|
||||||
|
Record.create(Record.MAGIC_VALUE_V1, timestampSeq(0), "hello".getBytes),
|
||||||
|
Record.create(Record.MAGIC_VALUE_V1, timestampSeq(1), "there".getBytes),
|
||||||
|
Record.create(Record.MAGIC_VALUE_V1, timestampSeq(2), "beautiful".getBytes))
|
||||||
|
|
||||||
|
val validatedResults =
|
||||||
|
LogValidator.validateMessagesAndAssignOffsets(records,
|
||||||
|
offsetCounter = new LongRef(0),
|
||||||
|
now = System.currentTimeMillis(),
|
||||||
|
sourceCodec = DefaultCompressionCodec,
|
||||||
|
targetCodec = DefaultCompressionCodec,
|
||||||
|
messageFormatVersion = Record.MAGIC_VALUE_V1,
|
||||||
|
messageTimestampType = TimestampType.CREATE_TIME,
|
||||||
|
messageTimestampDiffMaxMs = 1000L)
|
||||||
|
val validatedRecords = validatedResults.validatedRecords
|
||||||
|
|
||||||
|
var i = 0
|
||||||
|
for (logEntry <- validatedRecords.deepIterator.asScala) {
|
||||||
|
logEntry.record.ensureValid()
|
||||||
|
assertEquals(logEntry.record.timestamp, timestampSeq(i))
|
||||||
|
assertEquals(logEntry.record.timestampType, TimestampType.CREATE_TIME)
|
||||||
|
i += 1
|
||||||
|
}
|
||||||
|
assertEquals(s"Max timestamp should be ${now + 1}", now + 1, validatedResults.maxTimestamp)
|
||||||
|
assertEquals(s"Offset of max timestamp should be ${validatedRecords.deepIterator.asScala.size - 1}",
|
||||||
|
validatedRecords.deepIterator.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestamp)
|
||||||
|
assertFalse("Message size should not have been changed", validatedResults.messageSizeMaybeChanged)
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(expected = classOf[InvalidTimestampException])
|
||||||
|
def testInvalidCreateTimeNonCompressed() {
|
||||||
|
val now = System.currentTimeMillis()
|
||||||
|
val records = createRecords(magicValue = Record.MAGIC_VALUE_V1, timestamp = now - 1001L,
|
||||||
|
codec = CompressionType.NONE)
|
||||||
|
LogValidator.validateMessagesAndAssignOffsets(
|
||||||
|
records,
|
||||||
|
offsetCounter = new LongRef(0),
|
||||||
|
now = System.currentTimeMillis(),
|
||||||
|
sourceCodec = NoCompressionCodec,
|
||||||
|
targetCodec = NoCompressionCodec,
|
||||||
|
messageFormatVersion = Record.MAGIC_VALUE_V1,
|
||||||
|
messageTimestampType = TimestampType.CREATE_TIME,
|
||||||
|
messageTimestampDiffMaxMs = 1000L)
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(expected = classOf[InvalidTimestampException])
|
||||||
|
def testInvalidCreateTimeCompressed() {
|
||||||
|
val now = System.currentTimeMillis()
|
||||||
|
val records = createRecords(magicValue = Record.MAGIC_VALUE_V1, timestamp = now - 1001L,
|
||||||
|
codec = CompressionType.GZIP)
|
||||||
|
LogValidator.validateMessagesAndAssignOffsets(
|
||||||
|
records,
|
||||||
|
offsetCounter = new LongRef(0),
|
||||||
|
now = System.currentTimeMillis(),
|
||||||
|
sourceCodec = DefaultCompressionCodec,
|
||||||
|
targetCodec = DefaultCompressionCodec,
|
||||||
|
messageFormatVersion = Record.MAGIC_VALUE_V1,
|
||||||
|
messageTimestampType = TimestampType.CREATE_TIME,
|
||||||
|
messageTimestampDiffMaxMs = 1000L)
|
||||||
|
}
|
||||||
|
@Test
|
||||||
|
def testAbsoluteOffsetAssignmentNonCompressed() {
|
||||||
|
val records = createRecords(magicValue = Record.MAGIC_VALUE_V0, codec = CompressionType.NONE)
|
||||||
|
val offset = 1234567
|
||||||
|
checkOffsets(records, 0)
|
||||||
|
checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records,
|
||||||
|
offsetCounter = new LongRef(offset),
|
||||||
|
now = System.currentTimeMillis(),
|
||||||
|
sourceCodec = NoCompressionCodec,
|
||||||
|
targetCodec = NoCompressionCodec,
|
||||||
|
messageFormatVersion = Record.MAGIC_VALUE_V0,
|
||||||
|
messageTimestampType = TimestampType.CREATE_TIME,
|
||||||
|
messageTimestampDiffMaxMs = 1000L).validatedRecords, offset)
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
def testAbsoluteOffsetAssignmentCompressed() {
|
||||||
|
val records = createRecords(magicValue = Record.MAGIC_VALUE_V0, codec = CompressionType.GZIP)
|
||||||
|
val offset = 1234567
|
||||||
|
checkOffsets(records, 0)
|
||||||
|
checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records,
|
||||||
|
offsetCounter = new LongRef(offset),
|
||||||
|
now = System.currentTimeMillis(),
|
||||||
|
sourceCodec = DefaultCompressionCodec,
|
||||||
|
targetCodec = DefaultCompressionCodec,
|
||||||
|
messageFormatVersion = Record.MAGIC_VALUE_V0,
|
||||||
|
messageTimestampType = TimestampType.CREATE_TIME,
|
||||||
|
messageTimestampDiffMaxMs = 1000L).validatedRecords, offset)
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
def testRelativeOffsetAssignmentNonCompressed() {
|
||||||
|
val now = System.currentTimeMillis()
|
||||||
|
val records = createRecords(magicValue = Record.MAGIC_VALUE_V1, timestamp = now, codec = CompressionType.NONE)
|
||||||
|
val offset = 1234567
|
||||||
|
checkOffsets(records, 0)
|
||||||
|
val messageWithOffset = LogValidator.validateMessagesAndAssignOffsets(records,
|
||||||
|
offsetCounter = new LongRef(offset),
|
||||||
|
now = System.currentTimeMillis(),
|
||||||
|
sourceCodec = NoCompressionCodec,
|
||||||
|
targetCodec = NoCompressionCodec,
|
||||||
|
messageTimestampType = TimestampType.CREATE_TIME,
|
||||||
|
messageTimestampDiffMaxMs = 5000L).validatedRecords
|
||||||
|
checkOffsets(messageWithOffset, offset)
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
def testRelativeOffsetAssignmentCompressed() {
|
||||||
|
val now = System.currentTimeMillis()
|
||||||
|
val records = createRecords(magicValue = Record.MAGIC_VALUE_V1, timestamp = now, codec = CompressionType.GZIP)
|
||||||
|
val offset = 1234567
|
||||||
|
checkOffsets(records, 0)
|
||||||
|
val compressedMessagesWithOffset = LogValidator.validateMessagesAndAssignOffsets(
|
||||||
|
records,
|
||||||
|
offsetCounter = new LongRef(offset),
|
||||||
|
now = System.currentTimeMillis(),
|
||||||
|
sourceCodec = DefaultCompressionCodec,
|
||||||
|
targetCodec = DefaultCompressionCodec,
|
||||||
|
messageTimestampType = TimestampType.CREATE_TIME,
|
||||||
|
messageTimestampDiffMaxMs = 5000L).validatedRecords
|
||||||
|
checkOffsets(compressedMessagesWithOffset, offset)
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
def testOffsetAssignmentAfterMessageFormatConversionV0NonCompressed() {
|
||||||
|
val records = createRecords(magicValue = Record.MAGIC_VALUE_V0, codec = CompressionType.NONE)
|
||||||
|
checkOffsets(records, 0)
|
||||||
|
val offset = 1234567
|
||||||
|
checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records,
|
||||||
|
offsetCounter = new LongRef(offset),
|
||||||
|
now = System.currentTimeMillis(),
|
||||||
|
sourceCodec = NoCompressionCodec,
|
||||||
|
targetCodec = NoCompressionCodec,
|
||||||
|
messageFormatVersion = Record.MAGIC_VALUE_V1,
|
||||||
|
messageTimestampType = TimestampType.LOG_APPEND_TIME,
|
||||||
|
messageTimestampDiffMaxMs = 1000L).validatedRecords, offset)
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
def testOffsetAssignmentAfterMessageFormatConversionV0Compressed() {
|
||||||
|
val records = createRecords(magicValue = Record.MAGIC_VALUE_V0, codec = CompressionType.GZIP)
|
||||||
|
val offset = 1234567
|
||||||
|
checkOffsets(records, 0)
|
||||||
|
checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records,
|
||||||
|
offsetCounter = new LongRef(offset),
|
||||||
|
now = System.currentTimeMillis(),
|
||||||
|
sourceCodec = DefaultCompressionCodec,
|
||||||
|
targetCodec = DefaultCompressionCodec,
|
||||||
|
messageFormatVersion = Record.MAGIC_VALUE_V1,
|
||||||
|
messageTimestampType = TimestampType.LOG_APPEND_TIME,
|
||||||
|
messageTimestampDiffMaxMs = 1000L).validatedRecords, offset)
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
def testOffsetAssignmentAfterMessageFormatConversionV1NonCompressed() {
|
||||||
|
val offset = 1234567
|
||||||
|
val now = System.currentTimeMillis()
|
||||||
|
val records = createRecords(Record.MAGIC_VALUE_V1, now, codec = CompressionType.NONE)
|
||||||
|
checkOffsets(records, 0)
|
||||||
|
checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records,
|
||||||
|
offsetCounter = new LongRef(offset),
|
||||||
|
now = System.currentTimeMillis(),
|
||||||
|
sourceCodec = NoCompressionCodec,
|
||||||
|
targetCodec = NoCompressionCodec,
|
||||||
|
messageFormatVersion = Record.MAGIC_VALUE_V0,
|
||||||
|
messageTimestampType = TimestampType.CREATE_TIME,
|
||||||
|
messageTimestampDiffMaxMs = 5000L).validatedRecords, offset)
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
def testOffsetAssignmentAfterMessageFormatConversionV1Compressed() {
|
||||||
|
val offset = 1234567
|
||||||
|
val now = System.currentTimeMillis()
|
||||||
|
val records = createRecords(Record.MAGIC_VALUE_V1, now, CompressionType.GZIP)
|
||||||
|
checkOffsets(records, 0)
|
||||||
|
checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records,
|
||||||
|
offsetCounter = new LongRef(offset),
|
||||||
|
now = System.currentTimeMillis(),
|
||||||
|
sourceCodec = DefaultCompressionCodec,
|
||||||
|
targetCodec = DefaultCompressionCodec,
|
||||||
|
messageFormatVersion = Record.MAGIC_VALUE_V0,
|
||||||
|
messageTimestampType = TimestampType.CREATE_TIME,
|
||||||
|
messageTimestampDiffMaxMs = 5000L).validatedRecords, offset)
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test(expected = classOf[InvalidRecordException])
|
||||||
|
def testInvalidInnerMagicVersion(): Unit = {
|
||||||
|
val offset = 1234567
|
||||||
|
val records = recordsWithInvalidInnerMagic(offset)
|
||||||
|
LogValidator.validateMessagesAndAssignOffsets(records,
|
||||||
|
offsetCounter = new LongRef(offset),
|
||||||
|
now = System.currentTimeMillis(),
|
||||||
|
sourceCodec = SnappyCompressionCodec,
|
||||||
|
targetCodec = SnappyCompressionCodec,
|
||||||
|
messageTimestampType = TimestampType.CREATE_TIME,
|
||||||
|
messageTimestampDiffMaxMs = 5000L)
|
||||||
|
}
|
||||||
|
|
||||||
|
private def createRecords(magicValue: Byte = Message.CurrentMagicValue,
|
||||||
|
timestamp: Long = Message.NoTimestamp,
|
||||||
|
codec: CompressionType = CompressionType.NONE): MemoryRecords = {
|
||||||
|
if (magicValue == Record.MAGIC_VALUE_V0) {
|
||||||
|
MemoryRecords.withRecords(
|
||||||
|
codec,
|
||||||
|
Record.create(Record.MAGIC_VALUE_V0, Record.NO_TIMESTAMP, "hello".getBytes),
|
||||||
|
Record.create(Record.MAGIC_VALUE_V0, Record.NO_TIMESTAMP, "there".getBytes),
|
||||||
|
Record.create(Record.MAGIC_VALUE_V0, Record.NO_TIMESTAMP, "beautiful".getBytes))
|
||||||
|
} else {
|
||||||
|
MemoryRecords.withRecords(
|
||||||
|
codec,
|
||||||
|
Record.create(Record.MAGIC_VALUE_V1, timestamp, "hello".getBytes),
|
||||||
|
Record.create(Record.MAGIC_VALUE_V1, timestamp, "there".getBytes),
|
||||||
|
Record.create(Record.MAGIC_VALUE_V1, timestamp, "beautiful".getBytes))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/* check that offsets are assigned consecutively from the given base offset */
|
||||||
|
private def checkOffsets(records: MemoryRecords, baseOffset: Long) {
|
||||||
|
assertTrue("Message set should not be empty", records.deepIterator.asScala.nonEmpty)
|
||||||
|
var offset = baseOffset
|
||||||
|
for (entry <- records.deepIterator.asScala) {
|
||||||
|
assertEquals("Unexpected offset in message set iterator", offset, entry.offset)
|
||||||
|
offset += 1
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private def recordsWithInvalidInnerMagic(initialOffset: Long): MemoryRecords = {
|
||||||
|
val records = (0 until 20).map(id =>
|
||||||
|
Record.create(Record.MAGIC_VALUE_V0,
|
||||||
|
Record.NO_TIMESTAMP,
|
||||||
|
id.toString.getBytes,
|
||||||
|
id.toString.getBytes))
|
||||||
|
|
||||||
|
val buffer = ByteBuffer.allocate(math.min(math.max(records.map(_.sizeInBytes()).sum / 2, 1024), 1 << 16))
|
||||||
|
val builder = MemoryRecords.builder(buffer, Record.MAGIC_VALUE_V1, CompressionType.GZIP,
|
||||||
|
TimestampType.CREATE_TIME)
|
||||||
|
|
||||||
|
var offset = initialOffset
|
||||||
|
records.foreach { record =>
|
||||||
|
builder.appendUnchecked(offset, record)
|
||||||
|
offset += 1
|
||||||
|
}
|
||||||
|
|
||||||
|
builder.build()
|
||||||
|
}
|
||||||
|
|
||||||
|
def validateLogAppendTime(now: Long, record: Record) {
|
||||||
|
record.ensureValid()
|
||||||
|
assertEquals(s"Timestamp of message $record should be $now", now, record.timestamp)
|
||||||
|
assertEquals(TimestampType.LOG_APPEND_TIME, record.timestampType)
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -18,13 +18,11 @@
|
||||||
package kafka.message
|
package kafka.message
|
||||||
|
|
||||||
import java.nio.ByteBuffer
|
import java.nio.ByteBuffer
|
||||||
import java.nio.channels.{FileChannel, GatheringByteChannel}
|
import java.nio.channels.GatheringByteChannel
|
||||||
import java.nio.file.StandardOpenOption
|
|
||||||
|
|
||||||
import org.junit.Assert._
|
import org.junit.Assert._
|
||||||
import kafka.utils.TestUtils._
|
import kafka.utils.TestUtils._
|
||||||
import kafka.log.FileMessageSet
|
import org.apache.kafka.common.record.FileRecords
|
||||||
import kafka.utils.TestUtils
|
|
||||||
import org.scalatest.junit.JUnitSuite
|
import org.scalatest.junit.JUnitSuite
|
||||||
import org.junit.Test
|
import org.junit.Test
|
||||||
|
|
||||||
|
|
@ -94,7 +92,7 @@ trait BaseMessageSetTestCases extends JUnitSuite {
|
||||||
@Test
|
@Test
|
||||||
def testWriteToChannelThatConsumesPartially() {
|
def testWriteToChannelThatConsumesPartially() {
|
||||||
val bytesToConsumePerBuffer = 50
|
val bytesToConsumePerBuffer = 50
|
||||||
val messages = (0 until 10).map(_ => new Message(TestUtils.randomString(100).getBytes))
|
val messages = (0 until 10).map(_ => new Message(randomString(100).getBytes))
|
||||||
val messageSet = createMessageSet(messages)
|
val messageSet = createMessageSet(messages)
|
||||||
val messageSetSize = messageSet.sizeInBytes
|
val messageSetSize = messageSet.sizeInBytes
|
||||||
|
|
||||||
|
|
@ -119,15 +117,15 @@ trait BaseMessageSetTestCases extends JUnitSuite {
|
||||||
// do the write twice to ensure the message set is restored to its original state
|
// do the write twice to ensure the message set is restored to its original state
|
||||||
for (_ <- 0 to 1) {
|
for (_ <- 0 to 1) {
|
||||||
val file = tempFile()
|
val file = tempFile()
|
||||||
val channel = FileChannel.open(file.toPath, StandardOpenOption.READ, StandardOpenOption.WRITE)
|
val fileRecords = FileRecords.open(file, true)
|
||||||
try {
|
try {
|
||||||
val written = write(channel)
|
val written = write(fileRecords.channel)
|
||||||
|
fileRecords.resize() // resize since we wrote to the channel directly
|
||||||
|
|
||||||
assertEquals("Expect to write the number of bytes in the set.", set.sizeInBytes, written)
|
assertEquals("Expect to write the number of bytes in the set.", set.sizeInBytes, written)
|
||||||
val newSet = new FileMessageSet(file, channel)
|
checkEquals(set.asRecords.deepIterator, fileRecords.deepIterator())
|
||||||
checkEquals(set.iterator, newSet.iterator)
|
} finally fileRecords.close()
|
||||||
} finally channel.close()
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -17,13 +17,9 @@
|
||||||
|
|
||||||
package kafka.message
|
package kafka.message
|
||||||
|
|
||||||
import java.io.DataOutputStream
|
|
||||||
import java.nio._
|
import java.nio._
|
||||||
|
|
||||||
import kafka.common.LongRef
|
|
||||||
import kafka.utils.TestUtils
|
import kafka.utils.TestUtils
|
||||||
import org.apache.kafka.common.errors.InvalidTimestampException
|
|
||||||
import org.apache.kafka.common.record.TimestampType
|
|
||||||
import org.junit.Assert._
|
import org.junit.Assert._
|
||||||
import org.junit.Test
|
import org.junit.Test
|
||||||
|
|
||||||
|
|
@ -151,295 +147,6 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases {
|
||||||
assertEquals("second offset should be 2", 2L, iter.next().offset)
|
assertEquals("second offset should be 2", 2L, iter.next().offset)
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
|
||||||
def testLogAppendTime() {
|
|
||||||
val now = System.currentTimeMillis()
|
|
||||||
// The timestamps should be overwritten
|
|
||||||
val messages = getMessages(magicValue = Message.MagicValue_V1, timestamp = 0L, codec = NoCompressionCodec)
|
|
||||||
val compressedMessagesWithRecompresion = getMessages(magicValue = Message.MagicValue_V0, codec = DefaultCompressionCodec)
|
|
||||||
val compressedMessagesWithoutRecompression =
|
|
||||||
getMessages(magicValue = Message.MagicValue_V1, timestamp = 0L, codec = DefaultCompressionCodec)
|
|
||||||
|
|
||||||
val validatingResults = messages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0),
|
|
||||||
now = now,
|
|
||||||
sourceCodec = NoCompressionCodec,
|
|
||||||
targetCodec = NoCompressionCodec,
|
|
||||||
messageFormatVersion = 1,
|
|
||||||
messageTimestampType = TimestampType.LOG_APPEND_TIME,
|
|
||||||
messageTimestampDiffMaxMs = 1000L)
|
|
||||||
val validatedMessages = validatingResults.validatedMessages
|
|
||||||
|
|
||||||
val validatingCompressedMessagesResults =
|
|
||||||
compressedMessagesWithRecompresion.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0),
|
|
||||||
now = now,
|
|
||||||
sourceCodec = DefaultCompressionCodec,
|
|
||||||
targetCodec = DefaultCompressionCodec,
|
|
||||||
messageFormatVersion = 1,
|
|
||||||
messageTimestampType = TimestampType.LOG_APPEND_TIME,
|
|
||||||
messageTimestampDiffMaxMs = 1000L)
|
|
||||||
val validatedCompressedMessages = validatingCompressedMessagesResults.validatedMessages
|
|
||||||
|
|
||||||
val validatingCompressedMessagesWithoutRecompressionResults =
|
|
||||||
compressedMessagesWithoutRecompression.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0),
|
|
||||||
now = now,
|
|
||||||
sourceCodec = DefaultCompressionCodec,
|
|
||||||
targetCodec = DefaultCompressionCodec,
|
|
||||||
messageFormatVersion = 1,
|
|
||||||
messageTimestampType = TimestampType.LOG_APPEND_TIME,
|
|
||||||
messageTimestampDiffMaxMs = 1000L)
|
|
||||||
|
|
||||||
val validatedCompressedMessagesWithoutRecompression = validatingCompressedMessagesWithoutRecompressionResults.validatedMessages
|
|
||||||
|
|
||||||
assertEquals("message set size should not change", messages.size, validatedMessages.size)
|
|
||||||
validatedMessages.foreach(messageAndOffset => validateLogAppendTime(messageAndOffset.message))
|
|
||||||
assertEquals(s"Max timestamp should be $now", now, validatingResults.maxTimestamp)
|
|
||||||
assertEquals(s"The offset of max timestamp should be 0", 0, validatingResults.offsetOfMaxTimestamp)
|
|
||||||
assertFalse("Message size should not have been changed", validatingResults.messageSizeMaybeChanged)
|
|
||||||
|
|
||||||
assertEquals("message set size should not change", compressedMessagesWithRecompresion.size, validatedCompressedMessages.size)
|
|
||||||
validatedCompressedMessages.foreach(messageAndOffset => validateLogAppendTime(messageAndOffset.message))
|
|
||||||
assertTrue("MessageSet should still valid", validatedCompressedMessages.shallowIterator.next().message.isValid)
|
|
||||||
assertEquals(s"Max timestamp should be $now", now, validatingCompressedMessagesResults.maxTimestamp)
|
|
||||||
assertEquals(s"The offset of max timestamp should be ${compressedMessagesWithRecompresion.size - 1}",
|
|
||||||
compressedMessagesWithRecompresion.size - 1, validatingCompressedMessagesResults.offsetOfMaxTimestamp)
|
|
||||||
assertTrue("Message size may have been changed", validatingCompressedMessagesResults.messageSizeMaybeChanged)
|
|
||||||
|
|
||||||
assertEquals("message set size should not change", compressedMessagesWithoutRecompression.size,
|
|
||||||
validatedCompressedMessagesWithoutRecompression.size)
|
|
||||||
validatedCompressedMessagesWithoutRecompression.foreach(messageAndOffset => validateLogAppendTime(messageAndOffset.message))
|
|
||||||
assertTrue("MessageSet should still valid", validatedCompressedMessagesWithoutRecompression.shallowIterator.next().message.isValid)
|
|
||||||
assertEquals(s"Max timestamp should be $now", now, validatingCompressedMessagesWithoutRecompressionResults.maxTimestamp)
|
|
||||||
assertEquals(s"The offset of max timestamp should be ${compressedMessagesWithoutRecompression.size - 1}",
|
|
||||||
compressedMessagesWithoutRecompression.size - 1, validatingCompressedMessagesWithoutRecompressionResults.offsetOfMaxTimestamp)
|
|
||||||
assertFalse("Message size should not have been changed", validatingCompressedMessagesWithoutRecompressionResults.messageSizeMaybeChanged)
|
|
||||||
|
|
||||||
def validateLogAppendTime(message: Message) {
|
|
||||||
message.ensureValid()
|
|
||||||
assertEquals(s"Timestamp of message $message should be $now", now, message.timestamp)
|
|
||||||
assertEquals(TimestampType.LOG_APPEND_TIME, message.timestampType)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
def testCreateTime() {
|
|
||||||
val now = System.currentTimeMillis()
|
|
||||||
val timestampSeq = Seq(now - 1, now + 1, now)
|
|
||||||
val messages =
|
|
||||||
new ByteBufferMessageSet(NoCompressionCodec,
|
|
||||||
new Message("hello".getBytes, timestamp = timestampSeq(0), magicValue = Message.MagicValue_V1),
|
|
||||||
new Message("there".getBytes, timestamp = timestampSeq(1), magicValue = Message.MagicValue_V1),
|
|
||||||
new Message("beautiful".getBytes, timestamp = timestampSeq(2), magicValue = Message.MagicValue_V1))
|
|
||||||
val compressedMessages =
|
|
||||||
new ByteBufferMessageSet(DefaultCompressionCodec,
|
|
||||||
new Message("hello".getBytes, timestamp = timestampSeq(0), magicValue = Message.MagicValue_V1),
|
|
||||||
new Message("there".getBytes, timestamp = timestampSeq(1), magicValue = Message.MagicValue_V1),
|
|
||||||
new Message("beautiful".getBytes, timestamp = timestampSeq(2), magicValue = Message.MagicValue_V1))
|
|
||||||
|
|
||||||
val validatingResults = messages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0),
|
|
||||||
now = System.currentTimeMillis(),
|
|
||||||
sourceCodec = NoCompressionCodec,
|
|
||||||
targetCodec = NoCompressionCodec,
|
|
||||||
messageFormatVersion = 1,
|
|
||||||
messageTimestampType = TimestampType.CREATE_TIME,
|
|
||||||
messageTimestampDiffMaxMs = 1000L)
|
|
||||||
val validatedMessages = validatingResults.validatedMessages
|
|
||||||
|
|
||||||
val validatingCompressedMessagesResults =
|
|
||||||
compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0),
|
|
||||||
now = System.currentTimeMillis(),
|
|
||||||
sourceCodec = DefaultCompressionCodec,
|
|
||||||
targetCodec = DefaultCompressionCodec,
|
|
||||||
messageFormatVersion = 1,
|
|
||||||
messageTimestampType = TimestampType.CREATE_TIME,
|
|
||||||
messageTimestampDiffMaxMs = 1000L)
|
|
||||||
val validatedCompressedMessages = validatingCompressedMessagesResults.validatedMessages
|
|
||||||
|
|
||||||
var i = 0
|
|
||||||
for (messageAndOffset <- validatedMessages) {
|
|
||||||
messageAndOffset.message.ensureValid()
|
|
||||||
assertEquals(messageAndOffset.message.timestamp, timestampSeq(i))
|
|
||||||
assertEquals(messageAndOffset.message.timestampType, TimestampType.CREATE_TIME)
|
|
||||||
i += 1
|
|
||||||
}
|
|
||||||
assertEquals(s"Max timestamp should be ${now + 1}", now + 1, validatingResults.maxTimestamp)
|
|
||||||
assertEquals(s"Offset of max timestamp should be 1", 1, validatingResults.offsetOfMaxTimestamp)
|
|
||||||
assertFalse("Message size should not have been changed", validatingResults.messageSizeMaybeChanged)
|
|
||||||
i = 0
|
|
||||||
for (messageAndOffset <- validatedCompressedMessages) {
|
|
||||||
messageAndOffset.message.ensureValid()
|
|
||||||
assertEquals(messageAndOffset.message.timestamp, timestampSeq(i))
|
|
||||||
assertEquals(messageAndOffset.message.timestampType, TimestampType.CREATE_TIME)
|
|
||||||
i += 1
|
|
||||||
}
|
|
||||||
assertEquals(s"Max timestamp should be ${now + 1}", now + 1, validatingResults.maxTimestamp)
|
|
||||||
assertEquals(s"Offset of max timestamp should be ${validatedCompressedMessages.size - 1}",
|
|
||||||
validatedCompressedMessages.size - 1, validatingCompressedMessagesResults.offsetOfMaxTimestamp)
|
|
||||||
assertFalse("Message size should not have been changed", validatingCompressedMessagesResults.messageSizeMaybeChanged)
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
def testInvalidCreateTime() {
|
|
||||||
val now = System.currentTimeMillis()
|
|
||||||
val messages = getMessages(magicValue = Message.MagicValue_V1, timestamp = now - 1001L, codec = NoCompressionCodec)
|
|
||||||
val compressedMessages = getMessages(magicValue = Message.MagicValue_V1, timestamp = now - 1001L, codec = DefaultCompressionCodec)
|
|
||||||
|
|
||||||
try {
|
|
||||||
messages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0),
|
|
||||||
now = System.currentTimeMillis(),
|
|
||||||
sourceCodec = NoCompressionCodec,
|
|
||||||
targetCodec = NoCompressionCodec,
|
|
||||||
messageFormatVersion = 1,
|
|
||||||
messageTimestampType = TimestampType.CREATE_TIME,
|
|
||||||
messageTimestampDiffMaxMs = 1000L)
|
|
||||||
fail("Should throw InvalidMessageException.")
|
|
||||||
} catch {
|
|
||||||
case _: InvalidTimestampException =>
|
|
||||||
}
|
|
||||||
|
|
||||||
try {
|
|
||||||
compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0),
|
|
||||||
now = System.currentTimeMillis(),
|
|
||||||
sourceCodec = DefaultCompressionCodec,
|
|
||||||
targetCodec = DefaultCompressionCodec,
|
|
||||||
messageFormatVersion = 1,
|
|
||||||
messageTimestampType = TimestampType.CREATE_TIME,
|
|
||||||
messageTimestampDiffMaxMs = 1000L)
|
|
||||||
fail("Should throw InvalidMessageException.")
|
|
||||||
} catch {
|
|
||||||
case _: InvalidTimestampException =>
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
def testAbsoluteOffsetAssignment() {
|
|
||||||
val messages = getMessages(magicValue = Message.MagicValue_V0, codec = NoCompressionCodec)
|
|
||||||
val compressedMessages = getMessages(magicValue = Message.MagicValue_V0, codec = DefaultCompressionCodec)
|
|
||||||
// check uncompressed offsets
|
|
||||||
checkOffsets(messages, 0)
|
|
||||||
val offset = 1234567
|
|
||||||
checkOffsets(messages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
|
|
||||||
now = System.currentTimeMillis(),
|
|
||||||
sourceCodec = NoCompressionCodec,
|
|
||||||
targetCodec = NoCompressionCodec,
|
|
||||||
messageFormatVersion = 0,
|
|
||||||
messageTimestampType = TimestampType.CREATE_TIME,
|
|
||||||
messageTimestampDiffMaxMs = 1000L).validatedMessages, offset)
|
|
||||||
|
|
||||||
// check compressed messages
|
|
||||||
checkOffsets(compressedMessages, 0)
|
|
||||||
checkOffsets(compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
|
|
||||||
now = System.currentTimeMillis(),
|
|
||||||
sourceCodec = DefaultCompressionCodec,
|
|
||||||
targetCodec = DefaultCompressionCodec,
|
|
||||||
messageFormatVersion = 0,
|
|
||||||
messageTimestampType = TimestampType.CREATE_TIME,
|
|
||||||
messageTimestampDiffMaxMs = 1000L).validatedMessages, offset)
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
def testRelativeOffsetAssignment() {
|
|
||||||
val now = System.currentTimeMillis()
|
|
||||||
val messages = getMessages(magicValue = Message.MagicValue_V1, timestamp = now, codec = NoCompressionCodec)
|
|
||||||
val compressedMessages = getMessages(magicValue = Message.MagicValue_V1, timestamp = now, codec = DefaultCompressionCodec)
|
|
||||||
|
|
||||||
// check uncompressed offsets
|
|
||||||
checkOffsets(messages, 0)
|
|
||||||
val offset = 1234567
|
|
||||||
val messageWithOffset = messages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
|
|
||||||
now = System.currentTimeMillis(),
|
|
||||||
sourceCodec = NoCompressionCodec,
|
|
||||||
targetCodec = NoCompressionCodec,
|
|
||||||
messageTimestampType = TimestampType.CREATE_TIME,
|
|
||||||
messageTimestampDiffMaxMs = 5000L).validatedMessages
|
|
||||||
checkOffsets(messageWithOffset, offset)
|
|
||||||
|
|
||||||
// check compressed messages
|
|
||||||
checkOffsets(compressedMessages, 0)
|
|
||||||
val compressedMessagesWithOffset = compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
|
|
||||||
now = System.currentTimeMillis(),
|
|
||||||
sourceCodec = DefaultCompressionCodec,
|
|
||||||
targetCodec = DefaultCompressionCodec,
|
|
||||||
messageTimestampType = TimestampType.CREATE_TIME,
|
|
||||||
messageTimestampDiffMaxMs = 5000L).validatedMessages
|
|
||||||
checkOffsets(compressedMessagesWithOffset, offset)
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test(expected = classOf[InvalidMessageException])
|
|
||||||
def testInvalidInnerMagicVersion(): Unit = {
|
|
||||||
val offset = 1234567
|
|
||||||
val messages = messageSetWithInvalidInnerMagic(SnappyCompressionCodec, offset)
|
|
||||||
messages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
|
|
||||||
now = System.currentTimeMillis(),
|
|
||||||
sourceCodec = SnappyCompressionCodec,
|
|
||||||
targetCodec = SnappyCompressionCodec,
|
|
||||||
messageTimestampType = TimestampType.CREATE_TIME,
|
|
||||||
messageTimestampDiffMaxMs = 5000L).validatedMessages
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
@Test
|
|
||||||
def testOffsetAssignmentAfterMessageFormatConversion() {
|
|
||||||
// Check up conversion
|
|
||||||
val messagesV0 = getMessages(magicValue = Message.MagicValue_V0, codec = NoCompressionCodec)
|
|
||||||
val compressedMessagesV0 = getMessages(magicValue = Message.MagicValue_V0, codec = DefaultCompressionCodec)
|
|
||||||
// check uncompressed offsets
|
|
||||||
checkOffsets(messagesV0, 0)
|
|
||||||
val offset = 1234567
|
|
||||||
checkOffsets(messagesV0.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
|
|
||||||
now = System.currentTimeMillis(),
|
|
||||||
sourceCodec = NoCompressionCodec,
|
|
||||||
targetCodec = NoCompressionCodec,
|
|
||||||
messageFormatVersion = 1,
|
|
||||||
messageTimestampType = TimestampType.LOG_APPEND_TIME,
|
|
||||||
messageTimestampDiffMaxMs = 1000L).validatedMessages, offset)
|
|
||||||
|
|
||||||
// check compressed messages
|
|
||||||
checkOffsets(compressedMessagesV0, 0)
|
|
||||||
checkOffsets(compressedMessagesV0.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
|
|
||||||
now = System.currentTimeMillis(),
|
|
||||||
sourceCodec = DefaultCompressionCodec,
|
|
||||||
targetCodec = DefaultCompressionCodec,
|
|
||||||
messageFormatVersion = 1,
|
|
||||||
messageTimestampType = TimestampType.LOG_APPEND_TIME,
|
|
||||||
messageTimestampDiffMaxMs = 1000L).validatedMessages, offset)
|
|
||||||
|
|
||||||
// Check down conversion
|
|
||||||
val now = System.currentTimeMillis()
|
|
||||||
val messagesV1 = getMessages(Message.MagicValue_V1, now, NoCompressionCodec)
|
|
||||||
val compressedMessagesV1 = getMessages(Message.MagicValue_V1, now, DefaultCompressionCodec)
|
|
||||||
|
|
||||||
// check uncompressed offsets
|
|
||||||
checkOffsets(messagesV1, 0)
|
|
||||||
checkOffsets(messagesV1.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
|
|
||||||
now = System.currentTimeMillis(),
|
|
||||||
sourceCodec = NoCompressionCodec,
|
|
||||||
targetCodec = NoCompressionCodec,
|
|
||||||
messageFormatVersion = 0,
|
|
||||||
messageTimestampType = TimestampType.CREATE_TIME,
|
|
||||||
messageTimestampDiffMaxMs = 5000L).validatedMessages, offset)
|
|
||||||
|
|
||||||
// check compressed messages
|
|
||||||
checkOffsets(compressedMessagesV1, 0)
|
|
||||||
checkOffsets(compressedMessagesV1.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
|
|
||||||
now = System.currentTimeMillis(),
|
|
||||||
sourceCodec = DefaultCompressionCodec,
|
|
||||||
targetCodec = DefaultCompressionCodec,
|
|
||||||
messageFormatVersion = 0,
|
|
||||||
messageTimestampType = TimestampType.CREATE_TIME,
|
|
||||||
messageTimestampDiffMaxMs = 5000L).validatedMessages, offset)
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
def testWriteFullyTo() {
|
|
||||||
checkWriteFullyToWithMessageSet(createMessageSet(Array[Message]()))
|
|
||||||
checkWriteFullyToWithMessageSet(createMessageSet(messages))
|
|
||||||
}
|
|
||||||
|
|
||||||
def checkWriteFullyToWithMessageSet(messageSet: ByteBufferMessageSet) {
|
|
||||||
checkWriteWithMessageSet(messageSet, messageSet.writeFullyTo)
|
|
||||||
}
|
|
||||||
|
|
||||||
/* check that offsets are assigned based on byte offset from the given base offset */
|
/* check that offsets are assigned based on byte offset from the given base offset */
|
||||||
def checkOffsets(messages: ByteBufferMessageSet, baseOffset: Long) {
|
def checkOffsets(messages: ByteBufferMessageSet, baseOffset: Long) {
|
||||||
assertTrue("Message set should not be empty", messages.nonEmpty)
|
assertTrue("Message set should not be empty", messages.nonEmpty)
|
||||||
|
|
@ -457,59 +164,4 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases {
|
||||||
assertTrue(shallowOffsets.subsetOf(deepOffsets))
|
assertTrue(shallowOffsets.subsetOf(deepOffsets))
|
||||||
}
|
}
|
||||||
|
|
||||||
private def getMessages(magicValue: Byte = Message.CurrentMagicValue,
|
|
||||||
timestamp: Long = Message.NoTimestamp,
|
|
||||||
codec: CompressionCodec = NoCompressionCodec): ByteBufferMessageSet = {
|
|
||||||
if (magicValue == Message.MagicValue_V0) {
|
|
||||||
new ByteBufferMessageSet(
|
|
||||||
codec,
|
|
||||||
new Message("hello".getBytes, Message.NoTimestamp, Message.MagicValue_V0),
|
|
||||||
new Message("there".getBytes, Message.NoTimestamp, Message.MagicValue_V0),
|
|
||||||
new Message("beautiful".getBytes, Message.NoTimestamp, Message.MagicValue_V0))
|
|
||||||
} else {
|
|
||||||
new ByteBufferMessageSet(
|
|
||||||
codec,
|
|
||||||
new Message("hello".getBytes, timestamp = timestamp, magicValue = Message.MagicValue_V1),
|
|
||||||
new Message("there".getBytes, timestamp = timestamp, magicValue = Message.MagicValue_V1),
|
|
||||||
new Message("beautiful".getBytes, timestamp = timestamp, magicValue = Message.MagicValue_V1))
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private def messageSetWithInvalidInnerMagic(codec: CompressionCodec,
|
|
||||||
initialOffset: Long): ByteBufferMessageSet = {
|
|
||||||
val messages = (0 until 20).map(id =>
|
|
||||||
new Message(key = id.toString.getBytes,
|
|
||||||
bytes = id.toString.getBytes,
|
|
||||||
timestamp = Message.NoTimestamp,
|
|
||||||
magicValue = Message.MagicValue_V0))
|
|
||||||
|
|
||||||
val messageWriter = new MessageWriter(math.min(math.max(MessageSet.messageSetSize(messages) / 2, 1024), 1 << 16))
|
|
||||||
var lastOffset = initialOffset
|
|
||||||
|
|
||||||
messageWriter.write(
|
|
||||||
codec = codec,
|
|
||||||
timestamp = System.currentTimeMillis(),
|
|
||||||
timestampType = TimestampType.CREATE_TIME,
|
|
||||||
magicValue = Message.MagicValue_V1) { outputStream =>
|
|
||||||
|
|
||||||
val output = new DataOutputStream(CompressionFactory(codec, Message.MagicValue_V1, outputStream))
|
|
||||||
try {
|
|
||||||
for (message <- messages) {
|
|
||||||
val innerOffset = lastOffset - initialOffset
|
|
||||||
output.writeLong(innerOffset)
|
|
||||||
output.writeInt(message.size)
|
|
||||||
output.write(message.buffer.array, message.buffer.arrayOffset, message.buffer.limit)
|
|
||||||
lastOffset += 1
|
|
||||||
}
|
|
||||||
} finally {
|
|
||||||
output.close()
|
|
||||||
}
|
|
||||||
}
|
|
||||||
val buffer = ByteBuffer.allocate(messageWriter.size + MessageSet.LogOverhead)
|
|
||||||
ByteBufferMessageSet.writeMessage(buffer, messageWriter, lastOffset - 1)
|
|
||||||
buffer.rewind()
|
|
||||||
|
|
||||||
new ByteBufferMessageSet(buffer)
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -70,7 +70,7 @@ class MessageCompressionTest extends JUnitSuite {
|
||||||
testCompressSize(GZIPCompressionCodec, messages, 396)
|
testCompressSize(GZIPCompressionCodec, messages, 396)
|
||||||
|
|
||||||
if(isSnappyAvailable)
|
if(isSnappyAvailable)
|
||||||
testCompressSize(SnappyCompressionCodec, messages, 502)
|
testCompressSize(SnappyCompressionCodec, messages, 1063)
|
||||||
|
|
||||||
if(isLZ4Available)
|
if(isLZ4Available)
|
||||||
testCompressSize(LZ4CompressionCodec, messages, 387)
|
testCompressSize(LZ4CompressionCodec, messages, 387)
|
||||||
|
|
|
||||||
|
|
@ -48,7 +48,7 @@ class MessageTest extends JUnitSuite {
|
||||||
val magicValues = Array(Message.MagicValue_V0, Message.MagicValue_V1)
|
val magicValues = Array(Message.MagicValue_V0, Message.MagicValue_V1)
|
||||||
for(k <- keys; v <- vals; codec <- codecs; t <- timestamps; mv <- magicValues) {
|
for(k <- keys; v <- vals; codec <- codecs; t <- timestamps; mv <- magicValues) {
|
||||||
val timestamp = ensureValid(mv, t)
|
val timestamp = ensureValid(mv, t)
|
||||||
messages += new MessageTestVal(k, v, codec, timestamp, mv, new Message(v, k, timestamp, codec, mv))
|
messages += MessageTestVal(k, v, codec, timestamp, mv, new Message(v, k, timestamp, codec, mv))
|
||||||
}
|
}
|
||||||
|
|
||||||
def ensureValid(magicValue: Byte, timestamp: Long): Long =
|
def ensureValid(magicValue: Byte, timestamp: Long): Long =
|
||||||
|
|
@ -96,7 +96,7 @@ class MessageTest extends JUnitSuite {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
def testEquality() {
|
def testEquality() {
|
||||||
for(v <- messages) {
|
for (v <- messages) {
|
||||||
assertFalse("Should not equal null", v.message.equals(null))
|
assertFalse("Should not equal null", v.message.equals(null))
|
||||||
assertFalse("Should not equal a random string", v.message.equals("asdf"))
|
assertFalse("Should not equal a random string", v.message.equals("asdf"))
|
||||||
assertTrue("Should equal itself", v.message.equals(v.message))
|
assertTrue("Should equal itself", v.message.equals(v.message))
|
||||||
|
|
@ -105,40 +105,6 @@ class MessageTest extends JUnitSuite {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
|
||||||
def testMessageFormatConversion() {
|
|
||||||
|
|
||||||
def convertAndVerify(v: MessageTestVal, fromMessageFormat: Byte, toMessageFormat: Byte) {
|
|
||||||
assertEquals("Message should be the same when convert to the same version.",
|
|
||||||
v.message.toFormatVersion(fromMessageFormat), v.message)
|
|
||||||
val convertedMessage = v.message.toFormatVersion(toMessageFormat)
|
|
||||||
assertEquals("Size difference is not expected value", convertedMessage.size - v.message.size,
|
|
||||||
Message.headerSizeDiff(fromMessageFormat, toMessageFormat))
|
|
||||||
assertTrue("Message should still be valid", convertedMessage.isValid)
|
|
||||||
assertEquals("Timestamp should be NoTimestamp", convertedMessage.timestamp, Message.NoTimestamp)
|
|
||||||
assertEquals(s"Magic value should be $toMessageFormat now", convertedMessage.magic, toMessageFormat)
|
|
||||||
if (convertedMessage.hasKey)
|
|
||||||
assertEquals("Message key should not change", convertedMessage.key, ByteBuffer.wrap(v.key))
|
|
||||||
else
|
|
||||||
assertNull(convertedMessage.key)
|
|
||||||
if(v.payload == null) {
|
|
||||||
assertTrue(convertedMessage.isNull)
|
|
||||||
assertEquals("Payload should be null", null, convertedMessage.payload)
|
|
||||||
} else {
|
|
||||||
assertEquals("Message payload should not change", convertedMessage.payload, ByteBuffer.wrap(v.payload))
|
|
||||||
}
|
|
||||||
assertEquals("Compression codec should not change", convertedMessage.compressionCodec, v.codec)
|
|
||||||
}
|
|
||||||
|
|
||||||
for (v <- messages) {
|
|
||||||
if (v.magicValue == Message.MagicValue_V0) {
|
|
||||||
convertAndVerify(v, Message.MagicValue_V0, Message.MagicValue_V1)
|
|
||||||
} else if (v.magicValue == Message.MagicValue_V1) {
|
|
||||||
convertAndVerify(v, Message.MagicValue_V1, Message.MagicValue_V0)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test(expected = classOf[IllegalArgumentException])
|
@Test(expected = classOf[IllegalArgumentException])
|
||||||
def testInvalidTimestampAndMagicValueCombination() {
|
def testInvalidTimestampAndMagicValueCombination() {
|
||||||
new Message("hello".getBytes, 0L, Message.MagicValue_V0)
|
new Message("hello".getBytes, 0L, Message.MagicValue_V0)
|
||||||
|
|
|
||||||
|
|
@ -19,17 +19,17 @@ package kafka.server
|
||||||
|
|
||||||
import com.yammer.metrics.Metrics
|
import com.yammer.metrics.Metrics
|
||||||
import kafka.cluster.BrokerEndPoint
|
import kafka.cluster.BrokerEndPoint
|
||||||
import kafka.message.{ByteBufferMessageSet, Message, NoCompressionCodec}
|
|
||||||
import kafka.server.AbstractFetcherThread.{FetchRequest, PartitionData}
|
import kafka.server.AbstractFetcherThread.{FetchRequest, PartitionData}
|
||||||
import kafka.utils.TestUtils
|
import kafka.utils.TestUtils
|
||||||
import org.apache.kafka.common.TopicPartition
|
import org.apache.kafka.common.TopicPartition
|
||||||
import org.apache.kafka.common.protocol.Errors
|
import org.apache.kafka.common.protocol.Errors
|
||||||
import org.apache.kafka.common.utils.Utils
|
import org.apache.kafka.common.utils.Utils
|
||||||
|
import org.apache.kafka.common.record.{MemoryRecords, Record}
|
||||||
import org.junit.Assert.{assertFalse, assertTrue}
|
import org.junit.Assert.{assertFalse, assertTrue}
|
||||||
import org.junit.{Before, Test}
|
import org.junit.{Before, Test}
|
||||||
|
|
||||||
import scala.collection.JavaConverters._
|
import scala.collection.JavaConverters._
|
||||||
import scala.collection.{mutable, Map}
|
import scala.collection.{Map, mutable}
|
||||||
|
|
||||||
class AbstractFetcherThreadTest {
|
class AbstractFetcherThreadTest {
|
||||||
|
|
||||||
|
|
@ -91,10 +91,10 @@ class AbstractFetcherThreadTest {
|
||||||
override def offset(topicAndPartition: TopicPartition): Long = offsets(topicAndPartition)
|
override def offset(topicAndPartition: TopicPartition): Long = offsets(topicAndPartition)
|
||||||
}
|
}
|
||||||
|
|
||||||
class TestPartitionData(byteBufferMessageSet: ByteBufferMessageSet) extends PartitionData {
|
class TestPartitionData(records: MemoryRecords = MemoryRecords.EMPTY) extends PartitionData {
|
||||||
override def errorCode: Short = Errors.NONE.code
|
override def errorCode: Short = Errors.NONE.code
|
||||||
|
|
||||||
override def toByteBufferMessageSet: ByteBufferMessageSet = byteBufferMessageSet
|
override def toRecords: MemoryRecords = records
|
||||||
|
|
||||||
override def highWatermark: Long = 0L
|
override def highWatermark: Long = 0L
|
||||||
|
|
||||||
|
|
@ -119,7 +119,7 @@ class AbstractFetcherThreadTest {
|
||||||
override def handlePartitionsWithErrors(partitions: Iterable[TopicPartition]): Unit = {}
|
override def handlePartitionsWithErrors(partitions: Iterable[TopicPartition]): Unit = {}
|
||||||
|
|
||||||
override protected def fetch(fetchRequest: DummyFetchRequest): Seq[(TopicPartition, TestPartitionData)] =
|
override protected def fetch(fetchRequest: DummyFetchRequest): Seq[(TopicPartition, TestPartitionData)] =
|
||||||
fetchRequest.offsets.mapValues(_ => new TestPartitionData(new ByteBufferMessageSet())).toSeq
|
fetchRequest.offsets.mapValues(_ => new TestPartitionData()).toSeq
|
||||||
|
|
||||||
override protected def buildFetchRequest(partitionMap: collection.Seq[(TopicPartition, PartitionFetchState)]): DummyFetchRequest =
|
override protected def buildFetchRequest(partitionMap: collection.Seq[(TopicPartition, PartitionFetchState)]): DummyFetchRequest =
|
||||||
new DummyFetchRequest(partitionMap.map { case (k, v) => (k, v.offset) }.toMap)
|
new DummyFetchRequest(partitionMap.map { case (k, v) => (k, v.offset) }.toMap)
|
||||||
|
|
@ -156,8 +156,8 @@ class AbstractFetcherThreadTest {
|
||||||
@volatile var fetchCount = 0
|
@volatile var fetchCount = 0
|
||||||
|
|
||||||
private val normalPartitionDataSet = List(
|
private val normalPartitionDataSet = List(
|
||||||
new TestPartitionData(new ByteBufferMessageSet(NoCompressionCodec, Seq(0L), new Message("hello".getBytes))),
|
new TestPartitionData(MemoryRecords.withRecords(0L, Record.create("hello".getBytes()))),
|
||||||
new TestPartitionData(new ByteBufferMessageSet(NoCompressionCodec, Seq(1L), new Message("hello".getBytes)))
|
new TestPartitionData(MemoryRecords.withRecords(1L, Record.create("hello".getBytes())))
|
||||||
)
|
)
|
||||||
|
|
||||||
override def processPartitionData(topicAndPartition: TopicPartition,
|
override def processPartitionData(topicAndPartition: TopicPartition,
|
||||||
|
|
@ -170,10 +170,10 @@ class AbstractFetcherThreadTest {
|
||||||
.format(topicAndPartition, fetchOffset, logEndOffset))
|
.format(topicAndPartition, fetchOffset, logEndOffset))
|
||||||
|
|
||||||
// Now check message's crc
|
// Now check message's crc
|
||||||
val messages = partitionData.toByteBufferMessageSet
|
val records = partitionData.toRecords
|
||||||
for (messageAndOffset <- messages.shallowIterator) {
|
for (entry <- records.shallowIterator.asScala) {
|
||||||
messageAndOffset.message.ensureValid()
|
entry.record.ensureValid()
|
||||||
logEndOffset = messageAndOffset.nextOffset
|
logEndOffset = entry.nextOffset
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -181,12 +181,12 @@ class AbstractFetcherThreadTest {
|
||||||
fetchCount += 1
|
fetchCount += 1
|
||||||
// Set the first fetch to get a corrupted message
|
// Set the first fetch to get a corrupted message
|
||||||
if (fetchCount == 1) {
|
if (fetchCount == 1) {
|
||||||
val corruptedMessage = new Message("hello".getBytes)
|
val corruptedRecord = Record.create("hello".getBytes())
|
||||||
val badChecksum = (corruptedMessage.checksum + 1 % Int.MaxValue).toInt
|
val badChecksum = (corruptedRecord.checksum + 1 % Int.MaxValue).toInt
|
||||||
// Garble checksum
|
// Garble checksum
|
||||||
Utils.writeUnsignedInt(corruptedMessage.buffer, Message.CrcOffset, badChecksum)
|
Utils.writeUnsignedInt(corruptedRecord.buffer, Record.CRC_OFFSET, badChecksum)
|
||||||
val byteBufferMessageSet = new ByteBufferMessageSet(NoCompressionCodec, corruptedMessage)
|
val records = MemoryRecords.withRecords(corruptedRecord)
|
||||||
fetchRequest.offsets.mapValues(_ => new TestPartitionData(byteBufferMessageSet)).toSeq
|
fetchRequest.offsets.mapValues(_ => new TestPartitionData(records)).toSeq
|
||||||
} else
|
} else
|
||||||
// Then, the following fetches get the normal data
|
// Then, the following fetches get the normal data
|
||||||
fetchRequest.offsets.mapValues(v => normalPartitionDataSet(v.toInt)).toSeq
|
fetchRequest.offsets.mapValues(v => normalPartitionDataSet(v.toInt)).toSeq
|
||||||
|
|
|
||||||
|
|
@ -24,12 +24,12 @@ import kafka.utils.TestUtils
|
||||||
import kafka.utils.TestUtils._
|
import kafka.utils.TestUtils._
|
||||||
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
|
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
|
||||||
import org.apache.kafka.common.TopicPartition
|
import org.apache.kafka.common.TopicPartition
|
||||||
import org.apache.kafka.common.protocol.{ApiKeys, Errors, ProtoUtils}
|
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
|
||||||
import org.apache.kafka.common.record.{LogEntry, MemoryRecords}
|
import org.apache.kafka.common.record.LogEntry
|
||||||
import org.apache.kafka.common.requests.{FetchRequest, FetchResponse}
|
import org.apache.kafka.common.requests.{FetchRequest, FetchResponse}
|
||||||
import org.apache.kafka.common.serialization.StringSerializer
|
import org.apache.kafka.common.serialization.StringSerializer
|
||||||
import org.junit.Test
|
|
||||||
import org.junit.Assert._
|
import org.junit.Assert._
|
||||||
|
import org.junit.Test
|
||||||
|
|
||||||
import scala.collection.JavaConverters._
|
import scala.collection.JavaConverters._
|
||||||
import scala.util.Random
|
import scala.util.Random
|
||||||
|
|
@ -120,13 +120,13 @@ class FetchRequestTest extends BaseRequestTest {
|
||||||
val fetchResponse3 = sendFetchRequest(leaderId, fetchRequest3)
|
val fetchResponse3 = sendFetchRequest(leaderId, fetchRequest3)
|
||||||
assertEquals(shuffledTopicPartitions3, fetchResponse3.responseData.keySet.asScala.toSeq)
|
assertEquals(shuffledTopicPartitions3, fetchResponse3.responseData.keySet.asScala.toSeq)
|
||||||
val responseSize3 = fetchResponse3.responseData.asScala.values.map { partitionData =>
|
val responseSize3 = fetchResponse3.responseData.asScala.values.map { partitionData =>
|
||||||
logEntries(partitionData).map(_.size).sum
|
logEntries(partitionData).map(_.sizeInBytes).sum
|
||||||
}.sum
|
}.sum
|
||||||
assertTrue(responseSize3 <= maxResponseBytes)
|
assertTrue(responseSize3 <= maxResponseBytes)
|
||||||
val partitionData3 = fetchResponse3.responseData.get(partitionWithLargeMessage1)
|
val partitionData3 = fetchResponse3.responseData.get(partitionWithLargeMessage1)
|
||||||
assertEquals(Errors.NONE.code, partitionData3.errorCode)
|
assertEquals(Errors.NONE.code, partitionData3.errorCode)
|
||||||
assertTrue(partitionData3.highWatermark > 0)
|
assertTrue(partitionData3.highWatermark > 0)
|
||||||
val size3 = logEntries(partitionData3).map(_.size).sum
|
val size3 = logEntries(partitionData3).map(_.sizeInBytes).sum
|
||||||
assertTrue(s"Expected $size3 to be smaller than $maxResponseBytes", size3 <= maxResponseBytes)
|
assertTrue(s"Expected $size3 to be smaller than $maxResponseBytes", size3 <= maxResponseBytes)
|
||||||
assertTrue(s"Expected $size3 to be larger than $maxPartitionBytes", size3 > maxPartitionBytes)
|
assertTrue(s"Expected $size3 to be larger than $maxPartitionBytes", size3 > maxPartitionBytes)
|
||||||
assertTrue(maxPartitionBytes < partitionData3.records.sizeInBytes)
|
assertTrue(maxPartitionBytes < partitionData3.records.sizeInBytes)
|
||||||
|
|
@ -138,13 +138,13 @@ class FetchRequestTest extends BaseRequestTest {
|
||||||
val fetchResponse4 = sendFetchRequest(leaderId, fetchRequest4)
|
val fetchResponse4 = sendFetchRequest(leaderId, fetchRequest4)
|
||||||
assertEquals(shuffledTopicPartitions4, fetchResponse4.responseData.keySet.asScala.toSeq)
|
assertEquals(shuffledTopicPartitions4, fetchResponse4.responseData.keySet.asScala.toSeq)
|
||||||
val nonEmptyPartitions4 = fetchResponse4.responseData.asScala.toSeq.collect {
|
val nonEmptyPartitions4 = fetchResponse4.responseData.asScala.toSeq.collect {
|
||||||
case (tp, partitionData) if logEntries(partitionData).map(_.size).sum > 0 => tp
|
case (tp, partitionData) if logEntries(partitionData).map(_.sizeInBytes).sum > 0 => tp
|
||||||
}
|
}
|
||||||
assertEquals(Seq(partitionWithLargeMessage2), nonEmptyPartitions4)
|
assertEquals(Seq(partitionWithLargeMessage2), nonEmptyPartitions4)
|
||||||
val partitionData4 = fetchResponse4.responseData.get(partitionWithLargeMessage2)
|
val partitionData4 = fetchResponse4.responseData.get(partitionWithLargeMessage2)
|
||||||
assertEquals(Errors.NONE.code, partitionData4.errorCode)
|
assertEquals(Errors.NONE.code, partitionData4.errorCode)
|
||||||
assertTrue(partitionData4.highWatermark > 0)
|
assertTrue(partitionData4.highWatermark > 0)
|
||||||
val size4 = logEntries(partitionData4).map(_.size).sum
|
val size4 = logEntries(partitionData4).map(_.sizeInBytes).sum
|
||||||
assertTrue(s"Expected $size4 to be larger than $maxResponseBytes", size4 > maxResponseBytes)
|
assertTrue(s"Expected $size4 to be larger than $maxResponseBytes", size4 > maxResponseBytes)
|
||||||
assertTrue(maxResponseBytes < partitionData4.records.sizeInBytes)
|
assertTrue(maxResponseBytes < partitionData4.records.sizeInBytes)
|
||||||
}
|
}
|
||||||
|
|
@ -161,12 +161,11 @@ class FetchRequestTest extends BaseRequestTest {
|
||||||
assertEquals(Errors.NONE.code, partitionData.errorCode)
|
assertEquals(Errors.NONE.code, partitionData.errorCode)
|
||||||
assertTrue(partitionData.highWatermark > 0)
|
assertTrue(partitionData.highWatermark > 0)
|
||||||
assertEquals(maxPartitionBytes, partitionData.records.sizeInBytes)
|
assertEquals(maxPartitionBytes, partitionData.records.sizeInBytes)
|
||||||
assertEquals(0, logEntries(partitionData).map(_.size).sum)
|
assertEquals(0, logEntries(partitionData).map(_.sizeInBytes).sum)
|
||||||
}
|
}
|
||||||
|
|
||||||
private def logEntries(partitionData: FetchResponse.PartitionData): Seq[LogEntry] = {
|
private def logEntries(partitionData: FetchResponse.PartitionData): Seq[LogEntry] = {
|
||||||
val memoryRecords = partitionData.records
|
partitionData.records.deepIterator.asScala.toIndexedSeq
|
||||||
memoryRecords.iterator.asScala.toIndexedSeq
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private def checkFetchResponse(expectedPartitions: Seq[TopicPartition], fetchResponse: FetchResponse,
|
private def checkFetchResponse(expectedPartitions: Seq[TopicPartition], fetchResponse: FetchResponse,
|
||||||
|
|
@ -181,25 +180,25 @@ class FetchRequestTest extends BaseRequestTest {
|
||||||
assertEquals(Errors.NONE.code, partitionData.errorCode)
|
assertEquals(Errors.NONE.code, partitionData.errorCode)
|
||||||
assertTrue(partitionData.highWatermark > 0)
|
assertTrue(partitionData.highWatermark > 0)
|
||||||
|
|
||||||
val memoryRecords = partitionData.records
|
val records = partitionData.records
|
||||||
responseBufferSize += memoryRecords.sizeInBytes
|
responseBufferSize += records.sizeInBytes
|
||||||
|
|
||||||
val messages = memoryRecords.iterator.asScala.toIndexedSeq
|
val entries = records.shallowIterator.asScala.toIndexedSeq
|
||||||
assertTrue(messages.size < numMessagesPerPartition)
|
assertTrue(entries.size < numMessagesPerPartition)
|
||||||
val messagesSize = messages.map(_.size).sum
|
val entriesSize = entries.map(_.sizeInBytes).sum
|
||||||
responseSize += messagesSize
|
responseSize += entriesSize
|
||||||
if (messagesSize == 0 && !emptyResponseSeen) {
|
if (entriesSize == 0 && !emptyResponseSeen) {
|
||||||
assertEquals(0, memoryRecords.sizeInBytes)
|
assertEquals(0, records.sizeInBytes)
|
||||||
emptyResponseSeen = true
|
emptyResponseSeen = true
|
||||||
}
|
}
|
||||||
else if (messagesSize != 0 && !emptyResponseSeen) {
|
else if (entriesSize != 0 && !emptyResponseSeen) {
|
||||||
assertTrue(messagesSize <= maxPartitionBytes)
|
assertTrue(entriesSize <= maxPartitionBytes)
|
||||||
assertEquals(maxPartitionBytes, memoryRecords.sizeInBytes)
|
assertEquals(maxPartitionBytes, records.sizeInBytes)
|
||||||
}
|
}
|
||||||
else if (messagesSize != 0 && emptyResponseSeen)
|
else if (entriesSize != 0 && emptyResponseSeen)
|
||||||
fail(s"Expected partition with size 0, but found $tp with size $messagesSize")
|
fail(s"Expected partition with size 0, but found $tp with size $entriesSize")
|
||||||
else if (memoryRecords.sizeInBytes != 0 && emptyResponseSeen)
|
else if (records.sizeInBytes != 0 && emptyResponseSeen)
|
||||||
fail(s"Expected partition buffer with size 0, but found $tp with size ${memoryRecords.sizeInBytes}")
|
fail(s"Expected partition buffer with size 0, but found $tp with size ${records.sizeInBytes}")
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -208,7 +207,7 @@ class FetchRequestTest extends BaseRequestTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
private def createTopics(numTopics: Int, numPartitions: Int): Map[TopicPartition, Int] = {
|
private def createTopics(numTopics: Int, numPartitions: Int): Map[TopicPartition, Int] = {
|
||||||
val topics = (0 until numPartitions).map(t => s"topic${t}")
|
val topics = (0 until numPartitions).map(t => s"topic$t")
|
||||||
val topicConfig = new Properties
|
val topicConfig = new Properties
|
||||||
topicConfig.setProperty(LogConfig.MinInSyncReplicasProp, 2.toString)
|
topicConfig.setProperty(LogConfig.MinInSyncReplicasProp, 2.toString)
|
||||||
topics.flatMap { topic =>
|
topics.flatMap { topic =>
|
||||||
|
|
@ -223,7 +222,7 @@ class FetchRequestTest extends BaseRequestTest {
|
||||||
tp <- topicPartitions.toSeq
|
tp <- topicPartitions.toSeq
|
||||||
messageIndex <- 0 until numMessagesPerPartition
|
messageIndex <- 0 until numMessagesPerPartition
|
||||||
} yield {
|
} yield {
|
||||||
val suffix = s"${tp}-${messageIndex}"
|
val suffix = s"$tp-$messageIndex"
|
||||||
new ProducerRecord(tp.topic, tp.partition, s"key $suffix", s"value $suffix")
|
new ProducerRecord(tp.topic, tp.partition, s"key $suffix", s"value $suffix")
|
||||||
}
|
}
|
||||||
records.map(producer.send).foreach(_.get)
|
records.map(producer.send).foreach(_.get)
|
||||||
|
|
|
||||||
|
|
@ -17,21 +17,20 @@
|
||||||
package kafka.server
|
package kafka.server
|
||||||
|
|
||||||
import java.util.Properties
|
import java.util.Properties
|
||||||
|
|
||||||
import org.apache.kafka.common.metrics.Metrics
|
|
||||||
import org.junit.{After, Before, Test}
|
|
||||||
|
|
||||||
import collection.mutable.HashMap
|
|
||||||
import collection.mutable.Map
|
|
||||||
import kafka.cluster.{Partition, Replica}
|
|
||||||
import org.easymock.EasyMock
|
|
||||||
import kafka.log.Log
|
|
||||||
import org.junit.Assert._
|
|
||||||
import kafka.utils._
|
|
||||||
import java.util.concurrent.atomic.AtomicBoolean
|
import java.util.concurrent.atomic.AtomicBoolean
|
||||||
|
|
||||||
import kafka.message.MessageSet
|
import kafka.cluster.{Partition, Replica}
|
||||||
|
import kafka.log.Log
|
||||||
|
import kafka.utils._
|
||||||
|
import org.apache.kafka.common.metrics.Metrics
|
||||||
|
import org.apache.kafka.common.record.MemoryRecords
|
||||||
import org.apache.kafka.common.utils.Time
|
import org.apache.kafka.common.utils.Time
|
||||||
|
import org.easymock.EasyMock
|
||||||
|
import org.junit.Assert._
|
||||||
|
import org.junit.{After, Before, Test}
|
||||||
|
|
||||||
|
import scala.collection.mutable.{HashMap, Map}
|
||||||
|
|
||||||
|
|
||||||
class IsrExpirationTest {
|
class IsrExpirationTest {
|
||||||
|
|
||||||
|
|
@ -76,7 +75,7 @@ class IsrExpirationTest {
|
||||||
|
|
||||||
// let the follower catch up to the Leader logEndOffset (15)
|
// let the follower catch up to the Leader logEndOffset (15)
|
||||||
(partition0.assignedReplicas() - leaderReplica).foreach(
|
(partition0.assignedReplicas() - leaderReplica).foreach(
|
||||||
r => r.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(15L), MessageSet.Empty),
|
r => r.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(15L), MemoryRecords.EMPTY),
|
||||||
-1L,
|
-1L,
|
||||||
-1,
|
-1,
|
||||||
true)))
|
true)))
|
||||||
|
|
@ -127,7 +126,7 @@ class IsrExpirationTest {
|
||||||
|
|
||||||
// Make the remote replica not read to the end of log. It should be not be out of sync for at least 100 ms
|
// Make the remote replica not read to the end of log. It should be not be out of sync for at least 100 ms
|
||||||
for(replica <- partition0.assignedReplicas() - leaderReplica)
|
for(replica <- partition0.assignedReplicas() - leaderReplica)
|
||||||
replica.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(10L), MessageSet.Empty), -1L, -1, false))
|
replica.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(10L), MemoryRecords.EMPTY), -1L, -1, false))
|
||||||
|
|
||||||
// Simulate 2 fetch requests spanning more than 100 ms which do not read to the end of the log.
|
// Simulate 2 fetch requests spanning more than 100 ms which do not read to the end of the log.
|
||||||
// The replicas will no longer be in ISR. We do 2 fetches because we want to simulate the case where the replica is lagging but is not stuck
|
// The replicas will no longer be in ISR. We do 2 fetches because we want to simulate the case where the replica is lagging but is not stuck
|
||||||
|
|
@ -137,7 +136,7 @@ class IsrExpirationTest {
|
||||||
time.sleep(75)
|
time.sleep(75)
|
||||||
|
|
||||||
(partition0.assignedReplicas() - leaderReplica).foreach(
|
(partition0.assignedReplicas() - leaderReplica).foreach(
|
||||||
r => r.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(11L), MessageSet.Empty), -1L, -1, false)))
|
r => r.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(11L), MemoryRecords.EMPTY), -1L, -1, false)))
|
||||||
partition0OSR = partition0.getOutOfSyncReplicas(leaderReplica, configs.head.replicaLagTimeMaxMs)
|
partition0OSR = partition0.getOutOfSyncReplicas(leaderReplica, configs.head.replicaLagTimeMaxMs)
|
||||||
assertEquals("No replica should be out of sync", Set.empty[Int], partition0OSR.map(_.brokerId))
|
assertEquals("No replica should be out of sync", Set.empty[Int], partition0OSR.map(_.brokerId))
|
||||||
|
|
||||||
|
|
@ -149,7 +148,7 @@ class IsrExpirationTest {
|
||||||
|
|
||||||
// Now actually make a fetch to the end of the log. The replicas should be back in ISR
|
// Now actually make a fetch to the end of the log. The replicas should be back in ISR
|
||||||
(partition0.assignedReplicas() - leaderReplica).foreach(
|
(partition0.assignedReplicas() - leaderReplica).foreach(
|
||||||
r => r.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(15L), MessageSet.Empty), -1L, -1, true)))
|
r => r.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(15L), MemoryRecords.EMPTY), -1L, -1, true)))
|
||||||
partition0OSR = partition0.getOutOfSyncReplicas(leaderReplica, configs.head.replicaLagTimeMaxMs)
|
partition0OSR = partition0.getOutOfSyncReplicas(leaderReplica, configs.head.replicaLagTimeMaxMs)
|
||||||
assertEquals("No replica should be out of sync", Set.empty[Int], partition0OSR.map(_.brokerId))
|
assertEquals("No replica should be out of sync", Set.empty[Int], partition0OSR.map(_.brokerId))
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -26,12 +26,12 @@ import kafka.api.{FetchRequestBuilder, OffsetRequest, PartitionOffsetRequestInfo
|
||||||
import kafka.common.TopicAndPartition
|
import kafka.common.TopicAndPartition
|
||||||
import kafka.consumer.SimpleConsumer
|
import kafka.consumer.SimpleConsumer
|
||||||
import kafka.log.{Log, LogSegment}
|
import kafka.log.{Log, LogSegment}
|
||||||
import kafka.message.{ByteBufferMessageSet, Message, NoCompressionCodec}
|
|
||||||
import kafka.utils.TestUtils._
|
import kafka.utils.TestUtils._
|
||||||
import kafka.utils._
|
import kafka.utils._
|
||||||
import kafka.zk.ZooKeeperTestHarness
|
import kafka.zk.ZooKeeperTestHarness
|
||||||
import org.apache.kafka.common.TopicPartition
|
import org.apache.kafka.common.TopicPartition
|
||||||
import org.apache.kafka.common.protocol.Errors
|
import org.apache.kafka.common.protocol.Errors
|
||||||
|
import org.apache.kafka.common.record.{MemoryRecords, Record}
|
||||||
import org.apache.kafka.common.utils.{Time, Utils}
|
import org.apache.kafka.common.utils.{Time, Utils}
|
||||||
import org.easymock.{EasyMock, IAnswer}
|
import org.easymock.{EasyMock, IAnswer}
|
||||||
import org.junit.Assert._
|
import org.junit.Assert._
|
||||||
|
|
@ -89,9 +89,9 @@ class LogOffsetTest extends ZooKeeperTestHarness {
|
||||||
"Log for partition [topic,0] should be created")
|
"Log for partition [topic,0] should be created")
|
||||||
val log = logManager.getLog(TopicAndPartition(topic, part)).get
|
val log = logManager.getLog(TopicAndPartition(topic, part)).get
|
||||||
|
|
||||||
val message = new Message(Integer.toString(42).getBytes())
|
val record = Record.create(Integer.toString(42).getBytes())
|
||||||
for (_ <- 0 until 20)
|
for (_ <- 0 until 20)
|
||||||
log.append(new ByteBufferMessageSet(NoCompressionCodec, message))
|
log.append(MemoryRecords.withRecords(record))
|
||||||
log.flush()
|
log.flush()
|
||||||
|
|
||||||
val offsets = server.apis.fetchOffsets(logManager, new TopicPartition(topic, part), OffsetRequest.LatestTime, 15)
|
val offsets = server.apis.fetchOffsets(logManager, new TopicPartition(topic, part), OffsetRequest.LatestTime, 15)
|
||||||
|
|
@ -150,9 +150,9 @@ class LogOffsetTest extends ZooKeeperTestHarness {
|
||||||
|
|
||||||
val logManager = server.getLogManager
|
val logManager = server.getLogManager
|
||||||
val log = logManager.createLog(TopicAndPartition(topic, part), logManager.defaultConfig)
|
val log = logManager.createLog(TopicAndPartition(topic, part), logManager.defaultConfig)
|
||||||
val message = new Message(Integer.toString(42).getBytes())
|
val record = Record.create(Integer.toString(42).getBytes())
|
||||||
for (_ <- 0 until 20)
|
for (_ <- 0 until 20)
|
||||||
log.append(new ByteBufferMessageSet(NoCompressionCodec, message))
|
log.append(MemoryRecords.withRecords(record))
|
||||||
log.flush()
|
log.flush()
|
||||||
|
|
||||||
val now = time.milliseconds + 30000 // pretend it is the future to avoid race conditions with the fs
|
val now = time.milliseconds + 30000 // pretend it is the future to avoid race conditions with the fs
|
||||||
|
|
@ -179,9 +179,9 @@ class LogOffsetTest extends ZooKeeperTestHarness {
|
||||||
|
|
||||||
val logManager = server.getLogManager
|
val logManager = server.getLogManager
|
||||||
val log = logManager.createLog(TopicAndPartition(topic, part), logManager.defaultConfig)
|
val log = logManager.createLog(TopicAndPartition(topic, part), logManager.defaultConfig)
|
||||||
val message = new Message(Integer.toString(42).getBytes())
|
val record = Record.create(Integer.toString(42).getBytes())
|
||||||
for (_ <- 0 until 20)
|
for (_ <- 0 until 20)
|
||||||
log.append(new ByteBufferMessageSet(NoCompressionCodec, message))
|
log.append(MemoryRecords.withRecords(record))
|
||||||
log.flush()
|
log.flush()
|
||||||
|
|
||||||
val offsets = server.apis.fetchOffsets(logManager, new TopicPartition(topic, part), OffsetRequest.EarliestTime, 10)
|
val offsets = server.apis.fetchOffsets(logManager, new TopicPartition(topic, part), OffsetRequest.EarliestTime, 10)
|
||||||
|
|
|
||||||
|
|
@ -54,11 +54,11 @@ class ProduceRequestTest extends BaseRequestTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
sendAndCheck(JTestUtils.partitionRecordsBuffer(0, CompressionType.NONE,
|
sendAndCheck(JTestUtils.partitionRecordsBuffer(0, CompressionType.NONE,
|
||||||
new Record(System.currentTimeMillis(), "key".getBytes, "value".getBytes)), 0)
|
Record.create(System.currentTimeMillis(), "key".getBytes, "value".getBytes)), 0)
|
||||||
|
|
||||||
sendAndCheck(JTestUtils.partitionRecordsBuffer(0, CompressionType.GZIP,
|
sendAndCheck(JTestUtils.partitionRecordsBuffer(0, CompressionType.GZIP,
|
||||||
new Record(System.currentTimeMillis(), "key1".getBytes, "value1".getBytes),
|
Record.create(System.currentTimeMillis(), "key1".getBytes, "value1".getBytes),
|
||||||
new Record(System.currentTimeMillis(), "key2".getBytes, "value2".getBytes)), 1)
|
Record.create(System.currentTimeMillis(), "key2".getBytes, "value2".getBytes)), 1)
|
||||||
}
|
}
|
||||||
|
|
||||||
/* returns a pair of partition id and leader id */
|
/* returns a pair of partition id and leader id */
|
||||||
|
|
@ -74,7 +74,7 @@ class ProduceRequestTest extends BaseRequestTest {
|
||||||
val (partition, leader) = createTopicAndFindPartitionWithLeader("topic")
|
val (partition, leader) = createTopicAndFindPartitionWithLeader("topic")
|
||||||
val timestamp = 1000000
|
val timestamp = 1000000
|
||||||
val recordBuffer = JTestUtils.partitionRecordsBuffer(0, CompressionType.LZ4,
|
val recordBuffer = JTestUtils.partitionRecordsBuffer(0, CompressionType.LZ4,
|
||||||
new Record(timestamp, "key".getBytes, "value".getBytes))
|
Record.create(timestamp, "key".getBytes, "value".getBytes))
|
||||||
// Change the lz4 checksum value so that it doesn't match the contents
|
// Change the lz4 checksum value so that it doesn't match the contents
|
||||||
recordBuffer.array.update(40, 0)
|
recordBuffer.array.update(40, 0)
|
||||||
val topicPartition = new TopicPartition("topic", partition)
|
val topicPartition = new TopicPartition("topic", partition)
|
||||||
|
|
|
||||||
|
|
@ -16,28 +16,29 @@
|
||||||
*/
|
*/
|
||||||
package kafka.server
|
package kafka.server
|
||||||
|
|
||||||
|
|
||||||
import java.util.Properties
|
import java.util.Properties
|
||||||
import java.util.concurrent.atomic.AtomicBoolean
|
import java.util.concurrent.atomic.AtomicBoolean
|
||||||
|
|
||||||
import kafka.cluster.Replica
|
import kafka.cluster.Replica
|
||||||
import kafka.common.TopicAndPartition
|
import kafka.common.TopicAndPartition
|
||||||
import kafka.log.Log
|
import kafka.log.Log
|
||||||
import kafka.message.{ByteBufferMessageSet, Message}
|
|
||||||
import kafka.utils._
|
import kafka.utils._
|
||||||
import org.apache.kafka.common.TopicPartition
|
import org.apache.kafka.common.TopicPartition
|
||||||
import org.apache.kafka.common.metrics.Metrics
|
import org.apache.kafka.common.metrics.Metrics
|
||||||
|
import org.apache.kafka.common.record.{MemoryRecords, Record}
|
||||||
import org.apache.kafka.common.requests.FetchRequest.PartitionData
|
import org.apache.kafka.common.requests.FetchRequest.PartitionData
|
||||||
import org.easymock.EasyMock
|
import org.easymock.EasyMock
|
||||||
import EasyMock._
|
import EasyMock._
|
||||||
import org.junit.Assert._
|
import org.junit.Assert._
|
||||||
import org.junit.{After, Test}
|
import org.junit.{After, Test}
|
||||||
|
|
||||||
|
import scala.collection.JavaConverters._
|
||||||
|
|
||||||
class ReplicaManagerQuotasTest {
|
class ReplicaManagerQuotasTest {
|
||||||
val configs = TestUtils.createBrokerConfigs(2, TestUtils.MockZkConnect).map(KafkaConfig.fromProps(_, new Properties()))
|
val configs = TestUtils.createBrokerConfigs(2, TestUtils.MockZkConnect).map(KafkaConfig.fromProps(_, new Properties()))
|
||||||
val time = new MockTime
|
val time = new MockTime
|
||||||
val metrics = new Metrics
|
val metrics = new Metrics
|
||||||
val message = new Message("some-data-in-a-message".getBytes())
|
val record = Record.create("some-data-in-a-message".getBytes())
|
||||||
val topicAndPartition1 = TopicAndPartition("test-topic", 1)
|
val topicAndPartition1 = TopicAndPartition("test-topic", 1)
|
||||||
val topicAndPartition2 = TopicAndPartition("test-topic", 2)
|
val topicAndPartition2 = TopicAndPartition("test-topic", 2)
|
||||||
val fetchInfo = Seq(new TopicPartition(topicAndPartition1.topic, topicAndPartition1.partition) -> new PartitionData(0, 100),
|
val fetchInfo = Seq(new TopicPartition(topicAndPartition1.topic, topicAndPartition1.partition) -> new PartitionData(0, 100),
|
||||||
|
|
@ -63,10 +64,10 @@ class ReplicaManagerQuotasTest {
|
||||||
readPartitionInfo = fetchInfo,
|
readPartitionInfo = fetchInfo,
|
||||||
quota = quota)
|
quota = quota)
|
||||||
assertEquals("Given two partitions, with only one throttled, we should get the first", 1,
|
assertEquals("Given two partitions, with only one throttled, we should get the first", 1,
|
||||||
fetch.find(_._1 == topicAndPartition1).get._2.info.messageSet.size)
|
fetch.find(_._1 == topicAndPartition1).get._2.info.records.shallowIterator.asScala.size)
|
||||||
|
|
||||||
assertEquals("But we shouldn't get the second", 0,
|
assertEquals("But we shouldn't get the second", 0,
|
||||||
fetch.find(_._1 == topicAndPartition2).get._2.info.messageSet.size)
|
fetch.find(_._1 == topicAndPartition2).get._2.info.records.shallowIterator.asScala.size)
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
@ -88,9 +89,9 @@ class ReplicaManagerQuotasTest {
|
||||||
readPartitionInfo = fetchInfo,
|
readPartitionInfo = fetchInfo,
|
||||||
quota = quota)
|
quota = quota)
|
||||||
assertEquals("Given two partitions, with both throttled, we should get no messages", 0,
|
assertEquals("Given two partitions, with both throttled, we should get no messages", 0,
|
||||||
fetch.find(_._1 == topicAndPartition1).get._2.info.messageSet.size)
|
fetch.find(_._1 == topicAndPartition1).get._2.info.records.shallowIterator.asScala.size)
|
||||||
assertEquals("Given two partitions, with both throttled, we should get no messages", 0,
|
assertEquals("Given two partitions, with both throttled, we should get no messages", 0,
|
||||||
fetch.find(_._1 == topicAndPartition2).get._2.info.messageSet.size)
|
fetch.find(_._1 == topicAndPartition2).get._2.info.records.shallowIterator.asScala.size)
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
@ -112,9 +113,9 @@ class ReplicaManagerQuotasTest {
|
||||||
readPartitionInfo = fetchInfo,
|
readPartitionInfo = fetchInfo,
|
||||||
quota = quota)
|
quota = quota)
|
||||||
assertEquals("Given two partitions, with both non-throttled, we should get both messages", 1,
|
assertEquals("Given two partitions, with both non-throttled, we should get both messages", 1,
|
||||||
fetch.find(_._1 == topicAndPartition1).get._2.info.messageSet.size)
|
fetch.find(_._1 == topicAndPartition1).get._2.info.records.shallowIterator.asScala.size)
|
||||||
assertEquals("Given two partitions, with both non-throttled, we should get both messages", 1,
|
assertEquals("Given two partitions, with both non-throttled, we should get both messages", 1,
|
||||||
fetch.find(_._1 == topicAndPartition2).get._2.info.messageSet.size)
|
fetch.find(_._1 == topicAndPartition2).get._2.info.records.shallowIterator.asScala.size)
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
|
@ -136,13 +137,13 @@ class ReplicaManagerQuotasTest {
|
||||||
readPartitionInfo = fetchInfo,
|
readPartitionInfo = fetchInfo,
|
||||||
quota = quota)
|
quota = quota)
|
||||||
assertEquals("Given two partitions, with only one throttled, we should get the first", 1,
|
assertEquals("Given two partitions, with only one throttled, we should get the first", 1,
|
||||||
fetch.find(_._1 == topicAndPartition1).get._2.info.messageSet.size)
|
fetch.find(_._1 == topicAndPartition1).get._2.info.records.shallowIterator.asScala.size)
|
||||||
|
|
||||||
assertEquals("But we should get the second too since it's throttled but in sync", 1,
|
assertEquals("But we should get the second too since it's throttled but in sync", 1,
|
||||||
fetch.find(_._1 == topicAndPartition2).get._2.info.messageSet.size)
|
fetch.find(_._1 == topicAndPartition2).get._2.info.records.shallowIterator.asScala.size)
|
||||||
}
|
}
|
||||||
|
|
||||||
def setUpMocks(fetchInfo: Seq[(TopicPartition, PartitionData)], message: Message = this.message, bothReplicasInSync: Boolean = false) {
|
def setUpMocks(fetchInfo: Seq[(TopicPartition, PartitionData)], record: Record = this.record, bothReplicasInSync: Boolean = false) {
|
||||||
val zkUtils = createNiceMock(classOf[ZkUtils])
|
val zkUtils = createNiceMock(classOf[ZkUtils])
|
||||||
val scheduler = createNiceMock(classOf[KafkaScheduler])
|
val scheduler = createNiceMock(classOf[KafkaScheduler])
|
||||||
|
|
||||||
|
|
@ -153,16 +154,16 @@ class ReplicaManagerQuotasTest {
|
||||||
|
|
||||||
//if we ask for len 1 return a message
|
//if we ask for len 1 return a message
|
||||||
expect(log.read(anyObject(), geq(1), anyObject(), anyObject())).andReturn(
|
expect(log.read(anyObject(), geq(1), anyObject(), anyObject())).andReturn(
|
||||||
new FetchDataInfo(
|
FetchDataInfo(
|
||||||
new LogOffsetMetadata(0L, 0L, 0),
|
new LogOffsetMetadata(0L, 0L, 0),
|
||||||
new ByteBufferMessageSet(message)
|
MemoryRecords.withRecords(record)
|
||||||
)).anyTimes()
|
)).anyTimes()
|
||||||
|
|
||||||
//if we ask for len = 0, return 0 messages
|
//if we ask for len = 0, return 0 messages
|
||||||
expect(log.read(anyObject(), EasyMock.eq(0), anyObject(), anyObject())).andReturn(
|
expect(log.read(anyObject(), EasyMock.eq(0), anyObject(), anyObject())).andReturn(
|
||||||
new FetchDataInfo(
|
FetchDataInfo(
|
||||||
new LogOffsetMetadata(0L, 0L, 0),
|
new LogOffsetMetadata(0L, 0L, 0),
|
||||||
new ByteBufferMessageSet()
|
MemoryRecords.EMPTY
|
||||||
)).anyTimes()
|
)).anyTimes()
|
||||||
replay(log)
|
replay(log)
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -17,18 +17,16 @@
|
||||||
|
|
||||||
package kafka.server
|
package kafka.server
|
||||||
|
|
||||||
|
|
||||||
import java.io.File
|
import java.io.File
|
||||||
import java.util.concurrent.atomic.AtomicBoolean
|
import java.util.concurrent.atomic.AtomicBoolean
|
||||||
|
|
||||||
import kafka.api.FetchResponsePartitionData
|
|
||||||
import kafka.cluster.Broker
|
import kafka.cluster.Broker
|
||||||
import kafka.common.TopicAndPartition
|
import kafka.common.TopicAndPartition
|
||||||
import kafka.message.{ByteBufferMessageSet, Message, MessageSet}
|
|
||||||
import kafka.utils.{MockScheduler, MockTime, TestUtils, ZkUtils}
|
import kafka.utils.{MockScheduler, MockTime, TestUtils, ZkUtils}
|
||||||
import org.I0Itec.zkclient.ZkClient
|
import org.I0Itec.zkclient.ZkClient
|
||||||
import org.apache.kafka.common.metrics.Metrics
|
import org.apache.kafka.common.metrics.Metrics
|
||||||
import org.apache.kafka.common.protocol.Errors
|
import org.apache.kafka.common.protocol.Errors
|
||||||
|
import org.apache.kafka.common.record.{MemoryRecords, Record, Records}
|
||||||
import org.apache.kafka.common.requests.{LeaderAndIsrRequest, PartitionState}
|
import org.apache.kafka.common.requests.{LeaderAndIsrRequest, PartitionState}
|
||||||
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
|
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
|
||||||
import org.apache.kafka.common.requests.FetchRequest.PartitionData
|
import org.apache.kafka.common.requests.FetchRequest.PartitionData
|
||||||
|
|
@ -105,11 +103,11 @@ class ReplicaManagerTest {
|
||||||
def callback(responseStatus: Map[TopicPartition, PartitionResponse]) = {
|
def callback(responseStatus: Map[TopicPartition, PartitionResponse]) = {
|
||||||
assert(responseStatus.values.head.errorCode == Errors.INVALID_REQUIRED_ACKS.code)
|
assert(responseStatus.values.head.errorCode == Errors.INVALID_REQUIRED_ACKS.code)
|
||||||
}
|
}
|
||||||
rm.appendMessages(
|
rm.appendRecords(
|
||||||
timeout = 0,
|
timeout = 0,
|
||||||
requiredAcks = 3,
|
requiredAcks = 3,
|
||||||
internalTopicsAllowed = false,
|
internalTopicsAllowed = false,
|
||||||
messagesPerPartition = Map(new TopicPartition("test1", 0) -> new ByteBufferMessageSet(new Message("first message".getBytes))),
|
entriesPerPartition = Map(new TopicPartition("test1", 0) -> MemoryRecords.withRecords(Record.create("first message".getBytes()))),
|
||||||
responseCallback = callback)
|
responseCallback = callback)
|
||||||
} finally {
|
} finally {
|
||||||
rm.shutdown(checkpointHW = false)
|
rm.shutdown(checkpointHW = false)
|
||||||
|
|
@ -135,7 +133,7 @@ class ReplicaManagerTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
var fetchCallbackFired = false
|
var fetchCallbackFired = false
|
||||||
def fetchCallback(responseStatus: Seq[(TopicAndPartition, FetchResponsePartitionData)]) = {
|
def fetchCallback(responseStatus: Seq[(TopicAndPartition, FetchPartitionData)]) = {
|
||||||
assertEquals("Should give NotLeaderForPartitionException", Errors.NOT_LEADER_FOR_PARTITION.code, responseStatus.map(_._2).head.error)
|
assertEquals("Should give NotLeaderForPartitionException", Errors.NOT_LEADER_FOR_PARTITION.code, responseStatus.map(_._2).head.error)
|
||||||
fetchCallbackFired = true
|
fetchCallbackFired = true
|
||||||
}
|
}
|
||||||
|
|
@ -158,11 +156,11 @@ class ReplicaManagerTest {
|
||||||
rm.getLeaderReplicaIfLocal(topic, 0)
|
rm.getLeaderReplicaIfLocal(topic, 0)
|
||||||
|
|
||||||
// Append a message.
|
// Append a message.
|
||||||
rm.appendMessages(
|
rm.appendRecords(
|
||||||
timeout = 1000,
|
timeout = 1000,
|
||||||
requiredAcks = -1,
|
requiredAcks = -1,
|
||||||
internalTopicsAllowed = false,
|
internalTopicsAllowed = false,
|
||||||
messagesPerPartition = Map(new TopicPartition(topic, 0) -> new ByteBufferMessageSet(new Message("first message".getBytes))),
|
entriesPerPartition = Map(new TopicPartition(topic, 0) -> MemoryRecords.withRecords(Record.create("first message".getBytes()))),
|
||||||
responseCallback = produceCallback)
|
responseCallback = produceCallback)
|
||||||
|
|
||||||
// Fetch some messages
|
// Fetch some messages
|
||||||
|
|
@ -220,19 +218,19 @@ class ReplicaManagerTest {
|
||||||
|
|
||||||
// Append a couple of messages.
|
// Append a couple of messages.
|
||||||
for(i <- 1 to 2)
|
for(i <- 1 to 2)
|
||||||
rm.appendMessages(
|
rm.appendRecords(
|
||||||
timeout = 1000,
|
timeout = 1000,
|
||||||
requiredAcks = -1,
|
requiredAcks = -1,
|
||||||
internalTopicsAllowed = false,
|
internalTopicsAllowed = false,
|
||||||
messagesPerPartition = Map(new TopicPartition(topic, 0) -> new ByteBufferMessageSet(new Message("message %d".format(i).getBytes))),
|
entriesPerPartition = Map(new TopicPartition(topic, 0) -> MemoryRecords.withRecords(Record.create("message %d".format(i).getBytes))),
|
||||||
responseCallback = produceCallback)
|
responseCallback = produceCallback)
|
||||||
|
|
||||||
var fetchCallbackFired = false
|
var fetchCallbackFired = false
|
||||||
var fetchError = 0
|
var fetchError = 0
|
||||||
var fetchedMessages: MessageSet = null
|
var fetchedRecords: Records = null
|
||||||
def fetchCallback(responseStatus: Seq[(TopicAndPartition, FetchResponsePartitionData)]) = {
|
def fetchCallback(responseStatus: Seq[(TopicAndPartition, FetchPartitionData)]) = {
|
||||||
fetchError = responseStatus.map(_._2).head.error
|
fetchError = responseStatus.map(_._2).head.error
|
||||||
fetchedMessages = responseStatus.map(_._2).head.messages
|
fetchedRecords = responseStatus.map(_._2).head.records
|
||||||
fetchCallbackFired = true
|
fetchCallbackFired = true
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -249,7 +247,7 @@ class ReplicaManagerTest {
|
||||||
|
|
||||||
assertTrue(fetchCallbackFired)
|
assertTrue(fetchCallbackFired)
|
||||||
assertEquals("Should not give an exception", Errors.NONE.code, fetchError)
|
assertEquals("Should not give an exception", Errors.NONE.code, fetchError)
|
||||||
assertTrue("Should return some data", fetchedMessages.iterator.hasNext)
|
assertTrue("Should return some data", fetchedRecords.shallowIterator.hasNext)
|
||||||
fetchCallbackFired = false
|
fetchCallbackFired = false
|
||||||
|
|
||||||
// Fetch a message above the high watermark as a consumer
|
// Fetch a message above the high watermark as a consumer
|
||||||
|
|
@ -264,7 +262,7 @@ class ReplicaManagerTest {
|
||||||
|
|
||||||
assertTrue(fetchCallbackFired)
|
assertTrue(fetchCallbackFired)
|
||||||
assertEquals("Should not give an exception", Errors.NONE.code, fetchError)
|
assertEquals("Should not give an exception", Errors.NONE.code, fetchError)
|
||||||
assertEquals("Should return empty response", MessageSet.Empty, fetchedMessages)
|
assertEquals("Should return empty response", MemoryRecords.EMPTY, fetchedRecords)
|
||||||
} finally {
|
} finally {
|
||||||
rm.shutdown(checkpointHW = false)
|
rm.shutdown(checkpointHW = false)
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -21,7 +21,6 @@ import kafka.utils._
|
||||||
import kafka.cluster.Replica
|
import kafka.cluster.Replica
|
||||||
import kafka.common.TopicAndPartition
|
import kafka.common.TopicAndPartition
|
||||||
import kafka.log.Log
|
import kafka.log.Log
|
||||||
import kafka.message.{ByteBufferMessageSet, Message, MessageSet}
|
|
||||||
import kafka.server.QuotaFactory.UnboundedQuota
|
import kafka.server.QuotaFactory.UnboundedQuota
|
||||||
import org.apache.kafka.common.metrics.Metrics
|
import org.apache.kafka.common.metrics.Metrics
|
||||||
import org.apache.kafka.common.requests.FetchRequest.PartitionData
|
import org.apache.kafka.common.requests.FetchRequest.PartitionData
|
||||||
|
|
@ -30,8 +29,10 @@ import java.util.Properties
|
||||||
import java.util.concurrent.atomic.AtomicBoolean
|
import java.util.concurrent.atomic.AtomicBoolean
|
||||||
|
|
||||||
import org.apache.kafka.common.TopicPartition
|
import org.apache.kafka.common.TopicPartition
|
||||||
|
import org.apache.kafka.common.record.{MemoryRecords, Record}
|
||||||
import org.easymock.EasyMock
|
import org.easymock.EasyMock
|
||||||
import org.junit.Assert._
|
import org.junit.Assert._
|
||||||
|
import scala.collection.JavaConverters._
|
||||||
|
|
||||||
class SimpleFetchTest {
|
class SimpleFetchTest {
|
||||||
|
|
||||||
|
|
@ -53,8 +54,8 @@ class SimpleFetchTest {
|
||||||
val partitionHW = 5
|
val partitionHW = 5
|
||||||
|
|
||||||
val fetchSize = 100
|
val fetchSize = 100
|
||||||
val messagesToHW = new Message("messageToHW".getBytes())
|
val messagesToHW = Record.create("messageToHW".getBytes())
|
||||||
val messagesToLEO = new Message("messageToLEO".getBytes())
|
val messagesToLEO = Record.create("messageToLEO".getBytes())
|
||||||
|
|
||||||
val topic = "test-topic"
|
val topic = "test-topic"
|
||||||
val partitionId = 0
|
val partitionId = 0
|
||||||
|
|
@ -79,14 +80,14 @@ class SimpleFetchTest {
|
||||||
EasyMock.expect(log.logEndOffset).andReturn(leaderLEO).anyTimes()
|
EasyMock.expect(log.logEndOffset).andReturn(leaderLEO).anyTimes()
|
||||||
EasyMock.expect(log.logEndOffsetMetadata).andReturn(new LogOffsetMetadata(leaderLEO)).anyTimes()
|
EasyMock.expect(log.logEndOffsetMetadata).andReturn(new LogOffsetMetadata(leaderLEO)).anyTimes()
|
||||||
EasyMock.expect(log.read(0, fetchSize, Some(partitionHW), true)).andReturn(
|
EasyMock.expect(log.read(0, fetchSize, Some(partitionHW), true)).andReturn(
|
||||||
new FetchDataInfo(
|
FetchDataInfo(
|
||||||
new LogOffsetMetadata(0L, 0L, 0),
|
new LogOffsetMetadata(0L, 0L, 0),
|
||||||
new ByteBufferMessageSet(messagesToHW)
|
MemoryRecords.withRecords(messagesToHW)
|
||||||
)).anyTimes()
|
)).anyTimes()
|
||||||
EasyMock.expect(log.read(0, fetchSize, None, true)).andReturn(
|
EasyMock.expect(log.read(0, fetchSize, None, true)).andReturn(
|
||||||
new FetchDataInfo(
|
FetchDataInfo(
|
||||||
new LogOffsetMetadata(0L, 0L, 0),
|
new LogOffsetMetadata(0L, 0L, 0),
|
||||||
new ByteBufferMessageSet(messagesToLEO)
|
MemoryRecords.withRecords(messagesToLEO)
|
||||||
)).anyTimes()
|
)).anyTimes()
|
||||||
EasyMock.replay(log)
|
EasyMock.replay(log)
|
||||||
|
|
||||||
|
|
@ -110,7 +111,7 @@ class SimpleFetchTest {
|
||||||
// create the follower replica with defined log end offset
|
// create the follower replica with defined log end offset
|
||||||
val followerReplica= new Replica(configs(1).brokerId, partition, time)
|
val followerReplica= new Replica(configs(1).brokerId, partition, time)
|
||||||
val leo = new LogOffsetMetadata(followerLEO, 0L, followerLEO.toInt)
|
val leo = new LogOffsetMetadata(followerLEO, 0L, followerLEO.toInt)
|
||||||
followerReplica.updateLogReadResult(new LogReadResult(FetchDataInfo(leo, MessageSet.Empty), -1L, -1, true))
|
followerReplica.updateLogReadResult(new LogReadResult(FetchDataInfo(leo, MemoryRecords.EMPTY), -1L, -1, true))
|
||||||
|
|
||||||
// add both of them to ISR
|
// add both of them to ISR
|
||||||
val allReplicas = List(leaderReplica, followerReplica)
|
val allReplicas = List(leaderReplica, followerReplica)
|
||||||
|
|
@ -153,7 +154,7 @@ class SimpleFetchTest {
|
||||||
fetchMaxBytes = Int.MaxValue,
|
fetchMaxBytes = Int.MaxValue,
|
||||||
hardMaxBytesLimit = false,
|
hardMaxBytesLimit = false,
|
||||||
readPartitionInfo = fetchInfo,
|
readPartitionInfo = fetchInfo,
|
||||||
quota = UnboundedQuota).find(_._1 == topicAndPartition).get._2.info.messageSet.head.message)
|
quota = UnboundedQuota).find(_._1 == topicAndPartition).get._2.info.records.shallowIterator.next().record)
|
||||||
assertEquals("Reading any data can return messages up to the end of the log", messagesToLEO,
|
assertEquals("Reading any data can return messages up to the end of the log", messagesToLEO,
|
||||||
replicaManager.readFromLocalLog(
|
replicaManager.readFromLocalLog(
|
||||||
replicaId = Request.OrdinaryConsumerId,
|
replicaId = Request.OrdinaryConsumerId,
|
||||||
|
|
@ -162,7 +163,7 @@ class SimpleFetchTest {
|
||||||
fetchMaxBytes = Int.MaxValue,
|
fetchMaxBytes = Int.MaxValue,
|
||||||
hardMaxBytesLimit = false,
|
hardMaxBytesLimit = false,
|
||||||
readPartitionInfo = fetchInfo,
|
readPartitionInfo = fetchInfo,
|
||||||
quota = UnboundedQuota).find(_._1 == topicAndPartition).get._2.info.messageSet.head.message)
|
quota = UnboundedQuota).find(_._1 == topicAndPartition).get._2.info.records.shallowIterator().next().record)
|
||||||
|
|
||||||
assertEquals("Counts should increment after fetch", initialTopicCount+2, BrokerTopicStats.getBrokerTopicStats(topic).totalFetchRequestRate.count())
|
assertEquals("Counts should increment after fetch", initialTopicCount+2, BrokerTopicStats.getBrokerTopicStats(topic).totalFetchRequestRate.count())
|
||||||
assertEquals("Counts should increment after fetch", initialAllTopicsCount+2, BrokerTopicStats.getBrokerAllTopicsStats().totalFetchRequestRate.count())
|
assertEquals("Counts should increment after fetch", initialAllTopicsCount+2, BrokerTopicStats.getBrokerAllTopicsStats().totalFetchRequestRate.count())
|
||||||
|
|
|
||||||
|
|
@ -21,7 +21,7 @@ import java.io._
|
||||||
import java.nio._
|
import java.nio._
|
||||||
import java.nio.channels._
|
import java.nio.channels._
|
||||||
import java.util.concurrent.{Callable, Executors, TimeUnit}
|
import java.util.concurrent.{Callable, Executors, TimeUnit}
|
||||||
import java.util.{Properties, Random}
|
import java.util.Properties
|
||||||
import java.security.cert.X509Certificate
|
import java.security.cert.X509Certificate
|
||||||
import javax.net.ssl.X509TrustManager
|
import javax.net.ssl.X509TrustManager
|
||||||
import charset.Charset
|
import charset.Charset
|
||||||
|
|
@ -48,6 +48,7 @@ import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, Produce
|
||||||
import org.apache.kafka.clients.consumer.{KafkaConsumer, RangeAssignor}
|
import org.apache.kafka.clients.consumer.{KafkaConsumer, RangeAssignor}
|
||||||
import org.apache.kafka.clients.CommonClientConfigs
|
import org.apache.kafka.clients.CommonClientConfigs
|
||||||
import org.apache.kafka.common.network.Mode
|
import org.apache.kafka.common.network.Mode
|
||||||
|
import org.apache.kafka.common.record._
|
||||||
import org.apache.kafka.common.serialization.{ByteArraySerializer, Serializer}
|
import org.apache.kafka.common.serialization.{ByteArraySerializer, Serializer}
|
||||||
import org.apache.kafka.common.utils.Time
|
import org.apache.kafka.common.utils.Time
|
||||||
import org.apache.kafka.test.{TestUtils => JTestUtils}
|
import org.apache.kafka.test.{TestUtils => JTestUtils}
|
||||||
|
|
@ -269,16 +270,16 @@ object TestUtils extends Logging {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Wrap the message in a message set
|
* Wrap a single record log buffer.
|
||||||
*
|
|
||||||
* @param payload The bytes of the message
|
|
||||||
*/
|
*/
|
||||||
def singleMessageSet(payload: Array[Byte],
|
def singletonRecords(value: Array[Byte],
|
||||||
codec: CompressionCodec = NoCompressionCodec,
|
|
||||||
key: Array[Byte] = null,
|
key: Array[Byte] = null,
|
||||||
timestamp: Long = Message.NoTimestamp,
|
codec: CompressionType = CompressionType.NONE,
|
||||||
magicValue: Byte = Message.CurrentMagicValue) =
|
timestamp: Long = Record.NO_TIMESTAMP,
|
||||||
new ByteBufferMessageSet(compressionCodec = codec, messages = new Message(payload, key, timestamp, magicValue))
|
magicValue: Byte = Record.CURRENT_MAGIC_VALUE) = {
|
||||||
|
val record = Record.create(magicValue, timestamp, key, value)
|
||||||
|
MemoryRecords.withRecords(codec, record)
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Generate an array of random bytes
|
* Generate an array of random bytes
|
||||||
|
|
|
||||||
Loading…
Reference in New Issue