KIP-101: Alter Replication Protocol to use Leader Epoch rather than High Watermark for Truncation

This PR replaces https://github.com/apache/kafka/pull/2743 (just raising from Confluent repo)

This PR describes the addition of Partition Level Leader Epochs to messages in Kafka as a mechanism for fixing some known issues in the replication protocol. Full details can be found here:

[KIP-101 Reference](https://cwiki.apache.org/confluence/display/KAFKA/KIP-101+-+Alter+Replication+Protocol+to+use+Leader+Epoch+rather+than+High+Watermark+for+Truncation)

*The key elements are*:
- Epochs are stamped on messages as they enter the leader.
- Epochs are tracked in both leader and follower in a new checkpoint file.
- A new API allows followers to retrieve the leader's latest offset for a particular epoch.
- The logic for truncating the log, when a replica becomes a follower, has been moved from Partition into the ReplicaFetcherThread
- When partitions are added to the ReplicaFetcherThread they are added in an initialising state. Initialising partitions request leader epochs and then truncate their logs appropriately.

This test provides a good overview of the workflow `EpochDrivenReplicationProtocolAcceptanceTest.shouldFollowLeaderEpochBasicWorkflow()`

The corrupted log use case is covered by the test
`EpochDrivenReplicationProtocolAcceptanceTest.offsetsShouldNotGoBackwards()`

Remaining work: There is a do list here: https://docs.google.com/document/d/1edmMo70MfHEZH9x38OQfTWsHr7UGTvg-NOxeFhOeRew/edit?usp=sharing

Author: Ben Stopford <benstopford@gmail.com>
Author: Jun Rao <junrao@gmail.com>

Reviewers: Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>

Closes #2808 from benstopford/kip-101-v2
This commit is contained in:
Ben Stopford 2017-04-06 14:51:09 -07:00 committed by Jun Rao
parent b611cfa5c0
commit 0baea2ac13
53 changed files with 3837 additions and 256 deletions

View File

@ -232,5 +232,7 @@
<suppress checks="NPathComplexity"
files="KafkaLog4jAppender.java"/>
<suppress checks="JavaNCSS"
files="RequestResponseTest.java"/>
</suppressions>

View File

@ -47,7 +47,8 @@ public enum ApiKeys {
CREATE_TOPICS(19, "CreateTopics"),
DELETE_TOPICS(20, "DeleteTopics"),
DELETE_RECORDS(21, "DeleteRecords"),
INIT_PRODUCER_ID(22, "InitProducerId");
INIT_PRODUCER_ID(22, "InitProducerId"),
OFFSET_FOR_LEADER_EPOCH(23, "OffsetForLeaderEpoch");
private static final ApiKeys[] ID_TO_TYPE;
private static final int MIN_API_KEY = 0;

View File

@ -1199,6 +1199,56 @@ public class Protocol {
public static final Schema[] INIT_PRODUCER_ID_RESPONSE = new Schema[] {INIT_PRODUCER_ID_RESPONSE_V0};
/* Offsets for Leader Epoch api */
public static final Schema OFFSET_FOR_LEADER_EPOCH_REQUEST_PARTITION_V0 = new Schema(
new Field("partition_id",
INT32,
"The partition_id"),
new Field("leader_epoch",
INT32,
"The epoch")
);
public static final Schema OFFSET_FOR_LEADER_EPOCH_REQUEST_TOPIC_V0 = new Schema(
new Field("topic",
STRING,
"The topic"),
new Field("partitions",
new ArrayOf(OFFSET_FOR_LEADER_EPOCH_REQUEST_PARTITION_V0),
"The partition")
);
public static final Schema OFFSET_FOR_LEADER_EPOCH_REQUEST_V0 = new Schema(
new Field("topics",
new ArrayOf(OFFSET_FOR_LEADER_EPOCH_REQUEST_TOPIC_V0),
"An array of topics to get epochs for"));
public static final Schema OFFSET_FOR_LEADER_EPOCH_RESPONSE_PARTITION_V0 = new Schema(
new Field("error_code",
INT16,
"The error code"),
new Field("partition_id",
INT32,
"The partition id"),
new Field("end_offset",
INT64,
"The end offset")
);
public static final Schema OFFSET_FOR_LEADER_EPOCH_RESPONSE_TOPIC_V0 = new Schema(
new Field("topic",
STRING,
"The topic"),
new Field("partitions",
new ArrayOf(OFFSET_FOR_LEADER_EPOCH_RESPONSE_PARTITION_V0),
"The partition")
);
public static final Schema OFFSET_FOR_LEADER_EPOCH_RESPONSE_V0 = new Schema(
new Field("topics",
new ArrayOf(OFFSET_FOR_LEADER_EPOCH_RESPONSE_TOPIC_V0),
"An array of topics for which we have leader offsets for some requested Partition Leader Epoch"));
public static final Schema[] OFFSET_FOR_LEADER_EPOCH_REQUEST = new Schema[] {OFFSET_FOR_LEADER_EPOCH_REQUEST_V0};
public static final Schema[] OFFSET_FOR_LEADER_EPOCH_RESPONSE = new Schema[] {OFFSET_FOR_LEADER_EPOCH_RESPONSE_V0};
/* an array of all requests and responses with all schema versions; a null value in the inner array means that the
* particular version is not supported */
public static final Schema[][] REQUESTS = new Schema[ApiKeys.MAX_API_KEY + 1][];
@ -1232,6 +1282,7 @@ public class Protocol {
REQUESTS[ApiKeys.DELETE_TOPICS.id] = DELETE_TOPICS_REQUEST;
REQUESTS[ApiKeys.DELETE_RECORDS.id] = DELETE_RECORDS_REQUEST;
REQUESTS[ApiKeys.INIT_PRODUCER_ID.id] = INIT_PRODUCER_ID_REQUEST;
REQUESTS[ApiKeys.OFFSET_FOR_LEADER_EPOCH.id] = OFFSET_FOR_LEADER_EPOCH_REQUEST;
RESPONSES[ApiKeys.PRODUCE.id] = PRODUCE_RESPONSE;
RESPONSES[ApiKeys.FETCH.id] = FETCH_RESPONSE;
@ -1256,6 +1307,7 @@ public class Protocol {
RESPONSES[ApiKeys.DELETE_TOPICS.id] = DELETE_TOPICS_RESPONSE;
RESPONSES[ApiKeys.DELETE_RECORDS.id] = DELETE_RECORDS_RESPONSE;
RESPONSES[ApiKeys.INIT_PRODUCER_ID.id] = INIT_PRODUCER_ID_RESPONSE;
RESPONSES[ApiKeys.OFFSET_FOR_LEADER_EPOCH.id] = OFFSET_FOR_LEADER_EPOCH_RESPONSE;
/* set the minimum and maximum version of each api */
for (ApiKeys api : ApiKeys.values()) {

View File

@ -303,7 +303,18 @@ public class MemoryRecords extends AbstractRecords {
long baseOffset,
long logAppendTime) {
return builder(buffer, magic, compressionType, timestampType, baseOffset, logAppendTime,
RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE);
RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, RecordBatch.UNKNOWN_PARTITION_LEADER_EPOCH);
}
public static MemoryRecordsBuilder builder(ByteBuffer buffer,
byte magic,
CompressionType compressionType,
TimestampType timestampType,
long baseOffset,
long logAppendTime,
int partitionLeaderEpoch) {
return builder(buffer, magic, compressionType, timestampType, baseOffset, logAppendTime,
RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, partitionLeaderEpoch);
}
public static MemoryRecordsBuilder builder(ByteBuffer buffer,
@ -315,8 +326,22 @@ public class MemoryRecords extends AbstractRecords {
long pid,
short epoch,
int baseSequence) {
return builder(buffer, magic, compressionType, timestampType, baseOffset, logAppendTime,
pid, epoch, baseSequence, RecordBatch.UNKNOWN_PARTITION_LEADER_EPOCH);
}
public static MemoryRecordsBuilder builder(ByteBuffer buffer,
byte magic,
CompressionType compressionType,
TimestampType timestampType,
long baseOffset,
long logAppendTime,
long pid,
short epoch,
int baseSequence,
int partitionLeaderEpoch) {
return new MemoryRecordsBuilder(buffer, magic, compressionType, timestampType, baseOffset,
logAppendTime, pid, epoch, baseSequence, false, RecordBatch.UNKNOWN_PARTITION_LEADER_EPOCH,
logAppendTime, pid, epoch, baseSequence, false, partitionLeaderEpoch,
buffer.remaining());
}

View File

@ -174,6 +174,9 @@ public abstract class AbstractRequest extends AbstractRequestResponse {
case INIT_PRODUCER_ID:
request = new InitPidRequest(struct, version);
break;
case OFFSET_FOR_LEADER_EPOCH:
request = new OffsetsForLeaderEpochRequest(struct, version);
break;
default:
throw new AssertionError(String.format("ApiKey %s is not currently handled in `getRequest`, the " +
"code should be updated to do so.", apiKey));

View File

@ -95,6 +95,8 @@ public abstract class AbstractResponse extends AbstractRequestResponse {
return new DeleteRecordsResponse(struct);
case INIT_PRODUCER_ID:
return new InitPidResponse(struct);
case OFFSET_FOR_LEADER_EPOCH:
return new OffsetsForLeaderEpochResponse(struct);
default:
throw new AssertionError(String.format("ApiKey %s is not currently handled in `getResponse`, the " +
"code should be updated to do so.", apiKey));

View File

@ -0,0 +1,81 @@
/*
* 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.requests;
import org.apache.kafka.common.protocol.Errors;
import static org.apache.kafka.common.record.RecordBatch.UNKNOWN_PARTITION_LEADER_EPOCH;
/**
* The offset, fetched from a leader, for a particular partition.
*/
public class EpochEndOffset {
public static final long UNDEFINED_EPOCH_OFFSET = UNKNOWN_PARTITION_LEADER_EPOCH;
public static final int UNDEFINED_EPOCH = -1;
private Errors error;
private long endOffset;
public EpochEndOffset(Errors error, long endOffset) {
this.error = error;
this.endOffset = endOffset;
}
public EpochEndOffset(long endOffset) {
this.error = Errors.NONE;
this.endOffset = endOffset;
}
public Errors error() {
return error;
}
public boolean hasError() {
return error != Errors.NONE;
}
public long endOffset() {
return endOffset;
}
@Override
public String toString() {
return "EpochEndOffset{" +
"error=" + error +
", endOffset=" + endOffset +
'}';
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
EpochEndOffset that = (EpochEndOffset) o;
if (error != that.error) return false;
return endOffset == that.endOffset;
}
@Override
public int hashCode() {
int result = (int) error.code();
result = 31 * result + (int) (endOffset ^ (endOffset >>> 32));
return result;
}
}

View File

@ -0,0 +1,175 @@
/*
* 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.requests;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
public class OffsetsForLeaderEpochRequest extends AbstractRequest {
public static final String TOPICS = "topics";
public static final String TOPIC = "topic";
public static final String PARTITIONS = "partitions";
public static final String PARTITION_ID = "partition_id";
public static final String LEADER_EPOCH = "leader_epoch";
private Map<TopicPartition, Integer> epochsByPartition;
public Map<TopicPartition, Integer> epochsByTopicPartition() {
return epochsByPartition;
}
public static class Builder extends AbstractRequest.Builder<OffsetsForLeaderEpochRequest> {
private Map<TopicPartition, Integer> epochsByPartition = new HashMap();
public Builder() {
super(ApiKeys.OFFSET_FOR_LEADER_EPOCH);
}
public Builder(Map<TopicPartition, Integer> epochsByPartition) {
super(ApiKeys.OFFSET_FOR_LEADER_EPOCH);
this.epochsByPartition = epochsByPartition;
}
public Builder add(TopicPartition topicPartition, Integer epoch) {
epochsByPartition.put(topicPartition, epoch);
return this;
}
@Override
public OffsetsForLeaderEpochRequest build(short version) {
return new OffsetsForLeaderEpochRequest(epochsByPartition, version);
}
public static OffsetsForLeaderEpochRequest parse(ByteBuffer buffer, short version) {
return new OffsetsForLeaderEpochRequest(ApiKeys.OFFSET_FOR_LEADER_EPOCH.parseRequest(version, buffer), version);
}
@Override
public String toString() {
StringBuilder bld = new StringBuilder();
bld.append("(type=OffsetForLeaderEpochRequest, ").
append("epochsByTopic=").append(epochsByPartition).
append(")");
return bld.toString();
}
}
public OffsetsForLeaderEpochRequest(Map<TopicPartition, Integer> epochsByPartition, short version) {
super(version);
this.epochsByPartition = epochsByPartition;
}
public OffsetsForLeaderEpochRequest(Struct struct, short version) {
super(version);
epochsByPartition = new HashMap<>();
for (Object t : struct.getArray(TOPICS)) {
Struct topicAndEpochs = (Struct) t;
String topic = topicAndEpochs.getString(TOPIC);
for (Object e : topicAndEpochs.getArray(PARTITIONS)) {
Struct partitionAndEpoch = (Struct) e;
int partitionId = partitionAndEpoch.getInt(PARTITION_ID);
int epoch = partitionAndEpoch.getInt(LEADER_EPOCH);
TopicPartition tp = new TopicPartition(topic, partitionId);
epochsByPartition.put(tp, epoch);
}
}
}
public static OffsetsForLeaderEpochRequest parse(ByteBuffer buffer, short versionId) {
return new OffsetsForLeaderEpochRequest(ApiKeys.OFFSET_FOR_LEADER_EPOCH.parseRequest(versionId, buffer), versionId);
}
@Override
protected Struct toStruct() {
Struct struct = new Struct(ApiKeys.OFFSET_FOR_LEADER_EPOCH.requestSchema(version()));
//Group by topic
Map<String, List<PartitionLeaderEpoch>> topicsToPartitionEpochs = new HashMap<>();
for (TopicPartition tp : epochsByPartition.keySet()) {
List<PartitionLeaderEpoch> partitionEndOffsets = topicsToPartitionEpochs.get(tp.topic());
if (partitionEndOffsets == null)
partitionEndOffsets = new ArrayList<>();
partitionEndOffsets.add(new PartitionLeaderEpoch(tp.partition(), epochsByPartition.get(tp)));
topicsToPartitionEpochs.put(tp.topic(), partitionEndOffsets);
}
List<Struct> topics = new ArrayList<>();
for (Map.Entry<String, List<PartitionLeaderEpoch>> topicEpochs : topicsToPartitionEpochs.entrySet()) {
Struct partition = struct.instance(TOPICS);
String topic = topicEpochs.getKey();
partition.set(TOPIC, topic);
List<PartitionLeaderEpoch> partitionLeaderEpoches = topicEpochs.getValue();
List<Struct> partitions = new ArrayList<>(partitionLeaderEpoches.size());
for (PartitionLeaderEpoch partitionLeaderEpoch : partitionLeaderEpoches) {
Struct partitionRow = partition.instance(PARTITIONS);
partitionRow.set(PARTITION_ID, partitionLeaderEpoch.partitionId);
partitionRow.set(LEADER_EPOCH, partitionLeaderEpoch.epoch);
partitions.add(partitionRow);
}
partition.set(PARTITIONS, partitions.toArray());
topics.add(partition);
}
struct.set(TOPICS, topics.toArray());
return struct;
}
@Override
public AbstractResponse getErrorResponse(Throwable e) {
Errors error = Errors.forException(e);
Map<TopicPartition, EpochEndOffset> errorResponse = new HashMap();
for (TopicPartition tp : epochsByPartition.keySet()) {
errorResponse.put(tp, new EpochEndOffset(error, EpochEndOffset.UNDEFINED_EPOCH_OFFSET));
}
return new OffsetsForLeaderEpochResponse(errorResponse);
}
private class PartitionLeaderEpoch {
int partitionId;
int epoch;
public PartitionLeaderEpoch(int partitionId, int epoch) {
this.partitionId = partitionId;
this.epoch = epoch;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
PartitionLeaderEpoch other = (PartitionLeaderEpoch) o;
if (partitionId != other.partitionId) return false;
return epoch == other.epoch;
}
@Override
public int hashCode() {
int result = partitionId;
result = 31 * result + epoch;
return result;
}
}
}

View File

@ -0,0 +1,132 @@
/*
* 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.requests;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
public class OffsetsForLeaderEpochResponse extends AbstractResponse {
public static final String TOPICS = "topics";
public static final String TOPIC = "topic";
public static final String PARTITIONS = "partitions";
public static final String ERROR_CODE = "error_code";
public static final String PARTITION_ID = "partition_id";
public static final String END_OFFSET = "end_offset";
private Map<TopicPartition, EpochEndOffset> epochEndOffsetsByPartition;
public OffsetsForLeaderEpochResponse(Struct struct) {
epochEndOffsetsByPartition = new HashMap<>();
for (Object t : struct.getArray(TOPICS)) {
Struct topicAndEpochs = (Struct) t;
String topic = topicAndEpochs.getString(TOPIC);
for (Object e : topicAndEpochs.getArray(PARTITIONS)) {
Struct partitionAndEpoch = (Struct) e;
Errors error = Errors.forCode(partitionAndEpoch.getShort(ERROR_CODE));
int partitionId = partitionAndEpoch.getInt(PARTITION_ID);
TopicPartition tp = new TopicPartition(topic, partitionId);
long endOffset = partitionAndEpoch.getLong(END_OFFSET);
epochEndOffsetsByPartition.put(tp, new EpochEndOffset(error, endOffset));
}
}
}
public OffsetsForLeaderEpochResponse(Map<TopicPartition, EpochEndOffset> epochsByTopic) {
this.epochEndOffsetsByPartition = epochsByTopic;
}
public Map<TopicPartition, EpochEndOffset> responses() {
return epochEndOffsetsByPartition;
}
public static OffsetsForLeaderEpochResponse parse(ByteBuffer buffer, short versionId) {
return new OffsetsForLeaderEpochResponse(ApiKeys.OFFSET_FOR_LEADER_EPOCH.responseSchema(versionId).read(buffer));
}
@Override
protected Struct toStruct(short version) {
Struct struct = new Struct(ApiKeys.OFFSET_FOR_LEADER_EPOCH.responseSchema(version));
//Group by topic
Map<String, List<PartitionEndOffset>> topicsToPartitionEndOffsets = new HashMap<>();
for (TopicPartition tp : epochEndOffsetsByPartition.keySet()) {
List<PartitionEndOffset> partitionEndOffsets = topicsToPartitionEndOffsets.get(tp.topic());
if (partitionEndOffsets == null)
partitionEndOffsets = new ArrayList<>();
partitionEndOffsets.add(new PartitionEndOffset(tp.partition(), epochEndOffsetsByPartition.get(tp)));
topicsToPartitionEndOffsets.put(tp.topic(), partitionEndOffsets);
}
//Write struct
List<Struct> topics = new ArrayList<>(topicsToPartitionEndOffsets.size());
for (Map.Entry<String, List<PartitionEndOffset>> topicEpochs : topicsToPartitionEndOffsets.entrySet()) {
Struct partition = struct.instance(TOPICS);
String topic = topicEpochs.getKey();
partition.set(TOPIC, topic);
List<PartitionEndOffset> paritionEpochs = topicEpochs.getValue();
List<Struct> paritions = new ArrayList<>(paritionEpochs.size());
for (PartitionEndOffset peo : paritionEpochs) {
Struct partitionRow = partition.instance(PARTITIONS);
partitionRow.set(ERROR_CODE, peo.epochEndOffset.error().code());
partitionRow.set(PARTITION_ID, peo.partition);
partitionRow.set(END_OFFSET, peo.epochEndOffset.endOffset());
paritions.add(partitionRow);
}
partition.set(PARTITIONS, paritions.toArray());
topics.add(partition);
}
struct.set(TOPICS, topics.toArray());
return struct;
}
private class PartitionEndOffset {
private int partition;
private EpochEndOffset epochEndOffset;
PartitionEndOffset(int partition, EpochEndOffset epochEndOffset) {
this.partition = partition;
this.epochEndOffset = epochEndOffset;
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
PartitionEndOffset that = (PartitionEndOffset) o;
if (partition != that.partition) return false;
return epochEndOffset != null ? epochEndOffset.equals(that.epochEndOffset) : that.epochEndOffset == null;
}
@Override
public int hashCode() {
int result = partition;
result = 31 * result + (epochEndOffset != null ? epochEndOffset.hashCode() : 0);
return result;
}
}
}

View File

@ -163,6 +163,9 @@ public class RequestResponseTest {
checkRequest(createListOffsetRequest(0));
checkErrorResponse(createListOffsetRequest(0), new UnknownServerException());
checkResponse(createListOffsetResponse(0), 0);
checkRequest(createLeaderEpochRequest());
checkResponse(createLeaderEpochResponse(), 0);
checkErrorResponse(createLeaderEpochRequest(), new UnknownServerException());
}
@Test
@ -798,6 +801,26 @@ public class RequestResponseTest {
return new InitPidResponse(Errors.NONE, 3332, (short) 3);
}
private OffsetsForLeaderEpochRequest createLeaderEpochRequest() {
Map<TopicPartition, Integer> epochs = new HashMap<>();
epochs.put(new TopicPartition("topic1", 0), 1);
epochs.put(new TopicPartition("topic1", 1), 1);
epochs.put(new TopicPartition("topic2", 2), 3);
return new OffsetsForLeaderEpochRequest.Builder(epochs).build();
}
private OffsetsForLeaderEpochResponse createLeaderEpochResponse() {
Map<TopicPartition, EpochEndOffset> epochs = new HashMap<>();
epochs.put(new TopicPartition("topic1", 0), new EpochEndOffset(Errors.NONE, 0));
epochs.put(new TopicPartition("topic1", 1), new EpochEndOffset(Errors.NONE, 1));
epochs.put(new TopicPartition("topic2", 2), new EpochEndOffset(Errors.NONE, 2));
return new OffsetsForLeaderEpochResponse(epochs);
}
private static class ByteBufferChannel implements GatheringByteChannel {
private final ByteBuffer buf;
private boolean closed = false;

View File

@ -25,14 +25,21 @@ import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
public class MockDeserializer implements ClusterResourceListener, Deserializer<byte[]> {
public static final AtomicInteger INIT_COUNT = new AtomicInteger(0);
public static final AtomicInteger CLOSE_COUNT = new AtomicInteger(0);
public static final AtomicReference<ClusterResource> CLUSTER_META = new AtomicReference<>();
public static final ClusterResource NO_CLUSTER_ID = new ClusterResource("no_cluster_id");
public static final AtomicReference<ClusterResource> CLUSTER_ID_BEFORE_DESERIALIZE = new AtomicReference<>(NO_CLUSTER_ID);
public static AtomicInteger initCount = new AtomicInteger(0);
public static AtomicInteger closeCount = new AtomicInteger(0);
public static AtomicReference<ClusterResource> clusterMeta = new AtomicReference<>();
public static ClusterResource noClusterId = new ClusterResource("no_cluster_id");
public static AtomicReference<ClusterResource> clusterIdBeforeDeserialize = new AtomicReference<>(noClusterId);
public static void resetStaticVariables() {
initCount = new AtomicInteger(0);
closeCount = new AtomicInteger(0);
clusterMeta = new AtomicReference<>();
clusterIdBeforeDeserialize = new AtomicReference<>(noClusterId);
}
public MockDeserializer() {
INIT_COUNT.incrementAndGet();
initCount.incrementAndGet();
}
@Override
@ -43,17 +50,17 @@ public class MockDeserializer implements ClusterResourceListener, Deserializer<b
public byte[] deserialize(String topic, byte[] data) {
// This will ensure that we get the cluster metadata when deserialize is called for the first time
// as subsequent compareAndSet operations will fail.
CLUSTER_ID_BEFORE_DESERIALIZE.compareAndSet(NO_CLUSTER_ID, CLUSTER_META.get());
clusterIdBeforeDeserialize.compareAndSet(noClusterId, clusterMeta.get());
return data;
}
@Override
public void close() {
CLOSE_COUNT.incrementAndGet();
closeCount.incrementAndGet();
}
@Override
public void onUpdate(ClusterResource clusterResource) {
CLUSTER_META.set(clusterResource);
clusterMeta.set(clusterResource);
}
}

View File

@ -67,7 +67,10 @@ object ApiVersion {
"0.11.0-IV0" -> KAFKA_0_11_0_IV0,
// introduced DeleteRecordsRequest v0 and FetchRequest v4 in KIP-107
"0.11.0-IV1" -> KAFKA_0_11_0_IV1,
"0.11.0" -> KAFKA_0_11_0_IV1
"0.11.0" -> KAFKA_0_11_0_IV1,
// Introduce leader epoch fetches to the replica fetcher via KIP-101
"0.11.0-IV2" -> KAFKA_0_11_0_IV2,
"0.11.0" -> KAFKA_0_11_0_IV2
)
private val versionPattern = "\\.".r
@ -159,7 +162,13 @@ case object KAFKA_0_11_0_IV0 extends ApiVersion {
}
case object KAFKA_0_11_0_IV1 extends ApiVersion {
val version: String = "0.11.0-IV0"
val version: String = "0.11.0-IV1"
val messageFormatVersion: Byte = RecordBatch.MAGIC_VALUE_V2
val id: Int = 10
val id: Int = 11
}
case object KAFKA_0_11_0_IV2 extends ApiVersion {
val version: String = "0.11.0-IV2"
val messageFormatVersion: Byte = RecordBatch.MAGIC_VALUE_V2
val id: Int = 12
}

View File

@ -163,12 +163,21 @@ class Partition(val topic: String,
// to maintain the decision maker controller's epoch in the zookeeper path
controllerEpoch = partitionStateInfo.controllerEpoch
// add replicas that are new
allReplicas.foreach(replica => getOrCreateReplica(replica))
val newInSyncReplicas = partitionStateInfo.isr.asScala.map(r => getOrCreateReplica(r)).toSet
// remove assigned replicas that have been removed by the controller
(assignedReplicas.map(_.brokerId) -- allReplicas).foreach(removeReplica)
inSyncReplicas = newInSyncReplicas
info(s"$topicPartition starts at Leader Epoch ${partitionStateInfo.leaderEpoch} from offset ${getReplica().get.logEndOffset.messageOffset}. Previous Leader Epoch was: $leaderEpoch")
//We cache the leader epoch here, persisting it only if it's local (hence having a log dir)
leaderEpoch = partitionStateInfo.leaderEpoch
allReplicas.map(id => getOrCreateReplica(id))
.filter(_.isLocal)
.foreach { replica =>
replica.epochs.get.cacheLatestEpoch(leaderEpoch)
}
zkVersion = partitionStateInfo.zkVersion
val isNewLeader =
if (leaderReplicaIdOpt.isDefined && leaderReplicaIdOpt.get == localBrokerId) {

View File

@ -22,8 +22,8 @@ import kafka.utils.Logging
import kafka.server.{LogOffsetMetadata, LogReadResult}
import kafka.common.KafkaException
import org.apache.kafka.common.errors.OffsetOutOfRangeException
import kafka.server.checkpoints.{LeaderEpochCheckpointFile, LeaderEpochFile}
import kafka.server.epoch.{LeaderEpochCache, LeaderEpochFileCache}
import org.apache.kafka.common.utils.Time
class Replica(val brokerId: Int,
@ -58,6 +58,8 @@ class Replica(val brokerId: Int,
def lastCaughtUpTimeMs = _lastCaughtUpTimeMs
val epochs = log.map(_.leaderEpochCache)
/*
* If the FetchRequest reads up to the log end offset of the leader when the current fetch request is received,
* set `lastCaughtUpTimeMs` to the time when the current fetch request was received.

View File

@ -28,6 +28,7 @@ import ConsumerFetcherThread._
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.record.MemoryRecords
import org.apache.kafka.common.requests.EpochEndOffset
class ConsumerFetcherThread(name: String,
val config: ConsumerConfig,
@ -38,7 +39,8 @@ class ConsumerFetcherThread(name: String,
clientId = config.clientId,
sourceBroker = sourceBroker,
fetchBackOffMs = config.refreshLeaderBackoffMs,
isInterruptible = true) {
isInterruptible = true,
includeLogTruncation = false) {
type REQ = FetchRequest
type PD = PartitionData
@ -100,7 +102,7 @@ class ConsumerFetcherThread(name: String,
protected def buildFetchRequest(partitionMap: collection.Seq[(TopicPartition, PartitionFetchState)]): FetchRequest = {
partitionMap.foreach { case ((topicPartition, partitionFetchState)) =>
if (partitionFetchState.isActive)
if (partitionFetchState.isReadyForFetch)
fetchRequestBuilder.addFetch(topicPartition.topic, topicPartition.partition, partitionFetchState.fetchOffset, fetchSize)
}
@ -111,6 +113,12 @@ class ConsumerFetcherThread(name: String,
simpleConsumer.fetch(fetchRequest.underlying).data.map { case (TopicAndPartition(t, p), value) =>
new TopicPartition(t, p) -> new PartitionData(value)
}
override def buildLeaderEpochRequest(allPartitions: Seq[(TopicPartition, PartitionFetchState)]): Map[TopicPartition, Int] = { Map() }
override def fetchEpochsFromLeader(partitions: Map[TopicPartition, Int]): Map[TopicPartition, EpochEndOffset] = { Map() }
override def maybeTruncate(fetchedEpochs: Map[TopicPartition, EpochEndOffset]): Map[TopicPartition, Long] = { Map() }
}
object ConsumerFetcherThread {

View File

@ -37,6 +37,12 @@ import org.apache.kafka.common.utils.{Time, Utils}
import scala.collection.JavaConverters._
import scala.collection.{Seq, mutable}
import com.yammer.metrics.core.Gauge
import org.apache.kafka.common.utils.{Time, Utils}
import kafka.message.{BrokerCompressionCodec, CompressionCodec, NoCompressionCodec}
import kafka.server.checkpoints.{LeaderEpochCheckpointFile, LeaderEpochFile}
import kafka.server.epoch.{LeaderEpochCache, LeaderEpochFileCache}
import org.apache.kafka.common.TopicPartition
object LogAppendInfo {
val UnknownLogAppendInfo = LogAppendInfo(-1, -1, RecordBatch.NO_TIMESTAMP, -1L, RecordBatch.NO_TIMESTAMP,
@ -136,15 +142,24 @@ class Log(@volatile var dir: File,
/* the actual segments of the log */
private val segments: ConcurrentNavigableMap[java.lang.Long, LogSegment] = new ConcurrentSkipListMap[java.lang.Long, LogSegment]
val leaderEpochCache: LeaderEpochCache = initializeLeaderEpochCache()
locally {
val startMs = time.milliseconds
loadSegments()
/* Calculate the offset of the next message */
nextOffsetMetadata = new LogOffsetMetadata(activeSegment.nextOffset(), activeSegment.baseOffset,
activeSegment.size.toInt)
leaderEpochCache.clearLatest(nextOffsetMetadata.messageOffset)
logStartOffset = math.max(logStartOffset, segments.firstEntry().getValue.baseOffset)
// The earliest leader epoch may not be flushed during a hard failure. Recover it here.
leaderEpochCache.clearEarliest(logStartOffset)
buildAndRecoverPidMap(logEndOffset)
info("Completed load of log %s with %d log segments, log start offset %d and log end offset %d in %d ms"
@ -193,10 +208,15 @@ class Log(@volatile var dir: File,
/** The name of this log */
def name = dir.getName()
/* Load the log segments from the log files on disk */
private def loadSegments() {
private def initializeLeaderEpochCache(): LeaderEpochCache = {
// create the log directory if it doesn't exist
dir.mkdirs()
new LeaderEpochFileCache(topicPartition, () => logEndOffsetMetadata,
new LeaderEpochCheckpointFile(LeaderEpochFile.newFile(dir)))
}
/* Load the log segments from the log files on disk */
private def loadSegments() {
var swapFiles = Set[File]()
// first do a pass through the files in the log directory and remove any temporary files
@ -341,7 +361,7 @@ class Log(@volatile var dir: File,
info("Recovering unflushed segment %d in log %s.".format(curr.baseOffset, name))
val truncatedBytes =
try {
curr.recover(config.maxMessageSize)
curr.recover(config.maxMessageSize, Some(leaderEpochCache))
} catch {
case _: InvalidOffsetException =>
val startOffset = curr.baseOffset
@ -352,7 +372,7 @@ class Log(@volatile var dir: File,
if(truncatedBytes > 0) {
// we had an invalid message, delete all remaining log
warn("Corruption found in segment %d of log %s, truncating to offset %d.".format(curr.baseOffset, name, curr.nextOffset))
unflushed.foreach(deleteSegment)
unflushed.foreach(deleteSegment(_))
}
}
}
@ -427,11 +447,11 @@ class Log(@volatile var dir: File,
*
* @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 leaderEpochCache Optional cache of Leader Epoch Offsets.
* @throws KafkaStorageException If the append fails due to an I/O error.
* @return Information about the appended messages including the first and last offset.
*/
def append(records: MemoryRecords, assignOffsets: Boolean = true): LogAppendInfo = {
def append(records: MemoryRecords, assignOffsets: Boolean = true, leaderEpochCache: LeaderEpochCache = leaderEpochCache): LogAppendInfo = {
val appendInfo = analyzeAndValidateRecords(records, isFromClient = assignOffsets)
// return if we have no valid messages or if this is a duplicate of the last appended entry
@ -451,6 +471,7 @@ class Log(@volatile var dir: File,
appendInfo.firstOffset = offset.value
val now = time.milliseconds
val validateAndOffsetAssignResult = try {
leaderEpochCache.maybeAssignLatestCachedEpochToLeo()
LogValidator.validateMessagesAndAssignOffsets(validRecords,
offset,
now,
@ -459,7 +480,8 @@ class Log(@volatile var dir: File,
config.compact,
config.messageFormatVersion.messageFormatVersion,
config.messageTimestampType,
config.messageTimestampDifferenceMaxMs)
config.messageTimestampDifferenceMaxMs,
leaderEpochCache.latestUsedEpoch())
} catch {
case e: IOException => throw new KafkaException("Error in validating messages while appending to log '%s'".format(name), e)
}
@ -485,6 +507,12 @@ class Log(@volatile var dir: File,
}
}
} else {
//Update the epoch cache with the epoch stamped by the leader
records.batches().asScala.map { batch =>
if (batch.magic >= RecordBatch.MAGIC_VALUE_V2)
leaderEpochCache.assign(batch.partitionLeaderEpoch, batch.baseOffset())
}
// we are taking the offsets we are given
if (!appendInfo.offsetsMonotonic || appendInfo.firstOffset < nextOffsetMetadata.messageOffset)
throw new IllegalArgumentException("Out of order offsets found in " + records.records.asScala.map(_.offset))
@ -808,9 +836,12 @@ class Log(@volatile var dir: File,
// we must always have at least one segment, so if we are going to delete all the segments, create a new one first
if (segments.size == numToDelete)
roll()
lock synchronized {
// remove the segments for lookups
deletable.foreach(deleteSegment)
deletable.foreach(deleteSegment(_))
logStartOffset = math.max(logStartOffset, segments.firstEntry().getValue.baseOffset)
leaderEpochCache.clearEarliest(logStartOffset)
}
}
numToDelete
}
@ -1017,6 +1048,7 @@ class Log(@volatile var dir: File,
lock synchronized {
logSegments.foreach(_.delete())
segments.clear()
leaderEpochCache.clear()
Utils.delete(dir)
}
}
@ -1027,23 +1059,24 @@ class Log(@volatile var dir: File,
* @param targetOffset The offset to truncate to, an upper bound on all offsets in the log after truncation is complete.
*/
private[log] def truncateTo(targetOffset: Long) {
info("Truncating log %s to offset %d.".format(name, targetOffset))
if(targetOffset < 0)
throw new IllegalArgumentException("Cannot truncate to a negative offset (%d).".format(targetOffset))
if(targetOffset > logEndOffset) {
if(targetOffset >= logEndOffset) {
info("Truncating %s to %d has no effect as the largest offset in the log is %d.".format(name, targetOffset, logEndOffset-1))
return
}
info("Truncating log %s to offset %d.".format(name, targetOffset))
lock synchronized {
if(segments.firstEntry.getValue.baseOffset > targetOffset) {
truncateFullyAndStartAt(targetOffset)
} else {
val deletable = logSegments.filter(segment => segment.baseOffset > targetOffset)
deletable.foreach(deleteSegment)
deletable.foreach(deleteSegment(_))
activeSegment.truncateTo(targetOffset)
updateLogEndOffset(targetOffset)
this.recoveryPoint = math.min(targetOffset, this.recoveryPoint)
this.logStartOffset = math.min(targetOffset, this.logStartOffset)
leaderEpochCache.clearLatest(targetOffset)
}
buildAndRecoverPidMap(targetOffset)
}
@ -1058,7 +1091,7 @@ class Log(@volatile var dir: File,
debug("Truncate and start log '" + name + "' to " + newOffset)
lock synchronized {
val segmentsToDelete = logSegments.toList
segmentsToDelete.foreach(deleteSegment)
segmentsToDelete.foreach(deleteSegment(_))
addSegment(new LogSegment(dir,
newOffset,
indexIntervalBytes = config.indexInterval,
@ -1069,6 +1102,7 @@ class Log(@volatile var dir: File,
initFileSize = initFileSize,
preallocate = config.preallocate))
updateLogEndOffset(newOffset)
leaderEpochCache.clear()
this.recoveryPoint = math.min(newOffset, this.recoveryPoint)
this.logStartOffset = newOffset
}

View File

@ -24,7 +24,7 @@ import java.util.concurrent.locks.ReentrantLock
import com.yammer.metrics.core.Gauge
import kafka.common.LogCleaningAbortedException
import kafka.metrics.KafkaMetricsGroup
import kafka.server.OffsetCheckpoint
import kafka.server.checkpoints.{OffsetCheckpoint, OffsetCheckpointFile}
import kafka.utils.CoreUtils._
import kafka.utils.{Logging, Pool}
import org.apache.kafka.common.TopicPartition
@ -55,7 +55,7 @@ private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[To
private[log] val offsetCheckpointFile = "cleaner-offset-checkpoint"
/* the offset checkpoints holding the last cleaned point for each log */
private val checkpoints = logDirs.map(dir => (dir, new OffsetCheckpoint(new File(dir, offsetCheckpointFile)))).toMap
private val checkpoints = logDirs.map(dir => (dir, new OffsetCheckpointFile(new File(dir, offsetCheckpointFile)))).toMap
/* the set of logs currently being cleaned */
private val inProgress = mutable.HashMap[TopicPartition, LogCleaningState]()

View File

@ -18,20 +18,19 @@
package kafka.log
import java.io._
import java.util.concurrent.{LinkedBlockingQueue, TimeUnit}
import kafka.utils._
import scala.collection._
import scala.collection.JavaConverters._
import kafka.common.{KafkaException, KafkaStorageException}
import kafka.server._
import java.util.concurrent.{ExecutionException, ExecutorService, Executors, Future}
import java.util.concurrent._
import kafka.admin.AdminUtils
import kafka.common.{KafkaException, KafkaStorageException}
import kafka.server.checkpoints.OffsetCheckpointFile
import kafka.server.{BrokerState, RecoveringFromUncleanShutdown, _}
import kafka.utils._
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.utils.Time
import scala.collection.JavaConverters._
import scala.collection._
/**
* The entry point to the kafka log management subsystem. The log manager is responsible for log creation, retrieval, and cleaning.
* All read and write operations are delegated to the individual log instances.
@ -67,8 +66,8 @@ class LogManager(val logDirs: Array[File],
createAndValidateLogDirs(logDirs)
private val dirLocks = lockLogDirs(logDirs)
private val recoveryPointCheckpoints = logDirs.map(dir => (dir, new OffsetCheckpoint(new File(dir, RecoveryPointCheckpointFile)))).toMap
private val logStartOffsetCheckpoints = logDirs.map(dir => (dir, new OffsetCheckpoint(new File(dir, LogStartOffsetCheckpointFile)))).toMap
private val recoveryPointCheckpoints = logDirs.map(dir => (dir, new OffsetCheckpointFile(new File(dir, RecoveryPointCheckpointFile)))).toMap
private val logStartOffsetCheckpoints = logDirs.map(dir => (dir, new OffsetCheckpointFile(new File(dir, LogStartOffsetCheckpointFile)))).toMap
loadLogs()
// public, so we can access this from kafka.admin.DeleteTopicTest

View File

@ -21,6 +21,7 @@ import java.util.concurrent.TimeUnit
import kafka.common._
import kafka.metrics.{KafkaMetricsGroup, KafkaTimer}
import kafka.server.epoch.LeaderEpochCache
import kafka.server.{FetchDataInfo, LogOffsetMetadata}
import kafka.utils._
import org.apache.kafka.common.errors.CorruptRecordException
@ -213,10 +214,11 @@ class LogSegment(val log: FileRecords,
*
* @param maxMessageSize A bound the memory allocation in the case of a corrupt message size--we will assume any message larger than this
* is corrupt.
* @param leaderEpochCache Optionally a cache for updating the leader epoch during recovery.
* @return The number of bytes truncated from the log
*/
@nonthreadsafe
def recover(maxMessageSize: Int): Int = {
def recover(maxMessageSize: Int, leaderEpochCache: Option[LeaderEpochCache] = None): Int = {
index.truncate()
index.resize(index.maxIndexSize)
timeIndex.truncate()
@ -242,6 +244,13 @@ class LogSegment(val log: FileRecords,
lastIndexEntry = validBytes
}
validBytes += batch.sizeInBytes()
if (batch.magic >= RecordBatch.MAGIC_VALUE_V2) {
leaderEpochCache.foreach { cache =>
if (batch.partitionLeaderEpoch > cache.latestUsedEpoch()) // this is to avoid unnecessary warning in cache.assign()
cache.assign(batch.partitionLeaderEpoch, batch.baseOffset())
}
}
}
} catch {
case e: CorruptRecordException =>

View File

@ -51,20 +51,21 @@ private[kafka] object LogValidator extends Logging {
compactedTopic: Boolean = false,
messageFormatVersion: Byte = RecordBatch.CURRENT_MAGIC_VALUE,
messageTimestampType: TimestampType,
messageTimestampDiffMaxMs: Long): ValidationAndOffsetAssignResult = {
messageTimestampDiffMaxMs: Long,
partitionLeaderEpoch: Int = RecordBatch.UNKNOWN_PARTITION_LEADER_EPOCH): ValidationAndOffsetAssignResult = {
if (sourceCodec == NoCompressionCodec && targetCodec == NoCompressionCodec) {
// check the magic value
if (!records.hasMatchingMagic(messageFormatVersion))
convertAndAssignOffsetsNonCompressed(records, offsetCounter, compactedTopic, now, messageTimestampType,
messageTimestampDiffMaxMs, messageFormatVersion)
messageTimestampDiffMaxMs, messageFormatVersion, partitionLeaderEpoch)
else
// Do in-place validation, offset assignment and maybe set timestamp
assignOffsetsNonCompressed(records, offsetCounter, now, compactedTopic, messageTimestampType,
messageTimestampDiffMaxMs)
messageTimestampDiffMaxMs, partitionLeaderEpoch)
} else {
validateMessagesAndAssignOffsetsCompressed(records, offsetCounter, now, sourceCodec, targetCodec, compactedTopic,
messageFormatVersion, messageTimestampType, messageTimestampDiffMaxMs)
messageFormatVersion, messageTimestampType, messageTimestampDiffMaxMs, partitionLeaderEpoch)
}
}
@ -74,7 +75,8 @@ private[kafka] object LogValidator extends Logging {
now: Long,
timestampType: TimestampType,
messageTimestampDiffMaxMs: Long,
toMagicValue: Byte): ValidationAndOffsetAssignResult = {
toMagicValue: Byte,
partitionLeaderEpoch: Int): ValidationAndOffsetAssignResult = {
val sizeInBytesAfterConversion = AbstractRecords.estimateSizeInBytes(toMagicValue, offsetCounter.value,
CompressionType.NONE, records.records)
@ -85,7 +87,7 @@ private[kafka] object LogValidator extends Logging {
val newBuffer = ByteBuffer.allocate(sizeInBytesAfterConversion)
val builder = MemoryRecords.builder(newBuffer, toMagicValue, CompressionType.NONE, timestampType,
offsetCounter.value, now, pid, epoch, sequence)
offsetCounter.value, now, pid, epoch, sequence, partitionLeaderEpoch)
for (batch <- records.batches.asScala) {
ensureNonTransactional(batch)
@ -112,7 +114,8 @@ private[kafka] object LogValidator extends Logging {
currentTimestamp: Long,
compactedTopic: Boolean,
timestampType: TimestampType,
timestampDiffMaxMs: Long): ValidationAndOffsetAssignResult = {
timestampDiffMaxMs: Long,
partitionLeaderEpoch: Int): ValidationAndOffsetAssignResult = {
var maxTimestamp = RecordBatch.NO_TIMESTAMP
var offsetOfMaxTimestamp = -1L
val initialOffset = offsetCounter.value
@ -138,6 +141,9 @@ private[kafka] object LogValidator extends Logging {
batch.setLastOffset(offsetCounter.value - 1)
if(batch.magic >= RecordBatch.MAGIC_VALUE_V2)
batch.setPartitionLeaderEpoch(partitionLeaderEpoch)
// TODO: in the compressed path, we ensure that the batch max timestamp is correct.
// We should either do the same or (better) let those two paths converge.
if (batch.magic > RecordBatch.MAGIC_VALUE_V0 && timestampType == TimestampType.LOG_APPEND_TIME)
@ -171,7 +177,8 @@ private[kafka] object LogValidator extends Logging {
compactedTopic: Boolean = false,
messageFormatVersion: Byte = RecordBatch.CURRENT_MAGIC_VALUE,
messageTimestampType: TimestampType,
messageTimestampDiffMaxMs: Long): ValidationAndOffsetAssignResult = {
messageTimestampDiffMaxMs: Long,
partitionLeaderEpoch: Int): ValidationAndOffsetAssignResult = {
// No in place assignment situation 1 and 2
var inPlaceAssignment = sourceCodec == targetCodec && messageFormatVersion > RecordBatch.MAGIC_VALUE_V0
@ -223,7 +230,7 @@ private[kafka] object LogValidator extends Logging {
(first.producerId, first.producerEpoch, first.baseSequence)
}
buildRecordsAndAssignOffsets(messageFormatVersion, offsetCounter, messageTimestampType,
CompressionType.forId(targetCodec.codec), currentTimestamp, validatedRecords, pid, epoch, sequence)
CompressionType.forId(targetCodec.codec), currentTimestamp, validatedRecords, pid, epoch, sequence, partitionLeaderEpoch)
} else {
// we can update the batch only and write the compressed payload as is
val batch = records.batches.iterator.next()
@ -237,6 +244,9 @@ private[kafka] object LogValidator extends Logging {
if (messageFormatVersion >= RecordBatch.MAGIC_VALUE_V1)
batch.setMaxTimestamp(messageTimestampType, maxTimestamp)
if(messageFormatVersion >= RecordBatch.MAGIC_VALUE_V2)
batch.setPartitionLeaderEpoch(partitionLeaderEpoch)
ValidationAndOffsetAssignResult(validatedRecords = records,
maxTimestamp = maxTimestamp,
shallowOffsetOfMaxTimestamp = lastOffset,
@ -247,11 +257,11 @@ private[kafka] object LogValidator extends Logging {
private def buildRecordsAndAssignOffsets(magic: Byte, offsetCounter: LongRef, timestampType: TimestampType,
compressionType: CompressionType, logAppendTime: Long,
validatedRecords: Seq[Record],
producerId: Long, epoch: Short, baseSequence: Int): ValidationAndOffsetAssignResult = {
producerId: Long, epoch: Short, baseSequence: Int, partitionLeaderEpoch: Int): ValidationAndOffsetAssignResult = {
val estimatedSize = AbstractRecords.estimateSizeInBytes(magic, offsetCounter.value, compressionType, validatedRecords.asJava)
val buffer = ByteBuffer.allocate(estimatedSize)
val builder = MemoryRecords.builder(buffer, magic, compressionType, timestampType, offsetCounter.value,
logAppendTime, producerId, epoch, baseSequence)
logAppendTime, producerId, epoch, baseSequence, partitionLeaderEpoch)
validatedRecords.foreach { record =>
builder.appendWithOffset(offsetCounter.getAndIncrement(), record)

View File

@ -38,6 +38,7 @@ import com.yammer.metrics.core.Gauge
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.internals.{FatalExitError, PartitionStates}
import org.apache.kafka.common.record.MemoryRecords
import org.apache.kafka.common.requests.EpochEndOffset
/**
* Abstract class for fetching data from multiple partitions from the same broker.
@ -46,13 +47,15 @@ abstract class AbstractFetcherThread(name: String,
clientId: String,
val sourceBroker: BrokerEndPoint,
fetchBackOffMs: Int = 0,
isInterruptible: Boolean = true)
isInterruptible: Boolean = true,
includeLogTruncation: Boolean
)
extends ShutdownableThread(name, isInterruptible) {
type REQ <: FetchRequest
type PD <: PartitionData
private val partitionStates = new PartitionStates[PartitionFetchState]
private[server] val partitionStates = new PartitionStates[PartitionFetchState]
private val partitionMapLock = new ReentrantLock
private val partitionMapCond = partitionMapLock.newCondition()
@ -71,6 +74,12 @@ abstract class AbstractFetcherThread(name: String,
// deal with partitions with errors, potentially due to leadership changes
protected def handlePartitionsWithErrors(partitions: Iterable[TopicPartition])
protected def buildLeaderEpochRequest(allPartitions: Seq[(TopicPartition, PartitionFetchState)]): Map[TopicPartition, Int]
protected def fetchEpochsFromLeader(partitions: Map[TopicPartition, Int]): Map[TopicPartition, EpochEndOffset]
protected def maybeTruncate(fetchedEpochs: Map[TopicPartition, EpochEndOffset]): Map[TopicPartition, Long]
protected def buildFetchRequest(partitionMap: Seq[(TopicPartition, PartitionFetchState)]): REQ
protected def fetch(fetchRequest: REQ): Seq[(TopicPartition, PD)]
@ -87,12 +96,12 @@ abstract class AbstractFetcherThread(name: String,
fetcherLagStats.unregister()
}
override def doWork() {
private def states() = partitionStates.partitionStates.asScala.map { state => state.topicPartition -> state.value }
override def doWork() {
maybeTruncate()
val fetchRequest = inLock(partitionMapLock) {
val fetchRequest = buildFetchRequest(partitionStates.partitionStates.asScala.map { state =>
state.topicPartition -> state.value
})
val fetchRequest = buildFetchRequest(states)
if (fetchRequest.isEmpty) {
trace("There are no active partitions. Back off for %d ms before sending a fetch request".format(fetchBackOffMs))
partitionMapCond.await(fetchBackOffMs, TimeUnit.MILLISECONDS)
@ -103,6 +112,30 @@ abstract class AbstractFetcherThread(name: String,
processFetchRequest(fetchRequest)
}
/**
* - Build a leader epoch fetch based on partitions that are in the Truncating phase
* - Issue LeaderEpochRequeust, retrieving the latest offset for each partition's
* leader epoch. This is the offset the follower should truncate to ensure
* accurate log replication.
* - Finally truncate the logs for partitions in the truncating phase and mark them
* truncation complete. Do this within a lock to ensure no leadership changes can
* occur during truncation.
*/
def maybeTruncate(): Unit = {
val epochRequests = inLock(partitionMapLock) { buildLeaderEpochRequest(states) }
if (!epochRequests.isEmpty) {
val fetchedEpochs = fetchEpochsFromLeader(epochRequests)
//Ensure we hold a lock during truncation.
inLock(partitionMapLock) {
//Check no leadership changes happened whilst we were unlocked, fetching epochs
val leaderEpochs = fetchedEpochs.filter { case (tp, _) => partitionStates.contains(tp) }
val truncationPoints = maybeTruncate(leaderEpochs)
markTruncationComplete(truncationPoints)
}
}
}
private def processFetchRequest(fetchRequest: REQ) {
val partitionsWithError = mutable.Set[TopicPartition]()
@ -208,25 +241,41 @@ abstract class AbstractFetcherThread(name: String,
!partitionStates.contains(tp)
}.map { case (tp, offset) =>
val fetchState =
if (PartitionTopicInfo.isOffsetInvalid(offset)) new PartitionFetchState(handleOffsetOutOfRange(tp))
else new PartitionFetchState(offset)
if (PartitionTopicInfo.isOffsetInvalid(offset))
new PartitionFetchState(handleOffsetOutOfRange(tp), includeLogTruncation)
else
new PartitionFetchState(offset, includeLogTruncation)
tp -> fetchState
}
val existingPartitionToState = partitionStates.partitionStates.asScala.map { state =>
state.topicPartition -> state.value
}.toMap
val existingPartitionToState = states().toMap
partitionStates.set((existingPartitionToState ++ newPartitionToState).asJava)
partitionMapCond.signalAll()
} finally partitionMapLock.unlock()
}
/**
* Loop through all partitions, marking them as truncation complete and applying the correct offset
* @param partitions the partitions to mark truncation complete
*/
private def markTruncationComplete(partitions: Map[TopicPartition, Long]) {
val newStates: Map[TopicPartition, PartitionFetchState] = partitionStates.partitionStates.asScala
.map { state =>
val maybeTruncationComplete = partitions.get(state.topicPartition()) match {
case Some(offset) => new PartitionFetchState(offset, state.value.delay, truncatingLog = false)
case None => state.value()
}
(state.topicPartition(), maybeTruncationComplete)
}.toMap
partitionStates.set(newStates.asJava)
}
def delayPartitions(partitions: Iterable[TopicPartition], delay: Long) {
partitionMapLock.lockInterruptibly()
try {
for (partition <- partitions) {
Option(partitionStates.stateValue(partition)).foreach (currentPartitionFetchState =>
if (currentPartitionFetchState.isActive)
partitionStates.updateAndMoveToEnd(partition, new PartitionFetchState(currentPartitionFetchState.fetchOffset, new DelayedItem(delay)))
if (!currentPartitionFetchState.isDelayed)
partitionStates.updateAndMoveToEnd(partition, new PartitionFetchState(currentPartitionFetchState.fetchOffset, new DelayedItem(delay), currentPartitionFetchState.truncatingLog))
)
}
partitionMapCond.signalAll()
@ -348,13 +397,25 @@ case class ClientIdTopicPartition(clientId: String, topic: String, partitionId:
}
/**
* case class to keep partition offset and its state(active, inactive)
* case class to keep partition offset and its state(truncatingLog, delayed)
* This represents a partition as being either:
* (1) Truncating its log, for example having recently become a follower
* (2) Delayed, for example due to an error, where we subsequently back off a bit
* (3) ReadyForFetch, the is the active state where the thread is actively fetching data.
*/
case class PartitionFetchState(fetchOffset: Long, delay: DelayedItem) {
case class PartitionFetchState(fetchOffset: Long, delay: DelayedItem, truncatingLog: Boolean = false) {
def this(offset: Long, truncatingLog: Boolean) = this(offset, new DelayedItem(0), truncatingLog)
def this(offset: Long, delay: DelayedItem) = this(offset, new DelayedItem(0), false)
def this(fetchOffset: Long) = this(fetchOffset, new DelayedItem(0))
def isActive: Boolean = delay.getDelay(TimeUnit.MILLISECONDS) == 0
def isReadyForFetch: Boolean = delay.getDelay(TimeUnit.MILLISECONDS) == 0 && !truncatingLog
override def toString = "%d-%b".format(fetchOffset, isActive)
def isTruncatingLog: Boolean = delay.getDelay(TimeUnit.MILLISECONDS) == 0 && truncatingLog
def isDelayed: Boolean = delay.getDelay(TimeUnit.MILLISECONDS) > 0
override def toString = "offset:%d-isReadyForFetch:%b-isTruncatingLog:%b".format(fetchOffset, isReadyForFetch, truncatingLog)
}

View File

@ -103,6 +103,7 @@ class KafkaApis(val requestChannel: RequestChannel,
case ApiKeys.DELETE_TOPICS => handleDeleteTopicsRequest(request)
case ApiKeys.DELETE_RECORDS => handleDeleteRecordsRequest(request)
case ApiKeys.INIT_PRODUCER_ID => handleInitPidRequest(request)
case ApiKeys.OFFSET_FOR_LEADER_EPOCH => handleOffsetForLeaderEpochRequest(request)
case requestId => throw new KafkaException("Unknown api code " + requestId)
}
} catch {
@ -1322,6 +1323,17 @@ class KafkaApis(val requestChannel: RequestChannel,
txnCoordinator.handleInitPid(initPidRequest.transactionalId, sendResponseCallback)
}
def handleOffsetForLeaderEpochRequest(request: RequestChannel.Request): Unit = {
val offsetForEpoch = request.body[OffsetsForLeaderEpochRequest]
val requestInfo = offsetForEpoch.epochsByTopicPartition()
authorizeClusterAction(request)
val responseBody = new OffsetsForLeaderEpochResponse(
replicaManager.getResponseFor(requestInfo)
)
requestChannel.sendResponse(new RequestChannel.Response(request, responseBody))
}
def authorizeClusterAction(request: RequestChannel.Request): Unit = {
if (!authorize(request.session, ClusterAction, Resource.ClusterResource))
throw new ClusterAuthorizationException(s"Request $request is not authorized.")

View File

@ -0,0 +1,105 @@
/**
* 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.server
import java.net.SocketTimeoutException
import kafka.cluster.BrokerEndPoint
import org.apache.kafka.clients._
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.network._
import org.apache.kafka.common.requests.AbstractRequest
import org.apache.kafka.common.security.JaasContext
import org.apache.kafka.common.utils.Time
import org.apache.kafka.clients.{ApiVersions, ClientResponse, ManualMetadataUpdater, NetworkClient}
import org.apache.kafka.common.Node
import org.apache.kafka.common.requests.AbstractRequest.Builder
import scala.collection.JavaConverters._
trait BlockingSend {
def sendRequest(requestBuilder: AbstractRequest.Builder[_ <: AbstractRequest]): ClientResponse
def close()
}
class ReplicaFetcherBlockingSend(sourceBroker: BrokerEndPoint,
brokerConfig: KafkaConfig,
metrics: Metrics,
time: Time,
fetcherId: Int,
clientId: String) extends BlockingSend {
private val sourceNode = new Node(sourceBroker.id, sourceBroker.host, sourceBroker.port)
private val socketTimeout: Int = brokerConfig.replicaSocketTimeoutMs
private val networkClient = {
val channelBuilder = ChannelBuilders.clientChannelBuilder(
brokerConfig.interBrokerSecurityProtocol,
JaasContext.Type.SERVER,
brokerConfig,
brokerConfig.interBrokerListenerName,
brokerConfig.saslMechanismInterBrokerProtocol,
brokerConfig.saslInterBrokerHandshakeRequestEnable
)
val selector = new Selector(
NetworkReceive.UNLIMITED,
brokerConfig.connectionsMaxIdleMs,
metrics,
time,
"replica-fetcher",
Map("broker-id" -> sourceBroker.id.toString, "fetcher-id" -> fetcherId.toString).asJava,
false,
channelBuilder
)
new NetworkClient(
selector,
new ManualMetadataUpdater(),
clientId,
1,
0,
Selectable.USE_DEFAULT_BUFFER_SIZE,
brokerConfig.replicaSocketReceiveBufferBytes,
brokerConfig.requestTimeoutMs,
time,
false,
new ApiVersions
)
}
override def sendRequest(requestBuilder: Builder[_ <: AbstractRequest]): ClientResponse = {
try {
if (!NetworkClientUtils.awaitReady(networkClient, sourceNode, time, socketTimeout))
throw new SocketTimeoutException(s"Failed to connect within $socketTimeout ms")
else {
val clientRequest = networkClient.newClientRequest(sourceBroker.id.toString, requestBuilder,
time.milliseconds(), true)
NetworkClientUtils.sendAndReceive(networkClient, clientRequest, time)
}
}
catch {
case e: Throwable =>
networkClient.close(sourceBroker.id.toString)
throw e
}
}
def close(): Unit = {
networkClient.close()
}
}

View File

@ -17,30 +17,27 @@
package kafka.server
import java.net.SocketTimeoutException
import java.util
import kafka.admin.AdminUtils
import kafka.cluster.BrokerEndPoint
import kafka.log.LogConfig
import kafka.api.{FetchRequest => _, _}
import kafka.cluster.{BrokerEndPoint, Replica}
import kafka.common.KafkaStorageException
import ReplicaFetcherThread._
import kafka.log.LogConfig
import kafka.server.ReplicaFetcherThread._
import kafka.server.epoch.LeaderEpochCache
import org.apache.kafka.common.requests.EpochEndOffset._
import kafka.utils.Exit
import org.apache.kafka.clients._
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.internals.FatalExitError
import org.apache.kafka.common.network.{ChannelBuilders, NetworkReceive, Selectable, Selector}
import org.apache.kafka.common.requests.{AbstractRequest, FetchResponse, ListOffsetRequest, ListOffsetResponse}
import org.apache.kafka.common.requests.{FetchRequest => JFetchRequest}
import org.apache.kafka.common.{Node, TopicPartition}
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.record.MemoryRecords
import org.apache.kafka.common.security.JaasContext
import org.apache.kafka.common.requests.{EpochEndOffset, FetchResponse, ListOffsetRequest, ListOffsetResponse, OffsetsForLeaderEpochRequest, OffsetsForLeaderEpochResponse, FetchRequest => JFetchRequest}
import org.apache.kafka.common.utils.Time
import scala.collection.Map
import scala.collection.JavaConverters._
import scala.collection.{Map, mutable}
class ReplicaFetcherThread(name: String,
fetcherId: Int,
@ -49,16 +46,20 @@ class ReplicaFetcherThread(name: String,
replicaMgr: ReplicaManager,
metrics: Metrics,
time: Time,
quota: ReplicationQuotaManager)
quota: ReplicationQuotaManager,
leaderEndpointBlockingSend: Option[BlockingSend] = None)
extends AbstractFetcherThread(name = name,
clientId = name,
sourceBroker = sourceBroker,
fetchBackOffMs = brokerConfig.replicaFetchBackoffMs,
isInterruptible = false) {
isInterruptible = false,
includeLogTruncation = true) {
type REQ = FetchRequest
type PD = PartitionData
private val leaderEndpoint = leaderEndpointBlockingSend.getOrElse(
new ReplicaFetcherBlockingSend(sourceBroker, brokerConfig, metrics, time, fetcherId, s"broker-${brokerConfig.brokerId}-fetcher-$fetcherId"))
private val fetchRequestVersion: Short =
if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_11_0_IV1) 5
else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_11_0_IV0) 4
@ -66,57 +67,18 @@ class ReplicaFetcherThread(name: String,
else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_10_0_IV0) 2
else if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_9_0) 1
else 0
private val socketTimeout: Int = brokerConfig.replicaSocketTimeoutMs
private val replicaId = brokerConfig.brokerId
private val maxWait = brokerConfig.replicaFetchWaitMaxMs
private val minBytes = brokerConfig.replicaFetchMinBytes
private val maxBytes = brokerConfig.replicaFetchResponseMaxBytes
private val fetchSize = brokerConfig.replicaFetchMaxBytes
private val shouldSendLeaderEpochRequest: Boolean = brokerConfig.interBrokerProtocolVersion >= KAFKA_0_11_0_IV2
private def clientId = name
private val sourceNode = new Node(sourceBroker.id, sourceBroker.host, sourceBroker.port)
// we need to include both the broker id and the fetcher id
// as the metrics tag to avoid metric name conflicts with
// more than one fetcher thread to the same broker
private val networkClient = {
val channelBuilder = ChannelBuilders.clientChannelBuilder(
brokerConfig.interBrokerSecurityProtocol,
JaasContext.Type.SERVER,
brokerConfig,
brokerConfig.interBrokerListenerName,
brokerConfig.saslMechanismInterBrokerProtocol,
brokerConfig.saslInterBrokerHandshakeRequestEnable
)
val selector = new Selector(
NetworkReceive.UNLIMITED,
brokerConfig.connectionsMaxIdleMs,
metrics,
time,
"replica-fetcher",
Map("broker-id" -> sourceBroker.id.toString, "fetcher-id" -> fetcherId.toString).asJava,
false,
channelBuilder
)
new NetworkClient(
selector,
new ManualMetadataUpdater(),
clientId,
1,
0,
Selectable.USE_DEFAULT_BUFFER_SIZE,
brokerConfig.replicaSocketReceiveBufferBytes,
brokerConfig.requestTimeoutMs,
time,
false,
new ApiVersions
)
}
private def epochCache(tp: TopicPartition): LeaderEpochCache = replicaMgr.getReplica(tp).get.epochs.get
override def shutdown(): Unit = {
super.shutdown()
networkClient.close()
leaderEndpoint.close()
}
// process fetched data
@ -132,7 +94,10 @@ class ReplicaFetcherThread(name: String,
if (logger.isTraceEnabled)
trace("Follower %d has replica log end offset %d for partition %s. Received %d messages and leader hw %d"
.format(replica.brokerId, replica.logEndOffset.messageOffset, topicPartition, records.sizeInBytes, partitionData.highWatermark))
// Append the leader's messages to the log
replica.log.get.append(records, assignOffsets = false)
if (logger.isTraceEnabled)
trace("Follower %d has replica log end offset %d after appending %d bytes of messages for partition %s"
.format(replica.brokerId, replica.logEndOffset.messageOffset, records.sizeInBytes, topicPartition))
@ -179,8 +144,7 @@ class ReplicaFetcherThread(name: String,
*
* There is a potential for a mismatch between the logs of the two replicas here. We don't fix this mismatch as of now.
*/
val leaderEndOffset: Long = earliestOrLatestOffset(topicPartition, ListOffsetRequest.LATEST_TIMESTAMP,
brokerConfig.brokerId)
val leaderEndOffset: Long = earliestOrLatestOffset(topicPartition, ListOffsetRequest.LATEST_TIMESTAMP)
if (leaderEndOffset < replica.logEndOffset.messageOffset) {
// Prior to truncating the follower's log, ensure that doing so is not disallowed by the configuration for unclean leader election.
@ -222,8 +186,7 @@ class ReplicaFetcherThread(name: String,
* and the current leader's log start offset.
*
*/
val leaderStartOffset: Long = earliestOrLatestOffset(topicPartition, ListOffsetRequest.EARLIEST_TIMESTAMP,
brokerConfig.brokerId)
val leaderStartOffset: Long = earliestOrLatestOffset(topicPartition, ListOffsetRequest.EARLIEST_TIMESTAMP)
warn("Replica %d for partition %s reset its fetch offset from %d to current leader %d's start offset %d"
.format(brokerConfig.brokerId, topicPartition, replica.logEndOffset.messageOffset, sourceBroker.id, leaderStartOffset))
val offsetToFetch = Math.max(leaderStartOffset, replica.logEndOffset.messageOffset)
@ -240,32 +203,14 @@ class ReplicaFetcherThread(name: String,
}
protected def fetch(fetchRequest: FetchRequest): Seq[(TopicPartition, PartitionData)] = {
val clientResponse = sendRequest(fetchRequest.underlying)
val clientResponse = leaderEndpoint.sendRequest(fetchRequest.underlying)
val fetchResponse = clientResponse.responseBody.asInstanceOf[FetchResponse]
fetchResponse.responseData.asScala.toSeq.map { case (key, value) =>
key -> new PartitionData(value)
}
}
private def sendRequest(requestBuilder: AbstractRequest.Builder[_ <: AbstractRequest]): ClientResponse = {
try {
if (!NetworkClientUtils.awaitReady(networkClient, sourceNode, time, socketTimeout))
throw new SocketTimeoutException(s"Failed to connect within $socketTimeout ms")
else {
val clientRequest = networkClient.newClientRequest(sourceBroker.id.toString, requestBuilder,
time.milliseconds(), true)
NetworkClientUtils.sendAndReceive(networkClient, clientRequest, time)
}
}
catch {
case e: Throwable =>
networkClient.close(sourceBroker.id.toString)
throw e
}
}
private def earliestOrLatestOffset(topicPartition: TopicPartition, earliestOrLatest: Long, replicaId: Int): Long = {
private def earliestOrLatestOffset(topicPartition: TopicPartition, earliestOrLatest: Long): Long = {
val requestBuilder = if (brokerConfig.interBrokerProtocolVersion >= KAFKA_0_10_1_IV2) {
val partitions = Map(topicPartition -> (earliestOrLatest: java.lang.Long))
ListOffsetRequest.Builder.forReplica(1, replicaId).setTargetTimes(partitions.asJava)
@ -273,7 +218,7 @@ class ReplicaFetcherThread(name: String,
val partitions = Map(topicPartition -> new ListOffsetRequest.PartitionData(earliestOrLatest, 1))
ListOffsetRequest.Builder.forReplica(0, replicaId).setOffsetData(partitions.asJava)
}
val clientResponse = sendRequest(requestBuilder)
val clientResponse = leaderEndpoint.sendRequest(requestBuilder)
val response = clientResponse.responseBody.asInstanceOf[ListOffsetResponse]
val partitionData = response.responseData.get(topicPartition)
partitionData.error match {
@ -286,12 +231,12 @@ class ReplicaFetcherThread(name: String,
}
}
protected def buildFetchRequest(partitionMap: Seq[(TopicPartition, PartitionFetchState)]): FetchRequest = {
override def buildFetchRequest(partitionMap: Seq[(TopicPartition, PartitionFetchState)]): FetchRequest = {
val requestMap = new util.LinkedHashMap[TopicPartition, JFetchRequest.PartitionData]
partitionMap.foreach { case (topicPartition, partitionFetchState) =>
// We will not include a replica in the fetch request if it should be throttled.
if (partitionFetchState.isActive && !shouldFollowerThrottle(quota, topicPartition)) {
if (partitionFetchState.isReadyForFetch && !shouldFollowerThrottle(quota, topicPartition)) {
val logStartOffset = replicaMgr.getReplicaOrException(topicPartition).logStartOffset
requestMap.put(topicPartition, new JFetchRequest.PartitionData(partitionFetchState.fetchOffset, logStartOffset, fetchSize))
}
@ -302,6 +247,91 @@ class ReplicaFetcherThread(name: String,
new FetchRequest(requestBuilder)
}
/**
* - Truncate the log to the leader's offset for each partition's epoch.
* - If the leader's offset is greater, we stick with the Log End Offset
* otherwise we truncate to the leaders offset.
* - If the leader replied with undefined epoch offset we must use the high watermark
*/
override def maybeTruncate(fetchedEpochs: Map[TopicPartition, EpochEndOffset]): Map[TopicPartition, Long] = {
val truncationPoints = scala.collection.mutable.HashMap.empty[TopicPartition, Long]
val partitionsWithError = mutable.Set[TopicPartition]()
fetchedEpochs.foreach { case (tp, epochOffset) =>
val replica = replicaMgr.getReplica(tp).get
if (epochOffset.hasError) {
info(s"Retrying leaderEpoch request for partition ${replica.topicPartition} as the leader reported an error: ${epochOffset.error}")
partitionsWithError += tp
} else {
val truncationOffset =
if (epochOffset.endOffset() == UNDEFINED_EPOCH_OFFSET)
highWatermark(replica, epochOffset)
else if (epochOffset.endOffset() >= replica.logEndOffset.messageOffset)
logEndOffset(replica, epochOffset)
else
epochOffset.endOffset
truncationPoints.put(tp, truncationOffset)
}
}
replicaMgr.logManager.truncateTo(truncationPoints)
// For partitions that encountered an error, delay them a bit before retrying the leader epoch request
delayPartitions(partitionsWithError, brokerConfig.replicaFetchBackoffMs.toLong)
truncationPoints
}
override def buildLeaderEpochRequest(allPartitions: Seq[(TopicPartition, PartitionFetchState)]): Map[TopicPartition, Int] = {
val result = allPartitions
.filter { case (_, state) => state.isTruncatingLog }
.map { case (tp, _) => tp -> epochCache(tp).latestUsedEpoch }.toMap
debug(s"Build leaderEpoch request $result for broker $sourceBroker")
result
}
override def fetchEpochsFromLeader(partitions: Map[TopicPartition, Int]): Map[TopicPartition, EpochEndOffset] = {
var result: Map[TopicPartition, EpochEndOffset] = null
if (shouldSendLeaderEpochRequest) {
val partitionsAsJava = partitions.map { case (tp, epoch) => tp -> epoch.asInstanceOf[Integer] }.toMap.asJava
val epochRequest = new OffsetsForLeaderEpochRequest.Builder(partitionsAsJava)
try {
val response = leaderEndpoint.sendRequest(epochRequest)
result = response.responseBody.asInstanceOf[OffsetsForLeaderEpochResponse].responses.asScala
debug(s"Receive leaderEpoch response $result from broker $sourceBroker")
} catch {
case t: Throwable =>
warn(s"Error when sending leader epoch request for $partitions", t)
// if we get any unexpected exception, mark all partitions with an error
result = partitions.map { case (tp, _) =>
tp -> new EpochEndOffset(Errors.forException(t), UNDEFINED_EPOCH_OFFSET)
}
}
} else {
// just generate a response with no error but UNDEFINED_OFFSET so that we can fall back to truncating using
// high watermark in maybeTruncate()
result = partitions.map { case (tp, _) =>
tp -> new EpochEndOffset(Errors.NONE, UNDEFINED_EPOCH_OFFSET)
}
}
result
}
private def logEndOffset(replica: Replica, epochOffset: EpochEndOffset): Long = {
val logEndOffset = replica.logEndOffset.messageOffset
info(s"Based on follower's leader epoch, leader replied with an offset ${epochOffset.endOffset()} >= the follower's log end offset $logEndOffset in ${replica.topicPartition}. No truncation needed.")
logEndOffset
}
private def highWatermark(replica: Replica, epochOffset: EpochEndOffset): Long = {
warn(s"Based on follower's leader epoch, leader replied with an unknown offset in ${replica.topicPartition}. High watermark ${replica.highWatermark.messageOffset} will be used for truncation.")
replica.highWatermark.messageOffset
}
/**
* To avoid ISR thrashing, we only throttle a replica on the follower if it's in the throttled replica list,
* the quota is exceeded and the replica is not in sync.
@ -336,7 +366,5 @@ object ReplicaFetcherThread {
case Errors.NONE => None
case e => Some(e.exception)
}
}
}

View File

@ -20,28 +20,34 @@ import java.io.{File, IOException}
import java.util.concurrent.TimeUnit
import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong}
import org.apache.kafka.common.errors._
import com.yammer.metrics.core.Gauge
import kafka.api._
import kafka.cluster.{Partition, Replica}
import kafka.common._
import kafka.controller.KafkaController
import kafka.log.{Log, LogAppendInfo, LogManager}
import kafka.metrics.KafkaMetricsGroup
import kafka.server.QuotaFactory.UnboundedQuota
import kafka.server.checkpoints.OffsetCheckpointFile
import kafka.utils._
import org.apache.kafka.common.errors.{ControllerMovedException, CorruptRecordException, InvalidTimestampException, InvalidTopicException, PolicyViolationException}
import org.apache.kafka.common.errors.{NotLeaderForPartitionException, OffsetOutOfRangeException, RecordBatchTooLargeException, RecordTooLargeException, ReplicaNotAvailableException, UnknownTopicOrPartitionException}
import org.apache.kafka.common.errors.{ControllerMovedException, CorruptRecordException, InvalidTimestampException, InvalidTopicException, NotEnoughReplicasException, NotLeaderForPartitionException, OffsetOutOfRangeException, PolicyViolationException}
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.record._
import org.apache.kafka.common.requests.{LeaderAndIsrRequest, PartitionState, StopReplicaRequest, UpdateMetadataRequest, DeleteRecordsRequest, DeleteRecordsResponse}
import org.apache.kafka.common.requests.{DeleteRecordsRequest, DeleteRecordsResponse, LeaderAndIsrRequest, PartitionState, StopReplicaRequest, UpdateMetadataRequest}
import org.apache.kafka.common.requests._
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
import org.apache.kafka.common.utils.Time
import org.apache.kafka.common.requests.FetchRequest.PartitionData
import scala.collection._
import scala.collection.JavaConverters._
import java.util.{Map => JMap}
import kafka.common.{KafkaStorageException, Topic}
import org.apache.kafka.common.protocol.Errors._
import org.apache.kafka.common.requests.EpochEndOffset._
/*
* Result metadata of a log append operation on the log
@ -132,7 +138,7 @@ class ReplicaManager(val config: KafkaConfig,
private val replicaStateChangeLock = new Object
val replicaFetcherManager = createReplicaFetcherManager(metrics, time, threadNamePrefix, quotaManager)
private val highWatermarkCheckPointThreadStarted = new AtomicBoolean(false)
val highWatermarkCheckpoints = config.logDirs.map(dir => (new File(dir).getAbsolutePath, new OffsetCheckpoint(new File(dir, ReplicaManager.HighWatermarkFilename)))).toMap
val highWatermarkCheckpoints = config.logDirs.map(dir => (new File(dir).getAbsolutePath, new OffsetCheckpointFile(new File(dir, ReplicaManager.HighWatermarkFilename)))).toMap
private var hwThreadInitialized = false
this.logIdent = "[Replica Manager on Broker " + localBrokerId + "]: "
val stateChangeLogger = KafkaController.stateChangeLogger
@ -316,9 +322,11 @@ class ReplicaManager(val config: KafkaConfig,
}
}
def getReplica(topicPartition: TopicPartition, replicaId: Int = localBrokerId): Option[Replica] =
def getReplica(topicPartition: TopicPartition, replicaId: Int): Option[Replica] =
getPartition(topicPartition).flatMap(_.getReplica(replicaId))
def getReplica(tp: TopicPartition): Option[Replica] = getReplica(tp, localBrokerId)
/**
* 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
@ -974,9 +982,6 @@ class ReplicaManager(val config: KafkaConfig,
.format(localBrokerId, controllerId, epoch, correlationId, partition.topicPartition))
}
logManager.truncateTo(partitionsToMakeFollower.map { partition =>
(partition.topicPartition, partition.getOrCreateReplica().highWatermark.messageOffset)
}.toMap)
partitionsToMakeFollower.foreach { partition =>
val topicPartitionOperationKey = new TopicPartitionOperationKey(partition.topicPartition)
tryCompleteDelayedProduce(topicPartitionOperationKey)
@ -1089,4 +1094,22 @@ class ReplicaManager(val config: KafkaConfig,
new ReplicaFetcherManager(config, this, metrics, time, threadNamePrefix, quotaManager)
}
def getResponseFor(requestedEpochInfo: JMap[TopicPartition, Integer]): JMap[TopicPartition, EpochEndOffset] = {
OffsetsForLeaderEpoch.getResponseFor(this, requestedEpochInfo)
}
}
object OffsetsForLeaderEpoch extends Logging {
def getResponseFor(replicaManager: ReplicaManager, requestedEpochInfo: JMap[TopicPartition, Integer]): JMap[TopicPartition, EpochEndOffset] = {
debug(s"Processing OffsetForEpochRequest: $requestedEpochInfo")
requestedEpochInfo.asScala.map { case (tp, epoch) =>
val offset = try {
new EpochEndOffset(NONE, replicaManager.getLeaderReplicaIfLocal(tp).epochs.get.endOffsetFor(epoch))
} catch {
case e: NotLeaderForPartitionException => new EpochEndOffset(NOT_LEADER_FOR_PARTITION, UNDEFINED_EPOCH_OFFSET)
case e: UnknownTopicOrPartitionException => new EpochEndOffset(UNKNOWN_TOPIC_OR_PARTITION, UNDEFINED_EPOCH_OFFSET)
}
(tp, offset)
}.toMap.asJava
}
}

View File

@ -14,50 +14,41 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.server
package kafka.server.checkpoints
import java.nio.file.{FileSystems, Paths}
import java.util.regex.Pattern
import org.apache.kafka.common.utils.Utils
import scala.collection._
import kafka.utils.{Exit, Logging}
import kafka.common._
import java.io._
import java.nio.charset.StandardCharsets
import java.nio.file.{FileSystems, Paths}
import kafka.utils.{Exit, Logging}
import org.apache.kafka.common.utils.Utils
import scala.collection.{Seq, mutable}
import org.apache.kafka.common.TopicPartition
trait CheckpointFileFormatter[T]{
def toLine(entry: T): String
object OffsetCheckpoint {
private val WhiteSpacesPattern = Pattern.compile("\\s+")
private val CurrentVersion = 0
def fromLine(line: String): Option[T]
}
/**
* This class saves out a map of topic/partition=>offsets to a file
*/
class OffsetCheckpoint(val file: File) extends Logging {
import OffsetCheckpoint._
class CheckpointFile[T](val file: File, version: Int, formatter: CheckpointFileFormatter[T]) extends Logging {
private val path = file.toPath.toAbsolutePath
private val tempPath = Paths.get(path.toString + ".tmp")
private val lock = new Object()
file.createNewFile() // in case the file doesn't exist
file.createNewFile()
def write(offsets: Map[TopicPartition, Long]) {
def write(entries: Seq[T]) {
lock synchronized {
// write to temp file and then swap with the existing file
val fileOutputStream = new FileOutputStream(tempPath.toFile)
val writer = new BufferedWriter(new OutputStreamWriter(fileOutputStream, StandardCharsets.UTF_8))
try {
writer.write(CurrentVersion.toString)
writer.write(version.toString)
writer.newLine()
writer.write(offsets.size.toString)
writer.write(entries.size.toString)
writer.newLine()
offsets.foreach { case (topicPart, offset) =>
writer.write(s"${topicPart.topic} ${topicPart.partition} $offset")
entries.foreach { entry =>
writer.write(formatter.toLine(entry))
writer.newLine()
}
@ -78,8 +69,7 @@ class OffsetCheckpoint(val file: File) extends Logging {
}
}
def read(): Map[TopicPartition, Long] = {
def read(): Seq[T] = {
def malformedLineException(line: String) =
new IOException(s"Malformed line in offset checkpoint file: $line'")
@ -89,27 +79,28 @@ class OffsetCheckpoint(val file: File) extends Logging {
try {
line = reader.readLine()
if (line == null)
return Map.empty
return Seq.empty
val version = line.toInt
version match {
case CurrentVersion =>
case version =>
line = reader.readLine()
if (line == null)
return Map.empty
return Seq.empty
val expectedSize = line.toInt
val offsets = mutable.Map[TopicPartition, Long]()
val entries = mutable.Buffer[T]()
line = reader.readLine()
while (line != null) {
WhiteSpacesPattern.split(line) match {
case Array(topic, partition, offset) =>
offsets += new TopicPartition(topic, partition.toInt) -> offset.toLong
val entry = formatter.fromLine(line)
entry match {
case Some(e) =>
entries += e
line = reader.readLine()
case _ => throw malformedLineException(line)
}
}
if (offsets.size != expectedSize)
throw new IOException(s"Expected $expectedSize entries but found only ${offsets.size}")
offsets
if (entries.size != expectedSize)
throw new IOException(s"Expected $expectedSize entries but found only ${entries.size}")
entries
case _ =>
throw new IOException("Unrecognized version of the highwatermark checkpoint file: " + version)
}
@ -120,5 +111,4 @@ class OffsetCheckpoint(val file: File) extends Logging {
}
}
}
}

View File

@ -0,0 +1,67 @@
/**
* 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.server.checkpoints
import java.io._
import java.util.regex.Pattern
import kafka.server.checkpoints.LeaderEpochCheckpointConstants.{CurrentVersion, WhiteSpacesPattern}
import kafka.server.epoch.EpochEntry
import scala.collection._
trait LeaderEpochCheckpoint {
def write(epochs: Seq[EpochEntry])
def read(): Seq[EpochEntry]
}
object LeaderEpochFile {
private val LeaderEpochCheckpointFilename = "leader-epoch-checkpoint"
def newFile(dir: File) = {new File(dir, LeaderEpochCheckpointFilename)}
}
private object LeaderEpochCheckpointConstants {
val WhiteSpacesPattern = Pattern.compile("\\s+")
val CurrentVersion = 0
}
/**
* This class persists a map of (LeaderEpoch => Offsets) to a file (for a certain replica)
*/
class LeaderEpochCheckpointFile(val file: File) extends CheckpointFileFormatter[EpochEntry] with LeaderEpochCheckpoint {
val checkpoint = new CheckpointFile[EpochEntry](file, CurrentVersion, this)
override def toLine(entry: EpochEntry): String = {
s"${entry.epoch} ${entry.startOffset}"
}
override def fromLine(line: String): Option[EpochEntry] = {
WhiteSpacesPattern.split(line) match {
case Array(epoch, offset) =>
Some(EpochEntry(epoch.toInt, offset.toLong))
case _ => None
}
}
def write(epochs: Seq[EpochEntry]) = {
checkpoint.write(epochs)
}
def read(): Seq[EpochEntry] = {
checkpoint.read()
}
}

View File

@ -0,0 +1,60 @@
/**
* 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.server.checkpoints
import java.io._
import java.util.regex.Pattern
import kafka.server.epoch.EpochEntry
import org.apache.kafka.common.TopicPartition
import scala.collection._
private object OffsetCheckpointConstants {
val WhiteSpacesPattern = Pattern.compile("\\s+")
val CurrentVersion = 0
}
trait OffsetCheckpoint {
def write(epochs: Seq[EpochEntry])
def read(): Seq[EpochEntry]
}
/**
* This class persists a map of (Partition => Offsets) to a file (for a certain replica)
*/
class OffsetCheckpointFile(val f: File) extends CheckpointFileFormatter[(TopicPartition, Long)] {
val checkpoint = new CheckpointFile[(TopicPartition, Long)](f, OffsetCheckpointConstants.CurrentVersion, this)
override def toLine(entry: (TopicPartition, Long)): String = {
s"${entry._1.topic} ${entry._1.partition} ${entry._2}"
}
override def fromLine(line: String): Option[(TopicPartition, Long)] = {
OffsetCheckpointConstants.WhiteSpacesPattern.split(line) match {
case Array(topic, partition, offset) =>
Some(new TopicPartition(topic, partition.toInt), offset.toLong)
case _ => None
}
}
def write(offsets: Map[TopicPartition, Long]) = {
checkpoint.write(offsets.toSeq)
}
def read(): Map[TopicPartition, Long] = {
checkpoint.read().toMap
}
}

View File

@ -0,0 +1,224 @@
/**
* 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.server.epoch
import java.util.concurrent.locks.ReentrantReadWriteLock
import kafka.server.LogOffsetMetadata
import kafka.server.checkpoints.LeaderEpochCheckpoint
import org.apache.kafka.common.requests.EpochEndOffset.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET}
import kafka.utils.CoreUtils._
import kafka.utils.Logging
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.requests.EpochEndOffset
import scala.collection.mutable.ListBuffer
trait LeaderEpochCache {
def cacheLatestEpoch(leaderEpoch: Int)
def maybeAssignLatestCachedEpochToLeo()
def assign(leaderEpoch: Int, offset: Long)
def latestUsedEpoch(): Int
def endOffsetFor(epoch: Int): Long
def clearLatest(offset: Long)
def clearEarliest(offset: Long)
def clear()
}
/**
* Represents a cache of (LeaderEpoch => Offset) mappings for a particular replica.
*
* Leader Epoch = epoch assigned to each leader by the controller.
* Offset = offset of the first message in each epoch.
*
* @param leo a function that determines the log end offset
* @param checkpoint the checkpoint file
*/
class LeaderEpochFileCache(topicPartition: TopicPartition, leo: () => LogOffsetMetadata, checkpoint: LeaderEpochCheckpoint) extends LeaderEpochCache with Logging {
private val lock = new ReentrantReadWriteLock()
private var epochs: ListBuffer[EpochEntry] = lock synchronized { ListBuffer(checkpoint.read(): _*) }
private var cachedLatestEpoch: Option[Int] = None //epoch which has yet to be assigned to a message.
/**
* Assigns the supplied Leader Epoch to the supplied Offset
* Once the epoch is assigned it cannot be reassigned
*
* @param epoch
* @param offset
*/
override def assign(epoch: Int, offset: Long): Unit = {
inWriteLock(lock) {
if (epoch >= 0 && epoch > latestUsedEpoch && offset >= latestOffset) {
info(s"Updated PartitionLeaderEpoch. ${epochChangeMsg(epoch, offset)}. Cache now contains ${epochs.size} entries.")
epochs += EpochEntry(epoch, offset)
flush()
} else {
maybeWarn(epoch, offset)
}
}
}
/**
* Returns the current Leader Epoch. This is the latest epoch
* which has messages assigned to it.
*
* @return
*/
override def latestUsedEpoch(): Int = {
inReadLock(lock) {
if (epochs.isEmpty) UNDEFINED_EPOCH else epochs.last.epoch
}
}
/**
* Returns the End Offset for a requested Leader Epoch.
*
* This is defined as the start offset of the first Leader Epoch larger than the
* Leader Epoch requested, or else the Log End Offset if the latest epoch was requested.
*
* @param requestedEpoch
* @return offset
*/
override def endOffsetFor(requestedEpoch: Int): Long = {
inReadLock(lock) {
val offset =
if (requestedEpoch == latestUsedEpoch) {
leo().messageOffset
}
else {
val subsequentEpochs = epochs.filter(e => e.epoch > requestedEpoch)
if (subsequentEpochs.isEmpty || requestedEpoch < epochs.head.epoch)
UNDEFINED_EPOCH_OFFSET
else
subsequentEpochs.head.startOffset
}
debug(s"Processed offset for epoch request for partition ${topicPartition} epoch:$requestedEpoch and returning offset $offset from epoch list of size ${epochs.size}")
offset
}
}
/**
* Removes all epoch entries from the store with start offsets greater than or equal to the passed offset.
*
* @param offset
*/
override def clearLatest(offset: Long): Unit = {
inWriteLock(lock) {
val before = epochs
if (offset >= 0 && offset <= latestOffset()) {
epochs = epochs.filter(entry => entry.startOffset < offset)
flush()
info(s"Cleared latest ${before.toSet.filterNot(epochs.toSet)} entries from epoch cache based on passed offset $offset leaving ${epochs.size} in EpochFile for partition $topicPartition")
}
}
}
/**
* Clears old epoch entries. This method searches for the oldest epoch < offset, updates the saved epoch offset to
* be offset, then clears any previous epoch entries.
*
* This method is exclusive: so clearEarliest(6) will retain an entry at offset 6.
*
* @param offset the offset to clear up to
*/
override def clearEarliest(offset: Long): Unit = {
inWriteLock(lock) {
val before = epochs
if (offset >= 0 && earliestOffset() < offset) {
val earliest = epochs.filter(entry => entry.startOffset < offset)
if (earliest.size > 0) {
epochs = epochs --= earliest
//If the offset is less than the earliest offset remaining, add previous epoch back, but with an updated offset
if (offset < earliestOffset() || epochs.isEmpty)
new EpochEntry(earliest.last.epoch, offset) +=: epochs
flush()
info(s"Cleared earliest ${before.toSet.filterNot(epochs.toSet).size} entries from epoch cache based on passed offset $offset leaving ${epochs.size} in EpochFile for partition $topicPartition")
}
}
}
}
/**
* Delete all entries.
*/
override def clear() = {
inWriteLock(lock) {
epochs.clear()
flush()
}
}
def epochEntries(): ListBuffer[EpochEntry] = {
epochs
}
private def earliestOffset(): Long = {
if (epochs.isEmpty) -1 else epochs.head.startOffset
}
private def latestOffset(): Long = {
if (epochs.isEmpty) -1 else epochs.last.startOffset
}
private def flush(): Unit = {
checkpoint.write(epochs)
}
def epochChangeMsg(epoch: Int, offset: Long) = s"New: {epoch:$epoch, offset:$offset}, Latest: {epoch:$latestUsedEpoch, offset$latestOffset} for Partition: $topicPartition"
def maybeWarn(epoch: Int, offset: Long) = {
if (epoch < latestUsedEpoch())
warn(s"Received a PartitionLeaderEpoch assignment for an epoch < latestEpoch. " +
s"This implies messages have arrived out of order. ${epochChangeMsg(epoch, offset)}")
else if (epoch < 0)
warn(s"Received an PartitionLeaderEpoch assignment for an epoch < 0. This should not happen. ${epochChangeMsg(epoch, offset)}")
else if (offset < latestOffset() && epoch >= 0)
warn(s"Received an PartitionLeaderEpoch assignment for an offset < latest offset for the most recent, stored PartitionLeaderEpoch. " +
s"This implies messages have arrived out of order. ${epochChangeMsg(epoch, offset)}")
}
/**
* Registers a PartitionLeaderEpoch (typically in response to a leadership change).
* This will be cached until {@link #maybeAssignLatestCachedEpochToLeo} is called.
*
* This allows us to register an epoch in response to a leadership change, but not persist
* that epoch until a message arrives and is stamped. This asigns the aassignment of leadership
* on leader and follower, for eases debugability.
*
* @param epoch
*/
override def cacheLatestEpoch(epoch: Int) = {
inWriteLock(lock) {
cachedLatestEpoch = Some(epoch)
}
}
/**
* If there is a cached epoch, associate its start offset with the current log end offset if it's not in the epoch list yet.
*/
override def maybeAssignLatestCachedEpochToLeo() = {
inWriteLock(lock) {
if (cachedLatestEpoch == None) error("Attempt to assign log end offset to epoch before epoch has been set. This should never happen.")
cachedLatestEpoch.foreach { epoch =>
assign(epoch, leo().messageOffset)
}
}
}
}
// Mapping of epoch to the first offset of the subsequent epoch
case class EpochEntry(epoch: Int, startOffset: Long)

View File

@ -102,7 +102,8 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
ApiKeys.STOP_REPLICA -> classOf[requests.StopReplicaResponse],
ApiKeys.CONTROLLED_SHUTDOWN_KEY -> classOf[requests.ControlledShutdownResponse],
ApiKeys.CREATE_TOPICS -> classOf[CreateTopicsResponse],
ApiKeys.DELETE_TOPICS -> classOf[requests.DeleteTopicsResponse]
ApiKeys.DELETE_TOPICS -> classOf[requests.DeleteTopicsResponse],
ApiKeys.OFFSET_FOR_LEADER_EPOCH -> classOf[OffsetsForLeaderEpochResponse]
)
val RequestKeyToError = Map[ApiKeys, (Nothing) => Errors](
@ -122,7 +123,8 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
ApiKeys.STOP_REPLICA -> ((resp: requests.StopReplicaResponse) => resp.responses().asScala.find(_._1 == tp).get._2),
ApiKeys.CONTROLLED_SHUTDOWN_KEY -> ((resp: requests.ControlledShutdownResponse) => resp.error),
ApiKeys.CREATE_TOPICS -> ((resp: CreateTopicsResponse) => resp.errors().asScala.find(_._1 == createTopic).get._2.error),
ApiKeys.DELETE_TOPICS -> ((resp: requests.DeleteTopicsResponse) => resp.errors().asScala.find(_._1 == deleteTopic).get._2)
ApiKeys.DELETE_TOPICS -> ((resp: requests.DeleteTopicsResponse) => resp.errors().asScala.find(_._1 == deleteTopic).get._2),
ApiKeys.OFFSET_FOR_LEADER_EPOCH -> ((resp: OffsetsForLeaderEpochResponse) => resp.responses.asScala.get(tp).get.error())
)
val RequestKeysToAcls = Map[ApiKeys, Map[Resource, Set[Acl]]](
@ -142,7 +144,8 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
ApiKeys.STOP_REPLICA -> ClusterAcl,
ApiKeys.CONTROLLED_SHUTDOWN_KEY -> ClusterAcl,
ApiKeys.CREATE_TOPICS -> ClusterCreateAcl,
ApiKeys.DELETE_TOPICS -> TopicDeleteAcl
ApiKeys.DELETE_TOPICS -> TopicDeleteAcl,
ApiKeys.OFFSET_FOR_LEADER_EPOCH -> ClusterAcl
)
@Before
@ -201,6 +204,10 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
build()
}
private def offsetsForLeaderEpochRequest = {
new OffsetsForLeaderEpochRequest.Builder().add(tp, 7).build()
}
private def createOffsetFetchRequest = {
new requests.OffsetFetchRequest.Builder(group, List(tp).asJava).build()
}
@ -285,7 +292,8 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
ApiKeys.STOP_REPLICA -> createStopReplicaRequest,
ApiKeys.CONTROLLED_SHUTDOWN_KEY -> createControlledShutdownRequest,
ApiKeys.CREATE_TOPICS -> createTopicsRequest,
ApiKeys.DELETE_TOPICS -> deleteTopicsRequest
ApiKeys.DELETE_TOPICS -> deleteTopicsRequest,
ApiKeys.OFFSET_FOR_LEADER_EPOCH -> offsetsForLeaderEpochRequest
)
for ((key, request) <- requestKeyToRequest) {

View File

@ -112,6 +112,7 @@ class EndToEndClusterIdTest extends KafkaServerTestHarness {
@Before
override def setUp() {
super.setUp
MockDeserializer.resetStaticVariables
// create the consumer offset topic
TestUtils.createTopic(this.zkUtils, topic, 2, serverCount, this.servers)
}
@ -163,10 +164,10 @@ class EndToEndClusterIdTest extends KafkaServerTestHarness {
isValidClusterId(MockConsumerInterceptor.CLUSTER_META.get.clusterId)
assertEquals(MockConsumerInterceptor.CLUSTER_ID_BEFORE_ON_CONSUME.get.clusterId, MockConsumerInterceptor.CLUSTER_META.get.clusterId)
assertNotEquals(MockDeserializer.CLUSTER_ID_BEFORE_DESERIALIZE, MockDeserializer.NO_CLUSTER_ID)
assertNotNull(MockDeserializer.CLUSTER_META)
isValidClusterId(MockDeserializer.CLUSTER_META.get.clusterId)
assertEquals(MockDeserializer.CLUSTER_ID_BEFORE_DESERIALIZE.get.clusterId, MockDeserializer.CLUSTER_META.get.clusterId)
assertNotEquals(MockDeserializer.clusterIdBeforeDeserialize, MockDeserializer.noClusterId)
assertNotNull(MockDeserializer.clusterMeta)
isValidClusterId(MockDeserializer.clusterMeta.get.clusterId)
assertEquals(MockDeserializer.clusterIdBeforeDeserialize.get.clusterId, MockDeserializer.clusterMeta.get.clusterId)
assertNotNull(MockConsumerMetricsReporter.CLUSTER_META)
isValidClusterId(MockConsumerMetricsReporter.CLUSTER_META.get.clusterId)
@ -175,7 +176,7 @@ class EndToEndClusterIdTest extends KafkaServerTestHarness {
assertEquals(MockProducerInterceptor.CLUSTER_META.get.clusterId, MockSerializer.CLUSTER_META.get.clusterId)
assertEquals(MockProducerInterceptor.CLUSTER_META.get.clusterId, MockProducerMetricsReporter.CLUSTER_META.get.clusterId)
assertEquals(MockProducerInterceptor.CLUSTER_META.get.clusterId, MockConsumerInterceptor.CLUSTER_META.get.clusterId)
assertEquals(MockProducerInterceptor.CLUSTER_META.get.clusterId, MockDeserializer.CLUSTER_META.get.clusterId)
assertEquals(MockProducerInterceptor.CLUSTER_META.get.clusterId, MockDeserializer.clusterMeta.get.clusterId)
assertEquals(MockProducerInterceptor.CLUSTER_META.get.clusterId, MockConsumerMetricsReporter.CLUSTER_META.get.clusterId)
assertEquals(MockProducerInterceptor.CLUSTER_META.get.clusterId, MockBrokerMetricsReporter.CLUSTER_META.get.clusterId)

View File

@ -17,15 +17,14 @@
package kafka.api.test
import java.util.{Properties, Collection, ArrayList}
import java.util.{ArrayList, Collection, Properties}
import org.junit.runners.Parameterized
import org.junit.runner.RunWith
import org.junit.runners.Parameterized.Parameters
import org.junit.{After, Before, Test}
import org.apache.kafka.clients.producer.{ProducerRecord, KafkaProducer, ProducerConfig}
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord}
import org.junit.Assert._
import kafka.api.FetchRequestBuilder
import kafka.server.{KafkaConfig, KafkaServer}
import kafka.consumer.SimpleConsumer

View File

@ -20,8 +20,9 @@ package kafka.log
import java.io.File
import java.util.Properties
import kafka.api.{KAFKA_0_10_0_IV1, KAFKA_0_9_0, KAFKA_0_11_0_IV0}
import kafka.server.OffsetCheckpoint
import kafka.api.KAFKA_0_11_0_IV0
import kafka.api.{KAFKA_0_10_0_IV1, KAFKA_0_9_0}
import kafka.server.checkpoints.OffsetCheckpointFile
import kafka.utils._
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.record._
@ -87,7 +88,7 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
// and make sure its gone from checkpoint file
cleaner.logs.remove(topics(0))
cleaner.updateCheckpoints(logDir)
val checkpoints = new OffsetCheckpoint(new File(logDir,cleaner.cleanerManager.offsetCheckpointFile)).read()
val checkpoints = new OffsetCheckpointFile(new File(logDir,cleaner.cleanerManager.offsetCheckpointFile)).read()
// we expect partition 0 to be gone
assertFalse(checkpoints.contains(topics(0)))
}

View File

@ -21,7 +21,7 @@ import java.io._
import java.util.Properties
import kafka.common._
import kafka.server.OffsetCheckpoint
import kafka.server.checkpoints.{OffsetCheckpoint, OffsetCheckpointFile}
import kafka.utils._
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.errors.OffsetOutOfRangeException
@ -102,7 +102,9 @@ class LogManagerTest {
time.sleep(maxLogAgeMs + 1)
assertEquals("Now there should only be only one segment in the index.", 1, log.numberOfSegments)
time.sleep(log.config.fileDeleteDelayMs + 1)
assertEquals("Files should have been deleted", log.numberOfSegments * 3 + 1, log.dir.list.length)
//There should be a log file, two indexes, the leader epoch checkpoint and the pid snapshot dir
assertEquals("Files should have been deleted", log.numberOfSegments * 3 + 2, log.dir.list.length)
assertEquals("Should get empty fetch off new log.", 0, log.read(offset+1, 1024).records.sizeInBytes)
try {
@ -148,7 +150,9 @@ class LogManagerTest {
time.sleep(logManager.InitialTaskDelayMs)
assertEquals("Now there should be exactly 6 segments", 6, log.numberOfSegments)
time.sleep(log.config.fileDeleteDelayMs + 1)
assertEquals("Files should have been deleted", log.numberOfSegments * 3 + 1, log.dir.list.length)
//There should be a log file, two indexes, the leader epoch checkpoint and the pid snapshot dir
assertEquals("Files should have been deleted", log.numberOfSegments * 3 + 2, log.dir.list.length)
assertEquals("Should get empty fetch off new log.", 0, log.read(offset + 1, 1024).records.sizeInBytes)
try {
log.read(0, 1024)
@ -288,7 +292,7 @@ class LogManagerTest {
})
logManager.checkpointRecoveryPointOffsets()
val checkpoints = new OffsetCheckpoint(new File(logDir, logManager.RecoveryPointCheckpointFile)).read()
val checkpoints = new OffsetCheckpointFile(new File(logDir, logManager.RecoveryPointCheckpointFile)).read()
topicPartitions.zip(logs).foreach {
case(tp, log) => {

View File

@ -346,4 +346,29 @@ class LogSegmentTest {
assertEquals(oldSize, size)
assertEquals(size, fileSize)
}
@Test
def shouldTruncateEvenIfOffsetPointsToAGapInTheLog() {
val seg = createSegment(40)
val offset = 40
def records(offset: Long, record: String): MemoryRecords =
MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, offset, CompressionType.NONE, TimestampType.CREATE_TIME,
new SimpleRecord(offset * 1000, record.getBytes))
//Given two messages with a gap between them (e.g. mid offset compacted away)
val ms1 = records(offset, "first message")
seg.append(offset, offset, RecordBatch.NO_TIMESTAMP, -1L, ms1)
val ms2 = records(offset + 3, "message after gap")
seg.append(offset + 3, offset + 3, RecordBatch.NO_TIMESTAMP, -1L, ms2)
// When we truncate to an offset without a corresponding log entry
seg.truncateTo(offset + 1)
//Then we should still truncate the record that was present (i.e. offset + 3 is gone)
val log = seg.read(offset, None, 10000)
assertEquals(offset, log.records.batches.iterator.next().baseOffset())
assertEquals(1, log.records.batches.asScala.size)
}
}

View File

@ -28,10 +28,14 @@ import org.scalatest.junit.JUnitSuite
import org.junit.{After, Before, Test}
import kafka.utils._
import kafka.server.KafkaConfig
import org.apache.kafka.common.record._
import kafka.server.epoch.{EpochEntry, LeaderEpochCache, LeaderEpochFileCache}
import org.apache.kafka.common.record.{RecordBatch, _}
import org.apache.kafka.common.utils.Utils
import org.easymock.EasyMock
import org.easymock.EasyMock._
import scala.collection.JavaConverters._
import scala.collection.mutable.ListBuffer
class LogTest extends JUnitSuite {
@ -1235,7 +1239,7 @@ class LogTest extends JUnitSuite {
val config = LogConfig(logProps)
val set = TestUtils.singletonRecords(value = "test".getBytes, timestamp = time.milliseconds)
val recoveryPoint = 50L
for (_ <- 0 until 50) {
for (_ <- 0 until 10) {
// create a log and write some messages to it
logDir.mkdirs()
var log = new Log(logDir,
@ -1441,9 +1445,14 @@ class LogTest extends JUnitSuite {
for (_ <- 0 until 100)
log.append(set)
log.leaderEpochCache.assign(0, 40)
log.leaderEpochCache.assign(1, 90)
// expire all segments
log.deleteOldSegments()
assertEquals("The deleted segments should be gone.", 1, log.numberOfSegments)
assertEquals("Epoch entries should have gone.", 1, epochCache(log).epochEntries().size)
assertEquals("Epoch entry should be the latest epoch and the leo.", new EpochEntry(1, 100), epochCache(log).epochEntries().head)
// append some messages to create some segments
for (_ <- 0 until 100)
@ -1452,6 +1461,8 @@ class LogTest extends JUnitSuite {
log.delete()
assertEquals("The number of segments should be 0", 0, log.numberOfSegments)
assertEquals("The number of deleted segments should be zero.", 0, log.deleteOldSegments())
assertEquals("Epoch entries should have gone.", 0, epochCache(log).epochEntries().size)
}
@Test
@ -1480,6 +1491,10 @@ class LogTest extends JUnitSuite {
assertEquals(log.logStartOffset, 15)
}
def epochCache(log: Log): LeaderEpochFileCache = {
log.leaderEpochCache.asInstanceOf[LeaderEpochFileCache]
}
@Test
def shouldDeleteSizeBasedSegments() {
val set = TestUtils.singletonRecords("test".getBytes)
@ -1566,6 +1581,205 @@ class LogTest extends JUnitSuite {
assertEquals("There should be 1 segment remaining", 1, log.numberOfSegments)
}
@Test
def shouldApplyEpochToMessageOnAppendIfLeader() {
val messageIds = (0 until 50).toArray
val records = messageIds.map(id => new SimpleRecord(id.toString.getBytes))
//Given this partition is on leader epoch 72
val epoch = 72
val log = new Log(logDir, LogConfig(), recoveryPoint = 0L, scheduler = time.scheduler, time = time)
//When appending messages as a leader (i.e. assignOffsets = true)
for (i <- records.indices)
log.append(
MemoryRecords.withRecords(messageIds(i), CompressionType.NONE, records(i)),
leaderEpochCache = mockCache(epoch),
assignOffsets = true
)
//Then leader epoch should be set on messages
for (i <- records.indices) {
val read = log.read(i, 100, Some(i+1)).records.batches().iterator.next()
assertEquals("Should have set leader epoch", 72, read.partitionLeaderEpoch())
}
}
@Test
def followerShouldSaveEpochInformationFromReplicatedMessagesToTheEpochCache() {
val messageIds = (0 until 50).toArray
val records = messageIds.map(id => new SimpleRecord(id.toString.getBytes))
val cache = createMock(classOf[LeaderEpochCache])
//Given each message has an offset & epoch, as msgs from leader would
def recordsForEpoch(i: Int): MemoryRecords = {
val recs = MemoryRecords.withRecords(messageIds(i), CompressionType.NONE, records(i))
recs.batches().asScala.foreach{record =>
record.setPartitionLeaderEpoch(42)
record.setLastOffset(i)
}
recs
}
//Verify we save the epoch to the cache.
expect(cache.assign(EasyMock.eq(42), anyInt())).times(records.size)
replay(cache)
val log = new Log(logDir, LogConfig(), recoveryPoint = 0L, scheduler = time.scheduler, time = time)
//When appending as follower (assignOffsets = false)
for (i <- records.indices)
log.append(recordsForEpoch(i), assignOffsets = false, leaderEpochCache = cache)
verify(cache)
}
@Test
def shouldTruncateLeaderEpochsWhenDeletingSegments() {
val set = TestUtils.singletonRecords("test".getBytes)
val log = createLog(set.sizeInBytes, retentionBytes = set.sizeInBytes * 10)
val cache = epochCache(log)
// Given three segments of 5 messages each
for (e <- 0 until 15) {
log.append(set)
}
//Given epochs
cache.assign(0, 0)
cache.assign(1, 5)
cache.assign(2, 10)
//When first segment is removed
log.deleteOldSegments
//The oldest epoch entry should have been removed
assertEquals(ListBuffer(EpochEntry(1, 5), EpochEntry(2, 10)), cache.epochEntries)
}
@Test
def shouldUpdateOffsetForLeaderEpochsWhenDeletingSegments() {
val set = TestUtils.singletonRecords("test".getBytes)
val log = createLog(set.sizeInBytes, retentionBytes = set.sizeInBytes * 10)
val cache = epochCache(log)
// Given three segments of 5 messages each
for (e <- 0 until 15) {
log.append(set)
}
//Given epochs
cache.assign(0, 0)
cache.assign(1, 7)
cache.assign(2, 10)
//When first segment removed (up to offset 5)
log.deleteOldSegments
//The the first entry should have gone from (0,0) => (0,5)
assertEquals(ListBuffer(EpochEntry(0, 5), EpochEntry(1, 7), EpochEntry(2, 10)), cache.epochEntries)
}
@Test
def shouldTruncateLeaderEpochFileWhenTruncatingLog() {
val set = TestUtils.singletonRecords(value = "test".getBytes, timestamp = time.milliseconds)
val logProps = CoreUtils.propsWith(LogConfig.SegmentBytesProp, (10 * set.sizeInBytes).toString)
val log = new Log(logDir, LogConfig( logProps), recoveryPoint = 0L, scheduler = time.scheduler, time = time)
val cache = epochCache(log)
//Given 2 segments, 10 messages per segment
for (epoch <- 1 to 20)
log.append(set)
//Simulate some leader changes at specific offsets
cache.assign(0, 0)
cache.assign(1, 10)
cache.assign(2, 16)
assertEquals(2, log.numberOfSegments)
assertEquals(20, log.logEndOffset)
//When truncate to LEO (no op)
log.truncateTo(log.logEndOffset)
//Then no change
assertEquals(3, cache.epochEntries().size)
//When truncate
log.truncateTo(11)
//Then no change
assertEquals(2, cache.epochEntries().size)
//When truncate
log.truncateTo(10)
//Then
assertEquals(1, cache.epochEntries().size)
//When truncate all
log.truncateTo(0)
//Then
assertEquals(0, cache.epochEntries().size)
}
/**
* Append a bunch of messages to a log and then re-open it with recovery and check that the leader epochs are recovered properly.
*/
@Test
def testLogRecoversForLeaderEpoch() {
val log = new Log(logDir, LogConfig(new Properties()), recoveryPoint = 0L, scheduler = time.scheduler, time = time)
val leaderEpochCache = epochCache(log)
val firstBatch = singletonRecordsWithLeaderEpoch(value = "random".getBytes, leaderEpoch = 1, offset = 0)
log.append(records = firstBatch, assignOffsets = false)
val secondBatch = singletonRecordsWithLeaderEpoch(value = "random".getBytes, leaderEpoch = 2, offset = 1)
log.append(records = secondBatch, assignOffsets = false)
val thirdBatch = singletonRecordsWithLeaderEpoch(value = "random".getBytes, leaderEpoch = 2, offset = 2)
log.append(records = thirdBatch, assignOffsets = false)
val fourthBatch = singletonRecordsWithLeaderEpoch(value = "random".getBytes, leaderEpoch = 3, offset = 3)
log.append(records = fourthBatch, assignOffsets = false)
assertEquals(ListBuffer(EpochEntry(1, 0), EpochEntry(2, 1), EpochEntry(3, 3)), leaderEpochCache.epochEntries)
// deliberately remove some of the epoch entries
leaderEpochCache.clearLatest(2)
assertNotEquals(ListBuffer(EpochEntry(1, 0), EpochEntry(2, 1), EpochEntry(3, 3)), leaderEpochCache.epochEntries)
log.close()
// reopen the log and recover from the beginning
val recoveredLog = new Log(logDir, LogConfig(new Properties()), recoveryPoint = 0L, scheduler = time.scheduler, time = time)
val recoveredLeaderEpochCache = epochCache(recoveredLog)
// epoch entries should be recovered
assertEquals(ListBuffer(EpochEntry(1, 0), EpochEntry(2, 1), EpochEntry(3, 3)), recoveredLeaderEpochCache.epochEntries)
recoveredLog.close()
}
/**
* Wrap a single record log buffer with leader epoch.
*/
private def singletonRecordsWithLeaderEpoch(value: Array[Byte],
key: Array[Byte] = null,
leaderEpoch: Int,
offset: Long,
codec: CompressionType = CompressionType.NONE,
timestamp: Long = RecordBatch.NO_TIMESTAMP,
magicValue: Byte = RecordBatch.CURRENT_MAGIC_VALUE): MemoryRecords = {
val records = Seq(new SimpleRecord(timestamp, key, value))
val buf = ByteBuffer.allocate(DefaultRecordBatch.sizeInBytes(records.asJava))
val builder = MemoryRecords.builder(buf, magicValue, codec, TimestampType.CREATE_TIME, offset,
System.currentTimeMillis, leaderEpoch)
records.foreach(builder.append)
builder.build()
}
def createLog(messageSizeInBytes: Int, retentionMs: Int = -1,
retentionBytes: Int = -1, cleanupPolicy: String = "delete"): Log = {
val logProps = new Properties()
@ -1583,4 +1797,11 @@ class LogTest extends JUnitSuite {
time = time)
log
}
private def mockCache(epoch: Int) = {
val cache = EasyMock.createNiceMock(classOf[LeaderEpochCache])
EasyMock.expect(cache.latestUsedEpoch()).andReturn(epoch).anyTimes()
EasyMock.replay(cache)
cache
}
}

View File

@ -24,11 +24,12 @@ import kafka.utils.TestUtils
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord}
import org.apache.kafka.common.requests.EpochEndOffset
import org.junit.Assert.{assertFalse, assertTrue}
import org.junit.{Before, Test}
import scala.collection.JavaConverters._
import scala.collection.{Map, mutable}
import scala.collection.{Map, Set, mutable}
class AbstractFetcherThreadTest {
@ -104,7 +105,7 @@ class AbstractFetcherThreadTest {
clientId: String,
sourceBroker: BrokerEndPoint,
fetchBackOffMs: Int = 0)
extends AbstractFetcherThread(name, clientId, sourceBroker, fetchBackOffMs) {
extends AbstractFetcherThread(name, clientId, sourceBroker, fetchBackOffMs, isInterruptible = true, includeLogTruncation = false) {
type REQ = DummyFetchRequest
type PD = PartitionData
@ -122,6 +123,12 @@ class AbstractFetcherThreadTest {
override protected def buildFetchRequest(partitionMap: collection.Seq[(TopicPartition, PartitionFetchState)]): DummyFetchRequest =
new DummyFetchRequest(partitionMap.map { case (k, v) => (k, v.fetchOffset) }.toMap)
override def buildLeaderEpochRequest(allPartitions: Seq[(TopicPartition, PartitionFetchState)]): Map[TopicPartition, Int] = { Map() }
override def fetchEpochsFromLeader(partitions: Map[TopicPartition, Int]): Map[TopicPartition, EpochEndOffset] = { Map() }
override def maybeTruncate(fetchedEpochs: Map[TopicPartition, EpochEndOffset]): Map[TopicPartition, Long] = { Map() }
}
@ -137,7 +144,7 @@ class AbstractFetcherThreadTest {
fetcherThread.addPartitions(Map(partition -> 0L))
// Wait until fetcherThread finishes the work
TestUtils.waitUntilTrue(() => fetcherThread.fetchCount > 3, "Failed waiting for fetcherThread tp finish the work")
TestUtils.waitUntilTrue(() => fetcherThread.fetchCount > 3, "Failed waiting for fetcherThread to finish the work")
fetcherThread.shutdown()
@ -198,7 +205,7 @@ class AbstractFetcherThreadTest {
val requestMap = new mutable.HashMap[TopicPartition, Long]
partitionMap.foreach { case (topicPartition, partitionFetchState) =>
// Add backoff delay check
if (partitionFetchState.isActive)
if (partitionFetchState.isReadyForFetch)
requestMap.put(topicPartition, partitionFetchState.fetchOffset)
}
new DummyFetchRequest(requestMap)

View File

@ -21,6 +21,8 @@ import java.util.concurrent.atomic.AtomicBoolean
import kafka.cluster.{Partition, Replica}
import kafka.log.Log
import kafka.server.checkpoints.{LeaderEpochCheckpointFile, LeaderEpochFile}
import kafka.server.epoch.{LeaderEpochCache, LeaderEpochFileCache}
import kafka.utils._
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.metrics.Metrics
@ -205,6 +207,9 @@ class IsrExpirationTest {
private def logMock: Log = {
val log = EasyMock.createMock(classOf[kafka.log.Log])
val cache = EasyMock.createNiceMock(classOf[LeaderEpochCache])
EasyMock.expect(log.dir).andReturn(TestUtils.tempDir()).anyTimes()
EasyMock.expect(log.leaderEpochCache).andReturn(cache).anyTimes()
EasyMock.replay(log)
log
}

View File

@ -23,6 +23,7 @@ import TestUtils._
import kafka.zk.ZooKeeperTestHarness
import java.io.File
import kafka.server.checkpoints.{OffsetCheckpoint, OffsetCheckpointFile}
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.serialization.{IntegerSerializer, StringSerializer}
@ -56,8 +57,8 @@ class LogRecoveryTest extends ZooKeeperTestHarness {
val message = "hello"
var producer: KafkaProducer[Integer, String] = null
def hwFile1 = new OffsetCheckpoint(new File(configProps1.logDirs.head, ReplicaManager.HighWatermarkFilename))
def hwFile2 = new OffsetCheckpoint(new File(configProps2.logDirs.head, ReplicaManager.HighWatermarkFilename))
def hwFile1 = new OffsetCheckpointFile(new File(configProps1.logDirs.head, ReplicaManager.HighWatermarkFilename))
def hwFile2 = new OffsetCheckpointFile(new File(configProps2.logDirs.head, ReplicaManager.HighWatermarkFilename))
var servers = Seq.empty[KafkaServer]
// Some tests restart the brokers then produce more data. But since test brokers use random ports, we need

View File

@ -0,0 +1,402 @@
/**
* 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.server
import kafka.cluster.{BrokerEndPoint, Replica}
import kafka.log.LogManager
import kafka.server.epoch.LeaderEpochCache
import org.apache.kafka.common.requests.EpochEndOffset._
import kafka.server.epoch.util.ReplicaFetcherMockBlockingSend
import kafka.utils.TestUtils
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.protocol.Errors._
import org.apache.kafka.common.requests.EpochEndOffset
import org.apache.kafka.common.utils.SystemTime
import org.easymock.EasyMock._
import org.easymock.{Capture, CaptureType}
import org.junit.Assert._
import org.junit.Test
import scala.collection.JavaConverters._
import scala.collection.{Map, mutable}
class ReplicaFetcherThreadTest {
private val t1p0 = new TopicPartition("topic1", 0)
private val t1p1 = new TopicPartition("topic1", 1)
private val t2p1 = new TopicPartition("topic2", 1)
@Test
def shouldNotIssueLeaderEpochRequestIfInterbrokerVersionBelow11(): Unit = {
val props = TestUtils.createBrokerConfig(1, "localhost:1234")
props.put(KafkaConfig.InterBrokerProtocolVersionProp, "0.10.2")
props.put(KafkaConfig.LogMessageFormatVersionProp, "0.10.2")
val config = KafkaConfig.fromProps(props)
val endPoint = new BrokerEndPoint(0, "localhost", 1000)
val thread = new ReplicaFetcherThread(
name = "bob",
fetcherId = 0,
sourceBroker = endPoint,
brokerConfig = config,
replicaMgr = null,
metrics = new Metrics(),
time = new SystemTime(),
quota = null,
leaderEndpointBlockingSend = None)
val result = thread.fetchEpochsFromLeader(Map(t1p0 -> 0, t1p1 -> 0))
val expected = Map(
t1p0 -> new EpochEndOffset(Errors.NONE, UNDEFINED_EPOCH_OFFSET),
t1p1 -> new EpochEndOffset(Errors.NONE, UNDEFINED_EPOCH_OFFSET)
)
assertEquals("results from leader epoch request should have undefined offset", expected, result)
}
@Test
def shouldHandleExceptionFromBlockingSend(): Unit = {
val props = TestUtils.createBrokerConfig(1, "localhost:1234")
val config = KafkaConfig.fromProps(props)
val endPoint = new BrokerEndPoint(0, "localhost", 1000)
val mockBlockingSend = createMock(classOf[BlockingSend])
expect(mockBlockingSend.sendRequest(anyObject())).andThrow(new NullPointerException).once()
replay(mockBlockingSend)
val thread = new ReplicaFetcherThread(
name = "bob",
fetcherId = 0,
sourceBroker = endPoint,
brokerConfig = config,
replicaMgr = null,
metrics = new Metrics(),
time = new SystemTime(),
quota = null,
leaderEndpointBlockingSend = Some(mockBlockingSend))
val result = thread.fetchEpochsFromLeader(Map(t1p0 -> 0, t1p1 -> 0))
val expected = Map(
t1p0 -> new EpochEndOffset(Errors.UNKNOWN, UNDEFINED_EPOCH_OFFSET),
t1p1 -> new EpochEndOffset(Errors.UNKNOWN, UNDEFINED_EPOCH_OFFSET)
)
assertEquals("results from leader epoch request should have undefined offset", expected, result)
verify(mockBlockingSend)
}
@Test
def shouldFetchLeaderEpochOnFirstFetchOnly(): Unit = {
val config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(1, "localhost:1234"))
//Setup all dependencies
val quota = createNiceMock(classOf[ReplicationQuotaManager])
val leaderEpochs = createNiceMock(classOf[LeaderEpochCache])
val logManager = createMock(classOf[LogManager])
val replica = createNiceMock(classOf[Replica])
val replicaManager = createMock(classOf[ReplicaManager])
//Stubs
expect(replica.epochs).andReturn(Some(leaderEpochs)).anyTimes()
expect(replica.logEndOffset).andReturn(new LogOffsetMetadata(0)).anyTimes()
expect(leaderEpochs.latestUsedEpoch).andReturn(5)
expect(replicaManager.logManager).andReturn(logManager).anyTimes()
stub(replica, replicaManager)
//Expectations
expect(logManager.truncateTo(anyObject())).once
replay(leaderEpochs, replicaManager, logManager, quota, replica)
//Define the offsets for the OffsetsForLeaderEpochResponse
val offsets = Map(t1p0 -> new EpochEndOffset(1), t1p1 -> new EpochEndOffset(1)).asJava
//Create the fetcher thread
val endPoint = new BrokerEndPoint(0, "localhost", 1000)
val mockNetwork = new ReplicaFetcherMockBlockingSend(offsets, endPoint, new SystemTime())
val thread = new ReplicaFetcherThread("bob", 0, endPoint, config, replicaManager, new Metrics(), new SystemTime(), quota, Some(mockNetwork))
thread.addPartitions(Map(t1p0 -> 0, t1p1 -> 0))
//Loop 1
thread.doWork()
assertEquals(1, mockNetwork.epochFetchCount)
assertEquals(1, mockNetwork.fetchCount)
//Loop 2 we should not fetch epochs
thread.doWork()
assertEquals(1, mockNetwork.epochFetchCount)
assertEquals(2, mockNetwork.fetchCount)
//Loop 3 we should not fetch epochs
thread.doWork()
assertEquals(1, mockNetwork.epochFetchCount)
assertEquals(3, mockNetwork.fetchCount)
//Assert that truncate to is called exactly once (despite two loops)
verify(logManager)
}
@Test
def shouldTruncateToOffsetSpecifiedInEpochOffsetResponse(): Unit = {
//Create a capture to track what partitions/offsets are truncated
val truncateToCapture: Capture[Map[TopicPartition, Long]] = newCapture(CaptureType.ALL)
// Setup all the dependencies
val configs = TestUtils.createBrokerConfigs(1, "localhost:1234").map(KafkaConfig.fromProps)
val quota = createNiceMock(classOf[ReplicationQuotaManager])
val leaderEpochs = createMock(classOf[LeaderEpochCache])
val logManager = createMock(classOf[LogManager])
val replica = createNiceMock(classOf[Replica])
val replicaManager = createMock(classOf[ReplicaManager])
val initialLEO = 200
//Stubs
expect(logManager.truncateTo(capture(truncateToCapture))).once
expect(replica.epochs).andReturn(Some(leaderEpochs)).anyTimes()
expect(replica.logEndOffset).andReturn(new LogOffsetMetadata(initialLEO)).anyTimes()
expect(leaderEpochs.latestUsedEpoch).andReturn(5).anyTimes()
expect(replicaManager.logManager).andReturn(logManager).anyTimes()
stub(replica, replicaManager)
replay(leaderEpochs, replicaManager, logManager, quota, replica)
//Define the offsets for the OffsetsForLeaderEpochResponse, these are used for truncation
val offsetsReply = Map(t1p0 -> new EpochEndOffset(156), t2p1 -> new EpochEndOffset(172)).asJava
//Create the thread
val endPoint = new BrokerEndPoint(0, "localhost", 1000)
val mockNetwork = new ReplicaFetcherMockBlockingSend(offsetsReply, endPoint, new SystemTime())
val thread = new ReplicaFetcherThread("bob", 0, endPoint, configs(0), replicaManager, new Metrics(), new SystemTime(), quota, Some(mockNetwork))
thread.addPartitions(Map(t1p0 -> 0, t2p1 -> 0))
//Run it
thread.doWork()
//We should have truncated to the offsets in the response
assertEquals(156, truncateToCapture.getValue.get(t1p0).get)
assertEquals(172, truncateToCapture.getValue.get(t2p1).get)
}
@Test
def shouldTruncateToHighWatermarkIfLeaderReturnsUndefinedOffset(): Unit = {
//Create a capture to track what partitions/offsets are truncated
val truncated: Capture[Map[TopicPartition, Long]] = newCapture(CaptureType.ALL)
// Setup all the dependencies
val configs = TestUtils.createBrokerConfigs(1, "localhost:1234").map(KafkaConfig.fromProps)
val quota = createNiceMock(classOf[ReplicationQuotaManager])
val leaderEpochs = createNiceMock(classOf[LeaderEpochCache])
val logManager = createMock(classOf[LogManager])
val replica = createNiceMock(classOf[Replica])
val replicaManager = createMock(classOf[ReplicaManager])
val highWaterMark = 100
val initialLeo = 300
//Stubs
expect(replica.highWatermark).andReturn(new LogOffsetMetadata(highWaterMark)).anyTimes()
expect(logManager.truncateTo(capture(truncated))).once
expect(replica.epochs).andReturn(Some(leaderEpochs)).anyTimes()
expect(replica.logEndOffset).andReturn(new LogOffsetMetadata(initialLeo)).anyTimes()
expect(leaderEpochs.latestUsedEpoch).andReturn(5)
expect(replicaManager.logManager).andReturn(logManager).anyTimes()
stub(replica, replicaManager)
replay(leaderEpochs, replicaManager, logManager, quota, replica)
//Define the offsets for the OffsetsForLeaderEpochResponse, these are used for truncation
val offsetsReply = Map(t1p0 -> new EpochEndOffset(EpochEndOffset.UNDEFINED_EPOCH_OFFSET)).asJava
//Create the thread
val endPoint = new BrokerEndPoint(0, "localhost", 1000)
val mockNetwork = new ReplicaFetcherMockBlockingSend(offsetsReply, endPoint, new SystemTime())
val thread = new ReplicaFetcherThread("bob", 0, endPoint, configs(0), replicaManager, new Metrics(), new SystemTime(), quota, Some(mockNetwork))
thread.addPartitions(Map(t1p0 -> 0))
//Run it
thread.doWork()
//We should have truncated to the highwatermark for partitino 2 only
assertEquals(highWaterMark, truncated.getValue.get(t1p0).get)
}
@Test
def shouldPollIndefinitelyIfLeaderReturnsAnyException(): Unit = {
//Create a capture to track what partitions/offsets are truncated
val truncated: Capture[Map[TopicPartition, Long]] = newCapture(CaptureType.ALL)
// Setup all the dependencies
val configs = TestUtils.createBrokerConfigs(1, "localhost:1234").map(KafkaConfig.fromProps)
val quota = createNiceMock(classOf[kafka.server.ReplicationQuotaManager])
val leaderEpochs = createNiceMock(classOf[LeaderEpochCache])
val logManager = createMock(classOf[kafka.log.LogManager])
val replica = createNiceMock(classOf[Replica])
val replicaManager = createMock(classOf[kafka.server.ReplicaManager])
val highWaterMark = 100
val initialLeo = 300
//Stubs
expect(replica.highWatermark).andReturn(new LogOffsetMetadata(highWaterMark)).anyTimes()
expect(logManager.truncateTo(capture(truncated))).anyTimes()
expect(replica.epochs).andReturn(Some(leaderEpochs)).anyTimes()
expect(replica.logEndOffset).andReturn(new LogOffsetMetadata(initialLeo)).anyTimes()
expect(leaderEpochs.latestUsedEpoch).andReturn(5)
expect(replicaManager.logManager).andReturn(logManager).anyTimes()
stub(replica, replicaManager)
replay(leaderEpochs, replicaManager, logManager, quota, replica)
//Define the offsets for the OffsetsForLeaderEpochResponse, these are used for truncation
val offsetsReply = mutable.Map(
t1p0 -> new EpochEndOffset(NOT_LEADER_FOR_PARTITION, EpochEndOffset.UNDEFINED_EPOCH_OFFSET),
t1p1 -> new EpochEndOffset(UNKNOWN, EpochEndOffset.UNDEFINED_EPOCH_OFFSET)
).asJava
//Create the thread
val endPoint = new BrokerEndPoint(0, "localhost", 1000)
val mockNetwork = new ReplicaFetcherMockBlockingSend(offsetsReply, endPoint, new SystemTime())
val thread = new ReplicaFetcherThread("bob", 0, endPoint, configs(0), replicaManager, new Metrics(), new SystemTime(), quota, Some(mockNetwork))
thread.addPartitions(Map(t1p0 -> 0, t2p1 -> 0))
//Run thread 3 times
(0 to 3).foreach { _ =>
thread.doWork()
}
//Then should loop continuously while there is no leader
for (invocation <- truncated.getValues().asScala)
assertEquals(0, invocation.size)
//New leader elected and replies
offsetsReply.put(t1p0, new EpochEndOffset(156))
thread.doWork()
//Now the final call should have actually done a truncation (to offset 156)
assertEquals(156, truncated.getValues.asScala.last.get(t1p0).get)
}
@Test
def shouldMovePartitionsOutOfTruncatingLogState(): Unit = {
val config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(1, "localhost:1234"))
//Setup all stubs
val quota = createNiceMock(classOf[ReplicationQuotaManager])
val leaderEpochs = createNiceMock(classOf[LeaderEpochCache])
val logManager = createNiceMock(classOf[LogManager])
val replica = createNiceMock(classOf[Replica])
val replicaManager = createNiceMock(classOf[ReplicaManager])
//Stub return values
expect(replica.epochs).andReturn(Some(leaderEpochs)).anyTimes()
expect(replica.logEndOffset).andReturn(new LogOffsetMetadata(0)).anyTimes()
expect(leaderEpochs.latestUsedEpoch).andReturn(5)
expect(replicaManager.logManager).andReturn(logManager).anyTimes()
stub(replica, replicaManager)
replay(leaderEpochs, replicaManager, logManager, quota, replica)
//Define the offsets for the OffsetsForLeaderEpochResponse
val offsetsReply = Map(
t1p0 -> new EpochEndOffset(1), t1p1 -> new EpochEndOffset(1)
).asJava
//Create the fetcher thread
val endPoint = new BrokerEndPoint(0, "localhost", 1000)
val mockNetwork = new ReplicaFetcherMockBlockingSend(offsetsReply, endPoint, new SystemTime())
val thread = new ReplicaFetcherThread("bob", 0, endPoint, config, replicaManager, new Metrics(), new SystemTime(), quota, Some(mockNetwork))
//When
thread.addPartitions(Map(t1p0 -> 0, t1p1 -> 0))
//Then all partitions should start in an TruncatingLog state
assertTrue(thread.partitionStates.partitionStates().asScala.forall(_.value().truncatingLog))
//When
thread.doWork()
//Then none should be TruncatingLog anymore
assertFalse(thread.partitionStates.partitionStates().asScala.forall(_.value().truncatingLog))
}
@Test
def shouldFilterPartitionsMadeLeaderDuringLeaderEpochRequest(): Unit ={
val config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(1, "localhost:1234"))
val truncateToCapture: Capture[Map[TopicPartition, Long]] = newCapture(CaptureType.ALL)
val initialLEO = 100
//Setup all stubs
val quota = createNiceMock(classOf[ReplicationQuotaManager])
val leaderEpochs = createNiceMock(classOf[LeaderEpochCache])
val logManager = createNiceMock(classOf[LogManager])
val replica = createNiceMock(classOf[Replica])
val replicaManager = createNiceMock(classOf[ReplicaManager])
//Stub return values
expect(logManager.truncateTo(capture(truncateToCapture))).once
expect(replica.epochs).andReturn(Some(leaderEpochs)).anyTimes()
expect(replica.logEndOffset).andReturn(new LogOffsetMetadata(initialLEO)).anyTimes()
expect(leaderEpochs.latestUsedEpoch).andReturn(5)
expect(replicaManager.logManager).andReturn(logManager).anyTimes()
stub(replica, replicaManager)
replay(leaderEpochs, replicaManager, logManager, quota, replica)
//Define the offsets for the OffsetsForLeaderEpochResponse
val offsetsReply = Map(
t1p0 -> new EpochEndOffset(52), t1p1 -> new EpochEndOffset(49)
).asJava
//Create the fetcher thread
val endPoint = new BrokerEndPoint(0, "localhost", 1000)
val mockNetwork = new ReplicaFetcherMockBlockingSend(offsetsReply, endPoint, new SystemTime())
val thread = new ReplicaFetcherThread("bob", 0, endPoint, config, replicaManager, new Metrics(), new SystemTime(), quota, Some(mockNetwork))
//When
thread.addPartitions(Map(t1p0 -> 0, t1p1 -> 0))
//When the epoch request is outstanding, remove one of the partitions to simulate a leader change. We do this via a callback passed to the mock thread
val partitionThatBecameLeader = t1p0
mockNetwork.setEpochRequestCallback(() => {
thread.removePartitions(Set(partitionThatBecameLeader))
})
//When
thread.doWork()
//Then we should not have truncated the partition that became leader
assertEquals(None, truncateToCapture.getValue.get(partitionThatBecameLeader))
assertEquals(49, truncateToCapture.getValue.get(t1p1).get)
}
def stub(replica: Replica, replicaManager: ReplicaManager) = {
expect(replicaManager.getReplica(t1p0)).andReturn(Some(replica)).anyTimes()
expect(replicaManager.getReplicaOrException(t1p0)).andReturn(replica).anyTimes()
expect(replicaManager.getReplica(t1p1)).andReturn(Some(replica)).anyTimes()
expect(replicaManager.getReplicaOrException(t1p1)).andReturn(replica).anyTimes()
expect(replicaManager.getReplica(t2p1)).andReturn(Some(replica)).anyTimes()
expect(replicaManager.getReplicaOrException(t2p1)).andReturn(replica).anyTimes()
}
}

View File

@ -146,7 +146,7 @@ class ReplicaManagerQuotasTest {
val scheduler = createNiceMock(classOf[KafkaScheduler])
//Create log which handles both a regular read and a 0 bytes read
val log = createMock(classOf[Log])
val log = createNiceMock(classOf[Log])
expect(log.logStartOffset).andReturn(0L).anyTimes()
expect(log.logEndOffset).andReturn(20L).anyTimes()
expect(log.logEndOffsetMetadata).andReturn(new LogOffsetMetadata(20L)).anyTimes()

View File

@ -74,9 +74,10 @@ class SimpleFetchTest {
EasyMock.replay(scheduler)
// create the log which takes read with either HW max offset or none max offset
val log = EasyMock.createMock(classOf[Log])
val log = EasyMock.createNiceMock(classOf[Log])
EasyMock.expect(log.logStartOffset).andReturn(0).anyTimes()
EasyMock.expect(log.logEndOffset).andReturn(leaderLEO).anyTimes()
EasyMock.expect(log.dir).andReturn(TestUtils.tempDir()).anyTimes()
EasyMock.expect(log.logEndOffsetMetadata).andReturn(new LogOffsetMetadata(leaderLEO)).anyTimes()
EasyMock.expect(log.read(0, fetchSize, Some(partitionHW), true)).andReturn(
FetchDataInfo(

View File

@ -0,0 +1,72 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package kafka.server.checkpoints
import java.io.File
import kafka.server.epoch.EpochEntry
import kafka.utils.Logging
import org.junit.Assert._
import org.junit.Test
import org.scalatest.junit.JUnitSuite
class LeaderEpochCheckpointFileTest extends JUnitSuite with Logging{
@Test
def shouldPersistAndOverwriteAndReloadFile(): Unit ={
val file = File.createTempFile("temp-checkpoint-file", System.nanoTime().toString)
file.deleteOnExit()
val checkpoint = new LeaderEpochCheckpointFile(file)
//Given
val epochs = Seq(EpochEntry(0, 1L), EpochEntry(1, 2L), EpochEntry(2, 3L))
//When
checkpoint.write(epochs)
//Then
assertEquals(epochs, checkpoint.read())
//Given overwrite
val epochs2 = Seq(EpochEntry(3, 4L), EpochEntry(4, 5L))
//When
checkpoint.write(epochs2)
//Then
assertEquals(epochs2, checkpoint.read())
}
@Test
def shouldRetainValuesEvenIfCheckpointIsRecreated(): Unit ={
val file = File.createTempFile("temp-checkpoint-file", System.nanoTime().toString)
file.deleteOnExit()
//Given a file with data in
val checkpoint = new LeaderEpochCheckpointFile(file)
val epochs = Seq(EpochEntry(0, 1L), EpochEntry(1, 2L), EpochEntry(2, 3L))
checkpoint.write(epochs)
//When we recreate
val checkpoint2 = new LeaderEpochCheckpointFile(file)
//The data should still be there
assertEquals(epochs, checkpoint2.read())
}
}

View File

@ -0,0 +1,89 @@
/**
* 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 unit.kafka.server.checkpoints
import kafka.server.checkpoints.{OffsetCheckpointFile}
import kafka.utils.{Logging, TestUtils}
import org.apache.kafka.common.TopicPartition
import org.junit.Assert._
import org.junit.Test
import org.scalatest.junit.JUnitSuite
import scala.collection.Map
class OffsetCheckpointFileTest extends JUnitSuite with Logging {
@Test
def shouldPersistAndOverwriteAndReloadFile(): Unit = {
val checkpoint = new OffsetCheckpointFile(TestUtils.tempFile())
//Given
val offsets = Map(new TopicPartition("foo", 1) -> 5L, new TopicPartition("bar", 2) -> 10L)
//When
checkpoint.write(offsets)
//Then
assertEquals(offsets, checkpoint.read())
//Given overwrite
val offsets2 = Map(new TopicPartition("foo", 2) -> 15L, new TopicPartition("bar", 3) -> 20L)
//When
checkpoint.write(offsets2)
//Then
assertEquals(offsets2, checkpoint.read())
}
@Test
def shouldHandleMultipleLines(): Unit = {
val checkpoint = new OffsetCheckpointFile(TestUtils.tempFile())
//Given
val offsets = Map(
new TopicPartition("foo", 1) -> 5L, new TopicPartition("bar", 6) -> 10L,
new TopicPartition("foo", 2) -> 5L, new TopicPartition("bar", 7) -> 10L,
new TopicPartition("foo", 3) -> 5L, new TopicPartition("bar", 8) -> 10L,
new TopicPartition("foo", 4) -> 5L, new TopicPartition("bar", 9) -> 10L,
new TopicPartition("foo", 5) -> 5L, new TopicPartition("bar", 10) -> 10L
)
//When
checkpoint.write(offsets)
//Then
assertEquals(offsets, checkpoint.read())
}
@Test
def shouldReturnEmptyMapForEmptyFile(): Unit = {
//When
val checkpoint = new OffsetCheckpointFile(TestUtils.tempFile())
//Then
assertEquals(Map(), checkpoint.read())
//When
checkpoint.write(Map())
//Then
assertEquals(Map(), checkpoint.read())
}
}

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
*
* 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.server.epoch
import java.io.{File, RandomAccessFile}
import java.util
import java.util.Properties
import kafka.admin.AdminUtils
import kafka.api.KAFKA_0_11_0_IV1
import kafka.log.Log
import kafka.server.KafkaConfig._
import kafka.server.{KafkaConfig, KafkaServer}
import kafka.tools.DumpLogSegments
import kafka.utils.{CoreUtils, Logging, TestUtils}
import kafka.utils.TestUtils._
import kafka.zk.ZooKeeperTestHarness
import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer}
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord}
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.record.RecordBatch
import org.apache.kafka.common.serialization.Deserializer
import org.junit.Assert.{assertEquals, assertTrue}
import org.junit.{After, Before, Test}
import scala.collection.JavaConverters._
import scala.collection.mutable.{ListBuffer => Buffer}
/**
* These tests were written to assert the addition of leader epochs to the replication protocol fix the problems
* described in KIP-101. There is a boolean KIP_101_ENABLED which can be toggled to demonstrate the tests failing in the pre-KIP-101 case
*
* https://cwiki.apache.org/confluence/display/KAFKA/KIP-101+-+Alter+Replication+Protocol+to+use+Leader+Epoch+rather+than+High+Watermark+for+Truncation
*
* A test which validates the end to end workflow is also included.
*/
class EpochDrivenReplicationProtocolAcceptanceTest extends ZooKeeperTestHarness with Logging {
val topic = "topic1"
val msg = new Array[Byte](1000)
val msgBigger = new Array[Byte](10000)
var brokers: Seq[KafkaServer] = null
var producer: KafkaProducer[Array[Byte], Array[Byte]] = null
var consumer: KafkaConsumer[Array[Byte], Array[Byte]] = null
val KIP_101_ENABLED = true
@Before
override def setUp() {
super.setUp()
}
@After
override def tearDown() {
brokers.par.foreach(_.shutdown())
producer.close()
super.tearDown()
}
@Test
def shouldFollowLeaderEpochBasicWorkflow(): Unit = {
//Given 2 brokers
brokers = (100 to 101).map(createBroker(_))
//A single partition topic with 2 replicas
AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, Map(0 -> Seq(100, 101)))
producer = createProducer()
val tp = new TopicPartition(topic, 0)
//When one record is written to the leader
producer.send(new ProducerRecord(topic, 0, null, msg)).get
//The message should have epoch 0 stamped onto it in both leader and follower
assertEquals(0, latestRecord(leader).partitionLeaderEpoch())
assertEquals(0, latestRecord(follower).partitionLeaderEpoch())
//Both leader and follower should have recorded Epoch 0 at Offset 0
assertEquals(Buffer(EpochEntry(0, 0)), epochCache(leader).epochEntries())
assertEquals(Buffer(EpochEntry(0, 0)), epochCache(follower).epochEntries())
//Bounce the follower
bounce(follower)
awaitISR(tp)
//Nothing happens yet as we haven't sent any new messages.
assertEquals(Buffer(EpochEntry(0, 0)), epochCache(leader).epochEntries())
assertEquals(Buffer(EpochEntry(0, 0)), epochCache(follower).epochEntries())
//Send a message
producer.send(new ProducerRecord(topic, 0, null, msg)).get
//Epoch1 should now propagate to the follower with the written message
assertEquals(Buffer(EpochEntry(0, 0), EpochEntry(1, 1)), epochCache(leader).epochEntries())
assertEquals(Buffer(EpochEntry(0, 0), EpochEntry(1, 1)), epochCache(follower).epochEntries())
//The new message should have epoch 1 stamped
assertEquals(1, latestRecord(leader).partitionLeaderEpoch())
assertEquals(1, latestRecord(follower).partitionLeaderEpoch())
//Bounce the leader. Epoch -> 2
bounce(leader)
awaitISR(tp)
//Epochs 2 should be added to the leader, but not on the follower (yet), as there has been no replication.
assertEquals(Buffer(EpochEntry(0, 0), EpochEntry(1, 1)), epochCache(leader).epochEntries())
assertEquals(Buffer(EpochEntry(0, 0), EpochEntry(1, 1)), epochCache(follower).epochEntries())
//Send a message
producer.send(new ProducerRecord(topic, 0, null, msg)).get
//This should case epoch 2 to propagate to the follower
assertEquals(2, latestRecord(leader).partitionLeaderEpoch())
assertEquals(2, latestRecord(follower).partitionLeaderEpoch())
//The leader epoch files should now match on leader and follower
assertEquals(Buffer(EpochEntry(0, 0), EpochEntry(1, 1), EpochEntry(2, 2)), epochCache(leader).epochEntries())
assertEquals(Buffer(EpochEntry(0, 0), EpochEntry(1, 1), EpochEntry(2, 2)), epochCache(follower).epochEntries())
}
@Test
def shouldNotAllowDivergentLogs(): Unit = {
//Given two brokers
brokers = (100 to 101).map { id => createServer(fromProps(createBrokerConfig(id, zkConnect))) }
//A single partition topic with 2 replicas
AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, Map(
0 -> Seq(100, 101)
))
producer = createProducer()
//Write 10 messages
(0 until 10).foreach { i =>
producer.send(new ProducerRecord(topic, 0, null, msg))
producer.flush()
}
//Stop the brokers
brokers.foreach { b => b.shutdown() }
//Delete the clean shutdown file to simulate crash
new File(brokers(0).config.logDirs(0), Log.CleanShutdownFile).delete()
//Delete 5 messages from the leader's log on 100
deleteMessagesFromLogFile(5 * msg.length, brokers(0), 0)
//Restart broker 100
brokers(0).startup()
//Bounce the producer (this is required, although I'm unsure as to why?)
producer.close()
producer = createProducer()
//Write ten larger messages (so we can easily distinguish between messages written in the two phases)
(0 until 10).foreach { _ =>
producer.send(new ProducerRecord(topic, 0, null, msgBigger))
producer.flush()
}
//Start broker 101
brokers(1).startup()
//Wait for replication to resync
waitForLogsToMatch(brokers(0), brokers(1))
assertEquals("Log files should match Broker0 vs Broker 1", getLogFile(brokers(0), 0).length, getLogFile(brokers(1), 0).length)
}
//We can reproduce the pre-KIP-101 failure of this test by setting KafkaConfig.InterBrokerProtocolVersionProp = KAFKA_0_11_0_IV1
@Test
def offsetsShouldNotGoBackwards(): Unit = {
//Given two brokers
brokers = (100 to 101).map(createBroker(_))
//A single partition topic with 2 replicas
AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, Map(
0 -> Seq(100, 101)
))
producer = bufferingProducer()
//Write 100 messages
(0 until 100).foreach { i =>
producer.send(new ProducerRecord(topic, 0, null, msg))
producer.flush()
}
//Stop the brokers
brokers.foreach { b => b.shutdown() }
//Delete the clean shutdown file to simulate crash
new File(brokers(0).config.logDirs(0), Log.CleanShutdownFile).delete()
//Delete half the messages from the log file
deleteMessagesFromLogFile(getLogFile(brokers(0), 0).length() / 2, brokers(0), 0)
//Start broker 100 again
brokers(0).startup()
//Bounce the producer (this is required, although I'm unsure as to why?)
producer.close()
producer = bufferingProducer()
//Write two large batches of messages. This will ensure that the LeO of the follower's log aligns with the middle
//of the a compressed message set in the leader (which, when forwarded, will result in offsets going backwards)
(0 until 77).foreach { _ =>
producer.send(new ProducerRecord(topic, 0, null, msg))
}
producer.flush()
(0 until 77).foreach { _ =>
producer.send(new ProducerRecord(topic, 0, null, msg))
}
producer.flush()
printSegments()
//Start broker 101. When it comes up it should read a whole batch of messages from the leader.
//As the chronology is lost we would end up with non-monatonic offsets (pre kip-101)
brokers(1).startup()
//Wait for replication to resync
waitForLogsToMatch(brokers(0), brokers(1))
printSegments()
//Shut down broker 100, so we read from broker 101 which should have corrupted
brokers(0).shutdown()
//Search to see if we have non-monotonic offsets in the log
startConsumer()
val records = consumer.poll(1000).asScala
var prevOffset = -1L
records.foreach { r =>
assertTrue(s"Offset $prevOffset came before ${r.offset} ", r.offset > prevOffset)
prevOffset = r.offset
}
//Are the files identical?
assertEquals("Log files should match Broker0 vs Broker 1", getLogFile(brokers(0), 0).length, getLogFile(brokers(1), 0).length)
}
/**
* Unlike the tests above, this test doesn't fail prior to the Leader Epoch Change. I was unable to find a deterministic
* method for recreating the fast leader change bug.
*/
@Test
def shouldSurviveFastLeaderChange(): Unit = {
val tp = new TopicPartition(topic, 0)
//Given 2 brokers
brokers = (100 to 101).map(createBroker(_))
//A single partition topic with 2 replicas
AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, Map(0 -> Seq(100, 101)))
producer = createProducer()
//Kick off with a single record
producer.send(new ProducerRecord(topic, 0, null, msg)).get
var messagesWritten = 1
//Now invoke the fast leader change bug
(0 until 5).foreach { i =>
val leaderId = zkUtils.getLeaderForPartition(topic, 0).get
val leader = brokers.filter(_.config.brokerId == leaderId)(0)
val follower = brokers.filter(_.config.brokerId != leaderId)(0)
producer.send(new ProducerRecord(topic, 0, null, msg)).get
messagesWritten += 1
//As soon as it replicates, bounce the follower
bounce(follower)
log(leader, follower)
awaitISR(tp)
//Then bounce the leader
bounce(leader)
log(leader, follower)
awaitISR(tp)
//Ensure no data was lost
assertTrue(brokers.forall { broker => getLog(broker, 0).logEndOffset == messagesWritten })
}
}
private def log(leader: KafkaServer, follower: KafkaServer): Unit = {
info(s"Bounce complete for follower ${follower.config.brokerId}")
info(s"Leader: leo${leader.config.brokerId}: " + getLog(leader, 0).logEndOffset + " cache: " + epochCache(leader).epochEntries())
info(s"Follower: leo${follower.config.brokerId}: " + getLog(follower, 0).logEndOffset + " cache: " + epochCache(follower).epochEntries())
}
private def waitForLogsToMatch(b1: KafkaServer, b2: KafkaServer, partition: Int = 0): Unit = {
TestUtils.waitUntilTrue(() => {getLog(b1, partition).logEndOffset == getLog(b2, partition).logEndOffset}, "Logs didn't match.")
}
private def printSegments(): Unit = {
info("Broker0:")
DumpLogSegments.main(Seq("--files", getLogFile(brokers(0), 0).getCanonicalPath).toArray)
info("Broker1:")
DumpLogSegments.main(Seq("--files", getLogFile(brokers(1), 0).getCanonicalPath).toArray)
}
private def startConsumer(): KafkaConsumer[Array[Byte], Array[Byte]] = {
val consumerConfig = new Properties()
consumerConfig.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, getBrokerListStrFromServers(brokers))
consumerConfig.put(ConsumerConfig.FETCH_MAX_BYTES_CONFIG, String.valueOf(getLogFile(brokers(1), 0).length() * 2))
consumerConfig.put(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, String.valueOf(getLogFile(brokers(1), 0).length() * 2))
consumer = new KafkaConsumer(consumerConfig, new StubDeserializer, new StubDeserializer)
consumer.assign(List(new TopicPartition(topic, 0)).asJava)
consumer.seek(new TopicPartition(topic, 0), 0)
consumer
}
private def deleteMessagesFromLogFile(bytes: Long, broker: KafkaServer, partitionId: Int): Unit = {
val logFile = getLogFile(broker, partitionId)
val writable = new RandomAccessFile(logFile, "rwd")
writable.setLength(logFile.length() - bytes)
writable.close()
}
private def bufferingProducer(): KafkaProducer[Array[Byte], Array[Byte]] = {
createNewProducer(getBrokerListStrFromServers(brokers), retries = 5, acks = -1, lingerMs = 10000,
props = Option(CoreUtils.propsWith(
(ProducerConfig.BATCH_SIZE_CONFIG, String.valueOf(msg.length * 1000))
, (ProducerConfig.COMPRESSION_TYPE_CONFIG, "snappy")
)))
}
private def getLogFile(broker: KafkaServer, partition: Int): File = {
val log: Log = getLog(broker, partition)
log.flush()
log.dir.listFiles.filter(_.getName.endsWith(".log"))(0)
}
private def getLog(broker: KafkaServer, partition: Int): Log = {
broker.logManager.logsByTopicPartition.get(new TopicPartition(topic, partition)).get
}
private def bounce(follower: KafkaServer): Unit = {
follower.shutdown()
follower.startup()
producer.close()
producer = createProducer() //TODO not sure why we need to recreate the producer, but it doesn't reconnect if we don't
}
private def epochCache(broker: KafkaServer): LeaderEpochFileCache = {
getLog(broker, 0).leaderEpochCache.asInstanceOf[LeaderEpochFileCache]
}
private def latestRecord(leader: KafkaServer, offset: Int = -1, partition: Int = 0): RecordBatch = {
getLog(leader, partition).activeSegment.read(0, None, Integer.MAX_VALUE)
.records.batches().asScala.toSeq.last
}
private def awaitISR(tp: TopicPartition): Boolean = {
TestUtils.waitUntilTrue(() => {
leader.replicaManager.getReplicaOrException(tp).partition.inSyncReplicas.map(_.brokerId).size == 2
}, "")
}
private def createProducer(): KafkaProducer[Array[Byte], Array[Byte]] = {
createNewProducer(getBrokerListStrFromServers(brokers), retries = 5, acks = -1)
}
private def leader(): KafkaServer = {
assertEquals(2, brokers.size)
val leaderId = zkUtils.getLeaderForPartition(topic, 0).get
brokers.filter(_.config.brokerId == leaderId)(0)
}
private def follower(): KafkaServer = {
assertEquals(2, brokers.size)
val leader = zkUtils.getLeaderForPartition(topic, 0).get
brokers.filter(_.config.brokerId != leader)(0)
}
private def createBroker(id: Int): KafkaServer = {
val config = createBrokerConfig(id, zkConnect)
if(!KIP_101_ENABLED) {
config.setProperty(KafkaConfig.InterBrokerProtocolVersionProp, KAFKA_0_11_0_IV1.version)
config.setProperty(KafkaConfig.LogMessageFormatVersionProp, KAFKA_0_11_0_IV1.version)
}
createServer(fromProps(config))
}
private class StubDeserializer extends Deserializer[Array[Byte]] {
override def configure(configs: java.util.Map[String, _], isKey: Boolean): Unit = {}
override def deserialize(topic: String, data: Array[Byte]): Array[Byte] = { data }
override def close(): Unit = {}
}
}

View File

@ -0,0 +1,721 @@
/**
* 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.server.epoch
import java.io.File
import kafka.server.LogOffsetMetadata
import kafka.server.checkpoints.{LeaderEpochCheckpoint, LeaderEpochCheckpointFile}
import org.apache.kafka.common.requests.EpochEndOffset.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET}
import kafka.utils.TestUtils
import org.apache.kafka.common.TopicPartition
import org.junit.Assert._
import org.junit.{Before, Test}
import scala.collection.mutable.ListBuffer
/**
* Unit test for the LeaderEpochFileCache.
*/
class LeaderEpochFileCacheTest {
val tp = new TopicPartition("TestTopic", 5)
var checkpoint: LeaderEpochCheckpoint = _
@Test
def shouldAddEpochAndMessageOffsetToCache() = {
var leo = 0
def leoFinder() = new LogOffsetMetadata(leo)
//Given
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
//When
cache.assign(epoch = 2, offset = 10)
leo = 11
//Then
assertEquals(2, cache.latestUsedEpoch())
assertEquals(EpochEntry(2, 10), cache.epochEntries()(0))
assertEquals(11, cache.endOffsetFor(2)) //should match leo
}
@Test
def shouldUpdateEpochWithLogEndOffset() = {
var leo = 0
def leoFinder() = new LogOffsetMetadata(leo)
//Given
leo = 9
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
//When
cache.cacheLatestEpoch(2)
cache.maybeAssignLatestCachedEpochToLeo()
//Then
assertEquals(2, cache.latestUsedEpoch())
assertEquals(EpochEntry(2, 9), cache.epochEntries()(0))
}
@Test
def shouldReturnLogEndOffsetIfLatestEpochRequested() = {
var leo = 0
def leoFinder() = new LogOffsetMetadata(leo)
//Given
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
//When just one epoch
cache.assign(epoch = 2, offset = 11)
cache.assign(epoch = 2, offset = 12)
leo = 14
//Then
assertEquals(14, cache.endOffsetFor(2))
}
@Test
def shouldReturnUndefinedOffsetIfUndefinedEpochRequested() = {
def leoFinder() = new LogOffsetMetadata(0)
//Given cache with some data on leader
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
cache.assign(epoch = 2, offset = 11)
cache.assign(epoch = 3, offset = 12)
//When (say a bootstraping follower) sends request for UNDEFINED_EPOCH
val offsetFor = cache.endOffsetFor(UNDEFINED_EPOCH)
//Then
assertEquals(UNDEFINED_EPOCH_OFFSET, offsetFor)
}
@Test
def shouldNotOverwriteLogEndOffsetForALeaderEpochOnceItHasBeenAssigned() = {
var leo = 0
def leoFinder() = new LogOffsetMetadata(leo)
//Given
leo = 9
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
cache.cacheLatestEpoch(2)
cache.maybeAssignLatestCachedEpochToLeo()
//When called again later
leo = 10
cache.cacheLatestEpoch(2)
cache.maybeAssignLatestCachedEpochToLeo()
//Then the offset should NOT have been updated
assertEquals(9, cache.epochEntries()(0).startOffset)
}
@Test
def shouldAllowLeaderEpochToChangeEvenIfOffsetDoesNot() = {
var leo = 0
def leoFinder() = new LogOffsetMetadata(leo)
//Given
leo = 9
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
cache.cacheLatestEpoch(2)
cache.maybeAssignLatestCachedEpochToLeo()
//When update epoch with same leo
cache.cacheLatestEpoch(3)
cache.maybeAssignLatestCachedEpochToLeo()
//Then the offset should NOT have been updated
assertEquals(9, cache.endOffsetFor(3))
assertEquals(9, cache.endOffsetFor(2))
assertEquals(3, cache.latestUsedEpoch())
}
@Test
def shouldNotOverwriteOffsetForALeaderEpochOnceItHasBeenAssigned() = {
//Given
val cache = new LeaderEpochFileCache(tp, () => new LogOffsetMetadata(0), checkpoint)
cache.assign(2, 6)
//When called again later with a greater offset
cache.assign(2, 10)
//Then later update should have been ignored
assertEquals(6, cache.epochEntries()(0).startOffset)
}
@Test
def shouldReturnUnsupportedIfNoEpochRecorded(){
def leoFinder() = new LogOffsetMetadata(0)
//Given
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
//Then
assertEquals(UNDEFINED_EPOCH, cache.latestUsedEpoch())
assertEquals(UNDEFINED_EPOCH_OFFSET, cache.endOffsetFor(0))
}
@Test
def shouldReturnUnsupportedIfRequestedEpochLessThanFirstEpoch(){
def leoFinder() = new LogOffsetMetadata(0)
//Given
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
cache.assign(epoch = 5, offset = 11)
cache.assign(epoch = 6, offset = 12)
cache.assign(epoch = 7, offset = 13)
//When
val offset = cache.endOffsetFor(5 - 1)
//Then
assertEquals(UNDEFINED_EPOCH_OFFSET, offset)
}
@Test
def shouldGetFirstOffsetOfSubsequentEpochWhenOffsetRequestedForPreviousEpoch() = {
var leo = 0
def leoFinder() = new LogOffsetMetadata(leo)
//Given
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
//When several epochs
cache.assign(epoch = 1, offset = 11)
cache.assign(epoch = 1, offset = 12)
cache.assign(epoch = 2, offset = 13)
cache.assign(epoch = 2, offset = 14)
cache.assign(epoch = 3, offset = 15)
cache.assign(epoch = 3, offset = 16)
leo = 17
//Then get the start offset of the next epoch
assertEquals(15, cache.endOffsetFor(2))
}
@Test
def shouldReturnNextAvailableEpochIfThereIsNoExactEpochForTheOneRequested(){
def leoFinder() = new LogOffsetMetadata(0)
//Given
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
//When
cache.assign(epoch = 0, offset = 10)
cache.assign(epoch = 2, offset = 13)
cache.assign(epoch = 4, offset = 17)
//Then
assertEquals(13, cache.endOffsetFor(requestedEpoch = 1))
assertEquals(17, cache.endOffsetFor(requestedEpoch = 2))
}
@Test
def shouldNotUpdateEpochAndStartOffsetIfItDidNotChange() = {
def leoFinder() = new LogOffsetMetadata(0)
//Given
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
//When
cache.assign(epoch = 2, offset = 6)
cache.assign(epoch = 2, offset = 7)
//Then
assertEquals(1, cache.epochEntries.size)
assertEquals(EpochEntry(2, 6), cache.epochEntries.toList(0))
}
@Test
def shouldReturnInvalidOffsetIfEpochIsRequestedWhichIsNotCurrentlyTracked(): Unit = {
val leo = 100
def leoFinder() = new LogOffsetMetadata(leo)
//Given
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
//When
cache.cacheLatestEpoch(epoch = 2)
cache.maybeAssignLatestCachedEpochToLeo()
//Then
assertEquals(UNDEFINED_EPOCH_OFFSET, cache.endOffsetFor(3))
}
@Test
def shouldSupportEpochsThatDoNotStartFromZero(): Unit = {
var leo = 0
def leoFinder() = new LogOffsetMetadata(leo)
//Given
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
//When
cache.assign(epoch = 2, offset = 6)
leo = 7
//Then
assertEquals(leo, cache.endOffsetFor(2))
assertEquals(1, cache.epochEntries.size)
assertEquals(EpochEntry(2, 6), cache.epochEntries()(0))
}
@Test
def shouldPersistEpochsBetweenInstances(){
def leoFinder() = new LogOffsetMetadata(0)
val checkpointPath = TestUtils.tempFile().getAbsolutePath
checkpoint = new LeaderEpochCheckpointFile(new File(checkpointPath))
//Given
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
cache.assign(epoch = 2, offset = 6)
//When
val checkpoint2 = new LeaderEpochCheckpointFile(new File(checkpointPath))
val cache2 = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint2)
//Then
assertEquals(1, cache2.epochEntries.size)
assertEquals(EpochEntry(2, 6), cache2.epochEntries.toList(0))
}
@Test
def shouldNotLetEpochGoBackwardsEvenIfMessageEpochsDo(): Unit = {
var leo = 0
def leoFinder() = new LogOffsetMetadata(leo)
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
//Given
cache.assign(epoch = 1, offset = 5); leo = 6
cache.assign(epoch = 2, offset = 6); leo = 7
//When we update an epoch in the past with an earlier offset
cache.assign(epoch = 1, offset = 7); leo = 8
//Then epoch should not be changed
assertEquals(2, cache.latestUsedEpoch())
//Then end offset for epoch 1 shouldn't have changed
assertEquals(6, cache.endOffsetFor(1))
//Then end offset for epoch 2 has to be the offset of the epoch 1 message (I can't thing of a better option)
assertEquals(8, cache.endOffsetFor(2))
//Epoch history shouldn't have changed
assertEquals(EpochEntry(1, 5), cache.epochEntries()(0))
assertEquals(EpochEntry(2, 6), cache.epochEntries()(1))
}
@Test
def shouldNotLetOffsetsGoBackwardsEvenIfEpochsProgress() = {
def leoFinder() = new LogOffsetMetadata(0)
//Given
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
//When epoch goes forward but offset goes backwards
cache.assign(epoch = 2, offset = 6)
cache.assign(epoch = 3, offset = 5)
//Then latter assign should be ignored
assertEquals(EpochEntry(2, 6), cache.epochEntries.toList(0))
}
@Test
def shouldIncreaseAndTrackEpochsAsLeadersChangeManyTimes(): Unit = {
var leo = 0
def leoFinder() = new LogOffsetMetadata(leo)
//Given
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
cache.cacheLatestEpoch(epoch = 0) //leo=0
cache.maybeAssignLatestCachedEpochToLeo()
//When
cache.cacheLatestEpoch(epoch = 1) //leo=0
cache.maybeAssignLatestCachedEpochToLeo()
//Then epoch should go up
assertEquals(1, cache.latestUsedEpoch())
//offset for 1 should still be 0
assertEquals(0, cache.endOffsetFor(1))
//offset for 0 should the start offset of epoch(1) => 0
assertEquals(0, cache.endOffsetFor(0))
//When we write 5 messages as epoch 1
leo = 5
//Then end offset for epoch(1) should be leo => 5
assertEquals(5, cache.endOffsetFor(1))
//Epoch(0) should still show the start offset for Epoch(1) => 0
assertEquals(0, cache.endOffsetFor(0))
//When
cache.cacheLatestEpoch(epoch = 2) //leo=5
cache.maybeAssignLatestCachedEpochToLeo()
leo = 10 //write another 5 messages
//Then end offset for epoch(2) should be leo => 10
assertEquals(10, cache.endOffsetFor(2))
//end offset for epoch(1) should be the start offset of epoch(2) => 5
assertEquals(5, cache.endOffsetFor(1))
//epoch (0) should still be 0
assertEquals(0, cache.endOffsetFor(0))
}
@Test
def shouldIncreaseAndTrackEpochsAsFollowerReceivesManyMessages(): Unit = {
var leo = 0
def leoFinder() = new LogOffsetMetadata(leo)
//When new
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
//When Messages come in
cache.assign(epoch = 0, offset = 0); leo = 1
cache.assign(epoch = 0, offset = 1); leo = 2
cache.assign(epoch = 0, offset = 2); leo = 3
//Then epoch should stay, offsets should grow
assertEquals(0, cache.latestUsedEpoch())
assertEquals(leo, cache.endOffsetFor(0))
//When messages arrive with greater epoch
cache.assign(epoch = 1, offset = 3); leo = 4
cache.assign(epoch = 1, offset = 4); leo = 5
cache.assign(epoch = 1, offset = 5); leo = 6
assertEquals(1, cache.latestUsedEpoch())
assertEquals(leo, cache.endOffsetFor(1))
//When
cache.assign(epoch = 2, offset = 6); leo = 7
cache.assign(epoch = 2, offset = 7); leo = 8
cache.assign(epoch = 2, offset = 8); leo = 9
assertEquals(2, cache.latestUsedEpoch())
assertEquals(leo, cache.endOffsetFor(2))
//Older epochs should return the start offset of the first message in the subsequent epoch.
assertEquals(3, cache.endOffsetFor(0))
assertEquals(6, cache.endOffsetFor(1))
}
@Test
def shouldDropEntriesOnEpochBoundaryWhenRemovingLatestEntries(): Unit = {
def leoFinder() = new LogOffsetMetadata(0)
//Given
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
cache.assign(epoch = 2, offset = 6)
cache.assign(epoch = 3, offset = 8)
cache.assign(epoch = 4, offset = 11)
//When clear latest on epoch boundary
cache.clearLatest(offset = 8)
//Then should remove two latest epochs (remove is inclusive)
assertEquals(ListBuffer(EpochEntry(2, 6)), cache.epochEntries)
}
@Test
def shouldPreserveResetOffsetOnClearEarliestIfOneExists(): Unit = {
def leoFinder() = new LogOffsetMetadata(0)
//Given
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
cache.assign(epoch = 2, offset = 6)
cache.assign(epoch = 3, offset = 8)
cache.assign(epoch = 4, offset = 11)
//When reset to offset ON epoch boundary
cache.clearEarliest(offset = 8)
//Then should preserve (3, 8)
assertEquals(ListBuffer(EpochEntry(3, 8), EpochEntry(4, 11)), cache.epochEntries)
}
@Test
def shouldUpdateSavedOffsetWhenOffsetToClearToIsBetweenEpochs(): Unit = {
def leoFinder() = new LogOffsetMetadata(0)
//Given
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
cache.assign(epoch = 2, offset = 6)
cache.assign(epoch = 3, offset = 8)
cache.assign(epoch = 4, offset = 11)
//When reset to offset BETWEEN epoch boundaries
cache.clearEarliest(offset = 9)
//Then we should retain epoch 3, but update it's offset to 9 as 8 has been removed
assertEquals(ListBuffer(EpochEntry(3, 9), EpochEntry(4, 11)), cache.epochEntries)
}
@Test
def shouldNotClearAnythingIfOffsetToEarly(): Unit = {
def leoFinder() = new LogOffsetMetadata(0)
//Given
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
cache.assign(epoch = 2, offset = 6)
cache.assign(epoch = 3, offset = 8)
cache.assign(epoch = 4, offset = 11)
//When reset to offset before first epoch offset
cache.clearEarliest(offset = 1)
//Then nothing should change
assertEquals(ListBuffer(EpochEntry(2, 6),EpochEntry(3, 8), EpochEntry(4, 11)), cache.epochEntries)
}
@Test
def shouldNotClearAnythingIfOffsetToFirstOffset(): Unit = {
def leoFinder() = new LogOffsetMetadata(0)
//Given
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
cache.assign(epoch = 2, offset = 6)
cache.assign(epoch = 3, offset = 8)
cache.assign(epoch = 4, offset = 11)
//When reset to offset on earliest epoch boundary
cache.clearEarliest(offset = 6)
//Then nothing should change
assertEquals(ListBuffer(EpochEntry(2, 6),EpochEntry(3, 8), EpochEntry(4, 11)), cache.epochEntries)
}
@Test
def shouldRetainLatestEpochOnClearAllEarliest(): Unit = {
def leoFinder() = new LogOffsetMetadata(0)
//Given
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
cache.assign(epoch = 2, offset = 6)
cache.assign(epoch = 3, offset = 8)
cache.assign(epoch = 4, offset = 11)
//When
cache.clearEarliest(offset = 11)
//Then retain the last
assertEquals(ListBuffer(EpochEntry(4, 11)), cache.epochEntries)
}
@Test
def shouldUpdateOffsetBetweenEpochBoundariesOnClearEarliest(): Unit = {
def leoFinder() = new LogOffsetMetadata(0)
//Given
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
cache.assign(epoch = 2, offset = 6)
cache.assign(epoch = 3, offset = 8)
cache.assign(epoch = 4, offset = 11)
//When we clear from a postition between offset 8 & offset 11
cache.clearEarliest(offset = 9)
//Then we should update the middle epoch entry's offset
assertEquals(ListBuffer(EpochEntry(3, 9), EpochEntry(4, 11)), cache.epochEntries)
}
@Test
def shouldUpdateOffsetBetweenEpochBoundariesOnClearEarliest2(): Unit = {
def leoFinder() = new LogOffsetMetadata(0)
//Given
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
cache.assign(epoch = 0, offset = 0)
cache.assign(epoch = 1, offset = 7)
cache.assign(epoch = 2, offset = 10)
//When we clear from a postition between offset 0 & offset 7
cache.clearEarliest(offset = 5)
//Then we should keeep epoch 0 but update the offset appropriately
assertEquals(ListBuffer(EpochEntry(0,5), EpochEntry(1, 7), EpochEntry(2, 10)), cache.epochEntries)
}
@Test
def shouldRetainLatestEpochOnClearAllEarliestAndUpdateItsOffset(): Unit = {
def leoFinder() = new LogOffsetMetadata(0)
//Given
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
cache.assign(epoch = 2, offset = 6)
cache.assign(epoch = 3, offset = 8)
cache.assign(epoch = 4, offset = 11)
//When reset to offset beyond last epoch
cache.clearEarliest(offset = 15)
//Then update the last
assertEquals(ListBuffer(EpochEntry(4, 15)), cache.epochEntries)
}
@Test
def shouldDropEntriesBetweenEpochBoundaryWhenRemovingNewest(): Unit = {
def leoFinder() = new LogOffsetMetadata(0)
//Given
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
cache.assign(epoch = 2, offset = 6)
cache.assign(epoch = 3, offset = 8)
cache.assign(epoch = 4, offset = 11)
//When reset to offset BETWEEN epoch boundaries
cache.clearLatest(offset = 9)
//Then should keep the preceding epochs
assertEquals(3, cache.latestUsedEpoch())
assertEquals(ListBuffer(EpochEntry(2, 6), EpochEntry(3, 8)), cache.epochEntries)
}
@Test
def shouldClearAllEntries(): Unit = {
def leoFinder() = new LogOffsetMetadata(0)
//Given
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
cache.assign(epoch = 2, offset = 6)
cache.assign(epoch = 3, offset = 8)
cache.assign(epoch = 4, offset = 11)
//When
cache.clear()
//Then
assertEquals(0, cache.epochEntries.size)
}
@Test
def shouldNotResetEpochHistoryHeadIfUndefinedPassed(): Unit = {
def leoFinder() = new LogOffsetMetadata(0)
//Given
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
cache.assign(epoch = 2, offset = 6)
cache.assign(epoch = 3, offset = 8)
cache.assign(epoch = 4, offset = 11)
//When reset to offset on epoch boundary
cache.clearLatest(offset = UNDEFINED_EPOCH_OFFSET)
//Then should do nothing
assertEquals(3, cache.epochEntries.size)
}
@Test
def shouldNotResetEpochHistoryTailIfUndefinedPassed(): Unit = {
def leoFinder() = new LogOffsetMetadata(0)
//Given
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
cache.assign(epoch = 2, offset = 6)
cache.assign(epoch = 3, offset = 8)
cache.assign(epoch = 4, offset = 11)
//When reset to offset on epoch boundary
cache.clearEarliest(offset = UNDEFINED_EPOCH_OFFSET)
//Then should do nothing
assertEquals(3, cache.epochEntries.size)
}
@Test
def shouldFetchLatestEpochOfEmptyCache(): Unit = {
//Given
def leoFinder() = new LogOffsetMetadata(0)
//When
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
//Then
assertEquals(-1, cache.latestUsedEpoch)
}
@Test
def shouldFetchEndOffsetOfEmptyCache(): Unit = {
//Given
def leoFinder() = new LogOffsetMetadata(0)
//When
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
//Then
assertEquals(-1, cache.endOffsetFor(7))
}
@Test
def shouldClearEarliestOnEmptyCache(): Unit = {
//Given
def leoFinder() = new LogOffsetMetadata(0)
//When
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
//Then
cache.clearEarliest(7)
}
@Test
def shouldClearLatestOnEmptyCache(): Unit = {
//Given
def leoFinder() = new LogOffsetMetadata(0)
//When
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
//Then
cache.clearLatest(7)
}
@Test
def shouldUpdateEpochCacheOnLeadershipChangeThenCommit(): Unit ={
//Given
def leoFinder() = new LogOffsetMetadata(5)
val cache = new LeaderEpochFileCache(tp, () => leoFinder, checkpoint)
//When
cache.cacheLatestEpoch(2)
//Then
assertEquals(UNDEFINED_EPOCH, cache.latestUsedEpoch())
//When
cache.maybeAssignLatestCachedEpochToLeo()
//Then should have saved epoch
assertEquals(2, cache.latestUsedEpoch())
//Then should have applied LEO to epoch
assertEquals(5, cache.endOffsetFor(2))
}
@Before
def setUp() {
checkpoint = new LeaderEpochCheckpointFile(TestUtils.tempFile())
}
}

View File

@ -0,0 +1,283 @@
/**
* 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.server.epoch
import java.util.{Map => JMap}
import kafka.admin.AdminUtils
import kafka.server.KafkaConfig._
import kafka.server.{BlockingSend, KafkaConfig, KafkaServer, ReplicaFetcherBlockingSend}
import kafka.utils.TestUtils._
import kafka.utils.{Logging, TestUtils}
import kafka.zk.ZooKeeperTestHarness
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.protocol.Errors._
import org.apache.kafka.common.requests.EpochEndOffset._
import org.apache.kafka.common.serialization.StringSerializer
import org.apache.kafka.common.utils.SystemTime
import org.apache.kafka.common.TopicPartition
import org.junit.Assert._
import org.junit.{After, Before, Test}
import org.apache.kafka.common.requests.{EpochEndOffset, OffsetsForLeaderEpochRequest, OffsetsForLeaderEpochResponse}
import scala.collection.JavaConverters._
import scala.collection.Map
class LeaderEpochIntegrationTest extends ZooKeeperTestHarness with Logging {
var brokers: Seq[KafkaServer] = null
val topic1 = "foo"
val topic2 = "bar"
val t1p0 = new TopicPartition(topic1, 0)
val t1p1 = new TopicPartition(topic1, 1)
val t1p2 = new TopicPartition(topic1, 2)
val t2p0 = new TopicPartition(topic2, 0)
val t2p2 = new TopicPartition(topic2, 2)
val tp = t1p0
var producer: KafkaProducer[Array[Byte], Array[Byte]] = null
@Before
override def setUp() {
super.setUp()
val props = createBrokerConfigs(2, zkConnect)
brokers = props.map(KafkaConfig.fromProps).map(TestUtils.createServer(_))
}
@After
override def tearDown() {
brokers.foreach(_.shutdown())
if (producer != null)
producer.close()
super.tearDown()
}
@Test
def shouldAddCurrentLeaderEpochToMessagesAsTheyAreWrittenToLeader() {
// Given two topics with replication of a single partition
for (topic <- List(topic1, topic2)) {
createTopic(zkUtils, topic, Map(0 -> Seq(0, 1)), servers = brokers)
}
// When we send four messages
sendFourMessagesToEachTopic()
//Then they should be stamped with Leader Epoch 0
var expectedLeaderEpoch = 0
waitUntilTrue(() => messagesHaveLeaderEpoch(brokers(0), expectedLeaderEpoch, 0), "Leader epoch should be 0")
//Given we then bounce the leader
brokers(0).shutdown()
brokers(0).startup()
//Then LeaderEpoch should now have changed from 0 -> 1
expectedLeaderEpoch = 1
waitForEpochChangeTo(topic1, 0, expectedLeaderEpoch)
waitForEpochChangeTo(topic2, 0, expectedLeaderEpoch)
//Given we now send messages
sendFourMessagesToEachTopic()
//The new messages should be stamped with LeaderEpoch = 1
waitUntilTrue(() => messagesHaveLeaderEpoch(brokers(0), expectedLeaderEpoch, 4), "Leader epoch should be 1")
}
@Test
def shouldSendLeaderEpochRequestAndGetAResponse(): Unit = {
//3 brokers, put partition on 100/101 and then pretend to be 102
brokers = (100 to 102).map { id => createServer(fromProps(createBrokerConfig(id, zkConnect))) }
AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic1, Map(
0 -> Seq(100),
1 -> Seq(101)
))
AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic2, Map(
0 -> Seq(100)
))
//Send messages equally to the two partitions, then half as many to a third
producer = createNewProducer(getBrokerListStrFromServers(brokers), retries = 5, acks = -1)
(0 until 10).foreach { _ =>
producer.send(new ProducerRecord(topic1, 0, null, "IHeartLogs".getBytes))
}
(0 until 20).foreach { _ =>
producer.send(new ProducerRecord(topic1, 1, null, "OhAreThey".getBytes))
}
(0 until 30).foreach { _ =>
producer.send(new ProducerRecord(topic2, 0, null, "IReallyDo".getBytes))
}
producer.flush()
val fetcher0 = new TestFetcherThread(sender(from = brokers(2), to = brokers(0)))
val epochsRequested = Map(t1p0 -> 0, t1p1 -> 0, t2p0 -> 0, t2p2 -> 0)
//When
val offsetsForEpochs = fetcher0.leaderOffsetsFor(epochsRequested)
//Then end offset should be correct
assertEquals(10, offsetsForEpochs(t1p0).endOffset)
assertEquals(30, offsetsForEpochs(t2p0).endOffset)
//And should get no leader for partition error from t1p1 (as it's not on broker 0)
assertTrue(offsetsForEpochs(t1p1).hasError)
assertEquals(UNKNOWN_TOPIC_OR_PARTITION, offsetsForEpochs(t1p1).error)
assertEquals(UNDEFINED_EPOCH_OFFSET, offsetsForEpochs(t1p1).endOffset)
//Repointing to broker 1 we should get the correct offset for t1p1
val fetcher1 = new TestFetcherThread(sender(from = brokers(2), to = brokers(1)))
val offsetsForEpochs1 = fetcher1.leaderOffsetsFor(epochsRequested)
assertEquals(20, offsetsForEpochs1(t1p1).endOffset)
}
@Test
def shouldIncreaseLeaderEpochBetweenLeaderRestarts(): Unit = {
//Setup: we are only interested in the single partition on broker 101
brokers = Seq(100, 101).map { id => createServer(fromProps(createBrokerConfig(id, zkConnect))) }
def leo() = brokers(1).replicaManager.getReplica(tp).get.logEndOffset.messageOffset
AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, tp.topic, Map(tp.partition -> Seq(101)))
producer = createNewProducer(getBrokerListStrFromServers(brokers), retries = 10, acks = -1)
//1. Given a single message
producer.send(new ProducerRecord(tp.topic, tp.partition, null, "IHeartLogs".getBytes)).get
var fetcher = new TestFetcherThread(sender(brokers(0), brokers(1)))
//Then epoch should be 0 and leo: 1
var offset = fetcher.leaderOffsetsFor(Map(tp -> 0))(tp).endOffset()
assertEquals(1, offset)
assertEquals(leo(), offset)
//2. When broker is bounced
brokers(1).shutdown()
brokers(1).startup()
producer.send(new ProducerRecord(tp.topic, tp.partition, null, "IHeartLogs".getBytes)).get
fetcher = new TestFetcherThread(sender(brokers(0), brokers(1)))
//Then epoch 0 should still be the start offset of epoch 1
offset = fetcher.leaderOffsetsFor(Map(tp -> 0))(tp).endOffset()
assertEquals(1, offset)
//Then epoch 2 should be the leo (NB: The leader epoch goes up in factors of 2 - This is because we have to first change leader to -1 and then change it again to the live replica)
assertEquals(2, fetcher.leaderOffsetsFor(Map(tp -> 2))(tp).endOffset())
assertEquals(leo(), fetcher.leaderOffsetsFor(Map(tp -> 2))(tp).endOffset())
//3. When broker is bounced again
brokers(1).shutdown()
brokers(1).startup()
producer.send(new ProducerRecord(tp.topic, tp.partition, null, "IHeartLogs".getBytes)).get
fetcher = new TestFetcherThread(sender(brokers(0), brokers(1)))
//Then Epoch 0 should still map to offset 1
assertEquals(1, fetcher.leaderOffsetsFor(Map(tp -> 0))(tp).endOffset())
//Then Epoch 2 should still map to offset 2
assertEquals(2, fetcher.leaderOffsetsFor(Map(tp -> 2))(tp).endOffset())
//Then Epoch 4 should still map to offset 2
assertEquals(3, fetcher.leaderOffsetsFor(Map(tp -> 4))(tp).endOffset())
assertEquals(leo(), fetcher.leaderOffsetsFor(Map(tp -> 4))(tp).endOffset())
//Adding some extra assertions here to save test setup.
shouldSupportRequestsForEpochsNotOnTheLeader(fetcher)
}
//Appended onto the previous test to save on setup cost.
def shouldSupportRequestsForEpochsNotOnTheLeader(fetcher: TestFetcherThread): Unit = {
/**
* Asking for an epoch not present on the leader should return the
* next matching epoch, unless there isn't any, which should return
* undefined.
*/
val epoch1 = Map(t1p0 -> 1)
assertEquals(1, fetcher.leaderOffsetsFor(epoch1)(t1p0).endOffset())
val epoch3 = Map(t1p0 -> 3)
assertEquals(2, fetcher.leaderOffsetsFor(epoch3)(t1p0).endOffset())
val epoch5 = Map(t1p0 -> 5)
assertEquals(-1, fetcher.leaderOffsetsFor(epoch5)(t1p0).endOffset())
}
private def sender(from: KafkaServer, to: KafkaServer): BlockingSend = {
val endPoint = from.metadataCache.getAliveBrokers.find(_.id == to.config.brokerId).get.getBrokerEndPoint(from.config.interBrokerListenerName)
new ReplicaFetcherBlockingSend(endPoint, from.config, new Metrics(), new SystemTime(), 42, "TestFetcher")
}
private def waitForEpochChangeTo(topic: String, partition: Int, epoch: Int): Boolean = {
TestUtils.waitUntilTrue(() => {
brokers(0).metadataCache.getPartitionInfo(topic, partition) match {
case Some(m) => m.leaderIsrAndControllerEpoch.leaderAndIsr.leaderEpoch == epoch
case None => false
}
}, "Epoch didn't change")
}
private def messagesHaveLeaderEpoch(broker: KafkaServer, expectedLeaderEpoch: Int, minOffset: Int): Boolean = {
var result = true
for (topic <- List(topic1, topic2)) {
val tp = new TopicPartition(topic, 0)
val leo = broker.getLogManager().getLog(tp).get.logEndOffset
result = result && leo > 0 && brokers.forall { broker =>
broker.getLogManager().getLog(tp).get.logSegments.iterator.forall { segment =>
if (segment.read(minOffset, None, Integer.MAX_VALUE) == null) {
false
} else {
segment.read(minOffset, None, Integer.MAX_VALUE)
.records.batches().iterator().asScala.forall(
expectedLeaderEpoch == _.partitionLeaderEpoch()
)
}
}
}
}
result
}
private def sendFourMessagesToEachTopic() = {
val testMessageList1 = List("test1", "test2", "test3", "test4")
val testMessageList2 = List("test5", "test6", "test7", "test8")
val producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(brokers), retries = 5, keySerializer = new StringSerializer, valueSerializer = new StringSerializer)
val records =
testMessageList1.map(m => new ProducerRecord(topic1, m, m)) ++
testMessageList2.map(m => new ProducerRecord(topic2, m, m))
records.map(producer.send).foreach(_.get)
producer.close()
}
/**
* Simulates how the Replica Fetcher Thread requests leader offsets for epochs
*/
private class TestFetcherThread(sender: BlockingSend) extends Logging {
def leaderOffsetsFor(partitions: Map[TopicPartition, Int]): Map[TopicPartition, EpochEndOffset] = {
val request = new OffsetsForLeaderEpochRequest.Builder(toJavaFormat(partitions))
val response = sender.sendRequest(request)
response.responseBody.asInstanceOf[OffsetsForLeaderEpochResponse].responses.asScala
}
def toJavaFormat(partitions: Map[TopicPartition, Int]): JMap[TopicPartition, Integer] = {
partitions.map { case (tp, epoch) => tp -> epoch.asInstanceOf[Integer] }.toMap.asJava
}
}
}

View File

@ -0,0 +1,98 @@
/**
* 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.server.epoch
import kafka.server.OffsetsForLeaderEpoch
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.errors.{NotLeaderForPartitionException, UnknownTopicOrPartitionException}
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.requests.EpochEndOffset._
import org.apache.kafka.common.requests.EpochEndOffset
import org.easymock.EasyMock._
import org.junit.Test
import org.junit.Assert._
import scala.collection.JavaConverters._
import scala.collection.mutable
class OffsetsForLeaderEpochTest {
@Test
def shouldGetEpochsFromReplica(): Unit = {
val replicaManager = createNiceMock(classOf[kafka.server.ReplicaManager])
val replica = createNiceMock(classOf[kafka.cluster.Replica])
val cache = createNiceMock(classOf[kafka.server.epoch.LeaderEpochCache])
//Given
val tp = new TopicPartition("topic", 1)
val offset = 42
val epochRequested: Integer = 5
val request = mutable.Map(tp -> epochRequested).asJava
//Stubs
expect(replicaManager.getLeaderReplicaIfLocal(tp)).andReturn(replica)
expect(replica.epochs).andReturn(Some(cache))
expect(cache.endOffsetFor(epochRequested)).andReturn(offset)
replay(replica, replicaManager, cache)
//When
val response = OffsetsForLeaderEpoch.getResponseFor(replicaManager, request)
//Then
assertEquals(new EpochEndOffset(Errors.NONE, offset), response.get(tp))
}
@Test
def shonuldReturnNoLeaderForPartitionIfThrown(): Unit = {
val replicaManager = createNiceMock(classOf[kafka.server.ReplicaManager])
//Given
val tp = new TopicPartition("topic", 1)
val epochRequested: Integer = 5
val request = mutable.Map(tp -> epochRequested).asJava
//Stubs
expect(replicaManager.getLeaderReplicaIfLocal(tp)).andThrow(new NotLeaderForPartitionException())
replay(replicaManager)
//When
val response = OffsetsForLeaderEpoch.getResponseFor(replicaManager, request)
//Then
assertEquals(new EpochEndOffset(Errors.NOT_LEADER_FOR_PARTITION, UNDEFINED_EPOCH_OFFSET), response.get(tp))
}
@Test
def shouldReturnUnknownTopicOrPartitionIfThrown(): Unit = {
val replicaManager = createNiceMock(classOf[kafka.server.ReplicaManager])
//Given
val tp = new TopicPartition("topic", 1)
val epochRequested: Integer = 5
val request = mutable.Map(tp -> epochRequested).asJava
//Stubs
expect(replicaManager.getLeaderReplicaIfLocal(tp)).andThrow(new UnknownTopicOrPartitionException())
replay(replicaManager)
//When
val response = OffsetsForLeaderEpoch.getResponseFor(replicaManager, request)
//Then
assertEquals(new EpochEndOffset(Errors.UNKNOWN_TOPIC_OR_PARTITION, UNDEFINED_EPOCH_OFFSET), response.get(tp))
}
}

View File

@ -0,0 +1,80 @@
/**
* 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.server.epoch.util
import kafka.cluster.BrokerEndPoint
import kafka.server.BlockingSend
import org.apache.kafka.clients.{ClientRequest, ClientResponse, MockClient}
import org.apache.kafka.common.{Node, TopicPartition}
import org.apache.kafka.common.protocol.ApiKeys
import org.apache.kafka.common.requests.AbstractRequest.Builder
import org.apache.kafka.common.requests.FetchResponse.PartitionData
import org.apache.kafka.common.requests.{AbstractRequest, EpochEndOffset, FetchResponse, OffsetsForLeaderEpochResponse}
import org.apache.kafka.common.utils.{SystemTime, Time}
/**
* Stub network client used for testing the ReplicaFetcher, wraps the MockClient used for consumer testing
*/
class ReplicaFetcherMockBlockingSend(offsets: java.util.Map[TopicPartition, EpochEndOffset], destination: BrokerEndPoint, time: Time) extends BlockingSend {
private val client = new MockClient(new SystemTime)
var fetchCount = 0
var epochFetchCount = 0
var callback: Option[() => Unit] = None
def setEpochRequestCallback(postEpochFunction: () => Unit){
callback = Some(postEpochFunction)
}
override def sendRequest(requestBuilder: Builder[_ <: AbstractRequest]): ClientResponse = {
//Send the request to the mock client
val clientRequest = request(requestBuilder)
client.send(clientRequest, time.milliseconds())
//Create a suitable response based on the API key
val response = requestBuilder.apiKey() match {
case ApiKeys.OFFSET_FOR_LEADER_EPOCH =>
callback match {
case Some(f) => f()
case None => //nothing
}
epochFetchCount += 1
new OffsetsForLeaderEpochResponse(offsets)
case ApiKeys.FETCH =>
fetchCount += 1
new FetchResponse(new java.util.LinkedHashMap[TopicPartition, PartitionData], 0)
case _ =>
throw new UnsupportedOperationException
}
//Use mock client to create the appropriate response object
client.respondFrom(response, new Node(destination.id, destination.host, destination.port))
client.poll(30, time.milliseconds()).iterator().next()
}
private def request(requestBuilder: Builder[_ <: AbstractRequest]): ClientRequest = {
client.newClientRequest(
destination.id.toString,
requestBuilder,
time.milliseconds(),
true)
}
override def close(): Unit = {}
}

View File

@ -37,6 +37,7 @@ import kafka.producer._
import kafka.security.auth.{Acl, Authorizer, Resource}
import kafka.serializer.{DefaultEncoder, Encoder, StringEncoder}
import kafka.server._
import kafka.server.checkpoints.{OffsetCheckpoint, OffsetCheckpointFile}
import kafka.utils.ZkUtils._
import org.apache.kafka.clients.CommonClientConfigs
import org.apache.kafka.clients.consumer.{KafkaConsumer, RangeAssignor}
@ -1079,7 +1080,7 @@ object TestUtils extends Logging {
// ensure that topic is removed from all cleaner offsets
TestUtils.waitUntilTrue(() => servers.forall(server => topicPartitions.forall { tp =>
val checkpoints = server.getLogManager().logDirs.map { logDir =>
new OffsetCheckpoint(new File(logDir, "cleaner-offset-checkpoint")).read()
new OffsetCheckpointFile(new File(logDir, "cleaner-offset-checkpoint")).read()
}
checkpoints.forall(checkpointsPerLogDir => !checkpointsPerLogDir.contains(tp))
}), "Cleaner offset for deleted partition should have been removed")

View File

@ -93,7 +93,6 @@ class ZooKeeperSecurityUpgradeTest(ProduceConsumeValidateTest):
self.kafka.stop_node(node)
self.kafka.start_node(node)
@ignore
@cluster(num_nodes=9)
@matrix(security_protocol=["PLAINTEXT", "SSL", "SASL_SSL", "SASL_PLAINTEXT"])
def test_zk_security_upgrade(self, security_protocol):