mirror of https://github.com/apache/kafka.git
KIP-101: Alter Replication Protocol to use Leader Epoch rather than High Watermark for Truncation
This PR replaces https://github.com/apache/kafka/pull/2743 (just raising from Confluent repo) This PR describes the addition of Partition Level Leader Epochs to messages in Kafka as a mechanism for fixing some known issues in the replication protocol. Full details can be found here: [KIP-101 Reference](https://cwiki.apache.org/confluence/display/KAFKA/KIP-101+-+Alter+Replication+Protocol+to+use+Leader+Epoch+rather+than+High+Watermark+for+Truncation) *The key elements are*: - Epochs are stamped on messages as they enter the leader. - Epochs are tracked in both leader and follower in a new checkpoint file. - A new API allows followers to retrieve the leader's latest offset for a particular epoch. - The logic for truncating the log, when a replica becomes a follower, has been moved from Partition into the ReplicaFetcherThread - When partitions are added to the ReplicaFetcherThread they are added in an initialising state. Initialising partitions request leader epochs and then truncate their logs appropriately. This test provides a good overview of the workflow `EpochDrivenReplicationProtocolAcceptanceTest.shouldFollowLeaderEpochBasicWorkflow()` The corrupted log use case is covered by the test `EpochDrivenReplicationProtocolAcceptanceTest.offsetsShouldNotGoBackwards()` Remaining work: There is a do list here: https://docs.google.com/document/d/1edmMo70MfHEZH9x38OQfTWsHr7UGTvg-NOxeFhOeRew/edit?usp=sharing Author: Ben Stopford <benstopford@gmail.com> Author: Jun Rao <junrao@gmail.com> Reviewers: Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com> Closes #2808 from benstopford/kip-101-v2
This commit is contained in:
parent
b611cfa5c0
commit
0baea2ac13
|
@ -232,5 +232,7 @@
|
|||
|
||||
<suppress checks="NPathComplexity"
|
||||
files="KafkaLog4jAppender.java"/>
|
||||
<suppress checks="JavaNCSS"
|
||||
files="RequestResponseTest.java"/>
|
||||
|
||||
</suppressions>
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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()) {
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,175 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.kafka.common.requests;
|
||||
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.protocol.ApiKeys;
|
||||
import org.apache.kafka.common.protocol.Errors;
|
||||
import org.apache.kafka.common.protocol.types.Struct;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class OffsetsForLeaderEpochRequest extends AbstractRequest {
|
||||
public static final String TOPICS = "topics";
|
||||
public static final String TOPIC = "topic";
|
||||
public static final String PARTITIONS = "partitions";
|
||||
public static final String PARTITION_ID = "partition_id";
|
||||
public static final String LEADER_EPOCH = "leader_epoch";
|
||||
|
||||
private Map<TopicPartition, Integer> epochsByPartition;
|
||||
|
||||
public Map<TopicPartition, Integer> epochsByTopicPartition() {
|
||||
return epochsByPartition;
|
||||
}
|
||||
|
||||
public static class Builder extends AbstractRequest.Builder<OffsetsForLeaderEpochRequest> {
|
||||
private Map<TopicPartition, Integer> epochsByPartition = new HashMap();
|
||||
|
||||
public Builder() {
|
||||
super(ApiKeys.OFFSET_FOR_LEADER_EPOCH);
|
||||
}
|
||||
|
||||
public Builder(Map<TopicPartition, Integer> epochsByPartition) {
|
||||
super(ApiKeys.OFFSET_FOR_LEADER_EPOCH);
|
||||
this.epochsByPartition = epochsByPartition;
|
||||
}
|
||||
|
||||
public Builder add(TopicPartition topicPartition, Integer epoch) {
|
||||
epochsByPartition.put(topicPartition, epoch);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public OffsetsForLeaderEpochRequest build(short version) {
|
||||
return new OffsetsForLeaderEpochRequest(epochsByPartition, version);
|
||||
}
|
||||
|
||||
public static OffsetsForLeaderEpochRequest parse(ByteBuffer buffer, short version) {
|
||||
return new OffsetsForLeaderEpochRequest(ApiKeys.OFFSET_FOR_LEADER_EPOCH.parseRequest(version, buffer), version);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder bld = new StringBuilder();
|
||||
bld.append("(type=OffsetForLeaderEpochRequest, ").
|
||||
append("epochsByTopic=").append(epochsByPartition).
|
||||
append(")");
|
||||
return bld.toString();
|
||||
}
|
||||
}
|
||||
|
||||
public OffsetsForLeaderEpochRequest(Map<TopicPartition, Integer> epochsByPartition, short version) {
|
||||
super(version);
|
||||
this.epochsByPartition = epochsByPartition;
|
||||
}
|
||||
|
||||
public OffsetsForLeaderEpochRequest(Struct struct, short version) {
|
||||
super(version);
|
||||
epochsByPartition = new HashMap<>();
|
||||
for (Object t : struct.getArray(TOPICS)) {
|
||||
Struct topicAndEpochs = (Struct) t;
|
||||
String topic = topicAndEpochs.getString(TOPIC);
|
||||
for (Object e : topicAndEpochs.getArray(PARTITIONS)) {
|
||||
Struct partitionAndEpoch = (Struct) e;
|
||||
int partitionId = partitionAndEpoch.getInt(PARTITION_ID);
|
||||
int epoch = partitionAndEpoch.getInt(LEADER_EPOCH);
|
||||
TopicPartition tp = new TopicPartition(topic, partitionId);
|
||||
epochsByPartition.put(tp, epoch);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static OffsetsForLeaderEpochRequest parse(ByteBuffer buffer, short versionId) {
|
||||
return new OffsetsForLeaderEpochRequest(ApiKeys.OFFSET_FOR_LEADER_EPOCH.parseRequest(versionId, buffer), versionId);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Struct toStruct() {
|
||||
Struct struct = new Struct(ApiKeys.OFFSET_FOR_LEADER_EPOCH.requestSchema(version()));
|
||||
|
||||
//Group by topic
|
||||
Map<String, List<PartitionLeaderEpoch>> topicsToPartitionEpochs = new HashMap<>();
|
||||
for (TopicPartition tp : epochsByPartition.keySet()) {
|
||||
List<PartitionLeaderEpoch> partitionEndOffsets = topicsToPartitionEpochs.get(tp.topic());
|
||||
if (partitionEndOffsets == null)
|
||||
partitionEndOffsets = new ArrayList<>();
|
||||
partitionEndOffsets.add(new PartitionLeaderEpoch(tp.partition(), epochsByPartition.get(tp)));
|
||||
topicsToPartitionEpochs.put(tp.topic(), partitionEndOffsets);
|
||||
}
|
||||
|
||||
List<Struct> topics = new ArrayList<>();
|
||||
for (Map.Entry<String, List<PartitionLeaderEpoch>> topicEpochs : topicsToPartitionEpochs.entrySet()) {
|
||||
Struct partition = struct.instance(TOPICS);
|
||||
String topic = topicEpochs.getKey();
|
||||
partition.set(TOPIC, topic);
|
||||
List<PartitionLeaderEpoch> partitionLeaderEpoches = topicEpochs.getValue();
|
||||
List<Struct> partitions = new ArrayList<>(partitionLeaderEpoches.size());
|
||||
for (PartitionLeaderEpoch partitionLeaderEpoch : partitionLeaderEpoches) {
|
||||
Struct partitionRow = partition.instance(PARTITIONS);
|
||||
partitionRow.set(PARTITION_ID, partitionLeaderEpoch.partitionId);
|
||||
partitionRow.set(LEADER_EPOCH, partitionLeaderEpoch.epoch);
|
||||
partitions.add(partitionRow);
|
||||
}
|
||||
partition.set(PARTITIONS, partitions.toArray());
|
||||
topics.add(partition);
|
||||
}
|
||||
struct.set(TOPICS, topics.toArray());
|
||||
return struct;
|
||||
}
|
||||
|
||||
@Override
|
||||
public AbstractResponse getErrorResponse(Throwable e) {
|
||||
Errors error = Errors.forException(e);
|
||||
Map<TopicPartition, EpochEndOffset> errorResponse = new HashMap();
|
||||
for (TopicPartition tp : epochsByPartition.keySet()) {
|
||||
errorResponse.put(tp, new EpochEndOffset(error, EpochEndOffset.UNDEFINED_EPOCH_OFFSET));
|
||||
}
|
||||
return new OffsetsForLeaderEpochResponse(errorResponse);
|
||||
}
|
||||
|
||||
private class PartitionLeaderEpoch {
|
||||
int partitionId;
|
||||
int epoch;
|
||||
|
||||
public PartitionLeaderEpoch(int partitionId, int epoch) {
|
||||
this.partitionId = partitionId;
|
||||
this.epoch = epoch;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
|
||||
PartitionLeaderEpoch other = (PartitionLeaderEpoch) o;
|
||||
|
||||
if (partitionId != other.partitionId) return false;
|
||||
return epoch == other.epoch;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
int result = partitionId;
|
||||
result = 31 * result + epoch;
|
||||
return result;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,132 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.kafka.common.requests;
|
||||
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.protocol.ApiKeys;
|
||||
import org.apache.kafka.common.protocol.Errors;
|
||||
import org.apache.kafka.common.protocol.types.Struct;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class OffsetsForLeaderEpochResponse extends AbstractResponse {
|
||||
public static final String TOPICS = "topics";
|
||||
public static final String TOPIC = "topic";
|
||||
public static final String PARTITIONS = "partitions";
|
||||
public static final String ERROR_CODE = "error_code";
|
||||
public static final String PARTITION_ID = "partition_id";
|
||||
public static final String END_OFFSET = "end_offset";
|
||||
|
||||
private Map<TopicPartition, EpochEndOffset> epochEndOffsetsByPartition;
|
||||
|
||||
public OffsetsForLeaderEpochResponse(Struct struct) {
|
||||
epochEndOffsetsByPartition = new HashMap<>();
|
||||
for (Object t : struct.getArray(TOPICS)) {
|
||||
Struct topicAndEpochs = (Struct) t;
|
||||
String topic = topicAndEpochs.getString(TOPIC);
|
||||
for (Object e : topicAndEpochs.getArray(PARTITIONS)) {
|
||||
Struct partitionAndEpoch = (Struct) e;
|
||||
Errors error = Errors.forCode(partitionAndEpoch.getShort(ERROR_CODE));
|
||||
int partitionId = partitionAndEpoch.getInt(PARTITION_ID);
|
||||
TopicPartition tp = new TopicPartition(topic, partitionId);
|
||||
long endOffset = partitionAndEpoch.getLong(END_OFFSET);
|
||||
epochEndOffsetsByPartition.put(tp, new EpochEndOffset(error, endOffset));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public OffsetsForLeaderEpochResponse(Map<TopicPartition, EpochEndOffset> epochsByTopic) {
|
||||
this.epochEndOffsetsByPartition = epochsByTopic;
|
||||
}
|
||||
|
||||
public Map<TopicPartition, EpochEndOffset> responses() {
|
||||
return epochEndOffsetsByPartition;
|
||||
}
|
||||
|
||||
public static OffsetsForLeaderEpochResponse parse(ByteBuffer buffer, short versionId) {
|
||||
return new OffsetsForLeaderEpochResponse(ApiKeys.OFFSET_FOR_LEADER_EPOCH.responseSchema(versionId).read(buffer));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Struct toStruct(short version) {
|
||||
Struct struct = new Struct(ApiKeys.OFFSET_FOR_LEADER_EPOCH.responseSchema(version));
|
||||
|
||||
//Group by topic
|
||||
Map<String, List<PartitionEndOffset>> topicsToPartitionEndOffsets = new HashMap<>();
|
||||
for (TopicPartition tp : epochEndOffsetsByPartition.keySet()) {
|
||||
List<PartitionEndOffset> partitionEndOffsets = topicsToPartitionEndOffsets.get(tp.topic());
|
||||
if (partitionEndOffsets == null)
|
||||
partitionEndOffsets = new ArrayList<>();
|
||||
partitionEndOffsets.add(new PartitionEndOffset(tp.partition(), epochEndOffsetsByPartition.get(tp)));
|
||||
topicsToPartitionEndOffsets.put(tp.topic(), partitionEndOffsets);
|
||||
}
|
||||
|
||||
//Write struct
|
||||
List<Struct> topics = new ArrayList<>(topicsToPartitionEndOffsets.size());
|
||||
for (Map.Entry<String, List<PartitionEndOffset>> topicEpochs : topicsToPartitionEndOffsets.entrySet()) {
|
||||
Struct partition = struct.instance(TOPICS);
|
||||
String topic = topicEpochs.getKey();
|
||||
partition.set(TOPIC, topic);
|
||||
List<PartitionEndOffset> paritionEpochs = topicEpochs.getValue();
|
||||
List<Struct> paritions = new ArrayList<>(paritionEpochs.size());
|
||||
for (PartitionEndOffset peo : paritionEpochs) {
|
||||
Struct partitionRow = partition.instance(PARTITIONS);
|
||||
partitionRow.set(ERROR_CODE, peo.epochEndOffset.error().code());
|
||||
partitionRow.set(PARTITION_ID, peo.partition);
|
||||
partitionRow.set(END_OFFSET, peo.epochEndOffset.endOffset());
|
||||
paritions.add(partitionRow);
|
||||
}
|
||||
|
||||
partition.set(PARTITIONS, paritions.toArray());
|
||||
topics.add(partition);
|
||||
}
|
||||
struct.set(TOPICS, topics.toArray());
|
||||
return struct;
|
||||
}
|
||||
|
||||
private class PartitionEndOffset {
|
||||
private int partition;
|
||||
private EpochEndOffset epochEndOffset;
|
||||
|
||||
PartitionEndOffset(int partition, EpochEndOffset epochEndOffset) {
|
||||
this.partition = partition;
|
||||
this.epochEndOffset = epochEndOffset;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
|
||||
PartitionEndOffset that = (PartitionEndOffset) o;
|
||||
|
||||
if (partition != that.partition) return false;
|
||||
return epochEndOffset != null ? epochEndOffset.equals(that.epochEndOffset) : that.epochEndOffset == null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
int result = partition;
|
||||
result = 31 * result + (epochEndOffset != null ? epochEndOffset.hashCode() : 0);
|
||||
return result;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -163,6 +163,9 @@ public class RequestResponseTest {
|
|||
checkRequest(createListOffsetRequest(0));
|
||||
checkErrorResponse(createListOffsetRequest(0), new UnknownServerException());
|
||||
checkResponse(createListOffsetResponse(0), 0);
|
||||
checkRequest(createLeaderEpochRequest());
|
||||
checkResponse(createLeaderEpochResponse(), 0);
|
||||
checkErrorResponse(createLeaderEpochRequest(), new UnknownServerException());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -798,6 +801,26 @@ public class RequestResponseTest {
|
|||
return new InitPidResponse(Errors.NONE, 3332, (short) 3);
|
||||
}
|
||||
|
||||
private OffsetsForLeaderEpochRequest createLeaderEpochRequest() {
|
||||
Map<TopicPartition, Integer> epochs = new HashMap<>();
|
||||
|
||||
epochs.put(new TopicPartition("topic1", 0), 1);
|
||||
epochs.put(new TopicPartition("topic1", 1), 1);
|
||||
epochs.put(new TopicPartition("topic2", 2), 3);
|
||||
|
||||
return new OffsetsForLeaderEpochRequest.Builder(epochs).build();
|
||||
}
|
||||
|
||||
private OffsetsForLeaderEpochResponse createLeaderEpochResponse() {
|
||||
Map<TopicPartition, EpochEndOffset> epochs = new HashMap<>();
|
||||
|
||||
epochs.put(new TopicPartition("topic1", 0), new EpochEndOffset(Errors.NONE, 0));
|
||||
epochs.put(new TopicPartition("topic1", 1), new EpochEndOffset(Errors.NONE, 1));
|
||||
epochs.put(new TopicPartition("topic2", 2), new EpochEndOffset(Errors.NONE, 2));
|
||||
|
||||
return new OffsetsForLeaderEpochResponse(epochs);
|
||||
}
|
||||
|
||||
private static class ByteBufferChannel implements GatheringByteChannel {
|
||||
private final ByteBuffer buf;
|
||||
private boolean closed = false;
|
||||
|
|
|
@ -25,14 +25,21 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
public class MockDeserializer implements ClusterResourceListener, Deserializer<byte[]> {
|
||||
public static final AtomicInteger INIT_COUNT = new AtomicInteger(0);
|
||||
public static final AtomicInteger CLOSE_COUNT = new AtomicInteger(0);
|
||||
public static final AtomicReference<ClusterResource> CLUSTER_META = new AtomicReference<>();
|
||||
public static final ClusterResource NO_CLUSTER_ID = new ClusterResource("no_cluster_id");
|
||||
public static final AtomicReference<ClusterResource> CLUSTER_ID_BEFORE_DESERIALIZE = new AtomicReference<>(NO_CLUSTER_ID);
|
||||
public static AtomicInteger initCount = new AtomicInteger(0);
|
||||
public static AtomicInteger closeCount = new AtomicInteger(0);
|
||||
public static AtomicReference<ClusterResource> clusterMeta = new AtomicReference<>();
|
||||
public static ClusterResource noClusterId = new ClusterResource("no_cluster_id");
|
||||
public static AtomicReference<ClusterResource> clusterIdBeforeDeserialize = new AtomicReference<>(noClusterId);
|
||||
|
||||
public static void resetStaticVariables() {
|
||||
initCount = new AtomicInteger(0);
|
||||
closeCount = new AtomicInteger(0);
|
||||
clusterMeta = new AtomicReference<>();
|
||||
clusterIdBeforeDeserialize = new AtomicReference<>(noClusterId);
|
||||
}
|
||||
|
||||
public MockDeserializer() {
|
||||
INIT_COUNT.incrementAndGet();
|
||||
initCount.incrementAndGet();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -43,17 +50,17 @@ public class MockDeserializer implements ClusterResourceListener, Deserializer<b
|
|||
public byte[] deserialize(String topic, byte[] data) {
|
||||
// This will ensure that we get the cluster metadata when deserialize is called for the first time
|
||||
// as subsequent compareAndSet operations will fail.
|
||||
CLUSTER_ID_BEFORE_DESERIALIZE.compareAndSet(NO_CLUSTER_ID, CLUSTER_META.get());
|
||||
clusterIdBeforeDeserialize.compareAndSet(noClusterId, clusterMeta.get());
|
||||
return data;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
CLOSE_COUNT.incrementAndGet();
|
||||
closeCount.incrementAndGet();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onUpdate(ClusterResource clusterResource) {
|
||||
CLUSTER_META.set(clusterResource);
|
||||
clusterMeta.set(clusterResource);
|
||||
}
|
||||
}
|
|
@ -13,7 +13,7 @@
|
|||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
log4j.rootLogger=INFO, stdout
|
||||
log4j.rootLogger=INFO, stdout
|
||||
|
||||
log4j.appender.stdout=org.apache.log4j.ConsoleAppender
|
||||
log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
|
||||
|
|
|
@ -67,7 +67,10 @@ object ApiVersion {
|
|||
"0.11.0-IV0" -> KAFKA_0_11_0_IV0,
|
||||
// introduced DeleteRecordsRequest v0 and FetchRequest v4 in KIP-107
|
||||
"0.11.0-IV1" -> KAFKA_0_11_0_IV1,
|
||||
"0.11.0" -> KAFKA_0_11_0_IV1
|
||||
"0.11.0" -> KAFKA_0_11_0_IV1,
|
||||
// Introduce leader epoch fetches to the replica fetcher via KIP-101
|
||||
"0.11.0-IV2" -> KAFKA_0_11_0_IV2,
|
||||
"0.11.0" -> KAFKA_0_11_0_IV2
|
||||
)
|
||||
|
||||
private val versionPattern = "\\.".r
|
||||
|
@ -159,7 +162,13 @@ case object KAFKA_0_11_0_IV0 extends ApiVersion {
|
|||
}
|
||||
|
||||
case object KAFKA_0_11_0_IV1 extends ApiVersion {
|
||||
val version: String = "0.11.0-IV0"
|
||||
val version: String = "0.11.0-IV1"
|
||||
val messageFormatVersion: Byte = RecordBatch.MAGIC_VALUE_V2
|
||||
val id: Int = 10
|
||||
val id: Int = 11
|
||||
}
|
||||
|
||||
case object KAFKA_0_11_0_IV2 extends ApiVersion {
|
||||
val version: String = "0.11.0-IV2"
|
||||
val messageFormatVersion: Byte = RecordBatch.MAGIC_VALUE_V2
|
||||
val id: Int = 12
|
||||
}
|
|
@ -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) {
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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]()
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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 =>
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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.")
|
||||
|
|
|
@ -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()
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
}
|
|
@ -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
|
||||
}
|
||||
}
|
|
@ -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 {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
|
@ -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()
|
||||
}
|
||||
}
|
|
@ -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
|
||||
}
|
||||
}
|
|
@ -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)
|
|
@ -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) {
|
||||
|
|
|
@ -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)
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)))
|
||||
}
|
||||
|
|
|
@ -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) => {
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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()
|
||||
}
|
||||
}
|
|
@ -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()
|
||||
|
|
|
@ -242,4 +242,4 @@ class ReplicationQuotasTest extends ZooKeeperTestHarness {
|
|||
val metricName = broker.metrics.metricName("byte-rate", repType.toString)
|
||||
broker.metrics.metrics.asScala(metricName).value
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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(
|
||||
|
|
|
@ -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())
|
||||
}
|
||||
}
|
|
@ -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())
|
||||
}
|
||||
}
|
|
@ -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 = {}
|
||||
}
|
||||
}
|
|
@ -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())
|
||||
}
|
||||
}
|
|
@ -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
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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))
|
||||
}
|
||||
}
|
|
@ -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 = {}
|
||||
}
|
|
@ -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")
|
||||
|
|
|
@ -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):
|
||||
|
|
Loading…
Reference in New Issue