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:
Jason Gustafson 2016-12-13 10:26:25 -08:00
parent 6626b058c7
commit 67f1e5b91b
89 changed files with 5276 additions and 3921 deletions

View File

@ -89,6 +89,7 @@
<allow pkg="net.jpountz" />
<allow pkg="org.apache.kafka.common.record" />
<allow pkg="org.apache.kafka.common.network" />
<allow pkg="org.apache.kafka.common.errors" />
</subpackage>
<subpackage name="requests">

View File

@ -61,6 +61,7 @@ import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Locale;
@ -686,11 +687,13 @@ public class Fetcher<K, V> {
}
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.
if (logEntry.offset() >= position) {
parsed.add(parseRecord(tp, logEntry));
bytes += logEntry.size();
bytes += logEntry.sizeInBytes();
}
}

View File

@ -198,7 +198,7 @@ public final class BufferPool {
* memory as free.
*
* @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
*/
public void deallocate(ByteBuffer buffer, int size) {

View File

@ -27,8 +27,10 @@ import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.metrics.stats.Rate;
import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.MemoryRecordsBuilder;
import org.apache.kafka.common.record.Record;
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.Time;
import org.apache.kafka.common.utils.Utils;
@ -49,7 +51,7 @@ import java.util.concurrent.ConcurrentMap;
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.
* <p>
* 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
*
* @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 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
@ -190,13 +192,13 @@ public final class RecordAccumulator {
free.deallocate(buffer);
return appendResult;
}
MemoryRecords records = MemoryRecords.emptyRecords(buffer, compression, this.batchSize);
RecordBatch batch = new RecordBatch(tp, records, time.milliseconds());
MemoryRecordsBuilder recordsBuilder = MemoryRecords.builder(buffer, compression, TimestampType.CREATE_TIME, this.batchSize);
RecordBatch batch = new RecordBatch(tp, recordsBuilder, time.milliseconds());
FutureRecordMetadata future = Utils.notNull(batch.tryAppend(timestamp, key, value, callback, time.milliseconds()));
dq.addLast(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 {
appendsInProgress.decrementAndGet();
@ -212,9 +214,9 @@ public final class RecordAccumulator {
if (last != null) {
FutureRecordMetadata future = last.tryAppend(timestamp, key, value, callback, time.milliseconds());
if (future == null)
last.records.close();
last.close();
else
return new RecordAppendResult(future, deque.size() > 1 || last.records.isFull(), false);
return new RecordAppendResult(future, deque.size() > 1 || last.isFull(), false);
}
return null;
}
@ -240,7 +242,7 @@ public final class RecordAccumulator {
Iterator<RecordBatch> batchIterator = dq.iterator();
while (batchIterator.hasNext()) {
RecordBatch batch = batchIterator.next();
boolean isFull = batch != lastBatch || batch.records.isFull();
boolean isFull = batch != lastBatch || batch.isFull();
// check if the batch is expired
if (batch.maybeExpire(requestTimeout, retryBackoffMs, now, this.lingerMs, isFull)) {
expiredBatches.add(batch);
@ -319,7 +321,7 @@ public final class RecordAccumulator {
long waitedTimeMs = nowMs - batch.lastAttemptMs;
long timeToWaitMs = backingOff ? retryBackoffMs : lingerMs;
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 sendable = full || expired || exhausted || closed || flushInProgress();
if (sendable && !backingOff) {
@ -389,15 +391,15 @@ public final class RecordAccumulator {
boolean backoff = first.attempts > 0 && first.lastAttemptMs + retryBackoffMs > now;
// Only drain the batch if it is not during backoff period.
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
// to compression; in this case we will still eventually send this batch in a single
// request
break;
} else {
RecordBatch batch = deque.pollFirst();
batch.records.close();
size += batch.records.sizeInBytes();
batch.close();
size += batch.sizeInBytes();
ready.add(batch);
batch.drainedMs = now;
}
@ -437,7 +439,7 @@ public final class RecordAccumulator {
*/
public void deallocate(RecordBatch 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);
// Close the batch before aborting
synchronized (dq) {
batch.records.close();
batch.close();
dq.remove(batch);
}
batch.done(-1L, Record.NO_TIMESTAMP, new IllegalStateException("Producer is closed forcefully."));

View File

@ -12,18 +12,20 @@
*/
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.RecordMetadata;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.MemoryRecordsBuilder;
import org.apache.kafka.common.record.Record;
import org.slf4j.Logger;
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.
*
@ -39,21 +41,21 @@ public final class RecordBatch {
public final long createdMs;
public long drainedMs;
public long lastAttemptMs;
public final MemoryRecords records;
public final TopicPartition topicPartition;
public final ProduceRequestResult produceFuture;
public long lastAppendTime;
private final List<Thunk> thunks;
private long offsetCounter = 0L;
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.lastAttemptMs = now;
this.records = records;
this.recordsBuilder = recordsBuilder;
this.topicPartition = tp;
this.produceFuture = new ProduceRequestResult();
this.thunks = new ArrayList<Thunk>();
this.thunks = new ArrayList<>();
this.lastAppendTime = createdMs;
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.
*/
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;
} 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.lastAppendTime = now;
FutureRecordMetadata future = new FutureRecordMetadata(this.produceFuture, this.recordCount,
@ -94,9 +96,8 @@ public final class RecordBatch {
baseOffset,
exception);
// execute callbacks
for (int i = 0; i < this.thunks.size(); i++) {
for (Thunk thunk : thunks) {
try {
Thunk thunk = this.thunks.get(i);
if (exception == null) {
// 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(),
@ -156,7 +157,7 @@ public final class RecordBatch {
}
if (expire) {
this.records.close();
close();
this.done(-1L, Record.NO_TIMESTAMP,
new TimeoutException("Expiring " + recordCount + " record(s) for " + topicPartition + " due to " + errorMessage));
}
@ -177,4 +178,37 @@ public final class RecordBatch {
public void setRetry() {
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();
}
}

View File

@ -345,7 +345,7 @@ public class Sender implements Runnable {
final Map<TopicPartition, RecordBatch> recordsByPartition = new HashMap<>(batches.size());
for (RecordBatch batch : batches) {
TopicPartition tp = batch.topicPartition;
produceRecordsByPartition.put(tp, batch.records);
produceRecordsByPartition.put(tp, batch.records());
recordsByPartition.put(tp, batch);
}
@ -505,17 +505,17 @@ public class Sender implements Runnable {
// per-topic bytes send rate
String topicByteRateName = "topic." + topic + ".bytes";
Sensor topicByteRate = Utils.notNull(this.metrics.getSensor(topicByteRateName));
topicByteRate.record(batch.records.sizeInBytes());
topicByteRate.record(batch.sizeInBytes());
// per-topic compression rate
String topicCompressionRateName = "topic." + topic + ".compression-rate";
Sensor topicCompressionRate = Utils.notNull(this.metrics.getSensor(topicCompressionRateName));
topicCompressionRate.record(batch.records.compressionRate());
topicCompressionRate.record(batch.compressionRate());
// global metrics
this.batchSizeSensor.record(batch.records.sizeInBytes(), now);
this.batchSizeSensor.record(batch.sizeInBytes(), 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);
records += batch.recordCount;
}

View File

@ -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();
}
};
}
}

View File

@ -16,34 +16,41 @@
*/
package org.apache.kafka.common.record;
import java.io.DataInputStream;
import java.io.InputStream;
import java.nio.ByteBuffer;
/**
* A byte buffer backed input inputStream
*/
public class ByteBufferInputStream extends InputStream {
private ByteBuffer buffer;
public class ByteBufferInputStream extends DataInputStream {
public ByteBufferInputStream(ByteBuffer buffer) {
this.buffer = buffer;
super(new UnderlyingInputStream(buffer));
}
public int read() {
if (!buffer.hasRemaining()) {
return -1;
}
return buffer.get() & 0xFF;
}
private static class UnderlyingInputStream extends InputStream {
private ByteBuffer buffer;
public int read(byte[] bytes, int off, int len) {
if (!buffer.hasRemaining()) {
return -1;
public UnderlyingInputStream(ByteBuffer buffer) {
this.buffer = buffer;
}
len = Math.min(len, buffer.remaining());
buffer.get(bytes, off, len);
return len;
public int read() {
if (!buffer.hasRemaining()) {
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;
}
}
}

View File

@ -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;
}
}
}

View File

@ -16,42 +16,54 @@
*/
package org.apache.kafka.common.record;
import java.io.DataOutputStream;
import java.io.OutputStream;
import java.nio.ByteBuffer;
/**
* 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 ByteBuffer buffer;
public ByteBufferOutputStream(ByteBuffer buffer) {
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);
super(new UnderlyingOutputStream(buffer));
}
public ByteBuffer buffer() {
return buffer;
return ((UnderlyingOutputStream) out).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;
public static class UnderlyingOutputStream extends OutputStream {
private ByteBuffer buffer;
public UnderlyingOutputStream(ByteBuffer buffer) {
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;
}
}
}

View File

@ -26,7 +26,7 @@ public enum CompressionType {
public final String name;
public final float rate;
private CompressionType(int id, String name, float rate) {
CompressionType(int id, String name, float rate) {
this.id = id;
this.name = name;
this.rate = rate;

View File

@ -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;
}
}
}

View File

@ -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;
}
}
}

View File

@ -18,22 +18,31 @@ package org.apache.kafka.common.record;
import org.apache.kafka.common.KafkaException;
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.FileInputStream;
import java.io.IOException;
import java.io.RandomAccessFile;
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
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 {
private final File file;
public class FileRecords extends AbstractRecords implements Closeable {
private final boolean isSlice;
private final FileChannel channel;
private final long start;
private final long end;
private final long size;
private final int start;
private final int end;
private volatile File file;
private final AtomicInteger size;
public FileRecords(File file,
FileChannel channel,
@ -44,83 +53,435 @@ public class FileRecords implements Records {
this.channel = channel;
this.start = start;
this.end = end;
this.isSlice = isSlice;
this.size = new AtomicInteger();
if (isSlice)
this.size = end - start;
else
this.size = Math.min(channel.size(), end) - start;
// set the initial size of the buffer
resize();
}
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
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
public long writeTo(GatheringByteChannel destChannel, long offset, int length) throws IOException {
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));
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 count = Math.min(length, this.size - offset);
long count = Math.min(length, size.get());
final long bytesTransferred;
if (destChannel instanceof TransportLayer) {
TransportLayer tl = (TransportLayer) destChannel;
return tl.transferFrom(this.channel, position, count);
bytesTransferred = tl.transferFrom(channel, position, count);
} 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
public RecordsIterator iterator() {
return new RecordsIterator(new FileLogInputStream(channel, start, end), false);
public Iterator<LogEntry> deepIterator() {
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 {
private long position;
protected final long end;
protected final FileChannel channel;
private final ByteBuffer logHeaderBuffer = ByteBuffer.allocate(Records.LOG_OVERHEAD);
public static FileRecords open(File file,
boolean mutable,
boolean fileAlreadyExists,
int initFileSize,
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) {
this.channel = channel;
this.position = start;
this.end = end;
public static FileRecords open(File file,
boolean fileAlreadyExists,
int initFileSize,
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
public LogEntry nextEntry() throws IOException {
if (position + Records.LOG_OVERHEAD >= end)
return null;
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
logHeaderBuffer.rewind();
channel.read(logHeaderBuffer, position);
if (logHeaderBuffer.hasRemaining())
return null;
LogEntryPosition that = (LogEntryPosition) o;
logHeaderBuffer.rewind();
long offset = logHeaderBuffer.getLong();
int size = logHeaderBuffer.getInt();
if (size < 0)
throw new IllegalStateException("Record with size " + size);
if (offset != that.offset) return false;
if (position != that.position) return false;
return size == that.size;
if (position + Records.LOG_OVERHEAD + size > end)
return null;
}
ByteBuffer recordBuffer = ByteBuffer.allocate(size);
channel.read(recordBuffer, position + Records.LOG_OVERHEAD);
if (recordBuffer.hasRemaining())
return null;
recordBuffer.rewind();
@Override
public int hashCode() {
int result = (int) (offset ^ (offset >>> 32));
result = 31 * result + position;
result = 31 * result + size;
return result;
}
Record record = new Record(recordBuffer);
LogEntry logEntry = new LogEntry(offset, record);
position += logEntry.size();
return logEntry;
@Override
public String toString() {
return "LogEntryPosition(" +
"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 +
')';
}
}
}

View File

@ -16,9 +16,9 @@
*/
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;

View File

@ -16,33 +16,156 @@
*/
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
*/
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;
this.record = record;
/**
* Get the shallow record for this log entry.
* @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
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);
}
}

View File

@ -20,10 +20,14 @@ import java.io.IOException;
/**
* 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}
* for the deep iteration.
* returns only the shallow log entries, depending on {@link RecordsIterator.DeepRecordsIterator}
* 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.
@ -31,5 +35,5 @@ interface LogInputStream {
* @return The next log entry or null if there is none
* @throws IOException for any IO errors
*/
LogEntry nextEntry() throws IOException;
T nextEntry() throws IOException;
}

View File

@ -12,197 +12,185 @@
*/
package org.apache.kafka.common.record;
import java.io.DataInputStream;
import org.apache.kafka.common.record.ByteBufferLogInputStream.ByteBufferLogEntry;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.GatheringByteChannel;
import java.util.ArrayList;
import java.util.Arrays;
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));
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
private ByteBuffer buffer;
// indicate if the memory records is writable or not (i.e. used for appends or read-only)
private boolean writable;
private int validBytes = -1;
// Construct a writable memory records
private MemoryRecords(ByteBuffer buffer, CompressionType type, boolean writable, int writeLimit) {
this.writable = writable;
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;
}
private MemoryRecords(ByteBuffer buffer) {
this.buffer = buffer;
}
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
public int sizeInBytes() {
if (writable) {
return compressor.buffer().position();
} else {
return buffer.limit();
}
return buffer.limit();
}
@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();
int position = (int) offset;
dup.position(position);
dup.limit(position + length);
int pos = (int) position;
dup.position(pos);
dup.limit(pos + length);
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() {
if (compressor == null)
return 1.0;
else
return compressor.compressionRate();
public int writeFullyTo(GatheringByteChannel channel) throws IOException {
buffer.mark();
int written = 0;
while (written < sizeInBytes())
written += channel.write(buffer);
buffer.reset();
return written;
}
/**
* Return the capacity of the initial buffer, for writable records
* it may be different from the current buffer's capacity
* The total number of bytes in this message set not including any partial, trailing messages. This
* may be smaller than what is returned by {@link #sizeInBytes()}.
* @return The number of valid bytes
*/
public int initialCapacity() {
return this.initialCapacity;
public int validBytes() {
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() {
if (writable)
throw new IllegalStateException("The memory records must not be writable any more before getting its underlying buffer");
return buffer.duplicate();
}
@Override
public Iterator<LogEntry> iterator() {
ByteBuffer input = this.buffer.duplicate();
if (writable)
// flip on a duplicate buffer for reading
input.flip();
return new RecordsIterator(new ByteBufferLogInputStream(input), false);
public Iterator<ByteBufferLogEntry> shallowIterator() {
return RecordsIterator.shallowIterator(new ByteBufferLogInputStream(buffer.duplicate(), Integer.MAX_VALUE));
}
@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
public String toString() {
Iterator<LogEntry> iter = iterator();
Iterator<LogEntry> iter = deepIterator();
StringBuilder builder = new StringBuilder();
builder.append('[');
while (iter.hasNext()) {
@ -214,16 +202,13 @@ public class MemoryRecords implements Records {
builder.append("record=");
builder.append(entry.record());
builder.append(")");
if (iter.hasNext())
builder.append(", ");
}
builder.append(']');
return builder.toString();
}
/** Visible for testing */
public boolean isWritable() {
return writable;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
@ -232,7 +217,6 @@ public class MemoryRecords implements Records {
MemoryRecords that = (MemoryRecords) o;
return buffer.equals(that.buffer);
}
@Override
@ -240,28 +224,153 @@ public class MemoryRecords implements Records {
return buffer.hashCode();
}
private static class ByteBufferLogInputStream implements LogInputStream {
private final DataInputStream stream;
private final ByteBuffer buffer;
public interface LogEntryFilter {
boolean shouldRetain(LogEntry entry);
}
private ByteBufferLogInputStream(ByteBuffer buffer) {
this.stream = new DataInputStream(new ByteBufferInputStream(buffer));
this.buffer = buffer;
}
public static class FilterResult {
public final int messagesRead;
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 {
long offset = stream.readLong();
int size = stream.readInt();
if (size < 0)
throw new IllegalStateException("Record with size " + size);
ByteBuffer slice = buffer.slice();
int newPos = buffer.position() + size;
if (newPos > buffer.limit())
return null;
buffer.position(newPos);
slice.limit(size);
return new LogEntry(offset, new Record(slice));
public FilterResult(int messagesRead,
int bytesRead,
int messagesRetained,
int bytesRetained,
long maxTimestamp,
long shallowOffsetOfMaxTimestamp) {
this.messagesRead = messagesRead;
this.bytesRead = bytesRead;
this.messagesRetained = messagesRetained;
this.bytesRetained = bytesRetained;
this.maxTimestamp = maxTimestamp;
this.shallowOffsetOfMaxTimestamp = shallowOffsetOfMaxTimestamp;
}
}
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;
}
}

View File

@ -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;
}
}
}

View File

@ -16,11 +16,15 @@
*/
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.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
@ -53,7 +57,12 @@ public final class 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
@ -79,11 +88,6 @@ public final class Record {
public static final byte TIMESTAMP_TYPE_MASK = 0x08;
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
*/
@ -94,155 +98,20 @@ public final class Record {
private final TimestampType wrapperRecordTimestampType;
public Record(ByteBuffer buffer) {
this.buffer = buffer;
this.wrapperRecordTimestamp = null;
this.wrapperRecordTimestampType = null;
this(buffer, null, null);
}
// Package private constructor for inner iteration.
Record(ByteBuffer buffer, Long wrapperRecordTimestamp, TimestampType wrapperRecordTimestampType) {
public Record(ByteBuffer buffer, Long wrapperRecordTimestamp, TimestampType wrapperRecordTimestampType) {
this.buffer = buffer;
this.wrapperRecordTimestamp = wrapperRecordTimestamp;
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
*/
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
*/
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() {
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 "
+ "small, size = " + size() + ")");
+ "small, size = " + sizeInBytes() + ")");
else
throw new InvalidRecordException("Record is corrupt (stored crc = " + checksum()
+ ", 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();
}
/**
* The length of the key in bytes
* @return the size in bytes of the key (0 if the key is null)
*/
public int keySize() {
if (magic() == MAGIC_VALUE_V0)
@ -292,6 +172,7 @@ public final class Record {
/**
* Does the record have a key?
* @return true if so, false otherwise
*/
public boolean hasKey() {
return keySize() >= 0;
@ -309,13 +190,23 @@ public final class Record {
/**
* The length of the value in bytes
* @return the size in bytes of the value (0 if the value is null)
*/
public int valueSize() {
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() {
return buffer.get(MAGIC_OFFSET);
@ -323,6 +214,7 @@ public final class Record {
/**
* The attributes stored with this record
* @return the attributes
*/
public byte attributes() {
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.
* 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
*
* @return the timestamp as determined above
*/
public long timestamp() {
if (magic() == MAGIC_VALUE_V0)
@ -349,6 +243,8 @@ public final class Record {
/**
* 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() {
if (magic() == 0)
@ -366,36 +262,30 @@ public final class 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() {
return sliceDelimited(valueSizeOffset());
return Utils.sizeDelimited(buffer, valueSizeOffset());
}
/**
* A ByteBuffer containing the message key
* @return the buffer or null if the key for this record is null
*/
public ByteBuffer key() {
if (magic() == MAGIC_VALUE_V0)
return sliceDelimited(KEY_SIZE_OFFSET_V0);
return Utils.sizeDelimited(buffer, KEY_SIZE_OFFSET_V0);
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) {
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;
}
public ByteBuffer buffer() {
return this.buffer;
}
public String toString() {
@ -434,4 +324,316 @@ public final class Record {
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;
}
}

View File

@ -18,32 +18,74 @@ package org.apache.kafka.common.record;
import java.io.IOException;
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}
* for the in-memory representation.
* Interface for accessing the records contained in a log. The log itself is represented as a sequence of log entries.
* 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 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
* @return The size in bytes
* The size of these records in bytes.
* @return The size in bytes of the records
*/
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 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
* @return The number of bytes written to the channel (which may be fewer than requested)
* @throws IOException For any IO errors copying the
* @return The number of bytes written
* @throws IOException For any IO errors
*/
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);
}

View File

@ -17,6 +17,7 @@
package org.apache.kafka.common.record;
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.Utils;
@ -25,51 +26,58 @@ import java.io.EOFException;
import java.io.IOException;
import java.nio.ByteBuffer;
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> {
private final LogInputStream logStream;
private final boolean shallow;
private final boolean ensureMatchingMagic;
private final int masRecordSize;
private final ShallowRecordsIterator<?> shallowIter;
private DeepRecordsIterator innerIter;
public RecordsIterator(LogInputStream logStream, boolean shallow) {
this.logStream = logStream;
public RecordsIterator(LogInputStream<?> logInputStream,
boolean shallow,
boolean ensureMatchingMagic,
int masRecordSize) {
this.shallowIter = new ShallowRecordsIterator<>(logInputStream);
this.shallow = shallow;
this.ensureMatchingMagic = ensureMatchingMagic;
this.masRecordSize = masRecordSize;
}
/*
* Read the next record from the buffer.
*
* Note that in the compressed message set, each message value size is set as the size of the un-compressed
* version of the message value, so when we do de-compression allocating an array of the specified size for
* reading compressed value data is sufficient.
/**
* Get a shallow iterator over the given input stream.
* @param logInputStream The log input stream to read the entries from
* @param <T> The type of the log entry
* @return The shallow iterator.
*/
public static <T extends LogEntry> Iterator<T> shallowIterator(LogInputStream<T> logInputStream) {
return new ShallowRecordsIterator<>(logInputStream);
}
@Override
protected LogEntry makeNext() {
if (innerDone()) {
try {
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) {
if (!shallowIter.hasNext())
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 {
return innerIter.next();
@ -80,38 +88,70 @@ public class RecordsIterator extends AbstractIterator<LogEntry> {
return innerIter == null || !innerIter.hasNext();
}
private static class DataLogInputStream implements LogInputStream {
private static class DataLogInputStream implements LogInputStream<LogEntry> {
private final DataInputStream stream;
protected final int maxMessageSize;
private DataLogInputStream(DataInputStream stream) {
DataLogInputStream(DataInputStream stream, int maxMessageSize) {
this.stream = stream;
this.maxMessageSize = maxMessageSize;
}
public LogEntry nextEntry() throws IOException {
long offset = stream.readLong();
int size = stream.readInt();
if (size < 0)
throw new IllegalStateException("Record with size " + size);
try {
long offset = stream.readLong();
int size = stream.readInt();
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];
stream.readFully(recordBuffer, 0, size);
ByteBuffer buf = ByteBuffer.wrap(recordBuffer);
return new LogEntry(offset, new Record(buf));
byte[] recordBuffer = new byte[size];
stream.readFully(recordBuffer, 0, size);
ByteBuffer buf = ByteBuffer.wrap(recordBuffer);
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 long absoluteBaseOffset;
private final byte wrapperMagic;
private DeepRecordsIterator(LogEntry entry) {
CompressionType compressionType = entry.record().compressionType();
ByteBuffer buffer = entry.record().value();
DataInputStream stream = Compressor.wrapForInput(new ByteBufferInputStream(buffer), compressionType, entry.record().magic());
LogInputStream logStream = new DataLogInputStream(stream);
public DeepRecordsIterator(LogEntry wrapperEntry, boolean ensureMatchingMagic, int maxMessageSize) {
Record wrapperRecord = wrapperEntry.record();
this.wrapperMagic = wrapperRecord.magic();
long wrapperRecordOffset = entry.offset();
long wrapperRecordTimestamp = entry.record().timestamp();
CompressionType compressionType = wrapperRecord.compressionType();
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<>();
// 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
try {
while (true) {
try {
LogEntry logEntry = logStream.nextEntry();
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) {
LogEntry logEntry = logStream.nextEntry();
if (logEntry == null)
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();
else
this.absoluteBaseOffset = -1;
@ -155,12 +200,10 @@ public class RecordsIterator extends AbstractIterator<LogEntry> {
// Convert offset to absolute offset if needed.
if (absoluteBaseOffset >= 0) {
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
CompressionType compression = entry.record().compressionType();
if (compression != CompressionType.NONE)
if (entry.isCompressed())
throw new InvalidRecordException("Inner messages must not be compressed");
return entry;

View File

@ -27,6 +27,7 @@ public enum TimestampType {
public final int id;
public final String name;
TimestampType(int id, String name) {
this.id = id;
this.name = name;

View File

@ -265,6 +265,24 @@ public class Utils {
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
*/
@ -733,4 +751,37 @@ public class Utils {
public static int longHashcode(long value) {
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();
}
}

View File

@ -39,6 +39,8 @@ import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.MemoryRecordsBuilder;
import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.common.requests.AbstractRequest;
import org.apache.kafka.common.requests.FetchResponse;
import org.apache.kafka.common.requests.FetchResponse.PartitionData;
@ -1323,11 +1325,10 @@ public class KafkaConsumerTest {
TopicPartition partition = fetchEntry.getKey();
long fetchOffset = fetchEntry.getValue().offset;
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++)
records.append(fetchOffset + i, 0L, ("key-" + i).getBytes(), ("value-" + i).getBytes());
records.close();
tpResponses.put(partition, new FetchResponse.PartitionData(Errors.NONE.code(), 0, records));
tpResponses.put(partition, new FetchResponse.PartitionData(Errors.NONE.code(), 0, records.build()));
}
return new FetchResponse(tpResponses, 0);
}

View File

@ -37,10 +37,12 @@ import org.apache.kafka.common.errors.TopicAuthorizationException;
import org.apache.kafka.common.metrics.KafkaMetric;
import org.apache.kafka.common.metrics.Metrics;
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.Compressor;
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.TimestampType;
import org.apache.kafka.common.requests.AbstractRequest;
import org.apache.kafka.common.requests.FetchRequest;
import org.apache.kafka.common.requests.FetchResponse;
@ -93,8 +95,8 @@ public class FetcherTest {
private static final double EPSILON = 0.0001;
private ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(client, metadata, time, 100, 1000);
private MemoryRecords records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE);
private MemoryRecords nextRecords = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE);
private MemoryRecords records;
private MemoryRecords nextRecords;
private Fetcher<byte[], byte[]> fetcher = createFetcher(subscriptions, metrics);
private Metrics fetcherMetrics = new Metrics(time);
private Fetcher<byte[], byte[]> fetcherNoAutoReset = createFetcher(subscriptionsNoAutoReset, fetcherMetrics);
@ -104,14 +106,16 @@ public class FetcherTest {
metadata.update(cluster, time.milliseconds());
client.setNode(node);
records.append(1L, 0L, "key".getBytes(), "value-1".getBytes());
records.append(2L, 0L, "key".getBytes(), "value-2".getBytes());
records.append(3L, 0L, "key".getBytes(), "value-3".getBytes());
records.close();
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME);
builder.append(1L, 0L, "key".getBytes(), "value-1".getBytes());
builder.append(2L, 0L, "key".getBytes(), "value-2".getBytes());
builder.append(3L, 0L, "key".getBytes(), "value-3".getBytes());
records = builder.build();
nextRecords.append(4L, 0L, "key".getBytes(), "value-4".getBytes());
nextRecords.append(5L, 0L, "key".getBytes(), "value-5".getBytes());
nextRecords.close();
builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME);
builder.append(4L, 0L, "key".getBytes(), "value-4".getBytes());
builder.append(5L, 0L, "key".getBytes(), "value-5".getBytes());
nextRecords = builder.build();
}
@After
@ -129,7 +133,7 @@ public class FetcherTest {
assertEquals(1, fetcher.sendFetches());
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);
assertTrue(fetcher.hasCompletedFetches());
@ -154,7 +158,7 @@ public class FetcherTest {
assertEquals(1, fetcher.sendFetches());
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);
assertTrue(fetcher.hasCompletedFetches());
@ -192,7 +196,7 @@ public class FetcherTest {
subscriptions.assignFromUser(singleton(tp));
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());
consumerClient.poll(0);
@ -206,29 +210,30 @@ public class FetcherTest {
}
@Test
public void testParseInvalidRecord() {
public void testParseInvalidRecord() throws Exception {
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[] value = "baz".getBytes();
long offset = 0;
long timestamp = 500L;
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
compressor.putLong(offset);
compressor.putInt(size);
Record.write(compressor, crc, Record.computeAttributes(CompressionType.NONE), timestamp, key, value, 0, -1);
out.writeLong(offset);
out.writeInt(size);
Record.write(out, magic, crc, Record.computeAttributes(magic, CompressionType.NONE, TimestampType.CREATE_TIME), timestamp, key, value);
// and one invalid record (note the crc)
compressor.putLong(offset);
compressor.putInt(size);
Record.write(compressor, crc + 1, Record.computeAttributes(CompressionType.NONE), timestamp, key, value, 0, -1);
out.writeLong(offset);
out.writeInt(size);
Record.write(out, magic, crc + 1, Record.computeAttributes(magic, CompressionType.NONE, TimestampType.CREATE_TIME), timestamp, key, value);
compressor.close();
buffer.flip();
subscriptions.assignFromUser(singleton(tp));
@ -236,7 +241,7 @@ public class FetcherTest {
// normal fetch
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);
try {
fetcher.fetchedRecords();
@ -255,8 +260,8 @@ public class FetcherTest {
subscriptions.assignFromUser(singleton(tp));
subscriptions.seek(tp, 1);
client.prepareResponse(matchesOffset(tp, 1), fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0));
client.prepareResponse(matchesOffset(tp, 4), fetchResponse(this.nextRecords.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, Errors.NONE.code(), 100L, 0));
assertEquals(1, fetcher.sendFetches());
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
// this test verifies the fetcher updates the current fetched/consumed positions correctly for this case
MemoryRecords records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE);
records.append(15L, 0L, "key".getBytes(), "value-1".getBytes());
records.append(20L, 0L, "key".getBytes(), "value-2".getBytes());
records.append(30L, 0L, "key".getBytes(), "value-3".getBytes());
records.close();
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME);
builder.append(15L, 0L, "key".getBytes(), "value-1".getBytes());
builder.append(20L, 0L, "key".getBytes(), "value-2".getBytes());
builder.append(30L, 0L, "key".getBytes(), "value-3".getBytes());
MemoryRecords records = builder.build();
List<ConsumerRecord<byte[], byte[]>> consumerRecords;
subscriptions.assignFromUser(singleton(tp));
@ -299,7 +304,7 @@ public class FetcherTest {
// normal fetch
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);
consumerRecords = fetcher.fetchedRecords().get(tp);
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
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);
try {
fetcher.fetchedRecords();
@ -337,7 +342,7 @@ public class FetcherTest {
// Now the rebalance happens and fetch positions are cleared
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);
// The active fetch should be ignored since its position is no longer valid
@ -352,7 +357,7 @@ public class FetcherTest {
assertEquals(1, fetcher.sendFetches());
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);
assertNull(fetcher.fetchedRecords().get(tp));
}
@ -373,7 +378,7 @@ public class FetcherTest {
subscriptions.seek(tp, 0);
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);
assertEquals(0, fetcher.fetchedRecords().size());
assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds()));
@ -385,7 +390,7 @@ public class FetcherTest {
subscriptions.seek(tp, 0);
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);
assertEquals(0, fetcher.fetchedRecords().size());
assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds()));
@ -397,7 +402,7 @@ public class FetcherTest {
subscriptions.seek(tp, 0);
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);
assertEquals(0, fetcher.fetchedRecords().size());
assertTrue(subscriptions.isOffsetResetNeeded(tp));
@ -412,7 +417,7 @@ public class FetcherTest {
subscriptions.seek(tp, 0);
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);
consumerClient.poll(0);
assertEquals(0, fetcher.fetchedRecords().size());
@ -426,7 +431,7 @@ public class FetcherTest {
subscriptionsNoAutoReset.seek(tp, 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);
assertFalse(subscriptionsNoAutoReset.isOffsetResetNeeded(tp));
subscriptionsNoAutoReset.seek(tp, 2);
@ -439,7 +444,7 @@ public class FetcherTest {
subscriptionsNoAutoReset.seek(tp, 0);
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);
assertFalse(subscriptionsNoAutoReset.isOffsetResetNeeded(tp));
@ -459,7 +464,7 @@ public class FetcherTest {
subscriptions.seek(tp, 0);
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);
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
// and filtered out by consumer.
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++) {
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());
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);
records = fetcher.fetchedRecords().get(tp);
assertEquals(3, records.size());
@ -722,8 +727,7 @@ public class FetcherTest {
return new ListOffsetResponse(allPartitionData, 1);
}
private FetchResponse fetchResponse(ByteBuffer buffer, short error, long hw, int throttleTime) {
MemoryRecords records = MemoryRecords.readableRecords(buffer);
private FetchResponse fetchResponse(MemoryRecords records, short error, long hw, int throttleTime) {
return new FetchResponse(Collections.singletonMap(tp, new FetchResponse.PartitionData(error, hw, records)), throttleTime);
}

View File

@ -12,23 +12,6 @@
*/
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.RecordMetadata;
import org.apache.kafka.common.Cluster;
@ -45,6 +28,23 @@ import org.apache.kafka.common.utils.Time;
import org.junit.After;
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 {
private String topic = "test";
@ -84,7 +84,7 @@ public class RecordAccumulatorTest {
accum.append(tp1, 0L, key, value, null, maxBlockTimeMs);
Deque<RecordBatch> partitionBatches = accum.batches().get(tp1);
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());
}
@ -93,15 +93,15 @@ public class RecordAccumulatorTest {
Deque<RecordBatch> partitionBatches = accum.batches().get(tp1);
assertEquals(2, partitionBatches.size());
Iterator<RecordBatch> partitionBatchesIterator = partitionBatches.iterator();
assertFalse(partitionBatchesIterator.next().records.isWritable());
assertTrue(partitionBatchesIterator.next().records.isWritable());
assertFalse(partitionBatchesIterator.next().isWritable());
assertTrue(partitionBatchesIterator.next().isWritable());
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());
assertEquals(1, batches.size());
RecordBatch batch = batches.get(0);
Iterator<LogEntry> iter = batch.records.iterator();
Iterator<LogEntry> iter = batch.records().deepIterator();
for (int i = 0; i < appends; i++) {
LogEntry entry = iter.next();
assertEquals("Keys should match", ByteBuffer.wrap(key), entry.record().key());
@ -130,7 +130,7 @@ public class RecordAccumulatorTest {
assertEquals(1, batches.size());
RecordBatch batch = batches.get(0);
Iterator<LogEntry> iter = batch.records.iterator();
Iterator<LogEntry> iter = batch.records().deepIterator();
LogEntry entry = iter.next();
assertEquals("Keys should match", ByteBuffer.wrap(key), entry.record().key());
assertEquals("Values should match", ByteBuffer.wrap(value), entry.record().value());
@ -159,7 +159,7 @@ public class RecordAccumulatorTest {
final int msgs = 10000;
final int numParts = 2;
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++) {
threads.add(new Thread() {
public void run() {
@ -182,8 +182,11 @@ public class RecordAccumulatorTest {
List<RecordBatch> batches = accum.drain(cluster, nodes, 5 * 1024, 0).get(node1.id());
if (batches != null) {
for (RecordBatch batch : batches) {
for (LogEntry entry : batch.records)
Iterator<LogEntry> deepEntries = batch.records().deepIterator();
while (deepEntries.hasNext()) {
deepEntries.next();
read++;
}
accum.deallocate(batch);
}
}

View File

@ -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);
}
}

View File

@ -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;
}
}

View File

@ -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;
}
}

View File

@ -16,53 +16,64 @@
*/
package org.apache.kafka.common.record;
import static org.apache.kafka.common.utils.Utils.toArray;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import org.apache.kafka.test.TestUtils;
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 org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import static java.util.Arrays.asList;
import static org.apache.kafka.common.utils.Utils.toNullableArray;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
@RunWith(value = Parameterized.class)
public class MemoryRecordsTest {
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.firstOffset = firstOffset;
}
@Test
public void testIterator() {
MemoryRecords recs1 = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), compression);
MemoryRecords recs2 = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), compression);
List<Record> list = Arrays.asList(new Record(0L, "a".getBytes(), "1".getBytes()),
new Record(0L, "b".getBytes(), "2".getBytes()),
new Record(0L, "c".getBytes(), "3".getBytes()));
MemoryRecordsBuilder builder1 = MemoryRecords.builder(ByteBuffer.allocate(1024), magic, compression, TimestampType.CREATE_TIME, firstOffset);
MemoryRecordsBuilder builder2 = MemoryRecords.builder(ByteBuffer.allocate(1024), magic, compression, TimestampType.CREATE_TIME, firstOffset);
List<Record> list = asList(
Record.create(magic, 1L, "a".getBytes(), "1".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++) {
Record r = list.get(i);
recs1.append(i, r);
recs2.append(i, 0L, toArray(r.key()), toArray(r.value()));
builder1.append(firstOffset + i, r);
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 (MemoryRecords recs : Arrays.asList(recs1, recs2)) {
Iterator<LogEntry> iter = recs.iterator();
for (MemoryRecords recs : asList(recs1, recs2)) {
Iterator<LogEntry> iter = recs.deepIterator();
for (int i = 0; i < list.size(); i++) {
assertTrue(iter.hasNext());
LogEntry entry = iter.next();
assertEquals((long) i, entry.offset());
assertEquals(firstOffset + i, entry.offset());
assertEquals(list.get(i), entry.record());
entry.record().ensureValid();
}
@ -73,20 +84,145 @@ public class MemoryRecordsTest {
@Test
public void testHasRoomForMethod() {
MemoryRecords recs1 = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), compression);
recs1.append(0, new Record(0L, "a".getBytes(), "1".getBytes()));
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), magic, compression, TimestampType.CREATE_TIME);
builder.append(0, Record.create(magic, 0L, "a".getBytes(), "1".getBytes()));
assertTrue(recs1.hasRoomFor("b".getBytes(), "2".getBytes()));
recs1.close();
assertFalse(recs1.hasRoomFor("b".getBytes(), "2".getBytes()));
assertTrue(builder.hasRoomFor("b".getBytes(), "2".getBytes()));
builder.close();
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
public static Collection<Object[]> data() {
List<Object[]> values = new ArrayList<Object[]>();
for (CompressionType type: CompressionType.values())
values.add(new Object[] {type});
List<Object[]> values = new ArrayList<>();
for (long firstOffset : asList(0L, 57L))
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;
}
private static class RetainNonNullKeysFilter implements MemoryRecords.LogEntryFilter {
@Override
public boolean shouldRetain(LogEntry entry) {
return entry.record().hasKey();
}
}
}

View File

@ -35,18 +35,22 @@ import org.junit.runners.Parameterized.Parameters;
@RunWith(value = Parameterized.class)
public class RecordTest {
private byte magic;
private long timestamp;
private ByteBuffer key;
private ByteBuffer value;
private CompressionType compression;
private TimestampType timestampType;
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.timestampType = TimestampType.CREATE_TIME;
this.key = key == null ? null : ByteBuffer.wrap(key);
this.value = value == null ? null : ByteBuffer.wrap(value);
this.compression = compression;
this.record = new Record(timestamp, key, value, compression);
this.record = Record.create(magic, timestamp, key, value, compression, timestampType);
}
@Test
@ -56,22 +60,33 @@ public class RecordTest {
assertEquals(key, record.key());
if (key != null)
assertEquals(key.limit(), record.keySize());
assertEquals(Record.CURRENT_MAGIC_VALUE, record.magic());
assertEquals(magic, record.magic());
assertEquals(value, record.value());
if (value != null)
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
public void testChecksum() {
assertEquals(record.checksum(), record.computeChecksum());
byte attributes = Record.computeAttributes(magic, this.compression, TimestampType.CREATE_TIME);
assertEquals(record.checksum(), Record.computeChecksum(
this.timestamp,
this.key == null ? null : this.key.array(),
this.value == null ? null : this.value.array(),
this.compression, 0, -1));
magic,
attributes,
this.timestamp,
this.key == null ? null : this.key.array(),
this.value == null ? null : this.value.array()
));
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);
copy.buffer().put(i, (byte) 69);
assertFalse(copy.isValid());
@ -85,7 +100,7 @@ public class RecordTest {
}
private Record copyOf(Record record) {
ByteBuffer buffer = ByteBuffer.allocate(record.size());
ByteBuffer buffer = ByteBuffer.allocate(record.sizeInBytes());
record.buffer().put(buffer);
buffer.rewind();
record.buffer().rewind();
@ -101,12 +116,13 @@ public class RecordTest {
public static Collection<Object[]> data() {
byte[] payload = new byte[1000];
Arrays.fill(payload, (byte) 1);
List<Object[]> values = new ArrayList<Object[]>();
for (long timestamp : Arrays.asList(Record.NO_TIMESTAMP, 0L, 1L))
for (byte[] key : Arrays.asList(null, "".getBytes(), "key".getBytes(), payload))
for (byte[] value : Arrays.asList(null, "".getBytes(), "value".getBytes(), payload))
for (CompressionType compression : CompressionType.values())
values.add(new Object[] {timestamp, key, value, compression});
List<Object[]> values = new ArrayList<>();
for (byte magic : Arrays.asList(Record.MAGIC_VALUE_V0, Record.MAGIC_VALUE_V1))
for (long timestamp : Arrays.asList(Record.NO_TIMESTAMP, 0L, 1L))
for (byte[] key : Arrays.asList(null, "".getBytes(), "key".getBytes(), payload))
for (byte[] value : Arrays.asList(null, "".getBytes(), "value".getBytes(), payload))
for (CompressionType compression : CompressionType.values())
values.add(new Object[] {magic, timestamp, key, value, compression});
return values;
}

View File

@ -20,35 +20,29 @@ import org.junit.Test;
import java.nio.ByteBuffer;
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 SimpleRecordTest {
/* 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() {
ByteBuffer buffer = ByteBuffer.allocate(2);
Record record = new Record(buffer);
assertFalse(record.isValid());
try {
record.ensureValid();
fail("InvalidRecordException should have been thrown");
} catch (InvalidRecordException e) { }
record.ensureValid();
}
@Test
@Test(expected = InvalidRecordException.class)
public void testIsValidWithChecksumMismatch() {
ByteBuffer buffer = ByteBuffer.allocate(4);
// set checksum
buffer.putInt(2);
Record record = new Record(buffer);
assertFalse(record.isValid());
try {
record.ensureValid();
fail("InvalidRecordException should have been thrown");
} catch (InvalidRecordException e) { }
record.ensureValid();
}
@Test
@ -63,4 +57,40 @@ public class SimpleRecordTest {
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());
}
}
}

View File

@ -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));
}
}

View File

@ -23,8 +23,10 @@ import org.apache.kafka.common.Node;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.MemoryRecordsBuilder;
import org.apache.kafka.common.record.Record;
import org.apache.kafka.common.record.Records;
import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.common.utils.Utils;
import javax.xml.bind.DatatypeConverter;
@ -35,6 +37,7 @@ import java.nio.file.Files;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Properties;
@ -46,6 +49,7 @@ import java.util.regex.Pattern;
import static java.util.Arrays.asList;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
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) {
int bufferSize = 0;
for (final Record record : records)
bufferSize += Records.LOG_OVERHEAD + record.size();
bufferSize += Records.LOG_OVERHEAD + record.sizeInBytes();
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)
memoryRecords.append(offset, record);
memoryRecords.close();
return memoryRecords.buffer();
builder.append(nextOffset++, record);
return builder.build().buffer();
}
public static Properties producerConfig(final String bootstrapServers,
@ -309,4 +313,22 @@ public class TestUtils {
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;
}
}

View File

@ -17,7 +17,7 @@
package kafka.api
import kafka.message.Message
import org.apache.kafka.common.record.Record
/**
* This class contains the different Kafka versions.
@ -87,54 +87,54 @@ sealed trait ApiVersion extends Ordered[ApiVersion] {
// Keep the IDs in order of versions
case object KAFKA_0_8_0 extends ApiVersion {
val version: String = "0.8.0.X"
val messageFormatVersion: Byte = Message.MagicValue_V0
val messageFormatVersion: Byte = Record.MAGIC_VALUE_V0
val id: Int = 0
}
case object KAFKA_0_8_1 extends ApiVersion {
val version: String = "0.8.1.X"
val messageFormatVersion: Byte = Message.MagicValue_V0
val messageFormatVersion: Byte = Record.MAGIC_VALUE_V0
val id: Int = 1
}
case object KAFKA_0_8_2 extends ApiVersion {
val version: String = "0.8.2.X"
val messageFormatVersion: Byte = Message.MagicValue_V0
val messageFormatVersion: Byte = Record.MAGIC_VALUE_V0
val id: Int = 2
}
case object KAFKA_0_9_0 extends ApiVersion {
val version: String = "0.9.0.X"
val messageFormatVersion: Byte = Message.MagicValue_V0
val messageFormatVersion: Byte = Record.MAGIC_VALUE_V0
val id: Int = 3
}
case object KAFKA_0_10_0_IV0 extends ApiVersion {
val version: String = "0.10.0-IV0"
val messageFormatVersion: Byte = Message.MagicValue_V1
val messageFormatVersion: Byte = Record.MAGIC_VALUE_V1
val id: Int = 4
}
case object KAFKA_0_10_0_IV1 extends ApiVersion {
val version: String = "0.10.0-IV1"
val messageFormatVersion: Byte = Message.MagicValue_V1
val messageFormatVersion: Byte = Record.MAGIC_VALUE_V1
val id: Int = 5
}
case object KAFKA_0_10_1_IV0 extends ApiVersion {
val version: String = "0.10.1-IV0"
val messageFormatVersion: Byte = Message.MagicValue_V1
val messageFormatVersion: Byte = Record.MAGIC_VALUE_V1
val id: Int = 6
}
case object KAFKA_0_10_1_IV1 extends ApiVersion {
val version: String = "0.10.1-IV1"
val messageFormatVersion: Byte = Message.MagicValue_V1
val messageFormatVersion: Byte = Record.MAGIC_VALUE_V1
val id: Int = 7
}
case object KAFKA_0_10_1_IV2 extends ApiVersion {
val version: String = "0.10.1-IV2"
val messageFormatVersion: Byte = Message.MagicValue_V1
val messageFormatVersion: Byte = Record.MAGIC_VALUE_V1
val id: Int = 8
}

View File

@ -25,7 +25,6 @@ import kafka.log.LogConfig
import kafka.server._
import kafka.metrics.KafkaMetricsGroup
import kafka.controller.KafkaController
import kafka.message.ByteBufferMessageSet
import java.io.IOException
import java.util.concurrent.locks.ReentrantReadWriteLock
@ -34,6 +33,7 @@ import org.apache.kafka.common.protocol.Errors
import scala.collection.JavaConverters._
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.utils.Time
@ -190,7 +190,7 @@ class Partition(val topic: String,
allReplicas.foreach(replica => getOrCreateReplica(replica))
val newInSyncReplicas = partitionStateInfo.isr.asScala.map(r => getOrCreateReplica(r)).toSet
// remove assigned replicas that have been removed by the controller
(assignedReplicas().map(_.brokerId) -- allReplicas).foreach(removeReplica(_))
(assignedReplicas().map(_.brokerId) -- allReplicas).foreach(removeReplica)
inSyncReplicas = newInSyncReplicas
leaderEpoch = partitionStateInfo.leaderEpoch
zkVersion = partitionStateInfo.zkVersion
@ -440,7 +440,7 @@ class Partition(val topic: String,
laggingReplicas
}
def appendMessagesToLeader(messages: ByteBufferMessageSet, requiredAcks: Int = 0) = {
def appendRecordsToLeader(records: MemoryRecords, requiredAcks: Int = 0) = {
val (info, leaderHWIncremented) = inReadLock(leaderIsrUpdateLock) {
val leaderReplicaOpt = leaderReplicaIfLocal()
leaderReplicaOpt match {
@ -455,7 +455,7 @@ class Partition(val topic: String,
.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
replicaManager.tryCompleteDelayedFetch(TopicPartitionOperationKey(this.topic, this.partitionId))
// 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)
if(updateSucceeded) {
replicaManager.recordIsrChange(new TopicAndPartition(topic, partitionId))
replicaManager.recordIsrChange(TopicAndPartition(topic, partitionId))
inSyncReplicas = newIsr
zkVersion = newVersion
trace("ISR updated to [%s] and zkVersion updated to [%d]".format(newIsr.mkString(","), zkVersion))

View File

@ -17,14 +17,16 @@
package kafka.consumer
import kafka.api.{OffsetRequest, Request, FetchRequestBuilder, FetchResponsePartitionData}
import kafka.api.{FetchRequestBuilder, FetchResponsePartitionData, OffsetRequest, Request}
import kafka.cluster.BrokerEndPoint
import kafka.message.ByteBufferMessageSet
import kafka.server.{PartitionFetchState, AbstractFetcherThread}
import kafka.server.{AbstractFetcherThread, PartitionFetchState}
import kafka.common.{ErrorMapping, TopicAndPartition}
import scala.collection.Map
import ConsumerFetcherThread._
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.record.MemoryRecords
class ConsumerFetcherThread(name: String,
val config: ConsumerConfig,
@ -81,7 +83,7 @@ class ConsumerFetcherThread(name: String,
case OffsetRequest.LargestTimeString => 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 pti = partitionMap(topicPartition)
pti.resetFetchOffset(newOffset)
@ -123,7 +125,7 @@ object ConsumerFetcherThread {
class PartitionData(val underlying: FetchResponsePartitionData) extends AbstractFetcherThread.PartitionData {
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 exception: Option[Throwable] =
if (errorCode == ErrorMapping.NoError) None else Some(ErrorMapping.exceptionFor(errorCode))

View File

@ -458,14 +458,14 @@ class GroupCoordinator(val brokerId: Int,
def handleFetchOffsets(groupId: String,
partitions: Seq[TopicPartition]): Map[TopicPartition, OffsetFetchResponse.PartitionData] = {
if (!isActive.get) {
partitions.map { case topicPartition =>
partitions.map { topicPartition =>
(topicPartition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code))}.toMap
} else if (!isCoordinatorForGroup(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
} 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
} else {
// return offsets blindly regardless the current group state since the group may be using

View File

@ -17,38 +17,31 @@
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.nio.ByteBuffer
import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent.TimeUnit
import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent.locks.ReentrantLock
import com.yammer.metrics.core.Gauge
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._
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,
val interBrokerProtocolVersion: ApiVersion,
@ -57,6 +50,8 @@ class GroupMetadataManager(val brokerId: Int,
zkUtils: ZkUtils,
time: Time) extends Logging with KafkaMetricsGroup {
private val compressionType: CompressionType = CompressionType.forId(config.offsetsTopicCompressionCodec.codec)
private val groupMetadataCache = new Pool[String, GroupMetadata]
/* lock protecting access to loading and owned partition sets */
@ -135,13 +130,11 @@ class GroupMetadataManager(val brokerId: Int,
}
}
def prepareStoreGroup(group: GroupMetadata,
groupAssignment: Map[String, Array[Byte]],
responseCallback: Errors => Unit): Option[DelayedStore] = {
val magicValueAndTimestampOpt = getMessageFormatVersionAndTimestamp(partitionFor(group.groupId))
magicValueAndTimestampOpt match {
case Some((magicValue, timestamp)) =>
getMagicAndTimestamp(partitionFor(group.groupId)) match {
case Some((magicValue, timestampType, timestamp)) =>
val groupMetadataValueVersion = {
if (interBrokerProtocolVersion < KAFKA_0_10_1_IV0)
0.toShort
@ -149,17 +142,12 @@ class GroupMetadataManager(val brokerId: Int,
GroupMetadataManager.CURRENT_GROUP_VALUE_SCHEMA_VERSION
}
val message = new Message(
key = GroupMetadataManager.groupMetadataKey(group.groupId),
bytes = GroupMetadataManager.groupMetadataValue(group, groupAssignment, version = groupMetadataValueVersion),
timestamp = timestamp,
magicValue = magicValue)
val record = Record.create(magicValue, timestampType, timestamp,
GroupMetadataManager.groupMetadataKey(group.groupId),
GroupMetadataManager.groupMetadataValue(group, groupAssignment, version = groupMetadataValueVersion))
val groupMetadataPartition = new TopicPartition(Topic.GroupMetadataTopicName, partitionFor(group.groupId))
val groupMetadataMessageSet = Map(groupMetadataPartition ->
new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, message))
val groupMetadataRecords = Map(groupMetadataPartition -> MemoryRecords.withRecords(timestampType, compressionType, record))
val generationId = group.generationId
// 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)
}
Some(DelayedStore(groupMetadataMessageSet, putCacheCallback))
Some(DelayedStore(groupMetadataRecords, putCacheCallback))
case None =>
responseCallback(Errors.NOT_COORDINATOR_FOR_GROUP)
@ -222,11 +210,11 @@ class GroupMetadataManager(val brokerId: Int,
def store(delayedStore: DelayedStore) {
// call replica manager to append the group message
replicaManager.appendMessages(
replicaManager.appendRecords(
config.offsetCommitTimeoutMs.toLong,
config.offsetCommitRequiredAcks,
true, // allow appending to internal offset topic
delayedStore.messageSet,
delayedStore.partitionRecords,
delayedStore.callback)
}
@ -244,22 +232,17 @@ class GroupMetadataManager(val brokerId: Int,
}
// construct the message set to append
val magicValueAndTimestampOpt = getMessageFormatVersionAndTimestamp(partitionFor(group.groupId))
magicValueAndTimestampOpt match {
case Some((magicValue, timestamp)) =>
val messages = filteredOffsetMetadata.map { case (topicAndPartition, offsetAndMetadata) =>
new Message(
key = GroupMetadataManager.offsetCommitKey(group.groupId, topicAndPartition.topic, topicAndPartition.partition),
bytes = GroupMetadataManager.offsetCommitValue(offsetAndMetadata),
timestamp = timestamp,
magicValue = magicValue
)
getMagicAndTimestamp(partitionFor(group.groupId)) match {
case Some((magicValue, timestampType, timestamp)) =>
val records = filteredOffsetMetadata.map { case (topicAndPartition, offsetAndMetadata) =>
Record.create(magicValue, timestampType, timestamp,
GroupMetadataManager.offsetCommitKey(group.groupId, topicAndPartition.topic, topicAndPartition.partition),
GroupMetadataManager.offsetCommitValue(offsetAndMetadata))
}.toSeq
val offsetTopicPartition = new TopicPartition(Topic.GroupMetadataTopicName, partitionFor(group.groupId))
val offsetsAndMetadataMessageSet = Map(offsetTopicPartition ->
new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages:_*))
val entries = Map(offsetTopicPartition -> MemoryRecords.withRecords(timestampType, compressionType, records:_*))
// set the callback function to insert offsets into cache after log append completed
def putCacheCallback(responseStatus: Map[TopicPartition, PartitionResponse]) {
@ -330,7 +313,7 @@ class GroupMetadataManager(val brokerId: Int,
group.prepareOffsetCommit(offsetMetadata)
}
Some(DelayedStore(offsetsAndMetadataMessageSet, putCacheCallback))
Some(DelayedStore(entries, putCacheCallback))
case None =>
val commitStatus = offsetMetadata.map { case (topicAndPartition, offsetAndMetadata) =>
@ -412,28 +395,30 @@ class GroupMetadataManager(val brokerId: Int,
while (currOffset < getHighWatermark(offsetsPartition) && !shuttingDown.get()) {
buffer.clear()
val messages = log.read(currOffset, config.loadBufferSize, minOneMessage = true).messageSet.asInstanceOf[FileMessageSet]
messages.readInto(buffer, 0)
val messageSet = new ByteBufferMessageSet(buffer)
messageSet.foreach { msgAndOffset =>
require(msgAndOffset.message.key != null, "Offset entry key should not be null")
val baseKey = GroupMetadataManager.readMessageKey(msgAndOffset.message.key)
val fileRecords = log.read(currOffset, config.loadBufferSize, minOneMessage = true).records.asInstanceOf[FileRecords]
fileRecords.readInto(buffer, 0)
MemoryRecords.readableRecords(buffer).deepIterator.asScala.foreach { entry =>
val record = entry.record
require(record.hasKey, "Offset entry key should not be null")
val baseKey = GroupMetadataManager.readMessageKey(record.key)
if (baseKey.isInstanceOf[OffsetKey]) {
// load offset
val key = baseKey.key.asInstanceOf[GroupTopicPartition]
if (msgAndOffset.message.payload == null) {
if (record.hasNullValue) {
loadedOffsets.remove(key)
removedOffsets.add(key)
} else {
val value = GroupMetadataManager.readOffsetMessageValue(msgAndOffset.message.payload)
val value = GroupMetadataManager.readOffsetMessageValue(record.value)
loadedOffsets.put(key, value)
removedOffsets.remove(key)
}
} else {
// load group metadata
val groupId = baseKey.key.asInstanceOf[String]
val groupMetadata = GroupMetadataManager.readGroupMessageValue(groupId, msgAndOffset.message.payload)
val groupMetadata = GroupMetadataManager.readGroupMessageValue(groupId, record.value)
if (groupMetadata != null) {
trace(s"Loaded group metadata for group ${groupMetadata.groupId} with generation ${groupMetadata.generationId}")
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 =>
if (groupMetadataCache.contains(groupId))
throw new IllegalStateException(s"Unexpected unload of active group ${groupId} while " +
s"loading partition ${topicPartition}")
throw new IllegalStateException(s"Unexpected unload of active group $groupId while " +
s"loading partition $topicPartition")
}
if (!shuttingDown.get())
@ -572,15 +557,15 @@ class GroupMetadataManager(val brokerId: Int,
}
val offsetsPartition = partitionFor(groupId)
getMessageFormatVersionAndTimestamp(offsetsPartition) match {
case Some((magicValue, timestamp)) =>
getMagicAndTimestamp(offsetsPartition) match {
case Some((magicValue, timestampType, timestamp)) =>
val partitionOpt = replicaManager.getPartition(Topic.GroupMetadataTopicName, offsetsPartition)
partitionOpt.foreach { partition =>
val appendPartition = TopicAndPartition(Topic.GroupMetadataTopicName, offsetsPartition)
val tombstones = expiredOffsets.map { case (topicPartition, offsetAndMetadata) =>
trace(s"Removing expired offset and metadata for $groupId, $topicPartition: $offsetAndMetadata")
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
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,
// if we crash or leaders move) since the new leaders will still expire the consumers with heartbeat and
// retry removing this group.
tombstones += new Message(bytes = null, key = GroupMetadataManager.groupMetadataKey(group.groupId),
timestamp = timestamp, magicValue = magicValue)
tombstones += Record.create(magicValue, timestampType, timestamp, GroupMetadataManager.groupMetadataKey(group.groupId), null)
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 {
// do not need to require acks since even if the tombstone is lost,
// 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
trace(s"Successfully appended ${tombstones.size} tombstones to $appendPartition for expired offsets and/or metadata for group $groupId")
} catch {
@ -663,16 +647,11 @@ class GroupMetadataManager(val brokerId: Int,
* @param partition Partition of GroupMetadataTopic
* @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)
replicaManager.getMessageFormatVersion(groupMetadataTopicAndPartition).map { messageFormatVersion =>
val timestamp = {
if (messageFormatVersion == Message.MagicValue_V0)
Message.NoTimestamp
else
time.milliseconds()
}
(messageFormatVersion, timestamp)
replicaManager.getMagicAndTimestampType(groupMetadataTopicAndPartition).map { case (messageFormatVersion, timestampType) =>
val timestamp = if (messageFormatVersion == Record.MAGIC_VALUE_V0) Record.NO_TIMESTAMP else time.milliseconds()
(messageFormatVersion, timestampType, timestamp)
}
}
@ -964,7 +943,7 @@ object GroupMetadataManager {
* @return an offset-metadata object from the message
*/
def readOffsetMessageValue(buffer: ByteBuffer): OffsetAndMetadata = {
if(buffer == null) { // tombstone
if (buffer == null) { // tombstone
null
} else {
val version = buffer.getShort
@ -997,7 +976,7 @@ object GroupMetadataManager {
* @return a group metadata object from the message
*/
def readGroupMessageValue(groupId: String, buffer: ByteBuffer): GroupMetadata = {
if(buffer == null) { // tombstone
if (buffer == null) { // tombstone
null
} else {
val version = buffer.getShort
@ -1016,23 +995,22 @@ object GroupMetadataManager {
group.leaderId = value.get(LEADER_KEY).asInstanceOf[String]
group.protocol = value.get(PROTOCOL_KEY).asInstanceOf[String]
memberMetadataArray.foreach {
case memberMetadataObj =>
val memberMetadata = memberMetadataObj.asInstanceOf[Struct]
val memberId = memberMetadata.get(MEMBER_ID_KEY).asInstanceOf[String]
val clientId = memberMetadata.get(CLIENT_ID_KEY).asInstanceOf[String]
val clientHost = memberMetadata.get(CLIENT_HOST_KEY).asInstanceOf[String]
val sessionTimeout = memberMetadata.get(SESSION_TIMEOUT_KEY).asInstanceOf[Int]
val rebalanceTimeout = if (version == 0) sessionTimeout else memberMetadata.get(REBALANCE_TIMEOUT_KEY).asInstanceOf[Int]
memberMetadataArray.foreach { memberMetadataObj =>
val memberMetadata = memberMetadataObj.asInstanceOf[Struct]
val memberId = memberMetadata.get(MEMBER_ID_KEY).asInstanceOf[String]
val clientId = memberMetadata.get(CLIENT_ID_KEY).asInstanceOf[String]
val clientHost = memberMetadata.get(CLIENT_HOST_KEY).asInstanceOf[String]
val sessionTimeout = memberMetadata.get(SESSION_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,
protocolType, List((group.protocol, subscription)))
val member = new MemberMetadata(memberId, groupId, clientId, clientHost, rebalanceTimeout, sessionTimeout,
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
@ -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)
case class GroupTopicPartition(group: String, topicPartition: TopicPartition) {

View File

@ -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))
}

View File

@ -19,7 +19,6 @@ package kafka.log
import kafka.api.KAFKA_0_10_0_IV0
import kafka.utils._
import kafka.message._
import kafka.common._
import kafka.metrics.KafkaMetricsGroup
import kafka.server.{BrokerTopicStats, FetchDataInfo, LogOffsetMetadata}
@ -29,16 +28,18 @@ import java.util.concurrent.atomic._
import java.text.NumberFormat
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 scala.collection.Seq
import scala.collection.JavaConverters._
import com.yammer.metrics.core.Gauge
import org.apache.kafka.common.utils.{Time, Utils}
import kafka.message.{BrokerCompressionCodec, CompressionCodec, NoCompressionCodec}
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 timeIndexFile = new File(CoreUtils.replaceSuffix(logFile.getPath, LogFileSuffix, TimeIndexFileSuffix) + SwapFileSuffix)
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,
timeIndex = timeIndex,
baseOffset = startOffset,
@ -338,20 +339,20 @@ class Log(@volatile var dir: File,
* 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.
*
* @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
* @throws KafkaStorageException If the append fails due to an I/O error.
* @return Information about the appended messages including the first and last offset.
*/
def append(messages: ByteBufferMessageSet, assignOffsets: Boolean = true): LogAppendInfo = {
val appendInfo = analyzeAndValidateMessageSet(messages)
def append(records: MemoryRecords, assignOffsets: Boolean = true): LogAppendInfo = {
val appendInfo = analyzeAndValidateRecords(records)
// if we have any valid messages, append them to the log
if (appendInfo.shallowCount == 0)
return appendInfo
// 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 {
// they are valid, insert them in the log
@ -363,20 +364,21 @@ class Log(@volatile var dir: File,
appendInfo.firstOffset = offset.value
val now = time.milliseconds
val validateAndOffsetAssignResult = try {
validMessages.validateMessagesAndAssignOffsets(offset,
now,
appendInfo.sourceCodec,
appendInfo.targetCodec,
config.compact,
config.messageFormatVersion.messageFormatVersion,
config.messageTimestampType,
config.messageTimestampDifferenceMaxMs)
LogValidator.validateMessagesAndAssignOffsets(validRecords,
offset,
now,
appendInfo.sourceCodec,
appendInfo.targetCodec,
config.compact,
config.messageFormatVersion.messageFormatVersion,
config.messageTimestampType,
config.messageTimestampDifferenceMaxMs)
} catch {
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.offsetOfMaxTimestamp = validateAndOffsetAssignResult.offsetOfMaxTimestamp
appendInfo.offsetOfMaxTimestamp = validateAndOffsetAssignResult.shallowOffsetOfMaxTimestamp
appendInfo.lastOffset = offset.value - 1
if (config.messageTimestampType == TimestampType.LOG_APPEND_TIME)
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
// format conversion)
if (validateAndOffsetAssignResult.messageSizeMaybeChanged) {
for (messageAndOffset <- validMessages.shallowIterator) {
if (MessageSet.entrySize(messageAndOffset.message) > config.maxMessageSize) {
for (logEntry <- validRecords.shallowIterator.asScala) {
if (logEntry.sizeInBytes > config.maxMessageSize) {
// we record the original message set size instead of the trimmed size
// to be consistent with pre-compression bytesRejectedRate recording
BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesRejectedRate.mark(messages.sizeInBytes)
BrokerTopicStats.getBrokerAllTopicsStats.bytesRejectedRate.mark(messages.sizeInBytes)
BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesRejectedRate.mark(records.sizeInBytes)
BrokerTopicStats.getBrokerAllTopicsStats.bytesRejectedRate.mark(records.sizeInBytes)
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 {
// we are taking the offsets we are given
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
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."
.format(validMessages.sizeInBytes, config.segmentSize))
.format(validRecords.sizeInBytes, config.segmentSize))
}
// maybe roll the log if this segment is full
val segment = maybeRoll(messagesSize = validMessages.sizeInBytes,
maxTimestampInMessages = appendInfo.maxTimestamp)
val segment = maybeRoll(messagesSize = validRecords.sizeInBytes, maxTimestampInMessages = appendInfo.maxTimestamp)
// now append to the log
segment.append(firstOffset = appendInfo.firstOffset, largestTimestamp = appendInfo.maxTimestamp,
offsetOfLargestTimestamp = appendInfo.offsetOfMaxTimestamp, messages = validMessages)
shallowOffsetOfMaxTimestamp = appendInfo.offsetOfMaxTimestamp, records = validRecords)
// increment the log end offset
updateLogEndOffset(appendInfo.lastOffset + 1)
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)
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)
* </ol>
*/
private def analyzeAndValidateMessageSet(messages: ByteBufferMessageSet): LogAppendInfo = {
private def analyzeAndValidateRecords(records: MemoryRecords): LogAppendInfo = {
var shallowMessageCount = 0
var validBytesCount = 0
var firstOffset, lastOffset = -1L
var sourceCodec: CompressionCodec = NoCompressionCodec
var monotonic = true
var maxTimestamp = Message.NoTimestamp
var maxTimestamp = Record.NO_TIMESTAMP
var offsetOfMaxTimestamp = -1L
for(messageAndOffset <- messages.shallowIterator) {
for (entry <- records.shallowIterator.asScala) {
// update the first offset if on the first message
if(firstOffset < 0)
firstOffset = messageAndOffset.offset
firstOffset = entry.offset
// check that offsets are monotonically increasing
if(lastOffset >= messageAndOffset.offset)
if(lastOffset >= entry.offset)
monotonic = false
// 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.
val messageSize = MessageSet.entrySize(m)
val messageSize = entry.sizeInBytes
if(messageSize > config.maxMessageSize) {
BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesRejectedRate.mark(messages.sizeInBytes)
BrokerTopicStats.getBrokerAllTopicsStats.bytesRejectedRate.mark(messages.sizeInBytes)
BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesRejectedRate.mark(records.sizeInBytes)
BrokerTopicStats.getBrokerAllTopicsStats.bytesRejectedRate.mark(records.sizeInBytes)
throw new RecordTooLargeException("Message size is %d bytes which exceeds the maximum configured message size of %d."
.format(messageSize, config.maxMessageSize))
}
// check the validity of the message by checking CRC
m.ensureValid()
if (m.timestamp > maxTimestamp) {
maxTimestamp = m.timestamp
record.ensureValid()
if (record.timestamp > maxTimestamp) {
maxTimestamp = record.timestamp
offsetOfMaxTimestamp = lastOffset
}
shallowMessageCount += 1
validBytesCount += messageSize
val messageCodec = m.compressionCodec
if(messageCodec != NoCompressionCodec)
val messageCodec = CompressionCodec.getCompressionCodec(record.compressionType.id)
if (messageCodec != NoCompressionCodec)
sourceCodec = messageCodec
}
// Apply broker-side compression if any
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)
*
* @param messages The message set to trim
* @param records The records to trim
* @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.
*/
private def trimInvalidBytes(messages: ByteBufferMessageSet, info: LogAppendInfo): ByteBufferMessageSet = {
val messageSetValidBytes = info.validBytes
if(messageSetValidBytes < 0)
throw new CorruptRecordException("Illegal length of message set " + messageSetValidBytes + " Message set cannot be appended to log. Possible causes are corrupted produce requests")
if(messageSetValidBytes == messages.sizeInBytes) {
messages
private def trimInvalidBytes(records: MemoryRecords, info: LogAppendInfo): MemoryRecords = {
val validBytes = info.validBytes
if (validBytes < 0)
throw new CorruptRecordException("Illegal length of message set " + validBytes + " Message set cannot be appended to log. Possible causes are corrupted produce requests")
if (validBytes == records.sizeInBytes) {
records
} else {
// trim invalid bytes
val validByteBuffer = messages.buffer.duplicate()
validByteBuffer.limit(messageSetValidBytes)
new ByteBufferMessageSet(validByteBuffer)
val validByteBuffer = records.buffer.duplicate()
validByteBuffer.limit(validBytes)
MemoryRecords.readableRecords(validByteBuffer)
}
}
@ -538,7 +540,7 @@ class Log(@volatile var dir: File,
val currentNextOffsetMetadata = nextOffsetMetadata
val next = currentNextOffsetMetadata.messageOffset
if(startOffset == next)
return FetchDataInfo(currentNextOffsetMetadata, MessageSet.Empty)
return FetchDataInfo(currentNextOffsetMetadata, MemoryRecords.EMPTY)
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,
// 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
FetchDataInfo(nextOffsetMetadata, MessageSet.Empty)
FetchDataInfo(nextOffsetMetadata, MemoryRecords.EMPTY)
}
/**
@ -610,9 +612,9 @@ class Log(@volatile var dir: File,
val segmentsCopy = logSegments.toBuffer
// For the earliest and latest, we do not need to return the 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)
return Some(TimestampOffset(Message.NoTimestamp, logEndOffset))
return Some(TimestampOffset(Record.NO_TIMESTAMP, logEndOffset))
val targetSeg = {
// 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)
roll()
// remove the segments for lookups
deletable.foreach(deleteSegment(_))
deletable.foreach(deleteSegment)
}
numToDelete
}
@ -865,7 +867,7 @@ class Log(@volatile var dir: File,
truncateFullyAndStartAt(targetOffset)
} else {
val deletable = logSegments.filter(segment => segment.baseOffset > targetOffset)
deletable.foreach(deleteSegment(_))
deletable.foreach(deleteSegment)
activeSegment.truncateTo(targetOffset)
updateLogEndOffset(targetOffset)
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)
lock synchronized {
val segmentsToDelete = logSegments.toList
segmentsToDelete.foreach(deleteSegment(_))
segmentsToDelete.foreach(deleteSegment)
addSegment(new LogSegment(dir,
newOffset,
indexIntervalBytes = config.indexInterval,

View File

@ -17,20 +17,21 @@
package kafka.log
import java.io.{DataOutputStream, File}
import java.io.File
import java.nio._
import java.util.Date
import java.util.concurrent.{CountDownLatch, TimeUnit}
import com.yammer.metrics.core.Gauge
import kafka.common._
import kafka.message._
import kafka.metrics.KafkaMetricsGroup
import kafka.utils._
import org.apache.kafka.common.record.{FileRecords, LogEntry, MemoryRecords}
import org.apache.kafka.common.utils.Time
import MemoryRecords.LogEntryFilter
import scala.Iterable
import scala.collection._
import JavaConverters._
/**
* 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)
indexFile.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 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 {
// clean segments into the new destination segment
@ -449,8 +450,12 @@ private[log] class Cleaner(val id: Int,
retainDeletes: Boolean,
maxLogMessageSize: Int,
stats: CleanerStats) {
def shouldRetain(messageAndOffset: MessageAndOffset): Boolean =
shouldRetainMessage(source, map, retainDeletes, messageAndOffset, stats)
def shouldRetainEntry(logEntry: LogEntry): Boolean =
shouldRetainMessage(source, map, retainDeletes, logEntry, stats)
class LogCleanerFilter extends LogEntryFilter {
def shouldRetain(logEntry: LogEntry): Boolean = shouldRetainEntry(logEntry)
}
var position = 0
while (position < source.log.sizeInBytes) {
@ -460,10 +465,9 @@ private[log] class Cleaner(val id: Int,
writeBuffer.clear()
source.log.readInto(readBuffer, position)
val messages = new ByteBufferMessageSet(readBuffer)
throttler.maybeThrottle(messages.sizeInBytes)
val result = messages.filterInto(writeBuffer, shouldRetain)
val records = MemoryRecords.readableRecords(readBuffer)
throttler.maybeThrottle(records.sizeInBytes)
val result = records.filterTo(new LogCleanerFilter, writeBuffer)
stats.readMessages(result.messagesRead, result.bytesRead)
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 (writeBuffer.position > 0) {
writeBuffer.flip()
val retained = new ByteBufferMessageSet(writeBuffer)
dest.append(firstOffset = retained.head.offset, largestTimestamp = result.maxTimestamp,
offsetOfLargestTimestamp = result.offsetOfMaxTimestamp, messages = retained)
val retained = MemoryRecords.readableRecords(writeBuffer)
dest.append(firstOffset = retained.deepIterator().next().offset, largestTimestamp = result.maxTimestamp,
shallowOffsetOfMaxTimestamp = result.shallowOffsetOfMaxTimestamp, records = retained)
throttler.maybeThrottle(writeBuffer.limit)
}
@ -488,21 +493,22 @@ private[log] class Cleaner(val id: Int,
private def shouldRetainMessage(source: kafka.log.LogSegment,
map: kafka.log.OffsetMap,
retainDeletes: Boolean,
entry: kafka.message.MessageAndOffset,
entry: LogEntry,
stats: CleanerStats): Boolean = {
val pastLatestOffset = entry.offset > map.latestOffset
if (pastLatestOffset)
return true
val key = entry.message.key
if (key != null) {
if (entry.record.hasKey) {
val key = entry.record.key
val foundOffset = map.get(key)
/* two cases in which we can get rid of a message:
* 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
*/
val redundant = foundOffset >= 0 && entry.offset < foundOffset
val obsoleteDelete = !retainDeletes && entry.message.isNull
val obsoleteDelete = !retainDeletes && entry.record.hasNullValue
!redundant && !obsoleteDelete
} else {
stats.invalidMessage()
@ -620,12 +626,12 @@ private[log] class Cleaner(val id: Int,
checkDone(topicAndPartition)
readBuffer.clear()
segment.log.readInto(readBuffer, position)
val messages = new ByteBufferMessageSet(readBuffer)
throttler.maybeThrottle(messages.sizeInBytes)
val records = MemoryRecords.readableRecords(readBuffer)
throttler.maybeThrottle(records.sizeInBytes)
val startPosition = position
for (entry <- messages) {
val message = entry.message
for (entry <- records.deepIterator.asScala) {
val message = entry.record
if (message.hasKey && entry.offset >= start) {
if (map.size < maxDesiredMapSize)
map.put(message.key, entry.offset)
@ -634,8 +640,9 @@ private[log] class Cleaner(val id: Int,
}
stats.indexMessagesRead(1)
}
position += messages.validBytes
stats.indexBytesRead(messages.validBytes)
val bytesRead = records.validBytes
position += bytesRead
stats.indexBytesRead(bytesRead)
// if we didn't read even one complete message, our read buffer may be too small
if(position == startPosition)

View File

@ -440,7 +440,7 @@ class LogManager(val logDirs: Array[File],
removedLog.dir = renamedDir
// change the file pointers for log and index file
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)
}

View File

@ -16,15 +16,20 @@
*/
package kafka.log
import kafka.message._
import java.io.{File, IOException}
import java.util.concurrent.TimeUnit
import kafka.common._
import kafka.utils._
import kafka.metrics.{KafkaMetricsGroup, KafkaTimer}
import kafka.server.{FetchDataInfo, LogOffsetMetadata}
import kafka.utils._
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 scala.collection.JavaConverters._
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
@ -42,7 +47,7 @@ import java.io.{File, IOException}
* @param time The time instance
*/
@nonthreadsafe
class LogSegment(val log: FileMessageSet,
class LogSegment(val log: FileRecords,
val index: OffsetIndex,
val timeIndex: TimeIndex,
val baseOffset: Long,
@ -63,7 +68,7 @@ class LogSegment(val log: FileMessageSet,
@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) =
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 TimeIndex(Log.timeIndexFilename(dir, startOffset), baseOffset = startOffset, maxIndexSize = maxIndexSize),
startOffset,
@ -82,23 +87,25 @@ class LogSegment(val log: FileMessageSet,
*
* @param firstOffset The first offset 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 messages The messages to append.
* @param shallowOffsetOfMaxTimestamp The offset of the message that has the largest timestamp in the messages to append.
* @param records The log entries to append.
*/
@nonthreadsafe
def append(firstOffset: Long, largestTimestamp: Long, offsetOfLargestTimestamp: Long, messages: ByteBufferMessageSet) {
if (messages.sizeInBytes > 0) {
trace("Inserting %d bytes at offset %d at position %d with largest timestamp %d at offset %d"
.format(messages.sizeInBytes, firstOffset, log.sizeInBytes(), largestTimestamp, offsetOfLargestTimestamp))
def append(firstOffset: Long, largestTimestamp: Long, shallowOffsetOfMaxTimestamp: Long, records: MemoryRecords) {
if (records.sizeInBytes > 0) {
trace("Inserting %d bytes at offset %d at position %d with largest timestamp %d at shallow offset %d"
.format(records.sizeInBytes, firstOffset, log.sizeInBytes(), largestTimestamp, shallowOffsetOfMaxTimestamp))
val physicalPosition = log.sizeInBytes()
if (physicalPosition == 0)
rollingBasedTimestamp = Some(largestTimestamp)
// 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.
if (largestTimestamp > maxTimestampSoFar) {
maxTimestampSoFar = largestTimestamp
offsetOfMaxTimestamp = offsetOfLargestTimestamp
offsetOfMaxTimestamp = shallowOffsetOfMaxTimestamp
}
// append an entry to the index (if needed)
if(bytesSinceLastIndexEntry > indexIntervalBytes) {
@ -106,7 +113,7 @@ class LogSegment(val log: FileMessageSet,
timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestamp)
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.
*/
@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)
log.searchForOffsetWithSize(offset, max(mapping.position, startingFilePosition))
}
@ -154,40 +161,40 @@ class LogSegment(val log: FileMessageSet,
if (startOffsetAndSize == null)
return null
val (startPosition, messageSetSize) = startOffsetAndSize
val offsetMetadata = new LogOffsetMetadata(startOffset, this.baseOffset, startPosition.position)
val startPosition = startOffsetAndSize.position.toInt
val offsetMetadata = new LogOffsetMetadata(startOffset, this.baseOffset, startPosition)
val adjustedMaxSize =
if (minOneMessage) math.max(maxSize, messageSetSize)
if (minOneMessage) math.max(maxSize, startOffsetAndSize.size)
else maxSize
// return a log segment but with zero size in the case below
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
val length = maxOffset match {
case None =>
// no max offset, just read until the max position
min((maxPosition - startPosition.position).toInt, adjustedMaxSize)
min((maxPosition - startPosition).toInt, adjustedMaxSize)
case Some(offset) =>
// 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
// 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.
if (offset < startOffset)
return FetchDataInfo(offsetMetadata, MessageSet.Empty, firstMessageSetIncomplete = false)
val mapping = translateOffset(offset, startPosition.position)
return FetchDataInfo(offsetMetadata, MemoryRecords.EMPTY, firstEntryIncomplete = false)
val mapping = translateOffset(offset, startPosition)
val endPosition =
if (mapping == null)
logSize // the max offset is off the end of the log, use the end of the file
else
mapping._1.position
min(min(maxPosition, endPosition) - startPosition.position, adjustedMaxSize).toInt
mapping.position
min(min(maxPosition, endPosition) - startPosition, adjustedMaxSize).toInt
}
FetchDataInfo(offsetMetadata, log.read(startPosition.position, length),
firstMessageSetIncomplete = adjustedMaxSize < messageSetSize)
FetchDataInfo(offsetMetadata, log.read(startPosition, length),
firstEntryIncomplete = adjustedMaxSize < startOffsetAndSize.size)
}
/**
@ -205,16 +212,16 @@ class LogSegment(val log: FileMessageSet,
timeIndex.resize(timeIndex.maxIndexSize)
var validBytes = 0
var lastIndexEntry = 0
val iter = log.iterator(maxMessageSize)
maxTimestampSoFar = Message.NoTimestamp
val iter = log.shallowIterator(maxMessageSize)
maxTimestampSoFar = Record.NO_TIMESTAMP
try {
while(iter.hasNext) {
val entry = iter.next
entry.message.ensureValid()
for (entry <- iter.asScala) {
val record = entry.record
record.ensureValid()
// 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) {
maxTimestampSoFar = entry.message.timestamp
if (record.timestamp > maxTimestampSoFar) {
maxTimestampSoFar = record.timestamp
offsetOfMaxTimestamp = entry.offset
}
@ -225,11 +232,12 @@ class LogSegment(val log: FileMessageSet,
timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestamp)
lastIndexEntry = validBytes
}
validBytes += MessageSet.entrySize(entry.message)
validBytes += entry.sizeInBytes()
}
} catch {
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
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
index.resize(index.maxIndexSize)
timeIndex.resize(timeIndex.maxIndexSize)
val (offsetPosition, _) = mapping
val bytesTruncated = log.truncateTo(offsetPosition.position)
val bytesTruncated = log.truncateTo(mapping.position.toInt)
if(log.sizeInBytes == 0) {
created = time.milliseconds
rollingBasedTimestamp = None
@ -296,10 +303,10 @@ class LogSegment(val log: FileMessageSet,
@threadsafe
def nextOffset(): Long = {
val ms = read(index.lastOffset, None, log.sizeInBytes)
if(ms == null) {
if (ms == null) {
baseOffset
} else {
ms.messageSet.lastOption match {
ms.records.shallowIterator.asScala.toSeq.lastOption match {
case None => baseOffset
case Some(last) => last.nextOffset
}
@ -360,9 +367,9 @@ class LogSegment(val log: FileMessageSet,
def timeWaitedForRoll(now: Long, messageTimestamp: Long) : Long = {
// Load the timestamp of the first message into memory
if (rollingBasedTimestamp.isEmpty) {
val iter = log.iterator
val iter = log.shallowIterator
if (iter.hasNext)
rollingBasedTimestamp = Some(iter.next.message.timestamp)
rollingBasedTimestamp = Some(iter.next.record.timestamp)
}
rollingBasedTimestamp match {
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
val timestampOffset = timeIndex.lookup(timestamp)
val position = index.lookup(timestampOffset.offset).position
// 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)
}
}
object LogFlushStats extends KafkaMetricsGroup {
val logFlushTimer = new KafkaTimer(newTimer("LogFlushRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS))
}

View File

@ -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)
}

View File

@ -21,9 +21,9 @@ import java.io.File
import java.nio.ByteBuffer
import kafka.common.InvalidOffsetException
import kafka.message.Message
import kafka.utils.CoreUtils._
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
@ -69,7 +69,7 @@ class TimeIndex(file: File,
def lastEntry: TimestampOffset = {
inLock(lock) {
_entries match {
case 0 => TimestampOffset(Message.NoTimestamp, baseOffset)
case 0 => TimestampOffset(Record.NO_TIMESTAMP, baseOffset)
case s => parseEntry(mmap, s - 1).asInstanceOf[TimestampOffset]
}
}
@ -145,7 +145,7 @@ class TimeIndex(file: File,
val idx = mmap.duplicate
val slot = indexSlotFor(idx, targetTimestamp, IndexSearchType.KEY)
if (slot == -1)
TimestampOffset(Message.NoTimestamp, baseOffset)
TimestampOffset(Record.NO_TIMESTAMP, baseOffset)
else {
val entry = parseEntry(idx, slot).asInstanceOf[TimestampOffset]
TimestampOffset(entry.timestamp, entry.offset)

View File

@ -17,20 +17,13 @@
package kafka.message
import kafka.utils.{CoreUtils, IteratorTemplate, Logging}
import kafka.common.{KafkaException, LongRef}
import java.nio.ByteBuffer
import java.nio.channels._
import java.io._
import java.util.ArrayDeque
import kafka.message.ByteBufferMessageSet.FilterResult
import org.apache.kafka.common.errors.InvalidTimestampException
import org.apache.kafka.common.record.{MemoryRecords, TimestampType}
import org.apache.kafka.common.utils.Utils
import kafka.common.LongRef
import kafka.utils.Logging
import org.apache.kafka.common.record._
import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer
import scala.collection.JavaConverters._
object ByteBufferMessageSet {
@ -41,204 +34,19 @@ object ByteBufferMessageSet {
messages: Message*): ByteBuffer = {
if (messages.isEmpty)
MessageSet.Empty.buffer
else if (compressionCodec == NoCompressionCodec) {
val buffer = ByteBuffer.allocate(MessageSet.messageSetSize(messages))
for (message <- messages) writeMessage(buffer, message, offsetAssigner.nextAbsoluteOffset())
buffer.rewind()
buffer
} else {
else {
val magicAndTimestamp = wrapperMessageTimestamp match {
case Some(ts) => MagicAndTimestamp(messages.head.magic, ts)
case None => MessageSet.magicAndLargestTimestamp(messages)
}
val (messageWriter, lastOffset) = writeCompressedMessages(compressionCodec, offsetAssigner, magicAndTimestamp,
timestampType, messages)
val buffer = ByteBuffer.allocate(messageWriter.size + MessageSet.LogOverhead)
writeMessage(buffer, messageWriter, lastOffset)
buffer.rewind()
buffer
val entries = messages.map(message => LogEntry.create(offsetAssigner.nextAbsoluteOffset(), message.asRecord))
val builder = MemoryRecords.builderWithEntries(timestampType, CompressionType.forId(compressionCodec.codec),
magicAndTimestamp.timestamp, entries.asJava)
builder.build().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 {
@ -246,9 +54,6 @@ private object OffsetAssigner {
def apply(offsetCounter: LongRef, size: Int): OffsetAssigner =
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]) {
@ -322,7 +127,6 @@ private class OffsetAssigner(offsets: Seq[Long]) {
*
*/
class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Logging {
private var shallowValidByteCount = -1
private[kafka] def this(compressionCodec: CompressionCodec,
offsetCounter: LongRef,
@ -354,33 +158,6 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi
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 */
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)
/** 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] = {
new IteratorTemplate[MessageAndOffset] {
var topIter = buffer.slice()
var innerIter: Iterator[MessageAndOffset] = null
def innerDone(): Boolean = (innerIter == null || !innerIter.hasNext)
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.")
private def internalIterator(isShallow: Boolean = false): Iterator[MessageAndOffset] = {
val entries = if (isShallow)
asRecords.shallowIterator
else
asRecords.deepIterator
entries.asScala.map(MessageAndOffset.fromLogEntry)
}
/**
@ -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
*/
def validBytes: Int = shallowValidBytes
def validBytes: Int = asRecords.validBytes
/**
* 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
}
case class ValidationAndOffsetAssignResult(validatedMessages: ByteBufferMessageSet,
maxTimestamp: Long,
offsetOfMaxTimestamp: Long,
messageSizeMaybeChanged: Boolean)

View File

@ -19,7 +19,7 @@ package kafka.message
import java.nio._
import org.apache.kafka.common.record.TimestampType
import org.apache.kafka.common.record.{Record, TimestampType}
import scala.math._
import kafka.utils._
@ -98,6 +98,11 @@ object Message {
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._
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
* @param bytes The payload of the message
@ -327,52 +341,6 @@ class Message(val buffer: ByteBuffer,
*/
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
*/
@ -399,7 +367,7 @@ class Message(val buffer: ByteBuffer,
if (timestamp < 0 && timestamp != NoTimestamp)
throw new IllegalArgumentException(s"Invalid message timestamp $timestamp")
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 = {

View File

@ -17,6 +17,13 @@
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) {
@ -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.
*/
def firstOffset: Long = message.compressionCodec match {
case NoCompressionCodec => offset
case _ => ByteBufferMessageSet.deepIterator(this).next().offset
def firstOffset: Long = toLogEntry.firstOffset
def toLogEntry: LogEntry = {
LogEntry.create(offset, message.asRecord)
}
}

View File

@ -18,7 +18,6 @@
package kafka.message
import java.nio._
import java.nio.channels._
import org.apache.kafka.common.record.Records
@ -72,11 +71,6 @@ case class MagicAndTimestamp(magic: Byte, timestamp: Long)
*/
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
*/
@ -99,7 +93,7 @@ abstract class MessageSet extends Iterable[MessageAndOffset] {
override def toString: String = {
val builder = new StringBuilder()
builder.append(getClass.getSimpleName + "(")
val iter = this.iterator
val iter = this.asRecords.shallowIterator()
var i = 0
while(iter.hasNext && i < 100) {
val message = iter.next

View File

@ -21,7 +21,6 @@ import java.util.concurrent.locks.ReentrantLock
import kafka.cluster.BrokerEndPoint
import kafka.consumer.PartitionTopicInfo
import kafka.message.ByteBufferMessageSet
import kafka.utils.{DelayedItem, Pool, ShutdownableThread}
import kafka.common.{ClientIdAndBroker, KafkaException}
import kafka.metrics.KafkaMetricsGroup
@ -38,6 +37,7 @@ import java.util.concurrent.atomic.AtomicLong
import com.yammer.metrics.core.Gauge
import org.apache.kafka.common.TopicPartition
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.
@ -144,15 +144,15 @@ abstract class AbstractFetcherThread(name: String,
Errors.forCode(partitionData.errorCode) match {
case Errors.NONE =>
try {
val messages = partitionData.toByteBufferMessageSet
val newOffset = messages.shallowIterator.toSeq.lastOption.map(_.nextOffset).getOrElse(
val records = partitionData.toRecords
val newOffset = records.shallowIterator.asScala.toSeq.lastOption.map(_.nextOffset).getOrElse(
currentPartitionFetchState.offset)
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
processPartitionData(topicPartition, currentPartitionFetchState.offset, partitionData)
val validBytes = messages.validBytes
val validBytes = records.validBytes
if (validBytes > 0) {
// Update partitionStates only if there is no exception during processPartitionData
partitionStates.updateAndMoveToEnd(topicPartition, new PartitionFetchState(newOffset))
@ -260,7 +260,7 @@ object AbstractFetcherThread {
trait PartitionData {
def errorCode: Short
def exception: Option[Throwable]
def toByteBufferMessageSet: ByteBufferMessageSet
def toRecords: MemoryRecords
def highWatermark: Long
}

View File

@ -19,7 +19,6 @@ package kafka.server
import java.util.concurrent.TimeUnit
import kafka.api.FetchResponsePartitionData
import kafka.common.TopicAndPartition
import kafka.metrics.KafkaMetricsGroup
import org.apache.kafka.common.TopicPartition
@ -59,7 +58,7 @@ class DelayedFetch(delayMs: Long,
fetchMetadata: FetchMetadata,
replicaManager: ReplicaManager,
quota: ReplicaQuota,
responseCallback: Seq[(TopicAndPartition, FetchResponsePartitionData)] => Unit)
responseCallback: Seq[(TopicAndPartition, FetchPartitionData)] => Unit)
extends DelayedOperation(delayMs) {
/**
@ -152,7 +151,7 @@ class DelayedFetch(delayMs: Long,
)
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)

View File

@ -17,7 +17,8 @@
package kafka.server
import kafka.message.MessageSet
import org.apache.kafka.common.record.Records
case class FetchDataInfo(fetchOffsetMetadata: LogOffsetMetadata, messageSet: MessageSet,
firstMessageSetIncomplete: Boolean = false)
case class FetchDataInfo(fetchOffsetMetadata: LogOffsetMetadata,
records: Records,
firstEntryIncomplete: Boolean = false)

View File

@ -23,14 +23,13 @@ import java.util.{Collections, Properties}
import java.util
import kafka.admin.{AdminUtils, RackAwareMode}
import kafka.api.{ControlledShutdownRequest, ControlledShutdownResponse, FetchResponsePartitionData}
import kafka.api.{ControlledShutdownRequest, ControlledShutdownResponse}
import kafka.cluster.Partition
import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota}
import kafka.common._
import kafka.controller.KafkaController
import kafka.coordinator.{GroupCoordinator, JoinGroupResult}
import kafka.log._
import kafka.message.{ByteBufferMessageSet, Message}
import kafka.network._
import kafka.network.RequestChannel.{Response, Session}
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.metrics.Metrics
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.ProduceResponse.PartitionResponse
import org.apache.kafka.common.utils.{Time, Utils}
@ -359,9 +358,9 @@ class KafkaApis(val requestChannel: RequestChannel,
val mergedResponseStatus = responseStatus ++
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(_ =>
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
@ -422,17 +421,12 @@ class KafkaApis(val requestChannel: RequestChannel,
else {
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
replicaManager.appendMessages(
replicaManager.appendRecords(
produceRequest.timeout.toLong,
produceRequest.acks,
internalTopicsAllowed,
authorizedMessagesPerPartition,
authorizedRequestInfo,
sendResponseCallback)
// 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
def sendResponseCallback(responsePartitionData: Seq[(TopicAndPartition, FetchResponsePartitionData)]) {
def sendResponseCallback(responsePartitionData: Seq[(TopicAndPartition, FetchPartitionData)]) {
val convertedPartitionData = {
// Need to down-convert message when consumer only takes magic value 0.
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
// test might break because some messages in new message format can be delivered to consumers before 0.10.0.0
// without format down conversion.
val convertedData = if (versionId <= 1 && replicaManager.getMessageFormatVersion(tp).exists(_ > Message.MagicValue_V0) &&
!data.messages.isMagicValueInAllWrapperMessages(Message.MagicValue_V0)) {
val convertedData = if (versionId <= 1 && replicaManager.getMagicAndTimestampType(tp).exists(_._1 > Record.MAGIC_VALUE_V0) &&
!data.records.hasMatchingShallowMagic(Record.MAGIC_VALUE_V0)) {
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
val records = convertedData.messages.asRecords
new TopicPartition(tp.topic, tp.partition) -> new FetchResponse.PartitionData(convertedData.error, convertedData.hw, records)
new TopicPartition(tp.topic, tp.partition) -> new FetchResponse.PartitionData(convertedData.error, convertedData.hw, convertedData.records)
}
}
val mergedPartitionData = convertedPartitionData ++ unauthorizedForReadPartitionData ++ nonExistingOrUnauthorizedForDescribePartitionData
val fetchedPartitionData = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData]()
mergedPartitionData.foreach { case (topicPartition, data) =>
@ -660,7 +654,7 @@ class KafkaApis(val requestChannel: RequestChannel,
val found = {
if (fromConsumer && timestamp == ListOffsetRequest.LATEST_TIMESTAMP)
TimestampOffset(Message.NoTimestamp, localReplica.highWatermark.messageOffset)
TimestampOffset(Record.NO_TIMESTAMP, localReplica.highWatermark.messageOffset)
else {
def allowed(timestampOffset: TimestampOffset): Boolean =
!fromConsumer || timestampOffset.offset <= localReplica.highWatermark.messageOffset

View File

@ -23,7 +23,6 @@ import java.util
import kafka.admin.AdminUtils
import kafka.cluster.BrokerEndPoint
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.common.{KafkaStorageException, TopicAndPartition}
import ReplicaFetcherThread._
@ -119,19 +118,19 @@ class ReplicaFetcherThread(name: String,
val topic = topicPartition.topic
val partitionId = topicPartition.partition
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)
throw new RuntimeException("Offset mismatch for partition %s: fetched offset = %d, log end offset = %d.".format(topicPartition, fetchOffset, replica.logEndOffset.messageOffset))
if (logger.isTraceEnabled)
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))
replica.log.get.append(messageSet, assignOffsets = false)
.format(replica.brokerId, replica.logEndOffset.messageOffset, topicPartition, records.sizeInBytes, partitionData.highWatermark))
replica.log.get.append(records, assignOffsets = false)
if (logger.isTraceEnabled)
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)
// for the follower replica, we do not need to keep
// 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"
.format(replica.brokerId, topic, partitionId, followerHighWatermark))
if (quota.isThrottled(TopicAndPartition(topic, partitionId)))
quota.record(messageSet.sizeInBytes)
quota.record(records.sizeInBytes)
} catch {
case e: KafkaStorageException =>
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)
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. " +
"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 " +
@ -323,9 +322,8 @@ object ReplicaFetcherThread {
def errorCode: Short = underlying.errorCode
def toByteBufferMessageSet: ByteBufferMessageSet = {
val buffer = underlying.records.asInstanceOf[MemoryRecords].buffer
new ByteBufferMessageSet(buffer)
def toRecords: MemoryRecords = {
underlying.records.asInstanceOf[MemoryRecords]
}
def highWatermark: Long = underlying.highWatermark

View File

@ -26,7 +26,7 @@ import kafka.cluster.{Partition, Replica}
import kafka.common._
import kafka.controller.KafkaController
import kafka.log.{LogAppendInfo, LogManager}
import kafka.message.{ByteBufferMessageSet, InvalidMessageException, Message, MessageSet}
import kafka.message.InvalidMessageException
import kafka.metrics.KafkaMetricsGroup
import kafka.server.QuotaFactory.UnboundedQuota
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.metrics.Metrics
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.ProduceResponse.PartitionResponse
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 {
val UnknownLogReadResult = LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata,
MessageSet.Empty),
MemoryRecords.EMPTY),
-1L,
-1,
false)
@ -276,11 +279,7 @@ class ReplicaManager(val config: KafkaConfig,
}
def getPartition(topic: String, partitionId: Int): Option[Partition] = {
val partition = allPartitions.get((topic, partitionId))
if (partition == null)
None
else
Some(partition)
Option(allPartitions.get((topic, partitionId)))
}
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;
* the callback function will be triggered either when timeout or the required acks are satisfied
*/
def appendMessages(timeout: Long,
requiredAcks: Short,
internalTopicsAllowed: Boolean,
messagesPerPartition: Map[TopicPartition, MessageSet],
responseCallback: Map[TopicPartition, PartitionResponse] => Unit) {
def appendRecords(timeout: Long,
requiredAcks: Short,
internalTopicsAllowed: Boolean,
entriesPerPartition: Map[TopicPartition, MemoryRecords],
responseCallback: Map[TopicPartition, PartitionResponse] => Unit) {
if (isValidRequiredAcks(requiredAcks)) {
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))
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
}
if (delayedRequestRequired(requiredAcks, messagesPerPartition, localProduceResults)) {
if (delayedRequestRequired(requiredAcks, entriesPerPartition, localProduceResults)) {
// create delayed produce operation
val produceMetadata = ProduceMetadata(requiredAcks, produceStatus)
val delayedProduce = new DelayedProduce(timeout, produceMetadata, this, responseCallback)
// 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
// this is because while the delayed produce operation is being created, new
@ -357,9 +356,9 @@ class ReplicaManager(val config: KafkaConfig,
} else {
// 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
val responseStatus = messagesPerPartition.map { case (topicAndPartition, _) =>
val responseStatus = entriesPerPartition.map { case (topicAndPartition, _) =>
topicAndPartition -> new PartitionResponse(Errors.INVALID_REQUIRED_ACKS.code,
LogAppendInfo.UnknownLogAppendInfo.firstOffset, Message.NoTimestamp)
LogAppendInfo.UnknownLogAppendInfo.firstOffset, Record.NO_TIMESTAMP)
}
responseCallback(responseStatus)
}
@ -370,11 +369,12 @@ class ReplicaManager(val config: KafkaConfig,
// 1. required acks = -1
// 2. there is data to append
// 3. at least one partition append was successful (fewer errors than partitions)
private def delayedRequestRequired(requiredAcks: Short, messagesPerPartition: Map[TopicPartition, MessageSet],
localProduceResults: Map[TopicPartition, LogAppendResult]): Boolean = {
private def delayedRequestRequired(requiredAcks: Short,
entriesPerPartition: Map[TopicPartition, MemoryRecords],
localProduceResults: Map[TopicPartition, LogAppendResult]): Boolean = {
requiredAcks == -1 &&
messagesPerPartition.nonEmpty &&
localProduceResults.values.count(_.error.isDefined) < messagesPerPartition.size
entriesPerPartition.nonEmpty &&
localProduceResults.values.count(_.error.isDefined) < entriesPerPartition.size
}
private def isValidRequiredAcks(requiredAcks: Short): Boolean = {
@ -385,10 +385,10 @@ class ReplicaManager(val config: KafkaConfig,
* Append the messages to the local replica logs
*/
private def appendToLocalLog(internalTopicsAllowed: Boolean,
messagesPerPartition: Map[TopicPartition, MessageSet],
entriesPerPartition: Map[TopicPartition, MemoryRecords],
requiredAcks: Short): Map[TopicPartition, LogAppendResult] = {
trace("Append [%s] to local log ".format(messagesPerPartition))
messagesPerPartition.map { case (topicPartition, messages) =>
trace("Append [%s] to local log ".format(entriesPerPartition))
entriesPerPartition.map { case (topicPartition, records) =>
BrokerTopicStats.getBrokerTopicStats(topicPartition.topic).totalProduceRequestRate.mark()
BrokerTopicStats.getBrokerAllTopicsStats().totalProduceRequestRate.mark()
@ -402,7 +402,7 @@ class ReplicaManager(val config: KafkaConfig,
val partitionOpt = getPartition(topicPartition.topic, topicPartition.partition)
val info = partitionOpt match {
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"
.format(topicPartition, localBrokerId))
}
@ -414,13 +414,13 @@ class ReplicaManager(val config: KafkaConfig,
info.lastOffset - info.firstOffset + 1
// update stats for successfully appended bytes and messages as bytesInRate and messageInRate
BrokerTopicStats.getBrokerTopicStats(topicPartition.topic).bytesInRate.mark(messages.sizeInBytes)
BrokerTopicStats.getBrokerAllTopicsStats.bytesInRate.mark(messages.sizeInBytes)
BrokerTopicStats.getBrokerTopicStats(topicPartition.topic).bytesInRate.mark(records.sizeInBytes)
BrokerTopicStats.getBrokerAllTopicsStats.bytesInRate.mark(records.sizeInBytes)
BrokerTopicStats.getBrokerTopicStats(topicPartition.topic).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"
.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))
} catch {
// NOTE: Failed produce requests metric is not incremented for known exceptions
@ -434,6 +434,7 @@ class ReplicaManager(val config: KafkaConfig,
_: RecordTooLargeException |
_: RecordBatchTooLargeException |
_: CorruptRecordException |
_: InvalidRecordException |
_: InvalidMessageException |
_: InvalidTimestampException) =>
(topicPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(e)))
@ -458,7 +459,7 @@ class ReplicaManager(val config: KafkaConfig,
hardMaxBytesLimit: Boolean,
fetchInfos: Seq[(TopicPartition, PartitionData)],
quota: ReplicaQuota = UnboundedQuota,
responseCallback: Seq[(TopicAndPartition, FetchResponsePartitionData)] => Unit) {
responseCallback: Seq[(TopicAndPartition, FetchPartitionData)] => Unit) {
val isFromFollower = replicaId >= 0
val fetchOnlyFromLeader: Boolean = replicaId != Request.DebuggingConsumerId
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
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) =>
errorIncurred || (readResult.errorCode != Errors.NONE.code))
@ -490,7 +491,7 @@ class ReplicaManager(val config: KafkaConfig,
// 4) some error happens while reading data
if (timeout <= 0 || fetchInfos.isEmpty || bytesReadable >= fetchMinBytes || errorReadingData) {
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)
} else {
@ -568,16 +569,16 @@ class ReplicaManager(val config: KafkaConfig,
// If the partition is being throttled, simply return an empty set.
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
// progress in such cases and don't need to report a `RecordTooLargeException`
else if (!hardMaxBytesLimit && fetch.firstMessageSetIncomplete)
FetchDataInfo(fetch.fetchOffsetMetadata, MessageSet.Empty)
else if (!hardMaxBytesLimit && fetch.firstEntryIncomplete)
FetchDataInfo(fetch.fetchOffsetMetadata, MemoryRecords.EMPTY)
else fetch
case None =>
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
@ -590,12 +591,14 @@ class ReplicaManager(val config: KafkaConfig,
_: NotLeaderForPartitionException |
_: ReplicaNotAvailableException |
_: 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 =>
BrokerTopicStats.getBrokerTopicStats(topic).failedFetchRequestRate.mark()
BrokerTopicStats.getBrokerAllTopicsStats().failedFetchRequestRate.mark()
error(s"Error processing fetch operation on partition ${tp}, offset $offset", e)
LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, partitionFetchSize, false, Some(e))
error(s"Error processing fetch operation on partition $tp, offset $offset", e)
LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MemoryRecords.EMPTY), -1L,
partitionFetchSize, false, Some(e))
}
}
@ -604,7 +607,7 @@ class ReplicaManager(val config: KafkaConfig,
var minOneMessage = !hardMaxBytesLimit
readPartitionInfo.foreach { case (tp, fetchInfo) =>
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
if (messageSetSize > 0)
minOneMessage = false
@ -625,9 +628,9 @@ class ReplicaManager(val config: KafkaConfig,
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 =>
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) {

View File

@ -23,15 +23,16 @@ import java.nio.ByteBuffer
import joptsimple.OptionParser
import kafka.coordinator.{GroupMetadataKey, GroupMetadataManager, OffsetKey}
import kafka.log._
import kafka.message._
import kafka.serializer.Decoder
import kafka.utils._
import org.apache.kafka.clients.consumer.internals.ConsumerProtocol
import org.apache.kafka.common.KafkaException
import org.apache.kafka.common.record.{CompressionType, FileRecords, LogEntry, Record}
import org.apache.kafka.common.utils.Utils
import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer
import scala.collection.JavaConverters._
object DumpLogSegments {
@ -69,7 +70,7 @@ object DumpLogSegments {
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 indexSanityOnly = options.has(indexSanityOpt)
@ -132,7 +133,7 @@ object DumpLogSegments {
maxMessageSize: Int) {
val startOffset = file.getName().split("\\.")(0).toLong
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)
//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) {
val entry = index.entry(i)
val partialFileMessageSet: FileMessageSet = messageSet.read(entry.position, maxMessageSize)
val messageAndOffset = getIterator(partialFileMessageSet.head, isDeepIteration = true).next()
if(messageAndOffset.offset != entry.offset + index.baseOffset) {
val slice = fileRecords.read(entry.position, maxMessageSize)
val logEntry = getIterator(slice.shallowIterator.next, isDeepIteration = true).next()
if (logEntry.offset != entry.offset + index.baseOffset) {
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)
}
// 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,
timeIndexDumpErrors: TimeIndexDumpErrors,
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 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 index = new OffsetIndex(indexFile, baseOffset = startOffset)
val timeIndex = new TimeIndex(file, baseOffset = startOffset)
@ -178,26 +179,26 @@ object DumpLogSegments {
return
}
var prevTimestamp = Message.NoTimestamp
var prevTimestamp = Record.NO_TIMESTAMP
for(i <- 0 until timeIndex.entries) {
val entry = timeIndex.entry(i)
val position = index.lookup(entry.offset + timeIndex.baseOffset).position
val partialFileMessageSet: FileMessageSet = messageSet.read(position, Int.MaxValue)
val shallowIter = partialFileMessageSet.iterator
var maxTimestamp = Message.NoTimestamp
val partialFileRecords = fileRecords.read(position, Int.MaxValue)
val shallowEntries = partialFileRecords.shallowIterator.asScala
var maxTimestamp = Record.NO_TIMESTAMP
// We first find the message by offset then check if the timestamp is correct.
val wrapperMessageOpt = shallowIter.find(_.offset >= entry.offset + timeIndex.baseOffset)
wrapperMessageOpt match {
val maybeLogEntry = shallowEntries.find(_.offset >= entry.offset + timeIndex.baseOffset)
maybeLogEntry match {
case None =>
timeIndexDumpErrors.recordShallowOffsetNotFound(file, entry.offset + timeIndex.baseOffset,
-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,
wrapperMessage.offset)
case Some(wrapperMessage) =>
val deepIter = getIterator(wrapperMessage, isDeepIteration = true)
for (messageAndOffset <- deepIter)
maxTimestamp = math.max(maxTimestamp, messageAndOffset.message.timestamp)
logEntry.offset)
case Some(shallowLogEntry) =>
val deepIter = getIterator(shallowLogEntry, isDeepIteration = true)
for (deepLogEntry <- deepIter)
maxTimestamp = math.max(maxTimestamp, deepLogEntry.record.timestamp)
if (maxTimestamp != entry.timestamp)
timeIndexDumpErrors.recordMismatchTimeIndex(file, entry.timestamp, maxTimestamp)
@ -216,20 +217,20 @@ object DumpLogSegments {
}
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] {
override def parse(message: Message): (Option[K], Option[V]) = {
if (message.isNull) {
override def parse(record: Record): (Option[K], Option[V]) = {
if (record.hasNullValue) {
(None, None)
} else {
val key = if (message.hasKey)
Some(keyDecoder.fromBytes(Utils.readBytes(message.key)))
val key = if (record.hasKey)
Some(keyDecoder.fromBytes(Utils.readBytes(record.key)))
else
None
val payload = Some(valueDecoder.fromBytes(Utils.readBytes(message.payload)))
val payload = Some(valueDecoder.fromBytes(Utils.readBytes(record.value)))
(key, payload)
}
@ -249,7 +250,7 @@ object DumpLogSegments {
val topicPartition = offsetKey.key.topicPartition
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)
String.valueOf(offset.offset)
else
@ -271,27 +272,27 @@ object DumpLogSegments {
if (userData.isEmpty)
s"${member.memberId}=${partitionAssignment.partitions()}"
else
s"${member.memberId}=${partitionAssignment.partitions()}:${userData}"
s"${member.memberId}=${partitionAssignment.partitions()}:$userData"
} else {
s"${member.memberId}=${hex(member.assignment)}"
}
}.mkString("{", ",", "}")
val keyString = s"metadata::${groupId}"
val valueString = s"${protocolType}:${group.protocol}:${group.generationId}:${assignment}"
val keyString = s"metadata::$groupId"
val valueString = s"$protocolType:${group.protocol}:${group.generationId}:$assignment"
(Some(keyString), Some(valueString))
}
override def parse(message: Message): (Option[String], Option[String]) = {
if (message.isNull)
override def parse(record: Record): (Option[String], Option[String]) = {
if (record.hasNullValue)
(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)")
} else {
GroupMetadataManager.readMessageKey(message.key) match {
case offsetKey: OffsetKey => parseOffsets(offsetKey, message.payload)
case groupMetadataKey: GroupMetadataKey => parseGroupMetadata(groupMetadataKey, message.payload)
GroupMetadataManager.readMessageKey(record.key) match {
case offsetKey: OffsetKey => parseOffsets(offsetKey, record.value)
case groupMetadataKey: GroupMetadataKey => parseGroupMetadata(groupMetadataKey, record.value)
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[_, _]) {
val startOffset = file.getName().split("\\.")(0).toLong
println("Starting offset: " + startOffset)
val messageSet = new FileMessageSet(file, false)
val messageSet = FileRecords.open(file, false)
var validBytes = 0L
var lastOffset = -1l
val shallowIterator = messageSet.iterator(maxMessageSize)
for(shallowMessageAndOffset <- shallowIterator) { // this only does shallow iteration
val itr = getIterator(shallowMessageAndOffset, isDeepIteration)
for (messageAndOffset <- itr) {
val msg = messageAndOffset.message
val shallowIterator = messageSet.shallowIterator(maxMessageSize).asScala
for (shallowLogEntry <- shallowIterator) { // this only does shallow iteration
val itr = getIterator(shallowLogEntry, isDeepIteration)
for (deepLogEntry <- itr) {
val record = deepLogEntry.record()
if(lastOffset == -1)
lastOffset = messageAndOffset.offset
lastOffset = deepLogEntry.offset
// 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)]())
nonConsecutivePairsSeq ::=(lastOffset, messageAndOffset.offset)
nonConsecutivePairsSeq ::=(lastOffset, deepLogEntry.offset)
nonConsecutivePairsForLogFilesMap.put(file.getAbsolutePath, nonConsecutivePairsSeq)
}
lastOffset = messageAndOffset.offset
lastOffset = deepLogEntry.offset
print("offset: " + messageAndOffset.offset + " position: " + validBytes +
" " + msg.timestampType + ": " + msg.timestamp + " isvalid: " + msg.isValid +
" payloadsize: " + msg.payloadSize + " magic: " + msg.magic +
" compresscodec: " + msg.compressionCodec + " crc: " + msg.checksum)
if(msg.hasKey)
print(" keysize: " + msg.keySize)
if(printContents) {
val (key, payload) = parser.parse(msg)
key.map(key => print(s" key: ${key}"))
payload.map(payload => print(s" payload: ${payload}"))
print("offset: " + deepLogEntry.offset + " position: " + validBytes +
" " + record.timestampType + ": " + record.timestamp + " isvalid: " + record.isValid +
" payloadsize: " + record.valueSize + " magic: " + record.magic +
" compresscodec: " + record.compressionType + " crc: " + record.checksum)
if (record.hasKey)
print(" keysize: " + record.keySize)
if (printContents) {
val (key, payload) = parser.parse(record)
key.foreach(key => print(s" key: $key"))
payload.foreach(payload => print(s" payload: $payload"))
}
println()
}
validBytes += MessageSet.entrySize(shallowMessageAndOffset.message)
validBytes += shallowLogEntry.sizeInBytes
}
val trailingBytes = messageSet.sizeInBytes - validBytes
if(trailingBytes > 0)
println("Found %d invalid bytes at the end of %s".format(trailingBytes, file.getName))
}
private def getIterator(messageAndOffset: MessageAndOffset, isDeepIteration: Boolean) = {
if (isDeepIteration) {
val message = messageAndOffset.message
message.compressionCodec match {
case NoCompressionCodec =>
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()
}
}
private def getIterator(logEntry: LogEntry, isDeepIteration: Boolean): Iterator[LogEntry] = {
if (isDeepIteration)
logEntry.iterator.asScala
else
Iterator(logEntry)
}
class TimeIndexDumpErrors {

View File

@ -17,25 +17,26 @@
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.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.utils._
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.utils.Time
import scala.collection.JavaConverters._
/**
* For verifying the consistency among replicas.
*
@ -149,15 +150,15 @@ object ReplicaVerificationTool extends Logging {
debug("Selected topic partitions: " + topicPartitionReplicaList)
val topicAndPartitionsPerBroker: Map[Int, Seq[TopicAndPartition]] = topicPartitionReplicaList.groupBy(_.replicaId)
.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)
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 }
debug("Expected replicas per topic partition: " + expectedReplicasPerTopicAndPartition)
val leadersPerBroker: Map[Int, Seq[TopicAndPartition]] = filteredTopicMetadata.flatMap { topicMetadataResponse =>
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, _) =>
topicAndPartition
@ -200,8 +201,6 @@ object ReplicaVerificationTool extends Logging {
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 class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPartition, Int],
@ -276,41 +275,42 @@ private class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPa
assert(fetchResponsePerReplica.size == expectedReplicasPerTopicAndPartition(topicAndPartition),
"fetched " + fetchResponsePerReplica.size + " replicas for " + topicAndPartition + ", but expected "
+ expectedReplicasPerTopicAndPartition(topicAndPartition) + " replicas")
val messageIteratorMap = fetchResponsePerReplica.map {
val logEntryIteratorMap = fetchResponsePerReplica.map {
case(replicaId, fetchResponse) =>
replicaId -> fetchResponse.messages.asInstanceOf[ByteBufferMessageSet].shallowIterator}
replicaId -> fetchResponse.messages.asInstanceOf[ByteBufferMessageSet].asRecords.shallowIterator.asScala
}
val maxHw = fetchResponsePerReplica.values.map(_.hw).max
// Iterate one message at a time from every replica, until high watermark is reached.
var isMessageInAllReplicas = true
while (isMessageInAllReplicas) {
var messageInfoFromFirstReplicaOpt: Option[MessageInfo] = None
for ( (replicaId, messageIterator) <- messageIteratorMap) {
for ( (replicaId, logEntries) <- logEntryIteratorMap) {
try {
if (messageIterator.hasNext) {
val messageAndOffset = messageIterator.next()
if (logEntries.hasNext) {
val logEntry = logEntries.next()
// only verify up to the high watermark
if (messageAndOffset.offset >= fetchResponsePerReplica.get(replicaId).hw)
if (logEntry.offset >= fetchResponsePerReplica.get(replicaId).hw)
isMessageInAllReplicas = false
else {
messageInfoFromFirstReplicaOpt match {
case None =>
messageInfoFromFirstReplicaOpt = Some(
MessageInfo(replicaId, messageAndOffset.offset,messageAndOffset.nextOffset, messageAndOffset.message.checksum))
MessageInfo(replicaId, logEntry.offset,logEntry.nextOffset, logEntry.record.checksum))
case Some(messageInfoFromFirstReplica) =>
if (messageInfoFromFirstReplica.offset != messageAndOffset.offset) {
if (messageInfoFromFirstReplica.offset != logEntry.offset) {
println(ReplicaVerificationTool.getCurrentTimeString + ": partition " + topicAndPartition
+ ": replica " + messageInfoFromFirstReplica.replicaId + "'s offset "
+ messageInfoFromFirstReplica.offset + " doesn't match replica "
+ replicaId + "'s offset " + messageAndOffset.offset)
+ replicaId + "'s offset " + logEntry.offset)
System.exit(1)
}
if (messageInfoFromFirstReplica.checksum != messageAndOffset.message.checksum)
if (messageInfoFromFirstReplica.checksum != logEntry.record.checksum)
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
+ "; replica " + replicaId + "'s checksum " + messageAndOffset.message.checksum)
+ "; replica " + replicaId + "'s checksum " + logEntry.record.checksum)
}
}
} else

View File

@ -50,11 +50,13 @@ class GroupCoordinatorIntegrationTest extends KafkaServerTestHarness {
def getGroupMetadataLogOpt: Option[Log] =
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")
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)
consumer.close()

View File

@ -21,12 +21,15 @@ import joptsimple.OptionParser
import java.util.Properties
import java.util.Random
import java.io._
import kafka.consumer._
import kafka.serializer._
import kafka.utils._
import kafka.log.FileMessageSet
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:
@ -135,15 +138,15 @@ object TestLogCleaning {
def dumpLog(dir: File) {
require(dir.exists, "Non-existent directory: " + dir.getAbsolutePath)
for(file <- dir.list.sorted; if file.endsWith(Log.LogFileSuffix)) {
val ms = new FileMessageSet(new File(dir, file))
for(entry <- ms) {
val key = TestUtils.readString(entry.message.key)
for (file <- dir.list.sorted; if file.endsWith(Log.LogFileSuffix)) {
val fileRecords = FileRecords.open(new File(dir, file))
for (entry <- fileRecords.shallowIterator.asScala) {
val key = TestUtils.readString(entry.record.key)
val content =
if(entry.message.isNull)
if(entry.record.hasNullValue)
null
else
TestUtils.readString(entry.message.payload)
TestUtils.readString(entry.record.value)
println("offset = %s, key = %s, content = %s".format(entry.offset, key, content))
}
}

View File

@ -20,10 +20,10 @@ package kafka
import java.util.Properties
import java.util.concurrent.atomic._
import kafka.message._
import kafka.log._
import kafka.utils._
import org.apache.kafka.clients.consumer.OffsetOutOfRangeException
import org.apache.kafka.common.record.FileRecords
import org.apache.kafka.common.utils.Utils
/**
@ -36,13 +36,13 @@ object StressTestLog {
def main(args: Array[String]) {
val dir = TestUtils.randomPartitionLogDir(TestUtils.tempDir())
val time = new MockTime
val logProprties = new Properties()
logProprties.put(LogConfig.SegmentBytesProp, 64*1024*1024: java.lang.Integer)
logProprties.put(LogConfig.MaxMessageBytesProp, Int.MaxValue: java.lang.Integer)
logProprties.put(LogConfig.SegmentIndexBytesProp, 1024*1024: java.lang.Integer)
val logProperties = new Properties()
logProperties.put(LogConfig.SegmentBytesProp, 64*1024*1024: java.lang.Integer)
logProperties.put(LogConfig.MaxMessageBytesProp, Int.MaxValue: java.lang.Integer)
logProperties.put(LogConfig.SegmentIndexBytesProp, 1024*1024: java.lang.Integer)
val log = new Log(dir = dir,
config = LogConfig(logProprties),
config = LogConfig(logProperties),
recoveryPoint = 0L,
scheduler = time.scheduler,
time = time)
@ -84,7 +84,7 @@ object StressTestLog {
class WriterThread(val log: Log) extends WorkerThread {
@volatile var offset = 0
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)
offset += 1
if(offset % 1000 == 0)
@ -96,11 +96,11 @@ object StressTestLog {
@volatile var offset = 0
override def work() {
try {
log.read(offset, 1024, Some(offset+1)).messageSet match {
case read: FileMessageSet if read.sizeInBytes > 0 => {
val first = read.head
log.read(offset, 1024, Some(offset+1)).records match {
case read: FileRecords if read.sizeInBytes > 0 => {
val first = read.shallowIterator.next()
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
}
case _ =>

View File

@ -22,13 +22,14 @@ import java.nio._
import java.nio.channels._
import java.util.{Properties, Random}
import joptsimple._
import kafka.log._
import kafka.utils._
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 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.
@ -64,7 +65,7 @@ object TestLinearWriteSpeed {
.withRequiredArg
.describedAs("ms")
.ofType(classOf[java.lang.Long])
.defaultsTo(1000)
.defaultsTo(1000L)
val maxThroughputOpt = parser.accepts("max-throughput-mb", "The maximum throughput.")
.withRequiredArg
.describedAs("mb")
@ -81,7 +82,7 @@ object TestLinearWriteSpeed {
.ofType(classOf[java.lang.String])
.defaultsTo(NoCompressionCodec.name)
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 options = parser.parse(args : _*)
@ -101,9 +102,9 @@ object TestLinearWriteSpeed {
val rand = new Random
rand.nextBytes(buffer.array)
val numMessages = bufferSize / (messageSize + MessageSet.LogOverhead)
val messageSet = new ByteBufferMessageSet(compressionCodec = compressionCodec,
messages = (0 until numMessages).map(_ => new Message(new Array[Byte](messageSize))): _*)
val messageSet = MemoryRecords.withRecords(CompressionType.forId(compressionCodec.codec),
(0 until numMessages).map(_ => Record.create(new Array[Byte](messageSize))): _*)
val writables = new Array[Writable](numFiles)
val scheduler = new KafkaScheduler(1)
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)
val log = new Log(dir, config, 0L, scheduler, Time.SYSTEM)
def write(): Int = {

View File

@ -301,7 +301,7 @@ class DeleteTopicTest extends ZooKeeperTestHarness {
var counter = 0
for (_ <- 0 until numDups; key <- 0 until numKeys) yield {
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
(key, count)
}

View File

@ -18,20 +18,20 @@
package kafka.coordinator
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 kafka.common.{OffsetAndMetadata, Topic}
import kafka.message.{Message, MessageSet}
import kafka.server.{DelayedOperationPurgatory, ReplicaManager, KafkaConfig}
import kafka.server.{DelayedOperationPurgatory, KafkaConfig, ReplicaManager}
import kafka.utils._
import org.apache.kafka.common.TopicPartition
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.easymock.{Capture, IAnswer, EasyMock}
import org.easymock.{Capture, EasyMock, IAnswer}
import org.junit.{After, Before, Test}
import org.scalatest.junit.JUnitSuite
import java.util.concurrent.TimeUnit
import scala.collection._
import scala.concurrent.duration.Duration
import scala.concurrent.{Await, Future, Promise}
@ -305,7 +305,8 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
EasyMock.reset(replicaManager)
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)
timer.advanceClock(DefaultSessionTimeout + 100)
@ -988,17 +989,18 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
val capturedArgument: Capture[Map[TopicPartition, PartitionResponse] => Unit] = EasyMock.newCapture()
EasyMock.expect(replicaManager.appendMessages(EasyMock.anyLong(),
EasyMock.expect(replicaManager.appendRecords(EasyMock.anyLong(),
EasyMock.anyShort(),
EasyMock.anyBoolean(),
EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MessageSet]],
EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MemoryRecords]],
EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] {
override def answer = capturedArgument.getValue.apply(
Map(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId) ->
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)
groupCoordinator.handleSyncGroup(groupId, generation, leaderId, assignment, responseCallback)
@ -1069,17 +1071,18 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
val capturedArgument: Capture[Map[TopicPartition, PartitionResponse] => Unit] = EasyMock.newCapture()
EasyMock.expect(replicaManager.appendMessages(EasyMock.anyLong(),
EasyMock.expect(replicaManager.appendRecords(EasyMock.anyLong(),
EasyMock.anyShort(),
EasyMock.anyBoolean(),
EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MessageSet]],
EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MemoryRecords]],
EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] {
override def answer = capturedArgument.getValue.apply(
Map(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId) ->
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)
groupCoordinator.handleCommitOffsets(groupId, consumerId, generationId, offsets, responseCallback)
@ -1090,7 +1093,8 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
val (responseFuture, responseCallback) = setupHeartbeatCallback
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)
groupCoordinator.handleLeaveGroup(groupId, consumerId, responseCallback)

View File

@ -21,12 +21,11 @@ import kafka.api.ApiVersion
import kafka.cluster.Partition
import kafka.common.{OffsetAndMetadata, Topic}
import kafka.log.LogAppendInfo
import kafka.message.{ByteBufferMessageSet, Message, MessageSet}
import kafka.server.{KafkaConfig, ReplicaManager}
import kafka.utils.{KafkaScheduler, MockTime, TestUtils, ZkUtils}
import org.apache.kafka.common.TopicPartition
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.ProduceResponse.PartitionResponse
import org.easymock.{Capture, EasyMock, IAnswer}
@ -34,6 +33,7 @@ import org.junit.{After, Before, Test}
import org.junit.Assert._
import scala.collection._
import JavaConverters._
class GroupMetadataManagerTest {
@ -50,7 +50,6 @@ class GroupMetadataManagerTest {
val rebalanceTimeout = 60000
val sessionTimeout = 10000
@Before
def setUp() {
val config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(nodeId = 0, zkConnect = ""))
@ -176,7 +175,7 @@ class GroupMetadataManagerTest {
@Test
def testStoreNonEmptyGroupWhenCoordinatorHasMoved() {
EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andReturn(None)
EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject())).andReturn(None)
val memberId = "memberId"
val clientId = "clientId"
val clientHost = "localhost"
@ -245,7 +244,7 @@ class GroupMetadataManagerTest {
@Test
def testCommitOffsetWhenCoordinatorHasMoved() {
EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andReturn(None)
EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject())).andReturn(None)
val memberId = ""
val generationId = -1
val topicPartition = new TopicPartition("foo", 0)
@ -363,7 +362,7 @@ class GroupMetadataManagerTest {
time.sleep(2)
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)
EasyMock.replay(partition)
@ -391,24 +390,74 @@ class GroupMetadataManagerTest {
// expect the group metadata tombstone
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(partition.appendMessagesToLeader(EasyMock.capture(messageSetCapture), EasyMock.anyInt()))
EasyMock.expect(partition.appendRecordsToLeader(EasyMock.capture(recordsCapture), EasyMock.anyInt()))
.andReturn(LogAppendInfo.UnknownLogAppendInfo)
EasyMock.replay(replicaManager, partition)
groupMetadataManager.cleanupGroupMetadata()
assertTrue(messageSetCapture.hasCaptured)
assertTrue(recordsCapture.hasCaptured)
val messageSet = messageSetCapture.getValue
assertEquals(1, messageSet.size)
val records = recordsCapture.getValue.records.asScala.toList
assertEquals(1, records.size)
val metadataTombstone = messageSet.head.message
val metadataTombstone = records.head
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]
assertEquals(groupId, groupKey.key)
@ -463,22 +512,22 @@ class GroupMetadataManagerTest {
// expect the offset tombstone
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)
EasyMock.replay(partition)
groupMetadataManager.cleanupGroupMetadata()
assertTrue(messageSetCapture.hasCaptured)
assertTrue(recordsCapture.hasCaptured)
// verify the tombstones are correct and only for the expired offsets
val messageSet = messageSetCapture.getValue
assertEquals(2, messageSet.size)
messageSet.map(_.message).foreach { message =>
val records = recordsCapture.getValue.records.asScala.toList
assertEquals(2, records.size)
records.foreach { message =>
assertTrue(message.hasKey)
assertTrue(message.isNull)
assertTrue(message.hasNullValue)
val offsetKey = GroupMetadataManager.readMessageKey(message.key).asInstanceOf[OffsetKey]
assertEquals(groupId, offsetKey.key.group)
assertEquals("foo", offsetKey.key.topicPartition.topic)
@ -539,7 +588,7 @@ class GroupMetadataManagerTest {
// expect the offset tombstone
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)
EasyMock.replay(partition)
@ -557,17 +606,18 @@ class GroupMetadataManagerTest {
private def expectAppendMessage(error: Errors) {
val capturedArgument: Capture[Map[TopicPartition, PartitionResponse] => Unit] = EasyMock.newCapture()
EasyMock.expect(replicaManager.appendMessages(EasyMock.anyLong(),
EasyMock.expect(replicaManager.appendRecords(EasyMock.anyLong(),
EasyMock.anyShort(),
EasyMock.anyBoolean(),
EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MessageSet]],
EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MemoryRecords]],
EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] {
override def answer = capturedArgument.getValue.apply(
Map(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId) ->
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))
}
}

View File

@ -25,9 +25,10 @@ import org.junit.Assert._
import org.junit.runner.RunWith
import org.junit.runners.Parameterized
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 java.util.{Collection, Properties}
import scala.collection.JavaConverters._
@RunWith(value = classOf[Parameterized])
@ -50,22 +51,22 @@ class BrokerCompressionTest(messageCompression: String, brokerCompression: Strin
def testBrokerSideCompression() {
val messageCompressionCode = CompressionCodec.getCompressionCodec(messageCompression)
val logProps = new Properties()
logProps.put(LogConfig.CompressionTypeProp,brokerCompression)
logProps.put(LogConfig.CompressionTypeProp, brokerCompression)
/*configure broker-side compression */
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
/* 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")) {
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
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)
}
}

View File

@ -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
}
}
}
}

View File

@ -22,10 +22,9 @@ import java.util.Properties
import kafka.api.{KAFKA_0_10_0_IV1, KAFKA_0_9_0}
import kafka.common.TopicAndPartition
import kafka.message._
import kafka.server.OffsetCheckpoint
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.junit.Assert._
import org.junit._
@ -43,7 +42,7 @@ import scala.util.Random
@RunWith(value = classOf[Parameterized])
class LogCleanerIntegrationTest(compressionCodec: String) {
val codec = CompressionCodec.getCompressionCodec(compressionCodec)
val codec = CompressionType.forName(compressionCodec)
val time = new MockTime()
val segmentSize = 256
val deleteDelay = 1000
@ -56,7 +55,7 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
@Test
def cleanerTest() {
val largeMessageKey = 20
val (largeMessageValue, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, Message.MagicValue_V1)
val (largeMessageValue, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, Record.MAGIC_VALUE_V1)
val maxMessageSize = largeMessageSet.sizeInBytes
cleaner = makeCleaner(parts = 3, maxMessageSize = maxMessageSize)
@ -133,13 +132,13 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
}
// 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 = {
val random = new Random(0)
new String(random.alphanumeric.take(length).toArray)
}
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)
(value, messageSet)
}
@ -147,9 +146,9 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
@Test
def testCleanerWithMessageFormatV0(): Unit = {
val largeMessageKey = 20
val (largeMessageValue, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, Message.MagicValue_V0)
val (largeMessageValue, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, Record.MAGIC_VALUE_V0)
val maxMessageSize = codec match {
case NoCompressionCodec => largeMessageSet.sizeInBytes
case CompressionType.NONE => largeMessageSet.sizeInBytes
case _ =>
// the broker assigns absolute offsets for message format 0 which potentially causes the compressed size to
// increase because the broker offsets are larger than the ones assigned by the client
@ -165,7 +164,7 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
props.put(LogConfig.MessageFormatVersionProp, KAFKA_0_9_0.version)
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
cleaner.startup()
@ -177,14 +176,14 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
checkLogAfterAppendingDups(log, startSize, appends)
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 largeMessageOffset = appendInfo.firstOffset
// 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)
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
}
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
// all of the individual messages
var appendsV0 = writeDupsSingleMessageSet(numKeys = 2, numDups = 3, log = log, codec = codec, magicValue = Message.MagicValue_V0)
appendsV0 ++= writeDupsSingleMessageSet(numKeys = 2, startKey = 3, numDups = 2, 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 = Record.MAGIC_VALUE_V0)
props.put(LogConfig.MessageFormatVersionProp, KAFKA_0_10_0_IV1.version)
log.config = new LogConfig(props)
var 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 = Message.MagicValue_V1)
appendsV1 ++= writeDupsSingleMessageSet(startKey = 6, 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 = Record.MAGIC_VALUE_V1)
appendsV1 ++= writeDupsSingleMessageSet(startKey = 6, numKeys = 2, numDups = 2, log = log, codec = codec, magicValue = Record.MAGIC_VALUE_V1)
val appends = appendsV0 ++ appendsV1
@ -250,32 +249,27 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
}
private def readFromLog(log: Log): Iterable[(Int, String, Long)] = {
def messageIterator(entry: MessageAndOffset): Iterator[MessageAndOffset] =
// create single message iterator or deep iterator depending on compression codec
if (entry.message.compressionCodec == NoCompressionCodec) Iterator(entry)
else ByteBufferMessageSet.deepIterator(entry)
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)
import JavaConverters._
for (segment <- log.logSegments; deepLogEntry <- segment.log.deepIterator.asScala) yield {
val key = TestUtils.readString(deepLogEntry.record.key).toInt
val value = TestUtils.readString(deepLogEntry.record.value)
(key, value, deepLogEntry.offset)
}
}
private def writeDups(numKeys: Int, numDups: Int, log: Log, codec: CompressionCodec,
startKey: Int = 0, magicValue: Byte = Message.CurrentMagicValue): Seq[(Int, String, Long)] = {
private def writeDups(numKeys: Int, numDups: Int, log: Log, codec: CompressionType,
startKey: Int = 0, magicValue: Byte = Record.CURRENT_MAGIC_VALUE): Seq[(Int, String, Long)] = {
for(_ <- 0 until numDups; key <- startKey until (startKey + numKeys)) yield {
val payload = counter.toString
val appendInfo = log.append(TestUtils.singleMessageSet(payload = payload.toString.getBytes, codec = codec,
val value = counter.toString
val appendInfo = log.append(TestUtils.singletonRecords(value = value.toString.getBytes, codec = codec,
key = key.toString.getBytes, magicValue = magicValue), assignOffsets = true)
counter += 1
(key, payload, appendInfo.firstOffset)
(key, value, appendInfo.firstOffset)
}
}
private def writeDupsSingleMessageSet(numKeys: Int, numDups: Int, log: Log, codec: CompressionCodec,
startKey: Int = 0, magicValue: Byte = Message.CurrentMagicValue): Seq[(Int, String, Long)] = {
private def writeDupsSingleMessageSet(numKeys: Int, numDups: Int, log: Log, codec: CompressionType,
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 payload = counter.toString
counter += 1
@ -283,11 +277,11 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
}
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 appendInfo = log.append(messageSet, assignOffsets = true)
val records = MemoryRecords.withRecords(codec, messages: _*)
val appendInfo = log.append(records, assignOffsets = true)
val offsets = appendInfo.firstOffset to appendInfo.lastOffset
kvs.zip(offsets).map { case (kv, offset) => (kv._1, kv._2, offset) }

View File

@ -21,7 +21,6 @@ import java.io.File
import java.util.Properties
import kafka.common.TopicAndPartition
import kafka.message._
import kafka.utils._
import org.apache.kafka.common.record.CompressionType
import org.apache.kafka.common.utils.Utils
@ -33,7 +32,6 @@ import org.junit.runners.Parameterized.Parameters
import scala.collection._
/**
* 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()
var counter = 0
val topics = Array(TopicAndPartition("log", 0), TopicAndPartition("log", 1), TopicAndPartition("log", 2))
val compressionCodec = CompressionCodec.getCompressionCodec(compressionCodecName)
val compressionCodec = CompressionType.forName(compressionCodecName)
@Test
def cleanerTest(): Unit = {
@ -96,7 +94,7 @@ class LogCleanerLagIntegrationTest(compressionCodecName: String) extends Logging
val compactedSize = log.logSegments(0L, activeSegAtT0.baseOffset).map(_.size).sum
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 should have been compacted: size up to offset of active segment at T0=$sizeUpToActiveSegmentAtT0 compacted size=$compactedSize",
sizeUpToActiveSegmentAtT0 > compactedSize)
@ -106,23 +104,19 @@ class LogCleanerLagIntegrationTest(compressionCodecName: String) extends Logging
}
private def readFromLog(log: Log): Iterable[(Int, Int)] = {
for (segment <- log.logSegments; entry <- segment.log; messageAndOffset <- {
// create single message iterator or deep iterator depending on compression codec
if (entry.message.compressionCodec == NoCompressionCodec)
Stream.cons(entry, Stream.empty).iterator
else
ByteBufferMessageSet.deepIterator(entry)
}) yield {
val key = TestUtils.readString(messageAndOffset.message.key).toInt
val value = TestUtils.readString(messageAndOffset.message.payload).toInt
import JavaConverters._
for (segment <- log.logSegments; logEntry <- segment.log.deepIterator.asScala) yield {
val key = TestUtils.readString(logEntry.record.key).toInt
val value = TestUtils.readString(logEntry.record.value).toInt
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 {
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
(key, count)
}

View File

@ -21,8 +21,8 @@ import java.io.File
import java.util.Properties
import kafka.common._
import kafka.message._
import kafka.utils._
import org.apache.kafka.common.record.{MemoryRecords, Record}
import org.apache.kafka.common.utils.Utils
import org.junit.Assert._
import org.junit.{After, Test}
@ -54,8 +54,8 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
*/
@Test
def testLogsWithSegmentsToDeleteShouldNotConsiderCleanupPolicyDeleteLogs(): Unit = {
val messageSet = TestUtils.singleMessageSet("test".getBytes)
val log: Log = createLog(messageSet.sizeInBytes * 5, LogConfig.Delete)
val records = TestUtils.singletonRecords("test".getBytes)
val log: Log = createLog(records.sizeInBytes * 5, LogConfig.Delete)
val cleanerManager: LogCleanerManager = createCleanerManager(log)
val readyToDelete = cleanerManager.deletableLogs().size
@ -67,8 +67,8 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
*/
@Test
def testLogsWithSegmentsToDeleteShouldConsiderCleanupPolicyCompactDeleteLogs(): Unit = {
val messageSet = TestUtils.singleMessageSet("test".getBytes, key="test".getBytes)
val log: Log = createLog(messageSet.sizeInBytes * 5, LogConfig.Compact + "," + LogConfig.Delete)
val records = TestUtils.singletonRecords("test".getBytes, key="test".getBytes)
val log: Log = createLog(records.sizeInBytes * 5, LogConfig.Compact + "," + LogConfig.Delete)
val cleanerManager: LogCleanerManager = createCleanerManager(log)
val readyToDelete = cleanerManager.deletableLogs().size
@ -81,8 +81,8 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
*/
@Test
def testLogsWithSegmentsToDeleteShouldNotConsiderCleanupPolicyCompactLogs(): Unit = {
val messageSet = TestUtils.singleMessageSet("test".getBytes, key="test".getBytes)
val log: Log = createLog(messageSet.sizeInBytes * 5, LogConfig.Compact)
val records = TestUtils.singletonRecords("test".getBytes, key="test".getBytes)
val log: Log = createLog(records.sizeInBytes * 5, LogConfig.Compact)
val cleanerManager: LogCleanerManager = createCleanerManager(log)
val readyToDelete = cleanerManager.deletableLogs().size
@ -100,7 +100,7 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
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 lastClean = Map(topicAndPartition-> 0L)
@ -123,7 +123,7 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
val t0 = time.milliseconds
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
@ -131,7 +131,7 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
val t1 = time.milliseconds
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 lastClean = Map(topicAndPartition-> 0L)
@ -155,7 +155,7 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
val t0 = time.milliseconds
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)
@ -192,10 +192,7 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
private def makeLog(dir: File = logDir, config: LogConfig = logConfig) =
new Log(dir = dir, config = config, recoveryPoint = 0L, scheduler = time.scheduler, time = time)
private def message(key: Int, value: Int, timestamp: Long) =
new ByteBufferMessageSet(new Message(key = key.toString.getBytes,
bytes = value.toString.getBytes,
timestamp = timestamp,
magicValue = Message.MagicValue_V1))
private def logEntries(key: Int, value: Int, timestamp: Long) =
MemoryRecords.withRecords(Record.create(timestamp, key.toString.getBytes, value.toString.getBytes))
}

View File

@ -17,21 +17,21 @@
package kafka.log
import java.io.{DataOutputStream, File}
import java.io.File
import java.nio._
import java.nio.file.Paths
import java.util.Properties
import kafka.common._
import kafka.message._
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.junit.Assert._
import org.junit.{After, Test}
import org.scalatest.junit.JUnitSuite
import scala.collection._
import JavaConverters._
/**
* 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
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)
assertEquals(0L until log.logEndOffset, keysFound)
@ -100,7 +100,7 @@ class LogCleanerTest extends JUnitSuite {
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
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)
assertEquals(0L until log.logEndOffset, keysFound)
@ -123,23 +123,23 @@ class LogCleanerTest extends JUnitSuite {
logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer)
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
// append messages with the keys 0 through N
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
val leo = log.logEndOffset
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
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))
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(_)))
}
@ -151,11 +151,11 @@ class LogCleanerTest extends JUnitSuite {
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
log.append(message(0,0)) // offset 0
log.append(message(1,1)) // offset 1
log.append(message(0,0)) // offset 2
log.append(message(1,1)) // offset 3
log.append(message(0,0)) // offset 4
log.append(record(0,0)) // offset 0
log.append(record(1,1)) // offset 1
log.append(record(0,0)) // offset 2
log.append(record(1,1)) // offset 3
log.append(record(0,0)) // offset 4
// roll the segment, so we can clean the messages already appended
log.roll()
@ -180,11 +180,11 @@ class LogCleanerTest extends JUnitSuite {
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
log.append(message(0,0)) // offset 0
log.append(message(1,1)) // offset 1
log.append(message(0,0)) // offset 2
log.append(message(1,1)) // offset 3
log.append(message(0,0)) // offset 4
log.append(record(0,0)) // offset 0
log.append(record(1,1)) // offset 1
log.append(record(0,0)) // offset 2
log.append(record(1,1)) // offset 3
log.append(record(0,0)) // offset 4
// roll the segment, so we can clean the messages already appended
log.roll()
@ -218,18 +218,18 @@ class LogCleanerTest extends JUnitSuite {
// append messages with the keys 0 through N-1, values equal offset
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
// the entire segment containing the first uncleanable offset should not be cleaned.
val firstUncleanableOffset = log.logEndOffset + 1 // +1 so it is past the baseOffset
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
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
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))
// 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)
log.append(messageSet, assignOffsets = true)
@ -271,7 +271,7 @@ class LogCleanerTest extends JUnitSuite {
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
// 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)
log.append(messageSet, assignOffsets = true)
@ -305,14 +305,14 @@ class LogCleanerTest extends JUnitSuite {
// append unkeyed messages
while(log.numberOfSegments < 2)
log.append(unkeyedMessage(log.logEndOffset.toInt))
log.append(unkeyedRecord(log.logEndOffset.toInt))
val numInvalidMessages = unkeyedMessageCountInLog(log)
val sizeWithUnkeyedMessages = log.size
// append keyed messages
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 (_, 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("Cleaner should have seen %d invalid messages.", numInvalidMessages, stats.invalidMessagesRead)
}
/* extract all the keys from a log */
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 */
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) =
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 = {
throw new LogCleaningAbortedException()
@ -350,7 +350,7 @@ class LogCleanerTest extends JUnitSuite {
// append messages to the log until we have four segments
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 map = new FakeOffsetMap(Int.MaxValue)
@ -371,20 +371,20 @@ class LogCleanerTest extends JUnitSuite {
logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer)
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
// append some messages to the log
var i = 0
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
}
// 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)
assertEquals(1, groups.size)
assertEquals(log.numberOfSegments, groups.head.size)
checkSegmentOrder(groups)
// grouping by very small values should result in all groups having one entry
groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = 1, maxIndexSize = Int.MaxValue)
assertEquals(log.numberOfSegments, groups.size)
@ -396,20 +396,20 @@ class LogCleanerTest extends JUnitSuite {
checkSegmentOrder(groups)
val groupSize = 3
// check grouping by log size
val logSize = log.logSegments.take(groupSize).map(_.size).sum.toInt + 1
groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = logSize, maxIndexSize = Int.MaxValue)
checkSegmentOrder(groups)
assertTrue("All but the last group should be the target size.", groups.dropRight(1).forall(_.size == groupSize))
// check grouping by index size
val indexSize = log.logSegments.take(groupSize).map(_.index.sizeInBytes).sum + 1
groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = indexSize)
checkSegmentOrder(groups)
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
* 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)
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
// fill up first segment
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
val messageSet = new ByteBufferMessageSet(NoCompressionCodec, new LongRef(Int.MaxValue - 1),
new Message("hello".getBytes, "hello".getBytes, Message.NoTimestamp, Message.MagicValue_V1))
log.append(messageSet, assignOffsets = false)
log.append(TestUtils.singleMessageSet(payload = "hello".getBytes, key = "hello".getBytes))
val records = MemoryRecords.withLogEntries(LogEntry.create(Int.MaxValue - 1, Record.create("hello".getBytes, "hello".getBytes)))
log.append(records, assignOffsets = false)
log.append(TestUtils.singletonRecords(value = "hello".getBytes, key = "hello".getBytes))
assertEquals(Int.MaxValue, log.activeSegment.index.lastOffset)
// 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)
assertEquals(1, groups.size)
// 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
groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue)
assertEquals(2, groups.size)
checkSegmentOrder(groups)
// append more messages, creating new segments, further grouping should still occur
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)
assertEquals(log.numberOfSegments - 1, groups.size)
for (group <- groups)
assertTrue("Relative offset greater than Int.MaxValue", group.last.index.lastOffset - group.head.index.baseOffset <= Int.MaxValue)
checkSegmentOrder(groups)
}
private def checkSegmentOrder(groups: Seq[Seq[LogSegment]]): Unit = {
val offsets = groups.flatMap(_.map(_.baseOffset))
assertEquals("Offsets should be in increasing order.", offsets.sorted, offsets)
}
/**
* 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(3).baseOffset.toInt, log.logEndOffset.toInt)
}
/**
* Tests recovery if broker crashes at the following stages during the cleaning sequence
* <ol>
@ -516,8 +513,8 @@ class LogCleanerTest extends JUnitSuite {
logProps.put(LogConfig.FileDeleteDelayMsProp, 10: java.lang.Integer)
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
// and all temporary files have been deleted
val recoveredLog = makeLog(config = config)
@ -530,25 +527,25 @@ class LogCleanerTest extends JUnitSuite {
assertEquals(expectedKeys, keysInLog(recoveredLog))
recoveredLog
}
// create a log and append some messages
var log = makeLog(config = config)
var messageCount = 0
while(log.numberOfSegments < 10) {
log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt))
log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
messageCount += 1
}
val allKeys = keysInLog(log)
// pretend we have odd-numbered keys
val offsetMap = new FakeOffsetMap(Int.MaxValue)
for (k <- 1 until messageCount by 2)
offsetMap.put(key(k), Long.MaxValue)
// clean the log
cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats())
var cleanedKeys = keysInLog(log)
// 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
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, "")))
}
log = recoverAndCheck(config, allKeys)
// clean again
cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats())
cleanedKeys = keysInLog(log)
// 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)
for (file <- dir.listFiles if file.getName.endsWith(Log.DeletedFileSuffix)) {
Utils.atomicMoveWithFallback(file.toPath, Paths.get(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, "")))
}
}
log = recoverAndCheck(config, cleanedKeys)
// add some more messages and clean the log again
while(log.numberOfSegments < 10) {
log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt))
log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
messageCount += 1
}
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())
cleanedKeys = keysInLog(log)
// 3) Simulate recovery after swap file is created and old segments files are renamed
// to .deleted. Clean operation is resumed during recovery.
log.logSegments.head.changeFileSuffixes("", Log.SwapFileSuffix)
log = recoverAndCheck(config, cleanedKeys)
// add some more messages and clean the log again
while(log.numberOfSegments < 10) {
log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt))
log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
messageCount += 1
}
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())
cleanedKeys = keysInLog(log)
// 4) Simulate recovery after swap is complete, but async deletion
// is not yet complete. Clean operation is resumed during recovery.
recoverAndCheck(config, cleanedKeys)
@ -631,11 +628,11 @@ class LogCleanerTest extends JUnitSuite {
val log = makeLog()
val cleaner = makeCleaner(2)
log.append(message(0,0))
log.append(message(1,1))
log.append(message(2,2))
log.append(message(3,3))
log.append(message(4,4))
log.append(record(0,0))
log.append(record(1,1))
log.append(record(2,2))
log.append(record(3,3))
log.append(record(4,4))
log.roll()
val stats = new CleanerStats()
@ -653,7 +650,7 @@ class LogCleanerTest extends JUnitSuite {
*/
@Test
def testCleanCorruptMessageSet() {
val codec = SnappyCompressionCodec
val codec = CompressionType.GZIP
val logProps = new Properties()
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))
for (segment <- log.logSegments; shallowMessage <- segment.log.iterator; deepMessage <- ByteBufferMessageSet.deepIterator(shallowMessage)) {
assertEquals(shallowMessage.message.magic, deepMessage.message.magic)
val value = TestUtils.readString(deepMessage.message.payload).toLong
assertEquals(deepMessage.offset, value)
for (segment <- log.logSegments; shallowLogEntry <- segment.log.shallowIterator.asScala; deepLogEntry <- shallowLogEntry.asScala) {
assertEquals(shallowLogEntry.record.magic, deepLogEntry.record.magic)
val value = TestUtils.readString(deepLogEntry.record.value).toLong
assertEquals(deepLogEntry.offset, value)
}
}
@ -704,7 +701,7 @@ class LogCleanerTest extends JUnitSuite {
val corruptedMessage = invalidCleanedMessage(offset, set)
val records = MemoryRecords.readableRecords(corruptedMessage.buffer)
for (logEntry <- records.iterator.asScala) {
for (logEntry <- records.deepIterator.asScala) {
val offset = logEntry.offset
val value = TestUtils.readString(logEntry.record.value).toLong
assertEquals(offset, value)
@ -718,94 +715,64 @@ class LogCleanerTest extends JUnitSuite {
private def invalidCleanedMessage(initialOffset: Long,
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
// would write invalid compressed message sets with the outer magic set to 1 and the inner
// 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 =>
new Message(key = kv._1.toString.getBytes,
bytes = kv._2.toString.getBytes,
timestamp = Message.NoTimestamp,
magicValue = Message.MagicValue_V0))
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, codec, TimestampType.CREATE_TIME)
val messageWriter = new MessageWriter(math.min(math.max(MessageSet.messageSetSize(messages) / 2, 1024), 1 << 16))
var lastOffset = initialOffset
messageWriter.write(
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()
}
var offset = initialOffset
records.foreach { record =>
builder.appendUnchecked(offset, record)
offset += 1
}
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) =
new ByteBufferMessageSet(NoCompressionCodec, Seq(offset),
new Message(key = key.toString.getBytes,
bytes = value.toString.getBytes,
timestamp = Message.NoTimestamp,
magicValue = Message.MagicValue_V1))
MemoryRecords.withLogEntries(LogEntry.create(offset, Record.create(key.toString.getBytes, value.toString.getBytes)))
def makeLog(dir: File = dir, config: LogConfig = logConfig) =
new Log(dir = dir, config = config, recoveryPoint = 0L, scheduler = time.scheduler, time = time)
def noOpCheckDone(topicAndPartition: TopicAndPartition) { /* do nothing */ }
def makeCleaner(capacity: Int, checkDone: (TopicAndPartition) => Unit = noOpCheckDone, maxMessageSize: Int = 64*1024) =
new Cleaner(id = 0,
offsetMap = new FakeOffsetMap(capacity),
new Cleaner(id = 0,
offsetMap = new FakeOffsetMap(capacity),
ioBufferSize = maxMessageSize,
maxIoBufferSize = maxMessageSize,
dupBufferLoadFactor = 0.75,
throttler = throttler,
throttler = throttler,
time = time,
checkDone = checkDone )
def writeToLog(log: Log, seq: Iterable[(Int, Int)]): Iterable[Long] = {
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 message(key: Int, value: Int): ByteBufferMessageSet =
message(key, value.toString.getBytes)
def message(key: Int, value: Array[Byte]) =
new ByteBufferMessageSet(new Message(key = key.toString.getBytes,
bytes = value,
timestamp = Message.NoTimestamp,
magicValue = Message.MagicValue_V1))
def record(key: Int, value: Int): MemoryRecords =
record(key, value.toString.getBytes)
def unkeyedMessage(value: Int) =
new ByteBufferMessageSet(new Message(bytes = value.toString.getBytes))
def record(key: Int, value: Array[Byte]) =
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 {

View File

@ -67,7 +67,7 @@ class LogManagerTest {
val log = logManager.createLog(TopicAndPartition(name, 0), logConfig)
val logFile = new File(logDir, name + "-0")
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)
var offset = 0L
for(_ <- 0 until 200) {
val set = TestUtils.singleMessageSet("test".getBytes())
val set = TestUtils.singletonRecords("test".getBytes())
val info = log.append(set)
offset = info.lastOffset
}
@ -101,7 +101,7 @@ class LogManagerTest {
assertEquals("Now there should only be only one segment in the index.", 1, log.numberOfSegments)
time.sleep(log.config.fileDeleteDelayMs + 1)
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 {
log.read(0, 1024)
@ -110,7 +110,7 @@ class LogManagerTest {
case _: OffsetOutOfRangeException => // This is good.
}
// log should still be appendable
log.append(TestUtils.singleMessageSet("test".getBytes()))
log.append(TestUtils.singletonRecords("test".getBytes()))
}
/**
@ -118,7 +118,7 @@ class LogManagerTest {
*/
@Test
def testCleanupSegmentsToMaintainSize() {
val setSize = TestUtils.singleMessageSet("test".getBytes()).sizeInBytes
val setSize = TestUtils.singletonRecords("test".getBytes()).sizeInBytes
logManager.shutdown()
val logProps = new Properties()
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
val numMessages = 200
for (_ <- 0 until numMessages) {
val set = TestUtils.singleMessageSet("test".getBytes())
val set = TestUtils.singletonRecords("test".getBytes())
val info = log.append(set)
offset = info.firstOffset
}
@ -147,7 +147,7 @@ class LogManagerTest {
assertEquals("Now there should be exactly 6 segments", 6, log.numberOfSegments)
time.sleep(log.config.fileDeleteDelayMs + 1)
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 {
log.read(0, 1024)
fail("Should get exception from fetching earlier.")
@ -155,7 +155,7 @@ class LogManagerTest {
case _: OffsetOutOfRangeException => // This is good.
}
// 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))
var offset = 0L
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)
offset = info.lastOffset
}
@ -198,7 +198,7 @@ class LogManagerTest {
val log = logManager.createLog(TopicAndPartition(name, 0), config)
val lastFlush = log.lastFlushTime
for (_ <- 0 until 200) {
val set = TestUtils.singleMessageSet("test".getBytes())
val set = TestUtils.singletonRecords("test".getBytes())
log.append(set)
}
time.sleep(logManager.InitialTaskDelayMs)
@ -280,7 +280,7 @@ class LogManagerTest {
val logs = topicAndPartitions.map(this.logManager.createLog(_, logConfig))
logs.foreach(log => {
for (_ <- 0 until 50)
log.append(TestUtils.singleMessageSet("test".getBytes()))
log.append(TestUtils.singletonRecords("test".getBytes()))
log.flush()
})

View File

@ -16,15 +16,13 @@
*/
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.message._
import org.apache.kafka.common.record.{FileRecords, MemoryRecords, Record}
import org.apache.kafka.common.utils.Time
import org.junit.Assert._
import org.junit.{After, Test}
import scala.collection.JavaConverters._
import scala.collection._
class LogSegmentTest {
@ -34,7 +32,7 @@ class LogSegmentTest {
/* create a segment with the given base offset */
def createSegment(offset: Long, indexIntervalBytes: Int = 10): LogSegment = {
val msFile = TestUtils.tempFile()
val ms = new FileMessageSet(msFile)
val ms = FileRecords.open(msFile)
val idxFile = TestUtils.tempFile()
val timeIdxFile = TestUtils.tempFile()
idxFile.delete()
@ -47,12 +45,10 @@ class LogSegmentTest {
}
/* create a ByteBufferMessageSet for the given messages starting from the given offset */
def messages(offset: Long, messages: String*): ByteBufferMessageSet = {
new ByteBufferMessageSet(compressionCodec = NoCompressionCodec,
offsetCounter = new LongRef(offset),
messages = messages.map(s => new Message(s.getBytes, offset * 10, Message.MagicValue_V1)):_*)
def records(offset: Long, records: String*): MemoryRecords = {
MemoryRecords.withRecords(offset, records.map(s => Record.create(Record.MAGIC_VALUE_V1, offset * 10, s.getBytes)):_*)
}
@After
def teardown() {
for(seg <- segments) {
@ -60,7 +56,7 @@ class LogSegmentTest {
seg.log.delete()
}
}
/**
* 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)
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
* beginning with the first message in the segment
@ -78,12 +74,12 @@ class LogSegmentTest {
@Test
def testReadBeforeFirstOffset() {
val seg = createSegment(40)
val ms = messages(50, "hello", "there", "little", "bee")
seg.append(50, Message.NoTimestamp, -1L, ms)
val read = seg.read(startOffset = 41, maxSize = 300, maxOffset = None).messageSet
assertEquals(ms.toList, read.toList)
val ms = records(50, "hello", "there", "little", "bee")
seg.append(50, Record.NO_TIMESTAMP, -1L, ms)
val read = seg.read(startOffset = 41, maxSize = 300, maxOffset = None).records
assertEquals(ms.deepIterator.asScala.toList, read.deepIterator.asScala.toList)
}
/**
* 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
@ -92,28 +88,28 @@ class LogSegmentTest {
def testMaxOffset() {
val baseOffset = 50
val seg = createSegment(baseOffset)
val ms = messages(baseOffset, "hello", "there", "beautiful")
seg.append(baseOffset, Message.NoTimestamp, -1L, ms)
def validate(offset: Long) =
assertEquals(ms.filter(_.offset == offset).toList,
seg.read(startOffset = offset, maxSize = 1024, maxOffset = Some(offset+1)).messageSet.toList)
val ms = records(baseOffset, "hello", "there", "beautiful")
seg.append(baseOffset, Record.NO_TIMESTAMP, -1L, ms)
def validate(offset: Long) =
assertEquals(ms.deepIterator.asScala.filter(_.offset == offset).toList,
seg.read(startOffset = offset, maxSize = 1024, maxOffset = Some(offset+1)).records.deepIterator.asScala.toList)
validate(50)
validate(51)
validate(52)
}
/**
* If we read from an offset beyond the last offset in the segment we should get null
*/
@Test
def testReadAfterLast() {
val seg = createSegment(40)
val ms = messages(50, "hello", "there")
seg.append(50, Message.NoTimestamp, -1L, ms)
val ms = records(50, "hello", "there")
seg.append(50, Record.NO_TIMESTAMP, -1L, ms)
val read = seg.read(startOffset = 52, maxSize = 200, maxOffset = None)
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
* with the least offset greater than the given startOffset.
@ -121,14 +117,14 @@ class LogSegmentTest {
@Test
def testReadFromGap() {
val seg = createSegment(40)
val ms = messages(50, "hello", "there")
seg.append(50, Message.NoTimestamp, -1L, ms)
val ms2 = messages(60, "alpha", "beta")
seg.append(60, Message.NoTimestamp, -1L, ms2)
val ms = records(50, "hello", "there")
seg.append(50, Record.NO_TIMESTAMP, -1L, ms)
val ms2 = records(60, "alpha", "beta")
seg.append(60, Record.NO_TIMESTAMP, -1L, ms2)
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
* the first but not the second message.
@ -138,18 +134,18 @@ class LogSegmentTest {
val seg = createSegment(40)
var offset = 40
for (_ <- 0 until 30) {
val ms1 = messages(offset, "hello")
seg.append(offset, Message.NoTimestamp, -1L, ms1)
val ms2 = messages(offset + 1, "hello")
seg.append(offset + 1, Message.NoTimestamp, -1L, ms2)
val ms1 = records(offset, "hello")
seg.append(offset, Record.NO_TIMESTAMP, -1L, ms1)
val ms2 = records(offset + 1, "hello")
seg.append(offset + 1, Record.NO_TIMESTAMP, -1L, ms2)
// check that we can read back both messages
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
seg.truncateTo(offset + 1)
val read2 = seg.read(offset, None, 10000)
assertEquals(1, read2.messageSet.size)
assertEquals(ms1.head, read2.messageSet.head)
assertEquals(1, read2.records.deepIterator.asScala.size)
assertEquals(ms1.deepIterator.next(), read2.records.deepIterator.next())
offset += 1
}
}
@ -157,10 +153,10 @@ class LogSegmentTest {
@Test
def testReloadLargestTimestampAfterTruncation() {
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
for (_ <- 0 until numMessages) {
seg.append(offset, offset, offset, messages(offset, "hello"))
seg.append(offset, offset, offset, records(offset, "hello"))
offset += 1
}
val expectedNumEntries = numMessages / 2 - 1
@ -179,10 +175,10 @@ class LogSegmentTest {
def testTruncateFull() {
// test the case where we fully truncate the log
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)
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
def testFindOffsetByTimestamp() {
val messageSize = messages(0, s"msg00").sizeInBytes
val messageSize = records(0, s"msg00").sizeInBytes
val seg = createSegment(40, messageSize * 2 - 1)
// Produce some messages
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)
// Search for an indexed timestamp
@ -218,10 +214,10 @@ class LogSegmentTest {
def testNextOffsetCalculation() {
val seg = createSegment(40)
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())
}
/**
* 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.index.file.exists)
}
/**
* Create a segment with some data and an index. Then corrupt the index,
* and recover the segment, the entries should all be readable.
@ -245,12 +241,12 @@ class LogSegmentTest {
def testRecoveryFixesCorruptIndex() {
val seg = createSegment(0)
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
TestUtils.writeNonsenseToFile(indexFile, 5, indexFile.length.toInt)
seg.recover(64*1024)
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() {
val seg = createSegment(0)
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
TestUtils.writeNonsenseToFile(timeIndexFile, 5, timeIndexFile.length.toInt)
seg.recover(64*1024)
@ -271,7 +267,7 @@ class LogSegmentTest {
assertEquals(i + 1, seg.findOffsetByTimestamp(i * 10 + 1).get.offset)
}
}
/**
* Randomly corrupt a log a number of times and attempt recovery.
*/
@ -281,13 +277,15 @@ class LogSegmentTest {
for (_ <- 0 until 10) {
val seg = createSegment(0)
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)
// 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)
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()
}
}
@ -304,12 +302,12 @@ class LogSegmentTest {
@Test
def testCreateWithInitFileSizeAppendMessage() {
val seg = createSegment(40, false, 512*1024*1024, true)
val ms = messages(50, "hello", "there")
seg.append(50, Message.NoTimestamp, -1L, ms)
val ms2 = messages(60, "alpha", "beta")
seg.append(60, Message.NoTimestamp, -1L, ms2)
val ms = records(50, "hello", "there")
seg.append(50, Record.NO_TIMESTAMP, -1L, ms)
val ms2 = records(60, "alpha", "beta")
seg.append(60, Record.NO_TIMESTAMP, -1L, ms2)
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*/
@ -318,12 +316,12 @@ class LogSegmentTest {
val tempDir = TestUtils.tempDir()
val seg = new LogSegment(tempDir, 40, 10, 1000, 0, Time.SYSTEM, false, 512*1024*1024, true)
val ms = messages(50, "hello", "there")
seg.append(50, Message.NoTimestamp, -1L, ms)
val ms2 = messages(60, "alpha", "beta")
seg.append(60, Message.NoTimestamp, -1L, ms2)
val ms = records(50, "hello", "there")
seg.append(50, Record.NO_TIMESTAMP, -1L, ms)
val ms2 = records(60, "alpha", "beta")
seg.append(60, Record.NO_TIMESTAMP, -1L, ms2)
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 oldPosition = seg.log.channel.position
val oldFileSize = seg.log.file.length
@ -336,7 +334,7 @@ class LogSegmentTest {
segments += segReopen
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 position = segReopen.log.channel.position
val fileSize = segReopen.log.file.length

View File

@ -22,15 +22,16 @@ import java.util.Properties
import org.apache.kafka.common.errors.{CorruptRecordException, OffsetOutOfRangeException, RecordBatchTooLargeException, RecordTooLargeException}
import kafka.api.ApiVersion
import kafka.common.LongRef
import org.junit.Assert._
import org.scalatest.junit.JUnitSuite
import org.junit.{After, Before, Test}
import kafka.message._
import kafka.utils._
import kafka.server.KafkaConfig
import org.apache.kafka.common.record._
import org.apache.kafka.common.utils.Utils
import scala.collection.JavaConverters._
class LogTest extends JUnitSuite {
val tmpDir = TestUtils.tempDir()
@ -63,7 +64,7 @@ class LogTest extends JUnitSuite {
*/
@Test
def testTimeBasedLogRoll() {
val set = TestUtils.singleMessageSet("test".getBytes)
val set = TestUtils.singletonRecords("test".getBytes)
val logProps = new Properties()
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.
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)
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)
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)
assertEquals("Log should roll because the timestamp in the message should make the log segment expire.", 6, log.numberOfSegments)
val numSegments = log.numberOfSegments
time.sleep(log.config.segmentMs + 1)
log.append(new ByteBufferMessageSet())
assertEquals("Appending an empty message set should not roll log even if succient time has passed.", numSegments, log.numberOfSegments)
log.append(MemoryRecords.withLogEntries())
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
def testTimeBasedLogRollJitter() {
val set = TestUtils.singleMessageSet("test".getBytes)
val set = TestUtils.singletonRecords("test".getBytes)
val maxJitter = 20 * 60L
val logProps = new Properties()
@ -149,7 +150,7 @@ class LogTest extends JUnitSuite {
*/
@Test
def testSizeBasedLogRoll() {
val set = TestUtils.singleMessageSet("test".getBytes)
val set = TestUtils.singletonRecords("test".getBytes)
val setSize = set.sizeInBytes
val msgPerSeg = 10
val segmentSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages
@ -176,7 +177,7 @@ class LogTest extends JUnitSuite {
def testLoadEmptyLog() {
createEmptyLogs(logDir, 0)
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.
logProps.put(LogConfig.MessageFormatVersionProp, ApiVersion.latestVersion.toString)
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)
log.append(new ByteBufferMessageSet(NoCompressionCodec, messages = messages(i)))
for(i <- 0 until messages.length) {
val read = log.read(i, 100, Some(i+1)).messageSet.head
for(i <- records.indices)
log.append(MemoryRecords.withRecords(records(i)))
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("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)
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 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
for(i <- 0 until messages.length)
log.append(new ByteBufferMessageSet(NoCompressionCodec, new LongRef(messageIds(i)), messages = messages(i)), assignOffsets = false)
for(i <- records.indices)
log.append(MemoryRecords.withLogEntries(LogEntry.create(messageIds(i), records(i))), assignOffsets = false)
for(i <- 50 until messageIds.max) {
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("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
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
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
@ -252,12 +254,11 @@ class LogTest extends JUnitSuite {
logProps.put(LogConfig.SegmentBytesProp, 71: java.lang.Integer)
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 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
for (i <- 0 until messages.length)
log.append(new ByteBufferMessageSet(NoCompressionCodec, new LongRef(messageIds(i)), messages = messages(i)),
assignOffsets = false)
for (i <- records.indices)
log.append(MemoryRecords.withLogEntries(LogEntry.create(messageIds(i), records(i))), assignOffsets = false)
for (i <- 50 until messageIds.max) {
val idx = messageIds.indexWhere(_ >= i)
@ -265,13 +266,13 @@ class LogTest extends JUnitSuite {
log.read(i, 1, minOneMessage = true),
log.read(i, 100, minOneMessage = true),
log.read(i, 100, Some(10000), minOneMessage = true)
).map(_.messageSet.head)
).map(_.records.shallowIterator.next())
reads.foreach { read =>
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)
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 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
for (i <- 0 until messages.length)
log.append(new ByteBufferMessageSet(NoCompressionCodec, new LongRef(messageIds(i)), messages = messages(i)),
assignOffsets = false)
for (i <- records.indices)
log.append(MemoryRecords.withLogEntries(LogEntry.create(messageIds(i), records(i))), assignOffsets = false)
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 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
// partition
val fetchInfo = log.read(i, 1)
assertTrue(fetchInfo.firstMessageSetIncomplete)
assertTrue(fetchInfo.messageSet.isInstanceOf[FileMessageSet])
assertEquals(1, fetchInfo.messageSet.sizeInBytes)
assertTrue(fetchInfo.firstEntryIncomplete)
assertTrue(fetchInfo.records.isInstanceOf[FileRecords])
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)
logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer)
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 {
log.read(0, 1000)
@ -336,7 +336,7 @@ class LogTest extends JUnitSuite {
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)
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
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(_))
log.flush
/* do successive reads to ensure all our messages are there */
var offset = 0L
for(i <- 0 until numMessages) {
val messages = log.read(offset, 1024*1024).messageSet
assertEquals("Offsets not equal", offset, messages.head.offset)
assertEquals("Messages not equal at offset " + offset, messageSets(i).head.message,
messages.head.message.toFormatVersion(messageSets(i).head.message.magic))
offset = messages.head.offset + 1
val messages = log.read(offset, 1024*1024).records.shallowIterator
val head = messages.next()
assertEquals("Offsets not equal", offset, head.offset)
assertEquals("Messages not equal at offset " + offset, messageSets(i).shallowIterator.next().record,
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
assertEquals("Should be no more messages", 0, lastRead.size)
val lastRead = log.read(startOffset = numMessages, maxLength = 1024*1024, maxOffset = Some(numMessages + 1)).records
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
TestUtils.retry(1000L){
@ -383,10 +384,10 @@ class LogTest extends JUnitSuite {
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 */
log.append(new ByteBufferMessageSet(DefaultCompressionCodec, new Message("hello".getBytes), new Message("there".getBytes)))
log.append(new ByteBufferMessageSet(DefaultCompressionCodec, new Message("alpha".getBytes), new Message("beta".getBytes)))
log.append(MemoryRecords.withRecords(CompressionType.GZIP, Record.create("hello".getBytes), Record.create("there".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 */
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)
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
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
assertEquals(currOffset, messagesToAppend)
@ -422,7 +423,7 @@ class LogTest extends JUnitSuite {
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",
currOffset,
log.append(TestUtils.singleMessageSet("hello".getBytes)).firstOffset)
log.append(TestUtils.singletonRecords("hello".getBytes)).firstOffset)
// cleanup the log
log.delete()
@ -435,7 +436,7 @@ class LogTest extends JUnitSuite {
*/
@Test
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
val configSegmentSize = messageSet.sizeInBytes - 1
val logProps = new Properties()
@ -454,17 +455,17 @@ class LogTest extends JUnitSuite {
@Test
def testCompactedTopicConstraints() {
val keyedMessage = new Message(bytes = "this message has a key".getBytes, key = "and here it is".getBytes, Message.NoTimestamp, Message.CurrentMagicValue)
val anotherKeyedMessage = new Message(bytes = "this message also has a key".getBytes, key ="another key".getBytes, Message.NoTimestamp, Message.CurrentMagicValue)
val unkeyedMessage = new Message(bytes = "this message does not have a key".getBytes)
val keyedMessage = Record.create(Record.CURRENT_MAGIC_VALUE, Record.NO_TIMESTAMP, "and here it is".getBytes, "this message has a key".getBytes)
val anotherKeyedMessage = Record.create(Record.CURRENT_MAGIC_VALUE, Record.NO_TIMESTAMP, "another key".getBytes, "this message also has a key".getBytes)
val unkeyedMessage = Record.create("this message does not have a key".getBytes)
val messageSetWithUnkeyedMessage = new ByteBufferMessageSet(NoCompressionCodec, unkeyedMessage, keyedMessage)
val messageSetWithOneUnkeyedMessage = new ByteBufferMessageSet(NoCompressionCodec, unkeyedMessage)
val messageSetWithCompressedKeyedMessage = new ByteBufferMessageSet(GZIPCompressionCodec, keyedMessage)
val messageSetWithCompressedUnkeyedMessage = new ByteBufferMessageSet(GZIPCompressionCodec, keyedMessage, unkeyedMessage)
val messageSetWithUnkeyedMessage = MemoryRecords.withRecords(CompressionType.NONE, unkeyedMessage, keyedMessage)
val messageSetWithOneUnkeyedMessage = MemoryRecords.withRecords(CompressionType.NONE, unkeyedMessage)
val messageSetWithCompressedKeyedMessage = MemoryRecords.withRecords(CompressionType.GZIP, keyedMessage)
val messageSetWithCompressedUnkeyedMessage = MemoryRecords.withRecords(CompressionType.GZIP, keyedMessage, unkeyedMessage)
val messageSetWithKeyedMessage = new ByteBufferMessageSet(NoCompressionCodec, keyedMessage)
val messageSetWithKeyedMessages = new ByteBufferMessageSet(NoCompressionCodec, keyedMessage, anotherKeyedMessage)
val messageSetWithKeyedMessage = MemoryRecords.withRecords(CompressionType.NONE, keyedMessage)
val messageSetWithKeyedMessages = MemoryRecords.withRecords(CompressionType.NONE, keyedMessage, anotherKeyedMessage)
val logProps = new Properties()
logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Compact)
@ -502,8 +503,8 @@ class LogTest extends JUnitSuite {
*/
@Test
def testMessageSizeCheck() {
val first = new ByteBufferMessageSet(NoCompressionCodec, new Message ("You".getBytes), new Message("bethe".getBytes))
val second = new ByteBufferMessageSet(NoCompressionCodec, new Message("change (I need more bytes)".getBytes))
val first = MemoryRecords.withRecords(CompressionType.NONE, Record.create("You".getBytes), Record.create("bethe".getBytes))
val second = MemoryRecords.withRecords(CompressionType.NONE, Record.create("change (I need more bytes)".getBytes))
// append messages to log
val maxMessageSize = second.sizeInBytes - 1
@ -537,7 +538,7 @@ class LogTest extends JUnitSuite {
val config = LogConfig(logProps)
var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time)
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))
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
@ -585,7 +586,7 @@ class LogTest extends JUnitSuite {
val log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time)
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))
val timeIndexEntries = log.logSegments.foldLeft(0) { (entries, segment) => entries + segment.timeIndex.entries }
@ -608,7 +609,7 @@ class LogTest extends JUnitSuite {
val config = LogConfig(logProps)
var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time)
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 timeIndexFiles = log.logSegments.map(_.timeIndex.file)
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 time index should have been rebuilt", log.logSegments.head.timeIndex.entries > 0)
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)
assertEquals(log.logSegments.head.baseOffset, log.fetchOffsetsByTimestamp(time.milliseconds + i * 10).get.offset)
else
@ -647,7 +648,7 @@ class LogTest extends JUnitSuite {
val config = LogConfig(logProps)
var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time)
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)
log.close()
@ -676,7 +677,7 @@ class LogTest extends JUnitSuite {
val config = LogConfig(logProps)
var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time)
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 timeIndexFiles = log.logSegments.map(_.timeIndex.file)
log.close()
@ -699,7 +700,7 @@ class LogTest extends JUnitSuite {
log = new Log(logDir, config, recoveryPoint = 200L, time.scheduler, time)
assertEquals("Should have %d messages when log is reopened".format(numMessages), numMessages, log.logEndOffset)
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)
assertEquals(log.logSegments.head.baseOffset, log.fetchOffsetsByTimestamp(time.milliseconds + i * 10).get.offset)
else
@ -713,7 +714,7 @@ class LogTest extends JUnitSuite {
*/
@Test
def testTruncateTo() {
val set = TestUtils.singleMessageSet("test".getBytes)
val set = TestUtils.singletonRecords("test".getBytes)
val setSize = set.sizeInBytes
val msgPerSeg = 10
val segmentSize = msgPerSeg * setSize // each segment will be 10 messages
@ -770,7 +771,7 @@ class LogTest extends JUnitSuite {
*/
@Test
def testIndexResizingAtTruncation() {
val setSize = TestUtils.singleMessageSet(payload = "test".getBytes).sizeInBytes
val setSize = TestUtils.singletonRecords(value = "test".getBytes).sizeInBytes
val msgPerSeg = 10
val segmentSize = msgPerSeg * setSize // each segment will be 10 messages
val logProps = new Properties()
@ -781,12 +782,12 @@ class LogTest extends JUnitSuite {
assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
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)
time.sleep(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)
val expectedEntries = msgPerSeg - 1
@ -800,7 +801,7 @@ class LogTest extends JUnitSuite {
time.sleep(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)
}
@ -814,7 +815,7 @@ class LogTest extends JUnitSuite {
val bogusIndex2 = Log.indexFilename(logDir, 5)
val bogusTimeIndex2 = Log.timeIndexFilename(logDir, 5)
val set = TestUtils.singleMessageSet("test".getBytes)
val set = TestUtils.singletonRecords("test".getBytes)
val logProps = new Properties()
logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer)
logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer)
@ -842,7 +843,7 @@ class LogTest extends JUnitSuite {
*/
@Test
def testReopenThenTruncate() {
val set = TestUtils.singleMessageSet("test".getBytes)
val set = TestUtils.singletonRecords("test".getBytes)
val logProps = new Properties()
logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer)
logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer)
@ -875,7 +876,7 @@ class LogTest extends JUnitSuite {
*/
@Test
def testAsyncDelete() {
val set = TestUtils.singleMessageSet("test".getBytes)
val set = TestUtils.singletonRecords("test".getBytes)
val asyncDeleteMs = 1000
val logProps = new Properties()
logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer)
@ -921,7 +922,7 @@ class LogTest extends JUnitSuite {
*/
@Test
def testOpenDeletesObsoleteFiles() {
val set = TestUtils.singleMessageSet("test".getBytes)
val set = TestUtils.singletonRecords("test".getBytes)
val logProps = new Properties()
logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer)
logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer)
@ -957,10 +958,10 @@ class LogTest extends JUnitSuite {
recoveryPoint = 0L,
time.scheduler,
time)
log.append(new ByteBufferMessageSet(new Message(bytes = null)))
val messageSet = log.read(0, 4096, None).messageSet
assertEquals(0, messageSet.head.offset)
assertTrue("Message payload should be null.", messageSet.head.message.isNull)
log.append(MemoryRecords.withRecords(Record.create(null)))
val head = log.read(0, 4096, None).records.shallowIterator().next()
assertEquals(0, head.offset)
assertTrue("Message payload should be null.", head.record.hasNullValue)
}
@Test(expected = classOf[IllegalArgumentException])
@ -970,9 +971,9 @@ class LogTest extends JUnitSuite {
recoveryPoint = 0L,
time.scheduler,
time)
val messages = (0 until 2).map(id => new Message(id.toString.getBytes)).toArray
messages.foreach(message => log.append(new ByteBufferMessageSet(message)))
val invalidMessage = new ByteBufferMessageSet(new Message(1.toString.getBytes))
val messages = (0 until 2).map(id => Record.create(id.toString.getBytes)).toArray
messages.foreach(record => log.append(MemoryRecords.withRecords(record)))
val invalidMessage = MemoryRecords.withRecords(Record.create(1.toString.getBytes))
log.append(invalidMessage, assignOffsets = false)
}
@ -984,7 +985,7 @@ class LogTest extends JUnitSuite {
logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer)
logProps.put(LogConfig.MaxMessageBytesProp, 64*1024: java.lang.Integer)
val config = LogConfig(logProps)
val set = TestUtils.singleMessageSet("test".getBytes)
val set = TestUtils.singletonRecords("test".getBytes)
val recoveryPoint = 50L
for (_ <- 0 until 50) {
// create a log and write some messages to it
@ -997,7 +998,7 @@ class LogTest extends JUnitSuite {
val numMessages = 50 + TestUtils.random.nextInt(50)
for (_ <- 0 until numMessages)
log.append(set)
val messages = log.logSegments.flatMap(_.log.iterator.toList)
val messages = log.logSegments.flatMap(_.log.deepIterator.asScala.toList)
log.close()
// corrupt index and log by appending random bytes
@ -1007,7 +1008,8 @@ class LogTest extends JUnitSuite {
// attempt recovery
log = new Log(logDir, config, recoveryPoint, time.scheduler, time)
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)
}
}
@ -1020,7 +1022,7 @@ class LogTest extends JUnitSuite {
logProps.put(LogConfig.MaxMessageBytesProp, 64*1024: java.lang.Integer)
logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer)
val config = LogConfig(logProps)
val set = TestUtils.singleMessageSet("test".getBytes)
val set = TestUtils.singletonRecords("test".getBytes)
val parentLogDir = logDir.getParentFile
assertTrue("Data directory %s must exist", parentLogDir.isDirectory)
val cleanShutdownFile = new File(parentLogDir, Log.CleanShutdownFile)
@ -1121,7 +1123,7 @@ class LogTest extends JUnitSuite {
@Test
def testDeleteOldSegmentsMethod() {
val set = TestUtils.singleMessageSet("test".getBytes)
val set = TestUtils.singletonRecords("test".getBytes)
val logProps = new Properties()
logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer)
logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer)
@ -1154,7 +1156,7 @@ class LogTest extends JUnitSuite {
@Test
def shouldDeleteSizeBasedSegments() {
val set = TestUtils.singleMessageSet("test".getBytes)
val set = TestUtils.singletonRecords("test".getBytes)
val log = createLog(set.sizeInBytes, retentionBytes = set.sizeInBytes * 10)
// append some messages to create some segments
@ -1167,7 +1169,7 @@ class LogTest extends JUnitSuite {
@Test
def shouldNotDeleteSizeBasedSegmentsWhenUnderRetentionSize() {
val set = TestUtils.singleMessageSet("test".getBytes)
val set = TestUtils.singletonRecords("test".getBytes)
val log = createLog(set.sizeInBytes, retentionBytes = set.sizeInBytes * 15)
// append some messages to create some segments
@ -1180,7 +1182,7 @@ class LogTest extends JUnitSuite {
@Test
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)
// append some messages to create some segments
@ -1193,7 +1195,7 @@ class LogTest extends JUnitSuite {
@Test
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)
// append some messages to create some segments
@ -1206,7 +1208,7 @@ class LogTest extends JUnitSuite {
@Test
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,
retentionMs = 10000,
cleanupPolicy = "compact")
@ -1225,7 +1227,7 @@ class LogTest extends JUnitSuite {
@Test
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,
retentionMs = 10000,
cleanupPolicy = "compact,delete")

View File

@ -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)
}
}

View File

@ -18,13 +18,11 @@
package kafka.message
import java.nio.ByteBuffer
import java.nio.channels.{FileChannel, GatheringByteChannel}
import java.nio.file.StandardOpenOption
import java.nio.channels.GatheringByteChannel
import org.junit.Assert._
import kafka.utils.TestUtils._
import kafka.log.FileMessageSet
import kafka.utils.TestUtils
import org.apache.kafka.common.record.FileRecords
import org.scalatest.junit.JUnitSuite
import org.junit.Test
@ -94,7 +92,7 @@ trait BaseMessageSetTestCases extends JUnitSuite {
@Test
def testWriteToChannelThatConsumesPartially() {
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 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
for (_ <- 0 to 1) {
val file = tempFile()
val channel = FileChannel.open(file.toPath, StandardOpenOption.READ, StandardOpenOption.WRITE)
val fileRecords = FileRecords.open(file, true)
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)
val newSet = new FileMessageSet(file, channel)
checkEquals(set.iterator, newSet.iterator)
} finally channel.close()
checkEquals(set.asRecords.deepIterator, fileRecords.deepIterator())
} finally fileRecords.close()
}
}
}

View File

@ -17,13 +17,9 @@
package kafka.message
import java.io.DataOutputStream
import java.nio._
import kafka.common.LongRef
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.Test
@ -151,295 +147,6 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases {
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 */
def checkOffsets(messages: ByteBufferMessageSet, baseOffset: Long) {
assertTrue("Message set should not be empty", messages.nonEmpty)
@ -457,59 +164,4 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases {
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)
}
}

View File

@ -70,7 +70,7 @@ class MessageCompressionTest extends JUnitSuite {
testCompressSize(GZIPCompressionCodec, messages, 396)
if(isSnappyAvailable)
testCompressSize(SnappyCompressionCodec, messages, 502)
testCompressSize(SnappyCompressionCodec, messages, 1063)
if(isLZ4Available)
testCompressSize(LZ4CompressionCodec, messages, 387)

View File

@ -48,7 +48,7 @@ class MessageTest extends JUnitSuite {
val magicValues = Array(Message.MagicValue_V0, Message.MagicValue_V1)
for(k <- keys; v <- vals; codec <- codecs; t <- timestamps; mv <- magicValues) {
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 =
@ -96,7 +96,7 @@ class MessageTest extends JUnitSuite {
@Test
def testEquality() {
for(v <- messages) {
for (v <- messages) {
assertFalse("Should not equal null", v.message.equals(null))
assertFalse("Should not equal a random string", v.message.equals("asdf"))
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])
def testInvalidTimestampAndMagicValueCombination() {
new Message("hello".getBytes, 0L, Message.MagicValue_V0)

View File

@ -19,17 +19,17 @@ package kafka.server
import com.yammer.metrics.Metrics
import kafka.cluster.BrokerEndPoint
import kafka.message.{ByteBufferMessageSet, Message, NoCompressionCodec}
import kafka.server.AbstractFetcherThread.{FetchRequest, PartitionData}
import kafka.utils.TestUtils
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.utils.Utils
import org.apache.kafka.common.record.{MemoryRecords, Record}
import org.junit.Assert.{assertFalse, assertTrue}
import org.junit.{Before, Test}
import scala.collection.JavaConverters._
import scala.collection.{mutable, Map}
import scala.collection.{Map, mutable}
class AbstractFetcherThreadTest {
@ -91,10 +91,10 @@ class AbstractFetcherThreadTest {
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 toByteBufferMessageSet: ByteBufferMessageSet = byteBufferMessageSet
override def toRecords: MemoryRecords = records
override def highWatermark: Long = 0L
@ -119,7 +119,7 @@ class AbstractFetcherThreadTest {
override def handlePartitionsWithErrors(partitions: Iterable[TopicPartition]): Unit = {}
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 =
new DummyFetchRequest(partitionMap.map { case (k, v) => (k, v.offset) }.toMap)
@ -156,8 +156,8 @@ class AbstractFetcherThreadTest {
@volatile var fetchCount = 0
private val normalPartitionDataSet = List(
new TestPartitionData(new ByteBufferMessageSet(NoCompressionCodec, Seq(0L), new Message("hello".getBytes))),
new TestPartitionData(new ByteBufferMessageSet(NoCompressionCodec, Seq(1L), new Message("hello".getBytes)))
new TestPartitionData(MemoryRecords.withRecords(0L, Record.create("hello".getBytes()))),
new TestPartitionData(MemoryRecords.withRecords(1L, Record.create("hello".getBytes())))
)
override def processPartitionData(topicAndPartition: TopicPartition,
@ -170,10 +170,10 @@ class AbstractFetcherThreadTest {
.format(topicAndPartition, fetchOffset, logEndOffset))
// Now check message's crc
val messages = partitionData.toByteBufferMessageSet
for (messageAndOffset <- messages.shallowIterator) {
messageAndOffset.message.ensureValid()
logEndOffset = messageAndOffset.nextOffset
val records = partitionData.toRecords
for (entry <- records.shallowIterator.asScala) {
entry.record.ensureValid()
logEndOffset = entry.nextOffset
}
}
@ -181,12 +181,12 @@ class AbstractFetcherThreadTest {
fetchCount += 1
// Set the first fetch to get a corrupted message
if (fetchCount == 1) {
val corruptedMessage = new Message("hello".getBytes)
val badChecksum = (corruptedMessage.checksum + 1 % Int.MaxValue).toInt
val corruptedRecord = Record.create("hello".getBytes())
val badChecksum = (corruptedRecord.checksum + 1 % Int.MaxValue).toInt
// Garble checksum
Utils.writeUnsignedInt(corruptedMessage.buffer, Message.CrcOffset, badChecksum)
val byteBufferMessageSet = new ByteBufferMessageSet(NoCompressionCodec, corruptedMessage)
fetchRequest.offsets.mapValues(_ => new TestPartitionData(byteBufferMessageSet)).toSeq
Utils.writeUnsignedInt(corruptedRecord.buffer, Record.CRC_OFFSET, badChecksum)
val records = MemoryRecords.withRecords(corruptedRecord)
fetchRequest.offsets.mapValues(_ => new TestPartitionData(records)).toSeq
} else
// Then, the following fetches get the normal data
fetchRequest.offsets.mapValues(v => normalPartitionDataSet(v.toInt)).toSeq

View File

@ -24,12 +24,12 @@ import kafka.utils.TestUtils
import kafka.utils.TestUtils._
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.protocol.{ApiKeys, Errors, ProtoUtils}
import org.apache.kafka.common.record.{LogEntry, MemoryRecords}
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.record.LogEntry
import org.apache.kafka.common.requests.{FetchRequest, FetchResponse}
import org.apache.kafka.common.serialization.StringSerializer
import org.junit.Test
import org.junit.Assert._
import org.junit.Test
import scala.collection.JavaConverters._
import scala.util.Random
@ -120,13 +120,13 @@ class FetchRequestTest extends BaseRequestTest {
val fetchResponse3 = sendFetchRequest(leaderId, fetchRequest3)
assertEquals(shuffledTopicPartitions3, fetchResponse3.responseData.keySet.asScala.toSeq)
val responseSize3 = fetchResponse3.responseData.asScala.values.map { partitionData =>
logEntries(partitionData).map(_.size).sum
logEntries(partitionData).map(_.sizeInBytes).sum
}.sum
assertTrue(responseSize3 <= maxResponseBytes)
val partitionData3 = fetchResponse3.responseData.get(partitionWithLargeMessage1)
assertEquals(Errors.NONE.code, partitionData3.errorCode)
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 larger than $maxPartitionBytes", size3 > maxPartitionBytes)
assertTrue(maxPartitionBytes < partitionData3.records.sizeInBytes)
@ -138,13 +138,13 @@ class FetchRequestTest extends BaseRequestTest {
val fetchResponse4 = sendFetchRequest(leaderId, fetchRequest4)
assertEquals(shuffledTopicPartitions4, fetchResponse4.responseData.keySet.asScala.toSeq)
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)
val partitionData4 = fetchResponse4.responseData.get(partitionWithLargeMessage2)
assertEquals(Errors.NONE.code, partitionData4.errorCode)
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(maxResponseBytes < partitionData4.records.sizeInBytes)
}
@ -161,12 +161,11 @@ class FetchRequestTest extends BaseRequestTest {
assertEquals(Errors.NONE.code, partitionData.errorCode)
assertTrue(partitionData.highWatermark > 0)
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] = {
val memoryRecords = partitionData.records
memoryRecords.iterator.asScala.toIndexedSeq
partitionData.records.deepIterator.asScala.toIndexedSeq
}
private def checkFetchResponse(expectedPartitions: Seq[TopicPartition], fetchResponse: FetchResponse,
@ -181,25 +180,25 @@ class FetchRequestTest extends BaseRequestTest {
assertEquals(Errors.NONE.code, partitionData.errorCode)
assertTrue(partitionData.highWatermark > 0)
val memoryRecords = partitionData.records
responseBufferSize += memoryRecords.sizeInBytes
val records = partitionData.records
responseBufferSize += records.sizeInBytes
val messages = memoryRecords.iterator.asScala.toIndexedSeq
assertTrue(messages.size < numMessagesPerPartition)
val messagesSize = messages.map(_.size).sum
responseSize += messagesSize
if (messagesSize == 0 && !emptyResponseSeen) {
assertEquals(0, memoryRecords.sizeInBytes)
val entries = records.shallowIterator.asScala.toIndexedSeq
assertTrue(entries.size < numMessagesPerPartition)
val entriesSize = entries.map(_.sizeInBytes).sum
responseSize += entriesSize
if (entriesSize == 0 && !emptyResponseSeen) {
assertEquals(0, records.sizeInBytes)
emptyResponseSeen = true
}
else if (messagesSize != 0 && !emptyResponseSeen) {
assertTrue(messagesSize <= maxPartitionBytes)
assertEquals(maxPartitionBytes, memoryRecords.sizeInBytes)
else if (entriesSize != 0 && !emptyResponseSeen) {
assertTrue(entriesSize <= maxPartitionBytes)
assertEquals(maxPartitionBytes, records.sizeInBytes)
}
else if (messagesSize != 0 && emptyResponseSeen)
fail(s"Expected partition with size 0, but found $tp with size $messagesSize")
else if (memoryRecords.sizeInBytes != 0 && emptyResponseSeen)
fail(s"Expected partition buffer with size 0, but found $tp with size ${memoryRecords.sizeInBytes}")
else if (entriesSize != 0 && emptyResponseSeen)
fail(s"Expected partition with size 0, but found $tp with size $entriesSize")
else if (records.sizeInBytes != 0 && emptyResponseSeen)
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] = {
val topics = (0 until numPartitions).map(t => s"topic${t}")
val topics = (0 until numPartitions).map(t => s"topic$t")
val topicConfig = new Properties
topicConfig.setProperty(LogConfig.MinInSyncReplicasProp, 2.toString)
topics.flatMap { topic =>
@ -223,7 +222,7 @@ class FetchRequestTest extends BaseRequestTest {
tp <- topicPartitions.toSeq
messageIndex <- 0 until numMessagesPerPartition
} yield {
val suffix = s"${tp}-${messageIndex}"
val suffix = s"$tp-$messageIndex"
new ProducerRecord(tp.topic, tp.partition, s"key $suffix", s"value $suffix")
}
records.map(producer.send).foreach(_.get)

View File

@ -17,21 +17,20 @@
package kafka.server
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 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.easymock.EasyMock
import org.junit.Assert._
import org.junit.{After, Before, Test}
import scala.collection.mutable.{HashMap, Map}
class IsrExpirationTest {
@ -76,7 +75,7 @@ class IsrExpirationTest {
// let the follower catch up to the Leader logEndOffset (15)
(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,
-1,
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
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.
// 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)
(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)
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
(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)
assertEquals("No replica should be out of sync", Set.empty[Int], partition0OSR.map(_.brokerId))

View File

@ -26,12 +26,12 @@ import kafka.api.{FetchRequestBuilder, OffsetRequest, PartitionOffsetRequestInfo
import kafka.common.TopicAndPartition
import kafka.consumer.SimpleConsumer
import kafka.log.{Log, LogSegment}
import kafka.message.{ByteBufferMessageSet, Message, NoCompressionCodec}
import kafka.utils.TestUtils._
import kafka.utils._
import kafka.zk.ZooKeeperTestHarness
import org.apache.kafka.common.TopicPartition
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.easymock.{EasyMock, IAnswer}
import org.junit.Assert._
@ -89,9 +89,9 @@ class LogOffsetTest extends ZooKeeperTestHarness {
"Log for partition [topic,0] should be created")
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)
log.append(new ByteBufferMessageSet(NoCompressionCodec, message))
log.append(MemoryRecords.withRecords(record))
log.flush()
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 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)
log.append(new ByteBufferMessageSet(NoCompressionCodec, message))
log.append(MemoryRecords.withRecords(record))
log.flush()
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 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)
log.append(new ByteBufferMessageSet(NoCompressionCodec, message))
log.append(MemoryRecords.withRecords(record))
log.flush()
val offsets = server.apis.fetchOffsets(logManager, new TopicPartition(topic, part), OffsetRequest.EarliestTime, 10)

View File

@ -54,11 +54,11 @@ class ProduceRequestTest extends BaseRequestTest {
}
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,
new Record(System.currentTimeMillis(), "key1".getBytes, "value1".getBytes),
new Record(System.currentTimeMillis(), "key2".getBytes, "value2".getBytes)), 1)
Record.create(System.currentTimeMillis(), "key1".getBytes, "value1".getBytes),
Record.create(System.currentTimeMillis(), "key2".getBytes, "value2".getBytes)), 1)
}
/* returns a pair of partition id and leader id */
@ -74,7 +74,7 @@ class ProduceRequestTest extends BaseRequestTest {
val (partition, leader) = createTopicAndFindPartitionWithLeader("topic")
val timestamp = 1000000
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
recordBuffer.array.update(40, 0)
val topicPartition = new TopicPartition("topic", partition)

View File

@ -16,28 +16,29 @@
*/
package kafka.server
import java.util.Properties
import java.util.concurrent.atomic.AtomicBoolean
import kafka.cluster.Replica
import kafka.common.TopicAndPartition
import kafka.log.Log
import kafka.message.{ByteBufferMessageSet, Message}
import kafka.utils._
import org.apache.kafka.common.TopicPartition
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.easymock.EasyMock
import EasyMock._
import org.junit.Assert._
import org.junit.{After, Test}
import scala.collection.JavaConverters._
class ReplicaManagerQuotasTest {
val configs = TestUtils.createBrokerConfigs(2, TestUtils.MockZkConnect).map(KafkaConfig.fromProps(_, new Properties()))
val time = new MockTime
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 topicAndPartition2 = TopicAndPartition("test-topic", 2)
val fetchInfo = Seq(new TopicPartition(topicAndPartition1.topic, topicAndPartition1.partition) -> new PartitionData(0, 100),
@ -63,10 +64,10 @@ class ReplicaManagerQuotasTest {
readPartitionInfo = fetchInfo,
quota = quota)
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,
fetch.find(_._1 == topicAndPartition2).get._2.info.messageSet.size)
fetch.find(_._1 == topicAndPartition2).get._2.info.records.shallowIterator.asScala.size)
}
@Test
@ -88,9 +89,9 @@ class ReplicaManagerQuotasTest {
readPartitionInfo = fetchInfo,
quota = quota)
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,
fetch.find(_._1 == topicAndPartition2).get._2.info.messageSet.size)
fetch.find(_._1 == topicAndPartition2).get._2.info.records.shallowIterator.asScala.size)
}
@Test
@ -112,9 +113,9 @@ class ReplicaManagerQuotasTest {
readPartitionInfo = fetchInfo,
quota = quota)
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,
fetch.find(_._1 == topicAndPartition2).get._2.info.messageSet.size)
fetch.find(_._1 == topicAndPartition2).get._2.info.records.shallowIterator.asScala.size)
}
@Test
@ -136,13 +137,13 @@ class ReplicaManagerQuotasTest {
readPartitionInfo = fetchInfo,
quota = quota)
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,
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 scheduler = createNiceMock(classOf[KafkaScheduler])
@ -153,16 +154,16 @@ class ReplicaManagerQuotasTest {
//if we ask for len 1 return a message
expect(log.read(anyObject(), geq(1), anyObject(), anyObject())).andReturn(
new FetchDataInfo(
FetchDataInfo(
new LogOffsetMetadata(0L, 0L, 0),
new ByteBufferMessageSet(message)
MemoryRecords.withRecords(record)
)).anyTimes()
//if we ask for len = 0, return 0 messages
expect(log.read(anyObject(), EasyMock.eq(0), anyObject(), anyObject())).andReturn(
new FetchDataInfo(
FetchDataInfo(
new LogOffsetMetadata(0L, 0L, 0),
new ByteBufferMessageSet()
MemoryRecords.EMPTY
)).anyTimes()
replay(log)

View File

@ -17,18 +17,16 @@
package kafka.server
import java.io.File
import java.util.concurrent.atomic.AtomicBoolean
import kafka.api.FetchResponsePartitionData
import kafka.cluster.Broker
import kafka.common.TopicAndPartition
import kafka.message.{ByteBufferMessageSet, Message, MessageSet}
import kafka.utils.{MockScheduler, MockTime, TestUtils, ZkUtils}
import org.I0Itec.zkclient.ZkClient
import org.apache.kafka.common.metrics.Metrics
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.ProduceResponse.PartitionResponse
import org.apache.kafka.common.requests.FetchRequest.PartitionData
@ -105,11 +103,11 @@ class ReplicaManagerTest {
def callback(responseStatus: Map[TopicPartition, PartitionResponse]) = {
assert(responseStatus.values.head.errorCode == Errors.INVALID_REQUIRED_ACKS.code)
}
rm.appendMessages(
rm.appendRecords(
timeout = 0,
requiredAcks = 3,
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)
} finally {
rm.shutdown(checkpointHW = false)
@ -135,7 +133,7 @@ class ReplicaManagerTest {
}
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)
fetchCallbackFired = true
}
@ -158,11 +156,11 @@ class ReplicaManagerTest {
rm.getLeaderReplicaIfLocal(topic, 0)
// Append a message.
rm.appendMessages(
rm.appendRecords(
timeout = 1000,
requiredAcks = -1,
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)
// Fetch some messages
@ -220,19 +218,19 @@ class ReplicaManagerTest {
// Append a couple of messages.
for(i <- 1 to 2)
rm.appendMessages(
rm.appendRecords(
timeout = 1000,
requiredAcks = -1,
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)
var fetchCallbackFired = false
var fetchError = 0
var fetchedMessages: MessageSet = null
def fetchCallback(responseStatus: Seq[(TopicAndPartition, FetchResponsePartitionData)]) = {
var fetchedRecords: Records = null
def fetchCallback(responseStatus: Seq[(TopicAndPartition, FetchPartitionData)]) = {
fetchError = responseStatus.map(_._2).head.error
fetchedMessages = responseStatus.map(_._2).head.messages
fetchedRecords = responseStatus.map(_._2).head.records
fetchCallbackFired = true
}
@ -249,7 +247,7 @@ class ReplicaManagerTest {
assertTrue(fetchCallbackFired)
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
// Fetch a message above the high watermark as a consumer
@ -264,7 +262,7 @@ class ReplicaManagerTest {
assertTrue(fetchCallbackFired)
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 {
rm.shutdown(checkpointHW = false)
}

View File

@ -21,7 +21,6 @@ import kafka.utils._
import kafka.cluster.Replica
import kafka.common.TopicAndPartition
import kafka.log.Log
import kafka.message.{ByteBufferMessageSet, Message, MessageSet}
import kafka.server.QuotaFactory.UnboundedQuota
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.requests.FetchRequest.PartitionData
@ -30,8 +29,10 @@ import java.util.Properties
import java.util.concurrent.atomic.AtomicBoolean
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.record.{MemoryRecords, Record}
import org.easymock.EasyMock
import org.junit.Assert._
import scala.collection.JavaConverters._
class SimpleFetchTest {
@ -53,8 +54,8 @@ class SimpleFetchTest {
val partitionHW = 5
val fetchSize = 100
val messagesToHW = new Message("messageToHW".getBytes())
val messagesToLEO = new Message("messageToLEO".getBytes())
val messagesToHW = Record.create("messageToHW".getBytes())
val messagesToLEO = Record.create("messageToLEO".getBytes())
val topic = "test-topic"
val partitionId = 0
@ -79,14 +80,14 @@ class SimpleFetchTest {
EasyMock.expect(log.logEndOffset).andReturn(leaderLEO).anyTimes()
EasyMock.expect(log.logEndOffsetMetadata).andReturn(new LogOffsetMetadata(leaderLEO)).anyTimes()
EasyMock.expect(log.read(0, fetchSize, Some(partitionHW), true)).andReturn(
new FetchDataInfo(
FetchDataInfo(
new LogOffsetMetadata(0L, 0L, 0),
new ByteBufferMessageSet(messagesToHW)
MemoryRecords.withRecords(messagesToHW)
)).anyTimes()
EasyMock.expect(log.read(0, fetchSize, None, true)).andReturn(
new FetchDataInfo(
FetchDataInfo(
new LogOffsetMetadata(0L, 0L, 0),
new ByteBufferMessageSet(messagesToLEO)
MemoryRecords.withRecords(messagesToLEO)
)).anyTimes()
EasyMock.replay(log)
@ -110,7 +111,7 @@ class SimpleFetchTest {
// create the follower replica with defined log end offset
val followerReplica= new Replica(configs(1).brokerId, partition, time)
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
val allReplicas = List(leaderReplica, followerReplica)
@ -153,7 +154,7 @@ class SimpleFetchTest {
fetchMaxBytes = Int.MaxValue,
hardMaxBytesLimit = false,
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,
replicaManager.readFromLocalLog(
replicaId = Request.OrdinaryConsumerId,
@ -162,7 +163,7 @@ class SimpleFetchTest {
fetchMaxBytes = Int.MaxValue,
hardMaxBytesLimit = false,
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", initialAllTopicsCount+2, BrokerTopicStats.getBrokerAllTopicsStats().totalFetchRequestRate.count())

View File

@ -21,7 +21,7 @@ import java.io._
import java.nio._
import java.nio.channels._
import java.util.concurrent.{Callable, Executors, TimeUnit}
import java.util.{Properties, Random}
import java.util.Properties
import java.security.cert.X509Certificate
import javax.net.ssl.X509TrustManager
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.CommonClientConfigs
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.utils.Time
import org.apache.kafka.test.{TestUtils => JTestUtils}
@ -269,16 +270,16 @@ object TestUtils extends Logging {
}
/**
* Wrap the message in a message set
*
* @param payload The bytes of the message
* Wrap a single record log buffer.
*/
def singleMessageSet(payload: Array[Byte],
codec: CompressionCodec = NoCompressionCodec,
def singletonRecords(value: Array[Byte],
key: Array[Byte] = null,
timestamp: Long = Message.NoTimestamp,
magicValue: Byte = Message.CurrentMagicValue) =
new ByteBufferMessageSet(compressionCodec = codec, messages = new Message(payload, key, timestamp, magicValue))
codec: CompressionType = CompressionType.NONE,
timestamp: Long = Record.NO_TIMESTAMP,
magicValue: Byte = Record.CURRENT_MAGIC_VALUE) = {
val record = Record.create(magicValue, timestamp, key, value)
MemoryRecords.withRecords(codec, record)
}
/**
* Generate an array of random bytes