-
Notifications
You must be signed in to change notification settings - Fork 5
Add retry logic for RS3 WriteWalSegments #429
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 8 commits
a328171
09d873c
b42a77d
f7e5fb5
f891a8d
b77c732
142991e
d761c7f
27cf041
e8cf9bc
2931710
023d514
2a4ef94
3e6daee
fc5ae08
ac00ece
18db82b
9d4d02a
9401741
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,15 +17,20 @@ | |
| import dev.responsive.kafka.internal.db.partitioning.TablePartitioner; | ||
| import dev.responsive.kafka.internal.db.rs3.client.LssId; | ||
| import dev.responsive.kafka.internal.db.rs3.client.RS3Client; | ||
| import dev.responsive.kafka.internal.db.rs3.client.RS3TransientException; | ||
| import dev.responsive.kafka.internal.db.rs3.client.StreamSender; | ||
| import dev.responsive.kafka.internal.db.rs3.client.StreamSenderMessageReceiver; | ||
| import dev.responsive.kafka.internal.db.rs3.client.WalEntry; | ||
| import dev.responsive.kafka.internal.stores.RemoteWriteResult; | ||
| import java.util.ArrayList; | ||
| import java.util.HashMap; | ||
| import java.util.List; | ||
| import java.util.Objects; | ||
| import java.util.Optional; | ||
| import java.util.UUID; | ||
| import java.util.concurrent.CompletableFuture; | ||
| import java.util.concurrent.CompletionStage; | ||
| import java.util.function.Consumer; | ||
| import org.apache.kafka.common.utils.Bytes; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
@@ -114,7 +119,7 @@ public RemoteWriteResult<Integer> postFlush(final long consumedOffset) { | |
| return super.postFlush(consumedOffset); | ||
| } | ||
|
|
||
| Optional<Long> writtenOffset(final int pssId) { | ||
| public Optional<Long> writtenOffset(final int pssId) { | ||
| return writtenOffsets.get(pssId); | ||
| } | ||
|
|
||
|
|
@@ -156,64 +161,131 @@ public CompletionStage<RemoteWriteResult<Integer>> flush() { | |
| } | ||
| } | ||
|
|
||
| private static class RS3KVWriter implements RemoteWriter<Bytes, Integer> { | ||
| private final StreamSender<WalEntry> streamSender; | ||
| private final CompletionStage<Optional<Long>> resultFuture; | ||
| private RS3KVTable table; | ||
| private static final class RS3StreamFactory { | ||
| private final UUID storeId; | ||
| private final RS3Client rs3Client; | ||
| private final int pssId; | ||
| private final LssId lssId; | ||
| private final long endOffset; | ||
| private final int kafkaPartition; | ||
| private final Optional<Long> expectedWrittenOffset; | ||
|
|
||
| private RS3KVWriter( | ||
| private RS3StreamFactory( | ||
| final UUID storeId, | ||
| final RS3Client rs3Client, | ||
| final RS3KVTable table, | ||
| final int pssId, | ||
| final LssId lssId, | ||
| final long endOffset, | ||
| final Optional<Long> expectedWrittenOffset, | ||
| final int kafkaPartition | ||
| final Optional<Long> expectedWrittenOffset | ||
| ) { | ||
| this.table = Objects.requireNonNull(table); | ||
| this.storeId = storeId; | ||
| this.rs3Client = rs3Client; | ||
| this.pssId = pssId; | ||
| this.lssId = lssId; | ||
| this.endOffset = endOffset; | ||
| this.kafkaPartition = kafkaPartition; | ||
| final var sendRecv = rs3Client.writeWalSegmentAsync( | ||
| this.expectedWrittenOffset = expectedWrittenOffset; | ||
| } | ||
|
|
||
| StreamSenderMessageReceiver<WalEntry, Optional<Long>> writeWalSegmentAsync() { | ||
| return rs3Client.writeWalSegmentAsync( | ||
| storeId, | ||
| lssId, | ||
| pssId, | ||
| expectedWrittenOffset, | ||
| endOffset | ||
| ); | ||
| this.streamSender = sendRecv.sender(); | ||
| this.resultFuture = sendRecv.receiver(); | ||
| } | ||
|
|
||
| Optional<Long> writeWalSegmentSync(List<WalEntry> entries) { | ||
| return rs3Client.writeWalSegment( | ||
| storeId, | ||
| lssId, | ||
| pssId, | ||
| expectedWrittenOffset, | ||
| endOffset, | ||
| entries | ||
| ); | ||
| } | ||
|
|
||
| } | ||
|
|
||
| private static final class RS3KVWriter implements RemoteWriter<Bytes, Integer> { | ||
| private final RS3StreamFactory streamFactory; | ||
| private final RS3KVTable table; | ||
| private final int kafkaPartition; | ||
| private final List<WalEntry> retryBuffer = new ArrayList<>(); | ||
| private final StreamSenderMessageReceiver<WalEntry, Optional<Long>> sendRecv; | ||
|
|
||
| private RS3KVWriter( | ||
| final UUID storeId, | ||
| final RS3Client rs3Client, | ||
| final RS3KVTable table, | ||
| final int pssId, | ||
| final LssId lssId, | ||
| final long endOffset, | ||
| final Optional<Long> expectedWrittenOffset, | ||
| final int kafkaPartition | ||
| ) { | ||
| this.table = Objects.requireNonNull(table); | ||
| this.streamFactory = new RS3StreamFactory( | ||
| storeId, | ||
| rs3Client, | ||
| pssId, | ||
| lssId, | ||
| endOffset, | ||
| expectedWrittenOffset | ||
| ); | ||
| this.kafkaPartition = kafkaPartition; | ||
| this.sendRecv = streamFactory.writeWalSegmentAsync(); | ||
| } | ||
|
|
||
| long endOffset() { | ||
| return endOffset; | ||
| return streamFactory.endOffset; | ||
| } | ||
|
|
||
| @Override | ||
| public void insert(final Bytes key, final byte[] value, final long timestampMs) { | ||
| streamSender.sendNext(table.insert(kafkaPartition, key, value, timestampMs)); | ||
| maybeSendNext(table.insert(kafkaPartition, key, value, timestampMs)); | ||
| } | ||
|
|
||
| @Override | ||
| public void delete(final Bytes key) { | ||
| streamSender.sendNext(table.delete(kafkaPartition, key)); | ||
| maybeSendNext(table.delete(kafkaPartition, key)); | ||
| } | ||
|
|
||
| private void maybeSendNext(WalEntry entry) { | ||
| retryBuffer.add(entry); | ||
| ifActiveStream(sender -> sender.sendNext(entry)); | ||
| } | ||
|
|
||
| private void ifActiveStream(Consumer<StreamSender<WalEntry>> streamConsumer) { | ||
| if (sendRecv.isActive()) { | ||
| try { | ||
| streamConsumer.accept(sendRecv.sender()); | ||
| } catch (final RS3TransientException e) { | ||
| // Retry the stream in flush() | ||
| } | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public CompletionStage<RemoteWriteResult<Integer>> flush() { | ||
| streamSender.finish(); | ||
| return resultFuture.thenApply( | ||
| flushedOffset -> { | ||
| LOG.debug("last flushed offset for pss/lss {}/{} is {}", pssId, lssId, flushedOffset); | ||
| return RemoteWriteResult.success(kafkaPartition); | ||
| } | ||
| ); | ||
| ifActiveStream(StreamSender::finish); | ||
|
|
||
| return sendRecv.receiver().handle((result, throwable) -> { | ||
|
||
| Optional<Long> flushedOffset = result; | ||
| if (throwable instanceof RS3TransientException) { | ||
| flushedOffset = streamFactory.writeWalSegmentSync(retryBuffer); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. the retries for this call happen in the grpc client right?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes, that's right. I was trying to keep retry logic encapsulated in the grpc client. But there's not really a way I could think of to encapsulate retries for the async API. |
||
| } else if (throwable instanceof RuntimeException) { | ||
| throw (RuntimeException) throwable; | ||
| } else if (throwable != null) { | ||
| throw new RuntimeException(throwable); | ||
| } | ||
|
|
||
| LOG.debug("last flushed offset for pss/lss {}/{} is {}", | ||
| streamFactory.pssId, streamFactory.lssId, flushedOffset); | ||
| return RemoteWriteResult.success(kafkaPartition); | ||
| }); | ||
| } | ||
| } | ||
|
|
||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,21 @@ | ||
| /* | ||
| * Copyright 2024 Responsive Computing, Inc. | ||
| * | ||
| * This source code is licensed under the Responsive Business Source License Agreement v1.0 | ||
| * available at: | ||
| * | ||
| * https://www.responsive.dev/legal/responsive-bsl-10 | ||
| * | ||
| * This software requires a valid Commercial License Key for production use. Trial and commercial | ||
| * licenses can be obtained at https://www.responsive.dev | ||
| */ | ||
|
|
||
| package dev.responsive.kafka.internal.db.rs3.client; | ||
|
|
||
| public class RS3TransientException extends RS3Exception { | ||
| private static final long serialVersionUID = 0L; | ||
|
|
||
| public RS3TransientException(final Throwable cause) { | ||
| super(cause); | ||
| } | ||
| } |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
we probably need to catch exceptions from
finishas wellThere was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In
ifActiveStream, we catchRS3TransientException.