Skip to content

Commit

Permalink
KAFKA-18827: Initialize share group state persister impl [2/N]. (#18992)
Browse files Browse the repository at this point in the history
* In this PR, we have provided implementation for the initialize share
group state RPC from the persister perspective.
* Tests have been added wherever applicable.

Reviewers: Andrew Schofield <[email protected]>
  • Loading branch information
smjn authored Feb 23, 2025
1 parent a1372ce commit 6e76736
Show file tree
Hide file tree
Showing 4 changed files with 1,379 additions and 49 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.requests.DeleteShareGroupStateResponse;
import org.apache.kafka.common.requests.InitializeShareGroupStateResponse;
import org.apache.kafka.common.requests.ReadShareGroupStateResponse;
import org.apache.kafka.common.requests.ReadShareGroupStateSummaryResponse;
import org.apache.kafka.common.requests.WriteShareGroupStateResponse;
Expand All @@ -33,7 +34,6 @@
import java.util.List;
import java.util.Map;
import java.util.concurrent.CompletableFuture;
import java.util.stream.Collectors;

/**
* The default implementation of the {@link Persister} interface which is used by the
Expand Down Expand Up @@ -70,7 +70,48 @@ public void stop() {
* @return A completable future of InitializeShareGroupStateResult
*/
public CompletableFuture<InitializeShareGroupStateResult> initializeState(InitializeShareGroupStateParameters request) {
throw new RuntimeException("not implemented");
try {
validate(request);
} catch (Exception e) {
log.error("Unable to validate initialize state request", e);
return CompletableFuture.failedFuture(e);
}
GroupTopicPartitionData<PartitionStateData> gtp = request.groupTopicPartitionData();
String groupId = gtp.groupId();

Map<Uuid, Map<Integer, CompletableFuture<InitializeShareGroupStateResponse>>> futureMap = new HashMap<>();
List<PersisterStateManager.InitializeStateHandler> handlers = new ArrayList<>();

gtp.topicsData().forEach(topicData -> {
topicData.partitions().forEach(partitionData -> {
CompletableFuture<InitializeShareGroupStateResponse> future = futureMap
.computeIfAbsent(topicData.topicId(), k -> new HashMap<>())
.computeIfAbsent(partitionData.partition(), k -> new CompletableFuture<>());

handlers.add(
stateManager.new InitializeStateHandler(
groupId,
topicData.topicId(),
partitionData.partition(),
partitionData.stateEpoch(),
partitionData.startOffset(),
future,
null
)
);
});
});

for (PersisterStateManager.PersisterStateManagerHandler handler : handlers) {
stateManager.enqueue(handler);
}

CompletableFuture<Void> combinedFuture = CompletableFuture.allOf(
handlers.stream()
.map(PersisterStateManager.InitializeStateHandler::result)
.toArray(CompletableFuture[]::new));

return combinedFuture.thenApply(v -> initializeResponsesToResult(futureMap));
}

/**
Expand Down Expand Up @@ -125,6 +166,51 @@ stateManager.new WriteStateHandler(
return combinedFuture.thenApply(v -> writeResponsesToResult(futureMap));
}

/**
* Takes in a list of COMPLETED futures and combines the results,
* taking care of errors if any, into a single InitializeShareGroupStateResult
*
* @param futureMap - HashMap of {topic -> {partition -> future}}
* @return Object representing combined result of type InitializeShareGroupStateResult
*/
// visible for testing
InitializeShareGroupStateResult initializeResponsesToResult(
Map<Uuid, Map<Integer, CompletableFuture<InitializeShareGroupStateResponse>>> futureMap
) {
List<TopicData<PartitionErrorData>> topicsData = futureMap.keySet().stream()
.map(topicId -> {
List<PartitionErrorData> partitionErrData = futureMap.get(topicId).entrySet().stream()
.map(partitionFuture -> {
int partition = partitionFuture.getKey();
CompletableFuture<InitializeShareGroupStateResponse> future = partitionFuture.getValue();
try {
// already completed because of allOf application in the caller
InitializeShareGroupStateResponse partitionResponse = future.join();
return partitionResponse.data().results().get(0).partitions().stream()
.map(partitionResult -> PartitionFactory.newPartitionErrorData(
partitionResult.partition(),
partitionResult.errorCode(),
partitionResult.errorMessage()))
.toList();
} catch (Exception e) {
log.error("Unexpected exception while initializing data in share coordinator", e);
return List.of(PartitionFactory.newPartitionErrorData(
partition,
Errors.UNKNOWN_SERVER_ERROR.code(), // No specific public error code exists for InterruptedException / ExecutionException
"Error initializing state in share coordinator: " + e.getMessage())
);
}
})
.flatMap(List::stream)
.toList();
return new TopicData<>(topicId, partitionErrData);
})
.toList();
return new InitializeShareGroupStateResult.Builder()
.setTopicsData(topicsData)
.build();
}

/**
* Takes in a list of COMPLETED futures and combines the results,
* taking care of errors if any, into a single WriteShareGroupStateResult
Expand All @@ -150,21 +236,21 @@ WriteShareGroupStateResult writeResponsesToResult(
partitionResult.partition(),
partitionResult.errorCode(),
partitionResult.errorMessage()))
.collect(Collectors.toList());
.toList();
} catch (Exception e) {
log.error("Unexpected exception while writing data to share coordinator", e);
return Collections.singletonList(PartitionFactory.newPartitionErrorData(
return List.of(PartitionFactory.newPartitionErrorData(
partition,
Errors.UNKNOWN_SERVER_ERROR.code(), // No specific public error code exists for InterruptedException / ExecutionException
"Error writing state to share coordinator: " + e.getMessage())
);
}
})
.flatMap(List::stream)
.collect(Collectors.toList());
.toList();
return new TopicData<>(topicId, partitionErrData);
})
.collect(Collectors.toList());
.toList();
return new WriteShareGroupStateResult.Builder()
.setTopicsData(topicsData)
.build();
Expand Down Expand Up @@ -248,12 +334,12 @@ ReadShareGroupStateResult readResponsesToResult(
partitionResult.startOffset(),
partitionResult.errorCode(),
partitionResult.errorMessage(),
partitionResult.stateBatches().stream().map(PersisterStateBatch::from).collect(Collectors.toList())
partitionResult.stateBatches().stream().map(PersisterStateBatch::from).toList()
))
.collect(Collectors.toList());
.toList();
} catch (Exception e) {
log.error("Unexpected exception while getting data from share coordinator", e);
return Collections.singletonList(PartitionFactory.newPartitionAllData(
return List.of(PartitionFactory.newPartitionAllData(
partition,
-1,
-1,
Expand All @@ -264,10 +350,10 @@ ReadShareGroupStateResult readResponsesToResult(
}
})
.flatMap(List::stream)
.collect(Collectors.toList());
.toList();
return new TopicData<>(topicId, partitionAllData);
})
.collect(Collectors.toList());
.toList();
return new ReadShareGroupStateResult.Builder()
.setTopicsData(topicsData)
.build();
Expand Down Expand Up @@ -403,10 +489,10 @@ ReadShareGroupStateSummaryResult readSummaryResponsesToResult(
partitionResult.startOffset(),
partitionResult.errorCode(),
partitionResult.errorMessage()))
.collect(Collectors.toList());
.toList();
} catch (Exception e) {
log.error("Unexpected exception while getting data from share coordinator", e);
return Collections.singletonList(PartitionFactory.newPartitionStateSummaryData(
return List.of(PartitionFactory.newPartitionStateSummaryData(
partition,
-1,
-1,
Expand All @@ -415,10 +501,10 @@ ReadShareGroupStateSummaryResult readSummaryResponsesToResult(
}
})
.flatMap(List::stream)
.collect(Collectors.toList());
.toList();
return new TopicData<>(topicId, partitionStateErrorData);
})
.collect(Collectors.toList());
.toList();
return new ReadShareGroupStateSummaryResult.Builder()
.setTopicsData(topicsData)
.build();
Expand Down Expand Up @@ -464,15 +550,27 @@ DeleteShareGroupStateResult deleteResponsesToResult(
}
})
.flatMap(List::stream)
.collect(Collectors.toList());
.toList();
return new TopicData<>(topicId, partitionErrorData);
})
.collect(Collectors.toList());
.toList();
return new DeleteShareGroupStateResult.Builder()
.setTopicsData(topicsData)
.build();
}

private static void validate(InitializeShareGroupStateParameters params) {
String prefix = "Initialize share group parameters";
if (params == null) {
throw new IllegalArgumentException(prefix + " cannot be null.");
}
if (params.groupTopicPartitionData() == null) {
throw new IllegalArgumentException(prefix + " data cannot be null.");
}

validateGroupTopicPartitionData(prefix, params.groupTopicPartitionData());
}

private static void validate(WriteShareGroupStateParameters params) {
String prefix = "Write share group parameters";
if (params == null) {
Expand Down
Loading

0 comments on commit 6e76736

Please sign in to comment.