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="net.jpountz" />
<allow pkg="org.apache.kafka.common.record" /> <allow pkg="org.apache.kafka.common.record" />
<allow pkg="org.apache.kafka.common.network" /> <allow pkg="org.apache.kafka.common.network" />
<allow pkg="org.apache.kafka.common.errors" />
</subpackage> </subpackage>
<subpackage name="requests"> <subpackage name="requests">

View File

@ -61,6 +61,7 @@ import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedHashMap; import java.util.LinkedHashMap;
import java.util.List; import java.util.List;
import java.util.Locale; import java.util.Locale;
@ -686,11 +687,13 @@ public class Fetcher<K, V> {
} }
List<ConsumerRecord<K, V>> parsed = new ArrayList<>(); List<ConsumerRecord<K, V>> parsed = new ArrayList<>();
for (LogEntry logEntry : partition.records) { Iterator<LogEntry> deepIterator = partition.records.deepIterator();
while (deepIterator.hasNext()) {
LogEntry logEntry = deepIterator.next();
// Skip the messages earlier than current position. // Skip the messages earlier than current position.
if (logEntry.offset() >= position) { if (logEntry.offset() >= position) {
parsed.add(parseRecord(tp, logEntry)); parsed.add(parseRecord(tp, logEntry));
bytes += logEntry.size(); bytes += logEntry.sizeInBytes();
} }
} }

View File

@ -198,7 +198,7 @@ public final class BufferPool {
* memory as free. * memory as free.
* *
* @param buffer The buffer to return * @param buffer The buffer to return
* @param size The size of the buffer to mark as deallocated, note that this maybe smaller than buffer.capacity * @param size The size of the buffer to mark as deallocated, note that this may be smaller than buffer.capacity
* since the buffer may re-allocate itself during in-place compression * since the buffer may re-allocate itself during in-place compression
*/ */
public void deallocate(ByteBuffer buffer, int size) { public void deallocate(ByteBuffer buffer, int size) {

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.metrics.stats.Rate;
import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.MemoryRecordsBuilder;
import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.Record;
import org.apache.kafka.common.record.Records; import org.apache.kafka.common.record.Records;
import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.common.utils.CopyOnWriteMap; import org.apache.kafka.common.utils.CopyOnWriteMap;
import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.utils.Utils;
@ -49,7 +51,7 @@ import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
/** /**
* This class acts as a queue that accumulates records into {@link org.apache.kafka.common.record.MemoryRecords} * This class acts as a queue that accumulates records into {@link MemoryRecords}
* instances to be sent to the server. * instances to be sent to the server.
* <p> * <p>
* The accumulator uses a bounded amount of memory and append calls will block when that memory is exhausted, unless * The accumulator uses a bounded amount of memory and append calls will block when that memory is exhausted, unless
@ -77,7 +79,7 @@ public final class RecordAccumulator {
/** /**
* Create a new record accumulator * Create a new record accumulator
* *
* @param batchSize The size to use when allocating {@link org.apache.kafka.common.record.MemoryRecords} instances * @param batchSize The size to use when allocating {@link MemoryRecords} instances
* @param totalSize The maximum memory the record accumulator can use. * @param totalSize The maximum memory the record accumulator can use.
* @param compression The compression codec for the records * @param compression The compression codec for the records
* @param lingerMs An artificial delay time to add before declaring a records instance that isn't full ready for * @param lingerMs An artificial delay time to add before declaring a records instance that isn't full ready for
@ -190,13 +192,13 @@ public final class RecordAccumulator {
free.deallocate(buffer); free.deallocate(buffer);
return appendResult; return appendResult;
} }
MemoryRecords records = MemoryRecords.emptyRecords(buffer, compression, this.batchSize); MemoryRecordsBuilder recordsBuilder = MemoryRecords.builder(buffer, compression, TimestampType.CREATE_TIME, this.batchSize);
RecordBatch batch = new RecordBatch(tp, records, time.milliseconds()); RecordBatch batch = new RecordBatch(tp, recordsBuilder, time.milliseconds());
FutureRecordMetadata future = Utils.notNull(batch.tryAppend(timestamp, key, value, callback, time.milliseconds())); FutureRecordMetadata future = Utils.notNull(batch.tryAppend(timestamp, key, value, callback, time.milliseconds()));
dq.addLast(batch); dq.addLast(batch);
incomplete.add(batch); incomplete.add(batch);
return new RecordAppendResult(future, dq.size() > 1 || batch.records.isFull(), true); return new RecordAppendResult(future, dq.size() > 1 || batch.isFull(), true);
} }
} finally { } finally {
appendsInProgress.decrementAndGet(); appendsInProgress.decrementAndGet();
@ -212,9 +214,9 @@ public final class RecordAccumulator {
if (last != null) { if (last != null) {
FutureRecordMetadata future = last.tryAppend(timestamp, key, value, callback, time.milliseconds()); FutureRecordMetadata future = last.tryAppend(timestamp, key, value, callback, time.milliseconds());
if (future == null) if (future == null)
last.records.close(); last.close();
else else
return new RecordAppendResult(future, deque.size() > 1 || last.records.isFull(), false); return new RecordAppendResult(future, deque.size() > 1 || last.isFull(), false);
} }
return null; return null;
} }
@ -240,7 +242,7 @@ public final class RecordAccumulator {
Iterator<RecordBatch> batchIterator = dq.iterator(); Iterator<RecordBatch> batchIterator = dq.iterator();
while (batchIterator.hasNext()) { while (batchIterator.hasNext()) {
RecordBatch batch = batchIterator.next(); RecordBatch batch = batchIterator.next();
boolean isFull = batch != lastBatch || batch.records.isFull(); boolean isFull = batch != lastBatch || batch.isFull();
// check if the batch is expired // check if the batch is expired
if (batch.maybeExpire(requestTimeout, retryBackoffMs, now, this.lingerMs, isFull)) { if (batch.maybeExpire(requestTimeout, retryBackoffMs, now, this.lingerMs, isFull)) {
expiredBatches.add(batch); expiredBatches.add(batch);
@ -319,7 +321,7 @@ public final class RecordAccumulator {
long waitedTimeMs = nowMs - batch.lastAttemptMs; long waitedTimeMs = nowMs - batch.lastAttemptMs;
long timeToWaitMs = backingOff ? retryBackoffMs : lingerMs; long timeToWaitMs = backingOff ? retryBackoffMs : lingerMs;
long timeLeftMs = Math.max(timeToWaitMs - waitedTimeMs, 0); long timeLeftMs = Math.max(timeToWaitMs - waitedTimeMs, 0);
boolean full = deque.size() > 1 || batch.records.isFull(); boolean full = deque.size() > 1 || batch.isFull();
boolean expired = waitedTimeMs >= timeToWaitMs; boolean expired = waitedTimeMs >= timeToWaitMs;
boolean sendable = full || expired || exhausted || closed || flushInProgress(); boolean sendable = full || expired || exhausted || closed || flushInProgress();
if (sendable && !backingOff) { if (sendable && !backingOff) {
@ -389,15 +391,15 @@ public final class RecordAccumulator {
boolean backoff = first.attempts > 0 && first.lastAttemptMs + retryBackoffMs > now; boolean backoff = first.attempts > 0 && first.lastAttemptMs + retryBackoffMs > now;
// Only drain the batch if it is not during backoff period. // Only drain the batch if it is not during backoff period.
if (!backoff) { if (!backoff) {
if (size + first.records.sizeInBytes() > maxSize && !ready.isEmpty()) { if (size + first.sizeInBytes() > maxSize && !ready.isEmpty()) {
// there is a rare case that a single batch size is larger than the request size due // there is a rare case that a single batch size is larger than the request size due
// to compression; in this case we will still eventually send this batch in a single // to compression; in this case we will still eventually send this batch in a single
// request // request
break; break;
} else { } else {
RecordBatch batch = deque.pollFirst(); RecordBatch batch = deque.pollFirst();
batch.records.close(); batch.close();
size += batch.records.sizeInBytes(); size += batch.sizeInBytes();
ready.add(batch); ready.add(batch);
batch.drainedMs = now; batch.drainedMs = now;
} }
@ -437,7 +439,7 @@ public final class RecordAccumulator {
*/ */
public void deallocate(RecordBatch batch) { public void deallocate(RecordBatch batch) {
incomplete.remove(batch); incomplete.remove(batch);
free.deallocate(batch.records.buffer(), batch.records.initialCapacity()); free.deallocate(batch.buffer(), batch.initialCapacity());
} }
/** /**
@ -507,7 +509,7 @@ public final class RecordAccumulator {
Deque<RecordBatch> dq = getDeque(batch.topicPartition); Deque<RecordBatch> dq = getDeque(batch.topicPartition);
// Close the batch before aborting // Close the batch before aborting
synchronized (dq) { synchronized (dq) {
batch.records.close(); batch.close();
dq.remove(batch); dq.remove(batch);
} }
batch.done(-1L, Record.NO_TIMESTAMP, new IllegalStateException("Producer is closed forcefully.")); batch.done(-1L, Record.NO_TIMESTAMP, new IllegalStateException("Producer is closed forcefully."));

View File

@ -12,18 +12,20 @@
*/ */
package org.apache.kafka.clients.producer.internals; package org.apache.kafka.clients.producer.internals;
import java.util.ArrayList;
import java.util.List;
import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.Callback;
import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.MemoryRecordsBuilder;
import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.Record;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
/** /**
* A batch of records that is or will be sent. * A batch of records that is or will be sent.
* *
@ -39,21 +41,21 @@ public final class RecordBatch {
public final long createdMs; public final long createdMs;
public long drainedMs; public long drainedMs;
public long lastAttemptMs; public long lastAttemptMs;
public final MemoryRecords records;
public final TopicPartition topicPartition; public final TopicPartition topicPartition;
public final ProduceRequestResult produceFuture; public final ProduceRequestResult produceFuture;
public long lastAppendTime; public long lastAppendTime;
private final List<Thunk> thunks; private final List<Thunk> thunks;
private long offsetCounter = 0L; private long offsetCounter = 0L;
private boolean retry; private boolean retry;
private final MemoryRecordsBuilder recordsBuilder;
public RecordBatch(TopicPartition tp, MemoryRecords records, long now) { public RecordBatch(TopicPartition tp, MemoryRecordsBuilder recordsBuilder, long now) {
this.createdMs = now; this.createdMs = now;
this.lastAttemptMs = now; this.lastAttemptMs = now;
this.records = records; this.recordsBuilder = recordsBuilder;
this.topicPartition = tp; this.topicPartition = tp;
this.produceFuture = new ProduceRequestResult(); this.produceFuture = new ProduceRequestResult();
this.thunks = new ArrayList<Thunk>(); this.thunks = new ArrayList<>();
this.lastAppendTime = createdMs; this.lastAppendTime = createdMs;
this.retry = false; this.retry = false;
} }
@ -64,10 +66,10 @@ public final class RecordBatch {
* @return The RecordSend corresponding to this record or null if there isn't sufficient room. * @return The RecordSend corresponding to this record or null if there isn't sufficient room.
*/ */
public FutureRecordMetadata tryAppend(long timestamp, byte[] key, byte[] value, Callback callback, long now) { public FutureRecordMetadata tryAppend(long timestamp, byte[] key, byte[] value, Callback callback, long now) {
if (!this.records.hasRoomFor(key, value)) { if (!recordsBuilder.hasRoomFor(key, value)) {
return null; return null;
} else { } else {
long checksum = this.records.append(offsetCounter++, timestamp, key, value); long checksum = this.recordsBuilder.append(offsetCounter++, timestamp, key, value);
this.maxRecordSize = Math.max(this.maxRecordSize, Record.recordSize(key, value)); this.maxRecordSize = Math.max(this.maxRecordSize, Record.recordSize(key, value));
this.lastAppendTime = now; this.lastAppendTime = now;
FutureRecordMetadata future = new FutureRecordMetadata(this.produceFuture, this.recordCount, FutureRecordMetadata future = new FutureRecordMetadata(this.produceFuture, this.recordCount,
@ -94,9 +96,8 @@ public final class RecordBatch {
baseOffset, baseOffset,
exception); exception);
// execute callbacks // execute callbacks
for (int i = 0; i < this.thunks.size(); i++) { for (Thunk thunk : thunks) {
try { try {
Thunk thunk = this.thunks.get(i);
if (exception == null) { if (exception == null) {
// If the timestamp returned by server is NoTimestamp, that means CreateTime is used. Otherwise LogAppendTime is used. // If the timestamp returned by server is NoTimestamp, that means CreateTime is used. Otherwise LogAppendTime is used.
RecordMetadata metadata = new RecordMetadata(this.topicPartition, baseOffset, thunk.future.relativeOffset(), RecordMetadata metadata = new RecordMetadata(this.topicPartition, baseOffset, thunk.future.relativeOffset(),
@ -156,7 +157,7 @@ public final class RecordBatch {
} }
if (expire) { if (expire) {
this.records.close(); close();
this.done(-1L, Record.NO_TIMESTAMP, this.done(-1L, Record.NO_TIMESTAMP,
new TimeoutException("Expiring " + recordCount + " record(s) for " + topicPartition + " due to " + errorMessage)); new TimeoutException("Expiring " + recordCount + " record(s) for " + topicPartition + " due to " + errorMessage));
} }
@ -177,4 +178,37 @@ public final class RecordBatch {
public void setRetry() { public void setRetry() {
this.retry = true; this.retry = true;
} }
public MemoryRecords records() {
return recordsBuilder.build();
}
public int sizeInBytes() {
return recordsBuilder.sizeInBytes();
}
public double compressionRate() {
return recordsBuilder.compressionRate();
}
public boolean isFull() {
return recordsBuilder.isFull();
}
public void close() {
recordsBuilder.close();
}
public ByteBuffer buffer() {
return recordsBuilder.buffer();
}
public int initialCapacity() {
return recordsBuilder.initialCapacity();
}
public boolean isWritable() {
return !recordsBuilder.isClosed();
}
} }

View File

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

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

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; package org.apache.kafka.common.record;
import java.io.DataOutputStream;
import java.io.OutputStream; import java.io.OutputStream;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
/** /**
* A byte buffer backed output outputStream * A byte buffer backed output outputStream
*/ */
public class ByteBufferOutputStream extends OutputStream { public class ByteBufferOutputStream extends DataOutputStream {
private static final float REALLOCATION_FACTOR = 1.1f; private static final float REALLOCATION_FACTOR = 1.1f;
private ByteBuffer buffer;
public ByteBufferOutputStream(ByteBuffer buffer) { public ByteBufferOutputStream(ByteBuffer buffer) {
this.buffer = buffer; super(new UnderlyingOutputStream(buffer));
}
public void write(int b) {
if (buffer.remaining() < 1)
expandBuffer(buffer.capacity() + 1);
buffer.put((byte) b);
}
public void write(byte[] bytes, int off, int len) {
if (buffer.remaining() < len)
expandBuffer(buffer.capacity() + len);
buffer.put(bytes, off, len);
} }
public ByteBuffer buffer() { public ByteBuffer buffer() {
return buffer; return ((UnderlyingOutputStream) out).buffer;
} }
private void expandBuffer(int size) { public static class UnderlyingOutputStream extends OutputStream {
int expandSize = Math.max((int) (buffer.capacity() * REALLOCATION_FACTOR), size); private ByteBuffer buffer;
ByteBuffer temp = ByteBuffer.allocate(expandSize);
temp.put(buffer.array(), buffer.arrayOffset(), buffer.position()); public UnderlyingOutputStream(ByteBuffer buffer) {
buffer = temp; this.buffer = buffer;
}
public void write(int b) {
if (buffer.remaining() < 1)
expandBuffer(buffer.capacity() + 1);
buffer.put((byte) b);
}
public void write(byte[] bytes, int off, int len) {
if (buffer.remaining() < len)
expandBuffer(buffer.capacity() + len);
buffer.put(bytes, off, len);
}
public ByteBuffer buffer() {
return buffer;
}
private void expandBuffer(int size) {
int expandSize = Math.max((int) (buffer.capacity() * REALLOCATION_FACTOR), size);
ByteBuffer temp = ByteBuffer.allocate(expandSize);
temp.put(buffer.array(), buffer.arrayOffset(), buffer.position());
buffer = temp;
}
} }
} }

View File

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

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.KafkaException;
import org.apache.kafka.common.network.TransportLayer; import org.apache.kafka.common.network.TransportLayer;
import org.apache.kafka.common.record.FileLogInputStream.FileChannelLogEntry;
import org.apache.kafka.common.utils.Utils;
import java.io.Closeable;
import java.io.File; import java.io.File;
import java.io.FileInputStream;
import java.io.IOException; import java.io.IOException;
import java.io.RandomAccessFile;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.nio.channels.FileChannel; import java.nio.channels.FileChannel;
import java.nio.channels.GatheringByteChannel; import java.nio.channels.GatheringByteChannel;
import java.util.Iterator;
import java.util.concurrent.atomic.AtomicInteger;
/** /**
* File-backed record set. * A {@link Records} implementation backed by a file. An optional start and end position can be applied to this
* instance to enable slicing a range of the log records.
*/ */
public class FileRecords implements Records { public class FileRecords extends AbstractRecords implements Closeable {
private final File file; private final boolean isSlice;
private final FileChannel channel; private final FileChannel channel;
private final long start; private final int start;
private final long end; private final int end;
private final long size; private volatile File file;
private final AtomicInteger size;
public FileRecords(File file, public FileRecords(File file,
FileChannel channel, FileChannel channel,
@ -44,83 +53,435 @@ public class FileRecords implements Records {
this.channel = channel; this.channel = channel;
this.start = start; this.start = start;
this.end = end; this.end = end;
this.isSlice = isSlice;
this.size = new AtomicInteger();
if (isSlice) // set the initial size of the buffer
this.size = end - start; resize();
else }
this.size = Math.min(channel.size(), end) - start;
public void resize() throws IOException {
if (isSlice) {
size.set(end - start);
} else {
int limit = Math.min((int) channel.size(), end);
size.set(limit - start);
// if this is not a slice, update the file pointer to the end of the file
// set the file position to the last byte in the file
channel.position(limit);
}
} }
@Override @Override
public int sizeInBytes() { public int sizeInBytes() {
return (int) size; return size.get();
}
/**
* Get the underlying file.
* @return The file
*/
public File file() {
return file;
}
/**
* Get the underlying file channel.
* @return The file channel
*/
public FileChannel channel() {
return channel;
}
/**
* Read log entries into a given buffer.
* @param buffer The buffer to write the entries to
* @param position Position in the buffer to read from
* @return The same buffer
* @throws IOException
*/
public ByteBuffer readInto(ByteBuffer buffer, int position) throws IOException {
channel.read(buffer, position + this.start);
buffer.flip();
return buffer;
}
/**
* Return a slice of records from this instance, which is a view into this set starting from the given position
* and with the given size limit.
*
* If the size is beyond the end of the file, the end will be based on the size of the file at the time of the read.
*
* If this message set is already sliced, the position will be taken relative to that slicing.
*
* @param position The start position to begin the read from
* @param size The number of bytes after the start position to include
* @return A sliced wrapper on this message set limited based on the given position and size
*/
public FileRecords read(int position, int size) throws IOException {
if (position < 0)
throw new IllegalArgumentException("Invalid position: " + position);
if (size < 0)
throw new IllegalArgumentException("Invalid size: " + size);
final int end;
if (this.start + position + size < 0)
end = sizeInBytes();
else
end = Math.min(this.start + position + size, sizeInBytes());
return new FileRecords(file, channel, this.start + position, end, true);
}
/**
* Append log entries to the buffer
* @param records The records to append
* @return the number of bytes written to the underlying file
*/
public int append(MemoryRecords records) throws IOException {
int written = records.writeFullyTo(channel);
size.getAndAdd(written);
return written;
}
/**
* Commit all written data to the physical disk
*/
public void flush() throws IOException {
channel.force(true);
}
/**
* Close this record set
*/
public void close() throws IOException {
flush();
trim();
channel.close();
}
/**
* Delete this message set from the filesystem
* @return True iff this message set was deleted.
*/
public boolean delete() {
Utils.closeQuietly(channel, "FileChannel");
return file.delete();
}
/**
* Trim file when close or roll to next file
*/
public void trim() throws IOException {
truncateTo(sizeInBytes());
}
/**
* Update the file reference (to be used with caution since this does not reopen the file channel)
* @param file The new file to use
*/
public void setFile(File file) {
this.file = file;
}
/**
* Rename the file that backs this message set
* @throws IOException if rename fails.
*/
public void renameTo(File f) throws IOException {
try {
Utils.atomicMoveWithFallback(file.toPath(), f.toPath());
} finally {
this.file = f;
}
}
/**
* Truncate this file message set to the given size in bytes. Note that this API does no checking that the
* given size falls on a valid message boundary.
* In some versions of the JDK truncating to the same size as the file message set will cause an
* update of the files mtime, so truncate is only performed if the targetSize is smaller than the
* size of the underlying FileChannel.
* It is expected that no other threads will do writes to the log when this function is called.
* @param targetSize The size to truncate to. Must be between 0 and sizeInBytes.
* @return The number of bytes truncated off
*/
public int truncateTo(int targetSize) throws IOException {
int originalSize = sizeInBytes();
if (targetSize > originalSize || targetSize < 0)
throw new KafkaException("Attempt to truncate log segment to " + targetSize + " bytes failed, " +
" size of this log segment is " + originalSize + " bytes.");
if (targetSize < (int) channel.size()) {
channel.truncate(targetSize);
channel.position(targetSize);
size.set(targetSize);
}
return originalSize - targetSize;
} }
@Override @Override
public long writeTo(GatheringByteChannel destChannel, long offset, int length) throws IOException { public long writeTo(GatheringByteChannel destChannel, long offset, int length) throws IOException {
long newSize = Math.min(channel.size(), end) - start; long newSize = Math.min(channel.size(), end) - start;
if (newSize < size) if (newSize < size.get())
throw new KafkaException(String.format("Size of FileRecords %s has been truncated during write: old size %d, new size %d", file.getAbsolutePath(), size, newSize)); throw new KafkaException(String.format("Size of FileRecords %s has been truncated during write: old size %d, new size %d", file.getAbsolutePath(), size, newSize));
if (offset > size)
throw new KafkaException(String.format("The requested offset %d is out of range. The size of this FileRecords is %d.", offset, size));
long position = start + offset; long position = start + offset;
long count = Math.min(length, this.size - offset); long count = Math.min(length, size.get());
final long bytesTransferred;
if (destChannel instanceof TransportLayer) { if (destChannel instanceof TransportLayer) {
TransportLayer tl = (TransportLayer) destChannel; TransportLayer tl = (TransportLayer) destChannel;
return tl.transferFrom(this.channel, position, count); bytesTransferred = tl.transferFrom(channel, position, count);
} else { } else {
return this.channel.transferTo(position, count, destChannel); bytesTransferred = channel.transferTo(position, count, destChannel);
} }
return bytesTransferred;
}
/**
* Search forward for the file position of the last offset that is greater than or equal to the target offset
* and return its physical position and the size of the message (including log overhead) at the returned offset. If
* no such offsets are found, return null.
*
* @param targetOffset The offset to search for.
* @param startingPosition The starting position in the file to begin searching from.
*/
public LogEntryPosition searchForOffsetWithSize(long targetOffset, int startingPosition) {
Iterator<FileChannelLogEntry> iterator = shallowIteratorFrom(Integer.MAX_VALUE, startingPosition);
while (iterator.hasNext()) {
FileChannelLogEntry entry = iterator.next();
long offset = entry.offset();
if (offset >= targetOffset)
return new LogEntryPosition(offset, entry.position(), entry.sizeInBytes());
}
return null;
}
/**
* Search forward for the message whose timestamp is greater than or equals to the target timestamp.
*
* @param targetTimestamp The timestamp to search for.
* @param startingPosition The starting position to search.
* @return The timestamp and offset of the message found. None, if no message is found.
*/
public TimestampAndOffset searchForTimestamp(long targetTimestamp, int startingPosition) {
Iterator<FileChannelLogEntry> shallowIterator = shallowIteratorFrom(startingPosition);
while (shallowIterator.hasNext()) {
LogEntry shallowEntry = shallowIterator.next();
Record shallowRecord = shallowEntry.record();
if (shallowRecord.timestamp() >= targetTimestamp) {
// We found a message
for (LogEntry deepLogEntry : shallowEntry) {
long timestamp = deepLogEntry.record().timestamp();
if (timestamp >= targetTimestamp)
return new TimestampAndOffset(timestamp, deepLogEntry.offset());
}
throw new IllegalStateException(String.format("The message set (max timestamp = %s, max offset = %s" +
" should contain target timestamp %s, but does not.", shallowRecord.timestamp(),
shallowEntry.offset(), targetTimestamp));
}
}
return null;
}
/**
* Return the largest timestamp of the messages after a given position in this file message set.
* @param startingPosition The starting position.
* @return The largest timestamp of the messages after the given position.
*/
public TimestampAndOffset largestTimestampAfter(int startingPosition) {
long maxTimestamp = Record.NO_TIMESTAMP;
long offsetOfMaxTimestamp = -1L;
Iterator<FileChannelLogEntry> shallowIterator = shallowIteratorFrom(startingPosition);
while (shallowIterator.hasNext()) {
LogEntry shallowEntry = shallowIterator.next();
long timestamp = shallowEntry.record().timestamp();
if (timestamp > maxTimestamp) {
maxTimestamp = timestamp;
offsetOfMaxTimestamp = shallowEntry.offset();
}
}
return new TimestampAndOffset(maxTimestamp, offsetOfMaxTimestamp);
}
/**
* Get an iterator over the shallow entries in the file. Note that the entries are
* backed by the open file channel. When the channel is closed (i.e. when this instance
* is closed), the entries will generally no longer be readable.
* @return An iterator over the shallow entries
*/
@Override
public Iterator<FileChannelLogEntry> shallowIterator() {
return shallowIteratorFrom(start);
}
/**
* Get an iterator over the shallow entries, enforcing a maximum record size
* @param maxRecordSize The maximum allowable size of individual records (including compressed record sets)
* @return An iterator over the shallow entries
*/
public Iterator<FileChannelLogEntry> shallowIterator(int maxRecordSize) {
return shallowIteratorFrom(maxRecordSize, start);
}
private Iterator<FileChannelLogEntry> shallowIteratorFrom(int start) {
return shallowIteratorFrom(Integer.MAX_VALUE, start);
}
private Iterator<FileChannelLogEntry> shallowIteratorFrom(int maxRecordSize, int start) {
final int end;
if (isSlice)
end = this.end;
else
end = this.sizeInBytes();
FileLogInputStream inputStream = new FileLogInputStream(channel, maxRecordSize, start, end);
return RecordsIterator.shallowIterator(inputStream);
} }
@Override @Override
public RecordsIterator iterator() { public Iterator<LogEntry> deepIterator() {
return new RecordsIterator(new FileLogInputStream(channel, start, end), false); final int end;
if (isSlice)
end = this.end;
else
end = this.sizeInBytes();
FileLogInputStream inputStream = new FileLogInputStream(channel, Integer.MAX_VALUE, start, end);
return new RecordsIterator(inputStream, false, false, Integer.MAX_VALUE);
} }
private static class FileLogInputStream implements LogInputStream { public static FileRecords open(File file,
private long position; boolean mutable,
protected final long end; boolean fileAlreadyExists,
protected final FileChannel channel; int initFileSize,
private final ByteBuffer logHeaderBuffer = ByteBuffer.allocate(Records.LOG_OVERHEAD); boolean preallocate) throws IOException {
FileChannel channel = openChannel(file, mutable, fileAlreadyExists, initFileSize, preallocate);
int end = (!fileAlreadyExists && preallocate) ? 0 : Integer.MAX_VALUE;
return new FileRecords(file, channel, 0, end, false);
}
public FileLogInputStream(FileChannel channel, long start, long end) { public static FileRecords open(File file,
this.channel = channel; boolean fileAlreadyExists,
this.position = start; int initFileSize,
this.end = end; boolean preallocate) throws IOException {
return open(file, true, fileAlreadyExists, initFileSize, preallocate);
}
public static FileRecords open(File file, boolean mutable) throws IOException {
return open(file, mutable, false, 0, false);
}
public static FileRecords open(File file) throws IOException {
return open(file, true);
}
/**
* Open a channel for the given file
* For windows NTFS and some old LINUX file system, set preallocate to true and initFileSize
* with one value (for example 512 * 1025 *1024 ) can improve the kafka produce performance.
* @param file File path
* @param mutable mutable
* @param fileAlreadyExists File already exists or not
* @param initFileSize The size used for pre allocate file, for example 512 * 1025 *1024
* @param preallocate Pre allocate file or not, gotten from configuration.
*/
private static FileChannel openChannel(File file,
boolean mutable,
boolean fileAlreadyExists,
int initFileSize,
boolean preallocate) throws IOException {
if (mutable) {
if (fileAlreadyExists) {
return new RandomAccessFile(file, "rw").getChannel();
} else {
if (preallocate) {
RandomAccessFile randomAccessFile = new RandomAccessFile(file, "rw");
randomAccessFile.setLength(initFileSize);
return randomAccessFile.getChannel();
} else {
return new RandomAccessFile(file, "rw").getChannel();
}
}
} else {
return new FileInputStream(file).getChannel();
}
}
public static class LogEntryPosition {
public final long offset;
public final int position;
public final int size;
public LogEntryPosition(long offset, int position, int size) {
this.offset = offset;
this.position = position;
this.size = size;
} }
@Override @Override
public LogEntry nextEntry() throws IOException { public boolean equals(Object o) {
if (position + Records.LOG_OVERHEAD >= end) if (this == o) return true;
return null; if (o == null || getClass() != o.getClass()) return false;
logHeaderBuffer.rewind(); LogEntryPosition that = (LogEntryPosition) o;
channel.read(logHeaderBuffer, position);
if (logHeaderBuffer.hasRemaining())
return null;
logHeaderBuffer.rewind(); if (offset != that.offset) return false;
long offset = logHeaderBuffer.getLong(); if (position != that.position) return false;
int size = logHeaderBuffer.getInt(); return size == that.size;
if (size < 0)
throw new IllegalStateException("Record with size " + size);
if (position + Records.LOG_OVERHEAD + size > end) }
return null;
ByteBuffer recordBuffer = ByteBuffer.allocate(size); @Override
channel.read(recordBuffer, position + Records.LOG_OVERHEAD); public int hashCode() {
if (recordBuffer.hasRemaining()) int result = (int) (offset ^ (offset >>> 32));
return null; result = 31 * result + position;
recordBuffer.rewind(); result = 31 * result + size;
return result;
}
Record record = new Record(recordBuffer); @Override
LogEntry logEntry = new LogEntry(offset, record); public String toString() {
position += logEntry.size(); return "LogEntryPosition(" +
return logEntry; "offset=" + offset +
", position=" + position +
", size=" + size +
')';
} }
} }
public static class TimestampAndOffset {
public final long timestamp;
public final long offset;
public TimestampAndOffset(long timestamp, long offset) {
this.timestamp = timestamp;
this.offset = offset;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
TimestampAndOffset that = (TimestampAndOffset) o;
if (timestamp != that.timestamp) return false;
return offset == that.offset;
}
@Override
public int hashCode() {
int result = (int) (timestamp ^ (timestamp >>> 32));
result = 31 * result + (int) (offset ^ (offset >>> 32));
return result;
}
@Override
public String toString() {
return "TimestampAndOffset(" +
"timestamp=" + timestamp +
", offset=" + offset +
')';
}
}
} }

View File

@ -16,9 +16,9 @@
*/ */
package org.apache.kafka.common.record; package org.apache.kafka.common.record;
import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.errors.CorruptRecordException;
public class InvalidRecordException extends KafkaException { public class InvalidRecordException extends CorruptRecordException {
private static final long serialVersionUID = 1; private static final long serialVersionUID = 1;

View File

@ -16,33 +16,156 @@
*/ */
package org.apache.kafka.common.record; package org.apache.kafka.common.record;
import java.io.DataOutputStream;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Collections;
import java.util.Iterator;
import static org.apache.kafka.common.record.Records.LOG_OVERHEAD;
/** /**
* An offset and record pair * An offset and record pair
*/ */
public final class LogEntry { public abstract class LogEntry implements Iterable<LogEntry> {
private final long offset; /**
private final Record record; * Get the offset of this entry. Note that if this entry contains a compressed
* message set, then this offset will be the last offset of the nested entries
* @return the last offset contained in this entry
*/
public abstract long offset();
public LogEntry(long offset, Record record) { /**
this.offset = offset; * Get the shallow record for this log entry.
this.record = record; * @return the shallow record
*/
public abstract Record record();
/**
* Get the first offset of the records contained in this entry. Note that this
* generally requires deep iteration, which requires decompression, so this should
* be used with caution.
* @return The first offset contained in this entry
*/
public long firstOffset() {
return iterator().next().offset();
} }
public long offset() { /**
return this.offset; * Get the offset following this entry (i.e. the last offset contained in this entry plus one).
* @return the next consecutive offset following this entry
*/
public long nextOffset() {
return offset() + 1;
} }
public Record record() { /**
return this.record; * Get the message format version of this entry (i.e its magic value).
* @return the magic byte
*/
public byte magic() {
return record().magic();
} }
@Override @Override
public String toString() { public String toString() {
return "LogEntry(" + offset + ", " + record + ")"; return "LogEntry(" + offset() + ", " + record() + ")";
} }
public int size() { /**
return record.size() + Records.LOG_OVERHEAD; * Get the size in bytes of this entry, including the size of the record and the log overhead.
* @return The size in bytes of this entry
*/
public int sizeInBytes() {
return record().sizeInBytes() + LOG_OVERHEAD;
} }
/**
* Check whether this entry contains a compressed message set.
* @return true if so, false otherwise
*/
public boolean isCompressed() {
return record().compressionType() != CompressionType.NONE;
}
/**
* Write this entry into a buffer.
* @param buffer The buffer to write the entry to
*/
public void writeTo(ByteBuffer buffer) {
writeHeader(buffer, offset(), record().sizeInBytes());
buffer.put(record().buffer().duplicate());
}
/**
* Get an iterator for the nested entries contained within this log entry. Note that
* if the entry is not compressed, then this method will return an iterator over the
* shallow entry only (i.e. this object).
* @return An iterator over the entries contained within this log entry
*/
@Override
public Iterator<LogEntry> iterator() {
if (isCompressed())
return new RecordsIterator.DeepRecordsIterator(this, false, Integer.MAX_VALUE);
return Collections.singletonList(this).iterator();
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || !(o instanceof LogEntry)) return false;
LogEntry that = (LogEntry) o;
if (offset() != that.offset()) return false;
Record thisRecord = record();
Record thatRecord = that.record();
return thisRecord != null ? thisRecord.equals(thatRecord) : thatRecord == null;
}
@Override
public int hashCode() {
long offset = offset();
Record record = record();
int result = (int) (offset ^ (offset >>> 32));
result = 31 * result + (record != null ? record.hashCode() : 0);
return result;
}
public static void writeHeader(ByteBuffer buffer, long offset, int size) {
buffer.putLong(offset);
buffer.putInt(size);
}
public static void writeHeader(DataOutputStream out, long offset, int size) throws IOException {
out.writeLong(offset);
out.writeInt(size);
}
private static class SimpleLogEntry extends LogEntry {
private final long offset;
private final Record record;
public SimpleLogEntry(long offset, Record record) {
this.offset = offset;
this.record = record;
}
@Override
public long offset() {
return offset;
}
@Override
public Record record() {
return record;
}
}
public static LogEntry create(long offset, Record record) {
return new SimpleLogEntry(offset, record);
}
} }

View File

@ -20,10 +20,14 @@ import java.io.IOException;
/** /**
* An abstraction between an underlying input stream and record iterators, a LogInputStream * An abstraction between an underlying input stream and record iterators, a LogInputStream
* returns only the shallow log entries, depending on {@link org.apache.kafka.common.record.RecordsIterator.DeepRecordsIterator} * returns only the shallow log entries, depending on {@link RecordsIterator.DeepRecordsIterator}
* for the deep iteration. * for the deep iteration. The generic typing allows for implementations which present only
* a view of the log entries, which enables more efficient iteration when the record data is
* not actually needed. See for example {@link org.apache.kafka.common.record.FileLogInputStream.FileChannelLogEntry}
* in which the record is not brought into memory until needed.
* @param <T> Type parameter of the log entry
*/ */
interface LogInputStream { interface LogInputStream<T extends LogEntry> {
/** /**
* Get the next log entry from the underlying input stream. * Get the next log entry from the underlying input stream.
@ -31,5 +35,5 @@ interface LogInputStream {
* @return The next log entry or null if there is none * @return The next log entry or null if there is none
* @throws IOException for any IO errors * @throws IOException for any IO errors
*/ */
LogEntry nextEntry() throws IOException; T nextEntry() throws IOException;
} }

View File

@ -12,197 +12,185 @@
*/ */
package org.apache.kafka.common.record; package org.apache.kafka.common.record;
import java.io.DataInputStream; import org.apache.kafka.common.record.ByteBufferLogInputStream.ByteBufferLogEntry;
import java.io.IOException; import java.io.IOException;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.nio.channels.GatheringByteChannel; import java.nio.channels.GatheringByteChannel;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Iterator; import java.util.Iterator;
import java.util.List;
/** /**
* A {@link Records} implementation backed by a ByteBuffer. * A {@link Records} implementation backed by a ByteBuffer. This is used only for reading or
* modifying in-place an existing buffer of log entries. To create a new buffer see {@link MemoryRecordsBuilder},
* or one of the {@link #builder(ByteBuffer, byte, CompressionType, TimestampType) builder} variants.
*/ */
public class MemoryRecords implements Records { public class MemoryRecords extends AbstractRecords {
public final static MemoryRecords EMPTY = MemoryRecords.readableRecords(ByteBuffer.allocate(0)); public final static MemoryRecords EMPTY = MemoryRecords.readableRecords(ByteBuffer.allocate(0));
private final static int WRITE_LIMIT_FOR_READABLE_ONLY = -1;
// the compressor used for appends-only
private final Compressor compressor;
// the write limit for writable buffer, which may be smaller than the buffer capacity
private final int writeLimit;
// the capacity of the initial buffer, which is only used for de-allocation of writable records
private final int initialCapacity;
// the underlying buffer used for read; while the records are still writable it is null // the underlying buffer used for read; while the records are still writable it is null
private ByteBuffer buffer; private ByteBuffer buffer;
private int validBytes = -1;
// indicate if the memory records is writable or not (i.e. used for appends or read-only)
private boolean writable;
// Construct a writable memory records // Construct a writable memory records
private MemoryRecords(ByteBuffer buffer, CompressionType type, boolean writable, int writeLimit) { private MemoryRecords(ByteBuffer buffer) {
this.writable = writable; this.buffer = buffer;
this.writeLimit = writeLimit;
this.initialCapacity = buffer.capacity();
if (this.writable) {
this.buffer = null;
this.compressor = new Compressor(buffer, type);
} else {
this.buffer = buffer;
this.compressor = null;
}
} }
public static MemoryRecords emptyRecords(ByteBuffer buffer, CompressionType type, int writeLimit) {
return new MemoryRecords(buffer, type, true, writeLimit);
}
public static MemoryRecords emptyRecords(ByteBuffer buffer, CompressionType type) {
// use the buffer capacity as the default write limit
return emptyRecords(buffer, type, buffer.capacity());
}
public static MemoryRecords readableRecords(ByteBuffer buffer) {
return new MemoryRecords(buffer, CompressionType.NONE, false, WRITE_LIMIT_FOR_READABLE_ONLY);
}
/**
* Append the given record and offset to the buffer
*/
public void append(long offset, Record record) {
if (!writable)
throw new IllegalStateException("Memory records is not writable");
int size = record.size();
compressor.putLong(offset);
compressor.putInt(size);
compressor.put(record.buffer());
compressor.recordWritten(size + Records.LOG_OVERHEAD);
record.buffer().rewind();
}
/**
* Append a new record and offset to the buffer
* @return crc of the record
*/
public long append(long offset, long timestamp, byte[] key, byte[] value) {
if (!writable)
throw new IllegalStateException("Memory records is not writable");
int size = Record.recordSize(key, value);
compressor.putLong(offset);
compressor.putInt(size);
long crc = compressor.putRecord(timestamp, key, value);
compressor.recordWritten(size + Records.LOG_OVERHEAD);
return crc;
}
/**
* Check if we have room for a new record containing the given key/value pair
*
* Note that the return value is based on the estimate of the bytes written to the compressor, which may not be
* accurate if compression is really used. When this happens, the following append may cause dynamic buffer
* re-allocation in the underlying byte buffer stream.
*
* There is an exceptional case when appending a single message whose size is larger than the batch size, the
* capacity will be the message size which is larger than the write limit, i.e. the batch size. In this case
* the checking should be based on the capacity of the initialized buffer rather than the write limit in order
* to accept this single record.
*/
public boolean hasRoomFor(byte[] key, byte[] value) {
if (!this.writable)
return false;
return this.compressor.numRecordsWritten() == 0 ?
this.initialCapacity >= Records.LOG_OVERHEAD + Record.recordSize(key, value) :
this.writeLimit >= this.compressor.estimatedBytesWritten() + Records.LOG_OVERHEAD + Record.recordSize(key, value);
}
public boolean isFull() {
return !this.writable || this.writeLimit <= this.compressor.estimatedBytesWritten();
}
/**
* Close this batch for no more appends
*/
public void close() {
if (writable) {
// close the compressor to fill-in wrapper message metadata if necessary
compressor.close();
// flip the underlying buffer to be ready for reads
buffer = compressor.buffer();
buffer.flip();
// reset the writable flag
writable = false;
}
}
/**
* The size of this record set
*/
@Override @Override
public int sizeInBytes() { public int sizeInBytes() {
if (writable) { return buffer.limit();
return compressor.buffer().position();
} else {
return buffer.limit();
}
} }
@Override @Override
public long writeTo(GatheringByteChannel channel, long offset, int length) throws IOException { public long writeTo(GatheringByteChannel channel, long position, int length) throws IOException {
ByteBuffer dup = buffer.duplicate(); ByteBuffer dup = buffer.duplicate();
int position = (int) offset; int pos = (int) position;
dup.position(position); dup.position(pos);
dup.limit(position + length); dup.limit(pos + length);
return channel.write(dup); return channel.write(dup);
} }
/** /**
* The compression rate of this record set * Write all records to the given channel (including partial records).
* @param channel The channel to write to
* @return The number of bytes written
* @throws IOException For any IO errors writing to the channel
*/ */
public double compressionRate() { public int writeFullyTo(GatheringByteChannel channel) throws IOException {
if (compressor == null) buffer.mark();
return 1.0; int written = 0;
else while (written < sizeInBytes())
return compressor.compressionRate(); written += channel.write(buffer);
buffer.reset();
return written;
} }
/** /**
* Return the capacity of the initial buffer, for writable records * The total number of bytes in this message set not including any partial, trailing messages. This
* it may be different from the current buffer's capacity * may be smaller than what is returned by {@link #sizeInBytes()}.
* @return The number of valid bytes
*/ */
public int initialCapacity() { public int validBytes() {
return this.initialCapacity; if (validBytes >= 0)
return validBytes;
int bytes = 0;
Iterator<ByteBufferLogEntry> iterator = shallowIterator();
while (iterator.hasNext())
bytes += iterator.next().sizeInBytes();
this.validBytes = bytes;
return bytes;
} }
/** /**
* Get the byte buffer that backs this records instance for reading * Filter the records into the provided ByteBuffer.
* @param filter The filter function
* @param buffer The byte buffer to write the filtered records to
* @return A FilterResult with a summary of the output (for metrics)
*/
public FilterResult filterTo(LogEntryFilter filter, ByteBuffer buffer) {
long maxTimestamp = Record.NO_TIMESTAMP;
long shallowOffsetOfMaxTimestamp = -1L;
int messagesRead = 0;
int bytesRead = 0;
int messagesRetained = 0;
int bytesRetained = 0;
Iterator<ByteBufferLogEntry> shallowIterator = shallowIterator();
while (shallowIterator.hasNext()) {
ByteBufferLogEntry shallowEntry = shallowIterator.next();
bytesRead += shallowEntry.sizeInBytes();
// We use the absolute offset to decide whether to retain the message or not (this is handled by the
// deep iterator). Because of KAFKA-4298, we have to allow for the possibility that a previous version
// corrupted the log by writing a compressed message set with a wrapper magic value not matching the magic
// of the inner messages. This will be fixed as we recopy the messages to the destination buffer.
Record shallowRecord = shallowEntry.record();
byte shallowMagic = shallowRecord.magic();
boolean writeOriginalEntry = true;
List<LogEntry> retainedEntries = new ArrayList<>();
for (LogEntry deepEntry : shallowEntry) {
Record deepRecord = deepEntry.record();
messagesRead += 1;
if (filter.shouldRetain(deepEntry)) {
// Check for log corruption due to KAFKA-4298. If we find it, make sure that we overwrite
// the corrupted entry with correct data.
if (shallowMagic != deepRecord.magic())
writeOriginalEntry = false;
retainedEntries.add(deepEntry);
} else {
writeOriginalEntry = false;
}
}
if (writeOriginalEntry) {
// There are no messages compacted out and no message format conversion, write the original message set back
shallowEntry.writeTo(buffer);
messagesRetained += retainedEntries.size();
bytesRetained += shallowEntry.sizeInBytes();
if (shallowRecord.timestamp() > maxTimestamp) {
maxTimestamp = shallowRecord.timestamp();
shallowOffsetOfMaxTimestamp = shallowEntry.offset();
}
} else if (!retainedEntries.isEmpty()) {
ByteBuffer slice = buffer.slice();
MemoryRecordsBuilder builder = builderWithEntries(slice, shallowRecord.timestampType(), shallowRecord.compressionType(),
shallowRecord.timestamp(), retainedEntries);
MemoryRecords records = builder.build();
buffer.position(buffer.position() + slice.position());
messagesRetained += retainedEntries.size();
bytesRetained += records.sizeInBytes();
MemoryRecordsBuilder.RecordsInfo info = builder.info();
if (info.maxTimestamp > maxTimestamp) {
maxTimestamp = info.maxTimestamp;
shallowOffsetOfMaxTimestamp = info.shallowOffsetOfMaxTimestamp;
}
}
}
return new FilterResult(messagesRead, bytesRead, messagesRetained, bytesRetained, maxTimestamp, shallowOffsetOfMaxTimestamp);
}
/**
* Get the byte buffer that backs this instance for reading.
*/ */
public ByteBuffer buffer() { public ByteBuffer buffer() {
if (writable)
throw new IllegalStateException("The memory records must not be writable any more before getting its underlying buffer");
return buffer.duplicate(); return buffer.duplicate();
} }
@Override @Override
public Iterator<LogEntry> iterator() { public Iterator<ByteBufferLogEntry> shallowIterator() {
ByteBuffer input = this.buffer.duplicate(); return RecordsIterator.shallowIterator(new ByteBufferLogInputStream(buffer.duplicate(), Integer.MAX_VALUE));
if (writable)
// flip on a duplicate buffer for reading
input.flip();
return new RecordsIterator(new ByteBufferLogInputStream(input), false);
} }
@Override
public Iterator<LogEntry> deepIterator() {
return deepIterator(false);
}
public Iterator<LogEntry> deepIterator(boolean ensureMatchingMagic) {
return deepIterator(ensureMatchingMagic, Integer.MAX_VALUE);
}
public Iterator<LogEntry> deepIterator(boolean ensureMatchingMagic, int maxMessageSize) {
return new RecordsIterator(new ByteBufferLogInputStream(buffer.duplicate(), maxMessageSize), false,
ensureMatchingMagic, maxMessageSize);
}
@Override @Override
public String toString() { public String toString() {
Iterator<LogEntry> iter = iterator(); Iterator<LogEntry> iter = deepIterator();
StringBuilder builder = new StringBuilder(); StringBuilder builder = new StringBuilder();
builder.append('['); builder.append('[');
while (iter.hasNext()) { while (iter.hasNext()) {
@ -214,16 +202,13 @@ public class MemoryRecords implements Records {
builder.append("record="); builder.append("record=");
builder.append(entry.record()); builder.append(entry.record());
builder.append(")"); builder.append(")");
if (iter.hasNext())
builder.append(", ");
} }
builder.append(']'); builder.append(']');
return builder.toString(); return builder.toString();
} }
/** Visible for testing */
public boolean isWritable() {
return writable;
}
@Override @Override
public boolean equals(Object o) { public boolean equals(Object o) {
if (this == o) return true; if (this == o) return true;
@ -232,7 +217,6 @@ public class MemoryRecords implements Records {
MemoryRecords that = (MemoryRecords) o; MemoryRecords that = (MemoryRecords) o;
return buffer.equals(that.buffer); return buffer.equals(that.buffer);
} }
@Override @Override
@ -240,28 +224,153 @@ public class MemoryRecords implements Records {
return buffer.hashCode(); return buffer.hashCode();
} }
private static class ByteBufferLogInputStream implements LogInputStream { public interface LogEntryFilter {
private final DataInputStream stream; boolean shouldRetain(LogEntry entry);
private final ByteBuffer buffer; }
private ByteBufferLogInputStream(ByteBuffer buffer) { public static class FilterResult {
this.stream = new DataInputStream(new ByteBufferInputStream(buffer)); public final int messagesRead;
this.buffer = buffer; public final int bytesRead;
} public final int messagesRetained;
public final int bytesRetained;
public final long maxTimestamp;
public final long shallowOffsetOfMaxTimestamp;
public LogEntry nextEntry() throws IOException { public FilterResult(int messagesRead,
long offset = stream.readLong(); int bytesRead,
int size = stream.readInt(); int messagesRetained,
if (size < 0) int bytesRetained,
throw new IllegalStateException("Record with size " + size); long maxTimestamp,
long shallowOffsetOfMaxTimestamp) {
ByteBuffer slice = buffer.slice(); this.messagesRead = messagesRead;
int newPos = buffer.position() + size; this.bytesRead = bytesRead;
if (newPos > buffer.limit()) this.messagesRetained = messagesRetained;
return null; this.bytesRetained = bytesRetained;
buffer.position(newPos); this.maxTimestamp = maxTimestamp;
slice.limit(size); this.shallowOffsetOfMaxTimestamp = shallowOffsetOfMaxTimestamp;
return new LogEntry(offset, new Record(slice));
} }
} }
public static MemoryRecordsBuilder builder(ByteBuffer buffer,
CompressionType compressionType,
TimestampType timestampType,
int writeLimit) {
return new MemoryRecordsBuilder(buffer, Record.CURRENT_MAGIC_VALUE, compressionType, timestampType, 0L, System.currentTimeMillis(), writeLimit);
}
public static MemoryRecordsBuilder builder(ByteBuffer buffer,
byte magic,
CompressionType compressionType,
TimestampType timestampType,
long baseOffset,
long logAppendTime) {
return new MemoryRecordsBuilder(buffer, magic, compressionType, timestampType, baseOffset, logAppendTime, buffer.capacity());
}
public static MemoryRecordsBuilder builder(ByteBuffer buffer,
CompressionType compressionType,
TimestampType timestampType) {
// use the buffer capacity as the default write limit
return builder(buffer, compressionType, timestampType, buffer.capacity());
}
public static MemoryRecordsBuilder builder(ByteBuffer buffer,
byte magic,
CompressionType compressionType,
TimestampType timestampType) {
return builder(buffer, magic, compressionType, timestampType, 0L);
}
public static MemoryRecordsBuilder builder(ByteBuffer buffer,
byte magic,
CompressionType compressionType,
TimestampType timestampType,
long baseOffset) {
return builder(buffer, magic, compressionType, timestampType, baseOffset, System.currentTimeMillis());
}
public static MemoryRecords readableRecords(ByteBuffer buffer) {
return new MemoryRecords(buffer);
}
public static MemoryRecords withLogEntries(CompressionType compressionType, List<LogEntry> entries) {
return withLogEntries(TimestampType.CREATE_TIME, compressionType, System.currentTimeMillis(), entries);
}
public static MemoryRecords withLogEntries(LogEntry ... entries) {
return withLogEntries(CompressionType.NONE, Arrays.asList(entries));
}
public static MemoryRecords withRecords(CompressionType compressionType, long initialOffset, List<Record> records) {
return withRecords(initialOffset, TimestampType.CREATE_TIME, compressionType, System.currentTimeMillis(), records);
}
public static MemoryRecords withRecords(Record ... records) {
return withRecords(CompressionType.NONE, 0L, Arrays.asList(records));
}
public static MemoryRecords withRecords(long initialOffset, Record ... records) {
return withRecords(CompressionType.NONE, initialOffset, Arrays.asList(records));
}
public static MemoryRecords withRecords(CompressionType compressionType, Record ... records) {
return withRecords(compressionType, 0L, Arrays.asList(records));
}
public static MemoryRecords withRecords(TimestampType timestampType, CompressionType compressionType, Record ... records) {
return withRecords(0L, timestampType, compressionType, System.currentTimeMillis(), Arrays.asList(records));
}
public static MemoryRecords withRecords(long initialOffset,
TimestampType timestampType,
CompressionType compressionType,
long logAppendTime,
List<Record> records) {
return withLogEntries(timestampType, compressionType, logAppendTime, buildLogEntries(initialOffset, records));
}
private static MemoryRecords withLogEntries(TimestampType timestampType,
CompressionType compressionType,
long logAppendTime,
List<LogEntry> entries) {
if (entries.isEmpty())
return MemoryRecords.EMPTY;
return builderWithEntries(timestampType, compressionType, logAppendTime, entries).build();
}
private static List<LogEntry> buildLogEntries(long initialOffset, List<Record> records) {
List<LogEntry> entries = new ArrayList<>();
for (Record record : records)
entries.add(LogEntry.create(initialOffset++, record));
return entries;
}
public static MemoryRecordsBuilder builderWithEntries(TimestampType timestampType,
CompressionType compressionType,
long logAppendTime,
List<LogEntry> entries) {
ByteBuffer buffer = ByteBuffer.allocate(estimatedSize(compressionType, entries));
return builderWithEntries(buffer, timestampType, compressionType, logAppendTime, entries);
}
private static MemoryRecordsBuilder builderWithEntries(ByteBuffer buffer,
TimestampType timestampType,
CompressionType compressionType,
long logAppendTime,
List<LogEntry> entries) {
if (entries.isEmpty())
throw new IllegalArgumentException();
LogEntry firstEntry = entries.iterator().next();
long firstOffset = firstEntry.offset();
byte magic = firstEntry.record().magic();
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic, compressionType, timestampType,
firstOffset, logAppendTime);
for (LogEntry entry : entries)
builder.append(entry);
return builder;
}
} }

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; package org.apache.kafka.common.record;
import java.nio.ByteBuffer; import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.utils.Crc32; import org.apache.kafka.common.utils.Crc32;
import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.utils.Utils;
import java.io.DataOutputStream;
import java.io.IOException;
import java.nio.ByteBuffer;
import static org.apache.kafka.common.utils.Utils.wrapNullable;
/** /**
* A record: a serialized key and value along with the associated CRC and other fields * A record: a serialized key and value along with the associated CRC and other fields
@ -53,7 +57,12 @@ public final class Record {
/** /**
* The amount of overhead bytes in a record * The amount of overhead bytes in a record
*/ */
public static final int RECORD_OVERHEAD = HEADER_SIZE + TIMESTAMP_LENGTH + KEY_SIZE_LENGTH + VALUE_SIZE_LENGTH; public static final int RECORD_OVERHEAD_V0 = HEADER_SIZE + KEY_SIZE_LENGTH + VALUE_SIZE_LENGTH;
/**
* The amount of overhead bytes in a record
*/
public static final int RECORD_OVERHEAD_V1 = HEADER_SIZE + TIMESTAMP_LENGTH + KEY_SIZE_LENGTH + VALUE_SIZE_LENGTH;
/** /**
* The "magic" values * The "magic" values
@ -79,11 +88,6 @@ public final class Record {
public static final byte TIMESTAMP_TYPE_MASK = 0x08; public static final byte TIMESTAMP_TYPE_MASK = 0x08;
public static final int TIMESTAMP_TYPE_ATTRIBUTE_OFFSET = 3; public static final int TIMESTAMP_TYPE_ATTRIBUTE_OFFSET = 3;
/**
* Compression code for uncompressed records
*/
public static final int NO_COMPRESSION = 0;
/** /**
* Timestamp value for records without a timestamp * Timestamp value for records without a timestamp
*/ */
@ -94,155 +98,20 @@ public final class Record {
private final TimestampType wrapperRecordTimestampType; private final TimestampType wrapperRecordTimestampType;
public Record(ByteBuffer buffer) { public Record(ByteBuffer buffer) {
this.buffer = buffer; this(buffer, null, null);
this.wrapperRecordTimestamp = null;
this.wrapperRecordTimestampType = null;
} }
// Package private constructor for inner iteration. public Record(ByteBuffer buffer, Long wrapperRecordTimestamp, TimestampType wrapperRecordTimestampType) {
Record(ByteBuffer buffer, Long wrapperRecordTimestamp, TimestampType wrapperRecordTimestampType) {
this.buffer = buffer; this.buffer = buffer;
this.wrapperRecordTimestamp = wrapperRecordTimestamp; this.wrapperRecordTimestamp = wrapperRecordTimestamp;
this.wrapperRecordTimestampType = wrapperRecordTimestampType; this.wrapperRecordTimestampType = wrapperRecordTimestampType;
} }
/**
* A constructor to create a LogRecord. If the record's compression type is not none, then
* its value payload should be already compressed with the specified type; the constructor
* would always write the value payload as is and will not do the compression itself.
*
* @param timestamp The timestamp of the record
* @param key The key of the record (null, if none)
* @param value The record value
* @param type The compression type used on the contents of the record (if any)
* @param valueOffset The offset into the payload array used to extract payload
* @param valueSize The size of the payload to use
*/
public Record(long timestamp, byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) {
this(ByteBuffer.allocate(recordSize(key == null ? 0 : key.length,
value == null ? 0 : valueSize >= 0 ? valueSize : value.length - valueOffset)));
write(this.buffer, timestamp, key, value, type, valueOffset, valueSize);
this.buffer.rewind();
}
public Record(long timestamp, byte[] key, byte[] value, CompressionType type) {
this(timestamp, key, value, type, 0, -1);
}
public Record(long timestamp, byte[] value, CompressionType type) {
this(timestamp, null, value, type);
}
public Record(long timestamp, byte[] key, byte[] value) {
this(timestamp, key, value, CompressionType.NONE);
}
public Record(long timestamp, byte[] value) {
this(timestamp, null, value, CompressionType.NONE);
}
// Write a record to the buffer, if the record's compression type is none, then
// its value payload should be already compressed with the specified type
public static void write(ByteBuffer buffer, long timestamp, byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) {
// construct the compressor with compression type none since this function will not do any
//compression according to the input type, it will just write the record's payload as is
Compressor compressor = new Compressor(buffer, CompressionType.NONE);
try {
compressor.putRecord(timestamp, key, value, type, valueOffset, valueSize);
} finally {
compressor.close();
}
}
public static void write(Compressor compressor, long crc, byte attributes, long timestamp, byte[] key, byte[] value, int valueOffset, int valueSize) {
// write crc
compressor.putInt((int) (crc & 0xffffffffL));
// write magic value
compressor.putByte(CURRENT_MAGIC_VALUE);
// write attributes
compressor.putByte(attributes);
// write timestamp
compressor.putLong(timestamp);
// write the key
if (key == null) {
compressor.putInt(-1);
} else {
compressor.putInt(key.length);
compressor.put(key, 0, key.length);
}
// write the value
if (value == null) {
compressor.putInt(-1);
} else {
int size = valueSize >= 0 ? valueSize : (value.length - valueOffset);
compressor.putInt(size);
compressor.put(value, valueOffset, size);
}
}
public static int recordSize(byte[] key, byte[] value) {
return recordSize(key == null ? 0 : key.length, value == null ? 0 : value.length);
}
public static int recordSize(int keySize, int valueSize) {
return CRC_LENGTH + MAGIC_LENGTH + ATTRIBUTE_LENGTH + TIMESTAMP_LENGTH + KEY_SIZE_LENGTH + keySize + VALUE_SIZE_LENGTH + valueSize;
}
public ByteBuffer buffer() {
return this.buffer;
}
public static byte computeAttributes(CompressionType type) {
byte attributes = 0;
if (type.id > 0)
attributes = (byte) (attributes | (COMPRESSION_CODEC_MASK & type.id));
return attributes;
}
/**
* Compute the checksum of the record from the record contents
*/
public static long computeChecksum(ByteBuffer buffer, int position, int size) {
Crc32 crc = new Crc32();
crc.update(buffer.array(), buffer.arrayOffset() + position, size);
return crc.getValue();
}
/**
* Compute the checksum of the record from the attributes, key and value payloads
*/
public static long computeChecksum(long timestamp, byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) {
Crc32 crc = new Crc32();
crc.update(CURRENT_MAGIC_VALUE);
byte attributes = 0;
if (type.id > 0)
attributes = (byte) (attributes | (COMPRESSION_CODEC_MASK & type.id));
crc.update(attributes);
crc.updateLong(timestamp);
// update for the key
if (key == null) {
crc.updateInt(-1);
} else {
crc.updateInt(key.length);
crc.update(key, 0, key.length);
}
// update for the value
if (value == null) {
crc.updateInt(-1);
} else {
int size = valueSize >= 0 ? valueSize : (value.length - valueOffset);
crc.updateInt(size);
crc.update(value, valueOffset, size);
}
return crc.getValue();
}
/** /**
* Compute the checksum of the record from the record contents * Compute the checksum of the record from the record contents
*/ */
public long computeChecksum() { public long computeChecksum() {
return computeChecksum(buffer, MAGIC_OFFSET, buffer.limit() - MAGIC_OFFSET); return Utils.computeChecksum(buffer, MAGIC_OFFSET, buffer.limit() - MAGIC_OFFSET);
} }
/** /**
@ -256,7 +125,15 @@ public final class Record {
* Returns true if the crc stored with the record matches the crc computed off the record contents * Returns true if the crc stored with the record matches the crc computed off the record contents
*/ */
public boolean isValid() { public boolean isValid() {
return size() >= CRC_LENGTH && checksum() == computeChecksum(); return sizeInBytes() >= CRC_LENGTH && checksum() == computeChecksum();
}
public Long wrapperRecordTimestamp() {
return wrapperRecordTimestamp;
}
public TimestampType wrapperRecordTimestampType() {
return wrapperRecordTimestampType;
} }
/** /**
@ -264,9 +141,9 @@ public final class Record {
*/ */
public void ensureValid() { public void ensureValid() {
if (!isValid()) { if (!isValid()) {
if (size() < CRC_LENGTH) if (sizeInBytes() < CRC_LENGTH)
throw new InvalidRecordException("Record is corrupt (crc could not be retrieved as the record is too " throw new InvalidRecordException("Record is corrupt (crc could not be retrieved as the record is too "
+ "small, size = " + size() + ")"); + "small, size = " + sizeInBytes() + ")");
else else
throw new InvalidRecordException("Record is corrupt (stored crc = " + checksum() throw new InvalidRecordException("Record is corrupt (stored crc = " + checksum()
+ ", computed crc = " + computeChecksum() + ")"); + ", computed crc = " + computeChecksum() + ")");
@ -274,14 +151,17 @@ public final class Record {
} }
/** /**
* The complete serialized size of this record in bytes (including crc, header attributes, etc) * The complete serialized size of this record in bytes (including crc, header attributes, etc), but
* excluding the log overhead (offset and record size).
* @return the size in bytes
*/ */
public int size() { public int sizeInBytes() {
return buffer.limit(); return buffer.limit();
} }
/** /**
* The length of the key in bytes * The length of the key in bytes
* @return the size in bytes of the key (0 if the key is null)
*/ */
public int keySize() { public int keySize() {
if (magic() == MAGIC_VALUE_V0) if (magic() == MAGIC_VALUE_V0)
@ -292,6 +172,7 @@ public final class Record {
/** /**
* Does the record have a key? * Does the record have a key?
* @return true if so, false otherwise
*/ */
public boolean hasKey() { public boolean hasKey() {
return keySize() >= 0; return keySize() >= 0;
@ -309,13 +190,23 @@ public final class Record {
/** /**
* The length of the value in bytes * The length of the value in bytes
* @return the size in bytes of the value (0 if the value is null)
*/ */
public int valueSize() { public int valueSize() {
return buffer.getInt(valueSizeOffset()); return buffer.getInt(valueSizeOffset());
} }
/** /**
* The magic version of this record * Check whether the value field of this record is null.
* @return true if the value is null, false otherwise
*/
public boolean hasNullValue() {
return valueSize() < 0;
}
/**
* The magic value (i.e. message format version) of this record
* @return the magic value
*/ */
public byte magic() { public byte magic() {
return buffer.get(MAGIC_OFFSET); return buffer.get(MAGIC_OFFSET);
@ -323,6 +214,7 @@ public final class Record {
/** /**
* The attributes stored with this record * The attributes stored with this record
* @return the attributes
*/ */
public byte attributes() { public byte attributes() {
return buffer.get(ATTRIBUTES_OFFSET); return buffer.get(ATTRIBUTES_OFFSET);
@ -333,6 +225,8 @@ public final class Record {
* 1. wrapperRecordTimestampType = null and wrapperRecordTimestamp is null - Uncompressed message, timestamp is in the message. * 1. wrapperRecordTimestampType = null and wrapperRecordTimestamp is null - Uncompressed message, timestamp is in the message.
* 2. wrapperRecordTimestampType = LOG_APPEND_TIME and WrapperRecordTimestamp is not null - Compressed message using LOG_APPEND_TIME * 2. wrapperRecordTimestampType = LOG_APPEND_TIME and WrapperRecordTimestamp is not null - Compressed message using LOG_APPEND_TIME
* 3. wrapperRecordTimestampType = CREATE_TIME and wrapperRecordTimestamp is not null - Compressed message using CREATE_TIME * 3. wrapperRecordTimestampType = CREATE_TIME and wrapperRecordTimestamp is not null - Compressed message using CREATE_TIME
*
* @return the timestamp as determined above
*/ */
public long timestamp() { public long timestamp() {
if (magic() == MAGIC_VALUE_V0) if (magic() == MAGIC_VALUE_V0)
@ -349,6 +243,8 @@ public final class Record {
/** /**
* The timestamp of the message. * The timestamp of the message.
* @return the timstamp type or {@link TimestampType#NO_TIMESTAMP_TYPE} if the magic is 0 or the message has
* been up-converted.
*/ */
public TimestampType timestampType() { public TimestampType timestampType() {
if (magic() == 0) if (magic() == 0)
@ -366,36 +262,30 @@ public final class Record {
/** /**
* A ByteBuffer containing the value of this record * A ByteBuffer containing the value of this record
* @return the value or null if the value for this record is null
*/ */
public ByteBuffer value() { public ByteBuffer value() {
return sliceDelimited(valueSizeOffset()); return Utils.sizeDelimited(buffer, valueSizeOffset());
} }
/** /**
* A ByteBuffer containing the message key * A ByteBuffer containing the message key
* @return the buffer or null if the key for this record is null
*/ */
public ByteBuffer key() { public ByteBuffer key() {
if (magic() == MAGIC_VALUE_V0) if (magic() == MAGIC_VALUE_V0)
return sliceDelimited(KEY_SIZE_OFFSET_V0); return Utils.sizeDelimited(buffer, KEY_SIZE_OFFSET_V0);
else else
return sliceDelimited(KEY_SIZE_OFFSET_V1); return Utils.sizeDelimited(buffer, KEY_SIZE_OFFSET_V1);
} }
/** /**
* Read a size-delimited byte buffer starting at the given offset * Get the underlying buffer backing this record instance.
*
* @return the buffer
*/ */
private ByteBuffer sliceDelimited(int start) { public ByteBuffer buffer() {
int size = buffer.getInt(start); return this.buffer;
if (size < 0) {
return null;
} else {
ByteBuffer b = buffer.duplicate();
b.position(start + 4);
b = b.slice();
b.limit(size);
b.rewind();
return b;
}
} }
public String toString() { public String toString() {
@ -434,4 +324,316 @@ public final class Record {
return buffer.hashCode(); return buffer.hashCode();
} }
/**
* Get the size of this record if converted to the given format.
*
* @param toMagic The target magic version to convert to
* @return The size in bytes after conversion
*/
public int convertedSize(byte toMagic) {
return recordSize(toMagic, Math.max(0, keySize()), Math.max(0, valueSize()));
}
/**
* Convert this record to another message format.
*
* @param toMagic The target magic version to convert to
* @return A new record instance with a freshly allocated ByteBuffer.
*/
public Record convert(byte toMagic) {
if (toMagic == magic())
return this;
ByteBuffer buffer = ByteBuffer.allocate(convertedSize(toMagic));
TimestampType timestampType = wrapperRecordTimestampType != null ?
wrapperRecordTimestampType : TimestampType.forAttributes(attributes());
convertTo(buffer, toMagic, timestamp(), timestampType);
buffer.rewind();
return new Record(buffer);
}
private void convertTo(ByteBuffer buffer, byte toMagic, long timestamp, TimestampType timestampType) {
if (compressionType() != CompressionType.NONE)
throw new IllegalArgumentException("Cannot use convertTo for deep conversion");
write(buffer, toMagic, timestamp, key(), value(), CompressionType.NONE, timestampType);
}
/**
* Convert this record to another message format and write the converted data to the provided outputs stream.
*
* @param out The output stream to write the converted data to
* @param toMagic The target magic version for conversion
* @param timestamp The timestamp to use in the converted record (for up-conversion)
* @param timestampType The timestamp type to use in the converted record (for up-conversion)
* @throws IOException for any IO errors writing the converted record.
*/
public void convertTo(DataOutputStream out, byte toMagic, long timestamp, TimestampType timestampType) throws IOException {
if (compressionType() != CompressionType.NONE)
throw new IllegalArgumentException("Cannot use convertTo for deep conversion");
write(out, toMagic, timestamp, key(), value(), CompressionType.NONE, timestampType);
}
/**
* Create a new record instance. If the record's compression type is not none, then
* its value payload should be already compressed with the specified type; the constructor
* would always write the value payload as is and will not do the compression itself.
*
* @param magic The magic value to use
* @param timestamp The timestamp of the record
* @param key The key of the record (null, if none)
* @param value The record value
* @param compressionType The compression type used on the contents of the record (if any)
* @param timestampType The timestamp type to be used for this record
*/
public static Record create(byte magic,
long timestamp,
byte[] key,
byte[] value,
CompressionType compressionType,
TimestampType timestampType) {
int keySize = key == null ? 0 : key.length;
int valueSize = value == null ? 0 : value.length;
ByteBuffer buffer = ByteBuffer.allocate(recordSize(magic, keySize, valueSize));
write(buffer, magic, timestamp, wrapNullable(key), wrapNullable(value), compressionType, timestampType);
buffer.rewind();
return new Record(buffer);
}
public static Record create(long timestamp, byte[] key, byte[] value) {
return create(CURRENT_MAGIC_VALUE, timestamp, key, value, CompressionType.NONE, TimestampType.CREATE_TIME);
}
public static Record create(byte magic, long timestamp, byte[] key, byte[] value) {
return create(magic, timestamp, key, value, CompressionType.NONE, TimestampType.CREATE_TIME);
}
public static Record create(byte magic, TimestampType timestampType, long timestamp, byte[] key, byte[] value) {
return create(magic, timestamp, key, value, CompressionType.NONE, timestampType);
}
public static Record create(byte magic, long timestamp, byte[] value) {
return create(magic, timestamp, null, value, CompressionType.NONE, TimestampType.CREATE_TIME);
}
public static Record create(byte magic, byte[] key, byte[] value) {
return create(magic, NO_TIMESTAMP, key, value);
}
public static Record create(byte[] key, byte[] value) {
return create(NO_TIMESTAMP, key, value);
}
public static Record create(byte[] value) {
return create(CURRENT_MAGIC_VALUE, NO_TIMESTAMP, null, value, CompressionType.NONE, TimestampType.CREATE_TIME);
}
/**
* Write the header for a compressed record set in-place (i.e. assuming the compressed record data has already
* been written at the value offset in a wrapped record). This lets you dynamically create a compressed message
* set, and then go back later and fill in its size and CRC, which saves the need for copying to another buffer.
*
* @param buffer The buffer containing the compressed record data positioned at the first offset of the
* @param magic The magic value of the record set
* @param recordSize The size of the record (including record overhead)
* @param timestamp The timestamp of the wrapper record
* @param compressionType The compression type used
* @param timestampType The timestamp type of the wrapper record
*/
public static void writeCompressedRecordHeader(ByteBuffer buffer,
byte magic,
int recordSize,
long timestamp,
CompressionType compressionType,
TimestampType timestampType) {
int recordPosition = buffer.position();
int valueSize = recordSize - recordOverhead(magic);
// write the record header with a null value (the key is always null for the wrapper)
write(buffer, magic, timestamp, null, null, compressionType, timestampType);
// now fill in the value size
buffer.putInt(recordPosition + keyOffset(magic), valueSize);
// compute and fill the crc from the beginning of the message
long crc = Utils.computeChecksum(buffer, recordPosition + MAGIC_OFFSET, recordSize - MAGIC_OFFSET);
Utils.writeUnsignedInt(buffer, recordPosition + CRC_OFFSET, crc);
}
private static void write(ByteBuffer buffer,
byte magic,
long timestamp,
ByteBuffer key,
ByteBuffer value,
CompressionType compressionType,
TimestampType timestampType) {
try {
ByteBufferOutputStream out = new ByteBufferOutputStream(buffer);
write(out, magic, timestamp, key, value, compressionType, timestampType);
} catch (IOException e) {
throw new KafkaException(e);
}
}
/**
* Write the record data with the given compression type and return the computed crc.
*
* @param out The output stream to write to
* @param magic The magic value to be used
* @param timestamp The timestamp of the record
* @param key The record key
* @param value The record value
* @param compressionType The compression type
* @param timestampType The timestamp type
* @return the computed CRC for this record.
* @throws IOException for any IO errors writing to the output stream.
*/
public static long write(DataOutputStream out,
byte magic,
long timestamp,
byte[] key,
byte[] value,
CompressionType compressionType,
TimestampType timestampType) throws IOException {
return write(out, magic, timestamp, wrapNullable(key), wrapNullable(value), compressionType, timestampType);
}
private static long write(DataOutputStream out,
byte magic,
long timestamp,
ByteBuffer key,
ByteBuffer value,
CompressionType compressionType,
TimestampType timestampType) throws IOException {
byte attributes = computeAttributes(magic, compressionType, timestampType);
long crc = computeChecksum(magic, attributes, timestamp, key, value);
write(out, magic, crc, attributes, timestamp, key, value);
return crc;
}
/**
* Write a record using raw fields (without validation). This should only be used in testing.
*/
public static void write(DataOutputStream out,
byte magic,
long crc,
byte attributes,
long timestamp,
byte[] key,
byte[] value) throws IOException {
write(out, magic, crc, attributes, timestamp, wrapNullable(key), wrapNullable(value));
}
// Write a record to the buffer, if the record's compression type is none, then
// its value payload should be already compressed with the specified type
private static void write(DataOutputStream out,
byte magic,
long crc,
byte attributes,
long timestamp,
ByteBuffer key,
ByteBuffer value) throws IOException {
if (magic != MAGIC_VALUE_V0 && magic != MAGIC_VALUE_V1)
throw new IllegalArgumentException("Invalid magic value " + magic);
if (timestamp < 0 && timestamp != NO_TIMESTAMP)
throw new IllegalArgumentException("Invalid message timestamp " + timestamp);
// write crc
out.writeInt((int) (crc & 0xffffffffL));
// write magic value
out.writeByte(magic);
// write attributes
out.writeByte(attributes);
// maybe write timestamp
if (magic > 0)
out.writeLong(timestamp);
// write the key
if (key == null) {
out.writeInt(-1);
} else {
int size = key.remaining();
out.writeInt(size);
out.write(key.array(), key.arrayOffset(), size);
}
// write the value
if (value == null) {
out.writeInt(-1);
} else {
int size = value.remaining();
out.writeInt(size);
out.write(value.array(), value.arrayOffset(), size);
}
}
public static int recordSize(byte[] key, byte[] value) {
return recordSize(CURRENT_MAGIC_VALUE, key, value);
}
public static int recordSize(byte magic, byte[] key, byte[] value) {
return recordSize(magic, key == null ? 0 : key.length, value == null ? 0 : value.length);
}
private static int recordSize(byte magic, int keySize, int valueSize) {
return recordOverhead(magic) + keySize + valueSize;
}
// visible only for testing
public static byte computeAttributes(byte magic, CompressionType type, TimestampType timestampType) {
byte attributes = 0;
if (type.id > 0)
attributes = (byte) (attributes | (COMPRESSION_CODEC_MASK & type.id));
if (magic > 0)
return timestampType.updateAttributes(attributes);
return attributes;
}
// visible only for testing
public static long computeChecksum(byte magic, byte attributes, long timestamp, byte[] key, byte[] value) {
return computeChecksum(magic, attributes, timestamp, wrapNullable(key), wrapNullable(value));
}
/**
* Compute the checksum of the record from the attributes, key and value payloads
*/
private static long computeChecksum(byte magic, byte attributes, long timestamp, ByteBuffer key, ByteBuffer value) {
Crc32 crc = new Crc32();
crc.update(magic);
crc.update(attributes);
if (magic > 0)
crc.updateLong(timestamp);
// update for the key
if (key == null) {
crc.updateInt(-1);
} else {
int size = key.remaining();
crc.updateInt(size);
crc.update(key.array(), key.arrayOffset(), size);
}
// update for the value
if (value == null) {
crc.updateInt(-1);
} else {
int size = value.remaining();
crc.updateInt(size);
crc.update(value.array(), value.arrayOffset(), size);
}
return crc.getValue();
}
public static int recordOverhead(byte magic) {
if (magic == 0)
return RECORD_OVERHEAD_V0;
return RECORD_OVERHEAD_V1;
}
private static int keyOffset(byte magic) {
if (magic == 0)
return KEY_OFFSET_V0;
return KEY_OFFSET_V1;
}
} }

View File

@ -18,32 +18,74 @@ package org.apache.kafka.common.record;
import java.io.IOException; import java.io.IOException;
import java.nio.channels.GatheringByteChannel; import java.nio.channels.GatheringByteChannel;
import java.util.Iterator;
/** /**
* A binary format which consists of a 4 byte size, an 8 byte offset, and the record bytes. See {@link MemoryRecords} * Interface for accessing the records contained in a log. The log itself is represented as a sequence of log entries.
* for the in-memory representation. * Each log entry consists of an 8 byte offset, a 4 byte record size, and a "shallow" {@link Record record}.
* If the entry is not compressed, then each entry will have only the shallow record contained inside it. If it is
* compressed, the entry contains "deep" records, which are packed into the value field of the shallow record. To iterate
* over the shallow records, use {@link #shallowIterator()}; for the deep records, use {@link #deepIterator()}. Note
* that the deep iterator handles both compressed and non-compressed entries: if the entry is not compressed, the
* shallow record is returned; otherwise, the shallow record is decompressed and the deep entries are returned.
* See {@link MemoryRecords} for the in-memory representation and {@link FileRecords} for the on-disk representation.
*/ */
public interface Records extends Iterable<LogEntry> { public interface Records {
int SIZE_LENGTH = 4; int OFFSET_OFFSET = 0;
int OFFSET_LENGTH = 8; int OFFSET_LENGTH = 8;
int LOG_OVERHEAD = SIZE_LENGTH + OFFSET_LENGTH; int SIZE_OFFSET = OFFSET_OFFSET + OFFSET_LENGTH;
int SIZE_LENGTH = 4;
int LOG_OVERHEAD = SIZE_OFFSET + SIZE_LENGTH;
/** /**
* The size of these records in bytes * The size of these records in bytes.
* @return The size in bytes * @return The size in bytes of the records
*/ */
int sizeInBytes(); int sizeInBytes();
/** /**
* Write the messages in this set to the given channel starting at the given offset byte. * Write the contents of this buffer to a channel.
* @param channel The channel to write to * @param channel The channel to write to
* @param position The position within this record set to begin writing from * @param position The position in the buffer to write from
* @param length The number of bytes to write * @param length The number of bytes to write
* @return The number of bytes written to the channel (which may be fewer than requested) * @return The number of bytes written
* @throws IOException For any IO errors copying the * @throws IOException For any IO errors
*/ */
long writeTo(GatheringByteChannel channel, long position, int length) throws IOException; long writeTo(GatheringByteChannel channel, long position, int length) throws IOException;
/**
* Get the shallow log entries in this log buffer. Note that the signature allows subclasses
* to return a more specific log entry type. This enables optimizations such as in-place offset
* assignment (see {@link ByteBufferLogInputStream.ByteBufferLogEntry}), and partial reading of
* record data (see {@link FileLogInputStream.FileChannelLogEntry#magic()}.
* @return An iterator over the shallow entries of the log
*/
Iterator<? extends LogEntry> shallowIterator();
/**
* Get the deep log entries (i.e. descend into compressed message sets). For the deep records,
* there are fewer options for optimization since the data must be decompressed before it can be
* returned. Hence there is little advantage in allowing subclasses to return a more specific type
* as we do for {@link #shallowIterator()}.
* @return An iterator over the deep entries of the log
*/
Iterator<LogEntry> deepIterator();
/**
* Check whether all shallow entries in this buffer have a certain magic value.
* @param magic The magic value to check
* @return true if all shallow entries have a matching magic value, false otherwise
*/
boolean hasMatchingShallowMagic(byte magic);
/**
* Convert all entries in this buffer to the format passed as a parameter. Note that this requires
* deep iteration since all of the deep records must also be converted to the desired format.
* @param toMagic The magic value to convert to
* @return A Records (which may or may not be the same instance)
*/
Records toMessageFormat(byte toMagic);
} }

View File

@ -17,6 +17,7 @@
package org.apache.kafka.common.record; package org.apache.kafka.common.record;
import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.errors.CorruptRecordException;
import org.apache.kafka.common.utils.AbstractIterator; import org.apache.kafka.common.utils.AbstractIterator;
import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.utils.Utils;
@ -25,51 +26,58 @@ import java.io.EOFException;
import java.io.IOException; import java.io.IOException;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.ArrayDeque; import java.util.ArrayDeque;
import java.util.Iterator;
/**
* An iterator which handles both the shallow and deep iteration of record sets.
*/
public class RecordsIterator extends AbstractIterator<LogEntry> { public class RecordsIterator extends AbstractIterator<LogEntry> {
private final LogInputStream logStream;
private final boolean shallow; private final boolean shallow;
private final boolean ensureMatchingMagic;
private final int masRecordSize;
private final ShallowRecordsIterator<?> shallowIter;
private DeepRecordsIterator innerIter; private DeepRecordsIterator innerIter;
public RecordsIterator(LogInputStream logStream, boolean shallow) { public RecordsIterator(LogInputStream<?> logInputStream,
this.logStream = logStream; boolean shallow,
boolean ensureMatchingMagic,
int masRecordSize) {
this.shallowIter = new ShallowRecordsIterator<>(logInputStream);
this.shallow = shallow; this.shallow = shallow;
this.ensureMatchingMagic = ensureMatchingMagic;
this.masRecordSize = masRecordSize;
} }
/* /**
* Read the next record from the buffer. * Get a shallow iterator over the given input stream.
* * @param logInputStream The log input stream to read the entries from
* Note that in the compressed message set, each message value size is set as the size of the un-compressed * @param <T> The type of the log entry
* version of the message value, so when we do de-compression allocating an array of the specified size for * @return The shallow iterator.
* reading compressed value data is sufficient.
*/ */
public static <T extends LogEntry> Iterator<T> shallowIterator(LogInputStream<T> logInputStream) {
return new ShallowRecordsIterator<>(logInputStream);
}
@Override @Override
protected LogEntry makeNext() { protected LogEntry makeNext() {
if (innerDone()) { if (innerDone()) {
try { if (!shallowIter.hasNext())
LogEntry entry = logStream.nextEntry();
// No more record to return.
if (entry == null)
return allDone();
// decide whether to go shallow or deep iteration if it is compressed
CompressionType compressionType = entry.record().compressionType();
if (compressionType == CompressionType.NONE || shallow) {
return entry;
} else {
// init the inner iterator with the value payload of the message,
// which will de-compress the payload to a set of messages;
// since we assume nested compression is not allowed, the deep iterator
// would not try to further decompress underlying messages
// There will be at least one element in the inner iterator, so we don't
// need to call hasNext() here.
innerIter = new DeepRecordsIterator(entry);
return innerIter.next();
}
} catch (EOFException e) {
return allDone(); return allDone();
} catch (IOException e) {
throw new KafkaException(e); LogEntry entry = shallowIter.next();
// decide whether to go shallow or deep iteration if it is compressed
if (shallow || !entry.isCompressed()) {
return entry;
} else {
// init the inner iterator with the value payload of the message,
// which will de-compress the payload to a set of messages;
// since we assume nested compression is not allowed, the deep iterator
// would not try to further decompress underlying messages
// There will be at least one element in the inner iterator, so we don't
// need to call hasNext() here.
innerIter = new DeepRecordsIterator(entry, ensureMatchingMagic, masRecordSize);
return innerIter.next();
} }
} else { } else {
return innerIter.next(); return innerIter.next();
@ -80,38 +88,70 @@ public class RecordsIterator extends AbstractIterator<LogEntry> {
return innerIter == null || !innerIter.hasNext(); return innerIter == null || !innerIter.hasNext();
} }
private static class DataLogInputStream implements LogInputStream { private static class DataLogInputStream implements LogInputStream<LogEntry> {
private final DataInputStream stream; private final DataInputStream stream;
protected final int maxMessageSize;
private DataLogInputStream(DataInputStream stream) { DataLogInputStream(DataInputStream stream, int maxMessageSize) {
this.stream = stream; this.stream = stream;
this.maxMessageSize = maxMessageSize;
} }
public LogEntry nextEntry() throws IOException { public LogEntry nextEntry() throws IOException {
long offset = stream.readLong(); try {
int size = stream.readInt(); long offset = stream.readLong();
if (size < 0) int size = stream.readInt();
throw new IllegalStateException("Record with size " + size); if (size < Record.RECORD_OVERHEAD_V0)
throw new CorruptRecordException(String.format("Record size is less than the minimum record overhead (%d)", Record.RECORD_OVERHEAD_V0));
if (size > maxMessageSize)
throw new CorruptRecordException(String.format("Record size exceeds the largest allowable message size (%d).", maxMessageSize));
byte[] recordBuffer = new byte[size]; byte[] recordBuffer = new byte[size];
stream.readFully(recordBuffer, 0, size); stream.readFully(recordBuffer, 0, size);
ByteBuffer buf = ByteBuffer.wrap(recordBuffer); ByteBuffer buf = ByteBuffer.wrap(recordBuffer);
return new LogEntry(offset, new Record(buf)); return LogEntry.create(offset, new Record(buf));
} catch (EOFException e) {
return null;
}
} }
} }
private static class DeepRecordsIterator extends AbstractIterator<LogEntry> { private static class ShallowRecordsIterator<T extends LogEntry> extends AbstractIterator<T> {
private final LogInputStream<T> logStream;
public ShallowRecordsIterator(LogInputStream<T> logStream) {
this.logStream = logStream;
}
@Override
protected T makeNext() {
try {
T entry = logStream.nextEntry();
if (entry == null)
return allDone();
return entry;
} catch (IOException e) {
throw new KafkaException(e);
}
}
}
public static class DeepRecordsIterator extends AbstractIterator<LogEntry> {
private final ArrayDeque<LogEntry> logEntries; private final ArrayDeque<LogEntry> logEntries;
private final long absoluteBaseOffset; private final long absoluteBaseOffset;
private final byte wrapperMagic;
private DeepRecordsIterator(LogEntry entry) { public DeepRecordsIterator(LogEntry wrapperEntry, boolean ensureMatchingMagic, int maxMessageSize) {
CompressionType compressionType = entry.record().compressionType(); Record wrapperRecord = wrapperEntry.record();
ByteBuffer buffer = entry.record().value(); this.wrapperMagic = wrapperRecord.magic();
DataInputStream stream = Compressor.wrapForInput(new ByteBufferInputStream(buffer), compressionType, entry.record().magic());
LogInputStream logStream = new DataLogInputStream(stream);
long wrapperRecordOffset = entry.offset(); CompressionType compressionType = wrapperRecord.compressionType();
long wrapperRecordTimestamp = entry.record().timestamp(); ByteBuffer buffer = wrapperRecord.value();
DataInputStream stream = MemoryRecordsBuilder.wrapForInput(new ByteBufferInputStream(buffer), compressionType, wrapperRecord.magic());
LogInputStream logStream = new DataLogInputStream(stream, maxMessageSize);
long wrapperRecordOffset = wrapperEntry.offset();
long wrapperRecordTimestamp = wrapperRecord.timestamp();
this.logEntries = new ArrayDeque<>(); this.logEntries = new ArrayDeque<>();
// If relative offset is used, we need to decompress the entire message first to compute // If relative offset is used, we need to decompress the entire message first to compute
@ -119,22 +159,27 @@ public class RecordsIterator extends AbstractIterator<LogEntry> {
// do the same for message format version 0 // do the same for message format version 0
try { try {
while (true) { while (true) {
try { LogEntry logEntry = logStream.nextEntry();
LogEntry logEntry = logStream.nextEntry(); if (logEntry == null)
if (entry.record().magic() > Record.MAGIC_VALUE_V0) {
Record recordWithTimestamp = new Record(
logEntry.record().buffer(),
wrapperRecordTimestamp,
entry.record().timestampType()
);
logEntry = new LogEntry(logEntry.offset(), recordWithTimestamp);
}
logEntries.add(logEntry);
} catch (EOFException e) {
break; break;
Record record = logEntry.record();
byte magic = record.magic();
if (ensureMatchingMagic && magic != wrapperMagic)
throw new InvalidRecordException("Compressed message magic does not match wrapper magic");
if (magic > Record.MAGIC_VALUE_V0) {
Record recordWithTimestamp = new Record(
record.buffer(),
wrapperRecordTimestamp,
wrapperRecord.timestampType()
);
logEntry = LogEntry.create(logEntry.offset(), recordWithTimestamp);
} }
logEntries.addLast(logEntry);
} }
if (entry.record().magic() > Record.MAGIC_VALUE_V0) if (wrapperMagic > Record.MAGIC_VALUE_V0)
this.absoluteBaseOffset = wrapperRecordOffset - logEntries.getLast().offset(); this.absoluteBaseOffset = wrapperRecordOffset - logEntries.getLast().offset();
else else
this.absoluteBaseOffset = -1; this.absoluteBaseOffset = -1;
@ -155,12 +200,10 @@ public class RecordsIterator extends AbstractIterator<LogEntry> {
// Convert offset to absolute offset if needed. // Convert offset to absolute offset if needed.
if (absoluteBaseOffset >= 0) { if (absoluteBaseOffset >= 0) {
long absoluteOffset = absoluteBaseOffset + entry.offset(); long absoluteOffset = absoluteBaseOffset + entry.offset();
entry = new LogEntry(absoluteOffset, entry.record()); entry = LogEntry.create(absoluteOffset, entry.record());
} }
// decide whether to go shallow or deep iteration if it is compressed if (entry.isCompressed())
CompressionType compression = entry.record().compressionType();
if (compression != CompressionType.NONE)
throw new InvalidRecordException("Inner messages must not be compressed"); throw new InvalidRecordException("Inner messages must not be compressed");
return entry; return entry;

View File

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

View File

@ -265,6 +265,24 @@ public class Utils {
return toArray(buffer, 0, buffer.limit()); return toArray(buffer, 0, buffer.limit());
} }
/**
* Convert a ByteBuffer to a nullable array.
* @param buffer The buffer to convert
* @return The resulting array or null if the buffer is null
*/
public static byte[] toNullableArray(ByteBuffer buffer) {
return buffer == null ? null : toArray(buffer);
}
/**
* Wrap an array as a nullable ByteBuffer.
* @param array The nullable array to wrap
* @return The wrapping ByteBuffer or null if array is null
*/
public static ByteBuffer wrapNullable(byte[] array) {
return array == null ? null : ByteBuffer.wrap(array);
}
/** /**
* Read a byte array from the given offset and size in the buffer * Read a byte array from the given offset and size in the buffer
*/ */
@ -733,4 +751,37 @@ public class Utils {
public static int longHashcode(long value) { public static int longHashcode(long value) {
return (int) (value ^ (value >>> 32)); return (int) (value ^ (value >>> 32));
} }
/**
* Read a size-delimited byte buffer starting at the given offset.
* @param buffer Buffer containing the size and data
* @param start Offset in the buffer to read from
* @return A slice of the buffer containing only the delimited data (excluding the size)
*/
public static ByteBuffer sizeDelimited(ByteBuffer buffer, int start) {
int size = buffer.getInt(start);
if (size < 0) {
return null;
} else {
ByteBuffer b = buffer.duplicate();
b.position(start + 4);
b = b.slice();
b.limit(size);
b.rewind();
return b;
}
}
/**
* Compute the checksum of a range of data
* @param buffer Buffer containing the data to checksum
* @param start Offset in the buffer to read from
* @param size The number of bytes to include
*/
public static long computeChecksum(ByteBuffer buffer, int start, int size) {
Crc32 crc = new Crc32();
crc.update(buffer.array(), buffer.arrayOffset() + start, size);
return crc.getValue();
}
} }

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

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.KafkaMetric;
import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.record.ByteBufferOutputStream;
import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.record.Compressor;
import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.MemoryRecordsBuilder;
import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.Record;
import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.requests.AbstractRequest;
import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.requests.FetchRequest;
import org.apache.kafka.common.requests.FetchResponse; import org.apache.kafka.common.requests.FetchResponse;
@ -93,8 +95,8 @@ public class FetcherTest {
private static final double EPSILON = 0.0001; private static final double EPSILON = 0.0001;
private ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(client, metadata, time, 100, 1000); private ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(client, metadata, time, 100, 1000);
private MemoryRecords records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE); private MemoryRecords records;
private MemoryRecords nextRecords = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE); private MemoryRecords nextRecords;
private Fetcher<byte[], byte[]> fetcher = createFetcher(subscriptions, metrics); private Fetcher<byte[], byte[]> fetcher = createFetcher(subscriptions, metrics);
private Metrics fetcherMetrics = new Metrics(time); private Metrics fetcherMetrics = new Metrics(time);
private Fetcher<byte[], byte[]> fetcherNoAutoReset = createFetcher(subscriptionsNoAutoReset, fetcherMetrics); private Fetcher<byte[], byte[]> fetcherNoAutoReset = createFetcher(subscriptionsNoAutoReset, fetcherMetrics);
@ -104,14 +106,16 @@ public class FetcherTest {
metadata.update(cluster, time.milliseconds()); metadata.update(cluster, time.milliseconds());
client.setNode(node); client.setNode(node);
records.append(1L, 0L, "key".getBytes(), "value-1".getBytes()); MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME);
records.append(2L, 0L, "key".getBytes(), "value-2".getBytes()); builder.append(1L, 0L, "key".getBytes(), "value-1".getBytes());
records.append(3L, 0L, "key".getBytes(), "value-3".getBytes()); builder.append(2L, 0L, "key".getBytes(), "value-2".getBytes());
records.close(); builder.append(3L, 0L, "key".getBytes(), "value-3".getBytes());
records = builder.build();
nextRecords.append(4L, 0L, "key".getBytes(), "value-4".getBytes()); builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME);
nextRecords.append(5L, 0L, "key".getBytes(), "value-5".getBytes()); builder.append(4L, 0L, "key".getBytes(), "value-4".getBytes());
nextRecords.close(); builder.append(5L, 0L, "key".getBytes(), "value-5".getBytes());
nextRecords = builder.build();
} }
@After @After
@ -129,7 +133,7 @@ public class FetcherTest {
assertEquals(1, fetcher.sendFetches()); assertEquals(1, fetcher.sendFetches());
assertFalse(fetcher.hasCompletedFetches()); assertFalse(fetcher.hasCompletedFetches());
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0)); client.prepareResponse(fetchResponse(this.records, Errors.NONE.code(), 100L, 0));
consumerClient.poll(0); consumerClient.poll(0);
assertTrue(fetcher.hasCompletedFetches()); assertTrue(fetcher.hasCompletedFetches());
@ -154,7 +158,7 @@ public class FetcherTest {
assertEquals(1, fetcher.sendFetches()); assertEquals(1, fetcher.sendFetches());
assertFalse(fetcher.hasCompletedFetches()); assertFalse(fetcher.hasCompletedFetches());
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NOT_LEADER_FOR_PARTITION.code(), 100L, 0)); client.prepareResponse(fetchResponse(this.records, Errors.NOT_LEADER_FOR_PARTITION.code(), 100L, 0));
consumerClient.poll(0); consumerClient.poll(0);
assertTrue(fetcher.hasCompletedFetches()); assertTrue(fetcher.hasCompletedFetches());
@ -192,7 +196,7 @@ public class FetcherTest {
subscriptions.assignFromUser(singleton(tp)); subscriptions.assignFromUser(singleton(tp));
subscriptions.seek(tp, 1); subscriptions.seek(tp, 1);
client.prepareResponse(matchesOffset(tp, 1), fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0)); client.prepareResponse(matchesOffset(tp, 1), fetchResponse(this.records, Errors.NONE.code(), 100L, 0));
assertEquals(1, fetcher.sendFetches()); assertEquals(1, fetcher.sendFetches());
consumerClient.poll(0); consumerClient.poll(0);
@ -206,29 +210,30 @@ public class FetcherTest {
} }
@Test @Test
public void testParseInvalidRecord() { public void testParseInvalidRecord() throws Exception {
ByteBuffer buffer = ByteBuffer.allocate(1024); ByteBuffer buffer = ByteBuffer.allocate(1024);
Compressor compressor = new Compressor(buffer, CompressionType.NONE); ByteBufferOutputStream out = new ByteBufferOutputStream(buffer);
byte magic = Record.CURRENT_MAGIC_VALUE;
byte[] key = "foo".getBytes(); byte[] key = "foo".getBytes();
byte[] value = "baz".getBytes(); byte[] value = "baz".getBytes();
long offset = 0; long offset = 0;
long timestamp = 500L; long timestamp = 500L;
int size = Record.recordSize(key, value); int size = Record.recordSize(key, value);
long crc = Record.computeChecksum(timestamp, key, value, CompressionType.NONE, 0, -1); byte attributes = Record.computeAttributes(magic, CompressionType.NONE, TimestampType.CREATE_TIME);
long crc = Record.computeChecksum(magic, attributes, timestamp, key, value);
// write one valid record // write one valid record
compressor.putLong(offset); out.writeLong(offset);
compressor.putInt(size); out.writeInt(size);
Record.write(compressor, crc, Record.computeAttributes(CompressionType.NONE), timestamp, key, value, 0, -1); Record.write(out, magic, crc, Record.computeAttributes(magic, CompressionType.NONE, TimestampType.CREATE_TIME), timestamp, key, value);
// and one invalid record (note the crc) // and one invalid record (note the crc)
compressor.putLong(offset); out.writeLong(offset);
compressor.putInt(size); out.writeInt(size);
Record.write(compressor, crc + 1, Record.computeAttributes(CompressionType.NONE), timestamp, key, value, 0, -1); Record.write(out, magic, crc + 1, Record.computeAttributes(magic, CompressionType.NONE, TimestampType.CREATE_TIME), timestamp, key, value);
compressor.close();
buffer.flip(); buffer.flip();
subscriptions.assignFromUser(singleton(tp)); subscriptions.assignFromUser(singleton(tp));
@ -236,7 +241,7 @@ public class FetcherTest {
// normal fetch // normal fetch
assertEquals(1, fetcher.sendFetches()); assertEquals(1, fetcher.sendFetches());
client.prepareResponse(fetchResponse(buffer, Errors.NONE.code(), 100L, 0)); client.prepareResponse(fetchResponse(MemoryRecords.readableRecords(buffer), Errors.NONE.code(), 100L, 0));
consumerClient.poll(0); consumerClient.poll(0);
try { try {
fetcher.fetchedRecords(); fetcher.fetchedRecords();
@ -255,8 +260,8 @@ public class FetcherTest {
subscriptions.assignFromUser(singleton(tp)); subscriptions.assignFromUser(singleton(tp));
subscriptions.seek(tp, 1); subscriptions.seek(tp, 1);
client.prepareResponse(matchesOffset(tp, 1), fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0)); client.prepareResponse(matchesOffset(tp, 1), fetchResponse(this.records, Errors.NONE.code(), 100L, 0));
client.prepareResponse(matchesOffset(tp, 4), fetchResponse(this.nextRecords.buffer(), Errors.NONE.code(), 100L, 0)); client.prepareResponse(matchesOffset(tp, 4), fetchResponse(this.nextRecords, Errors.NONE.code(), 100L, 0));
assertEquals(1, fetcher.sendFetches()); assertEquals(1, fetcher.sendFetches());
consumerClient.poll(0); consumerClient.poll(0);
@ -287,11 +292,11 @@ public class FetcherTest {
// if we are fetching from a compacted topic, there may be gaps in the returned records // if we are fetching from a compacted topic, there may be gaps in the returned records
// this test verifies the fetcher updates the current fetched/consumed positions correctly for this case // this test verifies the fetcher updates the current fetched/consumed positions correctly for this case
MemoryRecords records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE); MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME);
records.append(15L, 0L, "key".getBytes(), "value-1".getBytes()); builder.append(15L, 0L, "key".getBytes(), "value-1".getBytes());
records.append(20L, 0L, "key".getBytes(), "value-2".getBytes()); builder.append(20L, 0L, "key".getBytes(), "value-2".getBytes());
records.append(30L, 0L, "key".getBytes(), "value-3".getBytes()); builder.append(30L, 0L, "key".getBytes(), "value-3".getBytes());
records.close(); MemoryRecords records = builder.build();
List<ConsumerRecord<byte[], byte[]>> consumerRecords; List<ConsumerRecord<byte[], byte[]>> consumerRecords;
subscriptions.assignFromUser(singleton(tp)); subscriptions.assignFromUser(singleton(tp));
@ -299,7 +304,7 @@ public class FetcherTest {
// normal fetch // normal fetch
assertEquals(1, fetcher.sendFetches()); assertEquals(1, fetcher.sendFetches());
client.prepareResponse(fetchResponse(records.buffer(), Errors.NONE.code(), 100L, 0)); client.prepareResponse(fetchResponse(records, Errors.NONE.code(), 100L, 0));
consumerClient.poll(0); consumerClient.poll(0);
consumerRecords = fetcher.fetchedRecords().get(tp); consumerRecords = fetcher.fetchedRecords().get(tp);
assertEquals(3, consumerRecords.size()); assertEquals(3, consumerRecords.size());
@ -317,7 +322,7 @@ public class FetcherTest {
// resize the limit of the buffer to pretend it is only fetch-size large // resize the limit of the buffer to pretend it is only fetch-size large
assertEquals(1, fetcher.sendFetches()); assertEquals(1, fetcher.sendFetches());
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.TOPIC_AUTHORIZATION_FAILED.code(), 100L, 0)); client.prepareResponse(fetchResponse(this.records, Errors.TOPIC_AUTHORIZATION_FAILED.code(), 100L, 0));
consumerClient.poll(0); consumerClient.poll(0);
try { try {
fetcher.fetchedRecords(); fetcher.fetchedRecords();
@ -337,7 +342,7 @@ public class FetcherTest {
// Now the rebalance happens and fetch positions are cleared // Now the rebalance happens and fetch positions are cleared
subscriptions.assignFromSubscribed(singleton(tp)); subscriptions.assignFromSubscribed(singleton(tp));
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0)); client.prepareResponse(fetchResponse(this.records, Errors.NONE.code(), 100L, 0));
consumerClient.poll(0); consumerClient.poll(0);
// The active fetch should be ignored since its position is no longer valid // The active fetch should be ignored since its position is no longer valid
@ -352,7 +357,7 @@ public class FetcherTest {
assertEquals(1, fetcher.sendFetches()); assertEquals(1, fetcher.sendFetches());
subscriptions.pause(tp); subscriptions.pause(tp);
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0)); client.prepareResponse(fetchResponse(this.records, Errors.NONE.code(), 100L, 0));
consumerClient.poll(0); consumerClient.poll(0);
assertNull(fetcher.fetchedRecords().get(tp)); assertNull(fetcher.fetchedRecords().get(tp));
} }
@ -373,7 +378,7 @@ public class FetcherTest {
subscriptions.seek(tp, 0); subscriptions.seek(tp, 0);
assertEquals(1, fetcher.sendFetches()); assertEquals(1, fetcher.sendFetches());
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NOT_LEADER_FOR_PARTITION.code(), 100L, 0)); client.prepareResponse(fetchResponse(this.records, Errors.NOT_LEADER_FOR_PARTITION.code(), 100L, 0));
consumerClient.poll(0); consumerClient.poll(0);
assertEquals(0, fetcher.fetchedRecords().size()); assertEquals(0, fetcher.fetchedRecords().size());
assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds())); assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds()));
@ -385,7 +390,7 @@ public class FetcherTest {
subscriptions.seek(tp, 0); subscriptions.seek(tp, 0);
assertEquals(1, fetcher.sendFetches()); assertEquals(1, fetcher.sendFetches());
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), 100L, 0)); client.prepareResponse(fetchResponse(this.records, Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), 100L, 0));
consumerClient.poll(0); consumerClient.poll(0);
assertEquals(0, fetcher.fetchedRecords().size()); assertEquals(0, fetcher.fetchedRecords().size());
assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds())); assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds()));
@ -397,7 +402,7 @@ public class FetcherTest {
subscriptions.seek(tp, 0); subscriptions.seek(tp, 0);
assertEquals(1, fetcher.sendFetches()); assertEquals(1, fetcher.sendFetches());
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0)); client.prepareResponse(fetchResponse(this.records, Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0));
consumerClient.poll(0); consumerClient.poll(0);
assertEquals(0, fetcher.fetchedRecords().size()); assertEquals(0, fetcher.fetchedRecords().size());
assertTrue(subscriptions.isOffsetResetNeeded(tp)); assertTrue(subscriptions.isOffsetResetNeeded(tp));
@ -412,7 +417,7 @@ public class FetcherTest {
subscriptions.seek(tp, 0); subscriptions.seek(tp, 0);
assertEquals(1, fetcher.sendFetches()); assertEquals(1, fetcher.sendFetches());
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0)); client.prepareResponse(fetchResponse(this.records, Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0));
subscriptions.seek(tp, 1); subscriptions.seek(tp, 1);
consumerClient.poll(0); consumerClient.poll(0);
assertEquals(0, fetcher.fetchedRecords().size()); assertEquals(0, fetcher.fetchedRecords().size());
@ -426,7 +431,7 @@ public class FetcherTest {
subscriptionsNoAutoReset.seek(tp, 0); subscriptionsNoAutoReset.seek(tp, 0);
assertTrue(fetcherNoAutoReset.sendFetches() > 0); assertTrue(fetcherNoAutoReset.sendFetches() > 0);
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0)); client.prepareResponse(fetchResponse(this.records, Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0));
consumerClient.poll(0); consumerClient.poll(0);
assertFalse(subscriptionsNoAutoReset.isOffsetResetNeeded(tp)); assertFalse(subscriptionsNoAutoReset.isOffsetResetNeeded(tp));
subscriptionsNoAutoReset.seek(tp, 2); subscriptionsNoAutoReset.seek(tp, 2);
@ -439,7 +444,7 @@ public class FetcherTest {
subscriptionsNoAutoReset.seek(tp, 0); subscriptionsNoAutoReset.seek(tp, 0);
fetcherNoAutoReset.sendFetches(); fetcherNoAutoReset.sendFetches();
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0)); client.prepareResponse(fetchResponse(this.records, Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0));
consumerClient.poll(0); consumerClient.poll(0);
assertFalse(subscriptionsNoAutoReset.isOffsetResetNeeded(tp)); assertFalse(subscriptionsNoAutoReset.isOffsetResetNeeded(tp));
@ -459,7 +464,7 @@ public class FetcherTest {
subscriptions.seek(tp, 0); subscriptions.seek(tp, 0);
assertEquals(1, fetcher.sendFetches()); assertEquals(1, fetcher.sendFetches());
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0), true); client.prepareResponse(fetchResponse(this.records, Errors.NONE.code(), 100L, 0), true);
consumerClient.poll(0); consumerClient.poll(0);
assertEquals(0, fetcher.fetchedRecords().size()); assertEquals(0, fetcher.fetchedRecords().size());
@ -611,14 +616,14 @@ public class FetcherTest {
// We need to make sure the message offset grows. Otherwise they will be considered as already consumed // We need to make sure the message offset grows. Otherwise they will be considered as already consumed
// and filtered out by consumer. // and filtered out by consumer.
if (i > 1) { if (i > 1) {
this.records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE); MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME);
for (int v = 0; v < 3; v++) { for (int v = 0; v < 3; v++) {
this.records.append((long) i * 3 + v, Record.NO_TIMESTAMP, "key".getBytes(), String.format("value-%d", v).getBytes()); builder.append((long) i * 3 + v, Record.NO_TIMESTAMP, "key".getBytes(), String.format("value-%d", v).getBytes());
} }
this.records.close(); this.records = builder.build();
} }
assertEquals(1, fetcher.sendFetches()); assertEquals(1, fetcher.sendFetches());
client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 100 * i)); client.prepareResponse(fetchResponse(this.records, Errors.NONE.code(), 100L, 100 * i));
consumerClient.poll(0); consumerClient.poll(0);
records = fetcher.fetchedRecords().get(tp); records = fetcher.fetchedRecords().get(tp);
assertEquals(3, records.size()); assertEquals(3, records.size());
@ -722,8 +727,7 @@ public class FetcherTest {
return new ListOffsetResponse(allPartitionData, 1); return new ListOffsetResponse(allPartitionData, 1);
} }
private FetchResponse fetchResponse(ByteBuffer buffer, short error, long hw, int throttleTime) { private FetchResponse fetchResponse(MemoryRecords records, short error, long hw, int throttleTime) {
MemoryRecords records = MemoryRecords.readableRecords(buffer);
return new FetchResponse(Collections.singletonMap(tp, new FetchResponse.PartitionData(error, hw, records)), throttleTime); return new FetchResponse(Collections.singletonMap(tp, new FetchResponse.PartitionData(error, hw, records)), throttleTime);
} }

View File

@ -12,23 +12,6 @@
*/ */
package org.apache.kafka.clients.producer.internals; package org.apache.kafka.clients.producer.internals;
import static java.util.Arrays.asList;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Deque;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.Callback;
import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Cluster;
@ -45,6 +28,23 @@ import org.apache.kafka.common.utils.Time;
import org.junit.After; import org.junit.After;
import org.junit.Test; import org.junit.Test;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Deque;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import static java.util.Arrays.asList;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
public class RecordAccumulatorTest { public class RecordAccumulatorTest {
private String topic = "test"; private String topic = "test";
@ -84,7 +84,7 @@ public class RecordAccumulatorTest {
accum.append(tp1, 0L, key, value, null, maxBlockTimeMs); accum.append(tp1, 0L, key, value, null, maxBlockTimeMs);
Deque<RecordBatch> partitionBatches = accum.batches().get(tp1); Deque<RecordBatch> partitionBatches = accum.batches().get(tp1);
assertEquals(1, partitionBatches.size()); assertEquals(1, partitionBatches.size());
assertTrue(partitionBatches.peekFirst().records.isWritable()); assertTrue(partitionBatches.peekFirst().isWritable());
assertEquals("No partitions should be ready.", 0, accum.ready(cluster, now).readyNodes.size()); assertEquals("No partitions should be ready.", 0, accum.ready(cluster, now).readyNodes.size());
} }
@ -93,15 +93,15 @@ public class RecordAccumulatorTest {
Deque<RecordBatch> partitionBatches = accum.batches().get(tp1); Deque<RecordBatch> partitionBatches = accum.batches().get(tp1);
assertEquals(2, partitionBatches.size()); assertEquals(2, partitionBatches.size());
Iterator<RecordBatch> partitionBatchesIterator = partitionBatches.iterator(); Iterator<RecordBatch> partitionBatchesIterator = partitionBatches.iterator();
assertFalse(partitionBatchesIterator.next().records.isWritable()); assertFalse(partitionBatchesIterator.next().isWritable());
assertTrue(partitionBatchesIterator.next().records.isWritable()); assertTrue(partitionBatchesIterator.next().isWritable());
assertEquals("Our partition's leader should be ready", Collections.singleton(node1), accum.ready(cluster, time.milliseconds()).readyNodes); assertEquals("Our partition's leader should be ready", Collections.singleton(node1), accum.ready(cluster, time.milliseconds()).readyNodes);
List<RecordBatch> batches = accum.drain(cluster, Collections.singleton(node1), Integer.MAX_VALUE, 0).get(node1.id()); List<RecordBatch> batches = accum.drain(cluster, Collections.singleton(node1), Integer.MAX_VALUE, 0).get(node1.id());
assertEquals(1, batches.size()); assertEquals(1, batches.size());
RecordBatch batch = batches.get(0); RecordBatch batch = batches.get(0);
Iterator<LogEntry> iter = batch.records.iterator(); Iterator<LogEntry> iter = batch.records().deepIterator();
for (int i = 0; i < appends; i++) { for (int i = 0; i < appends; i++) {
LogEntry entry = iter.next(); LogEntry entry = iter.next();
assertEquals("Keys should match", ByteBuffer.wrap(key), entry.record().key()); assertEquals("Keys should match", ByteBuffer.wrap(key), entry.record().key());
@ -130,7 +130,7 @@ public class RecordAccumulatorTest {
assertEquals(1, batches.size()); assertEquals(1, batches.size());
RecordBatch batch = batches.get(0); RecordBatch batch = batches.get(0);
Iterator<LogEntry> iter = batch.records.iterator(); Iterator<LogEntry> iter = batch.records().deepIterator();
LogEntry entry = iter.next(); LogEntry entry = iter.next();
assertEquals("Keys should match", ByteBuffer.wrap(key), entry.record().key()); assertEquals("Keys should match", ByteBuffer.wrap(key), entry.record().key());
assertEquals("Values should match", ByteBuffer.wrap(value), entry.record().value()); assertEquals("Values should match", ByteBuffer.wrap(value), entry.record().value());
@ -159,7 +159,7 @@ public class RecordAccumulatorTest {
final int msgs = 10000; final int msgs = 10000;
final int numParts = 2; final int numParts = 2;
final RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, 0L, 100L, metrics, time); final RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, 0L, 100L, metrics, time);
List<Thread> threads = new ArrayList<Thread>(); List<Thread> threads = new ArrayList<>();
for (int i = 0; i < numThreads; i++) { for (int i = 0; i < numThreads; i++) {
threads.add(new Thread() { threads.add(new Thread() {
public void run() { public void run() {
@ -182,8 +182,11 @@ public class RecordAccumulatorTest {
List<RecordBatch> batches = accum.drain(cluster, nodes, 5 * 1024, 0).get(node1.id()); List<RecordBatch> batches = accum.drain(cluster, nodes, 5 * 1024, 0).get(node1.id());
if (batches != null) { if (batches != null) {
for (RecordBatch batch : batches) { for (RecordBatch batch : batches) {
for (LogEntry entry : batch.records) Iterator<LogEntry> deepEntries = batch.records().deepIterator();
while (deepEntries.hasNext()) {
deepEntries.next();
read++; read++;
}
accum.deallocate(batch); accum.deallocate(batch);
} }
} }

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; package org.apache.kafka.common.record;
import static org.apache.kafka.common.utils.Utils.toArray; import org.apache.kafka.test.TestUtils;
import static org.junit.Assert.assertEquals; import org.junit.Test;
import static org.junit.Assert.assertFalse; import org.junit.runner.RunWith;
import static org.junit.Assert.assertTrue; import org.junit.runners.Parameterized;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import org.junit.Test; import static java.util.Arrays.asList;
import org.junit.runner.RunWith; import static org.apache.kafka.common.utils.Utils.toNullableArray;
import org.junit.runners.Parameterized; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
@RunWith(value = Parameterized.class) @RunWith(value = Parameterized.class)
public class MemoryRecordsTest { public class MemoryRecordsTest {
private CompressionType compression; private CompressionType compression;
private byte magic;
private long firstOffset;
public MemoryRecordsTest(CompressionType compression) { public MemoryRecordsTest(byte magic, long firstOffset, CompressionType compression) {
this.magic = magic;
this.compression = compression; this.compression = compression;
this.firstOffset = firstOffset;
} }
@Test @Test
public void testIterator() { public void testIterator() {
MemoryRecords recs1 = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), compression); MemoryRecordsBuilder builder1 = MemoryRecords.builder(ByteBuffer.allocate(1024), magic, compression, TimestampType.CREATE_TIME, firstOffset);
MemoryRecords recs2 = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), compression); MemoryRecordsBuilder builder2 = MemoryRecords.builder(ByteBuffer.allocate(1024), magic, compression, TimestampType.CREATE_TIME, firstOffset);
List<Record> list = Arrays.asList(new Record(0L, "a".getBytes(), "1".getBytes()), List<Record> list = asList(
new Record(0L, "b".getBytes(), "2".getBytes()), Record.create(magic, 1L, "a".getBytes(), "1".getBytes()),
new Record(0L, "c".getBytes(), "3".getBytes())); Record.create(magic, 2L, "b".getBytes(), "2".getBytes()),
Record.create(magic, 3L, "c".getBytes(), "3".getBytes()),
Record.create(magic, 4L, null, "4".getBytes()),
Record.create(magic, 5L, "e".getBytes(), null),
Record.create(magic, 6L, null, null));
for (int i = 0; i < list.size(); i++) { for (int i = 0; i < list.size(); i++) {
Record r = list.get(i); Record r = list.get(i);
recs1.append(i, r); builder1.append(firstOffset + i, r);
recs2.append(i, 0L, toArray(r.key()), toArray(r.value())); builder2.append(firstOffset + i, i + 1, toNullableArray(r.key()), toNullableArray(r.value()));
} }
recs1.close();
recs2.close(); MemoryRecords recs1 = builder1.build();
MemoryRecords recs2 = builder2.build();
for (int iteration = 0; iteration < 2; iteration++) { for (int iteration = 0; iteration < 2; iteration++) {
for (MemoryRecords recs : Arrays.asList(recs1, recs2)) { for (MemoryRecords recs : asList(recs1, recs2)) {
Iterator<LogEntry> iter = recs.iterator(); Iterator<LogEntry> iter = recs.deepIterator();
for (int i = 0; i < list.size(); i++) { for (int i = 0; i < list.size(); i++) {
assertTrue(iter.hasNext()); assertTrue(iter.hasNext());
LogEntry entry = iter.next(); LogEntry entry = iter.next();
assertEquals((long) i, entry.offset()); assertEquals(firstOffset + i, entry.offset());
assertEquals(list.get(i), entry.record()); assertEquals(list.get(i), entry.record());
entry.record().ensureValid(); entry.record().ensureValid();
} }
@ -73,20 +84,145 @@ public class MemoryRecordsTest {
@Test @Test
public void testHasRoomForMethod() { public void testHasRoomForMethod() {
MemoryRecords recs1 = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), compression); MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), magic, compression, TimestampType.CREATE_TIME);
recs1.append(0, new Record(0L, "a".getBytes(), "1".getBytes())); builder.append(0, Record.create(magic, 0L, "a".getBytes(), "1".getBytes()));
assertTrue(recs1.hasRoomFor("b".getBytes(), "2".getBytes())); assertTrue(builder.hasRoomFor("b".getBytes(), "2".getBytes()));
recs1.close(); builder.close();
assertFalse(recs1.hasRoomFor("b".getBytes(), "2".getBytes())); assertFalse(builder.hasRoomFor("b".getBytes(), "2".getBytes()));
}
@Test
public void testFilterTo() {
ByteBuffer buffer = ByteBuffer.allocate(2048);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME);
builder.append(0L, 10L, null, "a".getBytes());
builder.close();
builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME, 1L);
builder.append(1L, 11L, "1".getBytes(), "b".getBytes());
builder.append(2L, 12L, null, "c".getBytes());
builder.close();
builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME, 3L);
builder.append(3L, 13L, null, "d".getBytes());
builder.append(4L, 20L, "4".getBytes(), "e".getBytes());
builder.append(5L, 15L, "5".getBytes(), "f".getBytes());
builder.close();
builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME, 6L);
builder.append(6L, 16L, "6".getBytes(), "g".getBytes());
builder.close();
buffer.flip();
ByteBuffer filtered = ByteBuffer.allocate(2048);
MemoryRecords.FilterResult result = MemoryRecords.readableRecords(buffer).filterTo(new RetainNonNullKeysFilter(), filtered);
filtered.flip();
assertEquals(7, result.messagesRead);
assertEquals(4, result.messagesRetained);
assertEquals(buffer.limit(), result.bytesRead);
assertEquals(filtered.limit(), result.bytesRetained);
if (magic > 0) {
assertEquals(20L, result.maxTimestamp);
if (compression == CompressionType.NONE)
assertEquals(4L, result.shallowOffsetOfMaxTimestamp);
else
assertEquals(5L, result.shallowOffsetOfMaxTimestamp);
}
MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered);
List<ByteBufferLogInputStream.ByteBufferLogEntry> shallowEntries = TestUtils.toList(filteredRecords.shallowIterator());
List<Long> expectedOffsets = compression == CompressionType.NONE ? asList(1L, 4L, 5L, 6L) : asList(1L, 5L, 6L);
assertEquals(expectedOffsets.size(), shallowEntries.size());
for (int i = 0; i < expectedOffsets.size(); i++) {
LogEntry shallowEntry = shallowEntries.get(i);
assertEquals(expectedOffsets.get(i).longValue(), shallowEntry.offset());
assertEquals(magic, shallowEntry.record().magic());
assertEquals(compression, shallowEntry.record().compressionType());
assertEquals(magic == Record.MAGIC_VALUE_V0 ? TimestampType.NO_TIMESTAMP_TYPE : TimestampType.CREATE_TIME,
shallowEntry.record().timestampType());
}
List<LogEntry> deepEntries = TestUtils.toList(filteredRecords.deepIterator());
assertEquals(4, deepEntries.size());
LogEntry first = deepEntries.get(0);
assertEquals(1L, first.offset());
assertEquals(Record.create(magic, 11L, "1".getBytes(), "b".getBytes()), first.record());
LogEntry second = deepEntries.get(1);
assertEquals(4L, second.offset());
assertEquals(Record.create(magic, 20L, "4".getBytes(), "e".getBytes()), second.record());
LogEntry third = deepEntries.get(2);
assertEquals(5L, third.offset());
assertEquals(Record.create(magic, 15L, "5".getBytes(), "f".getBytes()), third.record());
LogEntry fourth = deepEntries.get(3);
assertEquals(6L, fourth.offset());
assertEquals(Record.create(magic, 16L, "6".getBytes(), "g".getBytes()), fourth.record());
}
@Test
public void testFilterToPreservesLogAppendTime() {
long logAppendTime = System.currentTimeMillis();
ByteBuffer buffer = ByteBuffer.allocate(2048);
MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic, compression,
TimestampType.LOG_APPEND_TIME, 0L, logAppendTime);
builder.append(0L, 10L, null, "a".getBytes());
builder.close();
builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.LOG_APPEND_TIME, 1L, logAppendTime);
builder.append(1L, 11L, "1".getBytes(), "b".getBytes());
builder.append(2L, 12L, null, "c".getBytes());
builder.close();
builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.LOG_APPEND_TIME, 3L, logAppendTime);
builder.append(3L, 13L, null, "d".getBytes());
builder.append(4L, 14L, "4".getBytes(), "e".getBytes());
builder.append(5L, 15L, "5".getBytes(), "f".getBytes());
builder.close();
buffer.flip();
ByteBuffer filtered = ByteBuffer.allocate(2048);
MemoryRecords.readableRecords(buffer).filterTo(new RetainNonNullKeysFilter(), filtered);
filtered.flip();
MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered);
List<ByteBufferLogInputStream.ByteBufferLogEntry> shallowEntries = TestUtils.toList(filteredRecords.shallowIterator());
assertEquals(compression == CompressionType.NONE ? 3 : 2, shallowEntries.size());
for (LogEntry shallowEntry : shallowEntries) {
assertEquals(compression, shallowEntry.record().compressionType());
if (magic > Record.MAGIC_VALUE_V0) {
assertEquals(TimestampType.LOG_APPEND_TIME, shallowEntry.record().timestampType());
assertEquals(logAppendTime, shallowEntry.record().timestamp());
}
}
} }
@Parameterized.Parameters @Parameterized.Parameters
public static Collection<Object[]> data() { public static Collection<Object[]> data() {
List<Object[]> values = new ArrayList<Object[]>(); List<Object[]> values = new ArrayList<>();
for (CompressionType type: CompressionType.values()) for (long firstOffset : asList(0L, 57L))
values.add(new Object[] {type}); for (byte magic : asList(Record.MAGIC_VALUE_V0, Record.MAGIC_VALUE_V1))
for (CompressionType type: CompressionType.values())
values.add(new Object[] {magic, firstOffset, type});
return values; return values;
} }
private static class RetainNonNullKeysFilter implements MemoryRecords.LogEntryFilter {
@Override
public boolean shouldRetain(LogEntry entry) {
return entry.record().hasKey();
}
}
} }

View File

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

View File

@ -20,35 +20,29 @@ import org.junit.Test;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
public class SimpleRecordTest { public class SimpleRecordTest {
/* This scenario can happen if the record size field is corrupt and we end up allocating a buffer that is too small */ /* This scenario can happen if the record size field is corrupt and we end up allocating a buffer that is too small */
@Test @Test(expected = InvalidRecordException.class)
public void testIsValidWithTooSmallBuffer() { public void testIsValidWithTooSmallBuffer() {
ByteBuffer buffer = ByteBuffer.allocate(2); ByteBuffer buffer = ByteBuffer.allocate(2);
Record record = new Record(buffer); Record record = new Record(buffer);
assertFalse(record.isValid()); assertFalse(record.isValid());
try { record.ensureValid();
record.ensureValid();
fail("InvalidRecordException should have been thrown");
} catch (InvalidRecordException e) { }
} }
@Test @Test(expected = InvalidRecordException.class)
public void testIsValidWithChecksumMismatch() { public void testIsValidWithChecksumMismatch() {
ByteBuffer buffer = ByteBuffer.allocate(4); ByteBuffer buffer = ByteBuffer.allocate(4);
// set checksum // set checksum
buffer.putInt(2); buffer.putInt(2);
Record record = new Record(buffer); Record record = new Record(buffer);
assertFalse(record.isValid()); assertFalse(record.isValid());
try { record.ensureValid();
record.ensureValid();
fail("InvalidRecordException should have been thrown");
} catch (InvalidRecordException e) { }
} }
@Test @Test
@ -63,4 +57,40 @@ public class SimpleRecordTest {
record.ensureValid(); record.ensureValid();
} }
@Test
public void testConvertFromV0ToV1() {
byte[][] keys = new byte[][] {"a".getBytes(), "".getBytes(), null, "b".getBytes()};
byte[][] values = new byte[][] {"1".getBytes(), "".getBytes(), "2".getBytes(), null};
for (int i = 0; i < keys.length; i++) {
Record record = Record.create(Record.MAGIC_VALUE_V0, Record.NO_TIMESTAMP, keys[i], values[i]);
Record converted = record.convert(Record.MAGIC_VALUE_V1);
assertEquals(Record.MAGIC_VALUE_V1, converted.magic());
assertEquals(Record.NO_TIMESTAMP, converted.timestamp());
assertEquals(record.key(), converted.key());
assertEquals(record.value(), converted.value());
assertTrue(record.isValid());
assertEquals(record.convertedSize(Record.MAGIC_VALUE_V1), converted.sizeInBytes());
}
}
@Test
public void testConvertFromV1ToV0() {
byte[][] keys = new byte[][] {"a".getBytes(), "".getBytes(), null, "b".getBytes()};
byte[][] values = new byte[][] {"1".getBytes(), "".getBytes(), "2".getBytes(), null};
for (int i = 0; i < keys.length; i++) {
Record record = Record.create(Record.MAGIC_VALUE_V1, System.currentTimeMillis(), keys[i], values[i]);
Record converted = record.convert(Record.MAGIC_VALUE_V0);
assertEquals(Record.MAGIC_VALUE_V0, converted.magic());
assertEquals(Record.NO_TIMESTAMP, converted.timestamp());
assertEquals(record.key(), converted.key());
assertEquals(record.value(), converted.value());
assertTrue(record.isValid());
assertEquals(record.convertedSize(Record.MAGIC_VALUE_V0), converted.sizeInBytes());
}
}
} }

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.PartitionInfo;
import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.record.MemoryRecordsBuilder;
import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.Record;
import org.apache.kafka.common.record.Records; import org.apache.kafka.common.record.Records;
import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.utils.Utils;
import javax.xml.bind.DatatypeConverter; import javax.xml.bind.DatatypeConverter;
@ -35,6 +37,7 @@ import java.nio.file.Files;
import java.nio.file.Path; import java.nio.file.Path;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Properties; import java.util.Properties;
@ -46,6 +49,7 @@ import java.util.regex.Pattern;
import static java.util.Arrays.asList; import static java.util.Arrays.asList;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail; import static org.junit.Assert.fail;
@ -185,13 +189,13 @@ public class TestUtils {
public static ByteBuffer partitionRecordsBuffer(final long offset, final CompressionType compressionType, final Record... records) { public static ByteBuffer partitionRecordsBuffer(final long offset, final CompressionType compressionType, final Record... records) {
int bufferSize = 0; int bufferSize = 0;
for (final Record record : records) for (final Record record : records)
bufferSize += Records.LOG_OVERHEAD + record.size(); bufferSize += Records.LOG_OVERHEAD + record.sizeInBytes();
final ByteBuffer buffer = ByteBuffer.allocate(bufferSize); final ByteBuffer buffer = ByteBuffer.allocate(bufferSize);
final MemoryRecords memoryRecords = MemoryRecords.emptyRecords(buffer, compressionType); MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, compressionType, TimestampType.CREATE_TIME);
long nextOffset = offset;
for (final Record record : records) for (final Record record : records)
memoryRecords.append(offset, record); builder.append(nextOffset++, record);
memoryRecords.close(); return builder.build().buffer();
return memoryRecords.buffer();
} }
public static Properties producerConfig(final String bootstrapServers, public static Properties producerConfig(final String bootstrapServers,
@ -309,4 +313,22 @@ public class TestUtils {
fail(clusterId + " cannot be converted back to UUID."); fail(clusterId + " cannot be converted back to UUID.");
} }
} }
/**
* Throw an exception if the two iterators are of differing lengths or contain
* different messages on their Nth element
*/
public static <T> void checkEquals(Iterator<T> s1, Iterator<T> s2) {
while (s1.hasNext() && s2.hasNext())
assertEquals(s1.next(), s2.next());
assertFalse("Iterators have uneven length--first has more", s1.hasNext());
assertFalse("Iterators have uneven length--second has more", s2.hasNext());
}
public static <T> List<T> toList(Iterator<T> iterator) {
List<T> res = new ArrayList<>();
while (iterator.hasNext())
res.add(iterator.next());
return res;
}
} }

View File

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

View File

@ -25,7 +25,6 @@ import kafka.log.LogConfig
import kafka.server._ import kafka.server._
import kafka.metrics.KafkaMetricsGroup import kafka.metrics.KafkaMetricsGroup
import kafka.controller.KafkaController import kafka.controller.KafkaController
import kafka.message.ByteBufferMessageSet
import java.io.IOException import java.io.IOException
import java.util.concurrent.locks.ReentrantReadWriteLock import java.util.concurrent.locks.ReentrantReadWriteLock
@ -34,6 +33,7 @@ import org.apache.kafka.common.protocol.Errors
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
import com.yammer.metrics.core.Gauge import com.yammer.metrics.core.Gauge
import org.apache.kafka.common.record.MemoryRecords
import org.apache.kafka.common.requests.PartitionState import org.apache.kafka.common.requests.PartitionState
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
@ -190,7 +190,7 @@ class Partition(val topic: String,
allReplicas.foreach(replica => getOrCreateReplica(replica)) allReplicas.foreach(replica => getOrCreateReplica(replica))
val newInSyncReplicas = partitionStateInfo.isr.asScala.map(r => getOrCreateReplica(r)).toSet val newInSyncReplicas = partitionStateInfo.isr.asScala.map(r => getOrCreateReplica(r)).toSet
// remove assigned replicas that have been removed by the controller // remove assigned replicas that have been removed by the controller
(assignedReplicas().map(_.brokerId) -- allReplicas).foreach(removeReplica(_)) (assignedReplicas().map(_.brokerId) -- allReplicas).foreach(removeReplica)
inSyncReplicas = newInSyncReplicas inSyncReplicas = newInSyncReplicas
leaderEpoch = partitionStateInfo.leaderEpoch leaderEpoch = partitionStateInfo.leaderEpoch
zkVersion = partitionStateInfo.zkVersion zkVersion = partitionStateInfo.zkVersion
@ -440,7 +440,7 @@ class Partition(val topic: String,
laggingReplicas laggingReplicas
} }
def appendMessagesToLeader(messages: ByteBufferMessageSet, requiredAcks: Int = 0) = { def appendRecordsToLeader(records: MemoryRecords, requiredAcks: Int = 0) = {
val (info, leaderHWIncremented) = inReadLock(leaderIsrUpdateLock) { val (info, leaderHWIncremented) = inReadLock(leaderIsrUpdateLock) {
val leaderReplicaOpt = leaderReplicaIfLocal() val leaderReplicaOpt = leaderReplicaIfLocal()
leaderReplicaOpt match { leaderReplicaOpt match {
@ -455,7 +455,7 @@ class Partition(val topic: String,
.format(topic, partitionId, inSyncSize, minIsr)) .format(topic, partitionId, inSyncSize, minIsr))
} }
val info = log.append(messages, assignOffsets = true) val info = log.append(records, assignOffsets = true)
// probably unblock some follower fetch requests since log end offset has been updated // probably unblock some follower fetch requests since log end offset has been updated
replicaManager.tryCompleteDelayedFetch(TopicPartitionOperationKey(this.topic, this.partitionId)) replicaManager.tryCompleteDelayedFetch(TopicPartitionOperationKey(this.topic, this.partitionId))
// we may need to increment high watermark since ISR could be down to 1 // we may need to increment high watermark since ISR could be down to 1
@ -480,7 +480,7 @@ class Partition(val topic: String,
newLeaderAndIsr, controllerEpoch, zkVersion) newLeaderAndIsr, controllerEpoch, zkVersion)
if(updateSucceeded) { if(updateSucceeded) {
replicaManager.recordIsrChange(new TopicAndPartition(topic, partitionId)) replicaManager.recordIsrChange(TopicAndPartition(topic, partitionId))
inSyncReplicas = newIsr inSyncReplicas = newIsr
zkVersion = newVersion zkVersion = newVersion
trace("ISR updated to [%s] and zkVersion updated to [%d]".format(newIsr.mkString(","), zkVersion)) trace("ISR updated to [%s] and zkVersion updated to [%d]".format(newIsr.mkString(","), zkVersion))

View File

@ -17,14 +17,16 @@
package kafka.consumer package kafka.consumer
import kafka.api.{OffsetRequest, Request, FetchRequestBuilder, FetchResponsePartitionData} import kafka.api.{FetchRequestBuilder, FetchResponsePartitionData, OffsetRequest, Request}
import kafka.cluster.BrokerEndPoint import kafka.cluster.BrokerEndPoint
import kafka.message.ByteBufferMessageSet import kafka.message.ByteBufferMessageSet
import kafka.server.{PartitionFetchState, AbstractFetcherThread} import kafka.server.{AbstractFetcherThread, PartitionFetchState}
import kafka.common.{ErrorMapping, TopicAndPartition} import kafka.common.{ErrorMapping, TopicAndPartition}
import scala.collection.Map import scala.collection.Map
import ConsumerFetcherThread._ import ConsumerFetcherThread._
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.record.MemoryRecords
class ConsumerFetcherThread(name: String, class ConsumerFetcherThread(name: String,
val config: ConsumerConfig, val config: ConsumerConfig,
@ -81,7 +83,7 @@ class ConsumerFetcherThread(name: String,
case OffsetRequest.LargestTimeString => OffsetRequest.LatestTime case OffsetRequest.LargestTimeString => OffsetRequest.LatestTime
case _ => OffsetRequest.LatestTime case _ => OffsetRequest.LatestTime
} }
val topicAndPartition = new TopicAndPartition(topicPartition.topic, topicPartition.partition) val topicAndPartition = TopicAndPartition(topicPartition.topic, topicPartition.partition)
val newOffset = simpleConsumer.earliestOrLatestOffset(topicAndPartition, startTimestamp, Request.OrdinaryConsumerId) val newOffset = simpleConsumer.earliestOrLatestOffset(topicAndPartition, startTimestamp, Request.OrdinaryConsumerId)
val pti = partitionMap(topicPartition) val pti = partitionMap(topicPartition)
pti.resetFetchOffset(newOffset) pti.resetFetchOffset(newOffset)
@ -123,7 +125,7 @@ object ConsumerFetcherThread {
class PartitionData(val underlying: FetchResponsePartitionData) extends AbstractFetcherThread.PartitionData { class PartitionData(val underlying: FetchResponsePartitionData) extends AbstractFetcherThread.PartitionData {
def errorCode: Short = underlying.error def errorCode: Short = underlying.error
def toByteBufferMessageSet: ByteBufferMessageSet = underlying.messages.asInstanceOf[ByteBufferMessageSet] def toRecords: MemoryRecords = underlying.messages.asInstanceOf[ByteBufferMessageSet].asRecords
def highWatermark: Long = underlying.hw def highWatermark: Long = underlying.hw
def exception: Option[Throwable] = def exception: Option[Throwable] =
if (errorCode == ErrorMapping.NoError) None else Some(ErrorMapping.exceptionFor(errorCode)) if (errorCode == ErrorMapping.NoError) None else Some(ErrorMapping.exceptionFor(errorCode))

View File

@ -458,14 +458,14 @@ class GroupCoordinator(val brokerId: Int,
def handleFetchOffsets(groupId: String, def handleFetchOffsets(groupId: String,
partitions: Seq[TopicPartition]): Map[TopicPartition, OffsetFetchResponse.PartitionData] = { partitions: Seq[TopicPartition]): Map[TopicPartition, OffsetFetchResponse.PartitionData] = {
if (!isActive.get) { if (!isActive.get) {
partitions.map { case topicPartition => partitions.map { topicPartition =>
(topicPartition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code))}.toMap (topicPartition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code))}.toMap
} else if (!isCoordinatorForGroup(groupId)) { } else if (!isCoordinatorForGroup(groupId)) {
debug("Could not fetch offsets for group %s (not group coordinator).".format(groupId)) debug("Could not fetch offsets for group %s (not group coordinator).".format(groupId))
partitions.map { case topicPartition => partitions.map { topicPartition =>
(topicPartition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.NOT_COORDINATOR_FOR_GROUP.code))}.toMap (topicPartition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.NOT_COORDINATOR_FOR_GROUP.code))}.toMap
} else if (isCoordinatorLoadingInProgress(groupId)) { } else if (isCoordinatorLoadingInProgress(groupId)) {
partitions.map { case topicPartition => partitions.map { topicPartition =>
(topicPartition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.GROUP_LOAD_IN_PROGRESS.code))}.toMap (topicPartition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.GROUP_LOAD_IN_PROGRESS.code))}.toMap
} else { } else {
// return offsets blindly regardless the current group state since the group may be using // return offsets blindly regardless the current group state since the group may be using

View File

@ -17,38 +17,31 @@
package kafka.coordinator package kafka.coordinator
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.protocol.types.{ArrayOf, Field, Schema, Struct}
import org.apache.kafka.common.protocol.types.Type.STRING
import org.apache.kafka.common.protocol.types.Type.NULLABLE_STRING
import org.apache.kafka.common.protocol.types.Type.INT32
import org.apache.kafka.common.protocol.types.Type.INT64
import org.apache.kafka.common.protocol.types.Type.BYTES
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
import org.apache.kafka.common.requests.OffsetFetchResponse
import org.apache.kafka.common.utils.Utils
import org.apache.kafka.common.utils.Time
import org.apache.kafka.clients.consumer.ConsumerRecord
import kafka.utils._
import kafka.common._
import kafka.message._
import kafka.log.FileMessageSet
import kafka.metrics.KafkaMetricsGroup
import kafka.common.TopicAndPartition
import kafka.common.MessageFormatter
import kafka.server.ReplicaManager
import scala.collection._
import java.io.PrintStream import java.io.PrintStream
import java.nio.ByteBuffer import java.nio.ByteBuffer
import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent.locks.ReentrantLock import java.util.concurrent.locks.ReentrantLock
import com.yammer.metrics.core.Gauge import com.yammer.metrics.core.Gauge
import kafka.api.{ApiVersion, KAFKA_0_10_1_IV0} import kafka.api.{ApiVersion, KAFKA_0_10_1_IV0}
import kafka.common.{MessageFormatter, TopicAndPartition, _}
import kafka.metrics.KafkaMetricsGroup
import kafka.server.ReplicaManager
import kafka.utils.CoreUtils.inLock import kafka.utils.CoreUtils.inLock
import kafka.utils._
import org.apache.kafka.clients.consumer.ConsumerRecord
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.protocol.types.Type._
import org.apache.kafka.common.protocol.types.{ArrayOf, Field, Schema, Struct}
import org.apache.kafka.common.record._
import org.apache.kafka.common.requests.OffsetFetchResponse
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
import org.apache.kafka.common.utils.{Time, Utils}
import scala.collection.JavaConverters._
import scala.collection._
class GroupMetadataManager(val brokerId: Int, class GroupMetadataManager(val brokerId: Int,
val interBrokerProtocolVersion: ApiVersion, val interBrokerProtocolVersion: ApiVersion,
@ -57,6 +50,8 @@ class GroupMetadataManager(val brokerId: Int,
zkUtils: ZkUtils, zkUtils: ZkUtils,
time: Time) extends Logging with KafkaMetricsGroup { time: Time) extends Logging with KafkaMetricsGroup {
private val compressionType: CompressionType = CompressionType.forId(config.offsetsTopicCompressionCodec.codec)
private val groupMetadataCache = new Pool[String, GroupMetadata] private val groupMetadataCache = new Pool[String, GroupMetadata]
/* lock protecting access to loading and owned partition sets */ /* lock protecting access to loading and owned partition sets */
@ -135,13 +130,11 @@ class GroupMetadataManager(val brokerId: Int,
} }
} }
def prepareStoreGroup(group: GroupMetadata, def prepareStoreGroup(group: GroupMetadata,
groupAssignment: Map[String, Array[Byte]], groupAssignment: Map[String, Array[Byte]],
responseCallback: Errors => Unit): Option[DelayedStore] = { responseCallback: Errors => Unit): Option[DelayedStore] = {
val magicValueAndTimestampOpt = getMessageFormatVersionAndTimestamp(partitionFor(group.groupId)) getMagicAndTimestamp(partitionFor(group.groupId)) match {
magicValueAndTimestampOpt match { case Some((magicValue, timestampType, timestamp)) =>
case Some((magicValue, timestamp)) =>
val groupMetadataValueVersion = { val groupMetadataValueVersion = {
if (interBrokerProtocolVersion < KAFKA_0_10_1_IV0) if (interBrokerProtocolVersion < KAFKA_0_10_1_IV0)
0.toShort 0.toShort
@ -149,17 +142,12 @@ class GroupMetadataManager(val brokerId: Int,
GroupMetadataManager.CURRENT_GROUP_VALUE_SCHEMA_VERSION GroupMetadataManager.CURRENT_GROUP_VALUE_SCHEMA_VERSION
} }
val message = new Message( val record = Record.create(magicValue, timestampType, timestamp,
key = GroupMetadataManager.groupMetadataKey(group.groupId), GroupMetadataManager.groupMetadataKey(group.groupId),
bytes = GroupMetadataManager.groupMetadataValue(group, groupAssignment, version = groupMetadataValueVersion), GroupMetadataManager.groupMetadataValue(group, groupAssignment, version = groupMetadataValueVersion))
timestamp = timestamp,
magicValue = magicValue)
val groupMetadataPartition = new TopicPartition(Topic.GroupMetadataTopicName, partitionFor(group.groupId)) val groupMetadataPartition = new TopicPartition(Topic.GroupMetadataTopicName, partitionFor(group.groupId))
val groupMetadataRecords = Map(groupMetadataPartition -> MemoryRecords.withRecords(timestampType, compressionType, record))
val groupMetadataMessageSet = Map(groupMetadataPartition ->
new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, message))
val generationId = group.generationId val generationId = group.generationId
// set the callback function to insert the created group into cache after log append completed // set the callback function to insert the created group into cache after log append completed
@ -212,7 +200,7 @@ class GroupMetadataManager(val brokerId: Int,
responseCallback(responseError) responseCallback(responseError)
} }
Some(DelayedStore(groupMetadataMessageSet, putCacheCallback)) Some(DelayedStore(groupMetadataRecords, putCacheCallback))
case None => case None =>
responseCallback(Errors.NOT_COORDINATOR_FOR_GROUP) responseCallback(Errors.NOT_COORDINATOR_FOR_GROUP)
@ -222,11 +210,11 @@ class GroupMetadataManager(val brokerId: Int,
def store(delayedStore: DelayedStore) { def store(delayedStore: DelayedStore) {
// call replica manager to append the group message // call replica manager to append the group message
replicaManager.appendMessages( replicaManager.appendRecords(
config.offsetCommitTimeoutMs.toLong, config.offsetCommitTimeoutMs.toLong,
config.offsetCommitRequiredAcks, config.offsetCommitRequiredAcks,
true, // allow appending to internal offset topic true, // allow appending to internal offset topic
delayedStore.messageSet, delayedStore.partitionRecords,
delayedStore.callback) delayedStore.callback)
} }
@ -244,22 +232,17 @@ class GroupMetadataManager(val brokerId: Int,
} }
// construct the message set to append // construct the message set to append
val magicValueAndTimestampOpt = getMessageFormatVersionAndTimestamp(partitionFor(group.groupId)) getMagicAndTimestamp(partitionFor(group.groupId)) match {
magicValueAndTimestampOpt match { case Some((magicValue, timestampType, timestamp)) =>
case Some((magicValue, timestamp)) => val records = filteredOffsetMetadata.map { case (topicAndPartition, offsetAndMetadata) =>
val messages = filteredOffsetMetadata.map { case (topicAndPartition, offsetAndMetadata) => Record.create(magicValue, timestampType, timestamp,
new Message( GroupMetadataManager.offsetCommitKey(group.groupId, topicAndPartition.topic, topicAndPartition.partition),
key = GroupMetadataManager.offsetCommitKey(group.groupId, topicAndPartition.topic, topicAndPartition.partition), GroupMetadataManager.offsetCommitValue(offsetAndMetadata))
bytes = GroupMetadataManager.offsetCommitValue(offsetAndMetadata),
timestamp = timestamp,
magicValue = magicValue
)
}.toSeq }.toSeq
val offsetTopicPartition = new TopicPartition(Topic.GroupMetadataTopicName, partitionFor(group.groupId)) val offsetTopicPartition = new TopicPartition(Topic.GroupMetadataTopicName, partitionFor(group.groupId))
val offsetsAndMetadataMessageSet = Map(offsetTopicPartition -> val entries = Map(offsetTopicPartition -> MemoryRecords.withRecords(timestampType, compressionType, records:_*))
new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages:_*))
// set the callback function to insert offsets into cache after log append completed // set the callback function to insert offsets into cache after log append completed
def putCacheCallback(responseStatus: Map[TopicPartition, PartitionResponse]) { def putCacheCallback(responseStatus: Map[TopicPartition, PartitionResponse]) {
@ -330,7 +313,7 @@ class GroupMetadataManager(val brokerId: Int,
group.prepareOffsetCommit(offsetMetadata) group.prepareOffsetCommit(offsetMetadata)
} }
Some(DelayedStore(offsetsAndMetadataMessageSet, putCacheCallback)) Some(DelayedStore(entries, putCacheCallback))
case None => case None =>
val commitStatus = offsetMetadata.map { case (topicAndPartition, offsetAndMetadata) => val commitStatus = offsetMetadata.map { case (topicAndPartition, offsetAndMetadata) =>
@ -412,28 +395,30 @@ class GroupMetadataManager(val brokerId: Int,
while (currOffset < getHighWatermark(offsetsPartition) && !shuttingDown.get()) { while (currOffset < getHighWatermark(offsetsPartition) && !shuttingDown.get()) {
buffer.clear() buffer.clear()
val messages = log.read(currOffset, config.loadBufferSize, minOneMessage = true).messageSet.asInstanceOf[FileMessageSet] val fileRecords = log.read(currOffset, config.loadBufferSize, minOneMessage = true).records.asInstanceOf[FileRecords]
messages.readInto(buffer, 0) fileRecords.readInto(buffer, 0)
val messageSet = new ByteBufferMessageSet(buffer)
messageSet.foreach { msgAndOffset => MemoryRecords.readableRecords(buffer).deepIterator.asScala.foreach { entry =>
require(msgAndOffset.message.key != null, "Offset entry key should not be null") val record = entry.record
val baseKey = GroupMetadataManager.readMessageKey(msgAndOffset.message.key)
require(record.hasKey, "Offset entry key should not be null")
val baseKey = GroupMetadataManager.readMessageKey(record.key)
if (baseKey.isInstanceOf[OffsetKey]) { if (baseKey.isInstanceOf[OffsetKey]) {
// load offset // load offset
val key = baseKey.key.asInstanceOf[GroupTopicPartition] val key = baseKey.key.asInstanceOf[GroupTopicPartition]
if (msgAndOffset.message.payload == null) { if (record.hasNullValue) {
loadedOffsets.remove(key) loadedOffsets.remove(key)
removedOffsets.add(key) removedOffsets.add(key)
} else { } else {
val value = GroupMetadataManager.readOffsetMessageValue(msgAndOffset.message.payload) val value = GroupMetadataManager.readOffsetMessageValue(record.value)
loadedOffsets.put(key, value) loadedOffsets.put(key, value)
removedOffsets.remove(key) removedOffsets.remove(key)
} }
} else { } else {
// load group metadata // load group metadata
val groupId = baseKey.key.asInstanceOf[String] val groupId = baseKey.key.asInstanceOf[String]
val groupMetadata = GroupMetadataManager.readGroupMessageValue(groupId, msgAndOffset.message.payload) val groupMetadata = GroupMetadataManager.readGroupMessageValue(groupId, record.value)
if (groupMetadata != null) { if (groupMetadata != null) {
trace(s"Loaded group metadata for group ${groupMetadata.groupId} with generation ${groupMetadata.generationId}") trace(s"Loaded group metadata for group ${groupMetadata.groupId} with generation ${groupMetadata.generationId}")
removedGroups.remove(groupId) removedGroups.remove(groupId)
@ -444,7 +429,7 @@ class GroupMetadataManager(val brokerId: Int,
} }
} }
currOffset = msgAndOffset.nextOffset currOffset = entry.nextOffset
} }
} }
@ -467,8 +452,8 @@ class GroupMetadataManager(val brokerId: Int,
removedGroups.foreach { groupId => removedGroups.foreach { groupId =>
if (groupMetadataCache.contains(groupId)) if (groupMetadataCache.contains(groupId))
throw new IllegalStateException(s"Unexpected unload of active group ${groupId} while " + throw new IllegalStateException(s"Unexpected unload of active group $groupId while " +
s"loading partition ${topicPartition}") s"loading partition $topicPartition")
} }
if (!shuttingDown.get()) if (!shuttingDown.get())
@ -572,15 +557,15 @@ class GroupMetadataManager(val brokerId: Int,
} }
val offsetsPartition = partitionFor(groupId) val offsetsPartition = partitionFor(groupId)
getMessageFormatVersionAndTimestamp(offsetsPartition) match { getMagicAndTimestamp(offsetsPartition) match {
case Some((magicValue, timestamp)) => case Some((magicValue, timestampType, timestamp)) =>
val partitionOpt = replicaManager.getPartition(Topic.GroupMetadataTopicName, offsetsPartition) val partitionOpt = replicaManager.getPartition(Topic.GroupMetadataTopicName, offsetsPartition)
partitionOpt.foreach { partition => partitionOpt.foreach { partition =>
val appendPartition = TopicAndPartition(Topic.GroupMetadataTopicName, offsetsPartition) val appendPartition = TopicAndPartition(Topic.GroupMetadataTopicName, offsetsPartition)
val tombstones = expiredOffsets.map { case (topicPartition, offsetAndMetadata) => val tombstones = expiredOffsets.map { case (topicPartition, offsetAndMetadata) =>
trace(s"Removing expired offset and metadata for $groupId, $topicPartition: $offsetAndMetadata") trace(s"Removing expired offset and metadata for $groupId, $topicPartition: $offsetAndMetadata")
val commitKey = GroupMetadataManager.offsetCommitKey(groupId, topicPartition.topic, topicPartition.partition) val commitKey = GroupMetadataManager.offsetCommitKey(groupId, topicPartition.topic, topicPartition.partition)
new Message(bytes = null, key = commitKey, timestamp = timestamp, magicValue = magicValue) Record.create(magicValue, timestampType, timestamp, commitKey, null)
}.toBuffer }.toBuffer
trace(s"Marked ${expiredOffsets.size} offsets in $appendPartition for deletion.") trace(s"Marked ${expiredOffsets.size} offsets in $appendPartition for deletion.")
@ -590,8 +575,7 @@ class GroupMetadataManager(val brokerId: Int,
// Append the tombstone messages to the partition. It is okay if the replicas don't receive these (say, // Append the tombstone messages to the partition. It is okay if the replicas don't receive these (say,
// if we crash or leaders move) since the new leaders will still expire the consumers with heartbeat and // if we crash or leaders move) since the new leaders will still expire the consumers with heartbeat and
// retry removing this group. // retry removing this group.
tombstones += new Message(bytes = null, key = GroupMetadataManager.groupMetadataKey(group.groupId), tombstones += Record.create(magicValue, timestampType, timestamp, GroupMetadataManager.groupMetadataKey(group.groupId), null)
timestamp = timestamp, magicValue = magicValue)
trace(s"Group $groupId removed from the metadata cache and marked for deletion in $appendPartition.") trace(s"Group $groupId removed from the metadata cache and marked for deletion in $appendPartition.")
} }
@ -599,7 +583,7 @@ class GroupMetadataManager(val brokerId: Int,
try { try {
// do not need to require acks since even if the tombstone is lost, // do not need to require acks since even if the tombstone is lost,
// it will be appended again in the next purge cycle // it will be appended again in the next purge cycle
partition.appendMessagesToLeader(new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, tombstones: _*)) partition.appendRecordsToLeader(MemoryRecords.withRecords(timestampType, compressionType, tombstones: _*))
offsetsRemoved += expiredOffsets.size offsetsRemoved += expiredOffsets.size
trace(s"Successfully appended ${tombstones.size} tombstones to $appendPartition for expired offsets and/or metadata for group $groupId") trace(s"Successfully appended ${tombstones.size} tombstones to $appendPartition for expired offsets and/or metadata for group $groupId")
} catch { } catch {
@ -663,16 +647,11 @@ class GroupMetadataManager(val brokerId: Int,
* @param partition Partition of GroupMetadataTopic * @param partition Partition of GroupMetadataTopic
* @return Option[(MessageFormatVersion, TimeStamp)] if replica is local, None otherwise * @return Option[(MessageFormatVersion, TimeStamp)] if replica is local, None otherwise
*/ */
private def getMessageFormatVersionAndTimestamp(partition: Int): Option[(Byte, Long)] = { private def getMagicAndTimestamp(partition: Int): Option[(Byte, TimestampType, Long)] = {
val groupMetadataTopicAndPartition = TopicAndPartition(Topic.GroupMetadataTopicName, partition) val groupMetadataTopicAndPartition = TopicAndPartition(Topic.GroupMetadataTopicName, partition)
replicaManager.getMessageFormatVersion(groupMetadataTopicAndPartition).map { messageFormatVersion => replicaManager.getMagicAndTimestampType(groupMetadataTopicAndPartition).map { case (messageFormatVersion, timestampType) =>
val timestamp = { val timestamp = if (messageFormatVersion == Record.MAGIC_VALUE_V0) Record.NO_TIMESTAMP else time.milliseconds()
if (messageFormatVersion == Message.MagicValue_V0) (messageFormatVersion, timestampType, timestamp)
Message.NoTimestamp
else
time.milliseconds()
}
(messageFormatVersion, timestamp)
} }
} }
@ -964,7 +943,7 @@ object GroupMetadataManager {
* @return an offset-metadata object from the message * @return an offset-metadata object from the message
*/ */
def readOffsetMessageValue(buffer: ByteBuffer): OffsetAndMetadata = { def readOffsetMessageValue(buffer: ByteBuffer): OffsetAndMetadata = {
if(buffer == null) { // tombstone if (buffer == null) { // tombstone
null null
} else { } else {
val version = buffer.getShort val version = buffer.getShort
@ -997,7 +976,7 @@ object GroupMetadataManager {
* @return a group metadata object from the message * @return a group metadata object from the message
*/ */
def readGroupMessageValue(groupId: String, buffer: ByteBuffer): GroupMetadata = { def readGroupMessageValue(groupId: String, buffer: ByteBuffer): GroupMetadata = {
if(buffer == null) { // tombstone if (buffer == null) { // tombstone
null null
} else { } else {
val version = buffer.getShort val version = buffer.getShort
@ -1016,23 +995,22 @@ object GroupMetadataManager {
group.leaderId = value.get(LEADER_KEY).asInstanceOf[String] group.leaderId = value.get(LEADER_KEY).asInstanceOf[String]
group.protocol = value.get(PROTOCOL_KEY).asInstanceOf[String] group.protocol = value.get(PROTOCOL_KEY).asInstanceOf[String]
memberMetadataArray.foreach { memberMetadataArray.foreach { memberMetadataObj =>
case memberMetadataObj => val memberMetadata = memberMetadataObj.asInstanceOf[Struct]
val memberMetadata = memberMetadataObj.asInstanceOf[Struct] val memberId = memberMetadata.get(MEMBER_ID_KEY).asInstanceOf[String]
val memberId = memberMetadata.get(MEMBER_ID_KEY).asInstanceOf[String] val clientId = memberMetadata.get(CLIENT_ID_KEY).asInstanceOf[String]
val clientId = memberMetadata.get(CLIENT_ID_KEY).asInstanceOf[String] val clientHost = memberMetadata.get(CLIENT_HOST_KEY).asInstanceOf[String]
val clientHost = memberMetadata.get(CLIENT_HOST_KEY).asInstanceOf[String] val sessionTimeout = memberMetadata.get(SESSION_TIMEOUT_KEY).asInstanceOf[Int]
val sessionTimeout = memberMetadata.get(SESSION_TIMEOUT_KEY).asInstanceOf[Int] val rebalanceTimeout = if (version == 0) sessionTimeout else memberMetadata.get(REBALANCE_TIMEOUT_KEY).asInstanceOf[Int]
val rebalanceTimeout = if (version == 0) sessionTimeout else memberMetadata.get(REBALANCE_TIMEOUT_KEY).asInstanceOf[Int]
val subscription = Utils.toArray(memberMetadata.get(SUBSCRIPTION_KEY).asInstanceOf[ByteBuffer]) val subscription = Utils.toArray(memberMetadata.get(SUBSCRIPTION_KEY).asInstanceOf[ByteBuffer])
val member = new MemberMetadata(memberId, groupId, clientId, clientHost, rebalanceTimeout, sessionTimeout, val member = new MemberMetadata(memberId, groupId, clientId, clientHost, rebalanceTimeout, sessionTimeout,
protocolType, List((group.protocol, subscription))) protocolType, List((group.protocol, subscription)))
member.assignment = Utils.toArray(memberMetadata.get(ASSIGNMENT_KEY).asInstanceOf[ByteBuffer]) member.assignment = Utils.toArray(memberMetadata.get(ASSIGNMENT_KEY).asInstanceOf[ByteBuffer])
group.add(memberId, member) group.add(memberId, member)
} }
group group
@ -1087,7 +1065,7 @@ object GroupMetadataManager {
} }
case class DelayedStore(messageSet: Map[TopicPartition, MessageSet], case class DelayedStore(partitionRecords: Map[TopicPartition, MemoryRecords],
callback: Map[TopicPartition, PartitionResponse] => Unit) callback: Map[TopicPartition, PartitionResponse] => Unit)
case class GroupTopicPartition(group: String, topicPartition: TopicPartition) { case class GroupTopicPartition(group: String, topicPartition: TopicPartition) {

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.api.KAFKA_0_10_0_IV0
import kafka.utils._ import kafka.utils._
import kafka.message._
import kafka.common._ import kafka.common._
import kafka.metrics.KafkaMetricsGroup import kafka.metrics.KafkaMetricsGroup
import kafka.server.{BrokerTopicStats, FetchDataInfo, LogOffsetMetadata} import kafka.server.{BrokerTopicStats, FetchDataInfo, LogOffsetMetadata}
@ -29,16 +28,18 @@ import java.util.concurrent.atomic._
import java.text.NumberFormat import java.text.NumberFormat
import org.apache.kafka.common.errors.{CorruptRecordException, OffsetOutOfRangeException, RecordBatchTooLargeException, RecordTooLargeException, UnsupportedForMessageFormatException} import org.apache.kafka.common.errors.{CorruptRecordException, OffsetOutOfRangeException, RecordBatchTooLargeException, RecordTooLargeException, UnsupportedForMessageFormatException}
import org.apache.kafka.common.record.TimestampType import org.apache.kafka.common.record._
import org.apache.kafka.common.requests.ListOffsetRequest import org.apache.kafka.common.requests.ListOffsetRequest
import scala.collection.Seq import scala.collection.Seq
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
import com.yammer.metrics.core.Gauge import com.yammer.metrics.core.Gauge
import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.utils.{Time, Utils}
import kafka.message.{BrokerCompressionCodec, CompressionCodec, NoCompressionCodec}
object LogAppendInfo { object LogAppendInfo {
val UnknownLogAppendInfo = LogAppendInfo(-1, -1, Message.NoTimestamp, -1L, Message.NoTimestamp, NoCompressionCodec, NoCompressionCodec, -1, -1, offsetsMonotonic = false) val UnknownLogAppendInfo = LogAppendInfo(-1, -1, Record.NO_TIMESTAMP, -1L, Record.NO_TIMESTAMP,
NoCompressionCodec, NoCompressionCodec, -1, -1, offsetsMonotonic = false)
} }
/** /**
@ -243,7 +244,7 @@ class Log(@volatile var dir: File,
val index = new OffsetIndex(indexFile, baseOffset = startOffset, maxIndexSize = config.maxIndexSize) val index = new OffsetIndex(indexFile, baseOffset = startOffset, maxIndexSize = config.maxIndexSize)
val timeIndexFile = new File(CoreUtils.replaceSuffix(logFile.getPath, LogFileSuffix, TimeIndexFileSuffix) + SwapFileSuffix) val timeIndexFile = new File(CoreUtils.replaceSuffix(logFile.getPath, LogFileSuffix, TimeIndexFileSuffix) + SwapFileSuffix)
val timeIndex = new TimeIndex(timeIndexFile, baseOffset = startOffset, maxIndexSize = config.maxIndexSize) val timeIndex = new TimeIndex(timeIndexFile, baseOffset = startOffset, maxIndexSize = config.maxIndexSize)
val swapSegment = new LogSegment(new FileMessageSet(file = swapFile), val swapSegment = new LogSegment(FileRecords.open(swapFile),
index = index, index = index,
timeIndex = timeIndex, timeIndex = timeIndex,
baseOffset = startOffset, baseOffset = startOffset,
@ -338,20 +339,20 @@ class Log(@volatile var dir: File,
* This method will generally be responsible for assigning offsets to the messages, * This method will generally be responsible for assigning offsets to the messages,
* however if the assignOffsets=false flag is passed we will only check that the existing offsets are valid. * however if the assignOffsets=false flag is passed we will only check that the existing offsets are valid.
* *
* @param messages The message set to append * @param records The log records to append
* @param assignOffsets Should the log assign offsets to this message set or blindly apply what it is given * @param assignOffsets Should the log assign offsets to this message set or blindly apply what it is given
* @throws KafkaStorageException If the append fails due to an I/O error. * @throws KafkaStorageException If the append fails due to an I/O error.
* @return Information about the appended messages including the first and last offset. * @return Information about the appended messages including the first and last offset.
*/ */
def append(messages: ByteBufferMessageSet, assignOffsets: Boolean = true): LogAppendInfo = { def append(records: MemoryRecords, assignOffsets: Boolean = true): LogAppendInfo = {
val appendInfo = analyzeAndValidateMessageSet(messages) val appendInfo = analyzeAndValidateRecords(records)
// if we have any valid messages, append them to the log // if we have any valid messages, append them to the log
if (appendInfo.shallowCount == 0) if (appendInfo.shallowCount == 0)
return appendInfo return appendInfo
// trim any invalid bytes or partial messages before appending it to the on-disk log // trim any invalid bytes or partial messages before appending it to the on-disk log
var validMessages = trimInvalidBytes(messages, appendInfo) var validRecords = trimInvalidBytes(records, appendInfo)
try { try {
// they are valid, insert them in the log // they are valid, insert them in the log
@ -363,20 +364,21 @@ class Log(@volatile var dir: File,
appendInfo.firstOffset = offset.value appendInfo.firstOffset = offset.value
val now = time.milliseconds val now = time.milliseconds
val validateAndOffsetAssignResult = try { val validateAndOffsetAssignResult = try {
validMessages.validateMessagesAndAssignOffsets(offset, LogValidator.validateMessagesAndAssignOffsets(validRecords,
now, offset,
appendInfo.sourceCodec, now,
appendInfo.targetCodec, appendInfo.sourceCodec,
config.compact, appendInfo.targetCodec,
config.messageFormatVersion.messageFormatVersion, config.compact,
config.messageTimestampType, config.messageFormatVersion.messageFormatVersion,
config.messageTimestampDifferenceMaxMs) config.messageTimestampType,
config.messageTimestampDifferenceMaxMs)
} catch { } catch {
case e: IOException => throw new KafkaException("Error in validating messages while appending to log '%s'".format(name), e) case e: IOException => throw new KafkaException("Error in validating messages while appending to log '%s'".format(name), e)
} }
validMessages = validateAndOffsetAssignResult.validatedMessages validRecords = validateAndOffsetAssignResult.validatedRecords
appendInfo.maxTimestamp = validateAndOffsetAssignResult.maxTimestamp appendInfo.maxTimestamp = validateAndOffsetAssignResult.maxTimestamp
appendInfo.offsetOfMaxTimestamp = validateAndOffsetAssignResult.offsetOfMaxTimestamp appendInfo.offsetOfMaxTimestamp = validateAndOffsetAssignResult.shallowOffsetOfMaxTimestamp
appendInfo.lastOffset = offset.value - 1 appendInfo.lastOffset = offset.value - 1
if (config.messageTimestampType == TimestampType.LOG_APPEND_TIME) if (config.messageTimestampType == TimestampType.LOG_APPEND_TIME)
appendInfo.logAppendTime = now appendInfo.logAppendTime = now
@ -384,14 +386,14 @@ class Log(@volatile var dir: File,
// re-validate message sizes if there's a possibility that they have changed (due to re-compression or message // re-validate message sizes if there's a possibility that they have changed (due to re-compression or message
// format conversion) // format conversion)
if (validateAndOffsetAssignResult.messageSizeMaybeChanged) { if (validateAndOffsetAssignResult.messageSizeMaybeChanged) {
for (messageAndOffset <- validMessages.shallowIterator) { for (logEntry <- validRecords.shallowIterator.asScala) {
if (MessageSet.entrySize(messageAndOffset.message) > config.maxMessageSize) { if (logEntry.sizeInBytes > config.maxMessageSize) {
// we record the original message set size instead of the trimmed size // we record the original message set size instead of the trimmed size
// to be consistent with pre-compression bytesRejectedRate recording // to be consistent with pre-compression bytesRejectedRate recording
BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesRejectedRate.mark(messages.sizeInBytes) BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesRejectedRate.mark(records.sizeInBytes)
BrokerTopicStats.getBrokerAllTopicsStats.bytesRejectedRate.mark(messages.sizeInBytes) BrokerTopicStats.getBrokerAllTopicsStats.bytesRejectedRate.mark(records.sizeInBytes)
throw new RecordTooLargeException("Message size is %d bytes which exceeds the maximum configured message size of %d." throw new RecordTooLargeException("Message size is %d bytes which exceeds the maximum configured message size of %d."
.format(MessageSet.entrySize(messageAndOffset.message), config.maxMessageSize)) .format(logEntry.sizeInBytes, config.maxMessageSize))
} }
} }
} }
@ -399,28 +401,27 @@ class Log(@volatile var dir: File,
} else { } else {
// we are taking the offsets we are given // we are taking the offsets we are given
if (!appendInfo.offsetsMonotonic || appendInfo.firstOffset < nextOffsetMetadata.messageOffset) if (!appendInfo.offsetsMonotonic || appendInfo.firstOffset < nextOffsetMetadata.messageOffset)
throw new IllegalArgumentException("Out of order offsets found in " + messages) throw new IllegalArgumentException("Out of order offsets found in " + records.deepIterator.asScala.map(_.offset))
} }
// check messages set size may be exceed config.segmentSize // check messages set size may be exceed config.segmentSize
if (validMessages.sizeInBytes > config.segmentSize) { if (validRecords.sizeInBytes > config.segmentSize) {
throw new RecordBatchTooLargeException("Message set size is %d bytes which exceeds the maximum configured segment size of %d." throw new RecordBatchTooLargeException("Message set size is %d bytes which exceeds the maximum configured segment size of %d."
.format(validMessages.sizeInBytes, config.segmentSize)) .format(validRecords.sizeInBytes, config.segmentSize))
} }
// maybe roll the log if this segment is full // maybe roll the log if this segment is full
val segment = maybeRoll(messagesSize = validMessages.sizeInBytes, val segment = maybeRoll(messagesSize = validRecords.sizeInBytes, maxTimestampInMessages = appendInfo.maxTimestamp)
maxTimestampInMessages = appendInfo.maxTimestamp)
// now append to the log // now append to the log
segment.append(firstOffset = appendInfo.firstOffset, largestTimestamp = appendInfo.maxTimestamp, segment.append(firstOffset = appendInfo.firstOffset, largestTimestamp = appendInfo.maxTimestamp,
offsetOfLargestTimestamp = appendInfo.offsetOfMaxTimestamp, messages = validMessages) shallowOffsetOfMaxTimestamp = appendInfo.offsetOfMaxTimestamp, records = validRecords)
// increment the log end offset // increment the log end offset
updateLogEndOffset(appendInfo.lastOffset + 1) updateLogEndOffset(appendInfo.lastOffset + 1)
trace("Appended message set to log %s with first offset: %d, next offset: %d, and messages: %s" trace("Appended message set to log %s with first offset: %d, next offset: %d, and messages: %s"
.format(this.name, appendInfo.firstOffset, nextOffsetMetadata.messageOffset, validMessages)) .format(this.name, appendInfo.firstOffset, nextOffsetMetadata.messageOffset, validRecords))
if (unflushedMessages >= config.flushInterval) if (unflushedMessages >= config.flushInterval)
flush() flush()
@ -449,73 +450,74 @@ class Log(@volatile var dir: File,
* <li> Whether any compression codec is used (if many are used, then the last one is given) * <li> Whether any compression codec is used (if many are used, then the last one is given)
* </ol> * </ol>
*/ */
private def analyzeAndValidateMessageSet(messages: ByteBufferMessageSet): LogAppendInfo = { private def analyzeAndValidateRecords(records: MemoryRecords): LogAppendInfo = {
var shallowMessageCount = 0 var shallowMessageCount = 0
var validBytesCount = 0 var validBytesCount = 0
var firstOffset, lastOffset = -1L var firstOffset, lastOffset = -1L
var sourceCodec: CompressionCodec = NoCompressionCodec var sourceCodec: CompressionCodec = NoCompressionCodec
var monotonic = true var monotonic = true
var maxTimestamp = Message.NoTimestamp var maxTimestamp = Record.NO_TIMESTAMP
var offsetOfMaxTimestamp = -1L var offsetOfMaxTimestamp = -1L
for(messageAndOffset <- messages.shallowIterator) { for (entry <- records.shallowIterator.asScala) {
// update the first offset if on the first message // update the first offset if on the first message
if(firstOffset < 0) if(firstOffset < 0)
firstOffset = messageAndOffset.offset firstOffset = entry.offset
// check that offsets are monotonically increasing // check that offsets are monotonically increasing
if(lastOffset >= messageAndOffset.offset) if(lastOffset >= entry.offset)
monotonic = false monotonic = false
// update the last offset seen // update the last offset seen
lastOffset = messageAndOffset.offset lastOffset = entry.offset
val m = messageAndOffset.message val record = entry.record
// Check if the message sizes are valid. // Check if the message sizes are valid.
val messageSize = MessageSet.entrySize(m) val messageSize = entry.sizeInBytes
if(messageSize > config.maxMessageSize) { if(messageSize > config.maxMessageSize) {
BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesRejectedRate.mark(messages.sizeInBytes) BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesRejectedRate.mark(records.sizeInBytes)
BrokerTopicStats.getBrokerAllTopicsStats.bytesRejectedRate.mark(messages.sizeInBytes) BrokerTopicStats.getBrokerAllTopicsStats.bytesRejectedRate.mark(records.sizeInBytes)
throw new RecordTooLargeException("Message size is %d bytes which exceeds the maximum configured message size of %d." throw new RecordTooLargeException("Message size is %d bytes which exceeds the maximum configured message size of %d."
.format(messageSize, config.maxMessageSize)) .format(messageSize, config.maxMessageSize))
} }
// check the validity of the message by checking CRC // check the validity of the message by checking CRC
m.ensureValid() record.ensureValid()
if (m.timestamp > maxTimestamp) { if (record.timestamp > maxTimestamp) {
maxTimestamp = m.timestamp maxTimestamp = record.timestamp
offsetOfMaxTimestamp = lastOffset offsetOfMaxTimestamp = lastOffset
} }
shallowMessageCount += 1 shallowMessageCount += 1
validBytesCount += messageSize validBytesCount += messageSize
val messageCodec = m.compressionCodec val messageCodec = CompressionCodec.getCompressionCodec(record.compressionType.id)
if(messageCodec != NoCompressionCodec) if (messageCodec != NoCompressionCodec)
sourceCodec = messageCodec sourceCodec = messageCodec
} }
// Apply broker-side compression if any // Apply broker-side compression if any
val targetCodec = BrokerCompressionCodec.getTargetCompressionCodec(config.compressionType, sourceCodec) val targetCodec = BrokerCompressionCodec.getTargetCompressionCodec(config.compressionType, sourceCodec)
LogAppendInfo(firstOffset, lastOffset, maxTimestamp, offsetOfMaxTimestamp, Message.NoTimestamp, sourceCodec, targetCodec, shallowMessageCount, validBytesCount, monotonic) LogAppendInfo(firstOffset, lastOffset, maxTimestamp, offsetOfMaxTimestamp, Record.NO_TIMESTAMP, sourceCodec,
targetCodec, shallowMessageCount, validBytesCount, monotonic)
} }
/** /**
* Trim any invalid bytes from the end of this message set (if there are any) * Trim any invalid bytes from the end of this message set (if there are any)
* *
* @param messages The message set to trim * @param records The records to trim
* @param info The general information of the message set * @param info The general information of the message set
* @return A trimmed message set. This may be the same as what was passed in or it may not. * @return A trimmed message set. This may be the same as what was passed in or it may not.
*/ */
private def trimInvalidBytes(messages: ByteBufferMessageSet, info: LogAppendInfo): ByteBufferMessageSet = { private def trimInvalidBytes(records: MemoryRecords, info: LogAppendInfo): MemoryRecords = {
val messageSetValidBytes = info.validBytes val validBytes = info.validBytes
if(messageSetValidBytes < 0) if (validBytes < 0)
throw new CorruptRecordException("Illegal length of message set " + messageSetValidBytes + " Message set cannot be appended to log. Possible causes are corrupted produce requests") throw new CorruptRecordException("Illegal length of message set " + validBytes + " Message set cannot be appended to log. Possible causes are corrupted produce requests")
if(messageSetValidBytes == messages.sizeInBytes) { if (validBytes == records.sizeInBytes) {
messages records
} else { } else {
// trim invalid bytes // trim invalid bytes
val validByteBuffer = messages.buffer.duplicate() val validByteBuffer = records.buffer.duplicate()
validByteBuffer.limit(messageSetValidBytes) validByteBuffer.limit(validBytes)
new ByteBufferMessageSet(validByteBuffer) MemoryRecords.readableRecords(validByteBuffer)
} }
} }
@ -538,7 +540,7 @@ class Log(@volatile var dir: File,
val currentNextOffsetMetadata = nextOffsetMetadata val currentNextOffsetMetadata = nextOffsetMetadata
val next = currentNextOffsetMetadata.messageOffset val next = currentNextOffsetMetadata.messageOffset
if(startOffset == next) if(startOffset == next)
return FetchDataInfo(currentNextOffsetMetadata, MessageSet.Empty) return FetchDataInfo(currentNextOffsetMetadata, MemoryRecords.EMPTY)
var entry = segments.floorEntry(startOffset) var entry = segments.floorEntry(startOffset)
@ -578,7 +580,7 @@ class Log(@volatile var dir: File,
// okay we are beyond the end of the last segment with no data fetched although the start offset is in range, // okay we are beyond the end of the last segment with no data fetched although the start offset is in range,
// this can happen when all messages with offset larger than start offsets have been deleted. // this can happen when all messages with offset larger than start offsets have been deleted.
// In this case, we will return the empty set with log end offset metadata // In this case, we will return the empty set with log end offset metadata
FetchDataInfo(nextOffsetMetadata, MessageSet.Empty) FetchDataInfo(nextOffsetMetadata, MemoryRecords.EMPTY)
} }
/** /**
@ -610,9 +612,9 @@ class Log(@volatile var dir: File,
val segmentsCopy = logSegments.toBuffer val segmentsCopy = logSegments.toBuffer
// For the earliest and latest, we do not need to return the timestamp. // For the earliest and latest, we do not need to return the timestamp.
if (targetTimestamp == ListOffsetRequest.EARLIEST_TIMESTAMP) if (targetTimestamp == ListOffsetRequest.EARLIEST_TIMESTAMP)
return Some(TimestampOffset(Message.NoTimestamp, segmentsCopy.head.baseOffset)) return Some(TimestampOffset(Record.NO_TIMESTAMP, segmentsCopy.head.baseOffset))
else if (targetTimestamp == ListOffsetRequest.LATEST_TIMESTAMP) else if (targetTimestamp == ListOffsetRequest.LATEST_TIMESTAMP)
return Some(TimestampOffset(Message.NoTimestamp, logEndOffset)) return Some(TimestampOffset(Record.NO_TIMESTAMP, logEndOffset))
val targetSeg = { val targetSeg = {
// Get all the segments whose largest timestamp is smaller than target timestamp // Get all the segments whose largest timestamp is smaller than target timestamp
@ -656,7 +658,7 @@ class Log(@volatile var dir: File,
if (segments.size == numToDelete) if (segments.size == numToDelete)
roll() roll()
// remove the segments for lookups // remove the segments for lookups
deletable.foreach(deleteSegment(_)) deletable.foreach(deleteSegment)
} }
numToDelete numToDelete
} }
@ -865,7 +867,7 @@ class Log(@volatile var dir: File,
truncateFullyAndStartAt(targetOffset) truncateFullyAndStartAt(targetOffset)
} else { } else {
val deletable = logSegments.filter(segment => segment.baseOffset > targetOffset) val deletable = logSegments.filter(segment => segment.baseOffset > targetOffset)
deletable.foreach(deleteSegment(_)) deletable.foreach(deleteSegment)
activeSegment.truncateTo(targetOffset) activeSegment.truncateTo(targetOffset)
updateLogEndOffset(targetOffset) updateLogEndOffset(targetOffset)
this.recoveryPoint = math.min(targetOffset, this.recoveryPoint) this.recoveryPoint = math.min(targetOffset, this.recoveryPoint)
@ -882,7 +884,7 @@ class Log(@volatile var dir: File,
debug("Truncate and start log '" + name + "' to " + newOffset) debug("Truncate and start log '" + name + "' to " + newOffset)
lock synchronized { lock synchronized {
val segmentsToDelete = logSegments.toList val segmentsToDelete = logSegments.toList
segmentsToDelete.foreach(deleteSegment(_)) segmentsToDelete.foreach(deleteSegment)
addSegment(new LogSegment(dir, addSegment(new LogSegment(dir,
newOffset, newOffset,
indexIntervalBytes = config.indexInterval, indexIntervalBytes = config.indexInterval,

View File

@ -17,20 +17,21 @@
package kafka.log package kafka.log
import java.io.{DataOutputStream, File} import java.io.File
import java.nio._ import java.nio._
import java.util.Date import java.util.Date
import java.util.concurrent.{CountDownLatch, TimeUnit} import java.util.concurrent.{CountDownLatch, TimeUnit}
import com.yammer.metrics.core.Gauge import com.yammer.metrics.core.Gauge
import kafka.common._ import kafka.common._
import kafka.message._
import kafka.metrics.KafkaMetricsGroup import kafka.metrics.KafkaMetricsGroup
import kafka.utils._ import kafka.utils._
import org.apache.kafka.common.record.{FileRecords, LogEntry, MemoryRecords}
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
import MemoryRecords.LogEntryFilter
import scala.Iterable
import scala.collection._ import scala.collection._
import JavaConverters._
/** /**
* The cleaner is responsible for removing obsolete records from logs which have the dedupe retention strategy. * The cleaner is responsible for removing obsolete records from logs which have the dedupe retention strategy.
@ -390,10 +391,10 @@ private[log] class Cleaner(val id: Int,
val timeIndexFile = new File(segments.head.timeIndex.file.getPath + Log.CleanedFileSuffix) val timeIndexFile = new File(segments.head.timeIndex.file.getPath + Log.CleanedFileSuffix)
indexFile.delete() indexFile.delete()
timeIndexFile.delete() timeIndexFile.delete()
val messages = new FileMessageSet(logFile, fileAlreadyExists = false, initFileSize = log.initFileSize(), preallocate = log.config.preallocate) val records = FileRecords.open(logFile, false, log.initFileSize(), log.config.preallocate)
val index = new OffsetIndex(indexFile, segments.head.baseOffset, segments.head.index.maxIndexSize) val index = new OffsetIndex(indexFile, segments.head.baseOffset, segments.head.index.maxIndexSize)
val timeIndex = new TimeIndex(timeIndexFile, segments.head.baseOffset, segments.head.timeIndex.maxIndexSize) val timeIndex = new TimeIndex(timeIndexFile, segments.head.baseOffset, segments.head.timeIndex.maxIndexSize)
val cleaned = new LogSegment(messages, index, timeIndex, segments.head.baseOffset, segments.head.indexIntervalBytes, log.config.randomSegmentJitter, time) val cleaned = new LogSegment(records, index, timeIndex, segments.head.baseOffset, segments.head.indexIntervalBytes, log.config.randomSegmentJitter, time)
try { try {
// clean segments into the new destination segment // clean segments into the new destination segment
@ -449,8 +450,12 @@ private[log] class Cleaner(val id: Int,
retainDeletes: Boolean, retainDeletes: Boolean,
maxLogMessageSize: Int, maxLogMessageSize: Int,
stats: CleanerStats) { stats: CleanerStats) {
def shouldRetain(messageAndOffset: MessageAndOffset): Boolean = def shouldRetainEntry(logEntry: LogEntry): Boolean =
shouldRetainMessage(source, map, retainDeletes, messageAndOffset, stats) shouldRetainMessage(source, map, retainDeletes, logEntry, stats)
class LogCleanerFilter extends LogEntryFilter {
def shouldRetain(logEntry: LogEntry): Boolean = shouldRetainEntry(logEntry)
}
var position = 0 var position = 0
while (position < source.log.sizeInBytes) { while (position < source.log.sizeInBytes) {
@ -460,10 +465,9 @@ private[log] class Cleaner(val id: Int,
writeBuffer.clear() writeBuffer.clear()
source.log.readInto(readBuffer, position) source.log.readInto(readBuffer, position)
val messages = new ByteBufferMessageSet(readBuffer) val records = MemoryRecords.readableRecords(readBuffer)
throttler.maybeThrottle(messages.sizeInBytes) throttler.maybeThrottle(records.sizeInBytes)
val result = messages.filterInto(writeBuffer, shouldRetain) val result = records.filterTo(new LogCleanerFilter, writeBuffer)
stats.readMessages(result.messagesRead, result.bytesRead) stats.readMessages(result.messagesRead, result.bytesRead)
stats.recopyMessages(result.messagesRetained, result.bytesRetained) stats.recopyMessages(result.messagesRetained, result.bytesRetained)
@ -472,9 +476,10 @@ private[log] class Cleaner(val id: Int,
// if any messages are to be retained, write them out // if any messages are to be retained, write them out
if (writeBuffer.position > 0) { if (writeBuffer.position > 0) {
writeBuffer.flip() writeBuffer.flip()
val retained = new ByteBufferMessageSet(writeBuffer)
dest.append(firstOffset = retained.head.offset, largestTimestamp = result.maxTimestamp, val retained = MemoryRecords.readableRecords(writeBuffer)
offsetOfLargestTimestamp = result.offsetOfMaxTimestamp, messages = retained) dest.append(firstOffset = retained.deepIterator().next().offset, largestTimestamp = result.maxTimestamp,
shallowOffsetOfMaxTimestamp = result.shallowOffsetOfMaxTimestamp, records = retained)
throttler.maybeThrottle(writeBuffer.limit) throttler.maybeThrottle(writeBuffer.limit)
} }
@ -488,21 +493,22 @@ private[log] class Cleaner(val id: Int,
private def shouldRetainMessage(source: kafka.log.LogSegment, private def shouldRetainMessage(source: kafka.log.LogSegment,
map: kafka.log.OffsetMap, map: kafka.log.OffsetMap,
retainDeletes: Boolean, retainDeletes: Boolean,
entry: kafka.message.MessageAndOffset, entry: LogEntry,
stats: CleanerStats): Boolean = { stats: CleanerStats): Boolean = {
val pastLatestOffset = entry.offset > map.latestOffset val pastLatestOffset = entry.offset > map.latestOffset
if (pastLatestOffset) if (pastLatestOffset)
return true return true
val key = entry.message.key
if (key != null) { if (entry.record.hasKey) {
val key = entry.record.key
val foundOffset = map.get(key) val foundOffset = map.get(key)
/* two cases in which we can get rid of a message: /* two cases in which we can get rid of a message:
* 1) if there exists a message with the same key but higher offset * 1) if there exists a message with the same key but higher offset
* 2) if the message is a delete "tombstone" marker and enough time has passed * 2) if the message is a delete "tombstone" marker and enough time has passed
*/ */
val redundant = foundOffset >= 0 && entry.offset < foundOffset val redundant = foundOffset >= 0 && entry.offset < foundOffset
val obsoleteDelete = !retainDeletes && entry.message.isNull val obsoleteDelete = !retainDeletes && entry.record.hasNullValue
!redundant && !obsoleteDelete !redundant && !obsoleteDelete
} else { } else {
stats.invalidMessage() stats.invalidMessage()
@ -620,12 +626,12 @@ private[log] class Cleaner(val id: Int,
checkDone(topicAndPartition) checkDone(topicAndPartition)
readBuffer.clear() readBuffer.clear()
segment.log.readInto(readBuffer, position) segment.log.readInto(readBuffer, position)
val messages = new ByteBufferMessageSet(readBuffer) val records = MemoryRecords.readableRecords(readBuffer)
throttler.maybeThrottle(messages.sizeInBytes) throttler.maybeThrottle(records.sizeInBytes)
val startPosition = position val startPosition = position
for (entry <- messages) { for (entry <- records.deepIterator.asScala) {
val message = entry.message val message = entry.record
if (message.hasKey && entry.offset >= start) { if (message.hasKey && entry.offset >= start) {
if (map.size < maxDesiredMapSize) if (map.size < maxDesiredMapSize)
map.put(message.key, entry.offset) map.put(message.key, entry.offset)
@ -634,8 +640,9 @@ private[log] class Cleaner(val id: Int,
} }
stats.indexMessagesRead(1) stats.indexMessagesRead(1)
} }
position += messages.validBytes val bytesRead = records.validBytes
stats.indexBytesRead(messages.validBytes) position += bytesRead
stats.indexBytesRead(bytesRead)
// if we didn't read even one complete message, our read buffer may be too small // if we didn't read even one complete message, our read buffer may be too small
if(position == startPosition) if(position == startPosition)

View File

@ -440,7 +440,7 @@ class LogManager(val logDirs: Array[File],
removedLog.dir = renamedDir removedLog.dir = renamedDir
// change the file pointers for log and index file // change the file pointers for log and index file
for (logSegment <- removedLog.logSegments) { for (logSegment <- removedLog.logSegments) {
logSegment.log.file = new File(renamedDir, logSegment.log.file.getName) logSegment.log.setFile(new File(renamedDir, logSegment.log.file.getName))
logSegment.index.file = new File(renamedDir, logSegment.index.file.getName) logSegment.index.file = new File(renamedDir, logSegment.index.file.getName)
} }

View File

@ -16,15 +16,20 @@
*/ */
package kafka.log package kafka.log
import kafka.message._ import java.io.{File, IOException}
import java.util.concurrent.TimeUnit
import kafka.common._ import kafka.common._
import kafka.utils._ import kafka.metrics.{KafkaMetricsGroup, KafkaTimer}
import kafka.server.{FetchDataInfo, LogOffsetMetadata} import kafka.server.{FetchDataInfo, LogOffsetMetadata}
import kafka.utils._
import org.apache.kafka.common.errors.CorruptRecordException import org.apache.kafka.common.errors.CorruptRecordException
import org.apache.kafka.common.record.FileRecords.LogEntryPosition
import org.apache.kafka.common.record._
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
import scala.collection.JavaConverters._
import scala.math._ import scala.math._
import java.io.{File, IOException}
/** /**
* A segment of the log. Each segment has two components: a log and an index. The log is a FileMessageSet containing * A segment of the log. Each segment has two components: a log and an index. The log is a FileMessageSet containing
@ -42,7 +47,7 @@ import java.io.{File, IOException}
* @param time The time instance * @param time The time instance
*/ */
@nonthreadsafe @nonthreadsafe
class LogSegment(val log: FileMessageSet, class LogSegment(val log: FileRecords,
val index: OffsetIndex, val index: OffsetIndex,
val timeIndex: TimeIndex, val timeIndex: TimeIndex,
val baseOffset: Long, val baseOffset: Long,
@ -63,7 +68,7 @@ class LogSegment(val log: FileMessageSet,
@volatile private var offsetOfMaxTimestamp = timeIndex.lastEntry.offset @volatile private var offsetOfMaxTimestamp = timeIndex.lastEntry.offset
def this(dir: File, startOffset: Long, indexIntervalBytes: Int, maxIndexSize: Int, rollJitterMs: Long, time: Time, fileAlreadyExists: Boolean = false, initFileSize: Int = 0, preallocate: Boolean = false) = def this(dir: File, startOffset: Long, indexIntervalBytes: Int, maxIndexSize: Int, rollJitterMs: Long, time: Time, fileAlreadyExists: Boolean = false, initFileSize: Int = 0, preallocate: Boolean = false) =
this(new FileMessageSet(file = Log.logFilename(dir, startOffset), fileAlreadyExists = fileAlreadyExists, initFileSize = initFileSize, preallocate = preallocate), this(FileRecords.open(Log.logFilename(dir, startOffset), fileAlreadyExists, initFileSize, preallocate),
new OffsetIndex(Log.indexFilename(dir, startOffset), baseOffset = startOffset, maxIndexSize = maxIndexSize), new OffsetIndex(Log.indexFilename(dir, startOffset), baseOffset = startOffset, maxIndexSize = maxIndexSize),
new TimeIndex(Log.timeIndexFilename(dir, startOffset), baseOffset = startOffset, maxIndexSize = maxIndexSize), new TimeIndex(Log.timeIndexFilename(dir, startOffset), baseOffset = startOffset, maxIndexSize = maxIndexSize),
startOffset, startOffset,
@ -82,23 +87,25 @@ class LogSegment(val log: FileMessageSet,
* *
* @param firstOffset The first offset in the message set. * @param firstOffset The first offset in the message set.
* @param largestTimestamp The largest timestamp in the message set. * @param largestTimestamp The largest timestamp in the message set.
* @param offsetOfLargestTimestamp The offset of the message that has the largest timestamp in the messages to append. * @param shallowOffsetOfMaxTimestamp The offset of the message that has the largest timestamp in the messages to append.
* @param messages The messages to append. * @param records The log entries to append.
*/ */
@nonthreadsafe @nonthreadsafe
def append(firstOffset: Long, largestTimestamp: Long, offsetOfLargestTimestamp: Long, messages: ByteBufferMessageSet) { def append(firstOffset: Long, largestTimestamp: Long, shallowOffsetOfMaxTimestamp: Long, records: MemoryRecords) {
if (messages.sizeInBytes > 0) { if (records.sizeInBytes > 0) {
trace("Inserting %d bytes at offset %d at position %d with largest timestamp %d at offset %d" trace("Inserting %d bytes at offset %d at position %d with largest timestamp %d at shallow offset %d"
.format(messages.sizeInBytes, firstOffset, log.sizeInBytes(), largestTimestamp, offsetOfLargestTimestamp)) .format(records.sizeInBytes, firstOffset, log.sizeInBytes(), largestTimestamp, shallowOffsetOfMaxTimestamp))
val physicalPosition = log.sizeInBytes() val physicalPosition = log.sizeInBytes()
if (physicalPosition == 0) if (physicalPosition == 0)
rollingBasedTimestamp = Some(largestTimestamp) rollingBasedTimestamp = Some(largestTimestamp)
// append the messages // append the messages
log.append(messages) val appendedBytes = log.append(records)
trace(s"Appended $appendedBytes to ${log.file()} at offset $firstOffset")
// Update the in memory max timestamp and corresponding offset. // Update the in memory max timestamp and corresponding offset.
if (largestTimestamp > maxTimestampSoFar) { if (largestTimestamp > maxTimestampSoFar) {
maxTimestampSoFar = largestTimestamp maxTimestampSoFar = largestTimestamp
offsetOfMaxTimestamp = offsetOfLargestTimestamp offsetOfMaxTimestamp = shallowOffsetOfMaxTimestamp
} }
// append an entry to the index (if needed) // append an entry to the index (if needed)
if(bytesSinceLastIndexEntry > indexIntervalBytes) { if(bytesSinceLastIndexEntry > indexIntervalBytes) {
@ -106,7 +113,7 @@ class LogSegment(val log: FileMessageSet,
timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestamp) timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestamp)
bytesSinceLastIndexEntry = 0 bytesSinceLastIndexEntry = 0
} }
bytesSinceLastIndexEntry += messages.sizeInBytes bytesSinceLastIndexEntry += records.sizeInBytes
} }
} }
@ -123,7 +130,7 @@ class LogSegment(val log: FileMessageSet,
* message or null if no message meets this criteria. * message or null if no message meets this criteria.
*/ */
@threadsafe @threadsafe
private[log] def translateOffset(offset: Long, startingFilePosition: Int = 0): (OffsetPosition, Int) = { private[log] def translateOffset(offset: Long, startingFilePosition: Int = 0): LogEntryPosition = {
val mapping = index.lookup(offset) val mapping = index.lookup(offset)
log.searchForOffsetWithSize(offset, max(mapping.position, startingFilePosition)) log.searchForOffsetWithSize(offset, max(mapping.position, startingFilePosition))
} }
@ -154,40 +161,40 @@ class LogSegment(val log: FileMessageSet,
if (startOffsetAndSize == null) if (startOffsetAndSize == null)
return null return null
val (startPosition, messageSetSize) = startOffsetAndSize val startPosition = startOffsetAndSize.position.toInt
val offsetMetadata = new LogOffsetMetadata(startOffset, this.baseOffset, startPosition.position) val offsetMetadata = new LogOffsetMetadata(startOffset, this.baseOffset, startPosition)
val adjustedMaxSize = val adjustedMaxSize =
if (minOneMessage) math.max(maxSize, messageSetSize) if (minOneMessage) math.max(maxSize, startOffsetAndSize.size)
else maxSize else maxSize
// return a log segment but with zero size in the case below // return a log segment but with zero size in the case below
if (adjustedMaxSize == 0) if (adjustedMaxSize == 0)
return FetchDataInfo(offsetMetadata, MessageSet.Empty) return FetchDataInfo(offsetMetadata, MemoryRecords.EMPTY)
// calculate the length of the message set to read based on whether or not they gave us a maxOffset // calculate the length of the message set to read based on whether or not they gave us a maxOffset
val length = maxOffset match { val length = maxOffset match {
case None => case None =>
// no max offset, just read until the max position // no max offset, just read until the max position
min((maxPosition - startPosition.position).toInt, adjustedMaxSize) min((maxPosition - startPosition).toInt, adjustedMaxSize)
case Some(offset) => case Some(offset) =>
// there is a max offset, translate it to a file position and use that to calculate the max read size; // there is a max offset, translate it to a file position and use that to calculate the max read size;
// when the leader of a partition changes, it's possible for the new leader's high watermark to be less than the // when the leader of a partition changes, it's possible for the new leader's high watermark to be less than the
// true high watermark in the previous leader for a short window. In this window, if a consumer fetches on an // true high watermark in the previous leader for a short window. In this window, if a consumer fetches on an
// offset between new leader's high watermark and the log end offset, we want to return an empty response. // offset between new leader's high watermark and the log end offset, we want to return an empty response.
if (offset < startOffset) if (offset < startOffset)
return FetchDataInfo(offsetMetadata, MessageSet.Empty, firstMessageSetIncomplete = false) return FetchDataInfo(offsetMetadata, MemoryRecords.EMPTY, firstEntryIncomplete = false)
val mapping = translateOffset(offset, startPosition.position) val mapping = translateOffset(offset, startPosition)
val endPosition = val endPosition =
if (mapping == null) if (mapping == null)
logSize // the max offset is off the end of the log, use the end of the file logSize // the max offset is off the end of the log, use the end of the file
else else
mapping._1.position mapping.position
min(min(maxPosition, endPosition) - startPosition.position, adjustedMaxSize).toInt min(min(maxPosition, endPosition) - startPosition, adjustedMaxSize).toInt
} }
FetchDataInfo(offsetMetadata, log.read(startPosition.position, length), FetchDataInfo(offsetMetadata, log.read(startPosition, length),
firstMessageSetIncomplete = adjustedMaxSize < messageSetSize) firstEntryIncomplete = adjustedMaxSize < startOffsetAndSize.size)
} }
/** /**
@ -205,16 +212,16 @@ class LogSegment(val log: FileMessageSet,
timeIndex.resize(timeIndex.maxIndexSize) timeIndex.resize(timeIndex.maxIndexSize)
var validBytes = 0 var validBytes = 0
var lastIndexEntry = 0 var lastIndexEntry = 0
val iter = log.iterator(maxMessageSize) val iter = log.shallowIterator(maxMessageSize)
maxTimestampSoFar = Message.NoTimestamp maxTimestampSoFar = Record.NO_TIMESTAMP
try { try {
while(iter.hasNext) { for (entry <- iter.asScala) {
val entry = iter.next val record = entry.record
entry.message.ensureValid() record.ensureValid()
// The max timestamp should have been put in the outer message, so we don't need to iterate over the inner messages. // The max timestamp should have been put in the outer message, so we don't need to iterate over the inner messages.
if (entry.message.timestamp > maxTimestampSoFar) { if (record.timestamp > maxTimestampSoFar) {
maxTimestampSoFar = entry.message.timestamp maxTimestampSoFar = record.timestamp
offsetOfMaxTimestamp = entry.offset offsetOfMaxTimestamp = entry.offset
} }
@ -225,11 +232,12 @@ class LogSegment(val log: FileMessageSet,
timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestamp) timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestamp)
lastIndexEntry = validBytes lastIndexEntry = validBytes
} }
validBytes += MessageSet.entrySize(entry.message) validBytes += entry.sizeInBytes()
} }
} catch { } catch {
case e: CorruptRecordException => case e: CorruptRecordException =>
logger.warn("Found invalid messages in log segment %s at byte offset %d: %s.".format(log.file.getAbsolutePath, validBytes, e.getMessage)) logger.warn("Found invalid messages in log segment %s at byte offset %d: %s."
.format(log.file.getAbsolutePath, validBytes, e.getMessage))
} }
val truncated = log.sizeInBytes - validBytes val truncated = log.sizeInBytes - validBytes
log.truncateTo(validBytes) log.truncateTo(validBytes)
@ -276,8 +284,7 @@ class LogSegment(val log: FileMessageSet,
// after truncation, reset and allocate more space for the (new currently active) index // after truncation, reset and allocate more space for the (new currently active) index
index.resize(index.maxIndexSize) index.resize(index.maxIndexSize)
timeIndex.resize(timeIndex.maxIndexSize) timeIndex.resize(timeIndex.maxIndexSize)
val (offsetPosition, _) = mapping val bytesTruncated = log.truncateTo(mapping.position.toInt)
val bytesTruncated = log.truncateTo(offsetPosition.position)
if(log.sizeInBytes == 0) { if(log.sizeInBytes == 0) {
created = time.milliseconds created = time.milliseconds
rollingBasedTimestamp = None rollingBasedTimestamp = None
@ -296,10 +303,10 @@ class LogSegment(val log: FileMessageSet,
@threadsafe @threadsafe
def nextOffset(): Long = { def nextOffset(): Long = {
val ms = read(index.lastOffset, None, log.sizeInBytes) val ms = read(index.lastOffset, None, log.sizeInBytes)
if(ms == null) { if (ms == null) {
baseOffset baseOffset
} else { } else {
ms.messageSet.lastOption match { ms.records.shallowIterator.asScala.toSeq.lastOption match {
case None => baseOffset case None => baseOffset
case Some(last) => last.nextOffset case Some(last) => last.nextOffset
} }
@ -360,9 +367,9 @@ class LogSegment(val log: FileMessageSet,
def timeWaitedForRoll(now: Long, messageTimestamp: Long) : Long = { def timeWaitedForRoll(now: Long, messageTimestamp: Long) : Long = {
// Load the timestamp of the first message into memory // Load the timestamp of the first message into memory
if (rollingBasedTimestamp.isEmpty) { if (rollingBasedTimestamp.isEmpty) {
val iter = log.iterator val iter = log.shallowIterator
if (iter.hasNext) if (iter.hasNext)
rollingBasedTimestamp = Some(iter.next.message.timestamp) rollingBasedTimestamp = Some(iter.next.record.timestamp)
} }
rollingBasedTimestamp match { rollingBasedTimestamp match {
case Some(t) if t >= 0 => messageTimestamp - t case Some(t) if t >= 0 => messageTimestamp - t
@ -394,8 +401,11 @@ class LogSegment(val log: FileMessageSet,
// Get the index entry with a timestamp less than or equal to the target timestamp // Get the index entry with a timestamp less than or equal to the target timestamp
val timestampOffset = timeIndex.lookup(timestamp) val timestampOffset = timeIndex.lookup(timestamp)
val position = index.lookup(timestampOffset.offset).position val position = index.lookup(timestampOffset.offset).position
// Search the timestamp // Search the timestamp
log.searchForTimestamp(timestamp, position) Option(log.searchForTimestamp(timestamp, position)).map { timestampAndOffset =>
TimestampOffset(timestampAndOffset.timestamp, timestampAndOffset.offset)
}
} }
/** /**
@ -444,3 +454,7 @@ class LogSegment(val log: FileMessageSet,
timeIndex.file.setLastModified(ms) timeIndex.file.setLastModified(ms)
} }
} }
object LogFlushStats extends KafkaMetricsGroup {
val logFlushTimer = new KafkaTimer(newTimer("LogFlushRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS))
}

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 java.nio.ByteBuffer
import kafka.common.InvalidOffsetException import kafka.common.InvalidOffsetException
import kafka.message.Message
import kafka.utils.CoreUtils._ import kafka.utils.CoreUtils._
import kafka.utils.Logging import kafka.utils.Logging
import org.apache.kafka.common.record.Record
/** /**
* An index that maps from the timestamp to the logical offsets of the messages in a segment. This index might be * An index that maps from the timestamp to the logical offsets of the messages in a segment. This index might be
@ -69,7 +69,7 @@ class TimeIndex(file: File,
def lastEntry: TimestampOffset = { def lastEntry: TimestampOffset = {
inLock(lock) { inLock(lock) {
_entries match { _entries match {
case 0 => TimestampOffset(Message.NoTimestamp, baseOffset) case 0 => TimestampOffset(Record.NO_TIMESTAMP, baseOffset)
case s => parseEntry(mmap, s - 1).asInstanceOf[TimestampOffset] case s => parseEntry(mmap, s - 1).asInstanceOf[TimestampOffset]
} }
} }
@ -145,7 +145,7 @@ class TimeIndex(file: File,
val idx = mmap.duplicate val idx = mmap.duplicate
val slot = indexSlotFor(idx, targetTimestamp, IndexSearchType.KEY) val slot = indexSlotFor(idx, targetTimestamp, IndexSearchType.KEY)
if (slot == -1) if (slot == -1)
TimestampOffset(Message.NoTimestamp, baseOffset) TimestampOffset(Record.NO_TIMESTAMP, baseOffset)
else { else {
val entry = parseEntry(idx, slot).asInstanceOf[TimestampOffset] val entry = parseEntry(idx, slot).asInstanceOf[TimestampOffset]
TimestampOffset(entry.timestamp, entry.offset) TimestampOffset(entry.timestamp, entry.offset)

View File

@ -17,20 +17,13 @@
package kafka.message package kafka.message
import kafka.utils.{CoreUtils, IteratorTemplate, Logging}
import kafka.common.{KafkaException, LongRef}
import java.nio.ByteBuffer import java.nio.ByteBuffer
import java.nio.channels._
import java.io._
import java.util.ArrayDeque
import kafka.message.ByteBufferMessageSet.FilterResult import kafka.common.LongRef
import org.apache.kafka.common.errors.InvalidTimestampException import kafka.utils.Logging
import org.apache.kafka.common.record.{MemoryRecords, TimestampType} import org.apache.kafka.common.record._
import org.apache.kafka.common.utils.Utils
import scala.collection.mutable import scala.collection.JavaConverters._
import scala.collection.mutable.ArrayBuffer
object ByteBufferMessageSet { object ByteBufferMessageSet {
@ -41,204 +34,19 @@ object ByteBufferMessageSet {
messages: Message*): ByteBuffer = { messages: Message*): ByteBuffer = {
if (messages.isEmpty) if (messages.isEmpty)
MessageSet.Empty.buffer MessageSet.Empty.buffer
else if (compressionCodec == NoCompressionCodec) { else {
val buffer = ByteBuffer.allocate(MessageSet.messageSetSize(messages))
for (message <- messages) writeMessage(buffer, message, offsetAssigner.nextAbsoluteOffset())
buffer.rewind()
buffer
} else {
val magicAndTimestamp = wrapperMessageTimestamp match { val magicAndTimestamp = wrapperMessageTimestamp match {
case Some(ts) => MagicAndTimestamp(messages.head.magic, ts) case Some(ts) => MagicAndTimestamp(messages.head.magic, ts)
case None => MessageSet.magicAndLargestTimestamp(messages) case None => MessageSet.magicAndLargestTimestamp(messages)
} }
val (messageWriter, lastOffset) = writeCompressedMessages(compressionCodec, offsetAssigner, magicAndTimestamp,
timestampType, messages) val entries = messages.map(message => LogEntry.create(offsetAssigner.nextAbsoluteOffset(), message.asRecord))
val buffer = ByteBuffer.allocate(messageWriter.size + MessageSet.LogOverhead) val builder = MemoryRecords.builderWithEntries(timestampType, CompressionType.forId(compressionCodec.codec),
writeMessage(buffer, messageWriter, lastOffset) magicAndTimestamp.timestamp, entries.asJava)
buffer.rewind() builder.build().buffer
buffer
} }
} }
/** Deep iterator that decompresses the message sets and adjusts timestamp and offset if needed. */
def deepIterator(wrapperMessageAndOffset: MessageAndOffset, ensureMatchingMagic: Boolean = false): Iterator[MessageAndOffset] = {
import Message._
new IteratorTemplate[MessageAndOffset] {
val MessageAndOffset(wrapperMessage, wrapperMessageOffset) = wrapperMessageAndOffset
if (wrapperMessage.payload == null)
throw new KafkaException(s"Message payload is null: $wrapperMessage")
val wrapperMessageTimestampOpt: Option[Long] =
if (wrapperMessage.magic > MagicValue_V0) Some(wrapperMessage.timestamp) else None
val wrapperMessageTimestampTypeOpt: Option[TimestampType] =
if (wrapperMessage.magic > MagicValue_V0) Some(wrapperMessage.timestampType) else None
var lastInnerOffset = -1L
val messageAndOffsets = {
val inputStream = new ByteBufferBackedInputStream(wrapperMessage.payload)
val compressed = try {
new DataInputStream(CompressionFactory(wrapperMessage.compressionCodec, wrapperMessage.magic, inputStream))
} catch {
case ioe: IOException =>
throw new InvalidMessageException(s"Failed to instantiate input stream compressed with ${wrapperMessage.compressionCodec}", ioe)
}
val innerMessageAndOffsets = new ArrayDeque[MessageAndOffset]()
try {
while (true)
innerMessageAndOffsets.add(readMessageFromStream(compressed))
} catch {
case _: EOFException =>
// we don't do anything at all here, because the finally
// will close the compressed input stream, and we simply
// want to return the innerMessageAndOffsets
case ioe: IOException =>
throw new InvalidMessageException(s"Error while reading message from stream compressed with ${wrapperMessage.compressionCodec}", ioe)
} finally {
CoreUtils.swallow(compressed.close())
}
innerMessageAndOffsets
}
private def readMessageFromStream(compressed: DataInputStream): MessageAndOffset = {
val innerOffset = compressed.readLong()
val recordSize = compressed.readInt()
if (recordSize < MinMessageOverhead)
throw new InvalidMessageException(s"Message found with corrupt size `$recordSize` in deep iterator")
// read the record into an intermediate record buffer (i.e. extra copy needed)
val bufferArray = new Array[Byte](recordSize)
compressed.readFully(bufferArray, 0, recordSize)
val buffer = ByteBuffer.wrap(bufferArray)
// Override the timestamp if necessary
val newMessage = new Message(buffer, wrapperMessageTimestampOpt, wrapperMessageTimestampTypeOpt)
// Due to KAFKA-4298, it is possible for the inner and outer magic values to differ. We ignore
// this and depend on the outer message in order to decide how to compute the respective offsets
// for the inner messages
if (ensureMatchingMagic && newMessage.magic != wrapperMessage.magic)
throw new InvalidMessageException(s"Compressed message has magic value ${wrapperMessage.magic} " +
s"but inner message has magic value ${newMessage.magic}")
lastInnerOffset = innerOffset
MessageAndOffset(newMessage, innerOffset)
}
override def makeNext(): MessageAndOffset = {
messageAndOffsets.pollFirst() match {
case null => allDone()
case nextMessage@ MessageAndOffset(message, offset) =>
if (wrapperMessage.magic > MagicValue_V0) {
val relativeOffset = offset - lastInnerOffset
val absoluteOffset = wrapperMessageOffset + relativeOffset
MessageAndOffset(message, absoluteOffset)
} else {
nextMessage
}
}
}
}
}
private def writeCompressedMessages(codec: CompressionCodec,
offsetAssigner: OffsetAssigner,
magicAndTimestamp: MagicAndTimestamp,
timestampType: TimestampType,
messages: Seq[Message]): (MessageWriter, Long) = {
require(codec != NoCompressionCodec, s"compressionCodec must not be $NoCompressionCodec")
require(messages.nonEmpty, "cannot write empty compressed message set")
var offset = -1L
val magic = magicAndTimestamp.magic
val messageWriter = new MessageWriter(math.min(math.max(MessageSet.messageSetSize(messages) / 2, 1024), 1 << 16))
messageWriter.write(
codec = codec,
timestamp = magicAndTimestamp.timestamp,
timestampType = timestampType,
magicValue = magic) { outputStream =>
val output = new DataOutputStream(CompressionFactory(codec, magic, outputStream))
try {
for (message <- messages) {
offset = offsetAssigner.nextAbsoluteOffset()
if (message.magic != magicAndTimestamp.magic)
throw new IllegalArgumentException("Messages in the message set must have same magic value")
// Use inner offset if magic value is greater than 0
val innerOffset = if (magicAndTimestamp.magic > Message.MagicValue_V0)
offsetAssigner.toInnerOffset(offset)
else
offset
output.writeLong(innerOffset)
output.writeInt(message.size)
output.write(message.buffer.array, message.buffer.arrayOffset, message.buffer.limit)
}
} finally {
output.close()
}
}
(messageWriter, offset)
}
private[kafka] def writeCompressedMessages(buffer: ByteBuffer,
codec: CompressionCodec,
messageAndOffsets: Seq[MessageAndOffset]): Int = {
require(codec != NoCompressionCodec, s"compressionCodec must not be $NoCompressionCodec")
if (messageAndOffsets.isEmpty)
0
else {
val messages = messageAndOffsets.map(_.message)
val magicAndTimestamp = MessageSet.magicAndLargestTimestamp(messages)
// ensure that we use the magic from the first message in the set when writing the wrapper
// message in order to fix message sets corrupted by KAFKA-4298
val magic = magicAndTimestamp.magic
val firstMessageAndOffset = messageAndOffsets.head
val firstAbsoluteOffset = firstMessageAndOffset.offset
val offsetAssigner = OffsetAssigner(firstAbsoluteOffset, magic, messageAndOffsets)
val timestampType = firstMessageAndOffset.message.timestampType
val (messageWriter, lastOffset) = writeCompressedMessages(codec, offsetAssigner, magicAndTimestamp,
timestampType, messages)
writeMessage(buffer, messageWriter, lastOffset)
messageWriter.size + MessageSet.LogOverhead
}
}
private[kafka] def writeMessage(buffer: ByteBuffer, message: Message, offset: Long) {
buffer.putLong(offset)
buffer.putInt(message.size)
buffer.put(message.buffer)
message.buffer.rewind()
}
private[kafka] def writeMessage(buffer: ByteBuffer, messageWriter: MessageWriter, offset: Long) {
buffer.putLong(offset)
buffer.putInt(messageWriter.size)
messageWriter.writeTo(buffer)
}
case class FilterResult(messagesRead: Int,
bytesRead: Int,
messagesRetained: Int,
bytesRetained: Int,
maxTimestamp: Long,
offsetOfMaxTimestamp: Long)
} }
private object OffsetAssigner { private object OffsetAssigner {
@ -246,9 +54,6 @@ private object OffsetAssigner {
def apply(offsetCounter: LongRef, size: Int): OffsetAssigner = def apply(offsetCounter: LongRef, size: Int): OffsetAssigner =
new OffsetAssigner(offsetCounter.value to offsetCounter.addAndGet(size)) new OffsetAssigner(offsetCounter.value to offsetCounter.addAndGet(size))
def apply(baseOffset: Long, magic: Byte, messageAndOffsets: Seq[MessageAndOffset]): OffsetAssigner =
new OffsetAssigner(messageAndOffsets.map(_.offset))
} }
private class OffsetAssigner(offsets: Seq[Long]) { private class OffsetAssigner(offsets: Seq[Long]) {
@ -322,7 +127,6 @@ private class OffsetAssigner(offsets: Seq[Long]) {
* *
*/ */
class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Logging { class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Logging {
private var shallowValidByteCount = -1
private[kafka] def this(compressionCodec: CompressionCodec, private[kafka] def this(compressionCodec: CompressionCodec,
offsetCounter: LongRef, offsetCounter: LongRef,
@ -354,33 +158,6 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi
override def asRecords: MemoryRecords = MemoryRecords.readableRecords(buffer.duplicate()) override def asRecords: MemoryRecords = MemoryRecords.readableRecords(buffer.duplicate())
private def shallowValidBytes: Int = {
if (shallowValidByteCount < 0) {
this.shallowValidByteCount = this.internalIterator(isShallow = true).map { messageAndOffset =>
MessageSet.entrySize(messageAndOffset.message)
}.sum
}
shallowValidByteCount
}
/** Write the messages in this set to the given channel */
def writeFullyTo(channel: GatheringByteChannel): Int = {
buffer.mark()
var written = 0
while (written < sizeInBytes)
written += channel.write(buffer)
buffer.reset()
written
}
override def isMagicValueInAllWrapperMessages(expectedMagicValue: Byte): Boolean = {
for (messageAndOffset <- shallowIterator) {
if (messageAndOffset.message.magic != expectedMagicValue)
return false
}
true
}
/** default iterator that iterates over decompressed messages */ /** default iterator that iterates over decompressed messages */
override def iterator: Iterator[MessageAndOffset] = internalIterator() override def iterator: Iterator[MessageAndOffset] = internalIterator()
@ -388,365 +165,12 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi
def shallowIterator: Iterator[MessageAndOffset] = internalIterator(isShallow = true) def shallowIterator: Iterator[MessageAndOffset] = internalIterator(isShallow = true)
/** When flag isShallow is set to be true, we do a shallow iteration: just traverse the first level of messages. **/ /** When flag isShallow is set to be true, we do a shallow iteration: just traverse the first level of messages. **/
private def internalIterator(isShallow: Boolean = false, ensureMatchingMagic: Boolean = false): Iterator[MessageAndOffset] = { private def internalIterator(isShallow: Boolean = false): Iterator[MessageAndOffset] = {
new IteratorTemplate[MessageAndOffset] { val entries = if (isShallow)
var topIter = buffer.slice() asRecords.shallowIterator
var innerIter: Iterator[MessageAndOffset] = null else
asRecords.deepIterator
def innerDone(): Boolean = (innerIter == null || !innerIter.hasNext) entries.asScala.map(MessageAndOffset.fromLogEntry)
def makeNextOuter: MessageAndOffset = {
// if there isn't at least an offset and size, we are done
if (topIter.remaining < 12)
return allDone()
val offset = topIter.getLong()
val size = topIter.getInt()
if(size < Message.MinMessageOverhead)
throw new InvalidMessageException("Message found with corrupt size (" + size + ") in shallow iterator")
// we have an incomplete message
if(topIter.remaining < size)
return allDone()
// read the current message and check correctness
val message = topIter.slice()
message.limit(size)
topIter.position(topIter.position + size)
val newMessage = new Message(message)
if(isShallow) {
MessageAndOffset(newMessage, offset)
} else {
newMessage.compressionCodec match {
case NoCompressionCodec =>
innerIter = null
MessageAndOffset(newMessage, offset)
case _ =>
innerIter = ByteBufferMessageSet.deepIterator(MessageAndOffset(newMessage, offset), ensureMatchingMagic)
if(!innerIter.hasNext)
innerIter = null
makeNext()
}
}
}
override def makeNext(): MessageAndOffset = {
if(isShallow){
makeNextOuter
} else {
if(innerDone())
makeNextOuter
else
innerIter.next()
}
}
}
}
def filterInto(buffer: ByteBuffer,
filter: MessageAndOffset => Boolean): FilterResult = {
var maxTimestamp = Message.NoTimestamp
var offsetOfMaxTimestamp = -1L
var messagesRead = 0
var bytesRead = 0
var messagesRetained = 0
var bytesRetained = 0
for (shallowMessageAndOffset <- shallowIterator) {
val shallowMessage = shallowMessageAndOffset.message
val shallowOffset = shallowMessageAndOffset.offset
val size = MessageSet.entrySize(shallowMessageAndOffset.message)
messagesRead += 1
bytesRead += size
if (shallowMessageAndOffset.message.compressionCodec == NoCompressionCodec) {
if (filter(shallowMessageAndOffset)) {
ByteBufferMessageSet.writeMessage(buffer, shallowMessage, shallowOffset)
messagesRetained += 1
bytesRetained += size
if (shallowMessage.timestamp > maxTimestamp) {
maxTimestamp = shallowMessage.timestamp
offsetOfMaxTimestamp = shallowOffset
}
}
messagesRead += 1
} else {
// We use the absolute offset to decide whether to retain the message or not (this is handled by the
// deep iterator). Because of KAFKA-4298, we have to allow for the possibility that a previous version
// corrupted the log by writing a compressed message set with a wrapper magic value not matching the magic
// of the inner messages. This will be fixed as we recopy the messages to the destination segment.
var writeOriginalMessageSet = true
val retainedMessages = ArrayBuffer[MessageAndOffset]()
val shallowMagic = shallowMessage.magic
for (deepMessageAndOffset <- ByteBufferMessageSet.deepIterator(shallowMessageAndOffset)) {
messagesRead += 1
if (filter(deepMessageAndOffset)) {
// Check for log corruption due to KAFKA-4298. If we find it, make sure that we overwrite
// the corrupted entry with correct data.
if (shallowMagic != deepMessageAndOffset.message.magic)
writeOriginalMessageSet = false
retainedMessages += deepMessageAndOffset
// We need the max timestamp and last offset for time index
if (deepMessageAndOffset.message.timestamp > maxTimestamp)
maxTimestamp = deepMessageAndOffset.message.timestamp
}
else writeOriginalMessageSet = false
}
offsetOfMaxTimestamp = if (retainedMessages.nonEmpty) retainedMessages.last.offset else -1L
// There are no messages compacted out and no message format conversion, write the original message set back
if (writeOriginalMessageSet)
ByteBufferMessageSet.writeMessage(buffer, shallowMessage, shallowOffset)
else if (retainedMessages.nonEmpty) {
val compressedSize = ByteBufferMessageSet.writeCompressedMessages(buffer, shallowMessage.compressionCodec, retainedMessages)
messagesRetained += 1
bytesRetained += compressedSize
}
}
}
FilterResult(messagesRead, bytesRead, messagesRetained, bytesRetained, maxTimestamp, offsetOfMaxTimestamp)
}
/**
* Update the offsets for this message set and do further validation on messages including:
* 1. Messages for compacted topics must have keys
* 2. When magic value = 1, inner messages of a compressed message set must have monotonically increasing offsets
* starting from 0.
* 3. When magic value = 1, validate and maybe overwrite timestamps of messages.
*
* This method will convert the messages in the following scenarios:
* A. Magic value of a message = 0 and messageFormatVersion is 1
* B. Magic value of a message = 1 and messageFormatVersion is 0
*
* If no format conversion or value overwriting is required for messages, this method will perform in-place
* operations and avoid re-compression.
*
* Returns a ValidationAndOffsetAssignResult containing the validated message set, maximum timestamp, the offset
* of the shallow message with the max timestamp and a boolean indicating whether the message sizes may have changed.
*/
private[kafka] def validateMessagesAndAssignOffsets(offsetCounter: LongRef,
now: Long,
sourceCodec: CompressionCodec,
targetCodec: CompressionCodec,
compactedTopic: Boolean = false,
messageFormatVersion: Byte = Message.CurrentMagicValue,
messageTimestampType: TimestampType,
messageTimestampDiffMaxMs: Long): ValidationAndOffsetAssignResult = {
if (sourceCodec == NoCompressionCodec && targetCodec == NoCompressionCodec) {
// check the magic value
if (!isMagicValueInAllWrapperMessages(messageFormatVersion))
convertNonCompressedMessages(offsetCounter, compactedTopic, now, messageTimestampType, messageTimestampDiffMaxMs,
messageFormatVersion)
else
// Do in-place validation, offset assignment and maybe set timestamp
validateNonCompressedMessagesAndAssignOffsetInPlace(offsetCounter, now, compactedTopic, messageTimestampType,
messageTimestampDiffMaxMs)
} else {
// Deal with compressed messages
// We cannot do in place assignment in one of the following situations:
// 1. Source and target compression codec are different
// 2. When magic value to use is 0 because offsets need to be overwritten
// 3. When magic value to use is above 0, but some fields of inner messages need to be overwritten.
// 4. Message format conversion is needed.
// No in place assignment situation 1 and 2
var inPlaceAssignment = sourceCodec == targetCodec && messageFormatVersion > Message.MagicValue_V0
var maxTimestamp = Message.NoTimestamp
var offsetOfMaxTimestamp = -1L
val expectedInnerOffset = new LongRef(0)
val validatedMessages = new mutable.ArrayBuffer[Message]
this.internalIterator(isShallow = false, ensureMatchingMagic = true).foreach { messageAndOffset =>
val message = messageAndOffset.message
validateMessageKey(message, compactedTopic)
if (message.magic > Message.MagicValue_V0 && messageFormatVersion > Message.MagicValue_V0) {
// No in place assignment situation 3
// Validate the timestamp
validateTimestamp(message, now, messageTimestampType, messageTimestampDiffMaxMs)
// Check if we need to overwrite offset
if (messageAndOffset.offset != expectedInnerOffset.getAndIncrement())
inPlaceAssignment = false
if (message.timestamp > maxTimestamp) {
maxTimestamp = message.timestamp
offsetOfMaxTimestamp = offsetCounter.value + expectedInnerOffset.value - 1
}
}
if (sourceCodec != NoCompressionCodec && message.compressionCodec != NoCompressionCodec)
throw new InvalidMessageException("Compressed outer message should not have an inner message with a " +
s"compression attribute set: $message")
// No in place assignment situation 4
if (message.magic != messageFormatVersion)
inPlaceAssignment = false
validatedMessages += message.toFormatVersion(messageFormatVersion)
}
if (!inPlaceAssignment) {
// Cannot do in place assignment.
val (largestTimestampOfMessageSet, offsetOfMaxTimestampInMessageSet) = {
if (messageFormatVersion == Message.MagicValue_V0)
(Some(Message.NoTimestamp), -1L)
else if (messageTimestampType == TimestampType.CREATE_TIME)
(Some(maxTimestamp), {if (targetCodec == NoCompressionCodec) offsetOfMaxTimestamp else offsetCounter.value + validatedMessages.length - 1})
else // Log append time
(Some(now), {if (targetCodec == NoCompressionCodec) offsetCounter.value else offsetCounter.value + validatedMessages.length - 1})
}
ValidationAndOffsetAssignResult(validatedMessages = new ByteBufferMessageSet(compressionCodec = targetCodec,
offsetCounter = offsetCounter,
wrapperMessageTimestamp = largestTimestampOfMessageSet,
timestampType = messageTimestampType,
messages = validatedMessages: _*),
maxTimestamp = largestTimestampOfMessageSet.get,
offsetOfMaxTimestamp = offsetOfMaxTimestampInMessageSet,
messageSizeMaybeChanged = true)
} else {
// Do not do re-compression but simply update the offset, timestamp and attributes field of the wrapper message.
buffer.putLong(0, offsetCounter.addAndGet(validatedMessages.size) - 1)
// validate the messages
validatedMessages.foreach(_.ensureValid())
var crcUpdateNeeded = true
val timestampOffset = MessageSet.LogOverhead + Message.TimestampOffset
val attributeOffset = MessageSet.LogOverhead + Message.AttributesOffset
val timestamp = buffer.getLong(timestampOffset)
val attributes = buffer.get(attributeOffset)
buffer.putLong(timestampOffset, maxTimestamp)
if (messageTimestampType == TimestampType.CREATE_TIME && timestamp == maxTimestamp)
// We don't need to recompute crc if the timestamp is not updated.
crcUpdateNeeded = false
else if (messageTimestampType == TimestampType.LOG_APPEND_TIME) {
// Set timestamp type and timestamp
buffer.putLong(timestampOffset, now)
buffer.put(attributeOffset, messageTimestampType.updateAttributes(attributes))
}
if (crcUpdateNeeded) {
// need to recompute the crc value
buffer.position(MessageSet.LogOverhead)
val wrapperMessage = new Message(buffer.slice())
Utils.writeUnsignedInt(buffer, MessageSet.LogOverhead + Message.CrcOffset, wrapperMessage.computeChecksum)
}
buffer.rewind()
// For compressed messages,
ValidationAndOffsetAssignResult(validatedMessages = this,
maxTimestamp = buffer.getLong(timestampOffset),
offsetOfMaxTimestamp = buffer.getLong(0),
messageSizeMaybeChanged = false)
}
}
}
// We create this method to avoid a memory copy. It reads from the original message set and directly
// writes the converted messages into new message set buffer. Hence we don't need to allocate memory for each
// individual message during message format conversion.
private def convertNonCompressedMessages(offsetCounter: LongRef,
compactedTopic: Boolean,
now: Long,
timestampType: TimestampType,
messageTimestampDiffMaxMs: Long,
toMagicValue: Byte): ValidationAndOffsetAssignResult = {
val sizeInBytesAfterConversion = shallowValidBytes + this.internalIterator(isShallow = true).map { messageAndOffset =>
Message.headerSizeDiff(messageAndOffset.message.magic, toMagicValue)
}.sum
val newBuffer = ByteBuffer.allocate(sizeInBytesAfterConversion)
var newMessagePosition = 0
var maxTimestamp = Message.NoTimestamp
var offsetOfMaxTimestamp = -1L
this.internalIterator(isShallow = true).foreach { case MessageAndOffset(message, _) =>
validateMessageKey(message, compactedTopic)
validateTimestamp(message, now, timestampType, messageTimestampDiffMaxMs)
newBuffer.position(newMessagePosition)
newBuffer.putLong(offsetCounter.getAndIncrement())
val newMessageSize = message.size + Message.headerSizeDiff(message.magic, toMagicValue)
newBuffer.putInt(newMessageSize)
val newMessageBuffer = newBuffer.slice()
newMessageBuffer.limit(newMessageSize)
message.convertToBuffer(toMagicValue, newMessageBuffer, now, timestampType)
if (toMagicValue > Message.MagicValue_V0) {
val timestamp = newMessageBuffer.getLong(Message.TimestampOffset)
if (maxTimestamp < timestamp) {
maxTimestamp = timestamp
offsetOfMaxTimestamp = offsetCounter.value - 1
}
}
newMessagePosition += MessageSet.LogOverhead + newMessageSize
}
newBuffer.rewind()
new ValidationAndOffsetAssignResult(validatedMessages = new ByteBufferMessageSet(newBuffer),
maxTimestamp = maxTimestamp,
offsetOfMaxTimestamp = offsetOfMaxTimestamp,
messageSizeMaybeChanged = true)
}
private def validateNonCompressedMessagesAndAssignOffsetInPlace(offsetCounter: LongRef,
now: Long,
compactedTopic: Boolean,
timestampType: TimestampType,
timestampDiffMaxMs: Long): ValidationAndOffsetAssignResult = {
// do in-place validation and offset assignment
var messagePosition = 0
var maxTimestamp = Message.NoTimestamp
var offsetOfMaxTimestamp = -1L
buffer.mark()
while (messagePosition < sizeInBytes - MessageSet.LogOverhead) {
buffer.position(messagePosition)
buffer.putLong(offsetCounter.getAndIncrement())
val messageSize = buffer.getInt()
val messageBuffer = buffer.slice()
messageBuffer.limit(messageSize)
val message = new Message(messageBuffer)
validateMessageKey(message, compactedTopic)
if (message.magic > Message.MagicValue_V0) {
validateTimestamp(message, now, timestampType, timestampDiffMaxMs)
if (timestampType == TimestampType.LOG_APPEND_TIME) {
message.buffer.putLong(Message.TimestampOffset, now)
message.buffer.put(Message.AttributesOffset, timestampType.updateAttributes(message.attributes))
Utils.writeUnsignedInt(message.buffer, Message.CrcOffset, message.computeChecksum)
}
if (message.timestamp > maxTimestamp) {
maxTimestamp = message.timestamp
offsetOfMaxTimestamp = offsetCounter.value - 1
}
}
messagePosition += MessageSet.LogOverhead + messageSize
}
buffer.reset()
ValidationAndOffsetAssignResult(validatedMessages = this,
maxTimestamp = maxTimestamp,
offsetOfMaxTimestamp = offsetOfMaxTimestamp,
messageSizeMaybeChanged = false)
}
private def validateMessageKey(message: Message, compactedTopic: Boolean) {
if (compactedTopic && !message.hasKey)
throw new InvalidMessageException("Compacted topic cannot accept message without key.")
}
/**
* This method validates the timestamps of a message.
* If the message is using create time, this method checks if it is within acceptable range.
*/
private def validateTimestamp(message: Message,
now: Long,
timestampType: TimestampType,
timestampDiffMaxMs: Long) {
if (timestampType == TimestampType.CREATE_TIME && math.abs(message.timestamp - now) > timestampDiffMaxMs)
throw new InvalidTimestampException(s"Timestamp ${message.timestamp} of message is out of range. " +
s"The timestamp should be within [${now - timestampDiffMaxMs}, ${now + timestampDiffMaxMs}")
if (message.timestampType == TimestampType.LOG_APPEND_TIME)
throw new InvalidTimestampException(s"Invalid timestamp type in message $message. Producer should not set " +
s"timestamp type to LogAppendTime.")
} }
/** /**
@ -757,7 +181,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi
/** /**
* The total number of bytes in this message set not including any partial, trailing messages * The total number of bytes in this message set not including any partial, trailing messages
*/ */
def validBytes: Int = shallowValidBytes def validBytes: Int = asRecords.validBytes
/** /**
* Two message sets are equal if their respective byte buffers are equal * Two message sets are equal if their respective byte buffers are equal
@ -773,8 +197,3 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi
override def hashCode: Int = buffer.hashCode override def hashCode: Int = buffer.hashCode
} }
case class ValidationAndOffsetAssignResult(validatedMessages: ByteBufferMessageSet,
maxTimestamp: Long,
offsetOfMaxTimestamp: Long,
messageSizeMaybeChanged: Boolean)

View File

@ -19,7 +19,7 @@ package kafka.message
import java.nio._ import java.nio._
import org.apache.kafka.common.record.TimestampType import org.apache.kafka.common.record.{Record, TimestampType}
import scala.math._ import scala.math._
import kafka.utils._ import kafka.utils._
@ -98,6 +98,11 @@ object Message {
MessageHeaderSizeMap(toMagicValue) - MessageHeaderSizeMap(fromMagicValue) MessageHeaderSizeMap(toMagicValue) - MessageHeaderSizeMap(fromMagicValue)
def fromRecord(record: Record): Message = {
val wrapperTimestamp: Option[Long] = if (record.wrapperRecordTimestamp() == null) None else Some(record.wrapperRecordTimestamp())
val wrapperTimestampType = Option(record.wrapperRecordTimestampType())
new Message(record.buffer, wrapperTimestamp, wrapperTimestampType)
}
} }
/** /**
@ -134,6 +139,15 @@ class Message(val buffer: ByteBuffer,
import kafka.message.Message._ import kafka.message.Message._
private[message] def asRecord: Record = {
wrapperMessageTimestamp match {
case None => new Record(buffer)
case Some(timestamp) =>
val timestampType = wrapperMessageTimestampType.orNull
new Record(buffer, timestamp, timestampType)
}
}
/** /**
* A constructor to create a Message * A constructor to create a Message
* @param bytes The payload of the message * @param bytes The payload of the message
@ -327,52 +341,6 @@ class Message(val buffer: ByteBuffer,
*/ */
def key: ByteBuffer = sliceDelimited(keySizeOffset) def key: ByteBuffer = sliceDelimited(keySizeOffset)
/**
* convert the message to specified format
*/
def toFormatVersion(toMagicValue: Byte): Message = {
if (magic == toMagicValue)
this
else {
val byteBuffer = ByteBuffer.allocate(size + Message.headerSizeDiff(magic, toMagicValue))
// Copy bytes from old messages to new message
convertToBuffer(toMagicValue, byteBuffer, Message.NoTimestamp)
new Message(byteBuffer)
}
}
def convertToBuffer(toMagicValue: Byte,
byteBuffer: ByteBuffer,
now: Long,
timestampType: TimestampType = wrapperMessageTimestampType.getOrElse(TimestampType.forAttributes(attributes))) {
if (byteBuffer.remaining() < size + headerSizeDiff(magic, toMagicValue))
throw new IndexOutOfBoundsException("The byte buffer does not have enough capacity to hold new message format " +
s"version $toMagicValue")
if (toMagicValue == Message.MagicValue_V1) {
// Up-conversion, reserve CRC and update magic byte
byteBuffer.position(Message.MagicOffset)
byteBuffer.put(Message.MagicValue_V1)
byteBuffer.put(timestampType.updateAttributes(attributes))
// Up-conversion, insert the timestamp field
if (timestampType == TimestampType.LOG_APPEND_TIME)
byteBuffer.putLong(now)
else
byteBuffer.putLong(Message.NoTimestamp)
byteBuffer.put(buffer.array(), buffer.arrayOffset() + Message.KeySizeOffset_V0, size - Message.KeySizeOffset_V0)
} else {
// Down-conversion, reserve CRC and update magic byte
byteBuffer.position(Message.MagicOffset)
byteBuffer.put(Message.MagicValue_V0)
byteBuffer.put(TimestampType.CREATE_TIME.updateAttributes(attributes))
// Down-conversion, skip the timestamp field
byteBuffer.put(buffer.array(), buffer.arrayOffset() + Message.KeySizeOffset_V1, size - Message.KeySizeOffset_V1)
}
// update crc value
val newMessage = new Message(byteBuffer)
Utils.writeUnsignedInt(byteBuffer, Message.CrcOffset, newMessage.computeChecksum)
byteBuffer.rewind()
}
/** /**
* Read a size-delimited byte buffer starting at the given offset * Read a size-delimited byte buffer starting at the given offset
*/ */
@ -399,7 +367,7 @@ class Message(val buffer: ByteBuffer,
if (timestamp < 0 && timestamp != NoTimestamp) if (timestamp < 0 && timestamp != NoTimestamp)
throw new IllegalArgumentException(s"Invalid message timestamp $timestamp") throw new IllegalArgumentException(s"Invalid message timestamp $timestamp")
if (magic == MagicValue_V0 && timestamp != NoTimestamp) if (magic == MagicValue_V0 && timestamp != NoTimestamp)
throw new IllegalArgumentException(s"Invalid timestamp $timestamp. Timestamp must be ${NoTimestamp} when magic = ${MagicValue_V0}") throw new IllegalArgumentException(s"Invalid timestamp $timestamp. Timestamp must be $NoTimestamp when magic = $MagicValue_V0")
} }
override def toString: String = { override def toString: String = {

View File

@ -17,6 +17,13 @@
package kafka.message package kafka.message
import org.apache.kafka.common.record.LogEntry
object MessageAndOffset {
def fromLogEntry(logEntry : LogEntry): MessageAndOffset = {
MessageAndOffset(Message.fromRecord(logEntry.record), logEntry.offset)
}
}
case class MessageAndOffset(message: Message, offset: Long) { case class MessageAndOffset(message: Message, offset: Long) {
@ -28,9 +35,10 @@ case class MessageAndOffset(message: Message, offset: Long) {
/** /**
* We need to decompress the message, if required, to get the offset of the first uncompressed message. * We need to decompress the message, if required, to get the offset of the first uncompressed message.
*/ */
def firstOffset: Long = message.compressionCodec match { def firstOffset: Long = toLogEntry.firstOffset
case NoCompressionCodec => offset
case _ => ByteBufferMessageSet.deepIterator(this).next().offset def toLogEntry: LogEntry = {
LogEntry.create(offset, message.asRecord)
} }
} }

View File

@ -18,7 +18,6 @@
package kafka.message package kafka.message
import java.nio._ import java.nio._
import java.nio.channels._
import org.apache.kafka.common.record.Records import org.apache.kafka.common.record.Records
@ -72,11 +71,6 @@ case class MagicAndTimestamp(magic: Byte, timestamp: Long)
*/ */
abstract class MessageSet extends Iterable[MessageAndOffset] { abstract class MessageSet extends Iterable[MessageAndOffset] {
/**
* Check if all the wrapper messages in the message set have the expected magic value
*/
def isMagicValueInAllWrapperMessages(expectedMagicValue: Byte): Boolean
/** /**
* Provides an iterator over the message/offset pairs in this set * Provides an iterator over the message/offset pairs in this set
*/ */
@ -99,7 +93,7 @@ abstract class MessageSet extends Iterable[MessageAndOffset] {
override def toString: String = { override def toString: String = {
val builder = new StringBuilder() val builder = new StringBuilder()
builder.append(getClass.getSimpleName + "(") builder.append(getClass.getSimpleName + "(")
val iter = this.iterator val iter = this.asRecords.shallowIterator()
var i = 0 var i = 0
while(iter.hasNext && i < 100) { while(iter.hasNext && i < 100) {
val message = iter.next val message = iter.next

View File

@ -21,7 +21,6 @@ import java.util.concurrent.locks.ReentrantLock
import kafka.cluster.BrokerEndPoint import kafka.cluster.BrokerEndPoint
import kafka.consumer.PartitionTopicInfo import kafka.consumer.PartitionTopicInfo
import kafka.message.ByteBufferMessageSet
import kafka.utils.{DelayedItem, Pool, ShutdownableThread} import kafka.utils.{DelayedItem, Pool, ShutdownableThread}
import kafka.common.{ClientIdAndBroker, KafkaException} import kafka.common.{ClientIdAndBroker, KafkaException}
import kafka.metrics.KafkaMetricsGroup import kafka.metrics.KafkaMetricsGroup
@ -38,6 +37,7 @@ import java.util.concurrent.atomic.AtomicLong
import com.yammer.metrics.core.Gauge import com.yammer.metrics.core.Gauge
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.internals.PartitionStates import org.apache.kafka.common.internals.PartitionStates
import org.apache.kafka.common.record.MemoryRecords
/** /**
* Abstract class for fetching data from multiple partitions from the same broker. * Abstract class for fetching data from multiple partitions from the same broker.
@ -144,15 +144,15 @@ abstract class AbstractFetcherThread(name: String,
Errors.forCode(partitionData.errorCode) match { Errors.forCode(partitionData.errorCode) match {
case Errors.NONE => case Errors.NONE =>
try { try {
val messages = partitionData.toByteBufferMessageSet val records = partitionData.toRecords
val newOffset = messages.shallowIterator.toSeq.lastOption.map(_.nextOffset).getOrElse( val newOffset = records.shallowIterator.asScala.toSeq.lastOption.map(_.nextOffset).getOrElse(
currentPartitionFetchState.offset) currentPartitionFetchState.offset)
fetcherLagStats.getAndMaybePut(topic, partitionId).lag = Math.max(0L, partitionData.highWatermark - newOffset) fetcherLagStats.getAndMaybePut(topic, partitionId).lag = Math.max(0L, partitionData.highWatermark - newOffset)
// Once we hand off the partition data to the subclass, we can't mess with it any more in this thread // Once we hand off the partition data to the subclass, we can't mess with it any more in this thread
processPartitionData(topicPartition, currentPartitionFetchState.offset, partitionData) processPartitionData(topicPartition, currentPartitionFetchState.offset, partitionData)
val validBytes = messages.validBytes val validBytes = records.validBytes
if (validBytes > 0) { if (validBytes > 0) {
// Update partitionStates only if there is no exception during processPartitionData // Update partitionStates only if there is no exception during processPartitionData
partitionStates.updateAndMoveToEnd(topicPartition, new PartitionFetchState(newOffset)) partitionStates.updateAndMoveToEnd(topicPartition, new PartitionFetchState(newOffset))
@ -260,7 +260,7 @@ object AbstractFetcherThread {
trait PartitionData { trait PartitionData {
def errorCode: Short def errorCode: Short
def exception: Option[Throwable] def exception: Option[Throwable]
def toByteBufferMessageSet: ByteBufferMessageSet def toRecords: MemoryRecords
def highWatermark: Long def highWatermark: Long
} }

View File

@ -19,7 +19,6 @@ package kafka.server
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import kafka.api.FetchResponsePartitionData
import kafka.common.TopicAndPartition import kafka.common.TopicAndPartition
import kafka.metrics.KafkaMetricsGroup import kafka.metrics.KafkaMetricsGroup
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
@ -59,7 +58,7 @@ class DelayedFetch(delayMs: Long,
fetchMetadata: FetchMetadata, fetchMetadata: FetchMetadata,
replicaManager: ReplicaManager, replicaManager: ReplicaManager,
quota: ReplicaQuota, quota: ReplicaQuota,
responseCallback: Seq[(TopicAndPartition, FetchResponsePartitionData)] => Unit) responseCallback: Seq[(TopicAndPartition, FetchPartitionData)] => Unit)
extends DelayedOperation(delayMs) { extends DelayedOperation(delayMs) {
/** /**
@ -152,7 +151,7 @@ class DelayedFetch(delayMs: Long,
) )
val fetchPartitionData = logReadResults.map { case (tp, result) => val fetchPartitionData = logReadResults.map { case (tp, result) =>
tp -> FetchResponsePartitionData(result.errorCode, result.hw, result.info.messageSet) tp -> FetchPartitionData(result.errorCode, result.hw, result.info.records)
} }
responseCallback(fetchPartitionData) responseCallback(fetchPartitionData)

View File

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

View File

@ -23,14 +23,13 @@ import java.util.{Collections, Properties}
import java.util import java.util
import kafka.admin.{AdminUtils, RackAwareMode} import kafka.admin.{AdminUtils, RackAwareMode}
import kafka.api.{ControlledShutdownRequest, ControlledShutdownResponse, FetchResponsePartitionData} import kafka.api.{ControlledShutdownRequest, ControlledShutdownResponse}
import kafka.cluster.Partition import kafka.cluster.Partition
import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota} import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota}
import kafka.common._ import kafka.common._
import kafka.controller.KafkaController import kafka.controller.KafkaController
import kafka.coordinator.{GroupCoordinator, JoinGroupResult} import kafka.coordinator.{GroupCoordinator, JoinGroupResult}
import kafka.log._ import kafka.log._
import kafka.message.{ByteBufferMessageSet, Message}
import kafka.network._ import kafka.network._
import kafka.network.RequestChannel.{Response, Session} import kafka.network.RequestChannel.{Response, Session}
import kafka.security.auth import kafka.security.auth
@ -39,7 +38,7 @@ import kafka.utils.{Logging, ZKGroupTopicDirs, ZkUtils}
import org.apache.kafka.common.errors.{ClusterAuthorizationException, NotLeaderForPartitionException, TopicExistsException, UnknownTopicOrPartitionException, UnsupportedForMessageFormatException} import org.apache.kafka.common.errors.{ClusterAuthorizationException, NotLeaderForPartitionException, TopicExistsException, UnknownTopicOrPartitionException, UnsupportedForMessageFormatException}
import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.protocol.{ApiKeys, Errors, Protocol, SecurityProtocol} import org.apache.kafka.common.protocol.{ApiKeys, Errors, Protocol, SecurityProtocol}
import org.apache.kafka.common.record.MemoryRecords import org.apache.kafka.common.record.{MemoryRecords, Record}
import org.apache.kafka.common.requests._ import org.apache.kafka.common.requests._
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.utils.{Time, Utils}
@ -359,9 +358,9 @@ class KafkaApis(val requestChannel: RequestChannel,
val mergedResponseStatus = responseStatus ++ val mergedResponseStatus = responseStatus ++
unauthorizedForWriteRequestInfo.mapValues(_ => unauthorizedForWriteRequestInfo.mapValues(_ =>
new PartitionResponse(Errors.TOPIC_AUTHORIZATION_FAILED.code, -1, Message.NoTimestamp)) ++ new PartitionResponse(Errors.TOPIC_AUTHORIZATION_FAILED.code, -1, Record.NO_TIMESTAMP)) ++
nonExistingOrUnauthorizedForDescribeTopics.mapValues(_ => nonExistingOrUnauthorizedForDescribeTopics.mapValues(_ =>
new PartitionResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION.code, -1, Message.NoTimestamp)) new PartitionResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION.code, -1, Record.NO_TIMESTAMP))
var errorInResponse = false var errorInResponse = false
@ -422,17 +421,12 @@ class KafkaApis(val requestChannel: RequestChannel,
else { else {
val internalTopicsAllowed = request.header.clientId == AdminUtils.AdminClientId val internalTopicsAllowed = request.header.clientId == AdminUtils.AdminClientId
// Convert Records to ByteBufferMessageSet
val authorizedMessagesPerPartition = authorizedRequestInfo.map {
case (topicPartition, records) => (topicPartition, new ByteBufferMessageSet(records.buffer))
}
// call the replica manager to append messages to the replicas // call the replica manager to append messages to the replicas
replicaManager.appendMessages( replicaManager.appendRecords(
produceRequest.timeout.toLong, produceRequest.timeout.toLong,
produceRequest.acks, produceRequest.acks,
internalTopicsAllowed, internalTopicsAllowed,
authorizedMessagesPerPartition, authorizedRequestInfo,
sendResponseCallback) sendResponseCallback)
// if the request is put into the purgatory, it will have a held reference // if the request is put into the purgatory, it will have a held reference
@ -467,7 +461,7 @@ class KafkaApis(val requestChannel: RequestChannel,
} }
// the callback for sending a fetch response // the callback for sending a fetch response
def sendResponseCallback(responsePartitionData: Seq[(TopicAndPartition, FetchResponsePartitionData)]) { def sendResponseCallback(responsePartitionData: Seq[(TopicAndPartition, FetchPartitionData)]) {
val convertedPartitionData = { val convertedPartitionData = {
// Need to down-convert message when consumer only takes magic value 0. // Need to down-convert message when consumer only takes magic value 0.
responsePartitionData.map { case (tp, data) => responsePartitionData.map { case (tp, data) =>
@ -480,18 +474,18 @@ class KafkaApis(val requestChannel: RequestChannel,
// Please note that if the message format is changed from a higher version back to lower version this // Please note that if the message format is changed from a higher version back to lower version this
// test might break because some messages in new message format can be delivered to consumers before 0.10.0.0 // test might break because some messages in new message format can be delivered to consumers before 0.10.0.0
// without format down conversion. // without format down conversion.
val convertedData = if (versionId <= 1 && replicaManager.getMessageFormatVersion(tp).exists(_ > Message.MagicValue_V0) && val convertedData = if (versionId <= 1 && replicaManager.getMagicAndTimestampType(tp).exists(_._1 > Record.MAGIC_VALUE_V0) &&
!data.messages.isMagicValueInAllWrapperMessages(Message.MagicValue_V0)) { !data.records.hasMatchingShallowMagic(Record.MAGIC_VALUE_V0)) {
trace(s"Down converting message to V0 for fetch request from $clientId") trace(s"Down converting message to V0 for fetch request from $clientId")
new FetchResponsePartitionData(data.error, data.hw, data.messages.asInstanceOf[FileMessageSet].toMessageFormat(Message.MagicValue_V0)) FetchPartitionData(data.error, data.hw, data.records.toMessageFormat(Record.MAGIC_VALUE_V0))
} else data } else data
val records = convertedData.messages.asRecords new TopicPartition(tp.topic, tp.partition) -> new FetchResponse.PartitionData(convertedData.error, convertedData.hw, convertedData.records)
new TopicPartition(tp.topic, tp.partition) -> new FetchResponse.PartitionData(convertedData.error, convertedData.hw, records)
} }
} }
val mergedPartitionData = convertedPartitionData ++ unauthorizedForReadPartitionData ++ nonExistingOrUnauthorizedForDescribePartitionData val mergedPartitionData = convertedPartitionData ++ unauthorizedForReadPartitionData ++ nonExistingOrUnauthorizedForDescribePartitionData
val fetchedPartitionData = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData]() val fetchedPartitionData = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData]()
mergedPartitionData.foreach { case (topicPartition, data) => mergedPartitionData.foreach { case (topicPartition, data) =>
@ -660,7 +654,7 @@ class KafkaApis(val requestChannel: RequestChannel,
val found = { val found = {
if (fromConsumer && timestamp == ListOffsetRequest.LATEST_TIMESTAMP) if (fromConsumer && timestamp == ListOffsetRequest.LATEST_TIMESTAMP)
TimestampOffset(Message.NoTimestamp, localReplica.highWatermark.messageOffset) TimestampOffset(Record.NO_TIMESTAMP, localReplica.highWatermark.messageOffset)
else { else {
def allowed(timestampOffset: TimestampOffset): Boolean = def allowed(timestampOffset: TimestampOffset): Boolean =
!fromConsumer || timestampOffset.offset <= localReplica.highWatermark.messageOffset !fromConsumer || timestampOffset.offset <= localReplica.highWatermark.messageOffset

View File

@ -23,7 +23,6 @@ import java.util
import kafka.admin.AdminUtils import kafka.admin.AdminUtils
import kafka.cluster.BrokerEndPoint import kafka.cluster.BrokerEndPoint
import kafka.log.LogConfig import kafka.log.LogConfig
import kafka.message.ByteBufferMessageSet
import kafka.api.{KAFKA_0_10_0_IV0, KAFKA_0_10_1_IV1, KAFKA_0_10_1_IV2, KAFKA_0_9_0} import kafka.api.{KAFKA_0_10_0_IV0, KAFKA_0_10_1_IV1, KAFKA_0_10_1_IV2, KAFKA_0_9_0}
import kafka.common.{KafkaStorageException, TopicAndPartition} import kafka.common.{KafkaStorageException, TopicAndPartition}
import ReplicaFetcherThread._ import ReplicaFetcherThread._
@ -119,19 +118,19 @@ class ReplicaFetcherThread(name: String,
val topic = topicPartition.topic val topic = topicPartition.topic
val partitionId = topicPartition.partition val partitionId = topicPartition.partition
val replica = replicaMgr.getReplica(topic, partitionId).get val replica = replicaMgr.getReplica(topic, partitionId).get
val messageSet = partitionData.toByteBufferMessageSet val records = partitionData.toRecords
maybeWarnIfMessageOversized(messageSet, topicPartition) maybeWarnIfOversizedRecords(records, topicPartition)
if (fetchOffset != replica.logEndOffset.messageOffset) if (fetchOffset != replica.logEndOffset.messageOffset)
throw new RuntimeException("Offset mismatch for partition %s: fetched offset = %d, log end offset = %d.".format(topicPartition, fetchOffset, replica.logEndOffset.messageOffset)) throw new RuntimeException("Offset mismatch for partition %s: fetched offset = %d, log end offset = %d.".format(topicPartition, fetchOffset, replica.logEndOffset.messageOffset))
if (logger.isTraceEnabled) if (logger.isTraceEnabled)
trace("Follower %d has replica log end offset %d for partition %s. Received %d messages and leader hw %d" trace("Follower %d has replica log end offset %d for partition %s. Received %d messages and leader hw %d"
.format(replica.brokerId, replica.logEndOffset.messageOffset, topicPartition, messageSet.sizeInBytes, partitionData.highWatermark)) .format(replica.brokerId, replica.logEndOffset.messageOffset, topicPartition, records.sizeInBytes, partitionData.highWatermark))
replica.log.get.append(messageSet, assignOffsets = false) replica.log.get.append(records, assignOffsets = false)
if (logger.isTraceEnabled) if (logger.isTraceEnabled)
trace("Follower %d has replica log end offset %d after appending %d bytes of messages for partition %s" trace("Follower %d has replica log end offset %d after appending %d bytes of messages for partition %s"
.format(replica.brokerId, replica.logEndOffset.messageOffset, messageSet.sizeInBytes, topicPartition)) .format(replica.brokerId, replica.logEndOffset.messageOffset, records.sizeInBytes, topicPartition))
val followerHighWatermark = replica.logEndOffset.messageOffset.min(partitionData.highWatermark) val followerHighWatermark = replica.logEndOffset.messageOffset.min(partitionData.highWatermark)
// for the follower replica, we do not need to keep // for the follower replica, we do not need to keep
// its segment base offset the physical position, // its segment base offset the physical position,
@ -141,7 +140,7 @@ class ReplicaFetcherThread(name: String,
trace("Follower %d set replica high watermark for partition [%s,%d] to %s" trace("Follower %d set replica high watermark for partition [%s,%d] to %s"
.format(replica.brokerId, topic, partitionId, followerHighWatermark)) .format(replica.brokerId, topic, partitionId, followerHighWatermark))
if (quota.isThrottled(TopicAndPartition(topic, partitionId))) if (quota.isThrottled(TopicAndPartition(topic, partitionId)))
quota.record(messageSet.sizeInBytes) quota.record(records.sizeInBytes)
} catch { } catch {
case e: KafkaStorageException => case e: KafkaStorageException =>
fatal(s"Disk error while replicating data for $topicPartition", e) fatal(s"Disk error while replicating data for $topicPartition", e)
@ -149,9 +148,9 @@ class ReplicaFetcherThread(name: String,
} }
} }
def maybeWarnIfMessageOversized(messageSet: ByteBufferMessageSet, topicPartition: TopicPartition): Unit = { def maybeWarnIfOversizedRecords(records: MemoryRecords, topicPartition: TopicPartition): Unit = {
// oversized messages don't cause replication to fail from fetch request version 3 (KIP-74) // oversized messages don't cause replication to fail from fetch request version 3 (KIP-74)
if (fetchRequestVersion <= 2 && messageSet.sizeInBytes > 0 && messageSet.validBytes <= 0) if (fetchRequestVersion <= 2 && records.sizeInBytes > 0 && records.validBytes <= 0)
error(s"Replication is failing due to a message that is greater than replica.fetch.max.bytes for partition $topicPartition. " + error(s"Replication is failing due to a message that is greater than replica.fetch.max.bytes for partition $topicPartition. " +
"This generally occurs when the max.message.bytes has been overridden to exceed this value and a suitably large " + "This generally occurs when the max.message.bytes has been overridden to exceed this value and a suitably large " +
"message has also been sent. To fix this problem increase replica.fetch.max.bytes in your broker config to be " + "message has also been sent. To fix this problem increase replica.fetch.max.bytes in your broker config to be " +
@ -323,9 +322,8 @@ object ReplicaFetcherThread {
def errorCode: Short = underlying.errorCode def errorCode: Short = underlying.errorCode
def toByteBufferMessageSet: ByteBufferMessageSet = { def toRecords: MemoryRecords = {
val buffer = underlying.records.asInstanceOf[MemoryRecords].buffer underlying.records.asInstanceOf[MemoryRecords]
new ByteBufferMessageSet(buffer)
} }
def highWatermark: Long = underlying.highWatermark def highWatermark: Long = underlying.highWatermark

View File

@ -26,7 +26,7 @@ import kafka.cluster.{Partition, Replica}
import kafka.common._ import kafka.common._
import kafka.controller.KafkaController import kafka.controller.KafkaController
import kafka.log.{LogAppendInfo, LogManager} import kafka.log.{LogAppendInfo, LogManager}
import kafka.message.{ByteBufferMessageSet, InvalidMessageException, Message, MessageSet} import kafka.message.InvalidMessageException
import kafka.metrics.KafkaMetricsGroup import kafka.metrics.KafkaMetricsGroup
import kafka.server.QuotaFactory.UnboundedQuota import kafka.server.QuotaFactory.UnboundedQuota
import kafka.utils._ import kafka.utils._
@ -34,6 +34,7 @@ import org.apache.kafka.common.errors.{ControllerMovedException, CorruptRecordEx
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.record._
import org.apache.kafka.common.requests.{LeaderAndIsrRequest, PartitionState, StopReplicaRequest, UpdateMetadataRequest} import org.apache.kafka.common.requests.{LeaderAndIsrRequest, PartitionState, StopReplicaRequest, UpdateMetadataRequest}
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
@ -78,9 +79,11 @@ case class LogReadResult(info: FetchDataInfo,
} }
} }
case class FetchPartitionData(error: Short = Errors.NONE.code, hw: Long = -1L, records: Records)
object LogReadResult { object LogReadResult {
val UnknownLogReadResult = LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, val UnknownLogReadResult = LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata,
MessageSet.Empty), MemoryRecords.EMPTY),
-1L, -1L,
-1, -1,
false) false)
@ -276,11 +279,7 @@ class ReplicaManager(val config: KafkaConfig,
} }
def getPartition(topic: String, partitionId: Int): Option[Partition] = { def getPartition(topic: String, partitionId: Int): Option[Partition] = {
val partition = allPartitions.get((topic, partitionId)) Option(allPartitions.get((topic, partitionId)))
if (partition == null)
None
else
Some(partition)
} }
def getReplicaOrException(topic: String, partition: Int): Replica = { def getReplicaOrException(topic: String, partition: Int): Replica = {
@ -318,15 +317,15 @@ class ReplicaManager(val config: KafkaConfig,
* Append messages to leader replicas of the partition, and wait for them to be replicated to other replicas; * Append messages to leader replicas of the partition, and wait for them to be replicated to other replicas;
* the callback function will be triggered either when timeout or the required acks are satisfied * the callback function will be triggered either when timeout or the required acks are satisfied
*/ */
def appendMessages(timeout: Long, def appendRecords(timeout: Long,
requiredAcks: Short, requiredAcks: Short,
internalTopicsAllowed: Boolean, internalTopicsAllowed: Boolean,
messagesPerPartition: Map[TopicPartition, MessageSet], entriesPerPartition: Map[TopicPartition, MemoryRecords],
responseCallback: Map[TopicPartition, PartitionResponse] => Unit) { responseCallback: Map[TopicPartition, PartitionResponse] => Unit) {
if (isValidRequiredAcks(requiredAcks)) { if (isValidRequiredAcks(requiredAcks)) {
val sTime = time.milliseconds val sTime = time.milliseconds
val localProduceResults = appendToLocalLog(internalTopicsAllowed, messagesPerPartition, requiredAcks) val localProduceResults = appendToLocalLog(internalTopicsAllowed, entriesPerPartition, requiredAcks)
debug("Produce to local log in %d ms".format(time.milliseconds - sTime)) debug("Produce to local log in %d ms".format(time.milliseconds - sTime))
val produceStatus = localProduceResults.map { case (topicPartition, result) => val produceStatus = localProduceResults.map { case (topicPartition, result) =>
@ -336,13 +335,13 @@ class ReplicaManager(val config: KafkaConfig,
new PartitionResponse(result.errorCode, result.info.firstOffset, result.info.logAppendTime)) // response status new PartitionResponse(result.errorCode, result.info.firstOffset, result.info.logAppendTime)) // response status
} }
if (delayedRequestRequired(requiredAcks, messagesPerPartition, localProduceResults)) { if (delayedRequestRequired(requiredAcks, entriesPerPartition, localProduceResults)) {
// create delayed produce operation // create delayed produce operation
val produceMetadata = ProduceMetadata(requiredAcks, produceStatus) val produceMetadata = ProduceMetadata(requiredAcks, produceStatus)
val delayedProduce = new DelayedProduce(timeout, produceMetadata, this, responseCallback) val delayedProduce = new DelayedProduce(timeout, produceMetadata, this, responseCallback)
// create a list of (topic, partition) pairs to use as keys for this delayed produce operation // create a list of (topic, partition) pairs to use as keys for this delayed produce operation
val producerRequestKeys = messagesPerPartition.keys.map(new TopicPartitionOperationKey(_)).toSeq val producerRequestKeys = entriesPerPartition.keys.map(new TopicPartitionOperationKey(_)).toSeq
// try to complete the request immediately, otherwise put it into the purgatory // try to complete the request immediately, otherwise put it into the purgatory
// this is because while the delayed produce operation is being created, new // this is because while the delayed produce operation is being created, new
@ -357,9 +356,9 @@ class ReplicaManager(val config: KafkaConfig,
} else { } else {
// If required.acks is outside accepted range, something is wrong with the client // If required.acks is outside accepted range, something is wrong with the client
// Just return an error and don't handle the request at all // Just return an error and don't handle the request at all
val responseStatus = messagesPerPartition.map { case (topicAndPartition, _) => val responseStatus = entriesPerPartition.map { case (topicAndPartition, _) =>
topicAndPartition -> new PartitionResponse(Errors.INVALID_REQUIRED_ACKS.code, topicAndPartition -> new PartitionResponse(Errors.INVALID_REQUIRED_ACKS.code,
LogAppendInfo.UnknownLogAppendInfo.firstOffset, Message.NoTimestamp) LogAppendInfo.UnknownLogAppendInfo.firstOffset, Record.NO_TIMESTAMP)
} }
responseCallback(responseStatus) responseCallback(responseStatus)
} }
@ -370,11 +369,12 @@ class ReplicaManager(val config: KafkaConfig,
// 1. required acks = -1 // 1. required acks = -1
// 2. there is data to append // 2. there is data to append
// 3. at least one partition append was successful (fewer errors than partitions) // 3. at least one partition append was successful (fewer errors than partitions)
private def delayedRequestRequired(requiredAcks: Short, messagesPerPartition: Map[TopicPartition, MessageSet], private def delayedRequestRequired(requiredAcks: Short,
localProduceResults: Map[TopicPartition, LogAppendResult]): Boolean = { entriesPerPartition: Map[TopicPartition, MemoryRecords],
localProduceResults: Map[TopicPartition, LogAppendResult]): Boolean = {
requiredAcks == -1 && requiredAcks == -1 &&
messagesPerPartition.nonEmpty && entriesPerPartition.nonEmpty &&
localProduceResults.values.count(_.error.isDefined) < messagesPerPartition.size localProduceResults.values.count(_.error.isDefined) < entriesPerPartition.size
} }
private def isValidRequiredAcks(requiredAcks: Short): Boolean = { private def isValidRequiredAcks(requiredAcks: Short): Boolean = {
@ -385,10 +385,10 @@ class ReplicaManager(val config: KafkaConfig,
* Append the messages to the local replica logs * Append the messages to the local replica logs
*/ */
private def appendToLocalLog(internalTopicsAllowed: Boolean, private def appendToLocalLog(internalTopicsAllowed: Boolean,
messagesPerPartition: Map[TopicPartition, MessageSet], entriesPerPartition: Map[TopicPartition, MemoryRecords],
requiredAcks: Short): Map[TopicPartition, LogAppendResult] = { requiredAcks: Short): Map[TopicPartition, LogAppendResult] = {
trace("Append [%s] to local log ".format(messagesPerPartition)) trace("Append [%s] to local log ".format(entriesPerPartition))
messagesPerPartition.map { case (topicPartition, messages) => entriesPerPartition.map { case (topicPartition, records) =>
BrokerTopicStats.getBrokerTopicStats(topicPartition.topic).totalProduceRequestRate.mark() BrokerTopicStats.getBrokerTopicStats(topicPartition.topic).totalProduceRequestRate.mark()
BrokerTopicStats.getBrokerAllTopicsStats().totalProduceRequestRate.mark() BrokerTopicStats.getBrokerAllTopicsStats().totalProduceRequestRate.mark()
@ -402,7 +402,7 @@ class ReplicaManager(val config: KafkaConfig,
val partitionOpt = getPartition(topicPartition.topic, topicPartition.partition) val partitionOpt = getPartition(topicPartition.topic, topicPartition.partition)
val info = partitionOpt match { val info = partitionOpt match {
case Some(partition) => case Some(partition) =>
partition.appendMessagesToLeader(messages.asInstanceOf[ByteBufferMessageSet], requiredAcks) partition.appendRecordsToLeader(records, requiredAcks)
case None => throw new UnknownTopicOrPartitionException("Partition %s doesn't exist on %d" case None => throw new UnknownTopicOrPartitionException("Partition %s doesn't exist on %d"
.format(topicPartition, localBrokerId)) .format(topicPartition, localBrokerId))
} }
@ -414,13 +414,13 @@ class ReplicaManager(val config: KafkaConfig,
info.lastOffset - info.firstOffset + 1 info.lastOffset - info.firstOffset + 1
// update stats for successfully appended bytes and messages as bytesInRate and messageInRate // update stats for successfully appended bytes and messages as bytesInRate and messageInRate
BrokerTopicStats.getBrokerTopicStats(topicPartition.topic).bytesInRate.mark(messages.sizeInBytes) BrokerTopicStats.getBrokerTopicStats(topicPartition.topic).bytesInRate.mark(records.sizeInBytes)
BrokerTopicStats.getBrokerAllTopicsStats.bytesInRate.mark(messages.sizeInBytes) BrokerTopicStats.getBrokerAllTopicsStats.bytesInRate.mark(records.sizeInBytes)
BrokerTopicStats.getBrokerTopicStats(topicPartition.topic).messagesInRate.mark(numAppendedMessages) BrokerTopicStats.getBrokerTopicStats(topicPartition.topic).messagesInRate.mark(numAppendedMessages)
BrokerTopicStats.getBrokerAllTopicsStats.messagesInRate.mark(numAppendedMessages) BrokerTopicStats.getBrokerAllTopicsStats.messagesInRate.mark(numAppendedMessages)
trace("%d bytes written to log %s-%d beginning at offset %d and ending at offset %d" trace("%d bytes written to log %s-%d beginning at offset %d and ending at offset %d"
.format(messages.sizeInBytes, topicPartition.topic, topicPartition.partition, info.firstOffset, info.lastOffset)) .format(records.sizeInBytes, topicPartition.topic, topicPartition.partition, info.firstOffset, info.lastOffset))
(topicPartition, LogAppendResult(info)) (topicPartition, LogAppendResult(info))
} catch { } catch {
// NOTE: Failed produce requests metric is not incremented for known exceptions // NOTE: Failed produce requests metric is not incremented for known exceptions
@ -434,6 +434,7 @@ class ReplicaManager(val config: KafkaConfig,
_: RecordTooLargeException | _: RecordTooLargeException |
_: RecordBatchTooLargeException | _: RecordBatchTooLargeException |
_: CorruptRecordException | _: CorruptRecordException |
_: InvalidRecordException |
_: InvalidMessageException | _: InvalidMessageException |
_: InvalidTimestampException) => _: InvalidTimestampException) =>
(topicPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(e))) (topicPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(e)))
@ -458,7 +459,7 @@ class ReplicaManager(val config: KafkaConfig,
hardMaxBytesLimit: Boolean, hardMaxBytesLimit: Boolean,
fetchInfos: Seq[(TopicPartition, PartitionData)], fetchInfos: Seq[(TopicPartition, PartitionData)],
quota: ReplicaQuota = UnboundedQuota, quota: ReplicaQuota = UnboundedQuota,
responseCallback: Seq[(TopicAndPartition, FetchResponsePartitionData)] => Unit) { responseCallback: Seq[(TopicAndPartition, FetchPartitionData)] => Unit) {
val isFromFollower = replicaId >= 0 val isFromFollower = replicaId >= 0
val fetchOnlyFromLeader: Boolean = replicaId != Request.DebuggingConsumerId val fetchOnlyFromLeader: Boolean = replicaId != Request.DebuggingConsumerId
val fetchOnlyCommitted: Boolean = ! Request.isValidBrokerId(replicaId) val fetchOnlyCommitted: Boolean = ! Request.isValidBrokerId(replicaId)
@ -480,7 +481,7 @@ class ReplicaManager(val config: KafkaConfig,
// check if this fetch request can be satisfied right away // check if this fetch request can be satisfied right away
val logReadResultValues = logReadResults.map { case (_, v) => v } val logReadResultValues = logReadResults.map { case (_, v) => v }
val bytesReadable = logReadResultValues.map(_.info.messageSet.sizeInBytes).sum val bytesReadable = logReadResultValues.map(_.info.records.sizeInBytes).sum
val errorReadingData = logReadResultValues.foldLeft(false) ((errorIncurred, readResult) => val errorReadingData = logReadResultValues.foldLeft(false) ((errorIncurred, readResult) =>
errorIncurred || (readResult.errorCode != Errors.NONE.code)) errorIncurred || (readResult.errorCode != Errors.NONE.code))
@ -490,7 +491,7 @@ class ReplicaManager(val config: KafkaConfig,
// 4) some error happens while reading data // 4) some error happens while reading data
if (timeout <= 0 || fetchInfos.isEmpty || bytesReadable >= fetchMinBytes || errorReadingData) { if (timeout <= 0 || fetchInfos.isEmpty || bytesReadable >= fetchMinBytes || errorReadingData) {
val fetchPartitionData = logReadResults.map { case (tp, result) => val fetchPartitionData = logReadResults.map { case (tp, result) =>
tp -> FetchResponsePartitionData(result.errorCode, result.hw, result.info.messageSet) tp -> FetchPartitionData(result.errorCode, result.hw, result.info.records)
} }
responseCallback(fetchPartitionData) responseCallback(fetchPartitionData)
} else { } else {
@ -568,16 +569,16 @@ class ReplicaManager(val config: KafkaConfig,
// If the partition is being throttled, simply return an empty set. // If the partition is being throttled, simply return an empty set.
if (shouldLeaderThrottle(quota, TopicAndPartition(tp.topic, tp.partition), replicaId)) if (shouldLeaderThrottle(quota, TopicAndPartition(tp.topic, tp.partition), replicaId))
FetchDataInfo(fetch.fetchOffsetMetadata, MessageSet.Empty) FetchDataInfo(fetch.fetchOffsetMetadata, MemoryRecords.EMPTY)
// For FetchRequest version 3, we replace incomplete message sets with an empty one as consumers can make // For FetchRequest version 3, we replace incomplete message sets with an empty one as consumers can make
// progress in such cases and don't need to report a `RecordTooLargeException` // progress in such cases and don't need to report a `RecordTooLargeException`
else if (!hardMaxBytesLimit && fetch.firstMessageSetIncomplete) else if (!hardMaxBytesLimit && fetch.firstEntryIncomplete)
FetchDataInfo(fetch.fetchOffsetMetadata, MessageSet.Empty) FetchDataInfo(fetch.fetchOffsetMetadata, MemoryRecords.EMPTY)
else fetch else fetch
case None => case None =>
error(s"Leader for partition $tp does not have a local log") error(s"Leader for partition $tp does not have a local log")
FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty) FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MemoryRecords.EMPTY)
} }
val readToEndOfLog = initialLogEndOffset.messageOffset - logReadInfo.fetchOffsetMetadata.messageOffset <= 0 val readToEndOfLog = initialLogEndOffset.messageOffset - logReadInfo.fetchOffsetMetadata.messageOffset <= 0
@ -590,12 +591,14 @@ class ReplicaManager(val config: KafkaConfig,
_: NotLeaderForPartitionException | _: NotLeaderForPartitionException |
_: ReplicaNotAvailableException | _: ReplicaNotAvailableException |
_: OffsetOutOfRangeException) => _: OffsetOutOfRangeException) =>
LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, partitionFetchSize, false, Some(e)) LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MemoryRecords.EMPTY), -1L,
partitionFetchSize, false, Some(e))
case e: Throwable => case e: Throwable =>
BrokerTopicStats.getBrokerTopicStats(topic).failedFetchRequestRate.mark() BrokerTopicStats.getBrokerTopicStats(topic).failedFetchRequestRate.mark()
BrokerTopicStats.getBrokerAllTopicsStats().failedFetchRequestRate.mark() BrokerTopicStats.getBrokerAllTopicsStats().failedFetchRequestRate.mark()
error(s"Error processing fetch operation on partition ${tp}, offset $offset", e) error(s"Error processing fetch operation on partition $tp, offset $offset", e)
LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, partitionFetchSize, false, Some(e)) LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MemoryRecords.EMPTY), -1L,
partitionFetchSize, false, Some(e))
} }
} }
@ -604,7 +607,7 @@ class ReplicaManager(val config: KafkaConfig,
var minOneMessage = !hardMaxBytesLimit var minOneMessage = !hardMaxBytesLimit
readPartitionInfo.foreach { case (tp, fetchInfo) => readPartitionInfo.foreach { case (tp, fetchInfo) =>
val readResult = read(tp, fetchInfo, limitBytes, minOneMessage) val readResult = read(tp, fetchInfo, limitBytes, minOneMessage)
val messageSetSize = readResult.info.messageSet.sizeInBytes val messageSetSize = readResult.info.records.sizeInBytes
// Once we read from a non-empty partition, we stop ignoring request and partition level size limits // Once we read from a non-empty partition, we stop ignoring request and partition level size limits
if (messageSetSize > 0) if (messageSetSize > 0)
minOneMessage = false minOneMessage = false
@ -625,9 +628,9 @@ class ReplicaManager(val config: KafkaConfig,
quota.isThrottled(topicPartition) && quota.isQuotaExceeded && !isReplicaInSync quota.isThrottled(topicPartition) && quota.isQuotaExceeded && !isReplicaInSync
} }
def getMessageFormatVersion(topicAndPartition: TopicAndPartition): Option[Byte] = def getMagicAndTimestampType(topicAndPartition: TopicAndPartition): Option[(Byte, TimestampType)] =
getReplica(topicAndPartition.topic, topicAndPartition.partition).flatMap { replica => getReplica(topicAndPartition.topic, topicAndPartition.partition).flatMap { replica =>
replica.log.map(_.config.messageFormatVersion.messageFormatVersion) replica.log.map(log => (log.config.messageFormatVersion.messageFormatVersion, log.config.messageTimestampType))
} }
def maybeUpdateMetadataCache(correlationId: Int, updateMetadataRequest: UpdateMetadataRequest, metadataCache: MetadataCache) { def maybeUpdateMetadataCache(correlationId: Int, updateMetadataRequest: UpdateMetadataRequest, metadataCache: MetadataCache) {

View File

@ -23,15 +23,16 @@ import java.nio.ByteBuffer
import joptsimple.OptionParser import joptsimple.OptionParser
import kafka.coordinator.{GroupMetadataKey, GroupMetadataManager, OffsetKey} import kafka.coordinator.{GroupMetadataKey, GroupMetadataManager, OffsetKey}
import kafka.log._ import kafka.log._
import kafka.message._
import kafka.serializer.Decoder import kafka.serializer.Decoder
import kafka.utils._ import kafka.utils._
import org.apache.kafka.clients.consumer.internals.ConsumerProtocol import org.apache.kafka.clients.consumer.internals.ConsumerProtocol
import org.apache.kafka.common.KafkaException import org.apache.kafka.common.KafkaException
import org.apache.kafka.common.record.{CompressionType, FileRecords, LogEntry, Record}
import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.utils.Utils
import scala.collection.mutable import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.ArrayBuffer
import scala.collection.JavaConverters._
object DumpLogSegments { object DumpLogSegments {
@ -69,7 +70,7 @@ object DumpLogSegments {
CommandLineUtils.checkRequiredArgs(parser, options, filesOpt) CommandLineUtils.checkRequiredArgs(parser, options, filesOpt)
val printDataLog = (options.has(printOpt) || options.has(offsetsOpt) || options.has(valueDecoderOpt) || options.has(keyDecoderOpt)) val printDataLog = options.has(printOpt) || options.has(offsetsOpt) || options.has(valueDecoderOpt) || options.has(keyDecoderOpt)
val verifyOnly = options.has(verifyOpt) val verifyOnly = options.has(verifyOpt)
val indexSanityOnly = options.has(indexSanityOpt) val indexSanityOnly = options.has(indexSanityOpt)
@ -132,7 +133,7 @@ object DumpLogSegments {
maxMessageSize: Int) { maxMessageSize: Int) {
val startOffset = file.getName().split("\\.")(0).toLong val startOffset = file.getName().split("\\.")(0).toLong
val logFile = new File(file.getAbsoluteFile.getParent, file.getName.split("\\.")(0) + Log.LogFileSuffix) val logFile = new File(file.getAbsoluteFile.getParent, file.getName.split("\\.")(0) + Log.LogFileSuffix)
val messageSet = new FileMessageSet(logFile, false) val fileRecords = FileRecords.open(logFile, false)
val index = new OffsetIndex(file, baseOffset = startOffset) val index = new OffsetIndex(file, baseOffset = startOffset)
//Check that index passes sanityCheck, this is the check that determines if indexes will be rebuilt on startup or not. //Check that index passes sanityCheck, this is the check that determines if indexes will be rebuilt on startup or not.
@ -144,11 +145,11 @@ object DumpLogSegments {
for(i <- 0 until index.entries) { for(i <- 0 until index.entries) {
val entry = index.entry(i) val entry = index.entry(i)
val partialFileMessageSet: FileMessageSet = messageSet.read(entry.position, maxMessageSize) val slice = fileRecords.read(entry.position, maxMessageSize)
val messageAndOffset = getIterator(partialFileMessageSet.head, isDeepIteration = true).next() val logEntry = getIterator(slice.shallowIterator.next, isDeepIteration = true).next()
if(messageAndOffset.offset != entry.offset + index.baseOffset) { if (logEntry.offset != entry.offset + index.baseOffset) {
var misMatchesSeq = misMatchesForIndexFilesMap.getOrElse(file.getAbsolutePath, List[(Long, Long)]()) var misMatchesSeq = misMatchesForIndexFilesMap.getOrElse(file.getAbsolutePath, List[(Long, Long)]())
misMatchesSeq ::=(entry.offset + index.baseOffset, messageAndOffset.offset) misMatchesSeq ::=(entry.offset + index.baseOffset, logEntry.offset)
misMatchesForIndexFilesMap.put(file.getAbsolutePath, misMatchesSeq) misMatchesForIndexFilesMap.put(file.getAbsolutePath, misMatchesSeq)
} }
// since it is a sparse file, in the event of a crash there may be many zero entries, stop if we see one // since it is a sparse file, in the event of a crash there may be many zero entries, stop if we see one
@ -164,9 +165,9 @@ object DumpLogSegments {
verifyOnly: Boolean, verifyOnly: Boolean,
timeIndexDumpErrors: TimeIndexDumpErrors, timeIndexDumpErrors: TimeIndexDumpErrors,
maxMessageSize: Int) { maxMessageSize: Int) {
val startOffset = file.getName().split("\\.")(0).toLong val startOffset = file.getName.split("\\.")(0).toLong
val logFile = new File(file.getAbsoluteFile.getParent, file.getName.split("\\.")(0) + Log.LogFileSuffix) val logFile = new File(file.getAbsoluteFile.getParent, file.getName.split("\\.")(0) + Log.LogFileSuffix)
val messageSet = new FileMessageSet(logFile, false) val fileRecords = FileRecords.open(logFile, false)
val indexFile = new File(file.getAbsoluteFile.getParent, file.getName.split("\\.")(0) + Log.IndexFileSuffix) val indexFile = new File(file.getAbsoluteFile.getParent, file.getName.split("\\.")(0) + Log.IndexFileSuffix)
val index = new OffsetIndex(indexFile, baseOffset = startOffset) val index = new OffsetIndex(indexFile, baseOffset = startOffset)
val timeIndex = new TimeIndex(file, baseOffset = startOffset) val timeIndex = new TimeIndex(file, baseOffset = startOffset)
@ -178,26 +179,26 @@ object DumpLogSegments {
return return
} }
var prevTimestamp = Message.NoTimestamp var prevTimestamp = Record.NO_TIMESTAMP
for(i <- 0 until timeIndex.entries) { for(i <- 0 until timeIndex.entries) {
val entry = timeIndex.entry(i) val entry = timeIndex.entry(i)
val position = index.lookup(entry.offset + timeIndex.baseOffset).position val position = index.lookup(entry.offset + timeIndex.baseOffset).position
val partialFileMessageSet: FileMessageSet = messageSet.read(position, Int.MaxValue) val partialFileRecords = fileRecords.read(position, Int.MaxValue)
val shallowIter = partialFileMessageSet.iterator val shallowEntries = partialFileRecords.shallowIterator.asScala
var maxTimestamp = Message.NoTimestamp var maxTimestamp = Record.NO_TIMESTAMP
// We first find the message by offset then check if the timestamp is correct. // We first find the message by offset then check if the timestamp is correct.
val wrapperMessageOpt = shallowIter.find(_.offset >= entry.offset + timeIndex.baseOffset) val maybeLogEntry = shallowEntries.find(_.offset >= entry.offset + timeIndex.baseOffset)
wrapperMessageOpt match { maybeLogEntry match {
case None => case None =>
timeIndexDumpErrors.recordShallowOffsetNotFound(file, entry.offset + timeIndex.baseOffset, timeIndexDumpErrors.recordShallowOffsetNotFound(file, entry.offset + timeIndex.baseOffset,
-1.toLong) -1.toLong)
case Some(wrapperMessage) if wrapperMessage.offset != entry.offset + timeIndex.baseOffset => case Some(logEntry) if logEntry.offset != entry.offset + timeIndex.baseOffset =>
timeIndexDumpErrors.recordShallowOffsetNotFound(file, entry.offset + timeIndex.baseOffset, timeIndexDumpErrors.recordShallowOffsetNotFound(file, entry.offset + timeIndex.baseOffset,
wrapperMessage.offset) logEntry.offset)
case Some(wrapperMessage) => case Some(shallowLogEntry) =>
val deepIter = getIterator(wrapperMessage, isDeepIteration = true) val deepIter = getIterator(shallowLogEntry, isDeepIteration = true)
for (messageAndOffset <- deepIter) for (deepLogEntry <- deepIter)
maxTimestamp = math.max(maxTimestamp, messageAndOffset.message.timestamp) maxTimestamp = math.max(maxTimestamp, deepLogEntry.record.timestamp)
if (maxTimestamp != entry.timestamp) if (maxTimestamp != entry.timestamp)
timeIndexDumpErrors.recordMismatchTimeIndex(file, entry.timestamp, maxTimestamp) timeIndexDumpErrors.recordMismatchTimeIndex(file, entry.timestamp, maxTimestamp)
@ -216,20 +217,20 @@ object DumpLogSegments {
} }
private trait MessageParser[K, V] { private trait MessageParser[K, V] {
def parse(message: Message): (Option[K], Option[V]) def parse(record: Record): (Option[K], Option[V])
} }
private class DecoderMessageParser[K, V](keyDecoder: Decoder[K], valueDecoder: Decoder[V]) extends MessageParser[K, V] { private class DecoderMessageParser[K, V](keyDecoder: Decoder[K], valueDecoder: Decoder[V]) extends MessageParser[K, V] {
override def parse(message: Message): (Option[K], Option[V]) = { override def parse(record: Record): (Option[K], Option[V]) = {
if (message.isNull) { if (record.hasNullValue) {
(None, None) (None, None)
} else { } else {
val key = if (message.hasKey) val key = if (record.hasKey)
Some(keyDecoder.fromBytes(Utils.readBytes(message.key))) Some(keyDecoder.fromBytes(Utils.readBytes(record.key)))
else else
None None
val payload = Some(valueDecoder.fromBytes(Utils.readBytes(message.payload))) val payload = Some(valueDecoder.fromBytes(Utils.readBytes(record.value)))
(key, payload) (key, payload)
} }
@ -249,7 +250,7 @@ object DumpLogSegments {
val topicPartition = offsetKey.key.topicPartition val topicPartition = offsetKey.key.topicPartition
val offset = GroupMetadataManager.readOffsetMessageValue(payload) val offset = GroupMetadataManager.readOffsetMessageValue(payload)
val keyString = s"offset::${group}:${topicPartition.topic}:${topicPartition.partition}" val keyString = s"offset::$group:${topicPartition.topic}:${topicPartition.partition}"
val valueString = if (offset.metadata.isEmpty) val valueString = if (offset.metadata.isEmpty)
String.valueOf(offset.offset) String.valueOf(offset.offset)
else else
@ -271,27 +272,27 @@ object DumpLogSegments {
if (userData.isEmpty) if (userData.isEmpty)
s"${member.memberId}=${partitionAssignment.partitions()}" s"${member.memberId}=${partitionAssignment.partitions()}"
else else
s"${member.memberId}=${partitionAssignment.partitions()}:${userData}" s"${member.memberId}=${partitionAssignment.partitions()}:$userData"
} else { } else {
s"${member.memberId}=${hex(member.assignment)}" s"${member.memberId}=${hex(member.assignment)}"
} }
}.mkString("{", ",", "}") }.mkString("{", ",", "}")
val keyString = s"metadata::${groupId}" val keyString = s"metadata::$groupId"
val valueString = s"${protocolType}:${group.protocol}:${group.generationId}:${assignment}" val valueString = s"$protocolType:${group.protocol}:${group.generationId}:$assignment"
(Some(keyString), Some(valueString)) (Some(keyString), Some(valueString))
} }
override def parse(message: Message): (Option[String], Option[String]) = { override def parse(record: Record): (Option[String], Option[String]) = {
if (message.isNull) if (record.hasNullValue)
(None, None) (None, None)
else if (!message.hasKey) { else if (!record.hasKey) {
throw new KafkaException("Failed to decode message using offset topic decoder (message had a missing key)") throw new KafkaException("Failed to decode message using offset topic decoder (message had a missing key)")
} else { } else {
GroupMetadataManager.readMessageKey(message.key) match { GroupMetadataManager.readMessageKey(record.key) match {
case offsetKey: OffsetKey => parseOffsets(offsetKey, message.payload) case offsetKey: OffsetKey => parseOffsets(offsetKey, record.value)
case groupMetadataKey: GroupMetadataKey => parseGroupMetadata(groupMetadataKey, message.payload) case groupMetadataKey: GroupMetadataKey => parseGroupMetadata(groupMetadataKey, record.value)
case _ => throw new KafkaException("Failed to decode message using offset topic decoder (message had an invalid key)") case _ => throw new KafkaException("Failed to decode message using offset topic decoder (message had an invalid key)")
} }
} }
@ -307,70 +308,51 @@ object DumpLogSegments {
parser: MessageParser[_, _]) { parser: MessageParser[_, _]) {
val startOffset = file.getName().split("\\.")(0).toLong val startOffset = file.getName().split("\\.")(0).toLong
println("Starting offset: " + startOffset) println("Starting offset: " + startOffset)
val messageSet = new FileMessageSet(file, false) val messageSet = FileRecords.open(file, false)
var validBytes = 0L var validBytes = 0L
var lastOffset = -1l var lastOffset = -1l
val shallowIterator = messageSet.iterator(maxMessageSize) val shallowIterator = messageSet.shallowIterator(maxMessageSize).asScala
for(shallowMessageAndOffset <- shallowIterator) { // this only does shallow iteration for (shallowLogEntry <- shallowIterator) { // this only does shallow iteration
val itr = getIterator(shallowMessageAndOffset, isDeepIteration) val itr = getIterator(shallowLogEntry, isDeepIteration)
for (messageAndOffset <- itr) { for (deepLogEntry <- itr) {
val msg = messageAndOffset.message val record = deepLogEntry.record()
if(lastOffset == -1) if(lastOffset == -1)
lastOffset = messageAndOffset.offset lastOffset = deepLogEntry.offset
// If we are iterating uncompressed messages, offsets must be consecutive // If we are iterating uncompressed messages, offsets must be consecutive
else if (msg.compressionCodec == NoCompressionCodec && messageAndOffset.offset != lastOffset +1) { else if (record.compressionType == CompressionType.NONE && deepLogEntry.offset != lastOffset +1) {
var nonConsecutivePairsSeq = nonConsecutivePairsForLogFilesMap.getOrElse(file.getAbsolutePath, List[(Long, Long)]()) var nonConsecutivePairsSeq = nonConsecutivePairsForLogFilesMap.getOrElse(file.getAbsolutePath, List[(Long, Long)]())
nonConsecutivePairsSeq ::=(lastOffset, messageAndOffset.offset) nonConsecutivePairsSeq ::=(lastOffset, deepLogEntry.offset)
nonConsecutivePairsForLogFilesMap.put(file.getAbsolutePath, nonConsecutivePairsSeq) nonConsecutivePairsForLogFilesMap.put(file.getAbsolutePath, nonConsecutivePairsSeq)
} }
lastOffset = messageAndOffset.offset lastOffset = deepLogEntry.offset
print("offset: " + messageAndOffset.offset + " position: " + validBytes + print("offset: " + deepLogEntry.offset + " position: " + validBytes +
" " + msg.timestampType + ": " + msg.timestamp + " isvalid: " + msg.isValid + " " + record.timestampType + ": " + record.timestamp + " isvalid: " + record.isValid +
" payloadsize: " + msg.payloadSize + " magic: " + msg.magic + " payloadsize: " + record.valueSize + " magic: " + record.magic +
" compresscodec: " + msg.compressionCodec + " crc: " + msg.checksum) " compresscodec: " + record.compressionType + " crc: " + record.checksum)
if(msg.hasKey) if (record.hasKey)
print(" keysize: " + msg.keySize) print(" keysize: " + record.keySize)
if(printContents) { if (printContents) {
val (key, payload) = parser.parse(msg) val (key, payload) = parser.parse(record)
key.map(key => print(s" key: ${key}")) key.foreach(key => print(s" key: $key"))
payload.map(payload => print(s" payload: ${payload}")) payload.foreach(payload => print(s" payload: $payload"))
} }
println() println()
} }
validBytes += MessageSet.entrySize(shallowMessageAndOffset.message)
validBytes += shallowLogEntry.sizeInBytes
} }
val trailingBytes = messageSet.sizeInBytes - validBytes val trailingBytes = messageSet.sizeInBytes - validBytes
if(trailingBytes > 0) if(trailingBytes > 0)
println("Found %d invalid bytes at the end of %s".format(trailingBytes, file.getName)) println("Found %d invalid bytes at the end of %s".format(trailingBytes, file.getName))
} }
private def getIterator(messageAndOffset: MessageAndOffset, isDeepIteration: Boolean) = { private def getIterator(logEntry: LogEntry, isDeepIteration: Boolean): Iterator[LogEntry] = {
if (isDeepIteration) { if (isDeepIteration)
val message = messageAndOffset.message logEntry.iterator.asScala
message.compressionCodec match { else
case NoCompressionCodec => Iterator(logEntry)
getSingleMessageIterator(messageAndOffset)
case _ =>
ByteBufferMessageSet.deepIterator(messageAndOffset)
}
} else
getSingleMessageIterator(messageAndOffset)
}
private def getSingleMessageIterator(messageAndOffset: MessageAndOffset) = {
new IteratorTemplate[MessageAndOffset] {
var messageIterated = false
override def makeNext(): MessageAndOffset = {
if (!messageIterated) {
messageIterated = true
messageAndOffset
} else
allDone()
}
}
} }
class TimeIndexDumpErrors { class TimeIndexDumpErrors {

View File

@ -17,25 +17,26 @@
package kafka.tools package kafka.tools
import joptsimple.OptionParser
import kafka.cluster.BrokerEndPoint
import kafka.message.{ByteBufferMessageSet, MessageAndOffset, MessageSet}
import java.util.concurrent.CountDownLatch
import java.util.concurrent.atomic.AtomicReference
import kafka.client.ClientUtils
import java.util.regex.{Pattern, PatternSyntaxException}
import kafka.api._
import java.text.SimpleDateFormat import java.text.SimpleDateFormat
import java.util.Date import java.util.Date
import java.util.concurrent.CountDownLatch
import java.util.concurrent.atomic.AtomicReference
import java.util.regex.{Pattern, PatternSyntaxException}
import joptsimple.OptionParser
import kafka.api._
import kafka.client.ClientUtils
import kafka.cluster.BrokerEndPoint
import kafka.common.TopicAndPartition import kafka.common.TopicAndPartition
import kafka.utils._
import kafka.consumer.{ConsumerConfig, SimpleConsumer, Whitelist} import kafka.consumer.{ConsumerConfig, SimpleConsumer, Whitelist}
import kafka.message.{ByteBufferMessageSet, MessageSet}
import kafka.utils._
import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
import scala.collection.JavaConverters._
/** /**
* For verifying the consistency among replicas. * For verifying the consistency among replicas.
* *
@ -149,15 +150,15 @@ object ReplicaVerificationTool extends Logging {
debug("Selected topic partitions: " + topicPartitionReplicaList) debug("Selected topic partitions: " + topicPartitionReplicaList)
val topicAndPartitionsPerBroker: Map[Int, Seq[TopicAndPartition]] = topicPartitionReplicaList.groupBy(_.replicaId) val topicAndPartitionsPerBroker: Map[Int, Seq[TopicAndPartition]] = topicPartitionReplicaList.groupBy(_.replicaId)
.map { case (brokerId, partitions) => .map { case (brokerId, partitions) =>
brokerId -> partitions.map { case partition => new TopicAndPartition(partition.topic, partition.partitionId) } } brokerId -> partitions.map { partition => TopicAndPartition(partition.topic, partition.partitionId) } }
debug("Topic partitions per broker: " + topicAndPartitionsPerBroker) debug("Topic partitions per broker: " + topicAndPartitionsPerBroker)
val expectedReplicasPerTopicAndPartition: Map[TopicAndPartition, Int] = val expectedReplicasPerTopicAndPartition: Map[TopicAndPartition, Int] =
topicPartitionReplicaList.groupBy(replica => new TopicAndPartition(replica.topic, replica.partitionId)) topicPartitionReplicaList.groupBy(replica => TopicAndPartition(replica.topic, replica.partitionId))
.map { case (topicAndPartition, replicaSet) => topicAndPartition -> replicaSet.size } .map { case (topicAndPartition, replicaSet) => topicAndPartition -> replicaSet.size }
debug("Expected replicas per topic partition: " + expectedReplicasPerTopicAndPartition) debug("Expected replicas per topic partition: " + expectedReplicasPerTopicAndPartition)
val leadersPerBroker: Map[Int, Seq[TopicAndPartition]] = filteredTopicMetadata.flatMap { topicMetadataResponse => val leadersPerBroker: Map[Int, Seq[TopicAndPartition]] = filteredTopicMetadata.flatMap { topicMetadataResponse =>
topicMetadataResponse.partitionsMetadata.map { partitionMetadata => topicMetadataResponse.partitionsMetadata.map { partitionMetadata =>
(new TopicAndPartition(topicMetadataResponse.topic, partitionMetadata.partitionId), partitionMetadata.leader.get.id) (TopicAndPartition(topicMetadataResponse.topic, partitionMetadata.partitionId), partitionMetadata.leader.get.id)
} }
}.groupBy(_._2).mapValues(topicAndPartitionAndLeaderIds => topicAndPartitionAndLeaderIds.map { case (topicAndPartition, _) => }.groupBy(_._2).mapValues(topicAndPartitionAndLeaderIds => topicAndPartitionAndLeaderIds.map { case (topicAndPartition, _) =>
topicAndPartition topicAndPartition
@ -200,8 +201,6 @@ object ReplicaVerificationTool extends Logging {
private case class TopicPartitionReplica(topic: String, partitionId: Int, replicaId: Int) private case class TopicPartitionReplica(topic: String, partitionId: Int, replicaId: Int)
private case class ReplicaAndMessageIterator(replicaId: Int, iterator: Iterator[MessageAndOffset])
private case class MessageInfo(replicaId: Int, offset: Long, nextOffset: Long, checksum: Long) private case class MessageInfo(replicaId: Int, offset: Long, nextOffset: Long, checksum: Long)
private class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPartition, Int], private class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPartition, Int],
@ -276,41 +275,42 @@ private class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPa
assert(fetchResponsePerReplica.size == expectedReplicasPerTopicAndPartition(topicAndPartition), assert(fetchResponsePerReplica.size == expectedReplicasPerTopicAndPartition(topicAndPartition),
"fetched " + fetchResponsePerReplica.size + " replicas for " + topicAndPartition + ", but expected " "fetched " + fetchResponsePerReplica.size + " replicas for " + topicAndPartition + ", but expected "
+ expectedReplicasPerTopicAndPartition(topicAndPartition) + " replicas") + expectedReplicasPerTopicAndPartition(topicAndPartition) + " replicas")
val messageIteratorMap = fetchResponsePerReplica.map { val logEntryIteratorMap = fetchResponsePerReplica.map {
case(replicaId, fetchResponse) => case(replicaId, fetchResponse) =>
replicaId -> fetchResponse.messages.asInstanceOf[ByteBufferMessageSet].shallowIterator} replicaId -> fetchResponse.messages.asInstanceOf[ByteBufferMessageSet].asRecords.shallowIterator.asScala
}
val maxHw = fetchResponsePerReplica.values.map(_.hw).max val maxHw = fetchResponsePerReplica.values.map(_.hw).max
// Iterate one message at a time from every replica, until high watermark is reached. // Iterate one message at a time from every replica, until high watermark is reached.
var isMessageInAllReplicas = true var isMessageInAllReplicas = true
while (isMessageInAllReplicas) { while (isMessageInAllReplicas) {
var messageInfoFromFirstReplicaOpt: Option[MessageInfo] = None var messageInfoFromFirstReplicaOpt: Option[MessageInfo] = None
for ( (replicaId, messageIterator) <- messageIteratorMap) { for ( (replicaId, logEntries) <- logEntryIteratorMap) {
try { try {
if (messageIterator.hasNext) { if (logEntries.hasNext) {
val messageAndOffset = messageIterator.next() val logEntry = logEntries.next()
// only verify up to the high watermark // only verify up to the high watermark
if (messageAndOffset.offset >= fetchResponsePerReplica.get(replicaId).hw) if (logEntry.offset >= fetchResponsePerReplica.get(replicaId).hw)
isMessageInAllReplicas = false isMessageInAllReplicas = false
else { else {
messageInfoFromFirstReplicaOpt match { messageInfoFromFirstReplicaOpt match {
case None => case None =>
messageInfoFromFirstReplicaOpt = Some( messageInfoFromFirstReplicaOpt = Some(
MessageInfo(replicaId, messageAndOffset.offset,messageAndOffset.nextOffset, messageAndOffset.message.checksum)) MessageInfo(replicaId, logEntry.offset,logEntry.nextOffset, logEntry.record.checksum))
case Some(messageInfoFromFirstReplica) => case Some(messageInfoFromFirstReplica) =>
if (messageInfoFromFirstReplica.offset != messageAndOffset.offset) { if (messageInfoFromFirstReplica.offset != logEntry.offset) {
println(ReplicaVerificationTool.getCurrentTimeString + ": partition " + topicAndPartition println(ReplicaVerificationTool.getCurrentTimeString + ": partition " + topicAndPartition
+ ": replica " + messageInfoFromFirstReplica.replicaId + "'s offset " + ": replica " + messageInfoFromFirstReplica.replicaId + "'s offset "
+ messageInfoFromFirstReplica.offset + " doesn't match replica " + messageInfoFromFirstReplica.offset + " doesn't match replica "
+ replicaId + "'s offset " + messageAndOffset.offset) + replicaId + "'s offset " + logEntry.offset)
System.exit(1) System.exit(1)
} }
if (messageInfoFromFirstReplica.checksum != messageAndOffset.message.checksum) if (messageInfoFromFirstReplica.checksum != logEntry.record.checksum)
println(ReplicaVerificationTool.getCurrentTimeString + ": partition " println(ReplicaVerificationTool.getCurrentTimeString + ": partition "
+ topicAndPartition + " has unmatched checksum at offset " + messageAndOffset.offset + "; replica " + topicAndPartition + " has unmatched checksum at offset " + logEntry.offset + "; replica "
+ messageInfoFromFirstReplica.replicaId + "'s checksum " + messageInfoFromFirstReplica.checksum + messageInfoFromFirstReplica.replicaId + "'s checksum " + messageInfoFromFirstReplica.checksum
+ "; replica " + replicaId + "'s checksum " + messageAndOffset.message.checksum) + "; replica " + replicaId + "'s checksum " + logEntry.record.checksum)
} }
} }
} else } else

View File

@ -50,11 +50,13 @@ class GroupCoordinatorIntegrationTest extends KafkaServerTestHarness {
def getGroupMetadataLogOpt: Option[Log] = def getGroupMetadataLogOpt: Option[Log] =
logManager.getLog(TopicAndPartition(Topic.GroupMetadataTopicName, 0)) logManager.getLog(TopicAndPartition(Topic.GroupMetadataTopicName, 0))
TestUtils.waitUntilTrue(() => getGroupMetadataLogOpt.exists(_.logSegments.exists(_.log.nonEmpty)), TestUtils.waitUntilTrue(() => getGroupMetadataLogOpt.exists(_.logSegments.exists(_.log.shallowIterator.asScala.nonEmpty)),
"Commit message not appended in time") "Commit message not appended in time")
val logSegments = getGroupMetadataLogOpt.get.logSegments val logSegments = getGroupMetadataLogOpt.get.logSegments
val incorrectCompressionCodecs = logSegments.flatMap(_.log.map(_.message.compressionCodec)).filter(_ != offsetsTopicCompressionCodec) val incorrectCompressionCodecs = logSegments
.flatMap(_.log.shallowIterator.asScala.map(_.record.compressionType.id))
.filter(_ != offsetsTopicCompressionCodec.codec)
assertEquals("Incorrect compression codecs should be empty", Seq.empty, incorrectCompressionCodecs) assertEquals("Incorrect compression codecs should be empty", Seq.empty, incorrectCompressionCodecs)
consumer.close() consumer.close()

View File

@ -21,12 +21,15 @@ import joptsimple.OptionParser
import java.util.Properties import java.util.Properties
import java.util.Random import java.util.Random
import java.io._ import java.io._
import kafka.consumer._ import kafka.consumer._
import kafka.serializer._ import kafka.serializer._
import kafka.utils._ import kafka.utils._
import kafka.log.FileMessageSet
import kafka.log.Log import kafka.log.Log
import org.apache.kafka.clients.producer.{ProducerRecord, KafkaProducer, ProducerConfig} import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord}
import org.apache.kafka.common.record.FileRecords
import scala.collection.JavaConverters._
/** /**
* This is a torture test that runs against an existing broker. Here is how it works: * This is a torture test that runs against an existing broker. Here is how it works:
@ -135,15 +138,15 @@ object TestLogCleaning {
def dumpLog(dir: File) { def dumpLog(dir: File) {
require(dir.exists, "Non-existent directory: " + dir.getAbsolutePath) require(dir.exists, "Non-existent directory: " + dir.getAbsolutePath)
for(file <- dir.list.sorted; if file.endsWith(Log.LogFileSuffix)) { for (file <- dir.list.sorted; if file.endsWith(Log.LogFileSuffix)) {
val ms = new FileMessageSet(new File(dir, file)) val fileRecords = FileRecords.open(new File(dir, file))
for(entry <- ms) { for (entry <- fileRecords.shallowIterator.asScala) {
val key = TestUtils.readString(entry.message.key) val key = TestUtils.readString(entry.record.key)
val content = val content =
if(entry.message.isNull) if(entry.record.hasNullValue)
null null
else else
TestUtils.readString(entry.message.payload) TestUtils.readString(entry.record.value)
println("offset = %s, key = %s, content = %s".format(entry.offset, key, content)) println("offset = %s, key = %s, content = %s".format(entry.offset, key, content))
} }
} }

View File

@ -20,10 +20,10 @@ package kafka
import java.util.Properties import java.util.Properties
import java.util.concurrent.atomic._ import java.util.concurrent.atomic._
import kafka.message._
import kafka.log._ import kafka.log._
import kafka.utils._ import kafka.utils._
import org.apache.kafka.clients.consumer.OffsetOutOfRangeException import org.apache.kafka.clients.consumer.OffsetOutOfRangeException
import org.apache.kafka.common.record.FileRecords
import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.utils.Utils
/** /**
@ -36,13 +36,13 @@ object StressTestLog {
def main(args: Array[String]) { def main(args: Array[String]) {
val dir = TestUtils.randomPartitionLogDir(TestUtils.tempDir()) val dir = TestUtils.randomPartitionLogDir(TestUtils.tempDir())
val time = new MockTime val time = new MockTime
val logProprties = new Properties() val logProperties = new Properties()
logProprties.put(LogConfig.SegmentBytesProp, 64*1024*1024: java.lang.Integer) logProperties.put(LogConfig.SegmentBytesProp, 64*1024*1024: java.lang.Integer)
logProprties.put(LogConfig.MaxMessageBytesProp, Int.MaxValue: java.lang.Integer) logProperties.put(LogConfig.MaxMessageBytesProp, Int.MaxValue: java.lang.Integer)
logProprties.put(LogConfig.SegmentIndexBytesProp, 1024*1024: java.lang.Integer) logProperties.put(LogConfig.SegmentIndexBytesProp, 1024*1024: java.lang.Integer)
val log = new Log(dir = dir, val log = new Log(dir = dir,
config = LogConfig(logProprties), config = LogConfig(logProperties),
recoveryPoint = 0L, recoveryPoint = 0L,
scheduler = time.scheduler, scheduler = time.scheduler,
time = time) time = time)
@ -84,7 +84,7 @@ object StressTestLog {
class WriterThread(val log: Log) extends WorkerThread { class WriterThread(val log: Log) extends WorkerThread {
@volatile var offset = 0 @volatile var offset = 0
override def work() { override def work() {
val logAppendInfo = log.append(TestUtils.singleMessageSet(offset.toString.getBytes)) val logAppendInfo = log.append(TestUtils.singletonRecords(offset.toString.getBytes))
require(logAppendInfo.firstOffset == offset && logAppendInfo.lastOffset == offset) require(logAppendInfo.firstOffset == offset && logAppendInfo.lastOffset == offset)
offset += 1 offset += 1
if(offset % 1000 == 0) if(offset % 1000 == 0)
@ -96,11 +96,11 @@ object StressTestLog {
@volatile var offset = 0 @volatile var offset = 0
override def work() { override def work() {
try { try {
log.read(offset, 1024, Some(offset+1)).messageSet match { log.read(offset, 1024, Some(offset+1)).records match {
case read: FileMessageSet if read.sizeInBytes > 0 => { case read: FileRecords if read.sizeInBytes > 0 => {
val first = read.head val first = read.shallowIterator.next()
require(first.offset == offset, "We should either read nothing or the message we asked for.") require(first.offset == offset, "We should either read nothing or the message we asked for.")
require(MessageSet.entrySize(first.message) == read.sizeInBytes, "Expected %d but got %d.".format(MessageSet.entrySize(first.message), read.sizeInBytes)) require(first.sizeInBytes == read.sizeInBytes, "Expected %d but got %d.".format(first.sizeInBytes, read.sizeInBytes))
offset += 1 offset += 1
} }
case _ => case _ =>

View File

@ -22,13 +22,14 @@ import java.nio._
import java.nio.channels._ import java.nio.channels._
import java.util.{Properties, Random} import java.util.{Properties, Random}
import joptsimple._
import kafka.log._ import kafka.log._
import kafka.utils._
import kafka.message._ import kafka.message._
import kafka.utils._
import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Record}
import org.apache.kafka.common.utils.{Time, Utils}
import scala.math._ import scala.math._
import joptsimple._
import org.apache.kafka.common.utils.{Time, Utils}
/** /**
* This test does linear writes using either a kafka log or a file and measures throughput and latency. * This test does linear writes using either a kafka log or a file and measures throughput and latency.
@ -64,7 +65,7 @@ object TestLinearWriteSpeed {
.withRequiredArg .withRequiredArg
.describedAs("ms") .describedAs("ms")
.ofType(classOf[java.lang.Long]) .ofType(classOf[java.lang.Long])
.defaultsTo(1000) .defaultsTo(1000L)
val maxThroughputOpt = parser.accepts("max-throughput-mb", "The maximum throughput.") val maxThroughputOpt = parser.accepts("max-throughput-mb", "The maximum throughput.")
.withRequiredArg .withRequiredArg
.describedAs("mb") .describedAs("mb")
@ -81,7 +82,7 @@ object TestLinearWriteSpeed {
.ofType(classOf[java.lang.String]) .ofType(classOf[java.lang.String])
.defaultsTo(NoCompressionCodec.name) .defaultsTo(NoCompressionCodec.name)
val mmapOpt = parser.accepts("mmap", "Do writes to memory-mapped files.") val mmapOpt = parser.accepts("mmap", "Do writes to memory-mapped files.")
val channelOpt = parser.accepts("channel", "Do writes to file channesl.") val channelOpt = parser.accepts("channel", "Do writes to file channels.")
val logOpt = parser.accepts("log", "Do writes to kafka logs.") val logOpt = parser.accepts("log", "Do writes to kafka logs.")
val options = parser.parse(args : _*) val options = parser.parse(args : _*)
@ -101,9 +102,9 @@ object TestLinearWriteSpeed {
val rand = new Random val rand = new Random
rand.nextBytes(buffer.array) rand.nextBytes(buffer.array)
val numMessages = bufferSize / (messageSize + MessageSet.LogOverhead) val numMessages = bufferSize / (messageSize + MessageSet.LogOverhead)
val messageSet = new ByteBufferMessageSet(compressionCodec = compressionCodec, val messageSet = MemoryRecords.withRecords(CompressionType.forId(compressionCodec.codec),
messages = (0 until numMessages).map(_ => new Message(new Array[Byte](messageSize))): _*) (0 until numMessages).map(_ => Record.create(new Array[Byte](messageSize))): _*)
val writables = new Array[Writable](numFiles) val writables = new Array[Writable](numFiles)
val scheduler = new KafkaScheduler(1) val scheduler = new KafkaScheduler(1)
scheduler.startup() scheduler.startup()
@ -199,7 +200,7 @@ object TestLinearWriteSpeed {
} }
} }
class LogWritable(val dir: File, config: LogConfig, scheduler: Scheduler, val messages: ByteBufferMessageSet) extends Writable { class LogWritable(val dir: File, config: LogConfig, scheduler: Scheduler, val messages: MemoryRecords) extends Writable {
Utils.delete(dir) Utils.delete(dir)
val log = new Log(dir, config, 0L, scheduler, Time.SYSTEM) val log = new Log(dir, config, 0L, scheduler, Time.SYSTEM)
def write(): Int = { def write(): Int = {

View File

@ -301,7 +301,7 @@ class DeleteTopicTest extends ZooKeeperTestHarness {
var counter = 0 var counter = 0
for (_ <- 0 until numDups; key <- 0 until numKeys) yield { for (_ <- 0 until numDups; key <- 0 until numKeys) yield {
val count = counter val count = counter
log.append(TestUtils.singleMessageSet(payload = counter.toString.getBytes, key = key.toString.getBytes), assignOffsets = true) log.append(TestUtils.singletonRecords(value = counter.toString.getBytes, key = key.toString.getBytes), assignOffsets = true)
counter += 1 counter += 1
(key, count) (key, count)
} }

View File

@ -18,20 +18,20 @@
package kafka.coordinator package kafka.coordinator
import kafka.utils.timer.MockTimer import kafka.utils.timer.MockTimer
import org.apache.kafka.common.record.Record import org.apache.kafka.common.record.{MemoryRecords, Record, TimestampType}
import org.junit.Assert._ import org.junit.Assert._
import kafka.common.{OffsetAndMetadata, Topic} import kafka.common.{OffsetAndMetadata, Topic}
import kafka.message.{Message, MessageSet} import kafka.server.{DelayedOperationPurgatory, KafkaConfig, ReplicaManager}
import kafka.server.{DelayedOperationPurgatory, ReplicaManager, KafkaConfig}
import kafka.utils._ import kafka.utils._
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.requests.{OffsetCommitRequest, JoinGroupRequest} import org.apache.kafka.common.requests.{JoinGroupRequest, OffsetCommitRequest}
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
import org.easymock.{Capture, IAnswer, EasyMock} import org.easymock.{Capture, EasyMock, IAnswer}
import org.junit.{After, Before, Test} import org.junit.{After, Before, Test}
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import scala.collection._ import scala.collection._
import scala.concurrent.duration.Duration import scala.concurrent.duration.Duration
import scala.concurrent.{Await, Future, Promise} import scala.concurrent.{Await, Future, Promise}
@ -305,7 +305,8 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
EasyMock.reset(replicaManager) EasyMock.reset(replicaManager)
EasyMock.expect(replicaManager.getPartition(Topic.GroupMetadataTopicName, groupPartitionId)).andReturn(None) EasyMock.expect(replicaManager.getPartition(Topic.GroupMetadataTopicName, groupPartitionId)).andReturn(None)
EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andReturn(Some(Message.MagicValue_V1)).anyTimes() EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject()))
.andReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME)).anyTimes()
EasyMock.replay(replicaManager) EasyMock.replay(replicaManager)
timer.advanceClock(DefaultSessionTimeout + 100) timer.advanceClock(DefaultSessionTimeout + 100)
@ -988,17 +989,18 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
val capturedArgument: Capture[Map[TopicPartition, PartitionResponse] => Unit] = EasyMock.newCapture() val capturedArgument: Capture[Map[TopicPartition, PartitionResponse] => Unit] = EasyMock.newCapture()
EasyMock.expect(replicaManager.appendMessages(EasyMock.anyLong(), EasyMock.expect(replicaManager.appendRecords(EasyMock.anyLong(),
EasyMock.anyShort(), EasyMock.anyShort(),
EasyMock.anyBoolean(), EasyMock.anyBoolean(),
EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MessageSet]], EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MemoryRecords]],
EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] { EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] {
override def answer = capturedArgument.getValue.apply( override def answer = capturedArgument.getValue.apply(
Map(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId) -> Map(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId) ->
new PartitionResponse(Errors.NONE.code, 0L, Record.NO_TIMESTAMP) new PartitionResponse(Errors.NONE.code, 0L, Record.NO_TIMESTAMP)
) )
)}) )})
EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andReturn(Some(Message.MagicValue_V1)).anyTimes() EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject()))
.andReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME)).anyTimes()
EasyMock.replay(replicaManager) EasyMock.replay(replicaManager)
groupCoordinator.handleSyncGroup(groupId, generation, leaderId, assignment, responseCallback) groupCoordinator.handleSyncGroup(groupId, generation, leaderId, assignment, responseCallback)
@ -1069,17 +1071,18 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
val capturedArgument: Capture[Map[TopicPartition, PartitionResponse] => Unit] = EasyMock.newCapture() val capturedArgument: Capture[Map[TopicPartition, PartitionResponse] => Unit] = EasyMock.newCapture()
EasyMock.expect(replicaManager.appendMessages(EasyMock.anyLong(), EasyMock.expect(replicaManager.appendRecords(EasyMock.anyLong(),
EasyMock.anyShort(), EasyMock.anyShort(),
EasyMock.anyBoolean(), EasyMock.anyBoolean(),
EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MessageSet]], EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MemoryRecords]],
EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] { EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] {
override def answer = capturedArgument.getValue.apply( override def answer = capturedArgument.getValue.apply(
Map(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId) -> Map(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId) ->
new PartitionResponse(Errors.NONE.code, 0L, Record.NO_TIMESTAMP) new PartitionResponse(Errors.NONE.code, 0L, Record.NO_TIMESTAMP)
) )
)}) )})
EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andReturn(Some(Message.MagicValue_V1)).anyTimes() EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject()))
.andReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME)).anyTimes()
EasyMock.replay(replicaManager) EasyMock.replay(replicaManager)
groupCoordinator.handleCommitOffsets(groupId, consumerId, generationId, offsets, responseCallback) groupCoordinator.handleCommitOffsets(groupId, consumerId, generationId, offsets, responseCallback)
@ -1090,7 +1093,8 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
val (responseFuture, responseCallback) = setupHeartbeatCallback val (responseFuture, responseCallback) = setupHeartbeatCallback
EasyMock.expect(replicaManager.getPartition(Topic.GroupMetadataTopicName, groupPartitionId)).andReturn(None) EasyMock.expect(replicaManager.getPartition(Topic.GroupMetadataTopicName, groupPartitionId)).andReturn(None)
EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andReturn(Some(Message.MagicValue_V1)).anyTimes() EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject()))
.andReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME)).anyTimes()
EasyMock.replay(replicaManager) EasyMock.replay(replicaManager)
groupCoordinator.handleLeaveGroup(groupId, consumerId, responseCallback) groupCoordinator.handleLeaveGroup(groupId, consumerId, responseCallback)

View File

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

View File

@ -25,9 +25,10 @@ import org.junit.Assert._
import org.junit.runner.RunWith import org.junit.runner.RunWith
import org.junit.runners.Parameterized import org.junit.runners.Parameterized
import org.junit.runners.Parameterized.Parameters import org.junit.runners.Parameterized.Parameters
import org.apache.kafka.common.record.CompressionType import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Record}
import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.utils.Utils
import java.util.{Collection, Properties} import java.util.{Collection, Properties}
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
@RunWith(value = classOf[Parameterized]) @RunWith(value = classOf[Parameterized])
@ -50,22 +51,22 @@ class BrokerCompressionTest(messageCompression: String, brokerCompression: Strin
def testBrokerSideCompression() { def testBrokerSideCompression() {
val messageCompressionCode = CompressionCodec.getCompressionCodec(messageCompression) val messageCompressionCode = CompressionCodec.getCompressionCodec(messageCompression)
val logProps = new Properties() val logProps = new Properties()
logProps.put(LogConfig.CompressionTypeProp,brokerCompression) logProps.put(LogConfig.CompressionTypeProp, brokerCompression)
/*configure broker-side compression */ /*configure broker-side compression */
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
/* append two messages */ /* append two messages */
log.append(new ByteBufferMessageSet(messageCompressionCode, new Message("hello".getBytes), new Message("there".getBytes))) log.append(MemoryRecords.withRecords(CompressionType.forId(messageCompressionCode.codec),
Record.create("hello".getBytes), Record.create("there".getBytes)))
def readMessage(offset: Int) = log.read(offset, 4096).messageSet.head.message def readMessage(offset: Int) = log.read(offset, 4096).records.shallowIterator.next().record
if (!brokerCompression.equals("producer")) { if (!brokerCompression.equals("producer")) {
val brokerCompressionCode = BrokerCompressionCodec.getCompressionCodec(brokerCompression) val brokerCompressionCode = BrokerCompressionCodec.getCompressionCodec(brokerCompression)
assertEquals("Compression at offset 0 should produce " + brokerCompressionCode.name, brokerCompressionCode, readMessage(0).compressionCodec) assertEquals("Compression at offset 0 should produce " + brokerCompressionCode.name, brokerCompressionCode.codec, readMessage(0).compressionType.id)
} }
else else
assertEquals("Compression at offset 0 should produce " + messageCompressionCode.name, messageCompressionCode, readMessage(0).compressionCodec) assertEquals("Compression at offset 0 should produce " + messageCompressionCode.name, messageCompressionCode.codec, readMessage(0).compressionType.id)
} }
} }

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

View File

@ -21,7 +21,6 @@ import java.io.File
import java.util.Properties import java.util.Properties
import kafka.common.TopicAndPartition import kafka.common.TopicAndPartition
import kafka.message._
import kafka.utils._ import kafka.utils._
import org.apache.kafka.common.record.CompressionType import org.apache.kafka.common.record.CompressionType
import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.utils.Utils
@ -33,7 +32,6 @@ import org.junit.runners.Parameterized.Parameters
import scala.collection._ import scala.collection._
/** /**
* This is an integration test that tests the fully integrated log cleaner * This is an integration test that tests the fully integrated log cleaner
*/ */
@ -52,7 +50,7 @@ class LogCleanerLagIntegrationTest(compressionCodecName: String) extends Logging
val logDir = TestUtils.tempDir() val logDir = TestUtils.tempDir()
var counter = 0 var counter = 0
val topics = Array(TopicAndPartition("log", 0), TopicAndPartition("log", 1), TopicAndPartition("log", 2)) val topics = Array(TopicAndPartition("log", 0), TopicAndPartition("log", 1), TopicAndPartition("log", 2))
val compressionCodec = CompressionCodec.getCompressionCodec(compressionCodecName) val compressionCodec = CompressionType.forName(compressionCodecName)
@Test @Test
def cleanerTest(): Unit = { def cleanerTest(): Unit = {
@ -96,7 +94,7 @@ class LogCleanerLagIntegrationTest(compressionCodecName: String) extends Logging
val compactedSize = log.logSegments(0L, activeSegAtT0.baseOffset).map(_.size).sum val compactedSize = log.logSegments(0L, activeSegAtT0.baseOffset).map(_.size).sum
debug(s"after cleaning the compacted size up to active segment at T0: $compactedSize") debug(s"after cleaning the compacted size up to active segment at T0: $compactedSize")
val lastCleaned = cleaner.cleanerManager.allCleanerCheckpoints.get(TopicAndPartition("log", 0)).get val lastCleaned = cleaner.cleanerManager.allCleanerCheckpoints(TopicAndPartition("log", 0))
assertTrue(s"log cleaner should have processed up to offset $firstBlock1SegmentBaseOffset, but lastCleaned=$lastCleaned", lastCleaned >= firstBlock1SegmentBaseOffset) assertTrue(s"log cleaner should have processed up to offset $firstBlock1SegmentBaseOffset, but lastCleaned=$lastCleaned", lastCleaned >= firstBlock1SegmentBaseOffset)
assertTrue(s"log should have been compacted: size up to offset of active segment at T0=$sizeUpToActiveSegmentAtT0 compacted size=$compactedSize", assertTrue(s"log should have been compacted: size up to offset of active segment at T0=$sizeUpToActiveSegmentAtT0 compacted size=$compactedSize",
sizeUpToActiveSegmentAtT0 > compactedSize) sizeUpToActiveSegmentAtT0 > compactedSize)
@ -106,23 +104,19 @@ class LogCleanerLagIntegrationTest(compressionCodecName: String) extends Logging
} }
private def readFromLog(log: Log): Iterable[(Int, Int)] = { private def readFromLog(log: Log): Iterable[(Int, Int)] = {
for (segment <- log.logSegments; entry <- segment.log; messageAndOffset <- { import JavaConverters._
// create single message iterator or deep iterator depending on compression codec
if (entry.message.compressionCodec == NoCompressionCodec) for (segment <- log.logSegments; logEntry <- segment.log.deepIterator.asScala) yield {
Stream.cons(entry, Stream.empty).iterator val key = TestUtils.readString(logEntry.record.key).toInt
else val value = TestUtils.readString(logEntry.record.value).toInt
ByteBufferMessageSet.deepIterator(entry)
}) yield {
val key = TestUtils.readString(messageAndOffset.message.key).toInt
val value = TestUtils.readString(messageAndOffset.message.payload).toInt
key -> value key -> value
} }
} }
private def writeDups(numKeys: Int, numDups: Int, log: Log, codec: CompressionCodec, timestamp: Long): Seq[(Int, Int)] = { private def writeDups(numKeys: Int, numDups: Int, log: Log, codec: CompressionType, timestamp: Long): Seq[(Int, Int)] = {
for (_ <- 0 until numDups; key <- 0 until numKeys) yield { for (_ <- 0 until numDups; key <- 0 until numKeys) yield {
val count = counter val count = counter
log.append(TestUtils.singleMessageSet(payload = counter.toString.getBytes, codec = codec, key = key.toString.getBytes, timestamp = timestamp), assignOffsets = true) log.append(TestUtils.singletonRecords(value = counter.toString.getBytes, codec = codec, key = key.toString.getBytes, timestamp = timestamp), assignOffsets = true)
counter += 1 counter += 1
(key, count) (key, count)
} }

View File

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

View File

@ -17,21 +17,21 @@
package kafka.log package kafka.log
import java.io.{DataOutputStream, File} import java.io.File
import java.nio._ import java.nio._
import java.nio.file.Paths import java.nio.file.Paths
import java.util.Properties import java.util.Properties
import kafka.common._ import kafka.common._
import kafka.message._
import kafka.utils._ import kafka.utils._
import org.apache.kafka.common.record.{MemoryRecords, TimestampType} import org.apache.kafka.common.record._
import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.utils.Utils
import org.junit.Assert._ import org.junit.Assert._
import org.junit.{After, Test} import org.junit.{After, Test}
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite
import scala.collection._ import scala.collection._
import JavaConverters._
/** /**
* Unit tests for the log cleaning logic * Unit tests for the log cleaning logic
@ -66,7 +66,7 @@ class LogCleanerTest extends JUnitSuite {
// append messages to the log until we have four segments // append messages to the log until we have four segments
while(log.numberOfSegments < 4) while(log.numberOfSegments < 4)
log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt)) log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
val keysFound = keysInLog(log) val keysFound = keysInLog(log)
assertEquals(0L until log.logEndOffset, keysFound) assertEquals(0L until log.logEndOffset, keysFound)
@ -100,7 +100,7 @@ class LogCleanerTest extends JUnitSuite {
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
while(log.numberOfSegments < 2) while(log.numberOfSegments < 2)
log.append(message(log.logEndOffset.toInt, Array.fill(largeMessageSize)(0: Byte))) log.append(record(log.logEndOffset.toInt, Array.fill(largeMessageSize)(0: Byte)))
val keysFound = keysInLog(log) val keysFound = keysInLog(log)
assertEquals(0L until log.logEndOffset, keysFound) assertEquals(0L until log.logEndOffset, keysFound)
@ -123,23 +123,23 @@ class LogCleanerTest extends JUnitSuite {
logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer)
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
// append messages with the keys 0 through N // append messages with the keys 0 through N
while(log.numberOfSegments < 2) while(log.numberOfSegments < 2)
log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt)) log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
// delete all even keys between 0 and N // delete all even keys between 0 and N
val leo = log.logEndOffset val leo = log.logEndOffset
for(key <- 0 until leo.toInt by 2) for(key <- 0 until leo.toInt by 2)
log.append(deleteMessage(key)) log.append(tombstoneRecord(key))
// append some new unique keys to pad out to a new active segment // append some new unique keys to pad out to a new active segment
while(log.numberOfSegments < 4) while(log.numberOfSegments < 4)
log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt)) log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
cleaner.clean(LogToClean(TopicAndPartition("test", 0), log, 0, log.activeSegment.baseOffset)) cleaner.clean(LogToClean(TopicAndPartition("test", 0), log, 0, log.activeSegment.baseOffset))
val keys = keysInLog(log).toSet val keys = keysInLog(log).toSet
assertTrue("None of the keys we deleted should still exist.", assertTrue("None of the keys we deleted should still exist.",
(0 until leo.toInt by 2).forall(!keys.contains(_))) (0 until leo.toInt by 2).forall(!keys.contains(_)))
} }
@ -151,11 +151,11 @@ class LogCleanerTest extends JUnitSuite {
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
log.append(message(0,0)) // offset 0 log.append(record(0,0)) // offset 0
log.append(message(1,1)) // offset 1 log.append(record(1,1)) // offset 1
log.append(message(0,0)) // offset 2 log.append(record(0,0)) // offset 2
log.append(message(1,1)) // offset 3 log.append(record(1,1)) // offset 3
log.append(message(0,0)) // offset 4 log.append(record(0,0)) // offset 4
// roll the segment, so we can clean the messages already appended // roll the segment, so we can clean the messages already appended
log.roll() log.roll()
@ -180,11 +180,11 @@ class LogCleanerTest extends JUnitSuite {
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
log.append(message(0,0)) // offset 0 log.append(record(0,0)) // offset 0
log.append(message(1,1)) // offset 1 log.append(record(1,1)) // offset 1
log.append(message(0,0)) // offset 2 log.append(record(0,0)) // offset 2
log.append(message(1,1)) // offset 3 log.append(record(1,1)) // offset 3
log.append(message(0,0)) // offset 4 log.append(record(0,0)) // offset 4
// roll the segment, so we can clean the messages already appended // roll the segment, so we can clean the messages already appended
log.roll() log.roll()
@ -218,18 +218,18 @@ class LogCleanerTest extends JUnitSuite {
// append messages with the keys 0 through N-1, values equal offset // append messages with the keys 0 through N-1, values equal offset
while(log.numberOfSegments <= numCleanableSegments) while(log.numberOfSegments <= numCleanableSegments)
log.append(message(log.logEndOffset.toInt % N, log.logEndOffset.toInt)) log.append(record(log.logEndOffset.toInt % N, log.logEndOffset.toInt))
// at this point one message past the cleanable segments has been added // at this point one message past the cleanable segments has been added
// the entire segment containing the first uncleanable offset should not be cleaned. // the entire segment containing the first uncleanable offset should not be cleaned.
val firstUncleanableOffset = log.logEndOffset + 1 // +1 so it is past the baseOffset val firstUncleanableOffset = log.logEndOffset + 1 // +1 so it is past the baseOffset
while(log.numberOfSegments < numTotalSegments - 1) while(log.numberOfSegments < numTotalSegments - 1)
log.append(message(log.logEndOffset.toInt % N, log.logEndOffset.toInt)) log.append(record(log.logEndOffset.toInt % N, log.logEndOffset.toInt))
// the last (active) segment has just one message // the last (active) segment has just one message
def distinctValuesBySegment = log.logSegments.map(s => s.log.map(m => TestUtils.readString(m.message.payload)).toSet.size).toSeq def distinctValuesBySegment = log.logSegments.map(s => s.log.shallowIterator.asScala.map(m => TestUtils.readString(m.record.value)).toSet.size).toSeq
val disctinctValuesBySegmentBeforeClean = distinctValuesBySegment val disctinctValuesBySegmentBeforeClean = distinctValuesBySegment
assertTrue("Test is not effective unless each segment contains duplicates. Increase segment size or decrease number of keys.", assertTrue("Test is not effective unless each segment contains duplicates. Increase segment size or decrease number of keys.",
@ -253,7 +253,7 @@ class LogCleanerTest extends JUnitSuite {
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
// create 6 segments with only one message in each segment // create 6 segments with only one message in each segment
val messageSet = TestUtils.singleMessageSet(payload = Array.fill[Byte](50)(0), key = 1.toString.getBytes) val messageSet = TestUtils.singletonRecords(value = Array.fill[Byte](50)(0), key = 1.toString.getBytes)
for (_ <- 0 until 6) for (_ <- 0 until 6)
log.append(messageSet, assignOffsets = true) log.append(messageSet, assignOffsets = true)
@ -271,7 +271,7 @@ class LogCleanerTest extends JUnitSuite {
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
// create 6 segments with only one message in each segment // create 6 segments with only one message in each segment
val messageSet = TestUtils.singleMessageSet(payload = Array.fill[Byte](50)(0), key = 1.toString.getBytes) val messageSet = TestUtils.singletonRecords(value = Array.fill[Byte](50)(0), key = 1.toString.getBytes)
for (_ <- 0 until 6) for (_ <- 0 until 6)
log.append(messageSet, assignOffsets = true) log.append(messageSet, assignOffsets = true)
@ -305,14 +305,14 @@ class LogCleanerTest extends JUnitSuite {
// append unkeyed messages // append unkeyed messages
while(log.numberOfSegments < 2) while(log.numberOfSegments < 2)
log.append(unkeyedMessage(log.logEndOffset.toInt)) log.append(unkeyedRecord(log.logEndOffset.toInt))
val numInvalidMessages = unkeyedMessageCountInLog(log) val numInvalidMessages = unkeyedMessageCountInLog(log)
val sizeWithUnkeyedMessages = log.size val sizeWithUnkeyedMessages = log.size
// append keyed messages // append keyed messages
while(log.numberOfSegments < 3) while(log.numberOfSegments < 3)
log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt)) log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
val expectedSizeAfterCleaning = log.size - sizeWithUnkeyedMessages val expectedSizeAfterCleaning = log.size - sizeWithUnkeyedMessages
val (_, stats) = cleaner.clean(LogToClean(TopicAndPartition("test", 0), log, 0, log.activeSegment.baseOffset)) val (_, stats) = cleaner.clean(LogToClean(TopicAndPartition("test", 0), log, 0, log.activeSegment.baseOffset))
@ -321,17 +321,17 @@ class LogCleanerTest extends JUnitSuite {
assertEquals("Log should only contain keyed messages after cleaning.", expectedSizeAfterCleaning, log.size) assertEquals("Log should only contain keyed messages after cleaning.", expectedSizeAfterCleaning, log.size)
assertEquals("Cleaner should have seen %d invalid messages.", numInvalidMessages, stats.invalidMessagesRead) assertEquals("Cleaner should have seen %d invalid messages.", numInvalidMessages, stats.invalidMessagesRead)
} }
/* extract all the keys from a log */ /* extract all the keys from a log */
def keysInLog(log: Log): Iterable[Int] = def keysInLog(log: Log): Iterable[Int] =
log.logSegments.flatMap(s => s.log.filter(!_.message.isNull).filter(_.message.hasKey).map(m => TestUtils.readString(m.message.key).toInt)) log.logSegments.flatMap(s => s.log.shallowIterator.asScala.filter(!_.record.hasNullValue).filter(_.record.hasKey).map(m => TestUtils.readString(m.record.key).toInt))
/* extract all the offsets from a log */ /* extract all the offsets from a log */
def offsetsInLog(log: Log): Iterable[Long] = def offsetsInLog(log: Log): Iterable[Long] =
log.logSegments.flatMap(s => s.log.filter(!_.message.isNull).filter(_.message.hasKey).map(m => m.offset)) log.logSegments.flatMap(s => s.log.shallowIterator.asScala.filter(!_.record.hasNullValue).filter(_.record.hasKey).map(m => m.offset))
def unkeyedMessageCountInLog(log: Log) = def unkeyedMessageCountInLog(log: Log) =
log.logSegments.map(s => s.log.filter(!_.message.isNull).count(m => !m.message.hasKey)).sum log.logSegments.map(s => s.log.shallowIterator.asScala.filter(!_.record.hasNullValue).count(m => !m.record.hasKey)).sum
def abortCheckDone(topicAndPartition: TopicAndPartition): Unit = { def abortCheckDone(topicAndPartition: TopicAndPartition): Unit = {
throw new LogCleaningAbortedException() throw new LogCleaningAbortedException()
@ -350,7 +350,7 @@ class LogCleanerTest extends JUnitSuite {
// append messages to the log until we have four segments // append messages to the log until we have four segments
while(log.numberOfSegments < 4) while(log.numberOfSegments < 4)
log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt)) log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
val keys = keysInLog(log) val keys = keysInLog(log)
val map = new FakeOffsetMap(Int.MaxValue) val map = new FakeOffsetMap(Int.MaxValue)
@ -371,20 +371,20 @@ class LogCleanerTest extends JUnitSuite {
logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer)
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
// append some messages to the log // append some messages to the log
var i = 0 var i = 0
while(log.numberOfSegments < 10) { while(log.numberOfSegments < 10) {
log.append(TestUtils.singleMessageSet(payload = "hello".getBytes, key = "hello".getBytes)) log.append(TestUtils.singletonRecords(value = "hello".getBytes, key = "hello".getBytes))
i += 1 i += 1
} }
// grouping by very large values should result in a single group with all the segments in it // grouping by very large values should result in a single group with all the segments in it
var groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue) var groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue)
assertEquals(1, groups.size) assertEquals(1, groups.size)
assertEquals(log.numberOfSegments, groups.head.size) assertEquals(log.numberOfSegments, groups.head.size)
checkSegmentOrder(groups) checkSegmentOrder(groups)
// grouping by very small values should result in all groups having one entry // grouping by very small values should result in all groups having one entry
groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = 1, maxIndexSize = Int.MaxValue) groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = 1, maxIndexSize = Int.MaxValue)
assertEquals(log.numberOfSegments, groups.size) assertEquals(log.numberOfSegments, groups.size)
@ -396,20 +396,20 @@ class LogCleanerTest extends JUnitSuite {
checkSegmentOrder(groups) checkSegmentOrder(groups)
val groupSize = 3 val groupSize = 3
// check grouping by log size // check grouping by log size
val logSize = log.logSegments.take(groupSize).map(_.size).sum.toInt + 1 val logSize = log.logSegments.take(groupSize).map(_.size).sum.toInt + 1
groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = logSize, maxIndexSize = Int.MaxValue) groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = logSize, maxIndexSize = Int.MaxValue)
checkSegmentOrder(groups) checkSegmentOrder(groups)
assertTrue("All but the last group should be the target size.", groups.dropRight(1).forall(_.size == groupSize)) assertTrue("All but the last group should be the target size.", groups.dropRight(1).forall(_.size == groupSize))
// check grouping by index size // check grouping by index size
val indexSize = log.logSegments.take(groupSize).map(_.index.sizeInBytes).sum + 1 val indexSize = log.logSegments.take(groupSize).map(_.index.sizeInBytes).sum + 1
groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = indexSize) groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = indexSize)
checkSegmentOrder(groups) checkSegmentOrder(groups)
assertTrue("All but the last group should be the target size.", groups.dropRight(1).forall(_.size == groupSize)) assertTrue("All but the last group should be the target size.", groups.dropRight(1).forall(_.size == groupSize))
} }
/** /**
* Validate the logic for grouping log segments together for cleaning when only a small number of * Validate the logic for grouping log segments together for cleaning when only a small number of
* messages are retained, but the range of offsets is greater than Int.MaxValue. A group should not * messages are retained, but the range of offsets is greater than Int.MaxValue. A group should not
@ -425,47 +425,45 @@ class LogCleanerTest extends JUnitSuite {
logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer)
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps)) val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
// fill up first segment // fill up first segment
while (log.numberOfSegments == 1) while (log.numberOfSegments == 1)
log.append(TestUtils.singleMessageSet(payload = "hello".getBytes, key = "hello".getBytes)) log.append(TestUtils.singletonRecords(value = "hello".getBytes, key = "hello".getBytes))
// forward offset and append message to next segment at offset Int.MaxValue // forward offset and append message to next segment at offset Int.MaxValue
val messageSet = new ByteBufferMessageSet(NoCompressionCodec, new LongRef(Int.MaxValue - 1), val records = MemoryRecords.withLogEntries(LogEntry.create(Int.MaxValue - 1, Record.create("hello".getBytes, "hello".getBytes)))
new Message("hello".getBytes, "hello".getBytes, Message.NoTimestamp, Message.MagicValue_V1)) log.append(records, assignOffsets = false)
log.append(messageSet, assignOffsets = false) log.append(TestUtils.singletonRecords(value = "hello".getBytes, key = "hello".getBytes))
log.append(TestUtils.singleMessageSet(payload = "hello".getBytes, key = "hello".getBytes))
assertEquals(Int.MaxValue, log.activeSegment.index.lastOffset) assertEquals(Int.MaxValue, log.activeSegment.index.lastOffset)
// grouping should result in a single group with maximum relative offset of Int.MaxValue // grouping should result in a single group with maximum relative offset of Int.MaxValue
var groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue) var groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue)
assertEquals(1, groups.size) assertEquals(1, groups.size)
// append another message, making last offset of second segment > Int.MaxValue // append another message, making last offset of second segment > Int.MaxValue
log.append(TestUtils.singleMessageSet(payload = "hello".getBytes, key = "hello".getBytes)) log.append(TestUtils.singletonRecords(value = "hello".getBytes, key = "hello".getBytes))
// grouping should not group the two segments to ensure that maximum relative offset in each group <= Int.MaxValue // grouping should not group the two segments to ensure that maximum relative offset in each group <= Int.MaxValue
groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue) groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue)
assertEquals(2, groups.size) assertEquals(2, groups.size)
checkSegmentOrder(groups) checkSegmentOrder(groups)
// append more messages, creating new segments, further grouping should still occur // append more messages, creating new segments, further grouping should still occur
while (log.numberOfSegments < 4) while (log.numberOfSegments < 4)
log.append(TestUtils.singleMessageSet(payload = "hello".getBytes, key = "hello".getBytes)) log.append(TestUtils.singletonRecords(value = "hello".getBytes, key = "hello".getBytes))
groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue) groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue)
assertEquals(log.numberOfSegments - 1, groups.size) assertEquals(log.numberOfSegments - 1, groups.size)
for (group <- groups) for (group <- groups)
assertTrue("Relative offset greater than Int.MaxValue", group.last.index.lastOffset - group.head.index.baseOffset <= Int.MaxValue) assertTrue("Relative offset greater than Int.MaxValue", group.last.index.lastOffset - group.head.index.baseOffset <= Int.MaxValue)
checkSegmentOrder(groups) checkSegmentOrder(groups)
} }
private def checkSegmentOrder(groups: Seq[Seq[LogSegment]]): Unit = { private def checkSegmentOrder(groups: Seq[Seq[LogSegment]]): Unit = {
val offsets = groups.flatMap(_.map(_.baseOffset)) val offsets = groups.flatMap(_.map(_.baseOffset))
assertEquals("Offsets should be in increasing order.", offsets.sorted, offsets) assertEquals("Offsets should be in increasing order.", offsets.sorted, offsets)
} }
/** /**
* Test building an offset map off the log * Test building an offset map off the log
*/ */
@ -496,8 +494,7 @@ class LogCleanerTest extends JUnitSuite {
checkRange(map, segments(1).baseOffset.toInt, segments(3).baseOffset.toInt) checkRange(map, segments(1).baseOffset.toInt, segments(3).baseOffset.toInt)
checkRange(map, segments(3).baseOffset.toInt, log.logEndOffset.toInt) checkRange(map, segments(3).baseOffset.toInt, log.logEndOffset.toInt)
} }
/** /**
* Tests recovery if broker crashes at the following stages during the cleaning sequence * Tests recovery if broker crashes at the following stages during the cleaning sequence
* <ol> * <ol>
@ -516,8 +513,8 @@ class LogCleanerTest extends JUnitSuite {
logProps.put(LogConfig.FileDeleteDelayMsProp, 10: java.lang.Integer) logProps.put(LogConfig.FileDeleteDelayMsProp, 10: java.lang.Integer)
val config = LogConfig.fromProps(logConfig.originals, logProps) val config = LogConfig.fromProps(logConfig.originals, logProps)
def recoverAndCheck(config: LogConfig, expectedKeys : Iterable[Int]) : Log = { def recoverAndCheck(config: LogConfig, expectedKeys : Iterable[Int]) : Log = {
// Recover log file and check that after recovery, keys are as expected // Recover log file and check that after recovery, keys are as expected
// and all temporary files have been deleted // and all temporary files have been deleted
val recoveredLog = makeLog(config = config) val recoveredLog = makeLog(config = config)
@ -530,25 +527,25 @@ class LogCleanerTest extends JUnitSuite {
assertEquals(expectedKeys, keysInLog(recoveredLog)) assertEquals(expectedKeys, keysInLog(recoveredLog))
recoveredLog recoveredLog
} }
// create a log and append some messages // create a log and append some messages
var log = makeLog(config = config) var log = makeLog(config = config)
var messageCount = 0 var messageCount = 0
while(log.numberOfSegments < 10) { while(log.numberOfSegments < 10) {
log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt)) log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
messageCount += 1 messageCount += 1
} }
val allKeys = keysInLog(log) val allKeys = keysInLog(log)
// pretend we have odd-numbered keys // pretend we have odd-numbered keys
val offsetMap = new FakeOffsetMap(Int.MaxValue) val offsetMap = new FakeOffsetMap(Int.MaxValue)
for (k <- 1 until messageCount by 2) for (k <- 1 until messageCount by 2)
offsetMap.put(key(k), Long.MaxValue) offsetMap.put(key(k), Long.MaxValue)
// clean the log // clean the log
cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats()) cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats())
var cleanedKeys = keysInLog(log) var cleanedKeys = keysInLog(log)
// 1) Simulate recovery just after .cleaned file is created, before rename to .swap // 1) Simulate recovery just after .cleaned file is created, before rename to .swap
// On recovery, clean operation is aborted. All messages should be present in the log // On recovery, clean operation is aborted. All messages should be present in the log
log.logSegments.head.changeFileSuffixes("", Log.CleanedFileSuffix) log.logSegments.head.changeFileSuffixes("", Log.CleanedFileSuffix)
@ -556,44 +553,44 @@ class LogCleanerTest extends JUnitSuite {
Utils.atomicMoveWithFallback(file.toPath, Paths.get(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, ""))) Utils.atomicMoveWithFallback(file.toPath, Paths.get(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, "")))
} }
log = recoverAndCheck(config, allKeys) log = recoverAndCheck(config, allKeys)
// clean again // clean again
cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats()) cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats())
cleanedKeys = keysInLog(log) cleanedKeys = keysInLog(log)
// 2) Simulate recovery just after swap file is created, before old segment files are // 2) Simulate recovery just after swap file is created, before old segment files are
// renamed to .deleted. Clean operation is resumed during recovery. // renamed to .deleted. Clean operation is resumed during recovery.
log.logSegments.head.changeFileSuffixes("", Log.SwapFileSuffix) log.logSegments.head.changeFileSuffixes("", Log.SwapFileSuffix)
for (file <- dir.listFiles if file.getName.endsWith(Log.DeletedFileSuffix)) { for (file <- dir.listFiles if file.getName.endsWith(Log.DeletedFileSuffix)) {
Utils.atomicMoveWithFallback(file.toPath, Paths.get(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, ""))) Utils.atomicMoveWithFallback(file.toPath, Paths.get(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, "")))
} }
log = recoverAndCheck(config, cleanedKeys) log = recoverAndCheck(config, cleanedKeys)
// add some more messages and clean the log again // add some more messages and clean the log again
while(log.numberOfSegments < 10) { while(log.numberOfSegments < 10) {
log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt)) log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
messageCount += 1 messageCount += 1
} }
for (k <- 1 until messageCount by 2) for (k <- 1 until messageCount by 2)
offsetMap.put(key(k), Long.MaxValue) offsetMap.put(key(k), Long.MaxValue)
cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats()) cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats())
cleanedKeys = keysInLog(log) cleanedKeys = keysInLog(log)
// 3) Simulate recovery after swap file is created and old segments files are renamed // 3) Simulate recovery after swap file is created and old segments files are renamed
// to .deleted. Clean operation is resumed during recovery. // to .deleted. Clean operation is resumed during recovery.
log.logSegments.head.changeFileSuffixes("", Log.SwapFileSuffix) log.logSegments.head.changeFileSuffixes("", Log.SwapFileSuffix)
log = recoverAndCheck(config, cleanedKeys) log = recoverAndCheck(config, cleanedKeys)
// add some more messages and clean the log again // add some more messages and clean the log again
while(log.numberOfSegments < 10) { while(log.numberOfSegments < 10) {
log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt)) log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
messageCount += 1 messageCount += 1
} }
for (k <- 1 until messageCount by 2) for (k <- 1 until messageCount by 2)
offsetMap.put(key(k), Long.MaxValue) offsetMap.put(key(k), Long.MaxValue)
cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats()) cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats())
cleanedKeys = keysInLog(log) cleanedKeys = keysInLog(log)
// 4) Simulate recovery after swap is complete, but async deletion // 4) Simulate recovery after swap is complete, but async deletion
// is not yet complete. Clean operation is resumed during recovery. // is not yet complete. Clean operation is resumed during recovery.
recoverAndCheck(config, cleanedKeys) recoverAndCheck(config, cleanedKeys)
@ -631,11 +628,11 @@ class LogCleanerTest extends JUnitSuite {
val log = makeLog() val log = makeLog()
val cleaner = makeCleaner(2) val cleaner = makeCleaner(2)
log.append(message(0,0)) log.append(record(0,0))
log.append(message(1,1)) log.append(record(1,1))
log.append(message(2,2)) log.append(record(2,2))
log.append(message(3,3)) log.append(record(3,3))
log.append(message(4,4)) log.append(record(4,4))
log.roll() log.roll()
val stats = new CleanerStats() val stats = new CleanerStats()
@ -653,7 +650,7 @@ class LogCleanerTest extends JUnitSuite {
*/ */
@Test @Test
def testCleanCorruptMessageSet() { def testCleanCorruptMessageSet() {
val codec = SnappyCompressionCodec val codec = CompressionType.GZIP
val logProps = new Properties() val logProps = new Properties()
logProps.put(LogConfig.CompressionTypeProp, codec.name) logProps.put(LogConfig.CompressionTypeProp, codec.name)
@ -682,10 +679,10 @@ class LogCleanerTest extends JUnitSuite {
cleaner.clean(LogToClean(TopicAndPartition("test", 0), log, 0, log.activeSegment.baseOffset)) cleaner.clean(LogToClean(TopicAndPartition("test", 0), log, 0, log.activeSegment.baseOffset))
for (segment <- log.logSegments; shallowMessage <- segment.log.iterator; deepMessage <- ByteBufferMessageSet.deepIterator(shallowMessage)) { for (segment <- log.logSegments; shallowLogEntry <- segment.log.shallowIterator.asScala; deepLogEntry <- shallowLogEntry.asScala) {
assertEquals(shallowMessage.message.magic, deepMessage.message.magic) assertEquals(shallowLogEntry.record.magic, deepLogEntry.record.magic)
val value = TestUtils.readString(deepMessage.message.payload).toLong val value = TestUtils.readString(deepLogEntry.record.value).toLong
assertEquals(deepMessage.offset, value) assertEquals(deepLogEntry.offset, value)
} }
} }
@ -704,7 +701,7 @@ class LogCleanerTest extends JUnitSuite {
val corruptedMessage = invalidCleanedMessage(offset, set) val corruptedMessage = invalidCleanedMessage(offset, set)
val records = MemoryRecords.readableRecords(corruptedMessage.buffer) val records = MemoryRecords.readableRecords(corruptedMessage.buffer)
for (logEntry <- records.iterator.asScala) { for (logEntry <- records.deepIterator.asScala) {
val offset = logEntry.offset val offset = logEntry.offset
val value = TestUtils.readString(logEntry.record.value).toLong val value = TestUtils.readString(logEntry.record.value).toLong
assertEquals(offset, value) assertEquals(offset, value)
@ -718,94 +715,64 @@ class LogCleanerTest extends JUnitSuite {
private def invalidCleanedMessage(initialOffset: Long, private def invalidCleanedMessage(initialOffset: Long,
keysAndValues: Iterable[(Int, Int)], keysAndValues: Iterable[(Int, Int)],
codec: CompressionCodec = SnappyCompressionCodec): ByteBufferMessageSet = { codec: CompressionType = CompressionType.GZIP): MemoryRecords = {
// this function replicates the old versions of the cleaner which under some circumstances // this function replicates the old versions of the cleaner which under some circumstances
// would write invalid compressed message sets with the outer magic set to 1 and the inner // would write invalid compressed message sets with the outer magic set to 1 and the inner
// magic set to 0 // magic set to 0
val records = keysAndValues.map(kv =>
Record.create(Record.MAGIC_VALUE_V0,
Record.NO_TIMESTAMP,
kv._1.toString.getBytes,
kv._2.toString.getBytes))
val messages = keysAndValues.map(kv => val buffer = ByteBuffer.allocate(math.min(math.max(records.map(_.sizeInBytes()).sum / 2, 1024), 1 << 16))
new Message(key = kv._1.toString.getBytes, val builder = MemoryRecords.builder(buffer, Record.MAGIC_VALUE_V1, codec, TimestampType.CREATE_TIME)
bytes = kv._2.toString.getBytes,
timestamp = Message.NoTimestamp,
magicValue = Message.MagicValue_V0))
val messageWriter = new MessageWriter(math.min(math.max(MessageSet.messageSetSize(messages) / 2, 1024), 1 << 16)) var offset = initialOffset
var lastOffset = initialOffset records.foreach { record =>
builder.appendUnchecked(offset, record)
messageWriter.write( offset += 1
codec = codec,
timestamp = Message.NoTimestamp,
timestampType = TimestampType.CREATE_TIME,
magicValue = Message.MagicValue_V1) { outputStream =>
val output = new DataOutputStream(CompressionFactory(codec, Message.MagicValue_V1, outputStream))
try {
for (message <- messages) {
val innerOffset = lastOffset - initialOffset
output.writeLong(innerOffset)
output.writeInt(message.size)
output.write(message.buffer.array, message.buffer.arrayOffset, message.buffer.limit)
lastOffset += 1
}
} finally {
output.close()
}
} }
val buffer = ByteBuffer.allocate(messageWriter.size + MessageSet.LogOverhead)
ByteBufferMessageSet.writeMessage(buffer, messageWriter, lastOffset - 1)
buffer.rewind()
new ByteBufferMessageSet(buffer) builder.build()
} }
private def messageWithOffset(key: Int, value: Int, offset: Long) = private def messageWithOffset(key: Int, value: Int, offset: Long) =
new ByteBufferMessageSet(NoCompressionCodec, Seq(offset), MemoryRecords.withLogEntries(LogEntry.create(offset, Record.create(key.toString.getBytes, value.toString.getBytes)))
new Message(key = key.toString.getBytes,
bytes = value.toString.getBytes,
timestamp = Message.NoTimestamp,
magicValue = Message.MagicValue_V1))
def makeLog(dir: File = dir, config: LogConfig = logConfig) = def makeLog(dir: File = dir, config: LogConfig = logConfig) =
new Log(dir = dir, config = config, recoveryPoint = 0L, scheduler = time.scheduler, time = time) new Log(dir = dir, config = config, recoveryPoint = 0L, scheduler = time.scheduler, time = time)
def noOpCheckDone(topicAndPartition: TopicAndPartition) { /* do nothing */ } def noOpCheckDone(topicAndPartition: TopicAndPartition) { /* do nothing */ }
def makeCleaner(capacity: Int, checkDone: (TopicAndPartition) => Unit = noOpCheckDone, maxMessageSize: Int = 64*1024) = def makeCleaner(capacity: Int, checkDone: (TopicAndPartition) => Unit = noOpCheckDone, maxMessageSize: Int = 64*1024) =
new Cleaner(id = 0, new Cleaner(id = 0,
offsetMap = new FakeOffsetMap(capacity), offsetMap = new FakeOffsetMap(capacity),
ioBufferSize = maxMessageSize, ioBufferSize = maxMessageSize,
maxIoBufferSize = maxMessageSize, maxIoBufferSize = maxMessageSize,
dupBufferLoadFactor = 0.75, dupBufferLoadFactor = 0.75,
throttler = throttler, throttler = throttler,
time = time, time = time,
checkDone = checkDone ) checkDone = checkDone )
def writeToLog(log: Log, seq: Iterable[(Int, Int)]): Iterable[Long] = { def writeToLog(log: Log, seq: Iterable[(Int, Int)]): Iterable[Long] = {
for((key, value) <- seq) for((key, value) <- seq)
yield log.append(message(key, value)).firstOffset yield log.append(record(key, value)).firstOffset
} }
def key(id: Int) = ByteBuffer.wrap(id.toString.getBytes) def key(id: Int) = ByteBuffer.wrap(id.toString.getBytes)
def message(key: Int, value: Int): ByteBufferMessageSet =
message(key, value.toString.getBytes)
def message(key: Int, value: Array[Byte]) = def record(key: Int, value: Int): MemoryRecords =
new ByteBufferMessageSet(new Message(key = key.toString.getBytes, record(key, value.toString.getBytes)
bytes = value,
timestamp = Message.NoTimestamp,
magicValue = Message.MagicValue_V1))
def unkeyedMessage(value: Int) = def record(key: Int, value: Array[Byte]) =
new ByteBufferMessageSet(new Message(bytes = value.toString.getBytes)) MemoryRecords.withRecords(Record.create(key.toString.getBytes, value))
def unkeyedRecord(value: Int) =
MemoryRecords.withRecords(Record.create(value.toString.getBytes))
def tombstoneRecord(key: Int) = record(key, null)
def deleteMessage(key: Int) =
new ByteBufferMessageSet(new Message(key = key.toString.getBytes,
bytes = null,
timestamp = Message.NoTimestamp,
magicValue = Message.MagicValue_V1))
} }
class FakeOffsetMap(val slots: Int) extends OffsetMap { class FakeOffsetMap(val slots: Int) extends OffsetMap {

View File

@ -67,7 +67,7 @@ class LogManagerTest {
val log = logManager.createLog(TopicAndPartition(name, 0), logConfig) val log = logManager.createLog(TopicAndPartition(name, 0), logConfig)
val logFile = new File(logDir, name + "-0") val logFile = new File(logDir, name + "-0")
assertTrue(logFile.exists) assertTrue(logFile.exists)
log.append(TestUtils.singleMessageSet("test".getBytes())) log.append(TestUtils.singletonRecords("test".getBytes()))
} }
/** /**
@ -89,7 +89,7 @@ class LogManagerTest {
val log = logManager.createLog(TopicAndPartition(name, 0), logConfig) val log = logManager.createLog(TopicAndPartition(name, 0), logConfig)
var offset = 0L var offset = 0L
for(_ <- 0 until 200) { for(_ <- 0 until 200) {
val set = TestUtils.singleMessageSet("test".getBytes()) val set = TestUtils.singletonRecords("test".getBytes())
val info = log.append(set) val info = log.append(set)
offset = info.lastOffset offset = info.lastOffset
} }
@ -101,7 +101,7 @@ class LogManagerTest {
assertEquals("Now there should only be only one segment in the index.", 1, log.numberOfSegments) assertEquals("Now there should only be only one segment in the index.", 1, log.numberOfSegments)
time.sleep(log.config.fileDeleteDelayMs + 1) time.sleep(log.config.fileDeleteDelayMs + 1)
assertEquals("Files should have been deleted", log.numberOfSegments * 3, log.dir.list.length) assertEquals("Files should have been deleted", log.numberOfSegments * 3, log.dir.list.length)
assertEquals("Should get empty fetch off new log.", 0, log.read(offset+1, 1024).messageSet.sizeInBytes) assertEquals("Should get empty fetch off new log.", 0, log.read(offset+1, 1024).records.sizeInBytes)
try { try {
log.read(0, 1024) log.read(0, 1024)
@ -110,7 +110,7 @@ class LogManagerTest {
case _: OffsetOutOfRangeException => // This is good. case _: OffsetOutOfRangeException => // This is good.
} }
// log should still be appendable // log should still be appendable
log.append(TestUtils.singleMessageSet("test".getBytes())) log.append(TestUtils.singletonRecords("test".getBytes()))
} }
/** /**
@ -118,7 +118,7 @@ class LogManagerTest {
*/ */
@Test @Test
def testCleanupSegmentsToMaintainSize() { def testCleanupSegmentsToMaintainSize() {
val setSize = TestUtils.singleMessageSet("test".getBytes()).sizeInBytes val setSize = TestUtils.singletonRecords("test".getBytes()).sizeInBytes
logManager.shutdown() logManager.shutdown()
val logProps = new Properties() val logProps = new Properties()
logProps.put(LogConfig.SegmentBytesProp, 10 * setSize: java.lang.Integer) logProps.put(LogConfig.SegmentBytesProp, 10 * setSize: java.lang.Integer)
@ -135,7 +135,7 @@ class LogManagerTest {
// add a bunch of messages that should be larger than the retentionSize // add a bunch of messages that should be larger than the retentionSize
val numMessages = 200 val numMessages = 200
for (_ <- 0 until numMessages) { for (_ <- 0 until numMessages) {
val set = TestUtils.singleMessageSet("test".getBytes()) val set = TestUtils.singletonRecords("test".getBytes())
val info = log.append(set) val info = log.append(set)
offset = info.firstOffset offset = info.firstOffset
} }
@ -147,7 +147,7 @@ class LogManagerTest {
assertEquals("Now there should be exactly 6 segments", 6, log.numberOfSegments) assertEquals("Now there should be exactly 6 segments", 6, log.numberOfSegments)
time.sleep(log.config.fileDeleteDelayMs + 1) time.sleep(log.config.fileDeleteDelayMs + 1)
assertEquals("Files should have been deleted", log.numberOfSegments * 3, log.dir.list.length) assertEquals("Files should have been deleted", log.numberOfSegments * 3, log.dir.list.length)
assertEquals("Should get empty fetch off new log.", 0, log.read(offset + 1, 1024).messageSet.sizeInBytes) assertEquals("Should get empty fetch off new log.", 0, log.read(offset + 1, 1024).records.sizeInBytes)
try { try {
log.read(0, 1024) log.read(0, 1024)
fail("Should get exception from fetching earlier.") fail("Should get exception from fetching earlier.")
@ -155,7 +155,7 @@ class LogManagerTest {
case _: OffsetOutOfRangeException => // This is good. case _: OffsetOutOfRangeException => // This is good.
} }
// log should still be appendable // log should still be appendable
log.append(TestUtils.singleMessageSet("test".getBytes())) log.append(TestUtils.singletonRecords("test".getBytes()))
} }
/** /**
@ -169,7 +169,7 @@ class LogManagerTest {
val log = logManager.createLog(TopicAndPartition(name, 0), LogConfig.fromProps(logConfig.originals, logProps)) val log = logManager.createLog(TopicAndPartition(name, 0), LogConfig.fromProps(logConfig.originals, logProps))
var offset = 0L var offset = 0L
for (_ <- 0 until 200) { for (_ <- 0 until 200) {
val set = TestUtils.singleMessageSet("test".getBytes(), key="test".getBytes()) val set = TestUtils.singletonRecords("test".getBytes(), key="test".getBytes())
val info = log.append(set) val info = log.append(set)
offset = info.lastOffset offset = info.lastOffset
} }
@ -198,7 +198,7 @@ class LogManagerTest {
val log = logManager.createLog(TopicAndPartition(name, 0), config) val log = logManager.createLog(TopicAndPartition(name, 0), config)
val lastFlush = log.lastFlushTime val lastFlush = log.lastFlushTime
for (_ <- 0 until 200) { for (_ <- 0 until 200) {
val set = TestUtils.singleMessageSet("test".getBytes()) val set = TestUtils.singletonRecords("test".getBytes())
log.append(set) log.append(set)
} }
time.sleep(logManager.InitialTaskDelayMs) time.sleep(logManager.InitialTaskDelayMs)
@ -280,7 +280,7 @@ class LogManagerTest {
val logs = topicAndPartitions.map(this.logManager.createLog(_, logConfig)) val logs = topicAndPartitions.map(this.logManager.createLog(_, logConfig))
logs.foreach(log => { logs.foreach(log => {
for (_ <- 0 until 50) for (_ <- 0 until 50)
log.append(TestUtils.singleMessageSet("test".getBytes())) log.append(TestUtils.singletonRecords("test".getBytes()))
log.flush() log.flush()
}) })

View File

@ -16,15 +16,13 @@
*/ */
package kafka.log package kafka.log
import org.junit.Assert._
import java.util.concurrent.atomic._
import kafka.common.LongRef
import org.junit.{After, Test}
import kafka.utils.TestUtils import kafka.utils.TestUtils
import kafka.message._ import org.apache.kafka.common.record.{FileRecords, MemoryRecords, Record}
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
import org.junit.Assert._
import org.junit.{After, Test}
import scala.collection.JavaConverters._
import scala.collection._ import scala.collection._
class LogSegmentTest { class LogSegmentTest {
@ -34,7 +32,7 @@ class LogSegmentTest {
/* create a segment with the given base offset */ /* create a segment with the given base offset */
def createSegment(offset: Long, indexIntervalBytes: Int = 10): LogSegment = { def createSegment(offset: Long, indexIntervalBytes: Int = 10): LogSegment = {
val msFile = TestUtils.tempFile() val msFile = TestUtils.tempFile()
val ms = new FileMessageSet(msFile) val ms = FileRecords.open(msFile)
val idxFile = TestUtils.tempFile() val idxFile = TestUtils.tempFile()
val timeIdxFile = TestUtils.tempFile() val timeIdxFile = TestUtils.tempFile()
idxFile.delete() idxFile.delete()
@ -47,12 +45,10 @@ class LogSegmentTest {
} }
/* create a ByteBufferMessageSet for the given messages starting from the given offset */ /* create a ByteBufferMessageSet for the given messages starting from the given offset */
def messages(offset: Long, messages: String*): ByteBufferMessageSet = { def records(offset: Long, records: String*): MemoryRecords = {
new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, MemoryRecords.withRecords(offset, records.map(s => Record.create(Record.MAGIC_VALUE_V1, offset * 10, s.getBytes)):_*)
offsetCounter = new LongRef(offset),
messages = messages.map(s => new Message(s.getBytes, offset * 10, Message.MagicValue_V1)):_*)
} }
@After @After
def teardown() { def teardown() {
for(seg <- segments) { for(seg <- segments) {
@ -60,7 +56,7 @@ class LogSegmentTest {
seg.log.delete() seg.log.delete()
} }
} }
/** /**
* A read on an empty log segment should return null * A read on an empty log segment should return null
*/ */
@ -70,7 +66,7 @@ class LogSegmentTest {
val read = seg.read(startOffset = 40, maxSize = 300, maxOffset = None) val read = seg.read(startOffset = 40, maxSize = 300, maxOffset = None)
assertNull("Read beyond the last offset in the segment should be null", read) assertNull("Read beyond the last offset in the segment should be null", read)
} }
/** /**
* Reading from before the first offset in the segment should return messages * Reading from before the first offset in the segment should return messages
* beginning with the first message in the segment * beginning with the first message in the segment
@ -78,12 +74,12 @@ class LogSegmentTest {
@Test @Test
def testReadBeforeFirstOffset() { def testReadBeforeFirstOffset() {
val seg = createSegment(40) val seg = createSegment(40)
val ms = messages(50, "hello", "there", "little", "bee") val ms = records(50, "hello", "there", "little", "bee")
seg.append(50, Message.NoTimestamp, -1L, ms) seg.append(50, Record.NO_TIMESTAMP, -1L, ms)
val read = seg.read(startOffset = 41, maxSize = 300, maxOffset = None).messageSet val read = seg.read(startOffset = 41, maxSize = 300, maxOffset = None).records
assertEquals(ms.toList, read.toList) assertEquals(ms.deepIterator.asScala.toList, read.deepIterator.asScala.toList)
} }
/** /**
* If we set the startOffset and maxOffset for the read to be the same value * If we set the startOffset and maxOffset for the read to be the same value
* we should get only the first message in the log * we should get only the first message in the log
@ -92,28 +88,28 @@ class LogSegmentTest {
def testMaxOffset() { def testMaxOffset() {
val baseOffset = 50 val baseOffset = 50
val seg = createSegment(baseOffset) val seg = createSegment(baseOffset)
val ms = messages(baseOffset, "hello", "there", "beautiful") val ms = records(baseOffset, "hello", "there", "beautiful")
seg.append(baseOffset, Message.NoTimestamp, -1L, ms) seg.append(baseOffset, Record.NO_TIMESTAMP, -1L, ms)
def validate(offset: Long) = def validate(offset: Long) =
assertEquals(ms.filter(_.offset == offset).toList, assertEquals(ms.deepIterator.asScala.filter(_.offset == offset).toList,
seg.read(startOffset = offset, maxSize = 1024, maxOffset = Some(offset+1)).messageSet.toList) seg.read(startOffset = offset, maxSize = 1024, maxOffset = Some(offset+1)).records.deepIterator.asScala.toList)
validate(50) validate(50)
validate(51) validate(51)
validate(52) validate(52)
} }
/** /**
* If we read from an offset beyond the last offset in the segment we should get null * If we read from an offset beyond the last offset in the segment we should get null
*/ */
@Test @Test
def testReadAfterLast() { def testReadAfterLast() {
val seg = createSegment(40) val seg = createSegment(40)
val ms = messages(50, "hello", "there") val ms = records(50, "hello", "there")
seg.append(50, Message.NoTimestamp, -1L, ms) seg.append(50, Record.NO_TIMESTAMP, -1L, ms)
val read = seg.read(startOffset = 52, maxSize = 200, maxOffset = None) val read = seg.read(startOffset = 52, maxSize = 200, maxOffset = None)
assertNull("Read beyond the last offset in the segment should give null", read) assertNull("Read beyond the last offset in the segment should give null", read)
} }
/** /**
* If we read from an offset which doesn't exist we should get a message set beginning * If we read from an offset which doesn't exist we should get a message set beginning
* with the least offset greater than the given startOffset. * with the least offset greater than the given startOffset.
@ -121,14 +117,14 @@ class LogSegmentTest {
@Test @Test
def testReadFromGap() { def testReadFromGap() {
val seg = createSegment(40) val seg = createSegment(40)
val ms = messages(50, "hello", "there") val ms = records(50, "hello", "there")
seg.append(50, Message.NoTimestamp, -1L, ms) seg.append(50, Record.NO_TIMESTAMP, -1L, ms)
val ms2 = messages(60, "alpha", "beta") val ms2 = records(60, "alpha", "beta")
seg.append(60, Message.NoTimestamp, -1L, ms2) seg.append(60, Record.NO_TIMESTAMP, -1L, ms2)
val read = seg.read(startOffset = 55, maxSize = 200, maxOffset = None) val read = seg.read(startOffset = 55, maxSize = 200, maxOffset = None)
assertEquals(ms2.toList, read.messageSet.toList) assertEquals(ms2.deepIterator.asScala.toList, read.records.deepIterator.asScala.toList)
} }
/** /**
* In a loop append two messages then truncate off the second of those messages and check that we can read * In a loop append two messages then truncate off the second of those messages and check that we can read
* the first but not the second message. * the first but not the second message.
@ -138,18 +134,18 @@ class LogSegmentTest {
val seg = createSegment(40) val seg = createSegment(40)
var offset = 40 var offset = 40
for (_ <- 0 until 30) { for (_ <- 0 until 30) {
val ms1 = messages(offset, "hello") val ms1 = records(offset, "hello")
seg.append(offset, Message.NoTimestamp, -1L, ms1) seg.append(offset, Record.NO_TIMESTAMP, -1L, ms1)
val ms2 = messages(offset + 1, "hello") val ms2 = records(offset + 1, "hello")
seg.append(offset + 1, Message.NoTimestamp, -1L, ms2) seg.append(offset + 1, Record.NO_TIMESTAMP, -1L, ms2)
// check that we can read back both messages // check that we can read back both messages
val read = seg.read(offset, None, 10000) val read = seg.read(offset, None, 10000)
assertEquals(List(ms1.head, ms2.head), read.messageSet.toList) assertEquals(List(ms1.deepIterator.next(), ms2.deepIterator.next()), read.records.deepIterator.asScala.toList)
// now truncate off the last message // now truncate off the last message
seg.truncateTo(offset + 1) seg.truncateTo(offset + 1)
val read2 = seg.read(offset, None, 10000) val read2 = seg.read(offset, None, 10000)
assertEquals(1, read2.messageSet.size) assertEquals(1, read2.records.deepIterator.asScala.size)
assertEquals(ms1.head, read2.messageSet.head) assertEquals(ms1.deepIterator.next(), read2.records.deepIterator.next())
offset += 1 offset += 1
} }
} }
@ -157,10 +153,10 @@ class LogSegmentTest {
@Test @Test
def testReloadLargestTimestampAfterTruncation() { def testReloadLargestTimestampAfterTruncation() {
val numMessages = 30 val numMessages = 30
val seg = createSegment(40, 2 * messages(0, "hello").sizeInBytes - 1) val seg = createSegment(40, 2 * records(0, "hello").sizeInBytes - 1)
var offset = 40 var offset = 40
for (_ <- 0 until numMessages) { for (_ <- 0 until numMessages) {
seg.append(offset, offset, offset, messages(offset, "hello")) seg.append(offset, offset, offset, records(offset, "hello"))
offset += 1 offset += 1
} }
val expectedNumEntries = numMessages / 2 - 1 val expectedNumEntries = numMessages / 2 - 1
@ -179,10 +175,10 @@ class LogSegmentTest {
def testTruncateFull() { def testTruncateFull() {
// test the case where we fully truncate the log // test the case where we fully truncate the log
val seg = createSegment(40) val seg = createSegment(40)
seg.append(40, Message.NoTimestamp, -1L, messages(40, "hello", "there")) seg.append(40, Record.NO_TIMESTAMP, -1L, records(40, "hello", "there"))
seg.truncateTo(0) seg.truncateTo(0)
assertNull("Segment should be empty.", seg.read(0, None, 1024)) assertNull("Segment should be empty.", seg.read(0, None, 1024))
seg.append(40, Message.NoTimestamp, -1L, messages(40, "hello", "there")) seg.append(40, Record.NO_TIMESTAMP, -1L, records(40, "hello", "there"))
} }
/** /**
@ -190,11 +186,11 @@ class LogSegmentTest {
*/ */
@Test @Test
def testFindOffsetByTimestamp() { def testFindOffsetByTimestamp() {
val messageSize = messages(0, s"msg00").sizeInBytes val messageSize = records(0, s"msg00").sizeInBytes
val seg = createSegment(40, messageSize * 2 - 1) val seg = createSegment(40, messageSize * 2 - 1)
// Produce some messages // Produce some messages
for (i <- 40 until 50) for (i <- 40 until 50)
seg.append(i, i * 10, i, messages(i, s"msg$i")) seg.append(i, i * 10, i, records(i, s"msg$i"))
assertEquals(490, seg.largestTimestamp) assertEquals(490, seg.largestTimestamp)
// Search for an indexed timestamp // Search for an indexed timestamp
@ -218,10 +214,10 @@ class LogSegmentTest {
def testNextOffsetCalculation() { def testNextOffsetCalculation() {
val seg = createSegment(40) val seg = createSegment(40)
assertEquals(40, seg.nextOffset) assertEquals(40, seg.nextOffset)
seg.append(50, Message.NoTimestamp, -1L, messages(50, "hello", "there", "you")) seg.append(50, Record.NO_TIMESTAMP, -1L, records(50, "hello", "there", "you"))
assertEquals(53, seg.nextOffset()) assertEquals(53, seg.nextOffset())
} }
/** /**
* Test that we can change the file suffixes for the log and index files * Test that we can change the file suffixes for the log and index files
*/ */
@ -236,7 +232,7 @@ class LogSegmentTest {
assertTrue(seg.log.file.exists) assertTrue(seg.log.file.exists)
assertTrue(seg.index.file.exists) assertTrue(seg.index.file.exists)
} }
/** /**
* Create a segment with some data and an index. Then corrupt the index, * Create a segment with some data and an index. Then corrupt the index,
* and recover the segment, the entries should all be readable. * and recover the segment, the entries should all be readable.
@ -245,12 +241,12 @@ class LogSegmentTest {
def testRecoveryFixesCorruptIndex() { def testRecoveryFixesCorruptIndex() {
val seg = createSegment(0) val seg = createSegment(0)
for(i <- 0 until 100) for(i <- 0 until 100)
seg.append(i, Message.NoTimestamp, -1L, messages(i, i.toString)) seg.append(i, Record.NO_TIMESTAMP, -1L, records(i, i.toString))
val indexFile = seg.index.file val indexFile = seg.index.file
TestUtils.writeNonsenseToFile(indexFile, 5, indexFile.length.toInt) TestUtils.writeNonsenseToFile(indexFile, 5, indexFile.length.toInt)
seg.recover(64*1024) seg.recover(64*1024)
for(i <- 0 until 100) for(i <- 0 until 100)
assertEquals(i, seg.read(i, Some(i + 1), 1024).messageSet.head.offset) assertEquals(i, seg.read(i, Some(i + 1), 1024).records.deepIterator.next().offset)
} }
/** /**
@ -261,7 +257,7 @@ class LogSegmentTest {
def testRecoveryFixesCorruptTimeIndex() { def testRecoveryFixesCorruptTimeIndex() {
val seg = createSegment(0) val seg = createSegment(0)
for(i <- 0 until 100) for(i <- 0 until 100)
seg.append(i, i * 10, i, messages(i, i.toString)) seg.append(i, i * 10, i, records(i, i.toString))
val timeIndexFile = seg.timeIndex.file val timeIndexFile = seg.timeIndex.file
TestUtils.writeNonsenseToFile(timeIndexFile, 5, timeIndexFile.length.toInt) TestUtils.writeNonsenseToFile(timeIndexFile, 5, timeIndexFile.length.toInt)
seg.recover(64*1024) seg.recover(64*1024)
@ -271,7 +267,7 @@ class LogSegmentTest {
assertEquals(i + 1, seg.findOffsetByTimestamp(i * 10 + 1).get.offset) assertEquals(i + 1, seg.findOffsetByTimestamp(i * 10 + 1).get.offset)
} }
} }
/** /**
* Randomly corrupt a log a number of times and attempt recovery. * Randomly corrupt a log a number of times and attempt recovery.
*/ */
@ -281,13 +277,15 @@ class LogSegmentTest {
for (_ <- 0 until 10) { for (_ <- 0 until 10) {
val seg = createSegment(0) val seg = createSegment(0)
for(i <- 0 until messagesAppended) for(i <- 0 until messagesAppended)
seg.append(i, Message.NoTimestamp, -1L, messages(i, i.toString)) seg.append(i, Record.NO_TIMESTAMP, -1L, records(i, i.toString))
val offsetToBeginCorruption = TestUtils.random.nextInt(messagesAppended) val offsetToBeginCorruption = TestUtils.random.nextInt(messagesAppended)
// start corrupting somewhere in the middle of the chosen record all the way to the end // start corrupting somewhere in the middle of the chosen record all the way to the end
val position = seg.log.searchForOffsetWithSize(offsetToBeginCorruption, 0)._1.position + TestUtils.random.nextInt(15)
TestUtils.writeNonsenseToFile(seg.log.file, position, seg.log.file.length.toInt - position) val recordPosition = seg.log.searchForOffsetWithSize(offsetToBeginCorruption, 0)
val position = recordPosition.position + TestUtils.random.nextInt(15)
TestUtils.writeNonsenseToFile(seg.log.file, position, (seg.log.file.length - position).toInt)
seg.recover(64*1024) seg.recover(64*1024)
assertEquals("Should have truncated off bad messages.", (0 until offsetToBeginCorruption).toList, seg.log.map(_.offset).toList) assertEquals("Should have truncated off bad messages.", (0 until offsetToBeginCorruption).toList, seg.log.shallowIterator.asScala.map(_.offset).toList)
seg.delete() seg.delete()
} }
} }
@ -304,12 +302,12 @@ class LogSegmentTest {
@Test @Test
def testCreateWithInitFileSizeAppendMessage() { def testCreateWithInitFileSizeAppendMessage() {
val seg = createSegment(40, false, 512*1024*1024, true) val seg = createSegment(40, false, 512*1024*1024, true)
val ms = messages(50, "hello", "there") val ms = records(50, "hello", "there")
seg.append(50, Message.NoTimestamp, -1L, ms) seg.append(50, Record.NO_TIMESTAMP, -1L, ms)
val ms2 = messages(60, "alpha", "beta") val ms2 = records(60, "alpha", "beta")
seg.append(60, Message.NoTimestamp, -1L, ms2) seg.append(60, Record.NO_TIMESTAMP, -1L, ms2)
val read = seg.read(startOffset = 55, maxSize = 200, maxOffset = None) val read = seg.read(startOffset = 55, maxSize = 200, maxOffset = None)
assertEquals(ms2.toList, read.messageSet.toList) assertEquals(ms2.deepIterator.asScala.toList, read.records.deepIterator.asScala.toList)
} }
/* create a segment with pre allocate and clearly shut down*/ /* create a segment with pre allocate and clearly shut down*/
@ -318,12 +316,12 @@ class LogSegmentTest {
val tempDir = TestUtils.tempDir() val tempDir = TestUtils.tempDir()
val seg = new LogSegment(tempDir, 40, 10, 1000, 0, Time.SYSTEM, false, 512*1024*1024, true) val seg = new LogSegment(tempDir, 40, 10, 1000, 0, Time.SYSTEM, false, 512*1024*1024, true)
val ms = messages(50, "hello", "there") val ms = records(50, "hello", "there")
seg.append(50, Message.NoTimestamp, -1L, ms) seg.append(50, Record.NO_TIMESTAMP, -1L, ms)
val ms2 = messages(60, "alpha", "beta") val ms2 = records(60, "alpha", "beta")
seg.append(60, Message.NoTimestamp, -1L, ms2) seg.append(60, Record.NO_TIMESTAMP, -1L, ms2)
val read = seg.read(startOffset = 55, maxSize = 200, maxOffset = None) val read = seg.read(startOffset = 55, maxSize = 200, maxOffset = None)
assertEquals(ms2.toList, read.messageSet.toList) assertEquals(ms2.deepIterator.asScala.toList, read.records.deepIterator.asScala.toList)
val oldSize = seg.log.sizeInBytes() val oldSize = seg.log.sizeInBytes()
val oldPosition = seg.log.channel.position val oldPosition = seg.log.channel.position
val oldFileSize = seg.log.file.length val oldFileSize = seg.log.file.length
@ -336,7 +334,7 @@ class LogSegmentTest {
segments += segReopen segments += segReopen
val readAgain = segReopen.read(startOffset = 55, maxSize = 200, maxOffset = None) val readAgain = segReopen.read(startOffset = 55, maxSize = 200, maxOffset = None)
assertEquals(ms2.toList, readAgain.messageSet.toList) assertEquals(ms2.deepIterator.asScala.toList, readAgain.records.deepIterator.asScala.toList)
val size = segReopen.log.sizeInBytes() val size = segReopen.log.sizeInBytes()
val position = segReopen.log.channel.position val position = segReopen.log.channel.position
val fileSize = segReopen.log.file.length val fileSize = segReopen.log.file.length

View File

@ -22,15 +22,16 @@ import java.util.Properties
import org.apache.kafka.common.errors.{CorruptRecordException, OffsetOutOfRangeException, RecordBatchTooLargeException, RecordTooLargeException} import org.apache.kafka.common.errors.{CorruptRecordException, OffsetOutOfRangeException, RecordBatchTooLargeException, RecordTooLargeException}
import kafka.api.ApiVersion import kafka.api.ApiVersion
import kafka.common.LongRef
import org.junit.Assert._ import org.junit.Assert._
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite
import org.junit.{After, Before, Test} import org.junit.{After, Before, Test}
import kafka.message._
import kafka.utils._ import kafka.utils._
import kafka.server.KafkaConfig import kafka.server.KafkaConfig
import org.apache.kafka.common.record._
import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.utils.Utils
import scala.collection.JavaConverters._
class LogTest extends JUnitSuite { class LogTest extends JUnitSuite {
val tmpDir = TestUtils.tempDir() val tmpDir = TestUtils.tempDir()
@ -63,7 +64,7 @@ class LogTest extends JUnitSuite {
*/ */
@Test @Test
def testTimeBasedLogRoll() { def testTimeBasedLogRoll() {
val set = TestUtils.singleMessageSet("test".getBytes) val set = TestUtils.singletonRecords("test".getBytes)
val logProps = new Properties() val logProps = new Properties()
logProps.put(LogConfig.SegmentMsProp, (1 * 60 * 60L): java.lang.Long) logProps.put(LogConfig.SegmentMsProp, (1 * 60 * 60L): java.lang.Long)
@ -91,7 +92,7 @@ class LogTest extends JUnitSuite {
// Append a message with timestamp to a segment whose first messgae do not have a timestamp. // Append a message with timestamp to a segment whose first messgae do not have a timestamp.
val setWithTimestamp = val setWithTimestamp =
TestUtils.singleMessageSet(payload = "test".getBytes, timestamp = time.milliseconds + log.config.segmentMs + 1) TestUtils.singletonRecords(value = "test".getBytes, timestamp = time.milliseconds + log.config.segmentMs + 1)
log.append(setWithTimestamp) log.append(setWithTimestamp)
assertEquals("Segment should not have been rolled out because the log rolling should be based on wall clock.", 4, log.numberOfSegments) assertEquals("Segment should not have been rolled out because the log rolling should be based on wall clock.", 4, log.numberOfSegments)
@ -105,14 +106,14 @@ class LogTest extends JUnitSuite {
log.append(setWithTimestamp) log.append(setWithTimestamp)
assertEquals("Log should not roll because the roll should depend on timestamp of the first message.", 5, log.numberOfSegments) assertEquals("Log should not roll because the roll should depend on timestamp of the first message.", 5, log.numberOfSegments)
val setWithExpiredTimestamp = TestUtils.singleMessageSet(payload = "test".getBytes, timestamp = time.milliseconds) val setWithExpiredTimestamp = TestUtils.singletonRecords(value = "test".getBytes, timestamp = time.milliseconds)
log.append(setWithExpiredTimestamp) log.append(setWithExpiredTimestamp)
assertEquals("Log should roll because the timestamp in the message should make the log segment expire.", 6, log.numberOfSegments) assertEquals("Log should roll because the timestamp in the message should make the log segment expire.", 6, log.numberOfSegments)
val numSegments = log.numberOfSegments val numSegments = log.numberOfSegments
time.sleep(log.config.segmentMs + 1) time.sleep(log.config.segmentMs + 1)
log.append(new ByteBufferMessageSet()) log.append(MemoryRecords.withLogEntries())
assertEquals("Appending an empty message set should not roll log even if succient time has passed.", numSegments, log.numberOfSegments) assertEquals("Appending an empty message set should not roll log even if sufficient time has passed.", numSegments, log.numberOfSegments)
} }
/** /**
@ -121,7 +122,7 @@ class LogTest extends JUnitSuite {
*/ */
@Test @Test
def testTimeBasedLogRollJitter() { def testTimeBasedLogRollJitter() {
val set = TestUtils.singleMessageSet("test".getBytes) val set = TestUtils.singletonRecords("test".getBytes)
val maxJitter = 20 * 60L val maxJitter = 20 * 60L
val logProps = new Properties() val logProps = new Properties()
@ -149,7 +150,7 @@ class LogTest extends JUnitSuite {
*/ */
@Test @Test
def testSizeBasedLogRoll() { def testSizeBasedLogRoll() {
val set = TestUtils.singleMessageSet("test".getBytes) val set = TestUtils.singletonRecords("test".getBytes)
val setSize = set.sizeInBytes val setSize = set.sizeInBytes
val msgPerSeg = 10 val msgPerSeg = 10
val segmentSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages val segmentSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages
@ -176,7 +177,7 @@ class LogTest extends JUnitSuite {
def testLoadEmptyLog() { def testLoadEmptyLog() {
createEmptyLogs(logDir, 0) createEmptyLogs(logDir, 0)
val log = new Log(logDir, logConfig, recoveryPoint = 0L, time.scheduler, time = time) val log = new Log(logDir, logConfig, recoveryPoint = 0L, time.scheduler, time = time)
log.append(TestUtils.singleMessageSet("test".getBytes)) log.append(TestUtils.singletonRecords("test".getBytes))
} }
/** /**
@ -189,16 +190,17 @@ class LogTest extends JUnitSuite {
// We use need to use magic value 1 here because the test is message size sensitive. // We use need to use magic value 1 here because the test is message size sensitive.
logProps.put(LogConfig.MessageFormatVersionProp, ApiVersion.latestVersion.toString) logProps.put(LogConfig.MessageFormatVersionProp, ApiVersion.latestVersion.toString)
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
val messages = (0 until 100 by 2).map(id => new Message(id.toString.getBytes)).toArray val records = (0 until 100 by 2).map(id => Record.create(id.toString.getBytes)).toArray
for(i <- 0 until messages.length) for(i <- records.indices)
log.append(new ByteBufferMessageSet(NoCompressionCodec, messages = messages(i))) log.append(MemoryRecords.withRecords(records(i)))
for(i <- 0 until messages.length) {
val read = log.read(i, 100, Some(i+1)).messageSet.head for(i <- records.indices) {
val read = log.read(i, 100, Some(i+1)).records.shallowIterator.next()
assertEquals("Offset read should match order appended.", i, read.offset) assertEquals("Offset read should match order appended.", i, read.offset)
assertEquals("Message should match appended.", messages(i), read.message) assertEquals("Message should match appended.", records(i), read.record)
} }
assertEquals("Reading beyond the last message returns nothing.", 0, log.read(messages.length, 100, None).messageSet.size) assertEquals("Reading beyond the last message returns nothing.", 0, log.read(records.length, 100, None).records.shallowIterator.asScala.size)
} }
/** /**
@ -211,16 +213,16 @@ class LogTest extends JUnitSuite {
logProps.put(LogConfig.SegmentBytesProp, 71: java.lang.Integer) logProps.put(LogConfig.SegmentBytesProp, 71: java.lang.Integer)
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray
val messages = messageIds.map(id => new Message(id.toString.getBytes)) val records = messageIds.map(id => Record.create(id.toString.getBytes))
// now test the case that we give the offsets and use non-sequential offsets // now test the case that we give the offsets and use non-sequential offsets
for(i <- 0 until messages.length) for(i <- records.indices)
log.append(new ByteBufferMessageSet(NoCompressionCodec, new LongRef(messageIds(i)), messages = messages(i)), assignOffsets = false) log.append(MemoryRecords.withLogEntries(LogEntry.create(messageIds(i), records(i))), assignOffsets = false)
for(i <- 50 until messageIds.max) { for(i <- 50 until messageIds.max) {
val idx = messageIds.indexWhere(_ >= i) val idx = messageIds.indexWhere(_ >= i)
val read = log.read(i, 100, None).messageSet.head val read = log.read(i, 100, None).records.shallowIterator.next()
assertEquals("Offset read should match message id.", messageIds(idx), read.offset) assertEquals("Offset read should match message id.", messageIds(idx), read.offset)
assertEquals("Message should match appended.", messages(idx), read.message) assertEquals("Message should match appended.", records(idx), read.record)
} }
} }
@ -238,12 +240,12 @@ class LogTest extends JUnitSuite {
// keep appending until we have two segments with only a single message in the second segment // keep appending until we have two segments with only a single message in the second segment
while(log.numberOfSegments == 1) while(log.numberOfSegments == 1)
log.append(new ByteBufferMessageSet(NoCompressionCodec, messages = new Message("42".getBytes))) log.append(MemoryRecords.withRecords(Record.create("42".getBytes)))
// now manually truncate off all but one message from the first segment to create a gap in the messages // now manually truncate off all but one message from the first segment to create a gap in the messages
log.logSegments.head.truncateTo(1) log.logSegments.head.truncateTo(1)
assertEquals("A read should now return the last message in the log", log.logEndOffset - 1, log.read(1, 200, None).messageSet.head.offset) assertEquals("A read should now return the last message in the log", log.logEndOffset - 1, log.read(1, 200, None).records.shallowIterator.next().offset)
} }
@Test @Test
@ -252,12 +254,11 @@ class LogTest extends JUnitSuite {
logProps.put(LogConfig.SegmentBytesProp, 71: java.lang.Integer) logProps.put(LogConfig.SegmentBytesProp, 71: java.lang.Integer)
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray
val messages = messageIds.map(id => new Message(id.toString.getBytes)) val records = messageIds.map(id => Record.create(id.toString.getBytes))
// now test the case that we give the offsets and use non-sequential offsets // now test the case that we give the offsets and use non-sequential offsets
for (i <- 0 until messages.length) for (i <- records.indices)
log.append(new ByteBufferMessageSet(NoCompressionCodec, new LongRef(messageIds(i)), messages = messages(i)), log.append(MemoryRecords.withLogEntries(LogEntry.create(messageIds(i), records(i))), assignOffsets = false)
assignOffsets = false)
for (i <- 50 until messageIds.max) { for (i <- 50 until messageIds.max) {
val idx = messageIds.indexWhere(_ >= i) val idx = messageIds.indexWhere(_ >= i)
@ -265,13 +266,13 @@ class LogTest extends JUnitSuite {
log.read(i, 1, minOneMessage = true), log.read(i, 1, minOneMessage = true),
log.read(i, 100, minOneMessage = true), log.read(i, 100, minOneMessage = true),
log.read(i, 100, Some(10000), minOneMessage = true) log.read(i, 100, Some(10000), minOneMessage = true)
).map(_.messageSet.head) ).map(_.records.shallowIterator.next())
reads.foreach { read => reads.foreach { read =>
assertEquals("Offset read should match message id.", messageIds(idx), read.offset) assertEquals("Offset read should match message id.", messageIds(idx), read.offset)
assertEquals("Message should match appended.", messages(idx), read.message) assertEquals("Message should match appended.", records(idx), read.record)
} }
assertEquals(Seq.empty, log.read(i, 1, Some(1), minOneMessage = true).messageSet.toIndexedSeq) assertEquals(Seq.empty, log.read(i, 1, Some(1), minOneMessage = true).records.shallowIterator.asScala.toIndexedSeq)
} }
} }
@ -282,15 +283,14 @@ class LogTest extends JUnitSuite {
logProps.put(LogConfig.SegmentBytesProp, 71: java.lang.Integer) logProps.put(LogConfig.SegmentBytesProp, 71: java.lang.Integer)
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray
val messages = messageIds.map(id => new Message(id.toString.getBytes)) val records = messageIds.map(id => Record.create(id.toString.getBytes))
// now test the case that we give the offsets and use non-sequential offsets // now test the case that we give the offsets and use non-sequential offsets
for (i <- 0 until messages.length) for (i <- records.indices)
log.append(new ByteBufferMessageSet(NoCompressionCodec, new LongRef(messageIds(i)), messages = messages(i)), log.append(MemoryRecords.withLogEntries(LogEntry.create(messageIds(i), records(i))), assignOffsets = false)
assignOffsets = false)
for (i <- 50 until messageIds.max) { for (i <- 50 until messageIds.max) {
assertEquals(MessageSet.Empty, log.read(i, 0).messageSet) assertEquals(MemoryRecords.EMPTY, log.read(i, 0).records)
// we return an incomplete message instead of an empty one for the case below // we return an incomplete message instead of an empty one for the case below
// we use this mechanism to tell consumers of the fetch request version 2 and below that the message size is // we use this mechanism to tell consumers of the fetch request version 2 and below that the message size is
@ -298,9 +298,9 @@ class LogTest extends JUnitSuite {
// in fetch request version 3, we no longer need this as we return oversized messages from the first non-empty // in fetch request version 3, we no longer need this as we return oversized messages from the first non-empty
// partition // partition
val fetchInfo = log.read(i, 1) val fetchInfo = log.read(i, 1)
assertTrue(fetchInfo.firstMessageSetIncomplete) assertTrue(fetchInfo.firstEntryIncomplete)
assertTrue(fetchInfo.messageSet.isInstanceOf[FileMessageSet]) assertTrue(fetchInfo.records.isInstanceOf[FileRecords])
assertEquals(1, fetchInfo.messageSet.sizeInBytes) assertEquals(1, fetchInfo.records.sizeInBytes)
} }
} }
@ -318,9 +318,9 @@ class LogTest extends JUnitSuite {
// set up replica log starting with offset 1024 and with one message (at offset 1024) // set up replica log starting with offset 1024 and with one message (at offset 1024)
logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer)
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
log.append(new ByteBufferMessageSet(NoCompressionCodec, messages = new Message("42".getBytes))) log.append(MemoryRecords.withRecords(Record.create("42".getBytes)))
assertEquals("Reading at the log end offset should produce 0 byte read.", 0, log.read(1025, 1000).messageSet.sizeInBytes) assertEquals("Reading at the log end offset should produce 0 byte read.", 0, log.read(1025, 1000).records.sizeInBytes)
try { try {
log.read(0, 1000) log.read(0, 1000)
@ -336,7 +336,7 @@ class LogTest extends JUnitSuite {
case _: OffsetOutOfRangeException => // This is good. case _: OffsetOutOfRangeException => // This is good.
} }
assertEquals("Reading from below the specified maxOffset should produce 0 byte read.", 0, log.read(1025, 1000, Some(1024)).messageSet.sizeInBytes) assertEquals("Reading from below the specified maxOffset should produce 0 byte read.", 0, log.read(1025, 1000, Some(1024)).records.sizeInBytes)
} }
/** /**
@ -350,21 +350,22 @@ class LogTest extends JUnitSuite {
logProps.put(LogConfig.SegmentBytesProp, 100: java.lang.Integer) logProps.put(LogConfig.SegmentBytesProp, 100: java.lang.Integer)
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
val numMessages = 100 val numMessages = 100
val messageSets = (0 until numMessages).map(i => TestUtils.singleMessageSet(i.toString.getBytes)) val messageSets = (0 until numMessages).map(i => TestUtils.singletonRecords(i.toString.getBytes))
messageSets.foreach(log.append(_)) messageSets.foreach(log.append(_))
log.flush log.flush
/* do successive reads to ensure all our messages are there */ /* do successive reads to ensure all our messages are there */
var offset = 0L var offset = 0L
for(i <- 0 until numMessages) { for(i <- 0 until numMessages) {
val messages = log.read(offset, 1024*1024).messageSet val messages = log.read(offset, 1024*1024).records.shallowIterator
assertEquals("Offsets not equal", offset, messages.head.offset) val head = messages.next()
assertEquals("Messages not equal at offset " + offset, messageSets(i).head.message, assertEquals("Offsets not equal", offset, head.offset)
messages.head.message.toFormatVersion(messageSets(i).head.message.magic)) assertEquals("Messages not equal at offset " + offset, messageSets(i).shallowIterator.next().record,
offset = messages.head.offset + 1 head.record.convert(messageSets(i).shallowIterator.next().record.magic))
offset = head.offset + 1
} }
val lastRead = log.read(startOffset = numMessages, maxLength = 1024*1024, maxOffset = Some(numMessages + 1)).messageSet val lastRead = log.read(startOffset = numMessages, maxLength = 1024*1024, maxOffset = Some(numMessages + 1)).records
assertEquals("Should be no more messages", 0, lastRead.size) assertEquals("Should be no more messages", 0, lastRead.shallowIterator.asScala.size)
// check that rolling the log forced a flushed the log--the flush is asyn so retry in case of failure // check that rolling the log forced a flushed the log--the flush is asyn so retry in case of failure
TestUtils.retry(1000L){ TestUtils.retry(1000L){
@ -383,10 +384,10 @@ class LogTest extends JUnitSuite {
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
/* append 2 compressed message sets, each with two messages giving offsets 0, 1, 2, 3 */ /* append 2 compressed message sets, each with two messages giving offsets 0, 1, 2, 3 */
log.append(new ByteBufferMessageSet(DefaultCompressionCodec, new Message("hello".getBytes), new Message("there".getBytes))) log.append(MemoryRecords.withRecords(CompressionType.GZIP, Record.create("hello".getBytes), Record.create("there".getBytes)))
log.append(new ByteBufferMessageSet(DefaultCompressionCodec, new Message("alpha".getBytes), new Message("beta".getBytes))) log.append(MemoryRecords.withRecords(CompressionType.GZIP, Record.create("alpha".getBytes), Record.create("beta".getBytes)))
def read(offset: Int) = ByteBufferMessageSet.deepIterator(log.read(offset, 4096).messageSet.head) def read(offset: Int) = log.read(offset, 4096).records.deepIterator
/* we should always get the first message in the compressed set when reading any offset in the set */ /* we should always get the first message in the compressed set when reading any offset in the set */
assertEquals("Read at offset 0 should produce 0", 0, read(0).next().offset) assertEquals("Read at offset 0 should produce 0", 0, read(0).next().offset)
@ -408,7 +409,7 @@ class LogTest extends JUnitSuite {
logProps.put(LogConfig.RetentionMsProp, 0: java.lang.Integer) logProps.put(LogConfig.RetentionMsProp, 0: java.lang.Integer)
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
for(i <- 0 until messagesToAppend) for(i <- 0 until messagesToAppend)
log.append(TestUtils.singleMessageSet(payload = i.toString.getBytes, timestamp = time.milliseconds - 10)) log.append(TestUtils.singletonRecords(value = i.toString.getBytes, timestamp = time.milliseconds - 10))
val currOffset = log.logEndOffset val currOffset = log.logEndOffset
assertEquals(currOffset, messagesToAppend) assertEquals(currOffset, messagesToAppend)
@ -422,7 +423,7 @@ class LogTest extends JUnitSuite {
assertEquals("Still no change in the logEndOffset", currOffset, log.logEndOffset) assertEquals("Still no change in the logEndOffset", currOffset, log.logEndOffset)
assertEquals("Should still be able to append and should get the logEndOffset assigned to the new append", assertEquals("Should still be able to append and should get the logEndOffset assigned to the new append",
currOffset, currOffset,
log.append(TestUtils.singleMessageSet("hello".getBytes)).firstOffset) log.append(TestUtils.singletonRecords("hello".getBytes)).firstOffset)
// cleanup the log // cleanup the log
log.delete() log.delete()
@ -435,7 +436,7 @@ class LogTest extends JUnitSuite {
*/ */
@Test @Test
def testMessageSetSizeCheck() { def testMessageSetSizeCheck() {
val messageSet = new ByteBufferMessageSet(NoCompressionCodec, new Message ("You".getBytes), new Message("bethe".getBytes)) val messageSet = MemoryRecords.withRecords(Record.create("You".getBytes), Record.create("bethe".getBytes))
// append messages to log // append messages to log
val configSegmentSize = messageSet.sizeInBytes - 1 val configSegmentSize = messageSet.sizeInBytes - 1
val logProps = new Properties() val logProps = new Properties()
@ -454,17 +455,17 @@ class LogTest extends JUnitSuite {
@Test @Test
def testCompactedTopicConstraints() { def testCompactedTopicConstraints() {
val keyedMessage = new Message(bytes = "this message has a key".getBytes, key = "and here it is".getBytes, Message.NoTimestamp, Message.CurrentMagicValue) val keyedMessage = Record.create(Record.CURRENT_MAGIC_VALUE, Record.NO_TIMESTAMP, "and here it is".getBytes, "this message has a key".getBytes)
val anotherKeyedMessage = new Message(bytes = "this message also has a key".getBytes, key ="another key".getBytes, Message.NoTimestamp, Message.CurrentMagicValue) val anotherKeyedMessage = Record.create(Record.CURRENT_MAGIC_VALUE, Record.NO_TIMESTAMP, "another key".getBytes, "this message also has a key".getBytes)
val unkeyedMessage = new Message(bytes = "this message does not have a key".getBytes) val unkeyedMessage = Record.create("this message does not have a key".getBytes)
val messageSetWithUnkeyedMessage = new ByteBufferMessageSet(NoCompressionCodec, unkeyedMessage, keyedMessage) val messageSetWithUnkeyedMessage = MemoryRecords.withRecords(CompressionType.NONE, unkeyedMessage, keyedMessage)
val messageSetWithOneUnkeyedMessage = new ByteBufferMessageSet(NoCompressionCodec, unkeyedMessage) val messageSetWithOneUnkeyedMessage = MemoryRecords.withRecords(CompressionType.NONE, unkeyedMessage)
val messageSetWithCompressedKeyedMessage = new ByteBufferMessageSet(GZIPCompressionCodec, keyedMessage) val messageSetWithCompressedKeyedMessage = MemoryRecords.withRecords(CompressionType.GZIP, keyedMessage)
val messageSetWithCompressedUnkeyedMessage = new ByteBufferMessageSet(GZIPCompressionCodec, keyedMessage, unkeyedMessage) val messageSetWithCompressedUnkeyedMessage = MemoryRecords.withRecords(CompressionType.GZIP, keyedMessage, unkeyedMessage)
val messageSetWithKeyedMessage = new ByteBufferMessageSet(NoCompressionCodec, keyedMessage) val messageSetWithKeyedMessage = MemoryRecords.withRecords(CompressionType.NONE, keyedMessage)
val messageSetWithKeyedMessages = new ByteBufferMessageSet(NoCompressionCodec, keyedMessage, anotherKeyedMessage) val messageSetWithKeyedMessages = MemoryRecords.withRecords(CompressionType.NONE, keyedMessage, anotherKeyedMessage)
val logProps = new Properties() val logProps = new Properties()
logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Compact) logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Compact)
@ -502,8 +503,8 @@ class LogTest extends JUnitSuite {
*/ */
@Test @Test
def testMessageSizeCheck() { def testMessageSizeCheck() {
val first = new ByteBufferMessageSet(NoCompressionCodec, new Message ("You".getBytes), new Message("bethe".getBytes)) val first = MemoryRecords.withRecords(CompressionType.NONE, Record.create("You".getBytes), Record.create("bethe".getBytes))
val second = new ByteBufferMessageSet(NoCompressionCodec, new Message("change (I need more bytes)".getBytes)) val second = MemoryRecords.withRecords(CompressionType.NONE, Record.create("change (I need more bytes)".getBytes))
// append messages to log // append messages to log
val maxMessageSize = second.sizeInBytes - 1 val maxMessageSize = second.sizeInBytes - 1
@ -537,7 +538,7 @@ class LogTest extends JUnitSuite {
val config = LogConfig(logProps) val config = LogConfig(logProps)
var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time) var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time)
for(i <- 0 until numMessages) for(i <- 0 until numMessages)
log.append(TestUtils.singleMessageSet(payload = TestUtils.randomBytes(messageSize), log.append(TestUtils.singletonRecords(value = TestUtils.randomBytes(messageSize),
timestamp = time.milliseconds + i * 10)) timestamp = time.milliseconds + i * 10))
assertEquals("After appending %d messages to an empty log, the log end offset should be %d".format(numMessages, numMessages), numMessages, log.logEndOffset) assertEquals("After appending %d messages to an empty log, the log end offset should be %d".format(numMessages, numMessages), numMessages, log.logEndOffset)
val lastIndexOffset = log.activeSegment.index.lastOffset val lastIndexOffset = log.activeSegment.index.lastOffset
@ -585,7 +586,7 @@ class LogTest extends JUnitSuite {
val log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time) val log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time)
val messages = (0 until numMessages).map { i => val messages = (0 until numMessages).map { i =>
new ByteBufferMessageSet(NoCompressionCodec, new LongRef(100 + i), new Message(i.toString.getBytes(), time.milliseconds + i, Message.MagicValue_V1)) MemoryRecords.withLogEntries(LogEntry.create(100 + i, Record.create(Record.MAGIC_VALUE_V1, time.milliseconds + i, i.toString.getBytes())))
} }
messages.foreach(log.append(_, assignOffsets = false)) messages.foreach(log.append(_, assignOffsets = false))
val timeIndexEntries = log.logSegments.foldLeft(0) { (entries, segment) => entries + segment.timeIndex.entries } val timeIndexEntries = log.logSegments.foldLeft(0) { (entries, segment) => entries + segment.timeIndex.entries }
@ -608,7 +609,7 @@ class LogTest extends JUnitSuite {
val config = LogConfig(logProps) val config = LogConfig(logProps)
var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time) var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time)
for(i <- 0 until numMessages) for(i <- 0 until numMessages)
log.append(TestUtils.singleMessageSet(payload = TestUtils.randomBytes(10), timestamp = time.milliseconds + i * 10)) log.append(TestUtils.singletonRecords(value = TestUtils.randomBytes(10), timestamp = time.milliseconds + i * 10))
val indexFiles = log.logSegments.map(_.index.file) val indexFiles = log.logSegments.map(_.index.file)
val timeIndexFiles = log.logSegments.map(_.timeIndex.file) val timeIndexFiles = log.logSegments.map(_.timeIndex.file)
log.close() log.close()
@ -623,7 +624,7 @@ class LogTest extends JUnitSuite {
assertTrue("The index should have been rebuilt", log.logSegments.head.index.entries > 0) assertTrue("The index should have been rebuilt", log.logSegments.head.index.entries > 0)
assertTrue("The time index should have been rebuilt", log.logSegments.head.timeIndex.entries > 0) assertTrue("The time index should have been rebuilt", log.logSegments.head.timeIndex.entries > 0)
for(i <- 0 until numMessages) { for(i <- 0 until numMessages) {
assertEquals(i, log.read(i, 100, None).messageSet.head.offset) assertEquals(i, log.read(i, 100, None).records.shallowIterator.next().offset)
if (i == 0) if (i == 0)
assertEquals(log.logSegments.head.baseOffset, log.fetchOffsetsByTimestamp(time.milliseconds + i * 10).get.offset) assertEquals(log.logSegments.head.baseOffset, log.fetchOffsetsByTimestamp(time.milliseconds + i * 10).get.offset)
else else
@ -647,7 +648,7 @@ class LogTest extends JUnitSuite {
val config = LogConfig(logProps) val config = LogConfig(logProps)
var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time) var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time)
for(i <- 0 until numMessages) for(i <- 0 until numMessages)
log.append(TestUtils.singleMessageSet(payload = TestUtils.randomBytes(10), timestamp = time.milliseconds + i * 10)) log.append(TestUtils.singletonRecords(value = TestUtils.randomBytes(10), timestamp = time.milliseconds + i * 10))
val timeIndexFiles = log.logSegments.map(_.timeIndex.file) val timeIndexFiles = log.logSegments.map(_.timeIndex.file)
log.close() log.close()
@ -676,7 +677,7 @@ class LogTest extends JUnitSuite {
val config = LogConfig(logProps) val config = LogConfig(logProps)
var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time) var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time)
for(i <- 0 until numMessages) for(i <- 0 until numMessages)
log.append(TestUtils.singleMessageSet(payload = TestUtils.randomBytes(10), timestamp = time.milliseconds + i * 10)) log.append(TestUtils.singletonRecords(value = TestUtils.randomBytes(10), timestamp = time.milliseconds + i * 10))
val indexFiles = log.logSegments.map(_.index.file) val indexFiles = log.logSegments.map(_.index.file)
val timeIndexFiles = log.logSegments.map(_.timeIndex.file) val timeIndexFiles = log.logSegments.map(_.timeIndex.file)
log.close() log.close()
@ -699,7 +700,7 @@ class LogTest extends JUnitSuite {
log = new Log(logDir, config, recoveryPoint = 200L, time.scheduler, time) log = new Log(logDir, config, recoveryPoint = 200L, time.scheduler, time)
assertEquals("Should have %d messages when log is reopened".format(numMessages), numMessages, log.logEndOffset) assertEquals("Should have %d messages when log is reopened".format(numMessages), numMessages, log.logEndOffset)
for(i <- 0 until numMessages) { for(i <- 0 until numMessages) {
assertEquals(i, log.read(i, 100, None).messageSet.head.offset) assertEquals(i, log.read(i, 100, None).records.shallowIterator.next().offset)
if (i == 0) if (i == 0)
assertEquals(log.logSegments.head.baseOffset, log.fetchOffsetsByTimestamp(time.milliseconds + i * 10).get.offset) assertEquals(log.logSegments.head.baseOffset, log.fetchOffsetsByTimestamp(time.milliseconds + i * 10).get.offset)
else else
@ -713,7 +714,7 @@ class LogTest extends JUnitSuite {
*/ */
@Test @Test
def testTruncateTo() { def testTruncateTo() {
val set = TestUtils.singleMessageSet("test".getBytes) val set = TestUtils.singletonRecords("test".getBytes)
val setSize = set.sizeInBytes val setSize = set.sizeInBytes
val msgPerSeg = 10 val msgPerSeg = 10
val segmentSize = msgPerSeg * setSize // each segment will be 10 messages val segmentSize = msgPerSeg * setSize // each segment will be 10 messages
@ -770,7 +771,7 @@ class LogTest extends JUnitSuite {
*/ */
@Test @Test
def testIndexResizingAtTruncation() { def testIndexResizingAtTruncation() {
val setSize = TestUtils.singleMessageSet(payload = "test".getBytes).sizeInBytes val setSize = TestUtils.singletonRecords(value = "test".getBytes).sizeInBytes
val msgPerSeg = 10 val msgPerSeg = 10
val segmentSize = msgPerSeg * setSize // each segment will be 10 messages val segmentSize = msgPerSeg * setSize // each segment will be 10 messages
val logProps = new Properties() val logProps = new Properties()
@ -781,12 +782,12 @@ class LogTest extends JUnitSuite {
assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments) assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
for (i<- 1 to msgPerSeg) for (i<- 1 to msgPerSeg)
log.append(TestUtils.singleMessageSet(payload = "test".getBytes, timestamp = time.milliseconds + i)) log.append(TestUtils.singletonRecords(value = "test".getBytes, timestamp = time.milliseconds + i))
assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments) assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
time.sleep(msgPerSeg) time.sleep(msgPerSeg)
for (i<- 1 to msgPerSeg) for (i<- 1 to msgPerSeg)
log.append(TestUtils.singleMessageSet(payload = "test".getBytes, timestamp = time.milliseconds + i)) log.append(TestUtils.singletonRecords(value = "test".getBytes, timestamp = time.milliseconds + i))
assertEquals("There should be exactly 2 segment.", 2, log.numberOfSegments) assertEquals("There should be exactly 2 segment.", 2, log.numberOfSegments)
val expectedEntries = msgPerSeg - 1 val expectedEntries = msgPerSeg - 1
@ -800,7 +801,7 @@ class LogTest extends JUnitSuite {
time.sleep(msgPerSeg) time.sleep(msgPerSeg)
for (i<- 1 to msgPerSeg) for (i<- 1 to msgPerSeg)
log.append(TestUtils.singleMessageSet(payload = "test".getBytes, timestamp = time.milliseconds + i)) log.append(TestUtils.singletonRecords(value = "test".getBytes, timestamp = time.milliseconds + i))
assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments) assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
} }
@ -814,7 +815,7 @@ class LogTest extends JUnitSuite {
val bogusIndex2 = Log.indexFilename(logDir, 5) val bogusIndex2 = Log.indexFilename(logDir, 5)
val bogusTimeIndex2 = Log.timeIndexFilename(logDir, 5) val bogusTimeIndex2 = Log.timeIndexFilename(logDir, 5)
val set = TestUtils.singleMessageSet("test".getBytes) val set = TestUtils.singletonRecords("test".getBytes)
val logProps = new Properties() val logProps = new Properties()
logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer) logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer)
logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer) logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer)
@ -842,7 +843,7 @@ class LogTest extends JUnitSuite {
*/ */
@Test @Test
def testReopenThenTruncate() { def testReopenThenTruncate() {
val set = TestUtils.singleMessageSet("test".getBytes) val set = TestUtils.singletonRecords("test".getBytes)
val logProps = new Properties() val logProps = new Properties()
logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer) logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer)
logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer) logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer)
@ -875,7 +876,7 @@ class LogTest extends JUnitSuite {
*/ */
@Test @Test
def testAsyncDelete() { def testAsyncDelete() {
val set = TestUtils.singleMessageSet("test".getBytes) val set = TestUtils.singletonRecords("test".getBytes)
val asyncDeleteMs = 1000 val asyncDeleteMs = 1000
val logProps = new Properties() val logProps = new Properties()
logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer) logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer)
@ -921,7 +922,7 @@ class LogTest extends JUnitSuite {
*/ */
@Test @Test
def testOpenDeletesObsoleteFiles() { def testOpenDeletesObsoleteFiles() {
val set = TestUtils.singleMessageSet("test".getBytes) val set = TestUtils.singletonRecords("test".getBytes)
val logProps = new Properties() val logProps = new Properties()
logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer) logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer)
logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer) logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer)
@ -957,10 +958,10 @@ class LogTest extends JUnitSuite {
recoveryPoint = 0L, recoveryPoint = 0L,
time.scheduler, time.scheduler,
time) time)
log.append(new ByteBufferMessageSet(new Message(bytes = null))) log.append(MemoryRecords.withRecords(Record.create(null)))
val messageSet = log.read(0, 4096, None).messageSet val head = log.read(0, 4096, None).records.shallowIterator().next()
assertEquals(0, messageSet.head.offset) assertEquals(0, head.offset)
assertTrue("Message payload should be null.", messageSet.head.message.isNull) assertTrue("Message payload should be null.", head.record.hasNullValue)
} }
@Test(expected = classOf[IllegalArgumentException]) @Test(expected = classOf[IllegalArgumentException])
@ -970,9 +971,9 @@ class LogTest extends JUnitSuite {
recoveryPoint = 0L, recoveryPoint = 0L,
time.scheduler, time.scheduler,
time) time)
val messages = (0 until 2).map(id => new Message(id.toString.getBytes)).toArray val messages = (0 until 2).map(id => Record.create(id.toString.getBytes)).toArray
messages.foreach(message => log.append(new ByteBufferMessageSet(message))) messages.foreach(record => log.append(MemoryRecords.withRecords(record)))
val invalidMessage = new ByteBufferMessageSet(new Message(1.toString.getBytes)) val invalidMessage = MemoryRecords.withRecords(Record.create(1.toString.getBytes))
log.append(invalidMessage, assignOffsets = false) log.append(invalidMessage, assignOffsets = false)
} }
@ -984,7 +985,7 @@ class LogTest extends JUnitSuite {
logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer)
logProps.put(LogConfig.MaxMessageBytesProp, 64*1024: java.lang.Integer) logProps.put(LogConfig.MaxMessageBytesProp, 64*1024: java.lang.Integer)
val config = LogConfig(logProps) val config = LogConfig(logProps)
val set = TestUtils.singleMessageSet("test".getBytes) val set = TestUtils.singletonRecords("test".getBytes)
val recoveryPoint = 50L val recoveryPoint = 50L
for (_ <- 0 until 50) { for (_ <- 0 until 50) {
// create a log and write some messages to it // create a log and write some messages to it
@ -997,7 +998,7 @@ class LogTest extends JUnitSuite {
val numMessages = 50 + TestUtils.random.nextInt(50) val numMessages = 50 + TestUtils.random.nextInt(50)
for (_ <- 0 until numMessages) for (_ <- 0 until numMessages)
log.append(set) log.append(set)
val messages = log.logSegments.flatMap(_.log.iterator.toList) val messages = log.logSegments.flatMap(_.log.deepIterator.asScala.toList)
log.close() log.close()
// corrupt index and log by appending random bytes // corrupt index and log by appending random bytes
@ -1007,7 +1008,8 @@ class LogTest extends JUnitSuite {
// attempt recovery // attempt recovery
log = new Log(logDir, config, recoveryPoint, time.scheduler, time) log = new Log(logDir, config, recoveryPoint, time.scheduler, time)
assertEquals(numMessages, log.logEndOffset) assertEquals(numMessages, log.logEndOffset)
assertEquals("Messages in the log after recovery should be the same.", messages, log.logSegments.flatMap(_.log.iterator.toList)) assertEquals("Messages in the log after recovery should be the same.", messages,
log.logSegments.flatMap(_.log.deepIterator.asScala.toList))
Utils.delete(logDir) Utils.delete(logDir)
} }
} }
@ -1020,7 +1022,7 @@ class LogTest extends JUnitSuite {
logProps.put(LogConfig.MaxMessageBytesProp, 64*1024: java.lang.Integer) logProps.put(LogConfig.MaxMessageBytesProp, 64*1024: java.lang.Integer)
logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer) logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer)
val config = LogConfig(logProps) val config = LogConfig(logProps)
val set = TestUtils.singleMessageSet("test".getBytes) val set = TestUtils.singletonRecords("test".getBytes)
val parentLogDir = logDir.getParentFile val parentLogDir = logDir.getParentFile
assertTrue("Data directory %s must exist", parentLogDir.isDirectory) assertTrue("Data directory %s must exist", parentLogDir.isDirectory)
val cleanShutdownFile = new File(parentLogDir, Log.CleanShutdownFile) val cleanShutdownFile = new File(parentLogDir, Log.CleanShutdownFile)
@ -1121,7 +1123,7 @@ class LogTest extends JUnitSuite {
@Test @Test
def testDeleteOldSegmentsMethod() { def testDeleteOldSegmentsMethod() {
val set = TestUtils.singleMessageSet("test".getBytes) val set = TestUtils.singletonRecords("test".getBytes)
val logProps = new Properties() val logProps = new Properties()
logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer) logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer)
logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer) logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer)
@ -1154,7 +1156,7 @@ class LogTest extends JUnitSuite {
@Test @Test
def shouldDeleteSizeBasedSegments() { def shouldDeleteSizeBasedSegments() {
val set = TestUtils.singleMessageSet("test".getBytes) val set = TestUtils.singletonRecords("test".getBytes)
val log = createLog(set.sizeInBytes, retentionBytes = set.sizeInBytes * 10) val log = createLog(set.sizeInBytes, retentionBytes = set.sizeInBytes * 10)
// append some messages to create some segments // append some messages to create some segments
@ -1167,7 +1169,7 @@ class LogTest extends JUnitSuite {
@Test @Test
def shouldNotDeleteSizeBasedSegmentsWhenUnderRetentionSize() { def shouldNotDeleteSizeBasedSegmentsWhenUnderRetentionSize() {
val set = TestUtils.singleMessageSet("test".getBytes) val set = TestUtils.singletonRecords("test".getBytes)
val log = createLog(set.sizeInBytes, retentionBytes = set.sizeInBytes * 15) val log = createLog(set.sizeInBytes, retentionBytes = set.sizeInBytes * 15)
// append some messages to create some segments // append some messages to create some segments
@ -1180,7 +1182,7 @@ class LogTest extends JUnitSuite {
@Test @Test
def shouldDeleteTimeBasedSegmentsReadyToBeDeleted() { def shouldDeleteTimeBasedSegmentsReadyToBeDeleted() {
val set = TestUtils.singleMessageSet("test".getBytes, timestamp = 10) val set = TestUtils.singletonRecords("test".getBytes, timestamp = 10)
val log = createLog(set.sizeInBytes, retentionMs = 10000) val log = createLog(set.sizeInBytes, retentionMs = 10000)
// append some messages to create some segments // append some messages to create some segments
@ -1193,7 +1195,7 @@ class LogTest extends JUnitSuite {
@Test @Test
def shouldNotDeleteTimeBasedSegmentsWhenNoneReadyToBeDeleted() { def shouldNotDeleteTimeBasedSegmentsWhenNoneReadyToBeDeleted() {
val set = TestUtils.singleMessageSet("test".getBytes, timestamp = time.milliseconds) val set = TestUtils.singletonRecords("test".getBytes, timestamp = time.milliseconds)
val log = createLog(set.sizeInBytes, retentionMs = 10000000) val log = createLog(set.sizeInBytes, retentionMs = 10000000)
// append some messages to create some segments // append some messages to create some segments
@ -1206,7 +1208,7 @@ class LogTest extends JUnitSuite {
@Test @Test
def shouldNotDeleteSegmentsWhenPolicyDoesNotIncludeDelete() { def shouldNotDeleteSegmentsWhenPolicyDoesNotIncludeDelete() {
val set = TestUtils.singleMessageSet("test".getBytes, key = "test".getBytes(), timestamp = 10L) val set = TestUtils.singletonRecords("test".getBytes, key = "test".getBytes(), timestamp = 10L)
val log = createLog(set.sizeInBytes, val log = createLog(set.sizeInBytes,
retentionMs = 10000, retentionMs = 10000,
cleanupPolicy = "compact") cleanupPolicy = "compact")
@ -1225,7 +1227,7 @@ class LogTest extends JUnitSuite {
@Test @Test
def shouldDeleteSegmentsReadyToBeDeletedWhenCleanupPolicyIsCompactAndDelete() { def shouldDeleteSegmentsReadyToBeDeletedWhenCleanupPolicyIsCompactAndDelete() {
val set = TestUtils.singleMessageSet("test".getBytes, key = "test".getBytes,timestamp = 10L) val set = TestUtils.singletonRecords("test".getBytes, key = "test".getBytes,timestamp = 10L)
val log = createLog(set.sizeInBytes, val log = createLog(set.sizeInBytes,
retentionMs = 10000, retentionMs = 10000,
cleanupPolicy = "compact,delete") cleanupPolicy = "compact,delete")

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 package kafka.message
import java.nio.ByteBuffer import java.nio.ByteBuffer
import java.nio.channels.{FileChannel, GatheringByteChannel} import java.nio.channels.GatheringByteChannel
import java.nio.file.StandardOpenOption
import org.junit.Assert._ import org.junit.Assert._
import kafka.utils.TestUtils._ import kafka.utils.TestUtils._
import kafka.log.FileMessageSet import org.apache.kafka.common.record.FileRecords
import kafka.utils.TestUtils
import org.scalatest.junit.JUnitSuite import org.scalatest.junit.JUnitSuite
import org.junit.Test import org.junit.Test
@ -94,7 +92,7 @@ trait BaseMessageSetTestCases extends JUnitSuite {
@Test @Test
def testWriteToChannelThatConsumesPartially() { def testWriteToChannelThatConsumesPartially() {
val bytesToConsumePerBuffer = 50 val bytesToConsumePerBuffer = 50
val messages = (0 until 10).map(_ => new Message(TestUtils.randomString(100).getBytes)) val messages = (0 until 10).map(_ => new Message(randomString(100).getBytes))
val messageSet = createMessageSet(messages) val messageSet = createMessageSet(messages)
val messageSetSize = messageSet.sizeInBytes val messageSetSize = messageSet.sizeInBytes
@ -119,15 +117,15 @@ trait BaseMessageSetTestCases extends JUnitSuite {
// do the write twice to ensure the message set is restored to its original state // do the write twice to ensure the message set is restored to its original state
for (_ <- 0 to 1) { for (_ <- 0 to 1) {
val file = tempFile() val file = tempFile()
val channel = FileChannel.open(file.toPath, StandardOpenOption.READ, StandardOpenOption.WRITE) val fileRecords = FileRecords.open(file, true)
try { try {
val written = write(channel) val written = write(fileRecords.channel)
fileRecords.resize() // resize since we wrote to the channel directly
assertEquals("Expect to write the number of bytes in the set.", set.sizeInBytes, written) assertEquals("Expect to write the number of bytes in the set.", set.sizeInBytes, written)
val newSet = new FileMessageSet(file, channel) checkEquals(set.asRecords.deepIterator, fileRecords.deepIterator())
checkEquals(set.iterator, newSet.iterator) } finally fileRecords.close()
} finally channel.close()
} }
} }
} }

View File

@ -17,13 +17,9 @@
package kafka.message package kafka.message
import java.io.DataOutputStream
import java.nio._ import java.nio._
import kafka.common.LongRef
import kafka.utils.TestUtils import kafka.utils.TestUtils
import org.apache.kafka.common.errors.InvalidTimestampException
import org.apache.kafka.common.record.TimestampType
import org.junit.Assert._ import org.junit.Assert._
import org.junit.Test import org.junit.Test
@ -151,295 +147,6 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases {
assertEquals("second offset should be 2", 2L, iter.next().offset) assertEquals("second offset should be 2", 2L, iter.next().offset)
} }
@Test
def testLogAppendTime() {
val now = System.currentTimeMillis()
// The timestamps should be overwritten
val messages = getMessages(magicValue = Message.MagicValue_V1, timestamp = 0L, codec = NoCompressionCodec)
val compressedMessagesWithRecompresion = getMessages(magicValue = Message.MagicValue_V0, codec = DefaultCompressionCodec)
val compressedMessagesWithoutRecompression =
getMessages(magicValue = Message.MagicValue_V1, timestamp = 0L, codec = DefaultCompressionCodec)
val validatingResults = messages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0),
now = now,
sourceCodec = NoCompressionCodec,
targetCodec = NoCompressionCodec,
messageFormatVersion = 1,
messageTimestampType = TimestampType.LOG_APPEND_TIME,
messageTimestampDiffMaxMs = 1000L)
val validatedMessages = validatingResults.validatedMessages
val validatingCompressedMessagesResults =
compressedMessagesWithRecompresion.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0),
now = now,
sourceCodec = DefaultCompressionCodec,
targetCodec = DefaultCompressionCodec,
messageFormatVersion = 1,
messageTimestampType = TimestampType.LOG_APPEND_TIME,
messageTimestampDiffMaxMs = 1000L)
val validatedCompressedMessages = validatingCompressedMessagesResults.validatedMessages
val validatingCompressedMessagesWithoutRecompressionResults =
compressedMessagesWithoutRecompression.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0),
now = now,
sourceCodec = DefaultCompressionCodec,
targetCodec = DefaultCompressionCodec,
messageFormatVersion = 1,
messageTimestampType = TimestampType.LOG_APPEND_TIME,
messageTimestampDiffMaxMs = 1000L)
val validatedCompressedMessagesWithoutRecompression = validatingCompressedMessagesWithoutRecompressionResults.validatedMessages
assertEquals("message set size should not change", messages.size, validatedMessages.size)
validatedMessages.foreach(messageAndOffset => validateLogAppendTime(messageAndOffset.message))
assertEquals(s"Max timestamp should be $now", now, validatingResults.maxTimestamp)
assertEquals(s"The offset of max timestamp should be 0", 0, validatingResults.offsetOfMaxTimestamp)
assertFalse("Message size should not have been changed", validatingResults.messageSizeMaybeChanged)
assertEquals("message set size should not change", compressedMessagesWithRecompresion.size, validatedCompressedMessages.size)
validatedCompressedMessages.foreach(messageAndOffset => validateLogAppendTime(messageAndOffset.message))
assertTrue("MessageSet should still valid", validatedCompressedMessages.shallowIterator.next().message.isValid)
assertEquals(s"Max timestamp should be $now", now, validatingCompressedMessagesResults.maxTimestamp)
assertEquals(s"The offset of max timestamp should be ${compressedMessagesWithRecompresion.size - 1}",
compressedMessagesWithRecompresion.size - 1, validatingCompressedMessagesResults.offsetOfMaxTimestamp)
assertTrue("Message size may have been changed", validatingCompressedMessagesResults.messageSizeMaybeChanged)
assertEquals("message set size should not change", compressedMessagesWithoutRecompression.size,
validatedCompressedMessagesWithoutRecompression.size)
validatedCompressedMessagesWithoutRecompression.foreach(messageAndOffset => validateLogAppendTime(messageAndOffset.message))
assertTrue("MessageSet should still valid", validatedCompressedMessagesWithoutRecompression.shallowIterator.next().message.isValid)
assertEquals(s"Max timestamp should be $now", now, validatingCompressedMessagesWithoutRecompressionResults.maxTimestamp)
assertEquals(s"The offset of max timestamp should be ${compressedMessagesWithoutRecompression.size - 1}",
compressedMessagesWithoutRecompression.size - 1, validatingCompressedMessagesWithoutRecompressionResults.offsetOfMaxTimestamp)
assertFalse("Message size should not have been changed", validatingCompressedMessagesWithoutRecompressionResults.messageSizeMaybeChanged)
def validateLogAppendTime(message: Message) {
message.ensureValid()
assertEquals(s"Timestamp of message $message should be $now", now, message.timestamp)
assertEquals(TimestampType.LOG_APPEND_TIME, message.timestampType)
}
}
@Test
def testCreateTime() {
val now = System.currentTimeMillis()
val timestampSeq = Seq(now - 1, now + 1, now)
val messages =
new ByteBufferMessageSet(NoCompressionCodec,
new Message("hello".getBytes, timestamp = timestampSeq(0), magicValue = Message.MagicValue_V1),
new Message("there".getBytes, timestamp = timestampSeq(1), magicValue = Message.MagicValue_V1),
new Message("beautiful".getBytes, timestamp = timestampSeq(2), magicValue = Message.MagicValue_V1))
val compressedMessages =
new ByteBufferMessageSet(DefaultCompressionCodec,
new Message("hello".getBytes, timestamp = timestampSeq(0), magicValue = Message.MagicValue_V1),
new Message("there".getBytes, timestamp = timestampSeq(1), magicValue = Message.MagicValue_V1),
new Message("beautiful".getBytes, timestamp = timestampSeq(2), magicValue = Message.MagicValue_V1))
val validatingResults = messages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0),
now = System.currentTimeMillis(),
sourceCodec = NoCompressionCodec,
targetCodec = NoCompressionCodec,
messageFormatVersion = 1,
messageTimestampType = TimestampType.CREATE_TIME,
messageTimestampDiffMaxMs = 1000L)
val validatedMessages = validatingResults.validatedMessages
val validatingCompressedMessagesResults =
compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0),
now = System.currentTimeMillis(),
sourceCodec = DefaultCompressionCodec,
targetCodec = DefaultCompressionCodec,
messageFormatVersion = 1,
messageTimestampType = TimestampType.CREATE_TIME,
messageTimestampDiffMaxMs = 1000L)
val validatedCompressedMessages = validatingCompressedMessagesResults.validatedMessages
var i = 0
for (messageAndOffset <- validatedMessages) {
messageAndOffset.message.ensureValid()
assertEquals(messageAndOffset.message.timestamp, timestampSeq(i))
assertEquals(messageAndOffset.message.timestampType, TimestampType.CREATE_TIME)
i += 1
}
assertEquals(s"Max timestamp should be ${now + 1}", now + 1, validatingResults.maxTimestamp)
assertEquals(s"Offset of max timestamp should be 1", 1, validatingResults.offsetOfMaxTimestamp)
assertFalse("Message size should not have been changed", validatingResults.messageSizeMaybeChanged)
i = 0
for (messageAndOffset <- validatedCompressedMessages) {
messageAndOffset.message.ensureValid()
assertEquals(messageAndOffset.message.timestamp, timestampSeq(i))
assertEquals(messageAndOffset.message.timestampType, TimestampType.CREATE_TIME)
i += 1
}
assertEquals(s"Max timestamp should be ${now + 1}", now + 1, validatingResults.maxTimestamp)
assertEquals(s"Offset of max timestamp should be ${validatedCompressedMessages.size - 1}",
validatedCompressedMessages.size - 1, validatingCompressedMessagesResults.offsetOfMaxTimestamp)
assertFalse("Message size should not have been changed", validatingCompressedMessagesResults.messageSizeMaybeChanged)
}
@Test
def testInvalidCreateTime() {
val now = System.currentTimeMillis()
val messages = getMessages(magicValue = Message.MagicValue_V1, timestamp = now - 1001L, codec = NoCompressionCodec)
val compressedMessages = getMessages(magicValue = Message.MagicValue_V1, timestamp = now - 1001L, codec = DefaultCompressionCodec)
try {
messages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0),
now = System.currentTimeMillis(),
sourceCodec = NoCompressionCodec,
targetCodec = NoCompressionCodec,
messageFormatVersion = 1,
messageTimestampType = TimestampType.CREATE_TIME,
messageTimestampDiffMaxMs = 1000L)
fail("Should throw InvalidMessageException.")
} catch {
case _: InvalidTimestampException =>
}
try {
compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0),
now = System.currentTimeMillis(),
sourceCodec = DefaultCompressionCodec,
targetCodec = DefaultCompressionCodec,
messageFormatVersion = 1,
messageTimestampType = TimestampType.CREATE_TIME,
messageTimestampDiffMaxMs = 1000L)
fail("Should throw InvalidMessageException.")
} catch {
case _: InvalidTimestampException =>
}
}
@Test
def testAbsoluteOffsetAssignment() {
val messages = getMessages(magicValue = Message.MagicValue_V0, codec = NoCompressionCodec)
val compressedMessages = getMessages(magicValue = Message.MagicValue_V0, codec = DefaultCompressionCodec)
// check uncompressed offsets
checkOffsets(messages, 0)
val offset = 1234567
checkOffsets(messages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
now = System.currentTimeMillis(),
sourceCodec = NoCompressionCodec,
targetCodec = NoCompressionCodec,
messageFormatVersion = 0,
messageTimestampType = TimestampType.CREATE_TIME,
messageTimestampDiffMaxMs = 1000L).validatedMessages, offset)
// check compressed messages
checkOffsets(compressedMessages, 0)
checkOffsets(compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
now = System.currentTimeMillis(),
sourceCodec = DefaultCompressionCodec,
targetCodec = DefaultCompressionCodec,
messageFormatVersion = 0,
messageTimestampType = TimestampType.CREATE_TIME,
messageTimestampDiffMaxMs = 1000L).validatedMessages, offset)
}
@Test
def testRelativeOffsetAssignment() {
val now = System.currentTimeMillis()
val messages = getMessages(magicValue = Message.MagicValue_V1, timestamp = now, codec = NoCompressionCodec)
val compressedMessages = getMessages(magicValue = Message.MagicValue_V1, timestamp = now, codec = DefaultCompressionCodec)
// check uncompressed offsets
checkOffsets(messages, 0)
val offset = 1234567
val messageWithOffset = messages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
now = System.currentTimeMillis(),
sourceCodec = NoCompressionCodec,
targetCodec = NoCompressionCodec,
messageTimestampType = TimestampType.CREATE_TIME,
messageTimestampDiffMaxMs = 5000L).validatedMessages
checkOffsets(messageWithOffset, offset)
// check compressed messages
checkOffsets(compressedMessages, 0)
val compressedMessagesWithOffset = compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
now = System.currentTimeMillis(),
sourceCodec = DefaultCompressionCodec,
targetCodec = DefaultCompressionCodec,
messageTimestampType = TimestampType.CREATE_TIME,
messageTimestampDiffMaxMs = 5000L).validatedMessages
checkOffsets(compressedMessagesWithOffset, offset)
}
@Test(expected = classOf[InvalidMessageException])
def testInvalidInnerMagicVersion(): Unit = {
val offset = 1234567
val messages = messageSetWithInvalidInnerMagic(SnappyCompressionCodec, offset)
messages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
now = System.currentTimeMillis(),
sourceCodec = SnappyCompressionCodec,
targetCodec = SnappyCompressionCodec,
messageTimestampType = TimestampType.CREATE_TIME,
messageTimestampDiffMaxMs = 5000L).validatedMessages
}
@Test
def testOffsetAssignmentAfterMessageFormatConversion() {
// Check up conversion
val messagesV0 = getMessages(magicValue = Message.MagicValue_V0, codec = NoCompressionCodec)
val compressedMessagesV0 = getMessages(magicValue = Message.MagicValue_V0, codec = DefaultCompressionCodec)
// check uncompressed offsets
checkOffsets(messagesV0, 0)
val offset = 1234567
checkOffsets(messagesV0.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
now = System.currentTimeMillis(),
sourceCodec = NoCompressionCodec,
targetCodec = NoCompressionCodec,
messageFormatVersion = 1,
messageTimestampType = TimestampType.LOG_APPEND_TIME,
messageTimestampDiffMaxMs = 1000L).validatedMessages, offset)
// check compressed messages
checkOffsets(compressedMessagesV0, 0)
checkOffsets(compressedMessagesV0.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
now = System.currentTimeMillis(),
sourceCodec = DefaultCompressionCodec,
targetCodec = DefaultCompressionCodec,
messageFormatVersion = 1,
messageTimestampType = TimestampType.LOG_APPEND_TIME,
messageTimestampDiffMaxMs = 1000L).validatedMessages, offset)
// Check down conversion
val now = System.currentTimeMillis()
val messagesV1 = getMessages(Message.MagicValue_V1, now, NoCompressionCodec)
val compressedMessagesV1 = getMessages(Message.MagicValue_V1, now, DefaultCompressionCodec)
// check uncompressed offsets
checkOffsets(messagesV1, 0)
checkOffsets(messagesV1.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
now = System.currentTimeMillis(),
sourceCodec = NoCompressionCodec,
targetCodec = NoCompressionCodec,
messageFormatVersion = 0,
messageTimestampType = TimestampType.CREATE_TIME,
messageTimestampDiffMaxMs = 5000L).validatedMessages, offset)
// check compressed messages
checkOffsets(compressedMessagesV1, 0)
checkOffsets(compressedMessagesV1.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
now = System.currentTimeMillis(),
sourceCodec = DefaultCompressionCodec,
targetCodec = DefaultCompressionCodec,
messageFormatVersion = 0,
messageTimestampType = TimestampType.CREATE_TIME,
messageTimestampDiffMaxMs = 5000L).validatedMessages, offset)
}
@Test
def testWriteFullyTo() {
checkWriteFullyToWithMessageSet(createMessageSet(Array[Message]()))
checkWriteFullyToWithMessageSet(createMessageSet(messages))
}
def checkWriteFullyToWithMessageSet(messageSet: ByteBufferMessageSet) {
checkWriteWithMessageSet(messageSet, messageSet.writeFullyTo)
}
/* check that offsets are assigned based on byte offset from the given base offset */ /* check that offsets are assigned based on byte offset from the given base offset */
def checkOffsets(messages: ByteBufferMessageSet, baseOffset: Long) { def checkOffsets(messages: ByteBufferMessageSet, baseOffset: Long) {
assertTrue("Message set should not be empty", messages.nonEmpty) assertTrue("Message set should not be empty", messages.nonEmpty)
@ -457,59 +164,4 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases {
assertTrue(shallowOffsets.subsetOf(deepOffsets)) assertTrue(shallowOffsets.subsetOf(deepOffsets))
} }
private def getMessages(magicValue: Byte = Message.CurrentMagicValue,
timestamp: Long = Message.NoTimestamp,
codec: CompressionCodec = NoCompressionCodec): ByteBufferMessageSet = {
if (magicValue == Message.MagicValue_V0) {
new ByteBufferMessageSet(
codec,
new Message("hello".getBytes, Message.NoTimestamp, Message.MagicValue_V0),
new Message("there".getBytes, Message.NoTimestamp, Message.MagicValue_V0),
new Message("beautiful".getBytes, Message.NoTimestamp, Message.MagicValue_V0))
} else {
new ByteBufferMessageSet(
codec,
new Message("hello".getBytes, timestamp = timestamp, magicValue = Message.MagicValue_V1),
new Message("there".getBytes, timestamp = timestamp, magicValue = Message.MagicValue_V1),
new Message("beautiful".getBytes, timestamp = timestamp, magicValue = Message.MagicValue_V1))
}
}
private def messageSetWithInvalidInnerMagic(codec: CompressionCodec,
initialOffset: Long): ByteBufferMessageSet = {
val messages = (0 until 20).map(id =>
new Message(key = id.toString.getBytes,
bytes = id.toString.getBytes,
timestamp = Message.NoTimestamp,
magicValue = Message.MagicValue_V0))
val messageWriter = new MessageWriter(math.min(math.max(MessageSet.messageSetSize(messages) / 2, 1024), 1 << 16))
var lastOffset = initialOffset
messageWriter.write(
codec = codec,
timestamp = System.currentTimeMillis(),
timestampType = TimestampType.CREATE_TIME,
magicValue = Message.MagicValue_V1) { outputStream =>
val output = new DataOutputStream(CompressionFactory(codec, Message.MagicValue_V1, outputStream))
try {
for (message <- messages) {
val innerOffset = lastOffset - initialOffset
output.writeLong(innerOffset)
output.writeInt(message.size)
output.write(message.buffer.array, message.buffer.arrayOffset, message.buffer.limit)
lastOffset += 1
}
} finally {
output.close()
}
}
val buffer = ByteBuffer.allocate(messageWriter.size + MessageSet.LogOverhead)
ByteBufferMessageSet.writeMessage(buffer, messageWriter, lastOffset - 1)
buffer.rewind()
new ByteBufferMessageSet(buffer)
}
} }

View File

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

View File

@ -48,7 +48,7 @@ class MessageTest extends JUnitSuite {
val magicValues = Array(Message.MagicValue_V0, Message.MagicValue_V1) val magicValues = Array(Message.MagicValue_V0, Message.MagicValue_V1)
for(k <- keys; v <- vals; codec <- codecs; t <- timestamps; mv <- magicValues) { for(k <- keys; v <- vals; codec <- codecs; t <- timestamps; mv <- magicValues) {
val timestamp = ensureValid(mv, t) val timestamp = ensureValid(mv, t)
messages += new MessageTestVal(k, v, codec, timestamp, mv, new Message(v, k, timestamp, codec, mv)) messages += MessageTestVal(k, v, codec, timestamp, mv, new Message(v, k, timestamp, codec, mv))
} }
def ensureValid(magicValue: Byte, timestamp: Long): Long = def ensureValid(magicValue: Byte, timestamp: Long): Long =
@ -96,7 +96,7 @@ class MessageTest extends JUnitSuite {
@Test @Test
def testEquality() { def testEquality() {
for(v <- messages) { for (v <- messages) {
assertFalse("Should not equal null", v.message.equals(null)) assertFalse("Should not equal null", v.message.equals(null))
assertFalse("Should not equal a random string", v.message.equals("asdf")) assertFalse("Should not equal a random string", v.message.equals("asdf"))
assertTrue("Should equal itself", v.message.equals(v.message)) assertTrue("Should equal itself", v.message.equals(v.message))
@ -105,40 +105,6 @@ class MessageTest extends JUnitSuite {
} }
} }
@Test
def testMessageFormatConversion() {
def convertAndVerify(v: MessageTestVal, fromMessageFormat: Byte, toMessageFormat: Byte) {
assertEquals("Message should be the same when convert to the same version.",
v.message.toFormatVersion(fromMessageFormat), v.message)
val convertedMessage = v.message.toFormatVersion(toMessageFormat)
assertEquals("Size difference is not expected value", convertedMessage.size - v.message.size,
Message.headerSizeDiff(fromMessageFormat, toMessageFormat))
assertTrue("Message should still be valid", convertedMessage.isValid)
assertEquals("Timestamp should be NoTimestamp", convertedMessage.timestamp, Message.NoTimestamp)
assertEquals(s"Magic value should be $toMessageFormat now", convertedMessage.magic, toMessageFormat)
if (convertedMessage.hasKey)
assertEquals("Message key should not change", convertedMessage.key, ByteBuffer.wrap(v.key))
else
assertNull(convertedMessage.key)
if(v.payload == null) {
assertTrue(convertedMessage.isNull)
assertEquals("Payload should be null", null, convertedMessage.payload)
} else {
assertEquals("Message payload should not change", convertedMessage.payload, ByteBuffer.wrap(v.payload))
}
assertEquals("Compression codec should not change", convertedMessage.compressionCodec, v.codec)
}
for (v <- messages) {
if (v.magicValue == Message.MagicValue_V0) {
convertAndVerify(v, Message.MagicValue_V0, Message.MagicValue_V1)
} else if (v.magicValue == Message.MagicValue_V1) {
convertAndVerify(v, Message.MagicValue_V1, Message.MagicValue_V0)
}
}
}
@Test(expected = classOf[IllegalArgumentException]) @Test(expected = classOf[IllegalArgumentException])
def testInvalidTimestampAndMagicValueCombination() { def testInvalidTimestampAndMagicValueCombination() {
new Message("hello".getBytes, 0L, Message.MagicValue_V0) new Message("hello".getBytes, 0L, Message.MagicValue_V0)

View File

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

View File

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

View File

@ -17,21 +17,20 @@
package kafka.server package kafka.server
import java.util.Properties import java.util.Properties
import org.apache.kafka.common.metrics.Metrics
import org.junit.{After, Before, Test}
import collection.mutable.HashMap
import collection.mutable.Map
import kafka.cluster.{Partition, Replica}
import org.easymock.EasyMock
import kafka.log.Log
import org.junit.Assert._
import kafka.utils._
import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.AtomicBoolean
import kafka.message.MessageSet import kafka.cluster.{Partition, Replica}
import kafka.log.Log
import kafka.utils._
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.record.MemoryRecords
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
import org.easymock.EasyMock
import org.junit.Assert._
import org.junit.{After, Before, Test}
import scala.collection.mutable.{HashMap, Map}
class IsrExpirationTest { class IsrExpirationTest {
@ -76,7 +75,7 @@ class IsrExpirationTest {
// let the follower catch up to the Leader logEndOffset (15) // let the follower catch up to the Leader logEndOffset (15)
(partition0.assignedReplicas() - leaderReplica).foreach( (partition0.assignedReplicas() - leaderReplica).foreach(
r => r.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(15L), MessageSet.Empty), r => r.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(15L), MemoryRecords.EMPTY),
-1L, -1L,
-1, -1,
true))) true)))
@ -127,7 +126,7 @@ class IsrExpirationTest {
// Make the remote replica not read to the end of log. It should be not be out of sync for at least 100 ms // Make the remote replica not read to the end of log. It should be not be out of sync for at least 100 ms
for(replica <- partition0.assignedReplicas() - leaderReplica) for(replica <- partition0.assignedReplicas() - leaderReplica)
replica.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(10L), MessageSet.Empty), -1L, -1, false)) replica.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(10L), MemoryRecords.EMPTY), -1L, -1, false))
// Simulate 2 fetch requests spanning more than 100 ms which do not read to the end of the log. // Simulate 2 fetch requests spanning more than 100 ms which do not read to the end of the log.
// The replicas will no longer be in ISR. We do 2 fetches because we want to simulate the case where the replica is lagging but is not stuck // The replicas will no longer be in ISR. We do 2 fetches because we want to simulate the case where the replica is lagging but is not stuck
@ -137,7 +136,7 @@ class IsrExpirationTest {
time.sleep(75) time.sleep(75)
(partition0.assignedReplicas() - leaderReplica).foreach( (partition0.assignedReplicas() - leaderReplica).foreach(
r => r.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(11L), MessageSet.Empty), -1L, -1, false))) r => r.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(11L), MemoryRecords.EMPTY), -1L, -1, false)))
partition0OSR = partition0.getOutOfSyncReplicas(leaderReplica, configs.head.replicaLagTimeMaxMs) partition0OSR = partition0.getOutOfSyncReplicas(leaderReplica, configs.head.replicaLagTimeMaxMs)
assertEquals("No replica should be out of sync", Set.empty[Int], partition0OSR.map(_.brokerId)) assertEquals("No replica should be out of sync", Set.empty[Int], partition0OSR.map(_.brokerId))
@ -149,7 +148,7 @@ class IsrExpirationTest {
// Now actually make a fetch to the end of the log. The replicas should be back in ISR // Now actually make a fetch to the end of the log. The replicas should be back in ISR
(partition0.assignedReplicas() - leaderReplica).foreach( (partition0.assignedReplicas() - leaderReplica).foreach(
r => r.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(15L), MessageSet.Empty), -1L, -1, true))) r => r.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(15L), MemoryRecords.EMPTY), -1L, -1, true)))
partition0OSR = partition0.getOutOfSyncReplicas(leaderReplica, configs.head.replicaLagTimeMaxMs) partition0OSR = partition0.getOutOfSyncReplicas(leaderReplica, configs.head.replicaLagTimeMaxMs)
assertEquals("No replica should be out of sync", Set.empty[Int], partition0OSR.map(_.brokerId)) assertEquals("No replica should be out of sync", Set.empty[Int], partition0OSR.map(_.brokerId))

View File

@ -26,12 +26,12 @@ import kafka.api.{FetchRequestBuilder, OffsetRequest, PartitionOffsetRequestInfo
import kafka.common.TopicAndPartition import kafka.common.TopicAndPartition
import kafka.consumer.SimpleConsumer import kafka.consumer.SimpleConsumer
import kafka.log.{Log, LogSegment} import kafka.log.{Log, LogSegment}
import kafka.message.{ByteBufferMessageSet, Message, NoCompressionCodec}
import kafka.utils.TestUtils._ import kafka.utils.TestUtils._
import kafka.utils._ import kafka.utils._
import kafka.zk.ZooKeeperTestHarness import kafka.zk.ZooKeeperTestHarness
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.record.{MemoryRecords, Record}
import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.utils.{Time, Utils}
import org.easymock.{EasyMock, IAnswer} import org.easymock.{EasyMock, IAnswer}
import org.junit.Assert._ import org.junit.Assert._
@ -89,9 +89,9 @@ class LogOffsetTest extends ZooKeeperTestHarness {
"Log for partition [topic,0] should be created") "Log for partition [topic,0] should be created")
val log = logManager.getLog(TopicAndPartition(topic, part)).get val log = logManager.getLog(TopicAndPartition(topic, part)).get
val message = new Message(Integer.toString(42).getBytes()) val record = Record.create(Integer.toString(42).getBytes())
for (_ <- 0 until 20) for (_ <- 0 until 20)
log.append(new ByteBufferMessageSet(NoCompressionCodec, message)) log.append(MemoryRecords.withRecords(record))
log.flush() log.flush()
val offsets = server.apis.fetchOffsets(logManager, new TopicPartition(topic, part), OffsetRequest.LatestTime, 15) val offsets = server.apis.fetchOffsets(logManager, new TopicPartition(topic, part), OffsetRequest.LatestTime, 15)
@ -150,9 +150,9 @@ class LogOffsetTest extends ZooKeeperTestHarness {
val logManager = server.getLogManager val logManager = server.getLogManager
val log = logManager.createLog(TopicAndPartition(topic, part), logManager.defaultConfig) val log = logManager.createLog(TopicAndPartition(topic, part), logManager.defaultConfig)
val message = new Message(Integer.toString(42).getBytes()) val record = Record.create(Integer.toString(42).getBytes())
for (_ <- 0 until 20) for (_ <- 0 until 20)
log.append(new ByteBufferMessageSet(NoCompressionCodec, message)) log.append(MemoryRecords.withRecords(record))
log.flush() log.flush()
val now = time.milliseconds + 30000 // pretend it is the future to avoid race conditions with the fs val now = time.milliseconds + 30000 // pretend it is the future to avoid race conditions with the fs
@ -179,9 +179,9 @@ class LogOffsetTest extends ZooKeeperTestHarness {
val logManager = server.getLogManager val logManager = server.getLogManager
val log = logManager.createLog(TopicAndPartition(topic, part), logManager.defaultConfig) val log = logManager.createLog(TopicAndPartition(topic, part), logManager.defaultConfig)
val message = new Message(Integer.toString(42).getBytes()) val record = Record.create(Integer.toString(42).getBytes())
for (_ <- 0 until 20) for (_ <- 0 until 20)
log.append(new ByteBufferMessageSet(NoCompressionCodec, message)) log.append(MemoryRecords.withRecords(record))
log.flush() log.flush()
val offsets = server.apis.fetchOffsets(logManager, new TopicPartition(topic, part), OffsetRequest.EarliestTime, 10) val offsets = server.apis.fetchOffsets(logManager, new TopicPartition(topic, part), OffsetRequest.EarliestTime, 10)

View File

@ -54,11 +54,11 @@ class ProduceRequestTest extends BaseRequestTest {
} }
sendAndCheck(JTestUtils.partitionRecordsBuffer(0, CompressionType.NONE, sendAndCheck(JTestUtils.partitionRecordsBuffer(0, CompressionType.NONE,
new Record(System.currentTimeMillis(), "key".getBytes, "value".getBytes)), 0) Record.create(System.currentTimeMillis(), "key".getBytes, "value".getBytes)), 0)
sendAndCheck(JTestUtils.partitionRecordsBuffer(0, CompressionType.GZIP, sendAndCheck(JTestUtils.partitionRecordsBuffer(0, CompressionType.GZIP,
new Record(System.currentTimeMillis(), "key1".getBytes, "value1".getBytes), Record.create(System.currentTimeMillis(), "key1".getBytes, "value1".getBytes),
new Record(System.currentTimeMillis(), "key2".getBytes, "value2".getBytes)), 1) Record.create(System.currentTimeMillis(), "key2".getBytes, "value2".getBytes)), 1)
} }
/* returns a pair of partition id and leader id */ /* returns a pair of partition id and leader id */
@ -74,7 +74,7 @@ class ProduceRequestTest extends BaseRequestTest {
val (partition, leader) = createTopicAndFindPartitionWithLeader("topic") val (partition, leader) = createTopicAndFindPartitionWithLeader("topic")
val timestamp = 1000000 val timestamp = 1000000
val recordBuffer = JTestUtils.partitionRecordsBuffer(0, CompressionType.LZ4, val recordBuffer = JTestUtils.partitionRecordsBuffer(0, CompressionType.LZ4,
new Record(timestamp, "key".getBytes, "value".getBytes)) Record.create(timestamp, "key".getBytes, "value".getBytes))
// Change the lz4 checksum value so that it doesn't match the contents // Change the lz4 checksum value so that it doesn't match the contents
recordBuffer.array.update(40, 0) recordBuffer.array.update(40, 0)
val topicPartition = new TopicPartition("topic", partition) val topicPartition = new TopicPartition("topic", partition)

View File

@ -16,28 +16,29 @@
*/ */
package kafka.server package kafka.server
import java.util.Properties import java.util.Properties
import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.AtomicBoolean
import kafka.cluster.Replica import kafka.cluster.Replica
import kafka.common.TopicAndPartition import kafka.common.TopicAndPartition
import kafka.log.Log import kafka.log.Log
import kafka.message.{ByteBufferMessageSet, Message}
import kafka.utils._ import kafka.utils._
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.record.{MemoryRecords, Record}
import org.apache.kafka.common.requests.FetchRequest.PartitionData import org.apache.kafka.common.requests.FetchRequest.PartitionData
import org.easymock.EasyMock import org.easymock.EasyMock
import EasyMock._ import EasyMock._
import org.junit.Assert._ import org.junit.Assert._
import org.junit.{After, Test} import org.junit.{After, Test}
import scala.collection.JavaConverters._
class ReplicaManagerQuotasTest { class ReplicaManagerQuotasTest {
val configs = TestUtils.createBrokerConfigs(2, TestUtils.MockZkConnect).map(KafkaConfig.fromProps(_, new Properties())) val configs = TestUtils.createBrokerConfigs(2, TestUtils.MockZkConnect).map(KafkaConfig.fromProps(_, new Properties()))
val time = new MockTime val time = new MockTime
val metrics = new Metrics val metrics = new Metrics
val message = new Message("some-data-in-a-message".getBytes()) val record = Record.create("some-data-in-a-message".getBytes())
val topicAndPartition1 = TopicAndPartition("test-topic", 1) val topicAndPartition1 = TopicAndPartition("test-topic", 1)
val topicAndPartition2 = TopicAndPartition("test-topic", 2) val topicAndPartition2 = TopicAndPartition("test-topic", 2)
val fetchInfo = Seq(new TopicPartition(topicAndPartition1.topic, topicAndPartition1.partition) -> new PartitionData(0, 100), val fetchInfo = Seq(new TopicPartition(topicAndPartition1.topic, topicAndPartition1.partition) -> new PartitionData(0, 100),
@ -63,10 +64,10 @@ class ReplicaManagerQuotasTest {
readPartitionInfo = fetchInfo, readPartitionInfo = fetchInfo,
quota = quota) quota = quota)
assertEquals("Given two partitions, with only one throttled, we should get the first", 1, assertEquals("Given two partitions, with only one throttled, we should get the first", 1,
fetch.find(_._1 == topicAndPartition1).get._2.info.messageSet.size) fetch.find(_._1 == topicAndPartition1).get._2.info.records.shallowIterator.asScala.size)
assertEquals("But we shouldn't get the second", 0, assertEquals("But we shouldn't get the second", 0,
fetch.find(_._1 == topicAndPartition2).get._2.info.messageSet.size) fetch.find(_._1 == topicAndPartition2).get._2.info.records.shallowIterator.asScala.size)
} }
@Test @Test
@ -88,9 +89,9 @@ class ReplicaManagerQuotasTest {
readPartitionInfo = fetchInfo, readPartitionInfo = fetchInfo,
quota = quota) quota = quota)
assertEquals("Given two partitions, with both throttled, we should get no messages", 0, assertEquals("Given two partitions, with both throttled, we should get no messages", 0,
fetch.find(_._1 == topicAndPartition1).get._2.info.messageSet.size) fetch.find(_._1 == topicAndPartition1).get._2.info.records.shallowIterator.asScala.size)
assertEquals("Given two partitions, with both throttled, we should get no messages", 0, assertEquals("Given two partitions, with both throttled, we should get no messages", 0,
fetch.find(_._1 == topicAndPartition2).get._2.info.messageSet.size) fetch.find(_._1 == topicAndPartition2).get._2.info.records.shallowIterator.asScala.size)
} }
@Test @Test
@ -112,9 +113,9 @@ class ReplicaManagerQuotasTest {
readPartitionInfo = fetchInfo, readPartitionInfo = fetchInfo,
quota = quota) quota = quota)
assertEquals("Given two partitions, with both non-throttled, we should get both messages", 1, assertEquals("Given two partitions, with both non-throttled, we should get both messages", 1,
fetch.find(_._1 == topicAndPartition1).get._2.info.messageSet.size) fetch.find(_._1 == topicAndPartition1).get._2.info.records.shallowIterator.asScala.size)
assertEquals("Given two partitions, with both non-throttled, we should get both messages", 1, assertEquals("Given two partitions, with both non-throttled, we should get both messages", 1,
fetch.find(_._1 == topicAndPartition2).get._2.info.messageSet.size) fetch.find(_._1 == topicAndPartition2).get._2.info.records.shallowIterator.asScala.size)
} }
@Test @Test
@ -136,13 +137,13 @@ class ReplicaManagerQuotasTest {
readPartitionInfo = fetchInfo, readPartitionInfo = fetchInfo,
quota = quota) quota = quota)
assertEquals("Given two partitions, with only one throttled, we should get the first", 1, assertEquals("Given two partitions, with only one throttled, we should get the first", 1,
fetch.find(_._1 == topicAndPartition1).get._2.info.messageSet.size) fetch.find(_._1 == topicAndPartition1).get._2.info.records.shallowIterator.asScala.size)
assertEquals("But we should get the second too since it's throttled but in sync", 1, assertEquals("But we should get the second too since it's throttled but in sync", 1,
fetch.find(_._1 == topicAndPartition2).get._2.info.messageSet.size) fetch.find(_._1 == topicAndPartition2).get._2.info.records.shallowIterator.asScala.size)
} }
def setUpMocks(fetchInfo: Seq[(TopicPartition, PartitionData)], message: Message = this.message, bothReplicasInSync: Boolean = false) { def setUpMocks(fetchInfo: Seq[(TopicPartition, PartitionData)], record: Record = this.record, bothReplicasInSync: Boolean = false) {
val zkUtils = createNiceMock(classOf[ZkUtils]) val zkUtils = createNiceMock(classOf[ZkUtils])
val scheduler = createNiceMock(classOf[KafkaScheduler]) val scheduler = createNiceMock(classOf[KafkaScheduler])
@ -153,16 +154,16 @@ class ReplicaManagerQuotasTest {
//if we ask for len 1 return a message //if we ask for len 1 return a message
expect(log.read(anyObject(), geq(1), anyObject(), anyObject())).andReturn( expect(log.read(anyObject(), geq(1), anyObject(), anyObject())).andReturn(
new FetchDataInfo( FetchDataInfo(
new LogOffsetMetadata(0L, 0L, 0), new LogOffsetMetadata(0L, 0L, 0),
new ByteBufferMessageSet(message) MemoryRecords.withRecords(record)
)).anyTimes() )).anyTimes()
//if we ask for len = 0, return 0 messages //if we ask for len = 0, return 0 messages
expect(log.read(anyObject(), EasyMock.eq(0), anyObject(), anyObject())).andReturn( expect(log.read(anyObject(), EasyMock.eq(0), anyObject(), anyObject())).andReturn(
new FetchDataInfo( FetchDataInfo(
new LogOffsetMetadata(0L, 0L, 0), new LogOffsetMetadata(0L, 0L, 0),
new ByteBufferMessageSet() MemoryRecords.EMPTY
)).anyTimes() )).anyTimes()
replay(log) replay(log)

View File

@ -17,18 +17,16 @@
package kafka.server package kafka.server
import java.io.File import java.io.File
import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.AtomicBoolean
import kafka.api.FetchResponsePartitionData
import kafka.cluster.Broker import kafka.cluster.Broker
import kafka.common.TopicAndPartition import kafka.common.TopicAndPartition
import kafka.message.{ByteBufferMessageSet, Message, MessageSet}
import kafka.utils.{MockScheduler, MockTime, TestUtils, ZkUtils} import kafka.utils.{MockScheduler, MockTime, TestUtils, ZkUtils}
import org.I0Itec.zkclient.ZkClient import org.I0Itec.zkclient.ZkClient
import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.record.{MemoryRecords, Record, Records}
import org.apache.kafka.common.requests.{LeaderAndIsrRequest, PartitionState} import org.apache.kafka.common.requests.{LeaderAndIsrRequest, PartitionState}
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
import org.apache.kafka.common.requests.FetchRequest.PartitionData import org.apache.kafka.common.requests.FetchRequest.PartitionData
@ -105,11 +103,11 @@ class ReplicaManagerTest {
def callback(responseStatus: Map[TopicPartition, PartitionResponse]) = { def callback(responseStatus: Map[TopicPartition, PartitionResponse]) = {
assert(responseStatus.values.head.errorCode == Errors.INVALID_REQUIRED_ACKS.code) assert(responseStatus.values.head.errorCode == Errors.INVALID_REQUIRED_ACKS.code)
} }
rm.appendMessages( rm.appendRecords(
timeout = 0, timeout = 0,
requiredAcks = 3, requiredAcks = 3,
internalTopicsAllowed = false, internalTopicsAllowed = false,
messagesPerPartition = Map(new TopicPartition("test1", 0) -> new ByteBufferMessageSet(new Message("first message".getBytes))), entriesPerPartition = Map(new TopicPartition("test1", 0) -> MemoryRecords.withRecords(Record.create("first message".getBytes()))),
responseCallback = callback) responseCallback = callback)
} finally { } finally {
rm.shutdown(checkpointHW = false) rm.shutdown(checkpointHW = false)
@ -135,7 +133,7 @@ class ReplicaManagerTest {
} }
var fetchCallbackFired = false var fetchCallbackFired = false
def fetchCallback(responseStatus: Seq[(TopicAndPartition, FetchResponsePartitionData)]) = { def fetchCallback(responseStatus: Seq[(TopicAndPartition, FetchPartitionData)]) = {
assertEquals("Should give NotLeaderForPartitionException", Errors.NOT_LEADER_FOR_PARTITION.code, responseStatus.map(_._2).head.error) assertEquals("Should give NotLeaderForPartitionException", Errors.NOT_LEADER_FOR_PARTITION.code, responseStatus.map(_._2).head.error)
fetchCallbackFired = true fetchCallbackFired = true
} }
@ -158,11 +156,11 @@ class ReplicaManagerTest {
rm.getLeaderReplicaIfLocal(topic, 0) rm.getLeaderReplicaIfLocal(topic, 0)
// Append a message. // Append a message.
rm.appendMessages( rm.appendRecords(
timeout = 1000, timeout = 1000,
requiredAcks = -1, requiredAcks = -1,
internalTopicsAllowed = false, internalTopicsAllowed = false,
messagesPerPartition = Map(new TopicPartition(topic, 0) -> new ByteBufferMessageSet(new Message("first message".getBytes))), entriesPerPartition = Map(new TopicPartition(topic, 0) -> MemoryRecords.withRecords(Record.create("first message".getBytes()))),
responseCallback = produceCallback) responseCallback = produceCallback)
// Fetch some messages // Fetch some messages
@ -220,19 +218,19 @@ class ReplicaManagerTest {
// Append a couple of messages. // Append a couple of messages.
for(i <- 1 to 2) for(i <- 1 to 2)
rm.appendMessages( rm.appendRecords(
timeout = 1000, timeout = 1000,
requiredAcks = -1, requiredAcks = -1,
internalTopicsAllowed = false, internalTopicsAllowed = false,
messagesPerPartition = Map(new TopicPartition(topic, 0) -> new ByteBufferMessageSet(new Message("message %d".format(i).getBytes))), entriesPerPartition = Map(new TopicPartition(topic, 0) -> MemoryRecords.withRecords(Record.create("message %d".format(i).getBytes))),
responseCallback = produceCallback) responseCallback = produceCallback)
var fetchCallbackFired = false var fetchCallbackFired = false
var fetchError = 0 var fetchError = 0
var fetchedMessages: MessageSet = null var fetchedRecords: Records = null
def fetchCallback(responseStatus: Seq[(TopicAndPartition, FetchResponsePartitionData)]) = { def fetchCallback(responseStatus: Seq[(TopicAndPartition, FetchPartitionData)]) = {
fetchError = responseStatus.map(_._2).head.error fetchError = responseStatus.map(_._2).head.error
fetchedMessages = responseStatus.map(_._2).head.messages fetchedRecords = responseStatus.map(_._2).head.records
fetchCallbackFired = true fetchCallbackFired = true
} }
@ -249,7 +247,7 @@ class ReplicaManagerTest {
assertTrue(fetchCallbackFired) assertTrue(fetchCallbackFired)
assertEquals("Should not give an exception", Errors.NONE.code, fetchError) assertEquals("Should not give an exception", Errors.NONE.code, fetchError)
assertTrue("Should return some data", fetchedMessages.iterator.hasNext) assertTrue("Should return some data", fetchedRecords.shallowIterator.hasNext)
fetchCallbackFired = false fetchCallbackFired = false
// Fetch a message above the high watermark as a consumer // Fetch a message above the high watermark as a consumer
@ -264,7 +262,7 @@ class ReplicaManagerTest {
assertTrue(fetchCallbackFired) assertTrue(fetchCallbackFired)
assertEquals("Should not give an exception", Errors.NONE.code, fetchError) assertEquals("Should not give an exception", Errors.NONE.code, fetchError)
assertEquals("Should return empty response", MessageSet.Empty, fetchedMessages) assertEquals("Should return empty response", MemoryRecords.EMPTY, fetchedRecords)
} finally { } finally {
rm.shutdown(checkpointHW = false) rm.shutdown(checkpointHW = false)
} }

View File

@ -21,7 +21,6 @@ import kafka.utils._
import kafka.cluster.Replica import kafka.cluster.Replica
import kafka.common.TopicAndPartition import kafka.common.TopicAndPartition
import kafka.log.Log import kafka.log.Log
import kafka.message.{ByteBufferMessageSet, Message, MessageSet}
import kafka.server.QuotaFactory.UnboundedQuota import kafka.server.QuotaFactory.UnboundedQuota
import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.requests.FetchRequest.PartitionData import org.apache.kafka.common.requests.FetchRequest.PartitionData
@ -30,8 +29,10 @@ import java.util.Properties
import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.AtomicBoolean
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.record.{MemoryRecords, Record}
import org.easymock.EasyMock import org.easymock.EasyMock
import org.junit.Assert._ import org.junit.Assert._
import scala.collection.JavaConverters._
class SimpleFetchTest { class SimpleFetchTest {
@ -53,8 +54,8 @@ class SimpleFetchTest {
val partitionHW = 5 val partitionHW = 5
val fetchSize = 100 val fetchSize = 100
val messagesToHW = new Message("messageToHW".getBytes()) val messagesToHW = Record.create("messageToHW".getBytes())
val messagesToLEO = new Message("messageToLEO".getBytes()) val messagesToLEO = Record.create("messageToLEO".getBytes())
val topic = "test-topic" val topic = "test-topic"
val partitionId = 0 val partitionId = 0
@ -79,14 +80,14 @@ class SimpleFetchTest {
EasyMock.expect(log.logEndOffset).andReturn(leaderLEO).anyTimes() EasyMock.expect(log.logEndOffset).andReturn(leaderLEO).anyTimes()
EasyMock.expect(log.logEndOffsetMetadata).andReturn(new LogOffsetMetadata(leaderLEO)).anyTimes() EasyMock.expect(log.logEndOffsetMetadata).andReturn(new LogOffsetMetadata(leaderLEO)).anyTimes()
EasyMock.expect(log.read(0, fetchSize, Some(partitionHW), true)).andReturn( EasyMock.expect(log.read(0, fetchSize, Some(partitionHW), true)).andReturn(
new FetchDataInfo( FetchDataInfo(
new LogOffsetMetadata(0L, 0L, 0), new LogOffsetMetadata(0L, 0L, 0),
new ByteBufferMessageSet(messagesToHW) MemoryRecords.withRecords(messagesToHW)
)).anyTimes() )).anyTimes()
EasyMock.expect(log.read(0, fetchSize, None, true)).andReturn( EasyMock.expect(log.read(0, fetchSize, None, true)).andReturn(
new FetchDataInfo( FetchDataInfo(
new LogOffsetMetadata(0L, 0L, 0), new LogOffsetMetadata(0L, 0L, 0),
new ByteBufferMessageSet(messagesToLEO) MemoryRecords.withRecords(messagesToLEO)
)).anyTimes() )).anyTimes()
EasyMock.replay(log) EasyMock.replay(log)
@ -110,7 +111,7 @@ class SimpleFetchTest {
// create the follower replica with defined log end offset // create the follower replica with defined log end offset
val followerReplica= new Replica(configs(1).brokerId, partition, time) val followerReplica= new Replica(configs(1).brokerId, partition, time)
val leo = new LogOffsetMetadata(followerLEO, 0L, followerLEO.toInt) val leo = new LogOffsetMetadata(followerLEO, 0L, followerLEO.toInt)
followerReplica.updateLogReadResult(new LogReadResult(FetchDataInfo(leo, MessageSet.Empty), -1L, -1, true)) followerReplica.updateLogReadResult(new LogReadResult(FetchDataInfo(leo, MemoryRecords.EMPTY), -1L, -1, true))
// add both of them to ISR // add both of them to ISR
val allReplicas = List(leaderReplica, followerReplica) val allReplicas = List(leaderReplica, followerReplica)
@ -153,7 +154,7 @@ class SimpleFetchTest {
fetchMaxBytes = Int.MaxValue, fetchMaxBytes = Int.MaxValue,
hardMaxBytesLimit = false, hardMaxBytesLimit = false,
readPartitionInfo = fetchInfo, readPartitionInfo = fetchInfo,
quota = UnboundedQuota).find(_._1 == topicAndPartition).get._2.info.messageSet.head.message) quota = UnboundedQuota).find(_._1 == topicAndPartition).get._2.info.records.shallowIterator.next().record)
assertEquals("Reading any data can return messages up to the end of the log", messagesToLEO, assertEquals("Reading any data can return messages up to the end of the log", messagesToLEO,
replicaManager.readFromLocalLog( replicaManager.readFromLocalLog(
replicaId = Request.OrdinaryConsumerId, replicaId = Request.OrdinaryConsumerId,
@ -162,7 +163,7 @@ class SimpleFetchTest {
fetchMaxBytes = Int.MaxValue, fetchMaxBytes = Int.MaxValue,
hardMaxBytesLimit = false, hardMaxBytesLimit = false,
readPartitionInfo = fetchInfo, readPartitionInfo = fetchInfo,
quota = UnboundedQuota).find(_._1 == topicAndPartition).get._2.info.messageSet.head.message) quota = UnboundedQuota).find(_._1 == topicAndPartition).get._2.info.records.shallowIterator().next().record)
assertEquals("Counts should increment after fetch", initialTopicCount+2, BrokerTopicStats.getBrokerTopicStats(topic).totalFetchRequestRate.count()) assertEquals("Counts should increment after fetch", initialTopicCount+2, BrokerTopicStats.getBrokerTopicStats(topic).totalFetchRequestRate.count())
assertEquals("Counts should increment after fetch", initialAllTopicsCount+2, BrokerTopicStats.getBrokerAllTopicsStats().totalFetchRequestRate.count()) assertEquals("Counts should increment after fetch", initialAllTopicsCount+2, BrokerTopicStats.getBrokerAllTopicsStats().totalFetchRequestRate.count())

View File

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