Skip to content

RATIS-2509. Introduce local read API to reduce serde cost#1448

Draft
ivandika3 wants to merge 1 commit intoapache:masterfrom
ivandika3:RATIS-2509
Draft

RATIS-2509. Introduce local read API to reduce serde cost#1448
ivandika3 wants to merge 1 commit intoapache:masterfrom
ivandika3:RATIS-2509

Conversation

@ivandika3
Copy link
Copy Markdown
Contributor

@ivandika3 ivandika3 commented May 5, 2026

What changes were proposed in this pull request?

See https://issues.apache.org/jira/browse/RATIS-2509

OM code integration

    public OMResponse submitReadRequest(OMRequest omRequest, Function<OMRequest, OMResponse> handler)
      throws ServiceException {
    try {
      return serverDivision.readOnlyAsync(() -> CompletableFuture.completedFuture(handler.apply(omRequest))).get();
    } catch (ExecutionException | IOException ex) {
      throw new ServiceException(ex.getMessage(), ex);
    } catch (InterruptedException ex) {
      Thread.currentThread().interrupt();
      throw new ServiceException(ex.getMessage(), ex);
    }

  }

What is the link to the Apache JIRA

https://issues.apache.org/jira/browse/RATIS-2509

How was this patch tested?

OM Leader read performance testing, the Ratis read performance is on par with the direct read.

  • Before patch: 235020 QPS -> 180433 QPS (24% QPS reduction)
  • After patch: 235020 QPS -> 228362 (3% QPS reduction)

(Putting this to draft since my GH account is currently blocked from GH actions).

Copy link
Copy Markdown
Contributor

@szetszwo szetszwo left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@ivandika3 , thanks for working on this! Please see the comments inlined and also https://issues.apache.org/jira/secure/attachment/13082106/1448_review.patch

* {@link org.apache.ratis.client.api.AsyncApi#sendReadOnly(org.apache.ratis.protocol.Message)}.
*/
default <T> CompletableFuture<T> readOnlyAsync(
CheckedSupplier<CompletableFuture<T>, IOException> query) throws IOException {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  • Since the query method in StateMachine does not throw IOException, let's just use Supplier here. It will force the StateMachine to handle IOException, if there is any.
//StateMachine
  CompletableFuture<Message> query(Message request);
  • Also, we need clientId and readRequestType to support read-after-write-consistency.
    default <T> CompletableFuture<T> readOnlyAsync(ClientId clientId, ReadRequestTypeProto readRequestType,
        Supplier<CompletableFuture<T>> query) throws IOException {

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good point, updated.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants