mirror of https://github.com/apache/kafka.git
KAFKA-17002: Integrated partition leader epoch for Persister APIs (KIP-932) (#16842)
The PR integrates leader epoch for partition while invoking Persister APIs. The write RPC is retried once on leader epoch failure. Reviewers: Abhinav Dixit <adixit@confluent.io>, Andrew Schofield <aschofield@confluent.io>, Jun Rao <junrao@gmail.com>, David Arthur <mumrah@gmail.com>
This commit is contained in:
parent
fb65dfeb11
commit
ff116df015
|
@ -127,7 +127,7 @@ public class DelayedShareFetch extends DelayedOperation {
|
||||||
shareFetchData.future().complete(result);
|
shareFetchData.future().complete(result);
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
log.error("Error processing delayed share fetch request", e);
|
log.error("Error processing delayed share fetch request", e);
|
||||||
shareFetchData.future().completeExceptionally(e);
|
sharePartitionManager.handleFetchException(shareFetchData.groupId(), topicPartitionData.keySet(), shareFetchData.future(), e);
|
||||||
} finally {
|
} finally {
|
||||||
// Releasing the lock to move ahead with the next request in queue.
|
// Releasing the lock to move ahead with the next request in queue.
|
||||||
releasePartitionLocks(shareFetchData.groupId(), topicPartitionData.keySet());
|
releasePartitionLocks(shareFetchData.groupId(), topicPartitionData.keySet());
|
||||||
|
|
|
@ -16,9 +16,12 @@
|
||||||
*/
|
*/
|
||||||
package kafka.server.share;
|
package kafka.server.share;
|
||||||
|
|
||||||
|
import kafka.cluster.Partition;
|
||||||
import kafka.server.ReplicaManager;
|
import kafka.server.ReplicaManager;
|
||||||
|
|
||||||
import org.apache.kafka.common.TopicIdPartition;
|
import org.apache.kafka.common.TopicIdPartition;
|
||||||
|
import org.apache.kafka.common.TopicPartition;
|
||||||
|
import org.apache.kafka.common.errors.NotLeaderOrFollowerException;
|
||||||
import org.apache.kafka.common.message.ShareFetchResponseData;
|
import org.apache.kafka.common.message.ShareFetchResponseData;
|
||||||
import org.apache.kafka.common.protocol.Errors;
|
import org.apache.kafka.common.protocol.Errors;
|
||||||
import org.apache.kafka.common.record.FileRecords;
|
import org.apache.kafka.common.record.FileRecords;
|
||||||
|
@ -128,4 +131,13 @@ public class ShareFetchUtils {
|
||||||
Optional.empty(), true).timestampAndOffsetOpt();
|
Optional.empty(), true).timestampAndOffsetOpt();
|
||||||
return timestampAndOffset.isEmpty() ? (long) 0 : timestampAndOffset.get().offset;
|
return timestampAndOffset.isEmpty() ? (long) 0 : timestampAndOffset.get().offset;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
static int leaderEpoch(ReplicaManager replicaManager, TopicPartition tp) {
|
||||||
|
Partition partition = replicaManager.getPartitionOrException(tp);
|
||||||
|
if (!partition.isLeader()) {
|
||||||
|
log.debug("The broker is not the leader for topic partition: {}-{}", tp.topic(), tp.partition());
|
||||||
|
throw new NotLeaderOrFollowerException();
|
||||||
|
}
|
||||||
|
return partition.getLeaderEpoch();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -23,11 +23,13 @@ import org.apache.kafka.common.TopicIdPartition;
|
||||||
import org.apache.kafka.common.Uuid;
|
import org.apache.kafka.common.Uuid;
|
||||||
import org.apache.kafka.common.errors.CoordinatorNotAvailableException;
|
import org.apache.kafka.common.errors.CoordinatorNotAvailableException;
|
||||||
import org.apache.kafka.common.errors.FencedStateEpochException;
|
import org.apache.kafka.common.errors.FencedStateEpochException;
|
||||||
|
import org.apache.kafka.common.errors.GroupIdNotFoundException;
|
||||||
import org.apache.kafka.common.errors.InvalidRecordStateException;
|
import org.apache.kafka.common.errors.InvalidRecordStateException;
|
||||||
import org.apache.kafka.common.errors.InvalidRequestException;
|
import org.apache.kafka.common.errors.InvalidRequestException;
|
||||||
import org.apache.kafka.common.errors.LeaderNotAvailableException;
|
import org.apache.kafka.common.errors.LeaderNotAvailableException;
|
||||||
import org.apache.kafka.common.errors.NotLeaderOrFollowerException;
|
import org.apache.kafka.common.errors.NotLeaderOrFollowerException;
|
||||||
import org.apache.kafka.common.errors.UnknownServerException;
|
import org.apache.kafka.common.errors.UnknownServerException;
|
||||||
|
import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
|
||||||
import org.apache.kafka.common.message.ShareFetchResponseData.AcquiredRecords;
|
import org.apache.kafka.common.message.ShareFetchResponseData.AcquiredRecords;
|
||||||
import org.apache.kafka.common.protocol.Errors;
|
import org.apache.kafka.common.protocol.Errors;
|
||||||
import org.apache.kafka.common.record.RecordBatch;
|
import org.apache.kafka.common.record.RecordBatch;
|
||||||
|
@ -103,7 +105,11 @@ public class SharePartition {
|
||||||
/**
|
/**
|
||||||
* The share partition failed to initialize with persisted state.
|
* The share partition failed to initialize with persisted state.
|
||||||
*/
|
*/
|
||||||
FAILED
|
FAILED,
|
||||||
|
/**
|
||||||
|
* The share partition is fenced and cannot be used.
|
||||||
|
*/
|
||||||
|
FENCED
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -181,6 +187,11 @@ public class SharePartition {
|
||||||
*/
|
*/
|
||||||
private final TopicIdPartition topicIdPartition;
|
private final TopicIdPartition topicIdPartition;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The leader epoch is used to track the partition epoch.
|
||||||
|
*/
|
||||||
|
private final int leaderEpoch;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The in-flight record is used to track the state of a record that has been fetched from the
|
* The in-flight record is used to track the state of a record that has been fetched from the
|
||||||
* leader. The state of the record is used to determine if the record should be re-fetched or if it
|
* leader. The state of the record is used to determine if the record should be re-fetched or if it
|
||||||
|
@ -280,6 +291,7 @@ public class SharePartition {
|
||||||
SharePartition(
|
SharePartition(
|
||||||
String groupId,
|
String groupId,
|
||||||
TopicIdPartition topicIdPartition,
|
TopicIdPartition topicIdPartition,
|
||||||
|
int leaderEpoch,
|
||||||
int maxInFlightMessages,
|
int maxInFlightMessages,
|
||||||
int maxDeliveryCount,
|
int maxDeliveryCount,
|
||||||
int defaultRecordLockDurationMs,
|
int defaultRecordLockDurationMs,
|
||||||
|
@ -288,9 +300,28 @@ public class SharePartition {
|
||||||
Persister persister,
|
Persister persister,
|
||||||
ReplicaManager replicaManager,
|
ReplicaManager replicaManager,
|
||||||
GroupConfigManager groupConfigManager
|
GroupConfigManager groupConfigManager
|
||||||
|
) {
|
||||||
|
this(groupId, topicIdPartition, leaderEpoch, maxInFlightMessages, maxDeliveryCount, defaultRecordLockDurationMs,
|
||||||
|
timer, time, persister, replicaManager, groupConfigManager, SharePartitionState.EMPTY);
|
||||||
|
}
|
||||||
|
|
||||||
|
SharePartition(
|
||||||
|
String groupId,
|
||||||
|
TopicIdPartition topicIdPartition,
|
||||||
|
int leaderEpoch,
|
||||||
|
int maxInFlightMessages,
|
||||||
|
int maxDeliveryCount,
|
||||||
|
int defaultRecordLockDurationMs,
|
||||||
|
Timer timer,
|
||||||
|
Time time,
|
||||||
|
Persister persister,
|
||||||
|
ReplicaManager replicaManager,
|
||||||
|
GroupConfigManager groupConfigManager,
|
||||||
|
SharePartitionState sharePartitionState
|
||||||
) {
|
) {
|
||||||
this.groupId = groupId;
|
this.groupId = groupId;
|
||||||
this.topicIdPartition = topicIdPartition;
|
this.topicIdPartition = topicIdPartition;
|
||||||
|
this.leaderEpoch = leaderEpoch;
|
||||||
this.maxInFlightMessages = maxInFlightMessages;
|
this.maxInFlightMessages = maxInFlightMessages;
|
||||||
this.maxDeliveryCount = maxDeliveryCount;
|
this.maxDeliveryCount = maxDeliveryCount;
|
||||||
this.cachedState = new ConcurrentSkipListMap<>();
|
this.cachedState = new ConcurrentSkipListMap<>();
|
||||||
|
@ -301,7 +332,7 @@ public class SharePartition {
|
||||||
this.timer = timer;
|
this.timer = timer;
|
||||||
this.time = time;
|
this.time = time;
|
||||||
this.persister = persister;
|
this.persister = persister;
|
||||||
this.partitionState = SharePartitionState.EMPTY;
|
this.partitionState = sharePartitionState;
|
||||||
this.replicaManager = replicaManager;
|
this.replicaManager = replicaManager;
|
||||||
this.groupConfigManager = groupConfigManager;
|
this.groupConfigManager = groupConfigManager;
|
||||||
}
|
}
|
||||||
|
@ -341,7 +372,7 @@ public class SharePartition {
|
||||||
.setGroupTopicPartitionData(new GroupTopicPartitionData.Builder<PartitionIdLeaderEpochData>()
|
.setGroupTopicPartitionData(new GroupTopicPartitionData.Builder<PartitionIdLeaderEpochData>()
|
||||||
.setGroupId(this.groupId)
|
.setGroupId(this.groupId)
|
||||||
.setTopicsData(Collections.singletonList(new TopicData<>(topicIdPartition.topicId(),
|
.setTopicsData(Collections.singletonList(new TopicData<>(topicIdPartition.topicId(),
|
||||||
Collections.singletonList(PartitionFactory.newPartitionIdLeaderEpochData(topicIdPartition.partition(), 0)))))
|
Collections.singletonList(PartitionFactory.newPartitionIdLeaderEpochData(topicIdPartition.partition(), leaderEpoch)))))
|
||||||
.build())
|
.build())
|
||||||
.build()
|
.build()
|
||||||
).whenComplete((result, exception) -> {
|
).whenComplete((result, exception) -> {
|
||||||
|
@ -520,13 +551,14 @@ public class SharePartition {
|
||||||
* @param fetchPartitionData The fetched records for the share partition.
|
* @param fetchPartitionData The fetched records for the share partition.
|
||||||
* @return The acquired records for the share partition.
|
* @return The acquired records for the share partition.
|
||||||
*/
|
*/
|
||||||
|
@SuppressWarnings("cyclomaticcomplexity") // Consider refactoring to avoid suppression
|
||||||
public ShareAcquiredRecords acquire(
|
public ShareAcquiredRecords acquire(
|
||||||
String memberId,
|
String memberId,
|
||||||
int maxFetchRecords,
|
int maxFetchRecords,
|
||||||
FetchPartitionData fetchPartitionData
|
FetchPartitionData fetchPartitionData
|
||||||
) {
|
) {
|
||||||
log.trace("Received acquire request for share partition: {}-{} memberId: {}", groupId, topicIdPartition, memberId);
|
log.trace("Received acquire request for share partition: {}-{} memberId: {}", groupId, topicIdPartition, memberId);
|
||||||
if (maxFetchRecords <= 0) {
|
if (stateNotActive() || maxFetchRecords <= 0) {
|
||||||
// Nothing to acquire.
|
// Nothing to acquire.
|
||||||
return ShareAcquiredRecords.empty();
|
return ShareAcquiredRecords.empty();
|
||||||
}
|
}
|
||||||
|
@ -1040,7 +1072,7 @@ public class SharePartition {
|
||||||
* @return A boolean which indicates whether the fetch lock is acquired.
|
* @return A boolean which indicates whether the fetch lock is acquired.
|
||||||
*/
|
*/
|
||||||
boolean maybeAcquireFetchLock() {
|
boolean maybeAcquireFetchLock() {
|
||||||
if (partitionState() != SharePartitionState.ACTIVE) {
|
if (stateNotActive()) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
return fetchLock.compareAndSet(false, true);
|
return fetchLock.compareAndSet(false, true);
|
||||||
|
@ -1053,6 +1085,22 @@ public class SharePartition {
|
||||||
fetchLock.set(false);
|
fetchLock.set(false);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Marks the share partition as fenced.
|
||||||
|
*/
|
||||||
|
void markFenced() {
|
||||||
|
lock.writeLock().lock();
|
||||||
|
try {
|
||||||
|
partitionState = SharePartitionState.FENCED;
|
||||||
|
} finally {
|
||||||
|
lock.writeLock().unlock();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean stateNotActive() {
|
||||||
|
return partitionState() != SharePartitionState.ACTIVE;
|
||||||
|
}
|
||||||
|
|
||||||
private void completeInitializationWithException(CompletableFuture<Void> future, Throwable exception) {
|
private void completeInitializationWithException(CompletableFuture<Void> future, Throwable exception) {
|
||||||
lock.writeLock().lock();
|
lock.writeLock().lock();
|
||||||
try {
|
try {
|
||||||
|
@ -1075,6 +1123,9 @@ public class SharePartition {
|
||||||
case INITIALIZING:
|
case INITIALIZING:
|
||||||
future.completeExceptionally(new LeaderNotAvailableException(String.format("Share partition is already initializing %s-%s", groupId, topicIdPartition)));
|
future.completeExceptionally(new LeaderNotAvailableException(String.format("Share partition is already initializing %s-%s", groupId, topicIdPartition)));
|
||||||
return;
|
return;
|
||||||
|
case FENCED:
|
||||||
|
future.completeExceptionally(new FencedStateEpochException(String.format("Share partition is fenced %s-%s", groupId, topicIdPartition)));
|
||||||
|
return;
|
||||||
case EMPTY:
|
case EMPTY:
|
||||||
// Do not complete the future as the share partition is not yet initialized.
|
// Do not complete the future as the share partition is not yet initialized.
|
||||||
break;
|
break;
|
||||||
|
@ -1743,7 +1794,7 @@ public class SharePartition {
|
||||||
.setGroupId(this.groupId)
|
.setGroupId(this.groupId)
|
||||||
.setTopicsData(Collections.singletonList(new TopicData<>(topicIdPartition.topicId(),
|
.setTopicsData(Collections.singletonList(new TopicData<>(topicIdPartition.topicId(),
|
||||||
Collections.singletonList(PartitionFactory.newPartitionStateBatchData(
|
Collections.singletonList(PartitionFactory.newPartitionStateBatchData(
|
||||||
topicIdPartition.partition(), stateEpoch, startOffset, 0, stateBatches))))
|
topicIdPartition.partition(), stateEpoch, startOffset, leaderEpoch, stateBatches))))
|
||||||
).build()).build())
|
).build()).build())
|
||||||
.whenComplete((result, exception) -> {
|
.whenComplete((result, exception) -> {
|
||||||
if (exception != null) {
|
if (exception != null) {
|
||||||
|
@ -1792,8 +1843,9 @@ public class SharePartition {
|
||||||
case COORDINATOR_LOAD_IN_PROGRESS:
|
case COORDINATOR_LOAD_IN_PROGRESS:
|
||||||
return new CoordinatorNotAvailableException(errorMessage);
|
return new CoordinatorNotAvailableException(errorMessage);
|
||||||
case GROUP_ID_NOT_FOUND:
|
case GROUP_ID_NOT_FOUND:
|
||||||
|
return new GroupIdNotFoundException(errorMessage);
|
||||||
case UNKNOWN_TOPIC_OR_PARTITION:
|
case UNKNOWN_TOPIC_OR_PARTITION:
|
||||||
return new InvalidRequestException(errorMessage);
|
return new UnknownTopicOrPartitionException(errorMessage);
|
||||||
case FENCED_STATE_EPOCH:
|
case FENCED_STATE_EPOCH:
|
||||||
return new FencedStateEpochException(errorMessage);
|
return new FencedStateEpochException(errorMessage);
|
||||||
case FENCED_LEADER_EPOCH:
|
case FENCED_LEADER_EPOCH:
|
||||||
|
|
|
@ -23,8 +23,10 @@ import org.apache.kafka.common.TopicIdPartition;
|
||||||
import org.apache.kafka.common.TopicPartition;
|
import org.apache.kafka.common.TopicPartition;
|
||||||
import org.apache.kafka.common.Uuid;
|
import org.apache.kafka.common.Uuid;
|
||||||
import org.apache.kafka.common.errors.FencedStateEpochException;
|
import org.apache.kafka.common.errors.FencedStateEpochException;
|
||||||
|
import org.apache.kafka.common.errors.GroupIdNotFoundException;
|
||||||
import org.apache.kafka.common.errors.LeaderNotAvailableException;
|
import org.apache.kafka.common.errors.LeaderNotAvailableException;
|
||||||
import org.apache.kafka.common.errors.NotLeaderOrFollowerException;
|
import org.apache.kafka.common.errors.NotLeaderOrFollowerException;
|
||||||
|
import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
|
||||||
import org.apache.kafka.common.message.ShareAcknowledgeResponseData;
|
import org.apache.kafka.common.message.ShareAcknowledgeResponseData;
|
||||||
import org.apache.kafka.common.message.ShareFetchResponseData.PartitionData;
|
import org.apache.kafka.common.message.ShareFetchResponseData.PartitionData;
|
||||||
import org.apache.kafka.common.metrics.Metrics;
|
import org.apache.kafka.common.metrics.Metrics;
|
||||||
|
@ -69,6 +71,7 @@ import java.util.Objects;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.CompletableFuture;
|
import java.util.concurrent.CompletableFuture;
|
||||||
import java.util.concurrent.ConcurrentHashMap;
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import scala.jdk.javaapi.CollectionConverters;
|
import scala.jdk.javaapi.CollectionConverters;
|
||||||
|
|
||||||
|
@ -271,11 +274,13 @@ public class SharePartitionManager implements AutoCloseable {
|
||||||
this.shareGroupMetrics.shareAcknowledgement();
|
this.shareGroupMetrics.shareAcknowledgement();
|
||||||
Map<TopicIdPartition, CompletableFuture<Errors>> futures = new HashMap<>();
|
Map<TopicIdPartition, CompletableFuture<Errors>> futures = new HashMap<>();
|
||||||
acknowledgeTopics.forEach((topicIdPartition, acknowledgePartitionBatches) -> {
|
acknowledgeTopics.forEach((topicIdPartition, acknowledgePartitionBatches) -> {
|
||||||
SharePartition sharePartition = partitionCacheMap.get(sharePartitionKey(groupId, topicIdPartition));
|
SharePartitionKey sharePartitionKey = sharePartitionKey(groupId, topicIdPartition);
|
||||||
|
SharePartition sharePartition = partitionCacheMap.get(sharePartitionKey);
|
||||||
if (sharePartition != null) {
|
if (sharePartition != null) {
|
||||||
CompletableFuture<Errors> future = new CompletableFuture<>();
|
CompletableFuture<Errors> future = new CompletableFuture<>();
|
||||||
sharePartition.acknowledge(memberId, acknowledgePartitionBatches).whenComplete((result, throwable) -> {
|
sharePartition.acknowledge(memberId, acknowledgePartitionBatches).whenComplete((result, throwable) -> {
|
||||||
if (throwable != null) {
|
if (throwable != null) {
|
||||||
|
handleFencedSharePartitionException(sharePartitionKey, throwable);
|
||||||
future.complete(Errors.forException(throwable));
|
future.complete(Errors.forException(throwable));
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
@ -339,7 +344,8 @@ public class SharePartitionManager implements AutoCloseable {
|
||||||
|
|
||||||
Map<TopicIdPartition, CompletableFuture<Errors>> futuresMap = new HashMap<>();
|
Map<TopicIdPartition, CompletableFuture<Errors>> futuresMap = new HashMap<>();
|
||||||
topicIdPartitions.forEach(topicIdPartition -> {
|
topicIdPartitions.forEach(topicIdPartition -> {
|
||||||
SharePartition sharePartition = partitionCacheMap.get(sharePartitionKey(groupId, topicIdPartition));
|
SharePartitionKey sharePartitionKey = sharePartitionKey(groupId, topicIdPartition);
|
||||||
|
SharePartition sharePartition = partitionCacheMap.get(sharePartitionKey);
|
||||||
if (sharePartition == null) {
|
if (sharePartition == null) {
|
||||||
log.error("No share partition found for groupId {} topicPartition {} while releasing acquired topic partitions", groupId, topicIdPartition);
|
log.error("No share partition found for groupId {} topicPartition {} while releasing acquired topic partitions", groupId, topicIdPartition);
|
||||||
futuresMap.put(topicIdPartition, CompletableFuture.completedFuture(Errors.UNKNOWN_TOPIC_OR_PARTITION));
|
futuresMap.put(topicIdPartition, CompletableFuture.completedFuture(Errors.UNKNOWN_TOPIC_OR_PARTITION));
|
||||||
|
@ -347,6 +353,7 @@ public class SharePartitionManager implements AutoCloseable {
|
||||||
CompletableFuture<Errors> future = new CompletableFuture<>();
|
CompletableFuture<Errors> future = new CompletableFuture<>();
|
||||||
sharePartition.releaseAcquiredRecords(memberId).whenComplete((result, throwable) -> {
|
sharePartition.releaseAcquiredRecords(memberId).whenComplete((result, throwable) -> {
|
||||||
if (throwable != null) {
|
if (throwable != null) {
|
||||||
|
handleFencedSharePartitionException(sharePartitionKey, throwable);
|
||||||
future.complete(Errors.forException(throwable));
|
future.complete(Errors.forException(throwable));
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
@ -490,6 +497,30 @@ public class SharePartitionManager implements AutoCloseable {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The handleFetchException method is used to handle the exception that occurred while reading from log.
|
||||||
|
* The method will handle the exception for each topic-partition in the request. The share partition
|
||||||
|
* might get removed from the cache.
|
||||||
|
* <p>
|
||||||
|
* The replica read request might error out for one share partition
|
||||||
|
* but as we cannot determine which share partition errored out, we might remove all the share partitions
|
||||||
|
* in the request.
|
||||||
|
*
|
||||||
|
* @param groupId The group id in the share fetch request.
|
||||||
|
* @param topicIdPartitions The topic-partitions in the replica read request.
|
||||||
|
* @param future The future to complete with the exception.
|
||||||
|
* @param throwable The exception that occurred while fetching messages.
|
||||||
|
*/
|
||||||
|
public void handleFetchException(
|
||||||
|
String groupId,
|
||||||
|
Set<TopicIdPartition> topicIdPartitions,
|
||||||
|
CompletableFuture<Map<TopicIdPartition, PartitionData>> future,
|
||||||
|
Throwable throwable
|
||||||
|
) {
|
||||||
|
topicIdPartitions.forEach(topicIdPartition -> handleFencedSharePartitionException(sharePartitionKey(groupId, topicIdPartition), throwable));
|
||||||
|
maybeCompleteShareFetchWithException(future, topicIdPartitions, throwable);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The cachedTopicIdPartitionsInShareSession method is used to get the cached topic-partitions in the share session.
|
* The cachedTopicIdPartitionsInShareSession method is used to get the cached topic-partitions in the share session.
|
||||||
*
|
*
|
||||||
|
@ -540,57 +571,76 @@ public class SharePartitionManager implements AutoCloseable {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
// Initialize lazily, if required.
|
||||||
shareFetchData.partitionMaxBytes().keySet().forEach(topicIdPartition -> {
|
Map<TopicIdPartition, Throwable> erroneous = null;
|
||||||
|
Set<DelayedShareFetchKey> delayedShareFetchWatchKeys = new HashSet<>();
|
||||||
|
for (TopicIdPartition topicIdPartition : shareFetchData.partitionMaxBytes().keySet()) {
|
||||||
SharePartitionKey sharePartitionKey = sharePartitionKey(
|
SharePartitionKey sharePartitionKey = sharePartitionKey(
|
||||||
shareFetchData.groupId(),
|
shareFetchData.groupId(),
|
||||||
topicIdPartition
|
topicIdPartition
|
||||||
);
|
);
|
||||||
SharePartition sharePartition = getOrCreateSharePartition(sharePartitionKey);
|
|
||||||
|
|
||||||
// The share partition is initialized asynchronously, so we need to wait for it to be initialized.
|
SharePartition sharePartition;
|
||||||
// But if the share partition is already initialized, then the future will be completed immediately.
|
try {
|
||||||
// Hence, it's safe to call the maybeInitialize method and then wait for the future to be completed.
|
sharePartition = getOrCreateSharePartition(sharePartitionKey);
|
||||||
// TopicPartitionData list will be populated only if the share partition is already initialized.
|
} catch (Exception e) {
|
||||||
sharePartition.maybeInitialize().whenComplete((result, throwable) -> {
|
// Complete the whole fetch request with an exception if there is an error processing.
|
||||||
if (throwable != null) {
|
// The exception currently can be thrown only if there is an error while initializing
|
||||||
maybeCompleteInitializationWithException(sharePartitionKey, shareFetchData.future(), throwable);
|
// the share partition. But skip the processing for other share partitions in the request
|
||||||
|
// as this situation is not expected.
|
||||||
|
log.error("Error processing share fetch request", e);
|
||||||
|
if (erroneous == null) {
|
||||||
|
erroneous = new HashMap<>();
|
||||||
|
}
|
||||||
|
erroneous.put(topicIdPartition, e);
|
||||||
|
// Continue iteration for other partitions in the request.
|
||||||
|
continue;
|
||||||
}
|
}
|
||||||
});
|
|
||||||
});
|
|
||||||
|
|
||||||
Set<DelayedShareFetchKey> delayedShareFetchWatchKeys = new HashSet<>();
|
|
||||||
shareFetchData.partitionMaxBytes().keySet().forEach(
|
|
||||||
topicIdPartition -> {
|
|
||||||
// We add a key corresponding to each share partition in the request in the group so that when there are
|
// We add a key corresponding to each share partition in the request in the group so that when there are
|
||||||
// acknowledgements/acquisition lock timeout etc, we have a way to perform checkAndComplete for all
|
// acknowledgements/acquisition lock timeout etc., we have a way to perform checkAndComplete for all
|
||||||
// such requests which are delayed because of lack of data to acquire for the share partition.
|
// such requests which are delayed because of lack of data to acquire for the share partition.
|
||||||
delayedShareFetchWatchKeys.add(new DelayedShareFetchGroupKey(shareFetchData.groupId(), topicIdPartition.topicId(), topicIdPartition.partition()));
|
delayedShareFetchWatchKeys.add(new DelayedShareFetchGroupKey(shareFetchData.groupId(), topicIdPartition.topicId(), topicIdPartition.partition()));
|
||||||
// We add a key corresponding to each topic partition in the request so that when the HWM is updated
|
// We add a key corresponding to each topic partition in the request so that when the HWM is updated
|
||||||
// for any topic partition, we have a way to perform checkAndComplete for all such requests which are
|
// for any topic partition, we have a way to perform checkAndComplete for all such requests which are
|
||||||
// delayed because of lack of data to acquire for the topic partition.
|
// delayed because of lack of data to acquire for the topic partition.
|
||||||
delayedShareFetchWatchKeys.add(new DelayedShareFetchPartitionKey(topicIdPartition.topicId(), topicIdPartition.partition()));
|
delayedShareFetchWatchKeys.add(new DelayedShareFetchPartitionKey(topicIdPartition.topicId(), topicIdPartition.partition()));
|
||||||
|
// The share partition is initialized asynchronously, so we need to wait for it to be initialized.
|
||||||
|
// But if the share partition is already initialized, then the future will be completed immediately.
|
||||||
|
// Hence, it's safe to call the maybeInitialize method and then wait for the future to be completed.
|
||||||
|
// TopicPartitionData list will be populated only if the share partition is already initialized.
|
||||||
|
sharePartition.maybeInitialize().whenComplete((result, throwable) -> {
|
||||||
|
if (throwable != null) {
|
||||||
|
// TODO: Complete error handling for initialization. We have to record the error
|
||||||
|
// for respective share partition as completing the full request might result in
|
||||||
|
// some acquired records to not being sent: https://issues.apache.org/jira/browse/KAFKA-17510
|
||||||
|
maybeCompleteInitializationWithException(sharePartitionKey, shareFetchData.future(), throwable);
|
||||||
|
}
|
||||||
});
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
// If all the partitions in the request errored out, then complete the fetch request with an exception.
|
||||||
|
if (erroneous != null && erroneous.size() == shareFetchData.partitionMaxBytes().size()) {
|
||||||
|
completeShareFetchWithException(shareFetchData.future(), erroneous);
|
||||||
|
// Do not proceed with share fetch processing as all the partitions errored out.
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
// TODO: If there exists some erroneous partitions then they will not be part of response.
|
||||||
|
|
||||||
// Add the share fetch to the delayed share fetch purgatory to process the fetch request.
|
// Add the share fetch to the delayed share fetch purgatory to process the fetch request.
|
||||||
addDelayedShareFetch(new DelayedShareFetch(shareFetchData, replicaManager, this),
|
addDelayedShareFetch(new DelayedShareFetch(shareFetchData, replicaManager, this), delayedShareFetchWatchKeys);
|
||||||
delayedShareFetchWatchKeys);
|
|
||||||
} catch (Exception e) {
|
|
||||||
// In case exception occurs then release the locks so queue can be further processed.
|
|
||||||
log.error("Error processing fetch queue for share partitions", e);
|
|
||||||
if (!shareFetchData.future().isDone()) {
|
|
||||||
shareFetchData.future().completeExceptionally(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private SharePartition getOrCreateSharePartition(SharePartitionKey sharePartitionKey) {
|
private SharePartition getOrCreateSharePartition(SharePartitionKey sharePartitionKey) {
|
||||||
return partitionCacheMap.computeIfAbsent(sharePartitionKey,
|
return partitionCacheMap.computeIfAbsent(sharePartitionKey,
|
||||||
k -> {
|
k -> {
|
||||||
long start = time.hiResClockMs();
|
long start = time.hiResClockMs();
|
||||||
|
int leaderEpoch = ShareFetchUtils.leaderEpoch(replicaManager, sharePartitionKey.topicIdPartition().topicPartition());
|
||||||
SharePartition partition = new SharePartition(
|
SharePartition partition = new SharePartition(
|
||||||
sharePartitionKey.groupId(),
|
sharePartitionKey.groupId(),
|
||||||
sharePartitionKey.topicIdPartition(),
|
sharePartitionKey.topicIdPartition(),
|
||||||
|
leaderEpoch,
|
||||||
maxInFlightMessages,
|
maxInFlightMessages,
|
||||||
maxDeliveryCount,
|
maxDeliveryCount,
|
||||||
defaultRecordLockDurationMs,
|
defaultRecordLockDurationMs,
|
||||||
|
@ -617,22 +667,47 @@ public class SharePartitionManager implements AutoCloseable {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
|
|
||||||
if (throwable instanceof NotLeaderOrFollowerException || throwable instanceof FencedStateEpochException) {
|
// Remove the partition from the cache as it's failed to initialize.
|
||||||
|
partitionCacheMap.remove(sharePartitionKey);
|
||||||
|
// The partition initialization failed, so complete the request with the exception.
|
||||||
|
// The server should not be in this state, so log the error on broker and surface the same
|
||||||
|
// to the client. The broker should not be in this state, investigate the root cause of the error.
|
||||||
|
log.error("Error initializing share partition with key {}", sharePartitionKey, throwable);
|
||||||
|
maybeCompleteShareFetchWithException(future, Collections.singletonList(sharePartitionKey.topicIdPartition()), throwable);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void handleFencedSharePartitionException(
|
||||||
|
SharePartitionKey sharePartitionKey,
|
||||||
|
Throwable throwable
|
||||||
|
) {
|
||||||
|
if (throwable instanceof NotLeaderOrFollowerException || throwable instanceof FencedStateEpochException ||
|
||||||
|
throwable instanceof GroupIdNotFoundException || throwable instanceof UnknownTopicOrPartitionException) {
|
||||||
log.info("The share partition with key {} is fenced: {}", sharePartitionKey, throwable.getMessage());
|
log.info("The share partition with key {} is fenced: {}", sharePartitionKey, throwable.getMessage());
|
||||||
// The share partition is fenced hence remove the partition from map and let the client retry.
|
// The share partition is fenced hence remove the partition from map and let the client retry.
|
||||||
// But surface the error to the client so client might take some action i.e. re-fetch
|
// But surface the error to the client so client might take some action i.e. re-fetch
|
||||||
// the metadata and retry the fetch on new leader.
|
// the metadata and retry the fetch on new leader.
|
||||||
partitionCacheMap.remove(sharePartitionKey);
|
SharePartition sharePartition = partitionCacheMap.remove(sharePartitionKey);
|
||||||
future.completeExceptionally(throwable);
|
if (sharePartition != null) {
|
||||||
return;
|
sharePartition.markFenced();
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// The partition initialization failed, so complete the request with the exception.
|
private void maybeCompleteShareFetchWithException(CompletableFuture<Map<TopicIdPartition, PartitionData>> future,
|
||||||
// The server should not be in this state, so log the error on broker and surface the same
|
Collection<TopicIdPartition> topicIdPartitions, Throwable throwable) {
|
||||||
// to the client. As of now this state is in-recoverable for the broker, and we should
|
if (!future.isDone()) {
|
||||||
// investigate the root cause of the error.
|
future.complete(topicIdPartitions.stream().collect(Collectors.toMap(
|
||||||
log.error("Error initializing share partition with key {}", sharePartitionKey, throwable);
|
tp -> tp, tp -> new PartitionData().setErrorCode(Errors.forException(throwable).code()).setErrorMessage(throwable.getMessage()))));
|
||||||
future.completeExceptionally(throwable);
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void completeShareFetchWithException(CompletableFuture<Map<TopicIdPartition, PartitionData>> future,
|
||||||
|
Map<TopicIdPartition, Throwable> erroneous) {
|
||||||
|
future.complete(erroneous.entrySet().stream().collect(Collectors.toMap(
|
||||||
|
Map.Entry::getKey, entry -> {
|
||||||
|
Throwable t = entry.getValue();
|
||||||
|
return new PartitionData().setErrorCode(Errors.forException(t).code()).setErrorMessage(t.getMessage());
|
||||||
|
})));
|
||||||
}
|
}
|
||||||
|
|
||||||
private SharePartitionKey sharePartitionKey(String groupId, TopicIdPartition topicIdPartition) {
|
private SharePartitionKey sharePartitionKey(String groupId, TopicIdPartition topicIdPartition) {
|
||||||
|
|
|
@ -4249,7 +4249,8 @@ class KafkaApis(val requestChannel: RequestChannel,
|
||||||
fetchMinBytes,
|
fetchMinBytes,
|
||||||
fetchMaxBytes,
|
fetchMaxBytes,
|
||||||
FetchIsolation.HIGH_WATERMARK,
|
FetchIsolation.HIGH_WATERMARK,
|
||||||
clientMetadata
|
clientMetadata,
|
||||||
|
true
|
||||||
)
|
)
|
||||||
|
|
||||||
// call the share partition manager to fetch messages from the local replica.
|
// call the share partition manager to fetch messages from the local replica.
|
||||||
|
|
|
@ -76,9 +76,9 @@ public class DelayedShareFetchTest {
|
||||||
private static final int MAX_FETCH_RECORDS = 100;
|
private static final int MAX_FETCH_RECORDS = 100;
|
||||||
private static final FetchParams FETCH_PARAMS = new FetchParams(ApiKeys.SHARE_FETCH.latestVersion(),
|
private static final FetchParams FETCH_PARAMS = new FetchParams(ApiKeys.SHARE_FETCH.latestVersion(),
|
||||||
FetchRequest.ORDINARY_CONSUMER_ID, -1, MAX_WAIT_MS, 1, 1024 * 1024, FetchIsolation.HIGH_WATERMARK,
|
FetchRequest.ORDINARY_CONSUMER_ID, -1, MAX_WAIT_MS, 1, 1024 * 1024, FetchIsolation.HIGH_WATERMARK,
|
||||||
Optional.empty());
|
Optional.empty(), true);
|
||||||
|
|
||||||
private static Timer mockTimer;
|
private Timer mockTimer;
|
||||||
|
|
||||||
@BeforeEach
|
@BeforeEach
|
||||||
public void setUp() {
|
public void setUp() {
|
||||||
|
|
|
@ -67,6 +67,10 @@ import static org.mockito.Mockito.when;
|
||||||
|
|
||||||
public class ShareFetchUtilsTest {
|
public class ShareFetchUtilsTest {
|
||||||
|
|
||||||
|
private static final FetchParams FETCH_PARAMS = new FetchParams(ApiKeys.SHARE_FETCH.latestVersion(),
|
||||||
|
FetchRequest.ORDINARY_CONSUMER_ID, -1, 0, 1, 1024 * 1024, FetchIsolation.HIGH_WATERMARK,
|
||||||
|
Optional.empty(), true);
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testProcessFetchResponse() {
|
public void testProcessFetchResponse() {
|
||||||
String groupId = "grp";
|
String groupId = "grp";
|
||||||
|
@ -97,9 +101,7 @@ public class ShareFetchUtilsTest {
|
||||||
when(sharePartitionManager.sharePartition(groupId, tp0)).thenReturn(sp0);
|
when(sharePartitionManager.sharePartition(groupId, tp0)).thenReturn(sp0);
|
||||||
when(sharePartitionManager.sharePartition(groupId, tp1)).thenReturn(sp1);
|
when(sharePartitionManager.sharePartition(groupId, tp1)).thenReturn(sp1);
|
||||||
|
|
||||||
ShareFetchData shareFetchData = new ShareFetchData(
|
ShareFetchData shareFetchData = new ShareFetchData(FETCH_PARAMS, groupId, memberId,
|
||||||
new FetchParams(ApiKeys.SHARE_FETCH.latestVersion(), FetchRequest.ORDINARY_CONSUMER_ID, -1, 0,
|
|
||||||
1, 1024 * 1024, FetchIsolation.HIGH_WATERMARK, Optional.empty()), groupId, memberId,
|
|
||||||
new CompletableFuture<>(), partitionMaxBytes, 100);
|
new CompletableFuture<>(), partitionMaxBytes, 100);
|
||||||
|
|
||||||
MemoryRecords records = MemoryRecords.withRecords(Compression.NONE,
|
MemoryRecords records = MemoryRecords.withRecords(Compression.NONE,
|
||||||
|
@ -165,9 +167,7 @@ public class ShareFetchUtilsTest {
|
||||||
when(sharePartitionManager.sharePartition(groupId, tp0)).thenReturn(sp0);
|
when(sharePartitionManager.sharePartition(groupId, tp0)).thenReturn(sp0);
|
||||||
when(sharePartitionManager.sharePartition(groupId, tp1)).thenReturn(sp1);
|
when(sharePartitionManager.sharePartition(groupId, tp1)).thenReturn(sp1);
|
||||||
|
|
||||||
ShareFetchData shareFetchData = new ShareFetchData(
|
ShareFetchData shareFetchData = new ShareFetchData(FETCH_PARAMS, groupId, memberId,
|
||||||
new FetchParams(ApiKeys.SHARE_FETCH.latestVersion(), FetchRequest.ORDINARY_CONSUMER_ID, -1, 0,
|
|
||||||
1, 1024 * 1024, FetchIsolation.HIGH_WATERMARK, Optional.empty()), groupId, memberId,
|
|
||||||
new CompletableFuture<>(), partitionMaxBytes, 100);
|
new CompletableFuture<>(), partitionMaxBytes, 100);
|
||||||
|
|
||||||
Map<TopicIdPartition, FetchPartitionData> responseData = new HashMap<>();
|
Map<TopicIdPartition, FetchPartitionData> responseData = new HashMap<>();
|
||||||
|
@ -209,10 +209,8 @@ public class ShareFetchUtilsTest {
|
||||||
when(sharePartitionManager.sharePartition(groupId, tp0)).thenReturn(sp0);
|
when(sharePartitionManager.sharePartition(groupId, tp0)).thenReturn(sp0);
|
||||||
when(sharePartitionManager.sharePartition(groupId, tp1)).thenReturn(sp1);
|
when(sharePartitionManager.sharePartition(groupId, tp1)).thenReturn(sp1);
|
||||||
|
|
||||||
ShareFetchData shareFetchData = new ShareFetchData(
|
ShareFetchData shareFetchData = new ShareFetchData(FETCH_PARAMS, groupId, Uuid.randomUuid().toString(),
|
||||||
new FetchParams(ApiKeys.SHARE_FETCH.latestVersion(), FetchRequest.ORDINARY_CONSUMER_ID, -1, 0,
|
new CompletableFuture<>(), partitionMaxBytes, 100);
|
||||||
1, 1024 * 1024, FetchIsolation.HIGH_WATERMARK, Optional.empty()),
|
|
||||||
groupId, Uuid.randomUuid().toString(), new CompletableFuture<>(), partitionMaxBytes, 100);
|
|
||||||
|
|
||||||
ReplicaManager replicaManager = mock(ReplicaManager.class);
|
ReplicaManager replicaManager = mock(ReplicaManager.class);
|
||||||
|
|
||||||
|
@ -222,6 +220,7 @@ public class ShareFetchUtilsTest {
|
||||||
|
|
||||||
when(sp0.nextFetchOffset()).thenReturn((long) 0, (long) 5);
|
when(sp0.nextFetchOffset()).thenReturn((long) 0, (long) 5);
|
||||||
when(sp1.nextFetchOffset()).thenReturn((long) 4, (long) 4);
|
when(sp1.nextFetchOffset()).thenReturn((long) 4, (long) 4);
|
||||||
|
|
||||||
when(sp0.acquire(anyString(), anyInt(), any(FetchPartitionData.class))).thenReturn(
|
when(sp0.acquire(anyString(), anyInt(), any(FetchPartitionData.class))).thenReturn(
|
||||||
ShareAcquiredRecords.empty(),
|
ShareAcquiredRecords.empty(),
|
||||||
ShareAcquiredRecords.fromAcquiredRecords(new ShareFetchResponseData.AcquiredRecords()
|
ShareAcquiredRecords.fromAcquiredRecords(new ShareFetchResponseData.AcquiredRecords()
|
||||||
|
@ -304,10 +303,8 @@ public class ShareFetchUtilsTest {
|
||||||
SharePartitionManager sharePartitionManager = mock(SharePartitionManager.class);
|
SharePartitionManager sharePartitionManager = mock(SharePartitionManager.class);
|
||||||
when(sharePartitionManager.sharePartition(groupId, tp0)).thenReturn(sp0);
|
when(sharePartitionManager.sharePartition(groupId, tp0)).thenReturn(sp0);
|
||||||
|
|
||||||
ShareFetchData shareFetchData = new ShareFetchData(
|
ShareFetchData shareFetchData = new ShareFetchData(FETCH_PARAMS, groupId, Uuid.randomUuid().toString(),
|
||||||
new FetchParams(ApiKeys.SHARE_FETCH.latestVersion(), FetchRequest.ORDINARY_CONSUMER_ID, -1, 0,
|
new CompletableFuture<>(), partitionMaxBytes, 100);
|
||||||
1, 1024 * 1024, FetchIsolation.HIGH_WATERMARK, Optional.empty()),
|
|
||||||
groupId, Uuid.randomUuid().toString(), new CompletableFuture<>(), partitionMaxBytes, 100);
|
|
||||||
|
|
||||||
ReplicaManager replicaManager = mock(ReplicaManager.class);
|
ReplicaManager replicaManager = mock(ReplicaManager.class);
|
||||||
|
|
||||||
|
|
|
@ -16,6 +16,7 @@
|
||||||
*/
|
*/
|
||||||
package kafka.server.share;
|
package kafka.server.share;
|
||||||
|
|
||||||
|
import kafka.cluster.Partition;
|
||||||
import kafka.server.DelayedOperationPurgatory;
|
import kafka.server.DelayedOperationPurgatory;
|
||||||
import kafka.server.LogReadResult;
|
import kafka.server.LogReadResult;
|
||||||
import kafka.server.ReplicaManager;
|
import kafka.server.ReplicaManager;
|
||||||
|
@ -31,11 +32,13 @@ import org.apache.kafka.common.errors.FencedStateEpochException;
|
||||||
import org.apache.kafka.common.errors.InvalidRecordStateException;
|
import org.apache.kafka.common.errors.InvalidRecordStateException;
|
||||||
import org.apache.kafka.common.errors.InvalidRequestException;
|
import org.apache.kafka.common.errors.InvalidRequestException;
|
||||||
import org.apache.kafka.common.errors.InvalidShareSessionEpochException;
|
import org.apache.kafka.common.errors.InvalidShareSessionEpochException;
|
||||||
|
import org.apache.kafka.common.errors.KafkaStorageException;
|
||||||
import org.apache.kafka.common.errors.LeaderNotAvailableException;
|
import org.apache.kafka.common.errors.LeaderNotAvailableException;
|
||||||
import org.apache.kafka.common.errors.NotLeaderOrFollowerException;
|
import org.apache.kafka.common.errors.NotLeaderOrFollowerException;
|
||||||
import org.apache.kafka.common.errors.ShareSessionNotFoundException;
|
import org.apache.kafka.common.errors.ShareSessionNotFoundException;
|
||||||
import org.apache.kafka.common.message.ShareAcknowledgeResponseData;
|
import org.apache.kafka.common.message.ShareAcknowledgeResponseData;
|
||||||
import org.apache.kafka.common.message.ShareFetchResponseData;
|
import org.apache.kafka.common.message.ShareFetchResponseData;
|
||||||
|
import org.apache.kafka.common.message.ShareFetchResponseData.PartitionData;
|
||||||
import org.apache.kafka.common.metrics.Metrics;
|
import org.apache.kafka.common.metrics.Metrics;
|
||||||
import org.apache.kafka.common.protocol.ApiKeys;
|
import org.apache.kafka.common.protocol.ApiKeys;
|
||||||
import org.apache.kafka.common.protocol.Errors;
|
import org.apache.kafka.common.protocol.Errors;
|
||||||
|
@ -56,6 +59,7 @@ import org.apache.kafka.server.share.acknowledge.ShareAcknowledgementBatch;
|
||||||
import org.apache.kafka.server.share.context.FinalContext;
|
import org.apache.kafka.server.share.context.FinalContext;
|
||||||
import org.apache.kafka.server.share.context.ShareFetchContext;
|
import org.apache.kafka.server.share.context.ShareFetchContext;
|
||||||
import org.apache.kafka.server.share.context.ShareSessionContext;
|
import org.apache.kafka.server.share.context.ShareSessionContext;
|
||||||
|
import org.apache.kafka.server.share.fetch.ShareAcquiredRecords;
|
||||||
import org.apache.kafka.server.share.fetch.ShareFetchData;
|
import org.apache.kafka.server.share.fetch.ShareFetchData;
|
||||||
import org.apache.kafka.server.share.persister.NoOpShareStatePersister;
|
import org.apache.kafka.server.share.persister.NoOpShareStatePersister;
|
||||||
import org.apache.kafka.server.share.persister.Persister;
|
import org.apache.kafka.server.share.persister.Persister;
|
||||||
|
@ -102,7 +106,6 @@ import scala.Tuple2;
|
||||||
import scala.collection.Seq;
|
import scala.collection.Seq;
|
||||||
import scala.jdk.javaapi.CollectionConverters;
|
import scala.jdk.javaapi.CollectionConverters;
|
||||||
|
|
||||||
import static org.apache.kafka.test.TestUtils.assertFutureThrows;
|
|
||||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||||
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
|
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
|
||||||
|
@ -113,9 +116,12 @@ import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||||
import static org.mockito.ArgumentMatchers.any;
|
import static org.mockito.ArgumentMatchers.any;
|
||||||
import static org.mockito.ArgumentMatchers.anyBoolean;
|
import static org.mockito.ArgumentMatchers.anyBoolean;
|
||||||
|
import static org.mockito.ArgumentMatchers.anyInt;
|
||||||
|
import static org.mockito.ArgumentMatchers.anyString;
|
||||||
import static org.mockito.Mockito.atLeast;
|
import static org.mockito.Mockito.atLeast;
|
||||||
import static org.mockito.Mockito.atMost;
|
import static org.mockito.Mockito.atMost;
|
||||||
import static org.mockito.Mockito.doAnswer;
|
import static org.mockito.Mockito.doAnswer;
|
||||||
|
import static org.mockito.Mockito.doThrow;
|
||||||
import static org.mockito.Mockito.mock;
|
import static org.mockito.Mockito.mock;
|
||||||
import static org.mockito.Mockito.spy;
|
import static org.mockito.Mockito.spy;
|
||||||
import static org.mockito.Mockito.times;
|
import static org.mockito.Mockito.times;
|
||||||
|
@ -133,12 +139,13 @@ public class SharePartitionManagerTest {
|
||||||
private static final int DELAYED_SHARE_FETCH_TIMEOUT_MS = 3000;
|
private static final int DELAYED_SHARE_FETCH_TIMEOUT_MS = 3000;
|
||||||
private static final FetchParams FETCH_PARAMS = new FetchParams(ApiKeys.SHARE_FETCH.latestVersion(),
|
private static final FetchParams FETCH_PARAMS = new FetchParams(ApiKeys.SHARE_FETCH.latestVersion(),
|
||||||
FetchRequest.ORDINARY_CONSUMER_ID, -1, DELAYED_SHARE_FETCH_MAX_WAIT_MS,
|
FetchRequest.ORDINARY_CONSUMER_ID, -1, DELAYED_SHARE_FETCH_MAX_WAIT_MS,
|
||||||
1, 1024 * 1024, FetchIsolation.HIGH_WATERMARK, Optional.empty());
|
1, 1024 * 1024, FetchIsolation.HIGH_WATERMARK, Optional.empty(), true);
|
||||||
|
|
||||||
static final int PARTITION_MAX_BYTES = 40000;
|
static final int PARTITION_MAX_BYTES = 40000;
|
||||||
static final int DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL = 1000;
|
static final int DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL = 1000;
|
||||||
|
|
||||||
private static Timer mockTimer;
|
private Timer mockTimer;
|
||||||
|
private ReplicaManager mockReplicaManager;
|
||||||
|
|
||||||
private static final List<TopicIdPartition> EMPTY_PART_LIST = Collections.unmodifiableList(new ArrayList<>());
|
private static final List<TopicIdPartition> EMPTY_PART_LIST = Collections.unmodifiableList(new ArrayList<>());
|
||||||
|
|
||||||
|
@ -146,6 +153,9 @@ public class SharePartitionManagerTest {
|
||||||
public void setUp() {
|
public void setUp() {
|
||||||
mockTimer = new SystemTimerReaper("sharePartitionManagerTestReaper",
|
mockTimer = new SystemTimerReaper("sharePartitionManagerTestReaper",
|
||||||
new SystemTimer("sharePartitionManagerTestTimer"));
|
new SystemTimer("sharePartitionManagerTestTimer"));
|
||||||
|
mockReplicaManager = mock(ReplicaManager.class);
|
||||||
|
Partition partition = mockPartition();
|
||||||
|
when(mockReplicaManager.getPartitionOrException(Mockito.any())).thenReturn(partition);
|
||||||
}
|
}
|
||||||
|
|
||||||
@AfterEach
|
@AfterEach
|
||||||
|
@ -1026,34 +1036,33 @@ public class SharePartitionManagerTest {
|
||||||
partitionMaxBytes.put(tp5, PARTITION_MAX_BYTES);
|
partitionMaxBytes.put(tp5, PARTITION_MAX_BYTES);
|
||||||
partitionMaxBytes.put(tp6, PARTITION_MAX_BYTES);
|
partitionMaxBytes.put(tp6, PARTITION_MAX_BYTES);
|
||||||
|
|
||||||
ReplicaManager replicaManager = mock(ReplicaManager.class);
|
|
||||||
Time time = mock(Time.class);
|
Time time = mock(Time.class);
|
||||||
when(time.hiResClockMs()).thenReturn(0L).thenReturn(100L);
|
when(time.hiResClockMs()).thenReturn(0L).thenReturn(100L);
|
||||||
Metrics metrics = new Metrics();
|
Metrics metrics = new Metrics();
|
||||||
DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>(
|
DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>(
|
||||||
"TestShareFetch", mockTimer, replicaManager.localBrokerId(),
|
"TestShareFetch", mockTimer, mockReplicaManager.localBrokerId(),
|
||||||
DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true);
|
DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true);
|
||||||
mockReplicaManagerDelayedShareFetch(replicaManager, delayedShareFetchPurgatory);
|
mockReplicaManagerDelayedShareFetch(mockReplicaManager, delayedShareFetchPurgatory);
|
||||||
|
|
||||||
SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder()
|
SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder()
|
||||||
.withReplicaManager(replicaManager)
|
.withReplicaManager(mockReplicaManager)
|
||||||
.withTime(time)
|
.withTime(time)
|
||||||
.withMetrics(metrics)
|
.withMetrics(metrics)
|
||||||
.withTimer(mockTimer)
|
.withTimer(mockTimer)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
doAnswer(invocation -> buildLogReadResult(partitionMaxBytes.keySet())).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean());
|
doAnswer(invocation -> buildLogReadResult(partitionMaxBytes.keySet())).when(mockReplicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean());
|
||||||
|
|
||||||
sharePartitionManager.fetchMessages(groupId, memberId1.toString(), FETCH_PARAMS, partitionMaxBytes);
|
sharePartitionManager.fetchMessages(groupId, memberId1.toString(), FETCH_PARAMS, partitionMaxBytes);
|
||||||
Mockito.verify(replicaManager, times(1)).readFromLog(
|
Mockito.verify(mockReplicaManager, times(1)).readFromLog(
|
||||||
any(), any(), any(ReplicaQuota.class), anyBoolean());
|
any(), any(), any(ReplicaQuota.class), anyBoolean());
|
||||||
|
|
||||||
sharePartitionManager.fetchMessages(groupId, memberId1.toString(), FETCH_PARAMS, partitionMaxBytes);
|
sharePartitionManager.fetchMessages(groupId, memberId1.toString(), FETCH_PARAMS, partitionMaxBytes);
|
||||||
Mockito.verify(replicaManager, times(2)).readFromLog(
|
Mockito.verify(mockReplicaManager, times(2)).readFromLog(
|
||||||
any(), any(), any(ReplicaQuota.class), anyBoolean());
|
any(), any(), any(ReplicaQuota.class), anyBoolean());
|
||||||
|
|
||||||
sharePartitionManager.fetchMessages(groupId, memberId1.toString(), FETCH_PARAMS, partitionMaxBytes);
|
sharePartitionManager.fetchMessages(groupId, memberId1.toString(), FETCH_PARAMS, partitionMaxBytes);
|
||||||
Mockito.verify(replicaManager, times(3)).readFromLog(
|
Mockito.verify(mockReplicaManager, times(3)).readFromLog(
|
||||||
any(), any(), any(ReplicaQuota.class), anyBoolean());
|
any(), any(), any(ReplicaQuota.class), anyBoolean());
|
||||||
|
|
||||||
Map<MetricName, Consumer<Double>> expectedMetrics = new HashMap<>();
|
Map<MetricName, Consumer<Double>> expectedMetrics = new HashMap<>();
|
||||||
|
@ -1089,15 +1098,14 @@ public class SharePartitionManagerTest {
|
||||||
partitionMaxBytes.put(tp3, PARTITION_MAX_BYTES);
|
partitionMaxBytes.put(tp3, PARTITION_MAX_BYTES);
|
||||||
|
|
||||||
final Time time = new MockTime(0, System.currentTimeMillis(), 0);
|
final Time time = new MockTime(0, System.currentTimeMillis(), 0);
|
||||||
ReplicaManager replicaManager = mock(ReplicaManager.class);
|
|
||||||
DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>(
|
DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>(
|
||||||
"TestShareFetch", mockTimer, replicaManager.localBrokerId(),
|
"TestShareFetch", mockTimer, mockReplicaManager.localBrokerId(),
|
||||||
DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true);
|
DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true);
|
||||||
mockReplicaManagerDelayedShareFetch(replicaManager, delayedShareFetchPurgatory);
|
mockReplicaManagerDelayedShareFetch(mockReplicaManager, delayedShareFetchPurgatory);
|
||||||
|
|
||||||
SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder()
|
SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder()
|
||||||
.withTime(time)
|
.withTime(time)
|
||||||
.withReplicaManager(replicaManager)
|
.withReplicaManager(mockReplicaManager)
|
||||||
.withTimer(mockTimer)
|
.withTimer(mockTimer)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
|
@ -1141,7 +1149,7 @@ public class SharePartitionManagerTest {
|
||||||
assertEquals(26, sp2.nextFetchOffset());
|
assertEquals(26, sp2.nextFetchOffset());
|
||||||
assertEquals(16, sp3.nextFetchOffset());
|
assertEquals(16, sp3.nextFetchOffset());
|
||||||
return buildLogReadResult(partitionMaxBytes.keySet());
|
return buildLogReadResult(partitionMaxBytes.keySet());
|
||||||
}).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean());
|
}).when(mockReplicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean());
|
||||||
|
|
||||||
int threadCount = 100;
|
int threadCount = 100;
|
||||||
ExecutorService executorService = Executors.newFixedThreadPool(threadCount);
|
ExecutorService executorService = Executors.newFixedThreadPool(threadCount);
|
||||||
|
@ -1160,9 +1168,9 @@ public class SharePartitionManagerTest {
|
||||||
executorService.shutdown();
|
executorService.shutdown();
|
||||||
}
|
}
|
||||||
// We are checking the number of replicaManager readFromLog() calls
|
// We are checking the number of replicaManager readFromLog() calls
|
||||||
Mockito.verify(replicaManager, atMost(100)).readFromLog(
|
Mockito.verify(mockReplicaManager, atMost(100)).readFromLog(
|
||||||
any(), any(), any(ReplicaQuota.class), anyBoolean());
|
any(), any(), any(ReplicaQuota.class), anyBoolean());
|
||||||
Mockito.verify(replicaManager, atLeast(10)).readFromLog(
|
Mockito.verify(mockReplicaManager, atLeast(10)).readFromLog(
|
||||||
any(), any(), any(ReplicaQuota.class), anyBoolean());
|
any(), any(), any(ReplicaQuota.class), anyBoolean());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1175,8 +1183,6 @@ public class SharePartitionManagerTest {
|
||||||
Map<TopicIdPartition, Integer> partitionMaxBytes = new HashMap<>();
|
Map<TopicIdPartition, Integer> partitionMaxBytes = new HashMap<>();
|
||||||
partitionMaxBytes.put(tp0, PARTITION_MAX_BYTES);
|
partitionMaxBytes.put(tp0, PARTITION_MAX_BYTES);
|
||||||
|
|
||||||
ReplicaManager replicaManager = mock(ReplicaManager.class);
|
|
||||||
|
|
||||||
SharePartition sp0 = mock(SharePartition.class);
|
SharePartition sp0 = mock(SharePartition.class);
|
||||||
when(sp0.maybeAcquireFetchLock()).thenReturn(true);
|
when(sp0.maybeAcquireFetchLock()).thenReturn(true);
|
||||||
when(sp0.canAcquireRecords()).thenReturn(false);
|
when(sp0.canAcquireRecords()).thenReturn(false);
|
||||||
|
@ -1185,19 +1191,19 @@ public class SharePartitionManagerTest {
|
||||||
partitionCacheMap.put(new SharePartitionKey(groupId, tp0), sp0);
|
partitionCacheMap.put(new SharePartitionKey(groupId, tp0), sp0);
|
||||||
|
|
||||||
DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>(
|
DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>(
|
||||||
"TestShareFetch", mockTimer, replicaManager.localBrokerId(),
|
"TestShareFetch", mockTimer, mockReplicaManager.localBrokerId(),
|
||||||
DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true);
|
DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true);
|
||||||
mockReplicaManagerDelayedShareFetch(replicaManager, delayedShareFetchPurgatory);
|
mockReplicaManagerDelayedShareFetch(mockReplicaManager, delayedShareFetchPurgatory);
|
||||||
|
|
||||||
SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder()
|
SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder()
|
||||||
.withPartitionCacheMap(partitionCacheMap)
|
.withPartitionCacheMap(partitionCacheMap)
|
||||||
.withReplicaManager(replicaManager)
|
.withReplicaManager(mockReplicaManager)
|
||||||
.withTimer(mockTimer)
|
.withTimer(mockTimer)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
CompletableFuture<Map<TopicIdPartition, ShareFetchResponseData.PartitionData>> future =
|
CompletableFuture<Map<TopicIdPartition, ShareFetchResponseData.PartitionData>> future =
|
||||||
sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, partitionMaxBytes);
|
sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, partitionMaxBytes);
|
||||||
Mockito.verify(replicaManager, times(0)).readFromLog(
|
Mockito.verify(mockReplicaManager, times(0)).readFromLog(
|
||||||
any(), any(), any(ReplicaQuota.class), anyBoolean());
|
any(), any(), any(ReplicaQuota.class), anyBoolean());
|
||||||
Map<TopicIdPartition, ShareFetchResponseData.PartitionData> result = future.join();
|
Map<TopicIdPartition, ShareFetchResponseData.PartitionData> result = future.join();
|
||||||
assertEquals(0, result.size());
|
assertEquals(0, result.size());
|
||||||
|
@ -1209,27 +1215,24 @@ public class SharePartitionManagerTest {
|
||||||
Uuid memberId = Uuid.randomUuid();
|
Uuid memberId = Uuid.randomUuid();
|
||||||
Uuid fooId = Uuid.randomUuid();
|
Uuid fooId = Uuid.randomUuid();
|
||||||
TopicIdPartition tp0 = new TopicIdPartition(fooId, new TopicPartition("foo", 0));
|
TopicIdPartition tp0 = new TopicIdPartition(fooId, new TopicPartition("foo", 0));
|
||||||
Map<TopicIdPartition, Integer> partitionMaxBytes = new HashMap<>();
|
Map<TopicIdPartition, Integer> partitionMaxBytes = Collections.singletonMap(tp0, PARTITION_MAX_BYTES);
|
||||||
partitionMaxBytes.put(tp0, PARTITION_MAX_BYTES);
|
|
||||||
|
|
||||||
ReplicaManager replicaManager = mock(ReplicaManager.class);
|
|
||||||
|
|
||||||
DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>(
|
DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>(
|
||||||
"TestShareFetch", mockTimer, replicaManager.localBrokerId(),
|
"TestShareFetch", mockTimer, mockReplicaManager.localBrokerId(),
|
||||||
DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true);
|
DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true);
|
||||||
mockReplicaManagerDelayedShareFetch(replicaManager, delayedShareFetchPurgatory);
|
mockReplicaManagerDelayedShareFetch(mockReplicaManager, delayedShareFetchPurgatory);
|
||||||
|
|
||||||
SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder()
|
SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder()
|
||||||
.withReplicaManager(replicaManager)
|
.withReplicaManager(mockReplicaManager)
|
||||||
.withTimer(mockTimer)
|
.withTimer(mockTimer)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
doAnswer(invocation -> buildLogReadResult(partitionMaxBytes.keySet())).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean());
|
doAnswer(invocation -> buildLogReadResult(partitionMaxBytes.keySet())).when(mockReplicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean());
|
||||||
|
|
||||||
sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, partitionMaxBytes);
|
sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, partitionMaxBytes);
|
||||||
// Since the nextFetchOffset does not point to endOffset + 1, i.e. some of the records in the cachedState are AVAILABLE,
|
// Since the nextFetchOffset does not point to endOffset + 1, i.e. some of the records in the cachedState are AVAILABLE,
|
||||||
// even though the maxInFlightMessages limit is exceeded, replicaManager.readFromLog should be called
|
// even though the maxInFlightMessages limit is exceeded, replicaManager.readFromLog should be called
|
||||||
Mockito.verify(replicaManager, times(1)).readFromLog(
|
Mockito.verify(mockReplicaManager, times(1)).readFromLog(
|
||||||
any(), any(), any(ReplicaQuota.class), anyBoolean());
|
any(), any(), any(ReplicaQuota.class), anyBoolean());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1643,12 +1646,11 @@ public class SharePartitionManagerTest {
|
||||||
new CompletableFuture<>(),
|
new CompletableFuture<>(),
|
||||||
partitionMaxBytes,
|
partitionMaxBytes,
|
||||||
100);
|
100);
|
||||||
ReplicaManager replicaManager = mock(ReplicaManager.class);
|
|
||||||
|
|
||||||
DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>(
|
DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>(
|
||||||
"TestShareFetch", mockTimer, replicaManager.localBrokerId(),
|
"TestShareFetch", mockTimer, mockReplicaManager.localBrokerId(),
|
||||||
DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true);
|
DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true);
|
||||||
mockReplicaManagerDelayedShareFetch(replicaManager, delayedShareFetchPurgatory);
|
mockReplicaManagerDelayedShareFetch(mockReplicaManager, delayedShareFetchPurgatory);
|
||||||
|
|
||||||
// Initially you cannot acquire records for both sp1 and sp2.
|
// Initially you cannot acquire records for both sp1 and sp2.
|
||||||
when(sp1.maybeAcquireFetchLock()).thenReturn(true);
|
when(sp1.maybeAcquireFetchLock()).thenReturn(true);
|
||||||
|
@ -1661,13 +1663,13 @@ public class SharePartitionManagerTest {
|
||||||
|
|
||||||
SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder()
|
SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder()
|
||||||
.withPartitionCacheMap(partitionCacheMap)
|
.withPartitionCacheMap(partitionCacheMap)
|
||||||
.withReplicaManager(replicaManager)
|
.withReplicaManager(mockReplicaManager)
|
||||||
.withTimer(mockTimer)
|
.withTimer(mockTimer)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
DelayedShareFetch delayedShareFetch = DelayedShareFetchTest.DelayedShareFetchBuilder.builder()
|
DelayedShareFetch delayedShareFetch = DelayedShareFetchTest.DelayedShareFetchBuilder.builder()
|
||||||
.withShareFetchData(shareFetchData)
|
.withShareFetchData(shareFetchData)
|
||||||
.withReplicaManager(replicaManager)
|
.withReplicaManager(mockReplicaManager)
|
||||||
.withSharePartitionManager(sharePartitionManager)
|
.withSharePartitionManager(sharePartitionManager)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
|
@ -1677,7 +1679,7 @@ public class SharePartitionManagerTest {
|
||||||
// Since acquisition lock for sp1 and sp2 cannot be acquired, we should have 2 watched keys.
|
// Since acquisition lock for sp1 and sp2 cannot be acquired, we should have 2 watched keys.
|
||||||
assertEquals(2, delayedShareFetchPurgatory.watched());
|
assertEquals(2, delayedShareFetchPurgatory.watched());
|
||||||
|
|
||||||
doAnswer(invocation -> buildLogReadResult(partitionMaxBytes.keySet())).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean());
|
doAnswer(invocation -> buildLogReadResult(partitionMaxBytes.keySet())).when(mockReplicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean());
|
||||||
|
|
||||||
Map<TopicIdPartition, List<ShareAcknowledgementBatch>> acknowledgeTopics = new HashMap<>();
|
Map<TopicIdPartition, List<ShareAcknowledgementBatch>> acknowledgeTopics = new HashMap<>();
|
||||||
acknowledgeTopics.put(tp1, Arrays.asList(
|
acknowledgeTopics.put(tp1, Arrays.asList(
|
||||||
|
@ -1739,12 +1741,11 @@ public class SharePartitionManagerTest {
|
||||||
new CompletableFuture<>(),
|
new CompletableFuture<>(),
|
||||||
partitionMaxBytes,
|
partitionMaxBytes,
|
||||||
100);
|
100);
|
||||||
ReplicaManager replicaManager = mock(ReplicaManager.class);
|
|
||||||
|
|
||||||
DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>(
|
DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>(
|
||||||
"TestShareFetch", mockTimer, replicaManager.localBrokerId(),
|
"TestShareFetch", mockTimer, mockReplicaManager.localBrokerId(),
|
||||||
DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true);
|
DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true);
|
||||||
mockReplicaManagerDelayedShareFetch(replicaManager, delayedShareFetchPurgatory);
|
mockReplicaManagerDelayedShareFetch(mockReplicaManager, delayedShareFetchPurgatory);
|
||||||
|
|
||||||
// Initially you cannot acquire records for both all 3 share partitions.
|
// Initially you cannot acquire records for both all 3 share partitions.
|
||||||
when(sp1.maybeAcquireFetchLock()).thenReturn(true);
|
when(sp1.maybeAcquireFetchLock()).thenReturn(true);
|
||||||
|
@ -1759,13 +1760,13 @@ public class SharePartitionManagerTest {
|
||||||
|
|
||||||
SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder()
|
SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder()
|
||||||
.withPartitionCacheMap(partitionCacheMap)
|
.withPartitionCacheMap(partitionCacheMap)
|
||||||
.withReplicaManager(replicaManager)
|
.withReplicaManager(mockReplicaManager)
|
||||||
.withTimer(mockTimer)
|
.withTimer(mockTimer)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
DelayedShareFetch delayedShareFetch = DelayedShareFetchTest.DelayedShareFetchBuilder.builder()
|
DelayedShareFetch delayedShareFetch = DelayedShareFetchTest.DelayedShareFetchBuilder.builder()
|
||||||
.withShareFetchData(shareFetchData)
|
.withShareFetchData(shareFetchData)
|
||||||
.withReplicaManager(replicaManager)
|
.withReplicaManager(mockReplicaManager)
|
||||||
.withSharePartitionManager(sharePartitionManager)
|
.withSharePartitionManager(sharePartitionManager)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
|
@ -1832,12 +1833,11 @@ public class SharePartitionManagerTest {
|
||||||
new CompletableFuture<>(),
|
new CompletableFuture<>(),
|
||||||
partitionMaxBytes,
|
partitionMaxBytes,
|
||||||
100);
|
100);
|
||||||
ReplicaManager replicaManager = mock(ReplicaManager.class);
|
|
||||||
|
|
||||||
DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>(
|
DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>(
|
||||||
"TestShareFetch", mockTimer, replicaManager.localBrokerId(),
|
"TestShareFetch", mockTimer, mockReplicaManager.localBrokerId(),
|
||||||
DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true);
|
DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true);
|
||||||
mockReplicaManagerDelayedShareFetch(replicaManager, delayedShareFetchPurgatory);
|
mockReplicaManagerDelayedShareFetch(mockReplicaManager, delayedShareFetchPurgatory);
|
||||||
|
|
||||||
// Initially you cannot acquire records for both sp1 and sp2.
|
// Initially you cannot acquire records for both sp1 and sp2.
|
||||||
when(sp1.maybeAcquireFetchLock()).thenReturn(true);
|
when(sp1.maybeAcquireFetchLock()).thenReturn(true);
|
||||||
|
@ -1851,13 +1851,13 @@ public class SharePartitionManagerTest {
|
||||||
SharePartitionManager sharePartitionManager = spy(SharePartitionManagerBuilder.builder()
|
SharePartitionManager sharePartitionManager = spy(SharePartitionManagerBuilder.builder()
|
||||||
.withPartitionCacheMap(partitionCacheMap)
|
.withPartitionCacheMap(partitionCacheMap)
|
||||||
.withCache(cache)
|
.withCache(cache)
|
||||||
.withReplicaManager(replicaManager)
|
.withReplicaManager(mockReplicaManager)
|
||||||
.withTimer(mockTimer)
|
.withTimer(mockTimer)
|
||||||
.build());
|
.build());
|
||||||
|
|
||||||
DelayedShareFetch delayedShareFetch = DelayedShareFetchTest.DelayedShareFetchBuilder.builder()
|
DelayedShareFetch delayedShareFetch = DelayedShareFetchTest.DelayedShareFetchBuilder.builder()
|
||||||
.withShareFetchData(shareFetchData)
|
.withShareFetchData(shareFetchData)
|
||||||
.withReplicaManager(replicaManager)
|
.withReplicaManager(mockReplicaManager)
|
||||||
.withSharePartitionManager(sharePartitionManager)
|
.withSharePartitionManager(sharePartitionManager)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
|
@ -1867,7 +1867,7 @@ public class SharePartitionManagerTest {
|
||||||
// Since acquisition lock for sp1 and sp2 cannot be acquired, we should have 2 watched keys.
|
// Since acquisition lock for sp1 and sp2 cannot be acquired, we should have 2 watched keys.
|
||||||
assertEquals(2, delayedShareFetchPurgatory.watched());
|
assertEquals(2, delayedShareFetchPurgatory.watched());
|
||||||
|
|
||||||
doAnswer(invocation -> buildLogReadResult(partitionMaxBytes.keySet())).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean());
|
doAnswer(invocation -> buildLogReadResult(partitionMaxBytes.keySet())).when(mockReplicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean());
|
||||||
|
|
||||||
assertEquals(2, delayedShareFetchPurgatory.watched());
|
assertEquals(2, delayedShareFetchPurgatory.watched());
|
||||||
|
|
||||||
|
@ -1932,12 +1932,11 @@ public class SharePartitionManagerTest {
|
||||||
new CompletableFuture<>(),
|
new CompletableFuture<>(),
|
||||||
partitionMaxBytes,
|
partitionMaxBytes,
|
||||||
100);
|
100);
|
||||||
ReplicaManager replicaManager = mock(ReplicaManager.class);
|
|
||||||
|
|
||||||
DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>(
|
DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>(
|
||||||
"TestShareFetch", mockTimer, replicaManager.localBrokerId(),
|
"TestShareFetch", mockTimer, mockReplicaManager.localBrokerId(),
|
||||||
DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true);
|
DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true);
|
||||||
mockReplicaManagerDelayedShareFetch(replicaManager, delayedShareFetchPurgatory);
|
mockReplicaManagerDelayedShareFetch(mockReplicaManager, delayedShareFetchPurgatory);
|
||||||
|
|
||||||
// Initially you cannot acquire records for both all 3 share partitions.
|
// Initially you cannot acquire records for both all 3 share partitions.
|
||||||
when(sp1.maybeAcquireFetchLock()).thenReturn(true);
|
when(sp1.maybeAcquireFetchLock()).thenReturn(true);
|
||||||
|
@ -1953,13 +1952,13 @@ public class SharePartitionManagerTest {
|
||||||
SharePartitionManager sharePartitionManager = spy(SharePartitionManagerBuilder.builder()
|
SharePartitionManager sharePartitionManager = spy(SharePartitionManagerBuilder.builder()
|
||||||
.withPartitionCacheMap(partitionCacheMap)
|
.withPartitionCacheMap(partitionCacheMap)
|
||||||
.withCache(cache)
|
.withCache(cache)
|
||||||
.withReplicaManager(replicaManager)
|
.withReplicaManager(mockReplicaManager)
|
||||||
.withTimer(mockTimer)
|
.withTimer(mockTimer)
|
||||||
.build());
|
.build());
|
||||||
|
|
||||||
DelayedShareFetch delayedShareFetch = DelayedShareFetchTest.DelayedShareFetchBuilder.builder()
|
DelayedShareFetch delayedShareFetch = DelayedShareFetchTest.DelayedShareFetchBuilder.builder()
|
||||||
.withShareFetchData(shareFetchData)
|
.withShareFetchData(shareFetchData)
|
||||||
.withReplicaManager(replicaManager)
|
.withReplicaManager(mockReplicaManager)
|
||||||
.withSharePartitionManager(sharePartitionManager)
|
.withSharePartitionManager(sharePartitionManager)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
|
@ -2000,16 +1999,13 @@ public class SharePartitionManagerTest {
|
||||||
CompletableFuture<Void> pendingInitializationFuture = new CompletableFuture<>();
|
CompletableFuture<Void> pendingInitializationFuture = new CompletableFuture<>();
|
||||||
when(sp0.maybeInitialize()).thenReturn(pendingInitializationFuture);
|
when(sp0.maybeInitialize()).thenReturn(pendingInitializationFuture);
|
||||||
|
|
||||||
// Mock replica manager to verify no calls are made to fetchMessages.
|
|
||||||
ReplicaManager replicaManager = mock(ReplicaManager.class);
|
|
||||||
|
|
||||||
DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>(
|
DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>(
|
||||||
"TestShareFetch", mockTimer, replicaManager.localBrokerId(),
|
"TestShareFetch", mockTimer, mockReplicaManager.localBrokerId(),
|
||||||
DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true);
|
DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true);
|
||||||
mockReplicaManagerDelayedShareFetch(replicaManager, delayedShareFetchPurgatory);
|
mockReplicaManagerDelayedShareFetch(mockReplicaManager, delayedShareFetchPurgatory);
|
||||||
|
|
||||||
SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder()
|
SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder()
|
||||||
.withPartitionCacheMap(partitionCacheMap).withReplicaManager(replicaManager).withTimer(mockTimer)
|
.withPartitionCacheMap(partitionCacheMap).withReplicaManager(mockReplicaManager).withTimer(mockTimer)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
CompletableFuture<Map<TopicIdPartition, ShareFetchResponseData.PartitionData>> future =
|
CompletableFuture<Map<TopicIdPartition, ShareFetchResponseData.PartitionData>> future =
|
||||||
|
@ -2021,7 +2017,7 @@ public class SharePartitionManagerTest {
|
||||||
() -> "Processing in delayed share fetch queue never ended.");
|
() -> "Processing in delayed share fetch queue never ended.");
|
||||||
assertTrue(future.join().isEmpty());
|
assertTrue(future.join().isEmpty());
|
||||||
// Verify that replica manager fetch is not called.
|
// Verify that replica manager fetch is not called.
|
||||||
Mockito.verify(replicaManager, times(0)).readFromLog(
|
Mockito.verify(mockReplicaManager, times(0)).readFromLog(
|
||||||
any(), any(), any(ReplicaQuota.class), anyBoolean());
|
any(), any(), any(ReplicaQuota.class), anyBoolean());
|
||||||
// Complete the pending initialization future.
|
// Complete the pending initialization future.
|
||||||
pendingInitializationFuture.complete(null);
|
pendingInitializationFuture.complete(null);
|
||||||
|
@ -2039,14 +2035,13 @@ public class SharePartitionManagerTest {
|
||||||
Map<SharePartitionKey, SharePartition> partitionCacheMap = new HashMap<>();
|
Map<SharePartitionKey, SharePartition> partitionCacheMap = new HashMap<>();
|
||||||
partitionCacheMap.put(new SharePartitionKey(groupId, tp0), sp0);
|
partitionCacheMap.put(new SharePartitionKey(groupId, tp0), sp0);
|
||||||
|
|
||||||
ReplicaManager replicaManager = mock(ReplicaManager.class);
|
|
||||||
DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>(
|
DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>(
|
||||||
"TestShareFetch", mockTimer, replicaManager.localBrokerId(),
|
"TestShareFetch", mockTimer, mockReplicaManager.localBrokerId(),
|
||||||
DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true);
|
DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true);
|
||||||
mockReplicaManagerDelayedShareFetch(replicaManager, delayedShareFetchPurgatory);
|
mockReplicaManagerDelayedShareFetch(mockReplicaManager, delayedShareFetchPurgatory);
|
||||||
|
|
||||||
SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder()
|
SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder()
|
||||||
.withPartitionCacheMap(partitionCacheMap).withReplicaManager(replicaManager).withTimer(mockTimer)
|
.withPartitionCacheMap(partitionCacheMap).withReplicaManager(mockReplicaManager).withTimer(mockTimer)
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
// Return LeaderNotAvailableException to simulate initialization failure.
|
// Return LeaderNotAvailableException to simulate initialization failure.
|
||||||
|
@ -2061,6 +2056,8 @@ public class SharePartitionManagerTest {
|
||||||
// between SharePartitionManager and SharePartition to retry the request as SharePartition is not yet ready.
|
// between SharePartitionManager and SharePartition to retry the request as SharePartition is not yet ready.
|
||||||
assertFalse(future.isCompletedExceptionally());
|
assertFalse(future.isCompletedExceptionally());
|
||||||
assertTrue(future.join().isEmpty());
|
assertTrue(future.join().isEmpty());
|
||||||
|
// Verify that the share partition is still in the cache on LeaderNotAvailableException.
|
||||||
|
assertEquals(1, partitionCacheMap.size());
|
||||||
|
|
||||||
// Return IllegalStateException to simulate initialization failure.
|
// Return IllegalStateException to simulate initialization failure.
|
||||||
when(sp0.maybeInitialize()).thenReturn(FutureUtils.failedFuture(new IllegalStateException("Illegal state")));
|
when(sp0.maybeInitialize()).thenReturn(FutureUtils.failedFuture(new IllegalStateException("Illegal state")));
|
||||||
|
@ -2069,9 +2066,11 @@ public class SharePartitionManagerTest {
|
||||||
future::isDone,
|
future::isDone,
|
||||||
DELAYED_SHARE_FETCH_TIMEOUT_MS,
|
DELAYED_SHARE_FETCH_TIMEOUT_MS,
|
||||||
() -> "Processing in delayed share fetch queue never ended.");
|
() -> "Processing in delayed share fetch queue never ended.");
|
||||||
assertTrue(future.isCompletedExceptionally());
|
validateShareFetchFutureException(future, tp0, Errors.UNKNOWN_SERVER_ERROR, "Illegal state");
|
||||||
assertFutureThrows(future, IllegalStateException.class);
|
assertTrue(partitionCacheMap.isEmpty());
|
||||||
|
|
||||||
|
// The last exception removes the share partition from the cache hence re-add the share partition to cache.
|
||||||
|
partitionCacheMap.put(new SharePartitionKey(groupId, tp0), sp0);
|
||||||
// Return CoordinatorNotAvailableException to simulate initialization failure.
|
// Return CoordinatorNotAvailableException to simulate initialization failure.
|
||||||
when(sp0.maybeInitialize()).thenReturn(FutureUtils.failedFuture(new CoordinatorNotAvailableException("Coordinator not available")));
|
when(sp0.maybeInitialize()).thenReturn(FutureUtils.failedFuture(new CoordinatorNotAvailableException("Coordinator not available")));
|
||||||
future = sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, partitionMaxBytes);
|
future = sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, partitionMaxBytes);
|
||||||
|
@ -2079,9 +2078,11 @@ public class SharePartitionManagerTest {
|
||||||
future::isDone,
|
future::isDone,
|
||||||
DELAYED_SHARE_FETCH_TIMEOUT_MS,
|
DELAYED_SHARE_FETCH_TIMEOUT_MS,
|
||||||
() -> "Processing in delayed share fetch queue never ended.");
|
() -> "Processing in delayed share fetch queue never ended.");
|
||||||
assertTrue(future.isCompletedExceptionally());
|
validateShareFetchFutureException(future, tp0, Errors.COORDINATOR_NOT_AVAILABLE, "Coordinator not available");
|
||||||
assertFutureThrows(future, CoordinatorNotAvailableException.class);
|
assertTrue(partitionCacheMap.isEmpty());
|
||||||
|
|
||||||
|
// The last exception removes the share partition from the cache hence re-add the share partition to cache.
|
||||||
|
partitionCacheMap.put(new SharePartitionKey(groupId, tp0), sp0);
|
||||||
// Return InvalidRequestException to simulate initialization failure.
|
// Return InvalidRequestException to simulate initialization failure.
|
||||||
when(sp0.maybeInitialize()).thenReturn(FutureUtils.failedFuture(new InvalidRequestException("Invalid request")));
|
when(sp0.maybeInitialize()).thenReturn(FutureUtils.failedFuture(new InvalidRequestException("Invalid request")));
|
||||||
future = sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, partitionMaxBytes);
|
future = sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, partitionMaxBytes);
|
||||||
|
@ -2089,21 +2090,19 @@ public class SharePartitionManagerTest {
|
||||||
future::isDone,
|
future::isDone,
|
||||||
DELAYED_SHARE_FETCH_TIMEOUT_MS,
|
DELAYED_SHARE_FETCH_TIMEOUT_MS,
|
||||||
() -> "Processing in delayed share fetch queue never ended.");
|
() -> "Processing in delayed share fetch queue never ended.");
|
||||||
assertTrue(future.isCompletedExceptionally());
|
validateShareFetchFutureException(future, tp0, Errors.INVALID_REQUEST, "Invalid request");
|
||||||
assertFutureThrows(future, InvalidRequestException.class);
|
assertTrue(partitionCacheMap.isEmpty());
|
||||||
|
|
||||||
|
// The last exception removes the share partition from the cache hence re-add the share partition to cache.
|
||||||
|
partitionCacheMap.put(new SharePartitionKey(groupId, tp0), sp0);
|
||||||
// Return FencedStateEpochException to simulate initialization failure.
|
// Return FencedStateEpochException to simulate initialization failure.
|
||||||
when(sp0.maybeInitialize()).thenReturn(FutureUtils.failedFuture(new FencedStateEpochException("Fenced state epoch")));
|
when(sp0.maybeInitialize()).thenReturn(FutureUtils.failedFuture(new FencedStateEpochException("Fenced state epoch")));
|
||||||
// Assert that partitionCacheMap contains instance before the fetch request.
|
|
||||||
assertEquals(1, partitionCacheMap.size());
|
|
||||||
future = sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, partitionMaxBytes);
|
future = sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, partitionMaxBytes);
|
||||||
TestUtils.waitForCondition(
|
TestUtils.waitForCondition(
|
||||||
future::isDone,
|
future::isDone,
|
||||||
DELAYED_SHARE_FETCH_TIMEOUT_MS,
|
DELAYED_SHARE_FETCH_TIMEOUT_MS,
|
||||||
() -> "Processing in delayed share fetch queue never ended.");
|
() -> "Processing in delayed share fetch queue never ended.");
|
||||||
assertTrue(future.isCompletedExceptionally());
|
validateShareFetchFutureException(future, tp0, Errors.FENCED_STATE_EPOCH, "Fenced state epoch");
|
||||||
assertFutureThrows(future, FencedStateEpochException.class);
|
|
||||||
// Verify that the share partition is removed from the cache.
|
|
||||||
assertTrue(partitionCacheMap.isEmpty());
|
assertTrue(partitionCacheMap.isEmpty());
|
||||||
|
|
||||||
// The last exception removes the share partition from the cache hence re-add the share partition to cache.
|
// The last exception removes the share partition from the cache hence re-add the share partition to cache.
|
||||||
|
@ -2115,9 +2114,7 @@ public class SharePartitionManagerTest {
|
||||||
future::isDone,
|
future::isDone,
|
||||||
DELAYED_SHARE_FETCH_TIMEOUT_MS,
|
DELAYED_SHARE_FETCH_TIMEOUT_MS,
|
||||||
() -> "Processing in delayed share fetch queue never ended.");
|
() -> "Processing in delayed share fetch queue never ended.");
|
||||||
assertTrue(future.isCompletedExceptionally());
|
validateShareFetchFutureException(future, tp0, Errors.NOT_LEADER_OR_FOLLOWER, "Not leader or follower");
|
||||||
assertFutureThrows(future, NotLeaderOrFollowerException.class);
|
|
||||||
// Verify that the share partition is removed from the cache.
|
|
||||||
assertTrue(partitionCacheMap.isEmpty());
|
assertTrue(partitionCacheMap.isEmpty());
|
||||||
|
|
||||||
// The last exception removes the share partition from the cache hence re-add the share partition to cache.
|
// The last exception removes the share partition from the cache hence re-add the share partition to cache.
|
||||||
|
@ -2129,10 +2126,11 @@ public class SharePartitionManagerTest {
|
||||||
future::isDone,
|
future::isDone,
|
||||||
DELAYED_SHARE_FETCH_TIMEOUT_MS,
|
DELAYED_SHARE_FETCH_TIMEOUT_MS,
|
||||||
() -> "Processing in delayed share fetch queue never ended.");
|
() -> "Processing in delayed share fetch queue never ended.");
|
||||||
assertTrue(future.isCompletedExceptionally());
|
validateShareFetchFutureException(future, tp0, Errors.UNKNOWN_SERVER_ERROR, "Runtime exception");
|
||||||
assertFutureThrows(future, RuntimeException.class);
|
assertTrue(partitionCacheMap.isEmpty());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
public void testShareFetchProcessingExceptions() throws Exception {
|
public void testShareFetchProcessingExceptions() throws Exception {
|
||||||
|
@ -2140,12 +2138,10 @@ public class SharePartitionManagerTest {
|
||||||
TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0));
|
TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0));
|
||||||
Map<TopicIdPartition, Integer> partitionMaxBytes = Collections.singletonMap(tp0, PARTITION_MAX_BYTES);
|
Map<TopicIdPartition, Integer> partitionMaxBytes = Collections.singletonMap(tp0, PARTITION_MAX_BYTES);
|
||||||
|
|
||||||
SharePartition sp0 = mock(SharePartition.class);
|
|
||||||
Map<SharePartitionKey, SharePartition> partitionCacheMap = (Map<SharePartitionKey, SharePartition>) mock(Map.class);
|
Map<SharePartitionKey, SharePartition> partitionCacheMap = (Map<SharePartitionKey, SharePartition>) mock(Map.class);
|
||||||
// Throw the exception for first fetch request. Return share partition for next.
|
// Throw the exception for first fetch request. Return share partition for next.
|
||||||
when(partitionCacheMap.computeIfAbsent(any(), any()))
|
when(partitionCacheMap.computeIfAbsent(any(), any()))
|
||||||
.thenThrow(new RuntimeException("Error creating instance"))
|
.thenThrow(new RuntimeException("Error creating instance"));
|
||||||
.thenReturn(sp0);
|
|
||||||
|
|
||||||
SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder()
|
SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder()
|
||||||
.withPartitionCacheMap(partitionCacheMap)
|
.withPartitionCacheMap(partitionCacheMap)
|
||||||
|
@ -2157,19 +2153,204 @@ public class SharePartitionManagerTest {
|
||||||
future::isDone,
|
future::isDone,
|
||||||
DELAYED_SHARE_FETCH_TIMEOUT_MS,
|
DELAYED_SHARE_FETCH_TIMEOUT_MS,
|
||||||
() -> "Processing for delayed share fetch request not finished.");
|
() -> "Processing for delayed share fetch request not finished.");
|
||||||
assertTrue(future.isCompletedExceptionally());
|
validateShareFetchFutureException(future, tp0, Errors.UNKNOWN_SERVER_ERROR, "Error creating instance");
|
||||||
assertFutureThrows(future, RuntimeException.class, "Error creating instance");
|
}
|
||||||
|
|
||||||
// Throw exception from share partition for second fetch request.
|
@Test
|
||||||
when(sp0.maybeInitialize()).thenThrow(new RuntimeException("Error initializing instance"));
|
public void testSharePartitionInitializationFailure() throws Exception {
|
||||||
|
String groupId = "grp";
|
||||||
|
TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0));
|
||||||
|
Map<TopicIdPartition, Integer> partitionMaxBytes = Collections.singletonMap(tp0, PARTITION_MAX_BYTES);
|
||||||
|
|
||||||
|
// Send map to check no share partition is created.
|
||||||
|
Map<SharePartitionKey, SharePartition> partitionCacheMap = new HashMap<>();
|
||||||
|
// Validate when partition is not the leader.
|
||||||
|
Partition partition = mock(Partition.class);
|
||||||
|
when(partition.isLeader()).thenReturn(false);
|
||||||
|
|
||||||
|
ReplicaManager replicaManager = mock(ReplicaManager.class);
|
||||||
|
// First check should throw KafkaStorageException, second check should return partition which
|
||||||
|
// is not leader.
|
||||||
|
when(replicaManager.getPartitionOrException(any()))
|
||||||
|
.thenThrow(new KafkaStorageException("Exception"))
|
||||||
|
.thenReturn(partition);
|
||||||
|
SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder()
|
||||||
|
.withReplicaManager(replicaManager)
|
||||||
|
.withPartitionCacheMap(partitionCacheMap)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
// Validate when exception is thrown.
|
||||||
|
CompletableFuture<Map<TopicIdPartition, ShareFetchResponseData.PartitionData>> future =
|
||||||
|
sharePartitionManager.fetchMessages(groupId, Uuid.randomUuid().toString(), FETCH_PARAMS, partitionMaxBytes);
|
||||||
|
TestUtils.waitForCondition(
|
||||||
|
future::isDone,
|
||||||
|
DELAYED_SHARE_FETCH_TIMEOUT_MS,
|
||||||
|
() -> "Processing for delayed share fetch request not finished.");
|
||||||
|
validateShareFetchFutureException(future, tp0, Errors.KAFKA_STORAGE_ERROR, "Exception");
|
||||||
|
assertTrue(partitionCacheMap.isEmpty());
|
||||||
|
|
||||||
|
// Validate when partition is not leader.
|
||||||
future = sharePartitionManager.fetchMessages(groupId, Uuid.randomUuid().toString(), FETCH_PARAMS, partitionMaxBytes);
|
future = sharePartitionManager.fetchMessages(groupId, Uuid.randomUuid().toString(), FETCH_PARAMS, partitionMaxBytes);
|
||||||
TestUtils.waitForCondition(
|
TestUtils.waitForCondition(
|
||||||
future::isDone,
|
future::isDone,
|
||||||
DELAYED_SHARE_FETCH_TIMEOUT_MS,
|
DELAYED_SHARE_FETCH_TIMEOUT_MS,
|
||||||
() -> "Processing for delayed share fetch request not finished.");
|
() -> "Processing for delayed share fetch request not finished.");
|
||||||
assertTrue(future.isCompletedExceptionally());
|
validateShareFetchFutureException(future, tp0, Errors.NOT_LEADER_OR_FOLLOWER);
|
||||||
assertFutureThrows(future, RuntimeException.class, "Error initializing instance");
|
assertTrue(partitionCacheMap.isEmpty());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSharePartitionPartialInitializationFailure() throws Exception {
|
||||||
|
String groupId = "grp";
|
||||||
|
Uuid memberId1 = Uuid.randomUuid();
|
||||||
|
TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0));
|
||||||
|
TopicIdPartition tp1 = new TopicIdPartition(memberId1, new TopicPartition("foo", 1));
|
||||||
|
Map<TopicIdPartition, Integer> partitionMaxBytes = Map.of(tp0, PARTITION_MAX_BYTES, tp1, PARTITION_MAX_BYTES);
|
||||||
|
|
||||||
|
// Mark partition1 as not the leader.
|
||||||
|
Partition partition1 = mock(Partition.class);
|
||||||
|
when(partition1.isLeader()).thenReturn(false);
|
||||||
|
|
||||||
|
ReplicaManager replicaManager = mock(ReplicaManager.class);
|
||||||
|
when(replicaManager.getPartitionOrException(any()))
|
||||||
|
.thenReturn(partition1);
|
||||||
|
|
||||||
|
SharePartition sp1 = mock(SharePartition.class);
|
||||||
|
Map<SharePartitionKey, SharePartition> partitionCacheMap = new HashMap<>();
|
||||||
|
partitionCacheMap.put(new SharePartitionKey(groupId, tp1), sp1);
|
||||||
|
|
||||||
|
when(sp1.maybeAcquireFetchLock()).thenReturn(true);
|
||||||
|
when(sp1.canAcquireRecords()).thenReturn(true);
|
||||||
|
when(sp1.maybeInitialize()).thenReturn(CompletableFuture.completedFuture(null));
|
||||||
|
when(sp1.acquire(anyString(), anyInt(), any())).thenReturn(new ShareAcquiredRecords(Collections.emptyList(), 0));
|
||||||
|
|
||||||
|
DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>(
|
||||||
|
"TestShareFetch", mockTimer, replicaManager.localBrokerId(),
|
||||||
|
DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true);
|
||||||
|
mockReplicaManagerDelayedShareFetch(replicaManager, delayedShareFetchPurgatory);
|
||||||
|
|
||||||
|
doAnswer(invocation -> buildLogReadResult(partitionMaxBytes.keySet())).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean());
|
||||||
|
|
||||||
|
SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder()
|
||||||
|
.withReplicaManager(replicaManager)
|
||||||
|
.withPartitionCacheMap(partitionCacheMap)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
// Validate when exception is thrown.
|
||||||
|
CompletableFuture<Map<TopicIdPartition, ShareFetchResponseData.PartitionData>> future =
|
||||||
|
sharePartitionManager.fetchMessages(groupId, Uuid.randomUuid().toString(), FETCH_PARAMS, partitionMaxBytes);
|
||||||
|
assertTrue(future.isDone());
|
||||||
|
assertFalse(future.isCompletedExceptionally());
|
||||||
|
|
||||||
|
Map<TopicIdPartition, PartitionData> partitionDataMap = future.get();
|
||||||
|
// For now only 1 successful partition is included, this will be fixed in subsequents PRs.
|
||||||
|
assertEquals(1, partitionDataMap.size());
|
||||||
|
assertTrue(partitionDataMap.containsKey(tp1));
|
||||||
|
assertEquals(Errors.NONE.code(), partitionDataMap.get(tp1).errorCode());
|
||||||
|
|
||||||
|
Mockito.verify(replicaManager, times(1)).readFromLog(
|
||||||
|
any(), any(), any(ReplicaQuota.class), anyBoolean());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testReplicaManagerFetchException() {
|
||||||
|
String groupId = "grp";
|
||||||
|
Uuid memberId = Uuid.randomUuid();
|
||||||
|
TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0));
|
||||||
|
Map<TopicIdPartition, Integer> partitionMaxBytes = Collections.singletonMap(tp0, PARTITION_MAX_BYTES);
|
||||||
|
|
||||||
|
SharePartition sp0 = mock(SharePartition.class);
|
||||||
|
when(sp0.maybeAcquireFetchLock()).thenReturn(true);
|
||||||
|
when(sp0.canAcquireRecords()).thenReturn(true);
|
||||||
|
when(sp0.maybeInitialize()).thenReturn(CompletableFuture.completedFuture(null));
|
||||||
|
Map<SharePartitionKey, SharePartition> partitionCacheMap = new HashMap<>();
|
||||||
|
partitionCacheMap.put(new SharePartitionKey(groupId, tp0), sp0);
|
||||||
|
|
||||||
|
DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>(
|
||||||
|
"TestShareFetch", mockTimer, mockReplicaManager.localBrokerId(),
|
||||||
|
DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true);
|
||||||
|
mockReplicaManagerDelayedShareFetch(mockReplicaManager, delayedShareFetchPurgatory);
|
||||||
|
|
||||||
|
doThrow(new RuntimeException("Exception")).when(mockReplicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean());
|
||||||
|
|
||||||
|
SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder()
|
||||||
|
.withPartitionCacheMap(partitionCacheMap)
|
||||||
|
.withReplicaManager(mockReplicaManager)
|
||||||
|
.withTimer(mockTimer)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
CompletableFuture<Map<TopicIdPartition, ShareFetchResponseData.PartitionData>> future =
|
||||||
|
sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, partitionMaxBytes);
|
||||||
|
validateShareFetchFutureException(future, tp0, Errors.UNKNOWN_SERVER_ERROR, "Exception");
|
||||||
|
// Verify that the share partition is still in the cache on exception.
|
||||||
|
assertEquals(1, partitionCacheMap.size());
|
||||||
|
|
||||||
|
// Throw NotLeaderOrFollowerException from replica manager fetch which should evict instance from the cache.
|
||||||
|
doThrow(new NotLeaderOrFollowerException("Leader exception")).when(mockReplicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean());
|
||||||
|
|
||||||
|
future = sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, partitionMaxBytes);
|
||||||
|
validateShareFetchFutureException(future, tp0, Errors.NOT_LEADER_OR_FOLLOWER, "Leader exception");
|
||||||
|
assertTrue(partitionCacheMap.isEmpty());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testReplicaManagerFetchMultipleSharePartitionsException() {
|
||||||
|
String groupId = "grp";
|
||||||
|
Uuid memberId = Uuid.randomUuid();
|
||||||
|
|
||||||
|
TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0));
|
||||||
|
TopicIdPartition tp1 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("bar", 0));
|
||||||
|
Map<TopicIdPartition, Integer> partitionMaxBytes = new HashMap<>();
|
||||||
|
partitionMaxBytes.put(tp0, PARTITION_MAX_BYTES);
|
||||||
|
partitionMaxBytes.put(tp1, PARTITION_MAX_BYTES);
|
||||||
|
|
||||||
|
SharePartition sp0 = mock(SharePartition.class);
|
||||||
|
when(sp0.maybeAcquireFetchLock()).thenReturn(true);
|
||||||
|
when(sp0.canAcquireRecords()).thenReturn(true);
|
||||||
|
when(sp0.maybeInitialize()).thenReturn(CompletableFuture.completedFuture(null));
|
||||||
|
|
||||||
|
SharePartition sp1 = mock(SharePartition.class);
|
||||||
|
// Do not make the share partition acquirable hence it shouldn't be removed from the cache,
|
||||||
|
// as it won't be part of replica manger readFromLog request.
|
||||||
|
when(sp1.maybeAcquireFetchLock()).thenReturn(false);
|
||||||
|
when(sp1.maybeInitialize()).thenReturn(CompletableFuture.completedFuture(null));
|
||||||
|
|
||||||
|
Map<SharePartitionKey, SharePartition> partitionCacheMap = new HashMap<>();
|
||||||
|
partitionCacheMap.put(new SharePartitionKey(groupId, tp0), sp0);
|
||||||
|
partitionCacheMap.put(new SharePartitionKey(groupId, tp1), sp1);
|
||||||
|
|
||||||
|
DelayedOperationPurgatory<DelayedShareFetch> delayedShareFetchPurgatory = new DelayedOperationPurgatory<>(
|
||||||
|
"TestShareFetch", mockTimer, mockReplicaManager.localBrokerId(),
|
||||||
|
DELAYED_SHARE_FETCH_PURGATORY_PURGE_INTERVAL, true, true);
|
||||||
|
mockReplicaManagerDelayedShareFetch(mockReplicaManager, delayedShareFetchPurgatory);
|
||||||
|
|
||||||
|
// Throw FencedStateEpochException from replica manager fetch which should evict instance from the cache.
|
||||||
|
doThrow(new FencedStateEpochException("Fenced exception")).when(mockReplicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean());
|
||||||
|
|
||||||
|
SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder()
|
||||||
|
.withPartitionCacheMap(partitionCacheMap)
|
||||||
|
.withReplicaManager(mockReplicaManager)
|
||||||
|
.withTimer(mockTimer)
|
||||||
|
.build();
|
||||||
|
|
||||||
|
CompletableFuture<Map<TopicIdPartition, ShareFetchResponseData.PartitionData>> future =
|
||||||
|
sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, partitionMaxBytes);
|
||||||
|
validateShareFetchFutureException(future, tp0, Errors.FENCED_STATE_EPOCH, "Fenced exception");
|
||||||
|
// Verify that tp1 is still in the cache on exception.
|
||||||
|
assertEquals(1, partitionCacheMap.size());
|
||||||
|
assertEquals(sp1, partitionCacheMap.get(new SharePartitionKey(groupId, tp1)));
|
||||||
|
|
||||||
|
// Make sp1 acquirable and add sp0 back in partition cache. Both share partitions should be
|
||||||
|
// removed from the cache.
|
||||||
|
when(sp1.maybeAcquireFetchLock()).thenReturn(true);
|
||||||
|
when(sp1.canAcquireRecords()).thenReturn(true);
|
||||||
|
partitionCacheMap.put(new SharePartitionKey(groupId, tp0), sp0);
|
||||||
|
// Throw FencedStateEpochException from replica manager fetch which should evict instance from the cache.
|
||||||
|
doThrow(new FencedStateEpochException("Fenced exception again")).when(mockReplicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean());
|
||||||
|
|
||||||
|
future = sharePartitionManager.fetchMessages(groupId, memberId.toString(), FETCH_PARAMS, partitionMaxBytes);
|
||||||
|
validateShareFetchFutureException(future, List.of(tp0, tp1), Errors.FENCED_STATE_EPOCH, "Fenced exception again");
|
||||||
|
assertTrue(partitionCacheMap.isEmpty());
|
||||||
}
|
}
|
||||||
|
|
||||||
private ShareFetchResponseData.PartitionData noErrorShareFetchResponse() {
|
private ShareFetchResponseData.PartitionData noErrorShareFetchResponse() {
|
||||||
|
@ -2233,6 +2414,37 @@ public class SharePartitionManagerTest {
|
||||||
assertEquals(expectedValidSet, actualValidPartitions);
|
assertEquals(expectedValidSet, actualValidPartitions);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private Partition mockPartition() {
|
||||||
|
Partition partition = mock(Partition.class);
|
||||||
|
when(partition.isLeader()).thenReturn(true);
|
||||||
|
when(partition.getLeaderEpoch()).thenReturn(1);
|
||||||
|
|
||||||
|
return partition;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void validateShareFetchFutureException(CompletableFuture<Map<TopicIdPartition, PartitionData>> future,
|
||||||
|
TopicIdPartition topicIdPartition, Errors error) {
|
||||||
|
validateShareFetchFutureException(future, Collections.singletonList(topicIdPartition), error, null);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void validateShareFetchFutureException(CompletableFuture<Map<TopicIdPartition, PartitionData>> future,
|
||||||
|
TopicIdPartition topicIdPartition, Errors error, String message) {
|
||||||
|
validateShareFetchFutureException(future, Collections.singletonList(topicIdPartition), error, message);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void validateShareFetchFutureException(CompletableFuture<Map<TopicIdPartition, PartitionData>> future,
|
||||||
|
List<TopicIdPartition> topicIdPartitions, Errors error, String message) {
|
||||||
|
assertFalse(future.isCompletedExceptionally());
|
||||||
|
Map<TopicIdPartition, ShareFetchResponseData.PartitionData> result = future.join();
|
||||||
|
assertEquals(topicIdPartitions.size(), result.size());
|
||||||
|
topicIdPartitions.forEach(topicIdPartition -> {
|
||||||
|
assertTrue(result.containsKey(topicIdPartition));
|
||||||
|
assertEquals(topicIdPartition.partition(), result.get(topicIdPartition).partitionIndex());
|
||||||
|
assertEquals(error.code(), result.get(topicIdPartition).errorCode());
|
||||||
|
assertEquals(message, result.get(topicIdPartition).errorMessage());
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
static Seq<Tuple2<TopicIdPartition, LogReadResult>> buildLogReadResult(Set<TopicIdPartition> topicIdPartitions) {
|
static Seq<Tuple2<TopicIdPartition, LogReadResult>> buildLogReadResult(Set<TopicIdPartition> topicIdPartitions) {
|
||||||
List<Tuple2<TopicIdPartition, LogReadResult>> logReadResults = new ArrayList<>();
|
List<Tuple2<TopicIdPartition, LogReadResult>> logReadResults = new ArrayList<>();
|
||||||
topicIdPartitions.forEach(topicIdPartition -> logReadResults.add(new Tuple2<>(topicIdPartition, new LogReadResult(
|
topicIdPartitions.forEach(topicIdPartition -> logReadResults.add(new Tuple2<>(topicIdPartition, new LogReadResult(
|
||||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -33,6 +33,7 @@ public class FetchParams {
|
||||||
public final int maxBytes;
|
public final int maxBytes;
|
||||||
public final FetchIsolation isolation;
|
public final FetchIsolation isolation;
|
||||||
public final Optional<ClientMetadata> clientMetadata;
|
public final Optional<ClientMetadata> clientMetadata;
|
||||||
|
public final boolean shareFetchRequest;
|
||||||
|
|
||||||
public FetchParams(short requestVersion,
|
public FetchParams(short requestVersion,
|
||||||
int replicaId,
|
int replicaId,
|
||||||
|
@ -42,6 +43,18 @@ public class FetchParams {
|
||||||
int maxBytes,
|
int maxBytes,
|
||||||
FetchIsolation isolation,
|
FetchIsolation isolation,
|
||||||
Optional<ClientMetadata> clientMetadata) {
|
Optional<ClientMetadata> clientMetadata) {
|
||||||
|
this(requestVersion, replicaId, replicaEpoch, maxWaitMs, minBytes, maxBytes, isolation, clientMetadata, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
public FetchParams(short requestVersion,
|
||||||
|
int replicaId,
|
||||||
|
long replicaEpoch,
|
||||||
|
long maxWaitMs,
|
||||||
|
int minBytes,
|
||||||
|
int maxBytes,
|
||||||
|
FetchIsolation isolation,
|
||||||
|
Optional<ClientMetadata> clientMetadata,
|
||||||
|
boolean shareFetchRequest) {
|
||||||
Objects.requireNonNull(isolation);
|
Objects.requireNonNull(isolation);
|
||||||
Objects.requireNonNull(clientMetadata);
|
Objects.requireNonNull(clientMetadata);
|
||||||
this.requestVersion = requestVersion;
|
this.requestVersion = requestVersion;
|
||||||
|
@ -52,6 +65,7 @@ public class FetchParams {
|
||||||
this.maxBytes = maxBytes;
|
this.maxBytes = maxBytes;
|
||||||
this.isolation = isolation;
|
this.isolation = isolation;
|
||||||
this.clientMetadata = clientMetadata;
|
this.clientMetadata = clientMetadata;
|
||||||
|
this.shareFetchRequest = shareFetchRequest;
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean isFromFollower() {
|
public boolean isFromFollower() {
|
||||||
|
@ -67,7 +81,7 @@ public class FetchParams {
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean fetchOnlyLeader() {
|
public boolean fetchOnlyLeader() {
|
||||||
return isFromFollower() || (isFromConsumer() && !clientMetadata.isPresent());
|
return isFromFollower() || (isFromConsumer() && !clientMetadata.isPresent()) || shareFetchRequest;
|
||||||
}
|
}
|
||||||
|
|
||||||
public boolean hardMaxBytesLimit() {
|
public boolean hardMaxBytesLimit() {
|
||||||
|
@ -86,7 +100,8 @@ public class FetchParams {
|
||||||
&& minBytes == that.minBytes
|
&& minBytes == that.minBytes
|
||||||
&& maxBytes == that.maxBytes
|
&& maxBytes == that.maxBytes
|
||||||
&& isolation.equals(that.isolation)
|
&& isolation.equals(that.isolation)
|
||||||
&& clientMetadata.equals(that.clientMetadata);
|
&& clientMetadata.equals(that.clientMetadata)
|
||||||
|
&& shareFetchRequest == that.shareFetchRequest;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -99,6 +114,7 @@ public class FetchParams {
|
||||||
result = 31 * result + maxBytes;
|
result = 31 * result + maxBytes;
|
||||||
result = 31 * result + isolation.hashCode();
|
result = 31 * result + isolation.hashCode();
|
||||||
result = 31 * result + clientMetadata.hashCode();
|
result = 31 * result + clientMetadata.hashCode();
|
||||||
|
result = 31 * result + Boolean.hashCode(shareFetchRequest);
|
||||||
return result;
|
return result;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -113,6 +129,7 @@ public class FetchParams {
|
||||||
", maxBytes=" + maxBytes +
|
", maxBytes=" + maxBytes +
|
||||||
", isolation=" + isolation +
|
", isolation=" + isolation +
|
||||||
", clientMetadata=" + clientMetadata +
|
", clientMetadata=" + clientMetadata +
|
||||||
|
", shareFetchRequest=" + shareFetchRequest +
|
||||||
')';
|
')';
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue