chia7712 commented on code in PR #18093:
URL: https://github.com/apache/kafka/pull/18093#discussion_r1874548857
##########
core/src/main/java/kafka/server/share/SharePartition.java:
##########
@@ -371,52 +370,35 @@ 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.
- InitializationResult initializationResult =
checkInitializationCompletion();
- if (initializationResult.isComplete()) {
- if (initializationResult.throwable() != null) {
- future.completeExceptionally(initializationResult.throwable());
- } else {
- future.complete(null);
+ try {
+ if (initializedOrThrowException()) {
+ return CompletableFuture.completedFuture(null);
}
- return future;
+ } catch (Exception e) {
+ return CompletableFuture.failedFuture(e);
}
+ // If code reaches here then the share partition is not initialized.
Initialize the share partition.
// 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.
- initializationResult = checkInitializationCompletion();
- if (initializationResult.isComplete()) {
- if (initializationResult.throwable() != null) {
-
futureException.set(Optional.of(initializationResult.throwable()));
- }
- shouldFutureBeCompleted = true;
- return future;
+ if (initializedOrThrowException()) {
+ return CompletableFuture.completedFuture(null);
Review Comment:
If a thread holds the write lock and the returned `CompletableFuture` is
registered to complete delayed operations, a deadlock could occur again.
However, this may not currently be an issue since we only call
`completeDelayedShareFetchRequest` when the future is done.
```java
CompletableFuture<Void> initializationFuture =
sharePartition.maybeInitialize();
final boolean initialized = initializationFuture.isDone();
initializationFuture.whenComplete((result, throwable) -> {
if (!initialized)
replicaManager.completeDelayedShareFetchRequest(delayedShareFetchKey);
});
```
However, I prefer not to hold the write lock while completing the future.
Perhaps we can add a new helper method to change the state from empty to
initializing.
```java
private boolean emptyToInitialState() {
lock.writeLock().lock();
try {
if (initializedOrThrowException()) return false;
partitionState = SharePartitionState.INITIALIZING;
return true;
} finally {
lock.writeLock().unlock();
}
}
```
Additionally, the second check can be simplified using the following style:
```java
// If code reaches here then the share partition is not initialized.
Initialize the share partition.
// 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.
try {
if (!emptyToInitialState()) return
CompletableFuture.completedFuture(null);
} catch (Exception e) {
return CompletableFuture.failedFuture(e);
}
```
##########
core/src/main/java/kafka/server/share/SharePartition.java:
##########
@@ -426,55 +408,50 @@ public CompletableFuture<Void> maybeInitialize() {
.build())
.build()
).whenComplete((result, exception) -> {
+ Throwable throwable = null;
lock.writeLock().lock();
try {
if (exception != null) {
log.error("Failed to initialize the share partition:
{}-{}", groupId, topicIdPartition, exception);
- completeInitializationWithException();
- futureException.set(Optional.of(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);
- completeInitializationWithException();
- futureException.set(Optional.of(new
IllegalStateException(String.format("Failed to initialize the share partition
%s-%s", groupId, topicIdPartition))));
+ throwable = new
IllegalStateException(String.format("Failed to initialize the share partition
%s-%s", groupId, topicIdPartition));
return;
}
TopicData<PartitionAllData> state = result.topicsData().get(0);
if (state.topicId() != topicIdPartition.topicId() ||
state.partitions().size() != 1) {
log.error("Failed to initialize the share partition:
{}-{}. Invalid topic partition response: {}.",
groupId, topicIdPartition, result);
- completeInitializationWithException();
- futureException.set(Optional.of(new
IllegalStateException(String.format("Failed to initialize the share partition
%s-%s", groupId, topicIdPartition))));
+ throwable = new
IllegalStateException(String.format("Failed to initialize the share partition
%s-%s", groupId, topicIdPartition));
return;
}
PartitionAllData partitionData = state.partitions().get(0);
if (partitionData.partition() != topicIdPartition.partition())
{
log.error("Failed to initialize the share partition:
{}-{}. Invalid partition response: {}.",
groupId, topicIdPartition, partitionData);
- completeInitializationWithException();
- futureException.set(Optional.of(new
IllegalStateException(String.format("Failed to initialize the share partition
%s-%s", groupId, topicIdPartition))));
+ throwable = new
IllegalStateException(String.format("Failed to initialize the share partition
%s-%s", groupId, topicIdPartition));
return;
}
if (partitionData.errorCode() != Errors.NONE.code()) {
KafkaException ex =
fetchPersisterError(partitionData.errorCode(), partitionData.errorMessage());
log.error("Failed to initialize the share partition:
{}-{}. Exception occurred: {}.",
groupId, topicIdPartition, partitionData);
- completeInitializationWithException();
- futureException.set(Optional.of(ex));
+ throwable = ex;
return;
}
try {
startOffset =
startOffsetDuringInitialization(partitionData.startOffset());
} catch (Exception e) {
Review Comment:
Could we remove this nested try-cache by moving out of this "catch"? for
example:
```java
try {
...
} catch (Exception e) {
throwable = e;
} finally {
boolean isFailed = throwable != 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]