apoorvmittal10 commented on code in PR #18053:
URL: https://github.com/apache/kafka/pull/18053#discussion_r1871678629
##########
core/src/main/java/kafka/server/share/SharePartition.java:
##########
@@ -371,30 +372,52 @@ public static RecordState forId(byte id) {
public CompletableFuture<Void> maybeInitialize() {
log.debug("Maybe initialize share partition: {}-{}", groupId,
topicIdPartition);
CompletableFuture<Void> future = new CompletableFuture<>();
+ AtomicReference<Optional<Throwable>> futureException = new
AtomicReference<>(Optional.empty());
// Check if the share partition is already initialized.
- maybeCompleteInitialization(future);
- if (future.isDone()) {
+ InitializationResult initializationResult =
maybeCompleteInitialization();
+ if (initializationResult.isComplete()) {
+ if (initializationResult.throwable() != null) {
+ future.completeExceptionally(initializationResult.throwable());
+ } else {
+ future.complete(null);
+ }
return future;
}
// All the pending requests should wait to get completed before the
share partition is initialized.
// Attain lock to avoid any concurrent requests to be processed.
lock.writeLock().lock();
+ boolean shouldFutureBeCompleted = false;
try {
// Re-check the state to verify if previous requests has already
initialized the share partition.
- maybeCompleteInitialization(future);
- if (future.isDone()) {
+ initializationResult = maybeCompleteInitialization();
+ if (initializationResult.isComplete()) {
+ if (initializationResult.throwable() != null) {
+
futureException.set(Optional.of(initializationResult.throwable()));
+ } else {
+ futureException.set(Optional.empty());
+ }
+ shouldFutureBeCompleted = true;
return future;
}
// Update state to initializing to avoid any concurrent requests
to be processed.
partitionState = SharePartitionState.INITIALIZING;
} catch (Exception e) {
log.error("Failed to initialize the share partition: {}-{}",
groupId, topicIdPartition, e);
- completeInitializationWithException(future, e);
+ completeInitializationWithException();
+ futureException.set(Optional.of(e));
+ shouldFutureBeCompleted = true;
return future;
} finally {
lock.writeLock().unlock();
+ if (shouldFutureBeCompleted) {
+ if (futureException.get().isPresent()) {
+ future.completeExceptionally(futureException.get().get());
+ } else {
+ future.complete(null);
+ }
+ }
Review Comment:
Instead of all these changes why can't the code be simpler i.e.:
Am I missing something here?
```
Throwable throwable = null;
try {
if (exception != null) {
log.error("Failed to initialize the share partition:
{}-{}", groupId, topicIdPartition, exception);
throwable = exception;
return;
}
if (result == null || result.topicsData() == null ||
result.topicsData().size() != 1) {
log.error("Failed to initialize the share partition:
{}-{}. Invalid state found: {}.",
groupId, topicIdPartition, result);
throwable = new
IllegalStateException(String.format("Failed to initialize the share partition
%s-%s", groupId, topicIdPartition));
return;
}
...
...
...
} finally {
lock.writeLock().unlock();
if (throwable != null) {
completeInitializationWithException(future, throwable);
} else {
future.complete(null);
}
}
--
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]