AndrewJSchofield commented on code in PR #16842:
URL: https://github.com/apache/kafka/pull/16842#discussion_r1812640859
##########
core/src/main/java/kafka/server/share/SharePartitionManager.java:
##########
@@ -606,22 +641,51 @@ private void maybeCompleteInitializationWithException(
return;
}
+ // 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);
+ maybeCompleteShareFetchExceptionally(future,
Collections.singletonList(topicIdPartition), throwable);
+ }
+
+ private void handleSharePartitionException(
Review Comment:
I suggest "fenced" should appear in the method name. This is essentially
seeing if the exception indicates fencing and then discarding the share
partition from the cache.
##########
server-common/src/main/java/org/apache/kafka/server/storage/log/FetchParams.java:
##########
@@ -42,6 +43,18 @@ public FetchParams(short requestVersion,
int maxBytes,
FetchIsolation isolation,
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) {
Review Comment:
I wonder whether this is a bit literal. Why are we supplying a
`ClientMetadata` here in the share fetch case? That seems to me to be concerned
with fetch-from-follower. If we didn't supply a `ClientMetadata`, then
`fetchOnlyLeader()` would already return true without needing the new flag.
##########
core/src/main/java/kafka/server/share/SharePartitionManager.java:
##########
@@ -606,22 +641,51 @@ private void maybeCompleteInitializationWithException(
return;
}
+ // 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);
+ maybeCompleteShareFetchExceptionally(future,
Collections.singletonList(topicIdPartition), throwable);
+ }
+
+ private void handleSharePartitionException(
+ SharePartitionKey sharePartitionKey,
+ Throwable throwable
+ ) {
if (throwable instanceof NotLeaderOrFollowerException || throwable
instanceof FencedStateEpochException) {
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.
// 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.
partitionCacheMap.remove(sharePartitionKey);
- future.completeExceptionally(throwable);
- return;
}
+ }
- // 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. As of now this state is in-recoverable for the
broker, and we should
- // investigate the root cause of the error.
- log.error("Error initializing share partition with key {}",
sharePartitionKey, throwable);
- future.completeExceptionally(throwable);
+ private void
maybeCompleteShareFetchExceptionally(CompletableFuture<Map<TopicIdPartition,
PartitionData>> future,
+ Collection<TopicIdPartition> topicIdPartitions, Throwable throwable) {
+ if (!future.isDone()) {
+ Errors error = Errors.forException(throwable);
+
future.complete(topicIdPartitions.stream().collect(Collectors.toMap(
+ tp -> tp, tp -> new
PartitionData().setErrorCode(error.code()).setErrorMessage(error.message()))));
+ }
+ }
+
+ private int leaderEpoch(TopicPartition tp) {
+ Either<Errors, Partition> partitionOrError =
replicaManager.getPartitionOrError(tp);
Review Comment:
I've checked the error codes here and the KIP looks right to me.
##########
core/src/main/java/kafka/server/share/SharePartitionManager.java:
##########
@@ -606,22 +641,51 @@ private void maybeCompleteInitializationWithException(
return;
}
+ // 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);
+ maybeCompleteShareFetchExceptionally(future,
Collections.singletonList(topicIdPartition), throwable);
+ }
+
+ private void handleSharePartitionException(
+ SharePartitionKey sharePartitionKey,
+ Throwable throwable
+ ) {
if (throwable instanceof NotLeaderOrFollowerException || throwable
instanceof FencedStateEpochException) {
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.
// 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.
partitionCacheMap.remove(sharePartitionKey);
Review Comment:
Does the `SharePartition` need to be put into a fenced state? Removing it
from the cache is good for future requests, but is that enough for the object
which is already in existence?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]