diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml
index 7bc55c8b65f..39d61e2faf9 100644
--- a/checkstyle/suppressions.xml
+++ b/checkstyle/suppressions.xml
@@ -232,5 +232,7 @@
+
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java
index 4183638e28d..b65defb6979 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java
@@ -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;
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
index 37eb75c2473..cc228c50153 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
@@ -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()) {
diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
index f810e394421..b3beed55180 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
@@ -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());
}
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java
index 1638556d4b1..7ce3518a9ae 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java
@@ -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));
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java
index 314aa42ce49..1ae30d1c8eb 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractResponse.java
@@ -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));
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/EpochEndOffset.java b/clients/src/main/java/org/apache/kafka/common/requests/EpochEndOffset.java
new file mode 100644
index 00000000000..2d49149d02e
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/EpochEndOffset.java
@@ -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;
+ }
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochRequest.java
new file mode 100644
index 00000000000..fed891073fc
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochRequest.java
@@ -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 epochsByPartition;
+
+ public Map epochsByTopicPartition() {
+ return epochsByPartition;
+ }
+
+ public static class Builder extends AbstractRequest.Builder {
+ private Map epochsByPartition = new HashMap();
+
+ public Builder() {
+ super(ApiKeys.OFFSET_FOR_LEADER_EPOCH);
+ }
+
+ public Builder(Map 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 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> topicsToPartitionEpochs = new HashMap<>();
+ for (TopicPartition tp : epochsByPartition.keySet()) {
+ List 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 topics = new ArrayList<>();
+ for (Map.Entry> topicEpochs : topicsToPartitionEpochs.entrySet()) {
+ Struct partition = struct.instance(TOPICS);
+ String topic = topicEpochs.getKey();
+ partition.set(TOPIC, topic);
+ List partitionLeaderEpoches = topicEpochs.getValue();
+ List 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 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;
+ }
+ }
+}
\ No newline at end of file
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochResponse.java
new file mode 100644
index 00000000000..03f3069a478
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetsForLeaderEpochResponse.java
@@ -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 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 epochsByTopic) {
+ this.epochEndOffsetsByPartition = epochsByTopic;
+ }
+
+ public Map 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> topicsToPartitionEndOffsets = new HashMap<>();
+ for (TopicPartition tp : epochEndOffsetsByPartition.keySet()) {
+ List 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 topics = new ArrayList<>(topicsToPartitionEndOffsets.size());
+ for (Map.Entry> topicEpochs : topicsToPartitionEndOffsets.entrySet()) {
+ Struct partition = struct.instance(TOPICS);
+ String topic = topicEpochs.getKey();
+ partition.set(TOPIC, topic);
+ List paritionEpochs = topicEpochs.getValue();
+ List 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;
+ }
+ }
+}
\ No newline at end of file
diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
index 8a7633e0498..2995882d29c 100644
--- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
@@ -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 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 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;
diff --git a/clients/src/test/java/org/apache/kafka/test/MockDeserializer.java b/clients/src/test/java/org/apache/kafka/test/MockDeserializer.java
index 97042069260..99551f718a9 100644
--- a/clients/src/test/java/org/apache/kafka/test/MockDeserializer.java
+++ b/clients/src/test/java/org/apache/kafka/test/MockDeserializer.java
@@ -25,14 +25,21 @@ import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
public class MockDeserializer implements ClusterResourceListener, Deserializer {
- public static final AtomicInteger INIT_COUNT = new AtomicInteger(0);
- public static final AtomicInteger CLOSE_COUNT = new AtomicInteger(0);
- public static final AtomicReference CLUSTER_META = new AtomicReference<>();
- public static final ClusterResource NO_CLUSTER_ID = new ClusterResource("no_cluster_id");
- public static final AtomicReference 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 clusterMeta = new AtomicReference<>();
+ public static ClusterResource noClusterId = new ClusterResource("no_cluster_id");
+ public static AtomicReference 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 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
}
\ No newline at end of file
diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala
index ddb24114e3a..4dd96c36488 100755
--- a/core/src/main/scala/kafka/cluster/Partition.scala
+++ b/core/src/main/scala/kafka/cluster/Partition.scala
@@ -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) {
diff --git a/core/src/main/scala/kafka/cluster/Replica.scala b/core/src/main/scala/kafka/cluster/Replica.scala
index 3995f9ef994..a604b87977b 100644
--- a/core/src/main/scala/kafka/cluster/Replica.scala
+++ b/core/src/main/scala/kafka/cluster/Replica.scala
@@ -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.
diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala
index c4b7ce6f9f4..8d712f4c1ba 100644
--- a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala
+++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala
@@ -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 {
diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala
index a052a9e0008..999c6aa6f28 100644
--- a/core/src/main/scala/kafka/log/Log.scala
+++ b/core/src/main/scala/kafka/log/Log.scala
@@ -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()
- // remove the segments for lookups
- deletable.foreach(deleteSegment)
- logStartOffset = math.max(logStartOffset, segments.firstEntry().getValue.baseOffset)
+ lock synchronized {
+ // remove the segments for lookups
+ 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
}
diff --git a/core/src/main/scala/kafka/log/LogCleanerManager.scala b/core/src/main/scala/kafka/log/LogCleanerManager.scala
index 01c4df4bb16..2b4d956d72d 100755
--- a/core/src/main/scala/kafka/log/LogCleanerManager.scala
+++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala
@@ -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]()
diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala
index ec164e2dff1..469c46b7a2d 100755
--- a/core/src/main/scala/kafka/log/LogManager.scala
+++ b/core/src/main/scala/kafka/log/LogManager.scala
@@ -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
diff --git a/core/src/main/scala/kafka/log/LogSegment.scala b/core/src/main/scala/kafka/log/LogSegment.scala
index 4e77625761b..b77be34d663 100755
--- a/core/src/main/scala/kafka/log/LogSegment.scala
+++ b/core/src/main/scala/kafka/log/LogSegment.scala
@@ -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 =>
diff --git a/core/src/main/scala/kafka/log/LogValidator.scala b/core/src/main/scala/kafka/log/LogValidator.scala
index c01a5debf3b..fa520ad8dc3 100644
--- a/core/src/main/scala/kafka/log/LogValidator.scala
+++ b/core/src/main/scala/kafka/log/LogValidator.scala
@@ -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)
diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
index 14e56bd665f..734c0063d73 100755
--- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
+++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
@@ -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)
}
diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala
index 600b84de5d6..1e8900b275b 100644
--- a/core/src/main/scala/kafka/server/KafkaApis.scala
+++ b/core/src/main/scala/kafka/server/KafkaApis.scala
@@ -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.")
diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherBlockingSend.scala b/core/src/main/scala/kafka/server/ReplicaFetcherBlockingSend.scala
new file mode 100644
index 00000000000..8ba3f600b49
--- /dev/null
+++ b/core/src/main/scala/kafka/server/ReplicaFetcherBlockingSend.scala
@@ -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()
+ }
+}
\ No newline at end of file
diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
index cce59ce2e2c..94ed66cc4e1 100644
--- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
+++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
@@ -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)
}
-
}
-
-}
+}
\ No newline at end of file
diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala
index 8f67425208c..b063b9e7936 100644
--- a/core/src/main/scala/kafka/server/ReplicaManager.scala
+++ b/core/src/main/scala/kafka/server/ReplicaManager.scala
@@ -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
+ }
+}
\ No newline at end of file
diff --git a/core/src/main/scala/kafka/server/OffsetCheckpoint.scala b/core/src/main/scala/kafka/server/checkpoints/CheckpointFile.scala
similarity index 52%
rename from core/src/main/scala/kafka/server/OffsetCheckpoint.scala
rename to core/src/main/scala/kafka/server/checkpoints/CheckpointFile.scala
index de2626cd2d1..890dde0e993 100644
--- a/core/src/main/scala/kafka/server/OffsetCheckpoint.scala
+++ b/core/src/main/scala/kafka/server/checkpoints/CheckpointFile.scala
@@ -1,63 +1,54 @@
/**
- * 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
+ * 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.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 {
}
}
}
-
-}
+}
\ No newline at end of file
diff --git a/core/src/main/scala/kafka/server/checkpoints/LeaderEpochCheckpointFile.scala b/core/src/main/scala/kafka/server/checkpoints/LeaderEpochCheckpointFile.scala
new file mode 100644
index 00000000000..9de75646c90
--- /dev/null
+++ b/core/src/main/scala/kafka/server/checkpoints/LeaderEpochCheckpointFile.scala
@@ -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()
+ }
+}
\ No newline at end of file
diff --git a/core/src/main/scala/kafka/server/checkpoints/OffsetCheckpointFile.scala b/core/src/main/scala/kafka/server/checkpoints/OffsetCheckpointFile.scala
new file mode 100644
index 00000000000..12ec98636d0
--- /dev/null
+++ b/core/src/main/scala/kafka/server/checkpoints/OffsetCheckpointFile.scala
@@ -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
+ }
+}
\ No newline at end of file
diff --git a/core/src/main/scala/kafka/server/epoch/LeaderEpochFileCache.scala b/core/src/main/scala/kafka/server/epoch/LeaderEpochFileCache.scala
new file mode 100644
index 00000000000..4a4727ea309
--- /dev/null
+++ b/core/src/main/scala/kafka/server/epoch/LeaderEpochFileCache.scala
@@ -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)
\ No newline at end of file
diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
index e4cece9b9e7..ec3eb88a17d 100644
--- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
@@ -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) {
diff --git a/core/src/test/scala/integration/kafka/api/EndToEndClusterIdTest.scala b/core/src/test/scala/integration/kafka/api/EndToEndClusterIdTest.scala
index d885d9b2894..6a4c552d132 100644
--- a/core/src/test/scala/integration/kafka/api/EndToEndClusterIdTest.scala
+++ b/core/src/test/scala/integration/kafka/api/EndToEndClusterIdTest.scala
@@ -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)
diff --git a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala
index 874637bcca7..b16591879be 100755
--- a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala
+++ b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala
@@ -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
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
index 2104842030f..5aeea89ca01 100755
--- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
@@ -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)))
}
diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
index 1400615fbe6..1248d1a5e1c 100755
--- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
@@ -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) => {
diff --git a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala
index 25b348009c7..3f531d97275 100644
--- a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala
@@ -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)
+ }
+
}
diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala
index a7af24e18b5..4fcf1c3e9f3 100755
--- a/core/src/test/scala/unit/kafka/log/LogTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogTest.scala
@@ -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
+ }
}
diff --git a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
index 3babfc831f0..00cda21fb97 100644
--- a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
+++ b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
@@ -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)
diff --git a/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala
index 3898d2b11ed..0c62a50bd51 100644
--- a/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala
@@ -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
}
diff --git a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala
index 40ac7eccd22..54cee6bbd5d 100755
--- a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala
+++ b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala
@@ -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
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala
new file mode 100644
index 00000000000..d4118c189e0
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala
@@ -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()
+ }
+}
\ No newline at end of file
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala
index a720a6a89ec..d6b1649611b 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala
@@ -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()
diff --git a/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala
index f8680325e18..15e77a0cf47 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala
@@ -242,4 +242,4 @@ class ReplicationQuotasTest extends ZooKeeperTestHarness {
val metricName = broker.metrics.metricName("byte-rate", repType.toString)
broker.metrics.metrics.asScala(metricName).value
}
-}
+}
\ No newline at end of file
diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
index 0129d5de1db..ba17db68e31 100644
--- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
+++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
@@ -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(
diff --git a/core/src/test/scala/unit/kafka/server/checkpoints/LeaderEpochCheckpointFileTest.scala b/core/src/test/scala/unit/kafka/server/checkpoints/LeaderEpochCheckpointFileTest.scala
new file mode 100644
index 00000000000..e7c6a9785bc
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/server/checkpoints/LeaderEpochCheckpointFileTest.scala
@@ -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())
+ }
+}
\ No newline at end of file
diff --git a/core/src/test/scala/unit/kafka/server/checkpoints/OffsetCheckpointFileTest.scala b/core/src/test/scala/unit/kafka/server/checkpoints/OffsetCheckpointFileTest.scala
new file mode 100644
index 00000000000..cc49ccf3c74
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/server/checkpoints/OffsetCheckpointFileTest.scala
@@ -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())
+ }
+}
diff --git a/core/src/test/scala/unit/kafka/server/epoch/EpochDrivenReplicationProtocolAcceptanceTest.scala b/core/src/test/scala/unit/kafka/server/epoch/EpochDrivenReplicationProtocolAcceptanceTest.scala
new file mode 100644
index 00000000000..a09e1ccda94
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/server/epoch/EpochDrivenReplicationProtocolAcceptanceTest.scala
@@ -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 = {}
+ }
+}
\ No newline at end of file
diff --git a/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochFileCacheTest.scala b/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochFileCacheTest.scala
new file mode 100644
index 00000000000..1a24c347d64
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochFileCacheTest.scala
@@ -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())
+ }
+}
\ No newline at end of file
diff --git a/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala b/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala
new file mode 100644
index 00000000000..c5bb5e4453a
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala
@@ -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
+ }
+ }
+}
\ No newline at end of file
diff --git a/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala b/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala
new file mode 100644
index 00000000000..77b90689195
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/server/epoch/OffsetsForLeaderEpochTest.scala
@@ -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))
+ }
+}
\ No newline at end of file
diff --git a/core/src/test/scala/unit/kafka/server/epoch/util/ReplicaFetcherMockBlockingSend.scala b/core/src/test/scala/unit/kafka/server/epoch/util/ReplicaFetcherMockBlockingSend.scala
new file mode 100644
index 00000000000..e04bd956b34
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/server/epoch/util/ReplicaFetcherMockBlockingSend.scala
@@ -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 = {}
+}
\ No newline at end of file
diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
index 87668553986..3dbe2dee9ec 100755
--- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala
+++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
@@ -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")
diff --git a/tests/kafkatest/tests/core/zookeeper_security_upgrade_test.py b/tests/kafkatest/tests/core/zookeeper_security_upgrade_test.py
index 155eb7d31bf..e5e140b6bcf 100644
--- a/tests/kafkatest/tests/core/zookeeper_security_upgrade_test.py
+++ b/tests/kafkatest/tests/core/zookeeper_security_upgrade_test.py
@@ -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):