Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -28,10 +28,12 @@ public abstract class AbstractLookupQuery<T> {

private final TableBucket tableBucket;
private final byte[] key;
private int retries;

public AbstractLookupQuery(TableBucket tableBucket, byte[] key) {
this.tableBucket = tableBucket;
this.key = key;
this.retries = 0;
}

public byte[] key() {
Expand All @@ -42,6 +44,14 @@ public TableBucket tableBucket() {
return tableBucket;
}

public int retries() {
return retries;
}

public void incrementRetries() {
retries++;
}

public abstract LookupType lookupType();

public abstract CompletableFuture<T> future();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -67,7 +67,8 @@ public LookupClient(Configuration conf, MetadataUpdater metadataUpdater) {
new LookupSender(
metadataUpdater,
lookupQueue,
conf.getInt(ConfigOptions.CLIENT_LOOKUP_MAX_INFLIGHT_SIZE));
conf.getInt(ConfigOptions.CLIENT_LOOKUP_MAX_INFLIGHT_SIZE),
conf.getInt(ConfigOptions.CLIENT_LOOKUP_MAX_RETRIES));
lookupSenderThreadPool.submit(lookupSender);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.apache.fluss.client.metadata.MetadataUpdater;
import org.apache.fluss.exception.FlussRuntimeException;
import org.apache.fluss.exception.LeaderNotAvailableException;
import org.apache.fluss.exception.RetriableException;
import org.apache.fluss.metadata.TableBucket;
import org.apache.fluss.rpc.gateway.TabletServerGateway;
import org.apache.fluss.rpc.messages.LookupRequest;
Expand Down Expand Up @@ -65,10 +66,17 @@ class LookupSender implements Runnable {

private final Semaphore maxInFlightReuqestsSemaphore;

LookupSender(MetadataUpdater metadataUpdater, LookupQueue lookupQueue, int maxFlightRequests) {
private final int maxRetries;

LookupSender(
MetadataUpdater metadataUpdater,
LookupQueue lookupQueue,
int maxFlightRequests,
int maxRetries) {
this.metadataUpdater = metadataUpdater;
this.lookupQueue = lookupQueue;
this.maxInFlightReuqestsSemaphore = new Semaphore(maxFlightRequests);
this.maxRetries = maxRetries;
this.running = true;
}

Expand Down Expand Up @@ -286,13 +294,8 @@ private void handleLookupResponse(
pbLookupRespForBucket.getBucketId());
LookupBatch lookupBatch = lookupsByBucket.get(tableBucket);
if (pbLookupRespForBucket.hasErrorCode()) {
// TODO for re-triable error, we should retry here instead of throwing exception.
ApiError error = ApiError.fromErrorMessage(pbLookupRespForBucket);
LOG.warn(
"Get error lookup response on table bucket {}, fail. Error: {}",
tableBucket,
error.formatErrMsg());
lookupBatch.completeExceptionally(error.exception());
handleLookupError(tableBucket, error, lookupBatch.lookups(), "");
} else {
List<byte[]> byteValues =
pbLookupRespForBucket.getValuesList().stream()
Expand Down Expand Up @@ -326,13 +329,8 @@ private void handlePrefixLookupResponse(

PrefixLookupBatch prefixLookupBatch = prefixLookupsByBucket.get(tableBucket);
if (pbRespForBucket.hasErrorCode()) {
// TODO for re-triable error, we should retry here instead of throwing exception.
ApiError error = ApiError.fromErrorMessage(pbRespForBucket);
LOG.warn(
"Get error prefix lookup response on table bucket {}, fail. Error: {}",
tableBucket,
error.formatErrMsg());
prefixLookupBatch.completeExceptionally(error.exception());
handleLookupError(tableBucket, error, prefixLookupBatch.lookups(), "prefix ");
} else {
List<List<byte[]>> result = new ArrayList<>(pbRespForBucket.getValueListsCount());
for (int i = 0; i < pbRespForBucket.getValueListsCount(); i++) {
Expand All @@ -352,22 +350,60 @@ private void handleLookupRequestException(
Throwable t, Map<TableBucket, LookupBatch> lookupsByBucket) {
ApiError error = ApiError.fromThrowable(t);
for (LookupBatch lookupBatch : lookupsByBucket.values()) {
// TODO for re-triable error, we should retry here instead of throwing exception.
LOG.warn(
"Get error lookup response on table bucket {}, fail. Error: {}",
lookupBatch.tableBucket(),
error.formatErrMsg());
lookupBatch.completeExceptionally(error.exception());
handleLookupError(lookupBatch.tableBucket(), error, lookupBatch.lookups(), "");
}
}

private void handlePrefixLookupException(
Throwable t, Map<TableBucket, PrefixLookupBatch> lookupsByBucket) {
ApiError error = ApiError.fromThrowable(t);
// TODO If error, we need to retry send the request instead of throw exception.
LOG.warn("Get error prefix lookup response. Error: {}", error.formatErrMsg());
for (PrefixLookupBatch lookupBatch : lookupsByBucket.values()) {
lookupBatch.completeExceptionally(error.exception());
handleLookupError(lookupBatch.tableBucket(), error, lookupBatch.lookups(), "prefix ");
}
}

private void reEnqueueLookup(AbstractLookupQuery<?> lookup) {
lookup.incrementRetries();
lookupQueue.appendLookup(lookup);
}

private boolean canRetry(AbstractLookupQuery<?> lookup, Exception exception) {
return lookup.retries() < maxRetries
&& !lookup.future().isDone()
&& exception instanceof RetriableException;
}

/**
* Handle lookup error with retry logic. For each lookup in the list, check if it can be
* retried. If yes, re-enqueue it; otherwise, complete it exceptionally.
*
* @param tableBucket the table bucket
* @param error the error from server response
* @param lookups the list of lookups to handle
* @param lookupType the type of lookup ("" for regular lookup, "prefix " for prefix lookup)
*/
private void handleLookupError(
TableBucket tableBucket,
ApiError error,
List<? extends AbstractLookupQuery<?>> lookups,
String lookupType) {
for (AbstractLookupQuery<?> lookup : lookups) {
if (canRetry(lookup, error.exception())) {
LOG.warn(
"Get error {}lookup response on table bucket {}, retrying ({} attempts left). Error: {}",
lookupType,
tableBucket,
maxRetries - lookup.retries(),
error.formatErrMsg());
reEnqueueLookup(lookup);
} else {
LOG.warn(
"Get error {}lookup response on table bucket {}, fail. Error: {}",
lookupType,
tableBucket,
error.formatErrMsg());
lookup.future().completeExceptionally(error.exception());
}
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import org.apache.fluss.types.RowType;

import javax.annotation.Nullable;
import javax.annotation.concurrent.NotThreadSafe;

import java.util.ArrayList;
import java.util.Collections;
Expand All @@ -46,6 +47,7 @@
* An implementation of {@link Lookuper} that lookups by prefix key. A prefix key is a prefix subset
* of the primary key.
*/
@NotThreadSafe
class PrefixKeyLookuper implements Lookuper {

private final TableInfo tableInfo;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import org.apache.fluss.types.RowType;

import javax.annotation.Nullable;
import javax.annotation.concurrent.NotThreadSafe;

import java.util.Collections;
import java.util.concurrent.CompletableFuture;
Expand All @@ -40,6 +41,7 @@
import static org.apache.fluss.utils.Preconditions.checkArgument;

/** An implementation of {@link Lookuper} that lookups by primary key. */
@NotThreadSafe
class PrimaryKeyLookuper implements Lookuper {

private final TableInfo tableInfo;
Expand Down
Loading