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):