From 1d8330ddc487ce43746a7df9ff52840e10c90069 Mon Sep 17 00:00:00 2001 From: yunhong <337361684@qq.com> Date: Wed, 18 Dec 2024 20:46:42 +0800 Subject: [PATCH 1/2] [kv] Support PrefixLookup for primary key table (#222) This closes #65 --- .../fluss/client/lookup/AbstractLookup.java | 43 ++++ .../client/lookup/AbstractLookupBatch.java | 58 ++++++ .../alibaba/fluss/client/lookup/Lookup.java | 12 +- .../fluss/client/lookup/LookupBatch.java | 15 +- .../fluss/client/lookup/LookupClient.java | 9 + .../fluss/client/lookup/LookupQueue.java | 44 +++-- .../{table => lookup}/LookupResult.java | 22 +-- .../fluss/client/lookup/LookupSender.java | 186 +++++++++++++++--- .../fluss/client/lookup/LookupType.java | 26 +++ .../fluss/client/lookup/PrefixLookup.java | 54 +++++ .../client/lookup/PrefixLookupBatch.java | 77 ++++++++ .../client/lookup/PrefixLookupResult.java | 61 ++++++ .../fluss/client/table/FlussTable.java | 62 ++++-- .../com/alibaba/fluss/client/table/Table.java | 20 ++ .../client/table/writer/UpsertWriter.java | 23 ++- .../client/utils/ClientRpcMessageUtils.java | 19 ++ .../fluss/client/table/FlussTableITCase.java | 59 ++++++ .../alibaba/fluss/config/ConfigOptions.java | 10 +- .../fluss/metadata/TableDescriptor.java | 80 ++++++-- .../alibaba/fluss/metrics/MetricNames.java | 4 + .../alibaba/fluss/row/encode/KeyEncoder.java | 2 +- .../fluss/metadata/TableDescriptorTest.java | 21 +- .../fluss/testutils/DataTestUtils.java | 22 ++- .../fluss/utils/json/JsonSerdeTestBase.java | 7 +- .../connector/flink/catalog/FlinkCatalog.java | 16 +- .../flink/catalog/FlinkTableFactory.java | 3 +- .../flink/source/FlinkTableSource.java | 10 +- .../lookup/FlinkAsyncLookupFunction.java | 160 ++++++++++----- .../source/lookup/FlinkLookupFunction.java | 45 +++-- .../flink/source/lookup/LookupNormalizer.java | 108 +++++++--- .../connector/flink/utils/PushdownUtils.java | 4 +- .../flink/source/FlinkTableSourceITCase.java | 4 +- .../entity/PrefixLookupResultForBucket.java | 47 +++++ .../rpc/gateway/TabletServerGateway.java | 10 + .../rpc/netty/server/RequestsMetrics.java | 2 + .../alibaba/fluss/rpc/protocol/ApiKeys.java | 3 +- fluss-rpc/src/main/proto/FlussApi.proto | 28 +++ .../com/alibaba/fluss/server/kv/KvTablet.java | 9 + .../fluss/server/kv/rocksdb/RocksDBKv.java | 68 ++++++- .../group/PhysicalTableMetricGroup.java | 28 +++ .../alibaba/fluss/server/replica/Replica.java | 41 ++++ .../fluss/server/replica/ReplicaManager.java | 53 ++++- .../fluss/server/tablet/TabletService.java | 16 +- .../fluss/server/utils/RpcMessageUtils.java | 61 ++++++ .../log/remote/RemoteLogManagerTest.java | 16 +- .../server/replica/ReplicaManagerTest.java | 178 +++++++++++++++-- .../fluss/server/replica/ReplicaTestBase.java | 44 +++-- .../server/tablet/TabletServiceITCase.java | 159 ++++++++++++++- .../tablet/TestTabletServerGateway.java | 7 + .../fluss/server/testutils/KvTestUtils.java | 25 ++- .../server/testutils/RpcMessageTestUtils.java | 13 ++ website/docs/maintenance/configuration.md | 3 +- website/docs/maintenance/monitor-metrics.md | 32 ++- 53 files changed, 1836 insertions(+), 293 deletions(-) create mode 100644 fluss-client/src/main/java/com/alibaba/fluss/client/lookup/AbstractLookup.java create mode 100644 fluss-client/src/main/java/com/alibaba/fluss/client/lookup/AbstractLookupBatch.java rename fluss-client/src/main/java/com/alibaba/fluss/client/{table => lookup}/LookupResult.java (73%) create mode 100644 fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupType.java create mode 100644 fluss-client/src/main/java/com/alibaba/fluss/client/lookup/PrefixLookup.java create mode 100644 fluss-client/src/main/java/com/alibaba/fluss/client/lookup/PrefixLookupBatch.java create mode 100644 fluss-client/src/main/java/com/alibaba/fluss/client/lookup/PrefixLookupResult.java create mode 100644 fluss-rpc/src/main/java/com/alibaba/fluss/rpc/entity/PrefixLookupResultForBucket.java diff --git a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/AbstractLookup.java b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/AbstractLookup.java new file mode 100644 index 00000000..b6aaef98 --- /dev/null +++ b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/AbstractLookup.java @@ -0,0 +1,43 @@ +/* + * Copyright (c) 2024 Alibaba Group Holding Ltd. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.alibaba.fluss.client.lookup; + +import com.alibaba.fluss.annotation.Internal; +import com.alibaba.fluss.metadata.TableBucket; + +import java.util.concurrent.CompletableFuture; + +/** Abstract Class to represent a lookup operation. */ +@Internal +public abstract class AbstractLookup { + + private final byte[] key; + + public AbstractLookup(byte[] key) { + this.key = key; + } + + public byte[] key() { + return key; + } + + public abstract TableBucket tableBucket(); + + public abstract LookupType lookupType(); + + public abstract CompletableFuture future(); +} diff --git a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/AbstractLookupBatch.java b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/AbstractLookupBatch.java new file mode 100644 index 00000000..0dcd52aa --- /dev/null +++ b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/AbstractLookupBatch.java @@ -0,0 +1,58 @@ +/* + * Copyright (c) 2024 Alibaba Group Holding Ltd. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.alibaba.fluss.client.lookup; + +import com.alibaba.fluss.annotation.Internal; +import com.alibaba.fluss.metadata.TableBucket; + +import java.util.ArrayList; +import java.util.List; + +/** An abstract lookup batch. */ +@Internal +public abstract class AbstractLookupBatch { + + protected final List> lookups; + private final TableBucket tableBucket; + + public AbstractLookupBatch(TableBucket tableBucket) { + this.lookups = new ArrayList<>(); + this.tableBucket = tableBucket; + } + + /** Complete the lookup operations using given values . */ + public abstract void complete(List values); + + public void addLookup(AbstractLookup lookup) { + lookups.add(lookup); + } + + public List> lookups() { + return lookups; + } + + public TableBucket tableBucket() { + return tableBucket; + } + + /** Complete the get operations with given exception. */ + public void completeExceptionally(Exception exception) { + for (AbstractLookup lookup : lookups) { + lookup.future().completeExceptionally(exception); + } + } +} diff --git a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/Lookup.java b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/Lookup.java index b4dd1ada..f8b18c3f 100644 --- a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/Lookup.java +++ b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/Lookup.java @@ -26,26 +26,28 @@ * from, the bytes of the key, and a future for the lookup operation. */ @Internal -public class Lookup { +public class Lookup extends AbstractLookup { private final TableBucket tableBucket; - private final byte[] key; private final CompletableFuture future; Lookup(TableBucket tableBucket, byte[] key) { + super(key); this.tableBucket = tableBucket; - this.key = key; this.future = new CompletableFuture<>(); } + @Override public TableBucket tableBucket() { return tableBucket; } - public byte[] key() { - return key; + @Override + public LookupType lookupType() { + return LookupType.LOOKUP; } + @Override public CompletableFuture future() { return future; } diff --git a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupBatch.java b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupBatch.java index 32bcb13f..751c6eea 100644 --- a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupBatch.java +++ b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupBatch.java @@ -19,7 +19,6 @@ import com.alibaba.fluss.annotation.Internal; import com.alibaba.fluss.exception.FlussRuntimeException; import com.alibaba.fluss.metadata.TableBucket; -import com.alibaba.fluss.rpc.messages.PbValue; import java.util.ArrayList; import java.util.List; @@ -51,21 +50,21 @@ public TableBucket tableBucket() { } /** Complete the lookup operations using given values . */ - public void complete(List pbValues) { + public void complete(List values) { // if the size of return values of lookup operation are not equal to the number of lookups, // should complete an exception. - if (pbValues.size() != lookups.size()) { + if (values.size() != lookups.size()) { completeExceptionally( new FlussRuntimeException( String.format( "The number of return values of lookup operation is not equal to the number of " + "lookups. Return %d values, but expected %d.", - pbValues.size(), lookups.size()))); + values.size(), lookups.size()))); } else { - for (int i = 0; i < pbValues.size(); i++) { - Lookup lookup = lookups.get(i); - PbValue pbValue = pbValues.get(i); - lookup.future().complete(pbValue.hasValues() ? pbValue.getValues() : null); + for (int i = 0; i < values.size(); i++) { + AbstractLookup lookup = lookups.get(i); + // single value. + lookup.future().complete(values.get(i)); } } } diff --git a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupClient.java b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupClient.java index 836a4d98..7d53e4c7 100644 --- a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupClient.java +++ b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupClient.java @@ -29,6 +29,7 @@ import javax.annotation.concurrent.ThreadSafe; import java.time.Duration; +import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -81,6 +82,14 @@ public CompletableFuture lookup(TableBucket tableBucket, byte[] keyBytes return lookup.future(); } + public CompletableFuture> prefixLookup( + long tableId, int bucketId, byte[] keyBytes) { + // TODO index lookup support partition table. + PrefixLookup prefixLookup = new PrefixLookup(new TableBucket(tableId, bucketId), keyBytes); + lookupQueue.appendLookup(prefixLookup); + return prefixLookup.future(); + } + public void close(Duration timeout) { LOG.info("Closing lookup client and lookup sender."); diff --git a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupQueue.java b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupQueue.java index fec36a1c..c1f2a40f 100644 --- a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupQueue.java +++ b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupQueue.java @@ -29,28 +29,32 @@ /** * A queue that buffers the pending lookup operations and provides a list of {@link Lookup} when - * call method {@link #drain()}. + * call method {@link #drain(LookupType)}. */ @ThreadSafe @Internal class LookupQueue { private volatile boolean closed; - private final ArrayBlockingQueue lookupQueue; + // Buffering both the Lookup and IndexLookup. + private final ArrayBlockingQueue> lookupQueue; private final int maxBatchSize; + private final long batchTimeoutMs; LookupQueue(Configuration conf) { this.lookupQueue = new ArrayBlockingQueue<>(conf.get(ConfigOptions.CLIENT_LOOKUP_QUEUE_SIZE)); this.maxBatchSize = conf.get(ConfigOptions.CLIENT_LOOKUP_MAX_BATCH_SIZE); + this.batchTimeoutMs = conf.get(ConfigOptions.CLIENT_LOOKUP_BATCH_TIMEOUT).toMillis(); this.closed = false; } - void appendLookup(Lookup lookup) { + void appendLookup(AbstractLookup lookup) { if (closed) { throw new IllegalStateException( "Can not append lookup operation since the LookupQueue is closed."); } + try { lookupQueue.put(lookup); } catch (InterruptedException e) { @@ -63,21 +67,33 @@ boolean hasUnDrained() { } /** Drain a batch of {@link Lookup}s from the lookup queue. */ - List drain() throws Exception { - List lookups = new ArrayList<>(maxBatchSize); - Lookup firstLookup = lookupQueue.poll(300, TimeUnit.MILLISECONDS); - if (firstLookup != null) { - lookups.add(firstLookup); - lookupQueue.drainTo(lookups, maxBatchSize - 1); + List> drain() throws Exception { + long start = System.currentTimeMillis(); + List> lookupOperations = new ArrayList<>(maxBatchSize); + int count = 0; + while (true) { + if (System.currentTimeMillis() - start > batchTimeoutMs) { + break; + } + + AbstractLookup lookup = lookupQueue.poll(300, TimeUnit.MILLISECONDS); + if (lookup == null) { + break; + } + count++; + lookupOperations.add(lookup); + if (count >= maxBatchSize) { + break; + } } - return lookups; + return lookupOperations; } /** Drain all the {@link Lookup}s from the lookup queue. */ - List drainAll() { - List lookups = new ArrayList<>(lookupQueue.size()); - lookupQueue.drainTo(lookups); - return lookups; + List> drainAll() { + List> lookupOperations = new ArrayList<>(lookupQueue.size()); + lookupQueue.drainTo(lookupOperations); + return lookupOperations; } public void close() { diff --git a/fluss-client/src/main/java/com/alibaba/fluss/client/table/LookupResult.java b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupResult.java similarity index 73% rename from fluss-client/src/main/java/com/alibaba/fluss/client/table/LookupResult.java rename to fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupResult.java index 9d8bee92..d11c8d19 100644 --- a/fluss-client/src/main/java/com/alibaba/fluss/client/table/LookupResult.java +++ b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupResult.java @@ -14,15 +14,12 @@ * limitations under the License. */ -package com.alibaba.fluss.client.table; +package com.alibaba.fluss.client.lookup; import com.alibaba.fluss.annotation.PublicEvolving; +import com.alibaba.fluss.client.table.Table; import com.alibaba.fluss.row.InternalRow; -import javax.annotation.Nullable; - -import java.util.Objects; - /** * The result of {@link Table#lookup(InternalRow)}. * @@ -30,13 +27,13 @@ */ @PublicEvolving public final class LookupResult { - private final @Nullable InternalRow row; + private final InternalRow row; - public LookupResult(@Nullable InternalRow row) { + public LookupResult(InternalRow row) { this.row = row; } - public @Nullable InternalRow getRow() { + public InternalRow getRow() { return row; } @@ -48,18 +45,17 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - - LookupResult lookupResult = (LookupResult) o; - return Objects.equals(row, lookupResult.row); + LookupResult that = (LookupResult) o; + return row.equals(that.row); } @Override public int hashCode() { - return Objects.hash(row); + return row.hashCode(); } @Override public String toString() { - return "LookupResult{row=" + row + '}'; + return "LookupResult{" + "row=" + row + '}'; } } diff --git a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupSender.java b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupSender.java index 87021397..bd142f14 100644 --- a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupSender.java +++ b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupSender.java @@ -24,8 +24,12 @@ import com.alibaba.fluss.rpc.messages.LookupRequest; import com.alibaba.fluss.rpc.messages.LookupResponse; import com.alibaba.fluss.rpc.messages.PbLookupRespForBucket; -import com.alibaba.fluss.rpc.messages.PbValue; +import com.alibaba.fluss.rpc.messages.PbPrefixLookupRespForBucket; +import com.alibaba.fluss.rpc.messages.PbValueList; +import com.alibaba.fluss.rpc.messages.PrefixLookupRequest; +import com.alibaba.fluss.rpc.messages.PrefixLookupResponse; import com.alibaba.fluss.rpc.protocol.ApiError; +import com.alibaba.fluss.utils.types.Tuple2; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -35,8 +39,10 @@ import java.util.List; import java.util.Map; import java.util.concurrent.Semaphore; +import java.util.stream.Collectors; import static com.alibaba.fluss.client.utils.ClientRpcMessageUtils.makeLookupRequest; +import static com.alibaba.fluss.client.utils.ClientRpcMessageUtils.makePrefixLookupRequest; /** * This background thread pool lookup operations from {@link #lookupQueue}, and send lookup requests @@ -96,57 +102,76 @@ public void run() { /** Run a single iteration of sending. */ private void runOnce(boolean drainAll) throws Exception { - List lookups = drainAll ? lookupQueue.drainAll() : lookupQueue.drain(); + List> lookups = drainAll ? lookupQueue.drainAll() : lookupQueue.drain(); sendLookups(lookups); } - private void sendLookups(List lookups) { + private void sendLookups(List> lookups) { if (lookups.isEmpty()) { return; } // group by to lookup batches - Map> lookupBatches = groupByLeader(lookups); + Map, List>> lookupBatches = + groupByLeaderAndType(lookups); // now, send the batches - lookupBatches.forEach(this::sendLookups); + lookupBatches.forEach( + (destinationAndType, batch) -> + sendLookups(destinationAndType.f0, destinationAndType.f1, batch)); } - private Map> groupByLeader(List lookups) { - // leader -> lookup batches - Map> lookupBatchesByLeader = new HashMap<>(); - for (Lookup lookup : lookups) { - // get the leader node - TableBucket tb = lookup.tableBucket(); - + private Map, List>> groupByLeaderAndType( + List> lookups) { + // -> lookup batches + Map, List>> lookupBatchesByLeader = + new HashMap<>(); + for (AbstractLookup abstractLookup : lookups) { int leader; + // lookup the leader node + TableBucket tb = abstractLookup.tableBucket(); try { - // TODO this can be a re-triable operation. We should retry here instead of throwing - // exception. + // TODO this can be a re-triable operation. We should retry here instead of + // throwing exception. leader = metadataUpdater.leaderFor(tb); } catch (Exception e) { - lookup.future().completeExceptionally(e); + abstractLookup.future().completeExceptionally(e); continue; } - - lookupBatchesByLeader.computeIfAbsent(leader, k -> new ArrayList<>()).add(lookup); + lookupBatchesByLeader + .computeIfAbsent( + Tuple2.of(leader, abstractLookup.lookupType()), k -> new ArrayList<>()) + .add(abstractLookup); } return lookupBatchesByLeader; } - private void sendLookups(int destination, List lookupBatches) { + private void sendLookups( + int destination, LookupType lookupType, List> lookupBatches) { TabletServerGateway gateway = metadataUpdater.newTabletServerClientForNode(destination); + if (lookupType == LookupType.LOOKUP) { + sendLookupRequest(gateway, lookupBatches); + } else if (lookupType == LookupType.PREFIX_LOOKUP) { + sendPrefixLookupRequest(gateway, lookupBatches); + } else { + throw new IllegalArgumentException("Unsupported lookup type: " + lookupType); + } + } + + private void sendLookupRequest( + TabletServerGateway gateway, List> lookupBatches) { // table id -> (bucket -> lookups) - Map> lookupsByTableId = new HashMap<>(); - for (Lookup lookup : lookupBatches) { + Map> lookupByTableId = new HashMap<>(); + for (AbstractLookup abstractLookup : lookupBatches) { + Lookup lookup = (Lookup) abstractLookup; TableBucket tb = lookup.tableBucket(); long tableId = tb.getTableId(); - lookupsByTableId + lookupByTableId .computeIfAbsent(tableId, k -> new HashMap<>()) .computeIfAbsent(tb, k -> new LookupBatch(tb)) .addLookup(lookup); } - lookupsByTableId.forEach( + lookupByTableId.forEach( (tableId, lookupsByBucket) -> sendLookupRequestAndHandleResponse( gateway, @@ -155,6 +180,29 @@ private void sendLookups(int destination, List lookupBatches) { lookupsByBucket)); } + private void sendPrefixLookupRequest( + TabletServerGateway gateway, List> indexLookupBatches) { + // table id -> (bucket -> lookups) + Map> lookupByTableId = new HashMap<>(); + for (AbstractLookup abstractLookup : indexLookupBatches) { + PrefixLookup prefixLookup = (PrefixLookup) abstractLookup; + TableBucket tb = prefixLookup.tableBucket(); + long tableId = tb.getTableId(); + lookupByTableId + .computeIfAbsent(tableId, k -> new HashMap<>()) + .computeIfAbsent(tb, k -> new PrefixLookupBatch(tb)) + .addLookup(prefixLookup); + } + + lookupByTableId.forEach( + (tableId, indexLookupBatch) -> + sendPrefixLookupRequestAndHandleResponse( + gateway, + makePrefixLookupRequest(tableId, indexLookupBatch.values()), + tableId, + indexLookupBatch)); + } + private void sendLookupRequestAndHandleResponse( TabletServerGateway gateway, LookupRequest lookupRequest, @@ -186,6 +234,38 @@ private void sendLookupRequestAndHandleResponse( }); } + private void sendPrefixLookupRequestAndHandleResponse( + TabletServerGateway gateway, + PrefixLookupRequest prefixLookupRequest, + long tableId, + Map lookupsByBucket) { + try { + maxInFlightReuqestsSemaphore.acquire(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new FlussRuntimeException("interrupted:", e); + } + gateway.prefixLookup(prefixLookupRequest) + .thenAccept( + prefixLookupResponse -> { + try { + handlePrefixLookupResponse( + tableId, prefixLookupResponse, lookupsByBucket); + } finally { + maxInFlightReuqestsSemaphore.release(); + } + }) + .exceptionally( + e -> { + try { + handlePrefixLookupException(e, lookupsByBucket); + return null; + } finally { + maxInFlightReuqestsSemaphore.release(); + } + }); + } + private void handleLookupResponse( long tableId, LookupResponse lookupResponse, @@ -208,8 +288,56 @@ private void handleLookupResponse( error.formatErrMsg()); lookupBatch.completeExceptionally(error.exception()); } else { - List pbValues = pbLookupRespForBucket.getValuesList(); - lookupBatch.complete(pbValues); + List byteValues = + pbLookupRespForBucket.getValuesList().stream() + .map( + pbValue -> { + if (pbValue.hasValues()) { + return pbValue.getValues(); + } else { + return null; + } + }) + .collect(Collectors.toList()); + lookupBatch.complete(byteValues); + } + } + } + + private void handlePrefixLookupResponse( + long tableId, + PrefixLookupResponse prefixLookupResponse, + Map indexLookupsByBucket) { + for (PbPrefixLookupRespForBucket pbRespForBucket : + prefixLookupResponse.getBucketsRespsList()) { + TableBucket tableBucket = + new TableBucket( + tableId, + pbRespForBucket.hasPartitionId() + ? pbRespForBucket.getPartitionId() + : null, + pbRespForBucket.getBucketId()); + + PrefixLookupBatch prefixLookupBatch = indexLookupsByBucket.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()); + } else { + List> result = new ArrayList<>(); + for (int i = 0; i < pbRespForBucket.getValueListsCount(); i++) { + PbValueList pbValueList = pbRespForBucket.getValueListAt(i); + List keyResult = new ArrayList<>(); + for (int j = 0; j < pbValueList.getValuesCount(); j++) { + keyResult.add(pbValueList.getValueAt(j)); + } + result.add(keyResult); + } + prefixLookupBatch.complete(result); } } } @@ -227,6 +355,16 @@ private void handleLookupRequestException( } } + private void handlePrefixLookupException( + Throwable t, Map 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()); + } + } + void forceClose() { forceClose = true; initiateClose(); diff --git a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupType.java b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupType.java new file mode 100644 index 00000000..5f461dbe --- /dev/null +++ b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupType.java @@ -0,0 +1,26 @@ +/* + * Copyright (c) 2024 Alibaba Group Holding Ltd. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.alibaba.fluss.client.lookup; + +import com.alibaba.fluss.annotation.Internal; + +/** Enum to represent the type of lookup operation. */ +@Internal +public enum LookupType { + LOOKUP, + PREFIX_LOOKUP; +} diff --git a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/PrefixLookup.java b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/PrefixLookup.java new file mode 100644 index 00000000..289623ee --- /dev/null +++ b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/PrefixLookup.java @@ -0,0 +1,54 @@ +/* + * Copyright (c) 2024 Alibaba Group Holding Ltd. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.alibaba.fluss.client.lookup; + +import com.alibaba.fluss.annotation.Internal; +import com.alibaba.fluss.metadata.TableBucket; + +import java.util.List; +import java.util.concurrent.CompletableFuture; + +/** + * Class to represent a prefix lookup operation, it contains the table id, bucketNums and related + * CompletableFuture. + */ +@Internal +public class PrefixLookup extends AbstractLookup> { + private final TableBucket tableBucket; + private final CompletableFuture> future; + + public PrefixLookup(TableBucket tableBucket, byte[] prefixKey) { + super(prefixKey); + this.tableBucket = tableBucket; + this.future = new CompletableFuture<>(); + } + + @Override + public TableBucket tableBucket() { + return tableBucket; + } + + @Override + public CompletableFuture> future() { + return future; + } + + @Override + public LookupType lookupType() { + return LookupType.PREFIX_LOOKUP; + } +} diff --git a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/PrefixLookupBatch.java b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/PrefixLookupBatch.java new file mode 100644 index 00000000..20188ff5 --- /dev/null +++ b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/PrefixLookupBatch.java @@ -0,0 +1,77 @@ +/* + * Copyright (c) 2024 Alibaba Group Holding Ltd. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.alibaba.fluss.client.lookup; + +import com.alibaba.fluss.annotation.Internal; +import com.alibaba.fluss.exception.FlussRuntimeException; +import com.alibaba.fluss.metadata.TableBucket; + +import java.util.ArrayList; +import java.util.List; + +/** + * A batch that contains the prefix lookup operations that send to same destination and some table + * together. + */ +@Internal +public class PrefixLookupBatch { + + /** The table bucket that the lookup operations should fall into. */ + private final TableBucket tableBucket; + + private final List prefixLookups; + + public PrefixLookupBatch(TableBucket tableBucket) { + this.tableBucket = tableBucket; + this.prefixLookups = new ArrayList<>(); + } + + public void addLookup(PrefixLookup lookup) { + prefixLookups.add(lookup); + } + + public List lookups() { + return prefixLookups; + } + + public TableBucket tableBucket() { + return tableBucket; + } + + public void complete(List> values) { + if (values.size() != prefixLookups.size()) { + completeExceptionally( + new FlussRuntimeException( + String.format( + "The number of values return by prefix lookup request is not equal to the number of " + + "index lookups send. Got %d values, but expected %d.", + values.size(), prefixLookups.size()))); + } else { + for (int i = 0; i < values.size(); i++) { + AbstractLookup> lookup = prefixLookups.get(i); + lookup.future().complete(values.get(i)); + } + } + } + + /** Complete the get operations with given exception. */ + public void completeExceptionally(Exception exception) { + for (PrefixLookup prefixLookup : prefixLookups) { + prefixLookup.future().completeExceptionally(exception); + } + } +} diff --git a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/PrefixLookupResult.java b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/PrefixLookupResult.java new file mode 100644 index 00000000..a9ff3044 --- /dev/null +++ b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/PrefixLookupResult.java @@ -0,0 +1,61 @@ +/* + * Copyright (c) 2024 Alibaba Group Holding Ltd. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.alibaba.fluss.client.lookup; + +import com.alibaba.fluss.client.table.Table; +import com.alibaba.fluss.row.InternalRow; + +import java.util.List; + +/** + * The result of {@link Table#prefixLookup(InternalRow)}}. + * + * @since 0.1 + */ +public class PrefixLookupResult { + private final List rowList; + + public PrefixLookupResult(List rowList) { + this.rowList = rowList; + } + + public List getRowList() { + return rowList; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PrefixLookupResult that = (PrefixLookupResult) o; + return rowList.equals(that.rowList); + } + + @Override + public int hashCode() { + return rowList.hashCode(); + } + + @Override + public String toString() { + return "PrefixLookupResult{" + "rowList=" + rowList + '}'; + } +} diff --git a/fluss-client/src/main/java/com/alibaba/fluss/client/table/FlussTable.java b/fluss-client/src/main/java/com/alibaba/fluss/client/table/FlussTable.java index 16dc10e0..385e20e1 100644 --- a/fluss-client/src/main/java/com/alibaba/fluss/client/table/FlussTable.java +++ b/fluss-client/src/main/java/com/alibaba/fluss/client/table/FlussTable.java @@ -19,6 +19,8 @@ import com.alibaba.fluss.annotation.PublicEvolving; import com.alibaba.fluss.client.lakehouse.LakeTableBucketAssigner; import com.alibaba.fluss.client.lookup.LookupClient; +import com.alibaba.fluss.client.lookup.LookupResult; +import com.alibaba.fluss.client.lookup.PrefixLookupResult; import com.alibaba.fluss.client.metadata.MetadataUpdater; import com.alibaba.fluss.client.scanner.RemoteFileDownloader; import com.alibaba.fluss.client.scanner.ScanRecord; @@ -104,14 +106,15 @@ public class FlussTable implements Table { private final boolean hasPrimaryKey; private final int numBuckets; private final RowType keyRowType; - // encode the key bytes for kv lookups - private final KeyEncoder keyEncoder; // decode the lookup bytes to result row private final ValueDecoder kvValueDecoder; // a getter to extract partition from key row, null when it's not a partitioned primary key // table private final @Nullable PartitionGetter keyRowPartitionGetter; + private final KeyEncoder bucketKeyEncoder; + private final KeyEncoder primaryKeyEncoder; + private final Supplier writerSupplier; private final Supplier lookupClientSupplier; private final AtomicBoolean closed; @@ -145,12 +148,10 @@ public FlussTable( this.tableId = tableInfo.getTableId(); TableDescriptor tableDescriptor = tableInfo.getTableDescriptor(); Schema schema = tableDescriptor.getSchema(); + RowType rowType = schema.toRowType(); this.hasPrimaryKey = tableDescriptor.hasPrimaryKey(); this.numBuckets = metadataUpdater.getBucketCount(tablePath); - this.keyRowType = getKeyRowType(schema); - this.keyEncoder = - KeyEncoder.createKeyEncoder( - keyRowType, keyRowType.getFieldNames(), tableDescriptor.getPartitionKeys()); + this.keyRowType = getKeyRowType(schema, schema.getPrimaryKeyIndexes()); this.keyRowPartitionGetter = tableDescriptor.isPartitioned() && tableDescriptor.hasPrimaryKey() ? new PartitionGetter(keyRowType, tableDescriptor.getPartitionKeys()) @@ -160,7 +161,17 @@ public FlussTable( new ValueDecoder( RowDecoder.create( tableDescriptor.getKvFormat(), - schema.toRowType().getChildren().toArray(new DataType[0]))); + rowType.getChildren().toArray(new DataType[0]))); + + this.primaryKeyEncoder = + KeyEncoder.createKeyEncoder( + keyRowType, keyRowType.getFieldNames(), tableDescriptor.getPartitionKeys()); + int[] bucketKeyIndexes = tableDescriptor.getBucketKeyIndexes(); + if (bucketKeyIndexes.length != 0) { + this.bucketKeyEncoder = new KeyEncoder(getKeyRowType(schema, bucketKeyIndexes)); + } else { + this.bucketKeyEncoder = primaryKeyEncoder; + } } @Override @@ -176,9 +187,10 @@ public CompletableFuture lookup(InternalRow key) { } // encoding the key row using a compacted way consisted with how the key is encoded when put // a row - byte[] keyBytes = keyEncoder.encode(key); + byte[] keyBytes = primaryKeyEncoder.encode(key); + byte[] lookupBucketKeyBytes = bucketKeyEncoder.encode(key); Long partitionId = keyRowPartitionGetter == null ? null : getPartitionId(key); - int bucketId = getBucketId(keyBytes, key); + int bucketId = getBucketId(lookupBucketKeyBytes, key); TableBucket tableBucket = new TableBucket(tableId, partitionId, bucketId); return lookupClientSupplier .get() @@ -193,6 +205,31 @@ public CompletableFuture lookup(InternalRow key) { }); } + @Override + public CompletableFuture prefixLookup(InternalRow prefixKey) { + if (!hasPrimaryKey) { + throw new FlussRuntimeException( + String.format("None-pk table %s don't support prefix lookup", tablePath)); + } + + byte[] prefixKeyBytes = bucketKeyEncoder.encode(prefixKey); + int bucketId = getBucketId(prefixKeyBytes, prefixKey); + return lookupClientSupplier + .get() + .prefixLookup(tableId, bucketId, prefixKeyBytes) + .thenApply( + result -> { + List rowList = new ArrayList<>(); + for (byte[] valueBytes : result) { + rowList.add( + valueBytes == null + ? null + : kvValueDecoder.decodeValue(valueBytes).row); + } + return new PrefixLookupResult(rowList); + }); + } + private int getBucketId(byte[] keyBytes, InternalRow key) { if (!tableInfo.getTableDescriptor().isDataLakeEnabled()) { return HashBucketAssigner.bucketForRowKey(keyBytes, numBuckets); @@ -336,11 +373,10 @@ private Long getPartitionId(InternalRow row) { return metadataUpdater.getCluster().getPartitionIdOrElseThrow(physicalTablePath); } - private RowType getKeyRowType(Schema schema) { - int[] pkIndex = schema.getPrimaryKeyIndexes(); - List keyRowFields = new ArrayList<>(pkIndex.length); + private RowType getKeyRowType(Schema schema, int[] keyIndexes) { + List keyRowFields = new ArrayList<>(keyIndexes.length); List rowFields = schema.toRowType().getFields(); - for (int index : pkIndex) { + for (int index : keyIndexes) { keyRowFields.add(rowFields.get(index)); } return new RowType(keyRowFields); diff --git a/fluss-client/src/main/java/com/alibaba/fluss/client/table/Table.java b/fluss-client/src/main/java/com/alibaba/fluss/client/table/Table.java index b2373f7b..0b4d6547 100644 --- a/fluss-client/src/main/java/com/alibaba/fluss/client/table/Table.java +++ b/fluss-client/src/main/java/com/alibaba/fluss/client/table/Table.java @@ -18,6 +18,8 @@ import com.alibaba.fluss.annotation.PublicEvolving; import com.alibaba.fluss.client.Connection; +import com.alibaba.fluss.client.lookup.LookupResult; +import com.alibaba.fluss.client.lookup.PrefixLookupResult; import com.alibaba.fluss.client.scanner.ScanRecord; import com.alibaba.fluss.client.scanner.log.LogScan; import com.alibaba.fluss.client.scanner.log.LogScanner; @@ -63,6 +65,24 @@ public interface Table extends AutoCloseable { */ CompletableFuture lookup(InternalRow key); + /** + * Prefix lookup certain rows from the given table by prefix key. + * + *

Only available for Primary Key Table. Will throw exception when the table isn't a Primary + * Key Table. + * + *

Note: Currently, if you want to use prefix lookup, the table you created must both define + * the primary key and the bucket key, in addition, the bucket key needs to be part of the + * primary key and must be a prefix of the primary key. For example, if a table has fields + * [a,b,c,d], and the primary key is set to [a, b, c], with the bucket key set to [a, b], then + * the prefix schema would also be [a, b]. This pattern can use PrefixLookup to lookup by prefix + * scan. + * + * @param prefixKey the given prefix key to do prefix lookup. + * @return the result of prefix lookup. + */ + CompletableFuture prefixLookup(InternalRow prefixKey); + /** * Extracts limit number of rows from the given table bucket. * diff --git a/fluss-client/src/main/java/com/alibaba/fluss/client/table/writer/UpsertWriter.java b/fluss-client/src/main/java/com/alibaba/fluss/client/table/writer/UpsertWriter.java index 01bcdfe7..5f8c5aef 100644 --- a/fluss-client/src/main/java/com/alibaba/fluss/client/table/writer/UpsertWriter.java +++ b/fluss-client/src/main/java/com/alibaba/fluss/client/table/writer/UpsertWriter.java @@ -42,6 +42,7 @@ public class UpsertWriter extends TableWriter { private final KeyEncoder keyEncoder; + private final KeyEncoder bucketKeyEncoder; private final @Nullable int[] targetColumns; public UpsertWriter( @@ -54,13 +55,21 @@ public UpsertWriter( Schema schema = tableDescriptor.getSchema(); sanityCheck(schema, upsertWrite.getPartialUpdateColumns()); + RowType rowType = schema.toRowType(); this.targetColumns = upsertWrite.getPartialUpdateColumns(); this.keyEncoder = KeyEncoder.createKeyEncoder( - schema.toRowType(), + rowType, schema.getPrimaryKey().get().getColumnNames(), tableDescriptor.getPartitionKeys()); + + int[] bucketKeyIndexes = tableDescriptor.getBucketKeyIndexes(); + if (bucketKeyIndexes.length != 0) { + this.bucketKeyEncoder = new KeyEncoder(rowType, bucketKeyIndexes); + } else { + this.bucketKeyEncoder = keyEncoder; + } } private static void sanityCheck(Schema schema, @Nullable int[] targetColumns) { @@ -111,8 +120,10 @@ private static void sanityCheck(Schema schema, @Nullable int[] targetColumns) { */ public CompletableFuture upsert(InternalRow row) { byte[] key = keyEncoder.encode(row); + byte[] bucketKey = bucketKeyEncoder.encode(row); return send( - new WriteRecord(getPhysicalPath(row), WriteKind.PUT, key, key, row, targetColumns)); + new WriteRecord( + getPhysicalPath(row), WriteKind.PUT, key, bucketKey, row, targetColumns)); } /** @@ -124,8 +135,14 @@ public CompletableFuture upsert(InternalRow row) { */ public CompletableFuture delete(InternalRow row) { byte[] key = keyEncoder.encode(row); + byte[] bucketKey = bucketKeyEncoder.encode(row); return send( new WriteRecord( - getPhysicalPath(row), WriteKind.DELETE, key, key, null, targetColumns)); + getPhysicalPath(row), + WriteKind.DELETE, + key, + bucketKey, + null, + targetColumns)); } } diff --git a/fluss-client/src/main/java/com/alibaba/fluss/client/utils/ClientRpcMessageUtils.java b/fluss-client/src/main/java/com/alibaba/fluss/client/utils/ClientRpcMessageUtils.java index cadccb37..a114e36c 100644 --- a/fluss-client/src/main/java/com/alibaba/fluss/client/utils/ClientRpcMessageUtils.java +++ b/fluss-client/src/main/java/com/alibaba/fluss/client/utils/ClientRpcMessageUtils.java @@ -18,6 +18,7 @@ import com.alibaba.fluss.client.admin.OffsetSpec; import com.alibaba.fluss.client.lookup.LookupBatch; +import com.alibaba.fluss.client.lookup.PrefixLookupBatch; import com.alibaba.fluss.client.table.lake.LakeTableSnapshotInfo; import com.alibaba.fluss.client.table.snapshot.BucketSnapshotInfo; import com.alibaba.fluss.client.table.snapshot.BucketsSnapshotInfo; @@ -54,12 +55,14 @@ import com.alibaba.fluss.rpc.messages.PbLakeStorageInfo; import com.alibaba.fluss.rpc.messages.PbLookupReqForBucket; import com.alibaba.fluss.rpc.messages.PbPhysicalTablePath; +import com.alibaba.fluss.rpc.messages.PbPrefixLookupReqForBucket; import com.alibaba.fluss.rpc.messages.PbProduceLogReqForBucket; import com.alibaba.fluss.rpc.messages.PbPutKvReqForBucket; import com.alibaba.fluss.rpc.messages.PbRemoteLogFetchInfo; import com.alibaba.fluss.rpc.messages.PbRemoteLogSegment; import com.alibaba.fluss.rpc.messages.PbRemotePathAndLocalFile; import com.alibaba.fluss.rpc.messages.PbSnapshotForBucket; +import com.alibaba.fluss.rpc.messages.PrefixLookupRequest; import com.alibaba.fluss.rpc.messages.ProduceLogRequest; import com.alibaba.fluss.rpc.messages.PutKvRequest; import com.alibaba.fluss.rpc.protocol.ApiError; @@ -185,6 +188,22 @@ public static LookupRequest makeLookupRequest( return request; } + public static PrefixLookupRequest makePrefixLookupRequest( + long tableId, Collection lookupBatches) { + PrefixLookupRequest request = new PrefixLookupRequest().setTableId(tableId); + lookupBatches.forEach( + (batch) -> { + TableBucket tb = batch.tableBucket(); + PbPrefixLookupReqForBucket pbPrefixLookupReqForBucket = + request.addBucketsReq().setBucketId(tb.getBucket()); + if (tb.getPartitionId() != null) { + pbPrefixLookupReqForBucket.setPartitionId(tb.getPartitionId()); + } + batch.lookups().forEach(get -> pbPrefixLookupReqForBucket.addKey(get.key())); + }); + return request; + } + public static FetchLogResultForBucket getFetchLogResultForBucket( TableBucket tb, TablePath tp, PbFetchLogRespForBucket respForBucket) { FetchLogResultForBucket fetchLogResultForBucket; diff --git a/fluss-client/src/test/java/com/alibaba/fluss/client/table/FlussTableITCase.java b/fluss-client/src/test/java/com/alibaba/fluss/client/table/FlussTableITCase.java index 48abf51c..3b8e2a49 100644 --- a/fluss-client/src/test/java/com/alibaba/fluss/client/table/FlussTableITCase.java +++ b/fluss-client/src/test/java/com/alibaba/fluss/client/table/FlussTableITCase.java @@ -19,6 +19,7 @@ import com.alibaba.fluss.client.Connection; import com.alibaba.fluss.client.ConnectionFactory; import com.alibaba.fluss.client.admin.ClientToServerITCaseBase; +import com.alibaba.fluss.client.lookup.PrefixLookupResult; import com.alibaba.fluss.client.scanner.ScanRecord; import com.alibaba.fluss.client.scanner.log.LogScanner; import com.alibaba.fluss.client.scanner.log.ScanRecords; @@ -58,6 +59,7 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; +import java.util.concurrent.CompletableFuture; import java.util.stream.Collectors; import static com.alibaba.fluss.record.TestData.DATA1_ROW_TYPE; @@ -215,6 +217,63 @@ void testPutAndLookup() throws Exception { verifyPutAndLookup(table2, schema, new Object[] {"a", 1}); } + @Test + void testPutAndPrefixLookup() throws Exception { + TablePath tablePath = TablePath.of("test_db_1", "test_put_and_prefix_lookup_table"); + Schema schema = + Schema.newBuilder() + .column("a", DataTypes.INT()) + .column("b", DataTypes.STRING()) + .column("c", DataTypes.BIGINT()) + .column("d", DataTypes.STRING()) + .primaryKey("a", "b", "c") + .build(); + TableDescriptor descriptor = + TableDescriptor.builder().schema(schema).distributedBy(3, "a", "b").build(); + createTable(tablePath, descriptor, false); + Table table = conn.getTable(tablePath); + verifyPutAndLookup(table, schema, new Object[] {1, "a", 1L, "value1"}); + verifyPutAndLookup(table, schema, new Object[] {1, "a", 2L, "value2"}); + verifyPutAndLookup(table, schema, new Object[] {1, "a", 3L, "value3"}); + verifyPutAndLookup(table, schema, new Object[] {2, "a", 4L, "value4"}); + RowType rowType = schema.toRowType(); + + // test prefix lookup. + Schema prefixKeySchema = + Schema.newBuilder() + .column("a", DataTypes.INT()) + .column("b", DataTypes.STRING()) + .build(); + CompletableFuture result = + table.prefixLookup( + compactedRow(prefixKeySchema.toRowType(), new Object[] {1, "a"})); + PrefixLookupResult prefixLookupResult = result.get(); + assertThat(prefixLookupResult).isNotNull(); + List rowList = prefixLookupResult.getRowList(); + assertThat(rowList.size()).isEqualTo(3); + for (int i = 0; i < rowList.size(); i++) { + assertRowValueEquals( + rowType, rowList.get(i), new Object[] {1, "a", i + 1L, "value" + (i + 1)}); + } + + result = + table.prefixLookup( + compactedRow(prefixKeySchema.toRowType(), new Object[] {2, "a"})); + prefixLookupResult = result.get(); + assertThat(prefixLookupResult).isNotNull(); + rowList = prefixLookupResult.getRowList(); + assertThat(rowList.size()).isEqualTo(1); + assertRowValueEquals(rowType, rowList.get(0), new Object[] {2, "a", 4L, "value4"}); + + result = + table.prefixLookup( + compactedRow(prefixKeySchema.toRowType(), new Object[] {3, "a"})); + prefixLookupResult = result.get(); + assertThat(prefixLookupResult).isNotNull(); + rowList = prefixLookupResult.getRowList(); + assertThat(rowList.size()).isEqualTo(0); + } + @Test void testLookupForNotReadyTable() throws Exception { TablePath tablePath = TablePath.of("test_db_1", "test_lookup_unready_table_t1"); diff --git a/fluss-common/src/main/java/com/alibaba/fluss/config/ConfigOptions.java b/fluss-common/src/main/java/com/alibaba/fluss/config/ConfigOptions.java index 12f842a5..a7c8079c 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/config/ConfigOptions.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/config/ConfigOptions.java @@ -717,7 +717,7 @@ public class ConfigOptions { public static final ConfigOption CLIENT_LOOKUP_QUEUE_SIZE = key("client.lookup.queue-size") .intType() - .defaultValue(256) + .defaultValue(25600) .withDescription("The maximum number of pending lookup operations."); public static final ConfigOption CLIENT_LOOKUP_MAX_BATCH_SIZE = @@ -734,6 +734,14 @@ public class ConfigOptions { .withDescription( "The maximum number of unacknowledged lookup requests for lookup operations."); + public static final ConfigOption CLIENT_LOOKUP_BATCH_TIMEOUT = + key("client.lookup.batch-timeout") + .durationType() + .defaultValue(Duration.ofMillis(100)) + .withDescription( + "The maximum time to wait for the lookup batch to full, if this timeout is reached, " + + "the lookup batch will be closed to send."); + public static final ConfigOption CLIENT_SCANNER_REMOTE_LOG_PREFETCH_NUM = key("client.scanner.remote-log.prefetch-num") .intType() diff --git a/fluss-common/src/main/java/com/alibaba/fluss/metadata/TableDescriptor.java b/fluss-common/src/main/java/com/alibaba/fluss/metadata/TableDescriptor.java index d3737b93..ec6c7a44 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/metadata/TableDescriptor.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/metadata/TableDescriptor.java @@ -22,6 +22,7 @@ import com.alibaba.fluss.config.ConfigOptions; import com.alibaba.fluss.config.Configuration; import com.alibaba.fluss.config.ConfigurationUtils; +import com.alibaba.fluss.types.RowType; import com.alibaba.fluss.utils.AutoPartitionStrategy; import com.alibaba.fluss.utils.Preconditions; import com.alibaba.fluss.utils.json.JsonSerdeUtils; @@ -156,6 +157,16 @@ public List getBucketKey() { .orElse(Collections.emptyList()); } + public int[] getBucketKeyIndexes() { + List bucketKey = getBucketKey(); + RowType rowType = schema.toRowType(); + int[] bucketKeyIndex = new int[bucketKey.size()]; + for (int i = 0; i < bucketKey.size(); i++) { + bucketKeyIndex[i] = rowType.getFieldIndex(bucketKey.get(i)); + } + return bucketKeyIndex; + } + /** * Check if the table is partitioned or not. * @@ -360,21 +371,28 @@ private static TableDistribution normalizeDistribution( originDistribution.getBucketCount().orElse(null), defaultBucketKeyOfPrimaryKeyTable(schema, partitionKeys)); } else { - // check the provided bucket key and expected bucket key - List expectedBucketKeys = - defaultBucketKeyOfPrimaryKeyTable(schema, partitionKeys); - List pkColumns = schema.getPrimaryKey().get().getColumnNames(); - - if (expectedBucketKeys.size() != bucketKeys.size() - || !new HashSet<>(expectedBucketKeys).containsAll(bucketKeys)) { - throw new IllegalArgumentException( - String.format( - "Currently, bucket keys must be equal to primary keys excluding partition keys for primary-key tables. " - + "The primary keys are %s, the partition keys are %s, " - + "the expected bucket keys are %s, but the user-defined bucket keys are %s.", - pkColumns, partitionKeys, expectedBucketKeys, bucketKeys)); + // For the primary key table match prefix lookup pattern, we allow the bucket + // keys different from primary keys. + if (!bucketKeysMatchPrefixLookupPattern(schema, bucketKeys)) { + // check the provided bucket key and expected bucket key + List expectedBucketKeys = + defaultBucketKeyOfPrimaryKeyTable(schema, partitionKeys); + List pkColumns = schema.getPrimaryKey().get().getColumnNames(); + + if (expectedBucketKeys.size() != bucketKeys.size() + || !new HashSet<>(expectedBucketKeys).containsAll(bucketKeys)) { + throw new IllegalArgumentException( + String.format( + "Currently, bucket keys must be equal to primary keys excluding partition " + + "keys for primary-key tables. The primary keys are %s, the " + + "partition keys are %s, the expected bucket keys are %s, but " + + "the user-defined bucket keys are %s.", + pkColumns, + partitionKeys, + expectedBucketKeys, + bucketKeys)); + } } - return new TableDistribution( originDistribution.getBucketCount().orElse(null), bucketKeys); } @@ -393,6 +411,40 @@ private static TableDistribution normalizeDistribution( } } + /** + * Check if the table supports prefix lookup. Currently, only pk-table are supported, and the + * bucket key needs to be part of the primary key and must be a prefix of the primary key. For + * example, if a table has fields [a,b,c,d], and the primary key is set to [a, b, c], with the + * bucket key set to [a, b]. + * + * @return true if the table supports prefix lookup; otherwise, false + */ + public static boolean bucketKeysMatchPrefixLookupPattern( + Schema schema, List bucketKeys) { + if (!schema.getPrimaryKey().isPresent()) { + return false; + } + + RowType rowType = schema.toRowType(); + int[] pkIndexes = schema.getPrimaryKeyIndexes(); + int[] bucketKeyIndexes = new int[bucketKeys.size()]; + for (int i = 0; i < bucketKeys.size(); i++) { + bucketKeyIndexes[i] = rowType.getFieldIndex(bucketKeys.get(i)); + } + + if (bucketKeyIndexes.length >= pkIndexes.length) { + return false; + } else { + for (int i = 0; i < bucketKeyIndexes.length; i++) { + if (bucketKeyIndexes[i] != pkIndexes[i]) { + return false; + } + } + + return true; + } + } + /** The default bucket key of primary key table is the primary key excluding partition keys. */ private static List defaultBucketKeyOfPrimaryKeyTable( Schema schema, List partitionKeys) { diff --git a/fluss-common/src/main/java/com/alibaba/fluss/metrics/MetricNames.java b/fluss-common/src/main/java/com/alibaba/fluss/metrics/MetricNames.java index c92e64df..bfd1c92b 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/metrics/MetricNames.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/metrics/MetricNames.java @@ -76,6 +76,10 @@ public class MetricNames { public static final String FAILED_PUT_KV_REQUESTS_RATE = "failedPutKvRequestsPerSecond"; public static final String TOTAL_LIMIT_SCAN_REQUESTS_RATE = "totalLimitScanRequestsPerSecond"; public static final String FAILED_LIMIT_SCAN_REQUESTS_RATE = "failedLimitScanRequestsPerSecond"; + public static final String TOTAL_PREFIX_LOOKUP_REQUESTS_RATE = + "totalPrefixLookupRequestsPerSecond"; + public static final String FAILED_PREFIX_LOOKUP_REQUESTS_RATE = + "failedPrefixLookupRequestsPerSecond"; // -------------------------------------------------------------------------------------------- // metrics for table bucket diff --git a/fluss-common/src/main/java/com/alibaba/fluss/row/encode/KeyEncoder.java b/fluss-common/src/main/java/com/alibaba/fluss/row/encode/KeyEncoder.java index eed6b665..227da452 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/row/encode/KeyEncoder.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/row/encode/KeyEncoder.java @@ -54,7 +54,7 @@ public static KeyEncoder createKeyEncoder( return new KeyEncoder(rowType, encodeColIndexes); } - protected KeyEncoder(RowType rowType) { + public KeyEncoder(RowType rowType) { this(rowType, IntStream.range(0, rowType.getFieldCount()).toArray()); } diff --git a/fluss-common/src/test/java/com/alibaba/fluss/metadata/TableDescriptorTest.java b/fluss-common/src/test/java/com/alibaba/fluss/metadata/TableDescriptorTest.java index 3ed0c4bc..fed0d8ce 100644 --- a/fluss-common/src/test/java/com/alibaba/fluss/metadata/TableDescriptorTest.java +++ b/fluss-common/src/test/java/com/alibaba/fluss/metadata/TableDescriptorTest.java @@ -144,24 +144,21 @@ void testPrimaryKeyDifferentWithBucketKeys() { + "The primary keys are [f0, f3], the partition keys are [], " + "the expected bucket keys are [f0, f3], but the user-defined bucket keys are [f0, f1]."); - // bucket key is the subset of primary key. + // bucket key is the subset of primary key. This pattern can be support for prefixLookup. Schema schema0 = Schema.newBuilder() .column("f0", DataTypes.STRING()) .column("f1", DataTypes.BIGINT()) .primaryKey("f0", "f1") .build(); - assertThatThrownBy( - () -> - TableDescriptor.builder() - .schema(schema0) - .distributedBy(12, "f0") - .build()) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage( - "Currently, bucket keys must be equal to primary keys excluding partition keys for primary-key tables. " - + "The primary keys are [f0, f1], the partition keys are [], " - + "the expected bucket keys are [f0, f1], but the user-defined bucket keys are [f0]."); + TableDescriptor tableDescriptor = + TableDescriptor.builder().schema(schema0).distributedBy(12, "f0").build(); + assertThat(tableDescriptor.getBucketKey()).containsExactlyInAnyOrder("f0"); + assertThat(tableDescriptor.getBucketKeyIndexes()).isEqualTo(new int[] {0}); + assertThat( + TableDescriptor.bucketKeysMatchPrefixLookupPattern( + schema0, Collections.singletonList("f0"))) + .isTrue(); } @Test diff --git a/fluss-common/src/test/java/com/alibaba/fluss/testutils/DataTestUtils.java b/fluss-common/src/test/java/com/alibaba/fluss/testutils/DataTestUtils.java index 5ec14006..0cc65f7e 100644 --- a/fluss-common/src/test/java/com/alibaba/fluss/testutils/DataTestUtils.java +++ b/fluss-common/src/test/java/com/alibaba/fluss/testutils/DataTestUtils.java @@ -191,23 +191,33 @@ public static MemoryLogRecords genLogRecordsWithBaseOffsetAndTimestamp( public static KvRecordBatch genKvRecordBatch(List> keyAndValues) throws Exception { - return genKvRecordBatchWithWriterId(keyAndValues, NO_WRITER_ID, NO_BATCH_SEQUENCE); + return genKvRecordBatch(DATA1_KEY_TYPE, DATA1_ROW_TYPE, keyAndValues); + } + + public static KvRecordBatch genKvRecordBatch( + RowType keyType, RowType valueType, List> keyAndValues) + throws Exception { + return genKvRecordBatchWithWriterId( + keyAndValues, keyType, valueType, NO_WRITER_ID, NO_BATCH_SEQUENCE); } public static KvRecordBatch genKvRecordBatchWithWriterId( - List> keyAndValues, long writerId, int batchSequence) + List> keyAndValues, + RowType keyType, + RowType valueType, + long writerId, + int batchSequence) throws Exception { - KeyEncoder keyEncoder = new KeyEncoder(DATA1_ROW_TYPE, new int[] {0}); + KeyEncoder keyEncoder = new KeyEncoder(keyType); KvRecordTestUtils.KvRecordBatchFactory kvRecordBatchFactory = KvRecordTestUtils.KvRecordBatchFactory.of(DEFAULT_SCHEMA_ID); KvRecordTestUtils.KvRecordFactory kvRecordFactory = - KvRecordTestUtils.KvRecordFactory.of(DATA1_ROW_TYPE); + KvRecordTestUtils.KvRecordFactory.of(valueType); List records = new ArrayList<>(); for (Tuple2 keyAndValue : keyAndValues) { records.add( kvRecordFactory.ofRecord( - keyEncoder.encode(row(DATA1_KEY_TYPE, keyAndValue.f0)), - keyAndValue.f1)); + keyEncoder.encode(row(keyType, keyAndValue.f0)), keyAndValue.f1)); } return kvRecordBatchFactory.ofRecords(records, writerId, batchSequence); } diff --git a/fluss-common/src/test/java/com/alibaba/fluss/utils/json/JsonSerdeTestBase.java b/fluss-common/src/test/java/com/alibaba/fluss/utils/json/JsonSerdeTestBase.java index 808c4669..5e6f9b47 100644 --- a/fluss-common/src/test/java/com/alibaba/fluss/utils/json/JsonSerdeTestBase.java +++ b/fluss-common/src/test/java/com/alibaba/fluss/utils/json/JsonSerdeTestBase.java @@ -23,7 +23,6 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; -import static com.alibaba.fluss.utils.Preconditions.checkArgument; import static org.assertj.core.api.Assertions.assertThat; /** Abstract test base for json serde. */ @@ -50,11 +49,7 @@ protected & JsonDeserializer> JsonSerdeTestBase void testJsonSerde() throws IOException { T[] testObjects = createObjects(); String[] expectedJsons = expectedJsons(); - checkArgument( - testObjects.length == expectedJsons.length, - "The length of createObjects() and expectedJsons() should be the same, but is %s and %s", - testObjects.length, - expectedJsons.length); + assertThat(testObjects.length).isEqualTo(expectedJsons.length); for (int i = 0; i < testObjects.length; i++) { T value = testObjects[i]; final byte[] json = JsonSerdeUtils.writeValueAsBytes(value, serializer); diff --git a/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/catalog/FlinkCatalog.java b/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/catalog/FlinkCatalog.java index ddbc8a5f..671b8b86 100644 --- a/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/catalog/FlinkCatalog.java +++ b/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/catalog/FlinkCatalog.java @@ -76,13 +76,13 @@ public class FlinkCatalog implements Catalog { public static final String LAKE_TABLE_SPLITTER = "$lake"; - private final ClassLoader classLoader; + protected final ClassLoader classLoader; - private final String catalogName; - private final @Nullable String defaultDatabase; - private final String bootstrapServers; - private Connection connection; - private Admin admin; + protected final String catalogName; + protected final @Nullable String defaultDatabase; + protected final String bootstrapServers; + protected Connection connection; + protected Admin admin; private volatile @Nullable LakeCatalog lakeCatalog; @@ -272,7 +272,7 @@ public CatalogBaseTable getTable(ObjectPath objectPath) } } - private CatalogBaseTable getLakeTable(String databaseName, String tableName) + protected CatalogBaseTable getLakeTable(String databaseName, String tableName) throws TableNotExistException, CatalogException { mayInitLakeCatalogCatalog(); String[] tableComponents = tableName.split("\\" + LAKE_TABLE_SPLITTER); @@ -514,7 +514,7 @@ public void alterPartitionColumnStatistics( throw new UnsupportedOperationException(); } - private TablePath toTablePath(ObjectPath objectPath) { + protected TablePath toTablePath(ObjectPath objectPath) { return TablePath.of(objectPath.getDatabaseName(), objectPath.getObjectName()); } diff --git a/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/catalog/FlinkTableFactory.java b/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/catalog/FlinkTableFactory.java index dbe033bc..62baf248 100644 --- a/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/catalog/FlinkTableFactory.java +++ b/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/catalog/FlinkTableFactory.java @@ -78,6 +78,7 @@ public DynamicTableSource createDynamicTableSource(Context context) { == RuntimeExecutionMode.STREAMING; final ReadableConfig tableOptions = helper.getOptions(); + RowType tableOutputType = (RowType) context.getPhysicalRowDataType().getLogicalType(); FlinkConnectorOptionsUtils.validateTableSourceOptions(tableOptions); ZoneId timeZone = @@ -90,8 +91,6 @@ public DynamicTableSource createDynamicTableSource(Context context) { ResolvedCatalogTable resolvedCatalogTable = context.getCatalogTable(); int[] primaryKeyIndexes = resolvedSchema.getPrimaryKeyIndexes(); - RowType tableOutputType = (RowType) context.getPhysicalRowDataType().getLogicalType(); - // options for lookup LookupCache cache = null; diff --git a/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/FlinkTableSource.java b/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/FlinkTableSource.java index a3e89c59..53250681 100644 --- a/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/FlinkTableSource.java +++ b/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/FlinkTableSource.java @@ -279,14 +279,18 @@ public boolean isBounded() { public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext context) { LookupNormalizer lookupNormalizer = LookupNormalizer.validateAndCreateLookupNormalizer( - context.getKeys(), primaryKeyIndexes, tableOutputType); + context.getKeys(), + primaryKeyIndexes, + tableOutputType, + tablePath, + flussConfig); if (lookupAsync) { AsyncLookupFunction asyncLookupFunction = new FlinkAsyncLookupFunction( flussConfig, tablePath, tableOutputType, - primaryKeyIndexes, + lookupNormalizer.getLookupKeyIndexes(), lookupMaxRetryTimes, lookupNormalizer, projectedFields); @@ -301,7 +305,7 @@ public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext context) { flussConfig, tablePath, tableOutputType, - primaryKeyIndexes, + lookupNormalizer.getLookupKeyIndexes(), lookupMaxRetryTimes, lookupNormalizer, projectedFields); diff --git a/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/lookup/FlinkAsyncLookupFunction.java b/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/lookup/FlinkAsyncLookupFunction.java index f3c10483..0c48a799 100644 --- a/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/lookup/FlinkAsyncLookupFunction.java +++ b/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/lookup/FlinkAsyncLookupFunction.java @@ -18,7 +18,9 @@ import com.alibaba.fluss.client.Connection; import com.alibaba.fluss.client.ConnectionFactory; -import com.alibaba.fluss.client.table.LookupResult; +import com.alibaba.fluss.client.lookup.LookupResult; +import com.alibaba.fluss.client.lookup.LookupType; +import com.alibaba.fluss.client.lookup.PrefixLookupResult; import com.alibaba.fluss.client.table.Table; import com.alibaba.fluss.config.Configuration; import com.alibaba.fluss.connector.flink.source.lookup.LookupNormalizer.RemainingFilter; @@ -40,8 +42,10 @@ import javax.annotation.Nullable; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.List; import java.util.concurrent.CompletableFuture; /** A flink async lookup function for fluss. */ @@ -55,9 +59,10 @@ public class FlinkAsyncLookupFunction extends AsyncLookupFunction { private final TablePath tablePath; private final int maxRetryTimes; private final RowType flinkRowType; - private final int[] pkIndexes; + private final int[] lookupKeyIndexes; private final LookupNormalizer lookupNormalizer; @Nullable private final int[] projection; + private final LookupType flussLookupType; private transient FlinkRowToFlussRowConverter flinkRowToFlussRowConverter; private transient FlussRowToFlinkRowConverter flussRowToFlinkRowConverter; @@ -68,7 +73,7 @@ public FlinkAsyncLookupFunction( Configuration flussConfig, TablePath tablePath, RowType flinkRowType, - int[] pkIndexes, + int[] lookupKeyIndexes, int maxRetryTimes, LookupNormalizer lookupNormalizer, @Nullable int[] projection) { @@ -76,9 +81,10 @@ public FlinkAsyncLookupFunction( this.tablePath = tablePath; this.maxRetryTimes = maxRetryTimes; this.flinkRowType = flinkRowType; - this.pkIndexes = pkIndexes; + this.lookupKeyIndexes = lookupKeyIndexes; this.lookupNormalizer = lookupNormalizer; this.projection = projection; + this.flussLookupType = lookupNormalizer.getFlussLookupType(); } @Override @@ -89,7 +95,7 @@ public void open(FunctionContext context) { // TODO: convert to Fluss GenericRow to avoid unnecessary deserialization flinkRowToFlussRowConverter = FlinkRowToFlussRowConverter.create( - FlinkUtils.projectRowType(flinkRowType, pkIndexes), + FlinkUtils.projectRowType(flinkRowType, lookupKeyIndexes), table.getDescriptor().getKvFormat()); final RowType outputRowType; @@ -132,53 +138,105 @@ private void fetchResult( int currentRetry, InternalRow keyRow, @Nullable RemainingFilter remainingFilter) { - CompletableFuture responseFuture = table.lookup(keyRow); - responseFuture.whenComplete( - (result, throwable) -> { - if (throwable != null) { - if (throwable instanceof TableNotExistException) { - LOG.error("Table '{}' not found ", tablePath, throwable); - resultFuture.completeExceptionally( - new RuntimeException( - "Fluss table '" + tablePath + "' not found.", - throwable)); - } else { - LOG.error( - "Fluss asyncLookup error, retry times = {}", - currentRetry, - throwable); - if (currentRetry >= maxRetryTimes) { - String exceptionMsg = - String.format( - "Execution of Fluss asyncLookup failed: %s, retry times = %d.", - throwable.getMessage(), currentRetry); - resultFuture.completeExceptionally( - new RuntimeException(exceptionMsg, throwable)); - } else { - try { - Thread.sleep(1000L * currentRetry); - } catch (InterruptedException e1) { - resultFuture.completeExceptionally(e1); + if (flussLookupType == LookupType.LOOKUP) { + table.lookup(keyRow) + .whenComplete( + (result, throwable) -> { + if (throwable != null) { + handleLookupFailed( + resultFuture, + throwable, + currentRetry, + keyRow, + remainingFilter); + } else { + handleLookupSuccess(resultFuture, result, remainingFilter); } - fetchResult( - resultFuture, currentRetry + 1, keyRow, remainingFilter); - } - } - } else { - InternalRow row = result.getRow(); - if (row == null) { - resultFuture.complete(Collections.emptyList()); - } else { - RowData flinkRow = - flussRowToFlinkRowConverter.toFlinkRowData(maybeProject(row)); - if (remainingFilter != null && !remainingFilter.isMatch(flinkRow)) { - resultFuture.complete(Collections.emptyList()); - } else { - resultFuture.complete(Collections.singletonList(flinkRow)); - } - } - } - }); + }); + } else if (flussLookupType == LookupType.PREFIX_LOOKUP) { + table.prefixLookup(keyRow) + .whenComplete( + (result, throwable) -> { + if (throwable != null) { + handleLookupFailed( + resultFuture, + throwable, + currentRetry, + keyRow, + remainingFilter); + } else { + handlePrefixLookupSuccess( + resultFuture, result, remainingFilter); + } + }); + } else { + resultFuture.completeExceptionally( + new UnsupportedOperationException( + "Unsupported Fluss lookup type. Currently, Fluss only " + + "support lookup by primary keys or prefix lookup by bucket keys.")); + } + } + + private void handleLookupFailed( + CompletableFuture> resultFuture, + Throwable throwable, + int currentRetry, + InternalRow keyRow, + @Nullable RemainingFilter remainingFilter) { + if (throwable instanceof TableNotExistException) { + LOG.error("Table '{}' not found ", tablePath, throwable); + resultFuture.completeExceptionally( + new RuntimeException("Fluss table '" + tablePath + "' not found.", throwable)); + } else { + LOG.error("Fluss asyncLookup error, retry times = {}", currentRetry, throwable); + if (currentRetry >= maxRetryTimes) { + String exceptionMsg = + String.format( + "Execution of Fluss asyncLookup failed: %s, retry times = %d.", + throwable.getMessage(), currentRetry); + resultFuture.completeExceptionally(new RuntimeException(exceptionMsg, throwable)); + } else { + try { + Thread.sleep(1000L * currentRetry); + } catch (InterruptedException e1) { + resultFuture.completeExceptionally(e1); + } + fetchResult(resultFuture, currentRetry + 1, keyRow, remainingFilter); + } + } + } + + private void handleLookupSuccess( + CompletableFuture> resultFuture, + LookupResult result, + @Nullable RemainingFilter remainingFilter) { + InternalRow row = result.getRow(); + if (row == null) { + resultFuture.complete(Collections.emptyList()); + } else { + RowData flinkRow = flussRowToFlinkRowConverter.toFlinkRowData(maybeProject(row)); + if (remainingFilter != null && !remainingFilter.isMatch(flinkRow)) { + resultFuture.complete(Collections.emptyList()); + } else { + resultFuture.complete(Collections.singletonList(flinkRow)); + } + } + } + + private void handlePrefixLookupSuccess( + CompletableFuture> resultFuture, + PrefixLookupResult result, + @Nullable RemainingFilter remainingFilter) { + List projectedRow = new ArrayList<>(); + for (InternalRow row : result.getRowList()) { + if (row != null) { + RowData flinkRow = flussRowToFlinkRowConverter.toFlinkRowData(maybeProject(row)); + if (remainingFilter == null || remainingFilter.isMatch(flinkRow)) { + projectedRow.add(flinkRow); + } + } + } + resultFuture.complete(projectedRow); } private InternalRow maybeProject(InternalRow row) { diff --git a/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/lookup/FlinkLookupFunction.java b/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/lookup/FlinkLookupFunction.java index ba57d57e..99bb3ced 100644 --- a/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/lookup/FlinkLookupFunction.java +++ b/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/lookup/FlinkLookupFunction.java @@ -18,6 +18,7 @@ import com.alibaba.fluss.client.Connection; import com.alibaba.fluss.client.ConnectionFactory; +import com.alibaba.fluss.client.lookup.LookupType; import com.alibaba.fluss.client.table.Table; import com.alibaba.fluss.config.Configuration; import com.alibaba.fluss.connector.flink.utils.FlinkConversions; @@ -37,8 +38,10 @@ import javax.annotation.Nullable; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.List; /** A flink lookup function for fluss. */ public class FlinkLookupFunction extends LookupFunction { @@ -50,9 +53,10 @@ public class FlinkLookupFunction extends LookupFunction { private final TablePath tablePath; private final int maxRetryTimes; private final RowType flinkRowType; - private final int[] pkIndexes; + private final int[] lookupKeyIndexes; private final LookupNormalizer lookupNormalizer; @Nullable private final int[] projection; + private final LookupType flussLookupType; private transient FlinkRowToFlussRowConverter flinkRowToFlussRowConverter; private transient FlussRowToFlinkRowConverter flussRowToFlinkRowConverter; @@ -64,7 +68,7 @@ public FlinkLookupFunction( Configuration flussConfig, TablePath tablePath, RowType flinkRowType, - int[] pkIndexes, + int[] lookupKeyIndexes, int maxRetryTimes, LookupNormalizer lookupNormalizer, @Nullable int[] projection) { @@ -72,9 +76,10 @@ public FlinkLookupFunction( this.tablePath = tablePath; this.maxRetryTimes = maxRetryTimes; this.flinkRowType = flinkRowType; - this.pkIndexes = pkIndexes; + this.lookupKeyIndexes = lookupKeyIndexes; this.lookupNormalizer = lookupNormalizer; this.projection = projection; + this.flussLookupType = lookupNormalizer.getFlussLookupType(); } @Override @@ -85,7 +90,7 @@ public void open(FunctionContext context) { // TODO: convert to Fluss GenericRow to avoid unnecessary deserialization flinkRowToFlussRowConverter = FlinkRowToFlussRowConverter.create( - FlinkUtils.projectRowType(flinkRowType, pkIndexes), + FlinkUtils.projectRowType(flinkRowType, lookupKeyIndexes), table.getDescriptor().getKvFormat()); final RowType outputRowType; @@ -121,15 +126,31 @@ public Collection lookup(RowData keyRow) { InternalRow flussKeyRow = flinkRowToFlussRowConverter.toInternalRow(normalizedKeyRow); for (int retry = 0; retry <= maxRetryTimes; retry++) { try { - InternalRow row = table.lookup(flussKeyRow).get().getRow(); - if (row != null) { - RowData flinkRow = - flussRowToFlinkRowConverter.toFlinkRowData(maybeProject(row)); - if (remainingFilter == null || remainingFilter.isMatch(flinkRow)) { - return Collections.singletonList(flinkRow); - } else { - return Collections.emptyList(); + if (flussLookupType == LookupType.LOOKUP) { + InternalRow row = table.lookup(flussKeyRow).get().getRow(); + if (row != null) { + RowData flinkRow = + flussRowToFlinkRowConverter.toFlinkRowData(maybeProject(row)); + if (remainingFilter == null || remainingFilter.isMatch(flinkRow)) { + return Collections.singletonList(flinkRow); + } else { + return Collections.emptyList(); + } } + } else { + List projectedRows = new ArrayList<>(); + List lookupRows = + table.prefixLookup(flussKeyRow).get().getRowList(); + for (InternalRow row : lookupRows) { + if (row != null) { + RowData flinkRow = + flussRowToFlinkRowConverter.toFlinkRowData(maybeProject(row)); + if (remainingFilter == null || remainingFilter.isMatch(flinkRow)) { + projectedRows.add(flinkRow); + } + } + } + return projectedRows; } } catch (Exception e) { LOG.error(String.format("Fluss lookup error, retry times = %d", retry), e); diff --git a/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/lookup/LookupNormalizer.java b/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/lookup/LookupNormalizer.java index 7c372d53..199c684a 100644 --- a/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/lookup/LookupNormalizer.java +++ b/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/lookup/LookupNormalizer.java @@ -16,6 +16,14 @@ package com.alibaba.fluss.connector.flink.source.lookup; +import com.alibaba.fluss.client.Connection; +import com.alibaba.fluss.client.ConnectionFactory; +import com.alibaba.fluss.client.lookup.LookupType; +import com.alibaba.fluss.client.table.Table; +import com.alibaba.fluss.config.Configuration; +import com.alibaba.fluss.metadata.TableDescriptor; +import com.alibaba.fluss.metadata.TablePath; + import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; @@ -50,7 +58,8 @@ public class LookupNormalizer implements Serializable { private static final long serialVersionUID = 1L; - public static final LookupNormalizer NOOP_NORMALIZER = new LookupNormalizer(null, null, null); + public static final LookupNormalizer NOOP_NORMALIZER = + new LookupNormalizer(LookupType.LOOKUP, new int[0], null, null, null); /** Mapping from normalized key index to the lookup key index (in the lookup row). */ @Nullable private final FieldGetter[] normalizedKeyGetters; @@ -61,10 +70,17 @@ public class LookupNormalizer implements Serializable { /** The field getter to get the remaining condition result from the lookup result row. */ @Nullable private final FieldGetter[] resultFieldGetters; + private final LookupType flussLookupType; + private final int[] lookupKeyIndexes; + private LookupNormalizer( + LookupType flussLookupType, + int[] lookupKeyIndexes, @Nullable FieldGetter[] normalizedKeyGetters, @Nullable FieldGetter[] conditionFieldGetters, @Nullable FieldGetter[] resultFieldGetters) { + this.flussLookupType = flussLookupType; + this.lookupKeyIndexes = lookupKeyIndexes; this.normalizedKeyGetters = normalizedKeyGetters; this.conditionFieldGetters = conditionFieldGetters; this.resultFieldGetters = resultFieldGetters; @@ -76,6 +92,14 @@ private LookupNormalizer( } } + public LookupType getFlussLookupType() { + return flussLookupType; + } + + public int[] getLookupKeyIndexes() { + return lookupKeyIndexes; + } + public RowData normalizeLookupKey(RowData lookupKey) { if (normalizedKeyGetters == null) { return lookupKey; @@ -141,45 +165,81 @@ public boolean fieldMatches(RowData result) { /** Validate the lookup key indexes and primary keys, and create a {@link LookupNormalizer}. */ public static LookupNormalizer validateAndCreateLookupNormalizer( - int[][] lookupKeyIndexes, int[] primaryKeys, RowType schema) { - if (primaryKeys.length == 0) { + int[][] lookupKeyIndexes, + int[] primaryKeys, + RowType schema, + TablePath tablePath, + Configuration flussConfig) { + int[] bucketKeys; + boolean supportPrefixLookup; + try (Connection connection = ConnectionFactory.createConnection(flussConfig); + Table table = connection.getTable(tablePath)) { + TableDescriptor descriptor = table.getDescriptor(); + bucketKeys = descriptor.getBucketKeyIndexes(); + supportPrefixLookup = + TableDescriptor.bucketKeysMatchPrefixLookupPattern( + descriptor.getSchema(), descriptor.getBucketKey()); + } catch (Exception e) { + throw new TableException( + "Failed execute validate and create lookup normalizer operation on Fluss table.", + e); + } + + if (primaryKeys.length == 0 || bucketKeys.length == 0) { throw new UnsupportedOperationException( - "Fluss lookup function only support lookup table with primary key."); + "Fluss lookup function only support lookup table with primary key or prefix lookup with bucket key."); } - // we compare string names rather than int index for better error message and readability, - // the length of lookup key and primary key shouldn't be large, so the overhead is low. - String[] columnNames = schema.getFieldNames().toArray(new String[0]); - String[] primaryKeyNames = - Arrays.stream(primaryKeys).mapToObj(i -> columnNames[i]).toArray(String[]::new); - // get the lookup keys int[] lookupKeys = new int[lookupKeyIndexes.length]; - String[] lookupKeyNames = new String[lookupKeyIndexes.length]; - for (int i = 0; i < lookupKeyNames.length; i++) { + for (int i = 0; i < lookupKeys.length; i++) { int[] innerKeyArr = lookupKeyIndexes[i]; checkArgument(innerKeyArr.length == 1, "Do not support nested lookup keys"); // lookupKeyIndexes passed by Flink is key indexed after projection pushdown, // we do remaining condition filter on the projected row, so no remapping needed. lookupKeys[i] = innerKeyArr[0]; - lookupKeyNames[i] = columnNames[innerKeyArr[0]]; } - if (Arrays.equals(lookupKeys, primaryKeys)) { - return NOOP_NORMALIZER; + if (supportPrefixLookup && lookupKeys.length == bucketKeys.length) { + // bucket key prefix lookup. + return createLookupNormalizer(lookupKeys, bucketKeys, schema, LookupType.PREFIX_LOOKUP); + } else { + // Primary key lookup. + return createLookupNormalizer(lookupKeys, primaryKeys, schema, LookupType.LOOKUP); + } + } + + /** create a {@link LookupNormalizer}. */ + private static LookupNormalizer createLookupNormalizer( + int[] lookupKeys, int[] keys, RowType schema, LookupType flussLookupType) { + // we compare string names rather than int index for better error message and readability, + // the length of lookup key and keys (primary key or index key) shouldn't be large, so the + // overhead is low. + String[] columnNames = schema.getFieldNames().toArray(new String[0]); + String[] keyNames = + Arrays.stream(keys).mapToObj(i -> columnNames[i]).toArray(String[]::new); + + // get the lookup keys + String[] lookupKeyNames = new String[lookupKeys.length]; + for (int i = 0; i < lookupKeyNames.length; i++) { + lookupKeyNames[i] = columnNames[lookupKeys[i]]; + } + + if (Arrays.equals(lookupKeys, keys)) { + return new LookupNormalizer(flussLookupType, keys, null, null, null); } - FieldGetter[] normalizedKeyGetters = new FieldGetter[primaryKeys.length]; - for (int i = 0; i < primaryKeyNames.length; i++) { - LogicalType fieldType = schema.getTypeAt(primaryKeys[i]); - int lookupKeyIndex = findIndex(lookupKeyNames, primaryKeyNames[i]); + FieldGetter[] normalizedKeyGetters = new FieldGetter[keys.length]; + for (int i = 0; i < keyNames.length; i++) { + LogicalType fieldType = schema.getTypeAt(keys[i]); + int lookupKeyIndex = findIndex(lookupKeyNames, keyNames[i]); normalizedKeyGetters[i] = RowData.createFieldGetter(fieldType, lookupKeyIndex); } - Set primaryKeySet = Arrays.stream(primaryKeys).boxed().collect(Collectors.toSet()); + Set keySet = Arrays.stream(keys).boxed().collect(Collectors.toSet()); List conditionFieldGetters = new ArrayList<>(); List resultFieldGetters = new ArrayList<>(); for (int i = 0; i < lookupKeys.length; i++) { - if (!primaryKeySet.contains(i)) { + if (!keySet.contains(i)) { LogicalType fieldType = schema.getTypeAt(lookupKeys[i]); conditionFieldGetters.add(RowData.createFieldGetter(fieldType, i)); resultFieldGetters.add(RowData.createFieldGetter(fieldType, lookupKeys[i])); @@ -187,6 +247,8 @@ public static LookupNormalizer validateAndCreateLookupNormalizer( } return new LookupNormalizer( + flussLookupType, + keys, normalizedKeyGetters, conditionFieldGetters.toArray(new FieldGetter[0]), resultFieldGetters.toArray(new FieldGetter[0])); @@ -199,8 +261,8 @@ private static int findIndex(String[] columnNames, String key) { } } throw new TableException( - "Fluss lookup function only supports lookup table with lookup keys contain all primary keys." - + " Can't find primary key '" + "Fluss lookup function only supports lookup table with lookup keys contain all primary keys or bucket keys." + + " Can't find primary key or bucket key '" + key + "' in lookup keys " + Arrays.toString(columnNames)); diff --git a/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/utils/PushdownUtils.java b/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/utils/PushdownUtils.java index 586e8465..841e551c 100644 --- a/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/utils/PushdownUtils.java +++ b/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/utils/PushdownUtils.java @@ -276,8 +276,8 @@ public static Collection querySingleRow( public static void deleteSingleRow( GenericRow deleteRow, TablePath tablePath, Configuration flussConfig) { - try (Connection connection = ConnectionFactory.createConnection(flussConfig)) { - Table table = connection.getTable(tablePath); + try (Connection connection = ConnectionFactory.createConnection(flussConfig); + Table table = connection.getTable(tablePath)) { UpsertWriter upsertWriter = table.getUpsertWriter(); upsertWriter.delete(deleteRow).get(); } catch (Exception e) { diff --git a/fluss-connectors/fluss-connector-flink/src/test/java/com/alibaba/fluss/connector/flink/source/FlinkTableSourceITCase.java b/fluss-connectors/fluss-connector-flink/src/test/java/com/alibaba/fluss/connector/flink/source/FlinkTableSourceITCase.java index 8ad2a6cd..19845bf3 100644 --- a/fluss-connectors/fluss-connector-flink/src/test/java/com/alibaba/fluss/connector/flink/source/FlinkTableSourceITCase.java +++ b/fluss-connectors/fluss-connector-flink/src/test/java/com/alibaba/fluss/connector/flink/source/FlinkTableSourceITCase.java @@ -768,8 +768,8 @@ void testLookup2PkTableWith1KeyInCondition(Caching caching, boolean async) throw assertThatThrownBy(() -> tEnv.executeSql(dimJoinQuery)) .hasStackTraceContaining( "Fluss lookup function only supports lookup table with " - + "lookup keys contain all primary keys. Can't find primary " - + "key 'name' in lookup keys [id]"); + + "lookup keys contain all primary keys or bucket keys. Can't find primary " + + "key or bucket key 'name' in lookup keys [id]"); } /** diff --git a/fluss-rpc/src/main/java/com/alibaba/fluss/rpc/entity/PrefixLookupResultForBucket.java b/fluss-rpc/src/main/java/com/alibaba/fluss/rpc/entity/PrefixLookupResultForBucket.java new file mode 100644 index 00000000..508401eb --- /dev/null +++ b/fluss-rpc/src/main/java/com/alibaba/fluss/rpc/entity/PrefixLookupResultForBucket.java @@ -0,0 +1,47 @@ +/* + * Copyright (c) 2024 Alibaba Group Holding Ltd. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.alibaba.fluss.rpc.entity; + +import com.alibaba.fluss.metadata.TableBucket; +import com.alibaba.fluss.rpc.messages.PrefixLookupRequest; +import com.alibaba.fluss.rpc.protocol.ApiError; + +import java.util.List; + +/** The Result of {@link PrefixLookupRequest} for each table bucket. */ +public class PrefixLookupResultForBucket extends ResultForBucket { + + private final List> values; + + public PrefixLookupResultForBucket(TableBucket tableBucket, List> values) { + this(tableBucket, values, ApiError.NONE); + } + + public PrefixLookupResultForBucket(TableBucket tableBucket, ApiError error) { + this(tableBucket, null, error); + } + + public PrefixLookupResultForBucket( + TableBucket tableBucket, List> values, ApiError error) { + super(tableBucket, error); + this.values = values; + } + + public List> prefixLookupValues() { + return values; + } +} diff --git a/fluss-rpc/src/main/java/com/alibaba/fluss/rpc/gateway/TabletServerGateway.java b/fluss-rpc/src/main/java/com/alibaba/fluss/rpc/gateway/TabletServerGateway.java index 543dfab1..efc96fde 100644 --- a/fluss-rpc/src/main/java/com/alibaba/fluss/rpc/gateway/TabletServerGateway.java +++ b/fluss-rpc/src/main/java/com/alibaba/fluss/rpc/gateway/TabletServerGateway.java @@ -35,6 +35,8 @@ import com.alibaba.fluss.rpc.messages.NotifyLeaderAndIsrResponse; import com.alibaba.fluss.rpc.messages.NotifyRemoteLogOffsetsRequest; import com.alibaba.fluss.rpc.messages.NotifyRemoteLogOffsetsResponse; +import com.alibaba.fluss.rpc.messages.PrefixLookupRequest; +import com.alibaba.fluss.rpc.messages.PrefixLookupResponse; import com.alibaba.fluss.rpc.messages.ProduceLogRequest; import com.alibaba.fluss.rpc.messages.ProduceLogResponse; import com.alibaba.fluss.rpc.messages.PutKvRequest; @@ -99,6 +101,14 @@ CompletableFuture notifyLeaderAndIsr( @RPC(api = ApiKeys.LOOKUP) CompletableFuture lookup(LookupRequest request); + /** + * Prefix lookup to get value by index key. + * + * @return Index lookup response. + */ + @RPC(api = ApiKeys.PREFIX_LOOKUP) + CompletableFuture prefixLookup(PrefixLookupRequest request); + /** * Get limit number of values from the specified table bucket. * diff --git a/fluss-rpc/src/main/java/com/alibaba/fluss/rpc/netty/server/RequestsMetrics.java b/fluss-rpc/src/main/java/com/alibaba/fluss/rpc/netty/server/RequestsMetrics.java index 212f9536..74741c49 100644 --- a/fluss-rpc/src/main/java/com/alibaba/fluss/rpc/netty/server/RequestsMetrics.java +++ b/fluss-rpc/src/main/java/com/alibaba/fluss/rpc/netty/server/RequestsMetrics.java @@ -93,6 +93,8 @@ private static String toRequestName(ApiKeys apiKeys, boolean isFromFollower) { return "putKv"; case LOOKUP: return "lookup"; + case PREFIX_LOOKUP: + return "indexLookup"; case FETCH_LOG: return isFromFollower ? "fetchLogFollower" : "fetchLogClient"; default: diff --git a/fluss-rpc/src/main/java/com/alibaba/fluss/rpc/protocol/ApiKeys.java b/fluss-rpc/src/main/java/com/alibaba/fluss/rpc/protocol/ApiKeys.java index 949917b2..8bdbe50d 100644 --- a/fluss-rpc/src/main/java/com/alibaba/fluss/rpc/protocol/ApiKeys.java +++ b/fluss-rpc/src/main/java/com/alibaba/fluss/rpc/protocol/ApiKeys.java @@ -61,7 +61,8 @@ public enum ApiKeys { NOTIFY_LAKE_TABLE_OFFSET(1031, 0, 0, PRIVATE), DESCRIBE_LAKE_STORAGE(1032, 0, 0, PUBLIC), GET_LAKE_TABLE_SNAPSHOT(1033, 0, 0, PUBLIC), - LIMIT_SCAN(1034, 0, 0, PUBLIC); + LIMIT_SCAN(1034, 0, 0, PUBLIC), + PREFIX_LOOKUP(1035, 0, 0, PUBLIC); private static final Map ID_TO_TYPE = Arrays.stream(ApiKeys.values()) diff --git a/fluss-rpc/src/main/proto/FlussApi.proto b/fluss-rpc/src/main/proto/FlussApi.proto index 4c68064a..dea96366 100644 --- a/fluss-rpc/src/main/proto/FlussApi.proto +++ b/fluss-rpc/src/main/proto/FlussApi.proto @@ -202,6 +202,16 @@ message LookupResponse { repeated PbLookupRespForBucket buckets_resp = 1; } +// Prefix Lookup request and response +message PrefixLookupRequest { + required int64 table_id = 1; + repeated PbPrefixLookupReqForBucket buckets_req = 2; +} + +message PrefixLookupResponse { + repeated PbPrefixLookupRespForBucket buckets_resp = 1; +} + // limit scan request and response message LimitScanRequest { @@ -550,6 +560,24 @@ message PbValue { optional bytes values = 1; } +message PbValueList { + repeated bytes values = 1; +} + +message PbPrefixLookupReqForBucket { + optional int64 partition_id = 1; + required int32 bucket_id = 2; + repeated bytes keys = 3; +} + +message PbPrefixLookupRespForBucket { + optional int64 partition_id = 1; + required int32 bucket_id = 2; + optional int32 error_code = 3; + optional string error_message = 4; + repeated PbValueList value_lists = 5; +} + message PbTableBucket { required int64 table_id = 1; optional int64 partition_id = 2; diff --git a/fluss-server/src/main/java/com/alibaba/fluss/server/kv/KvTablet.java b/fluss-server/src/main/java/com/alibaba/fluss/server/kv/KvTablet.java index 7dc740ce..dfb5d85a 100644 --- a/fluss-server/src/main/java/com/alibaba/fluss/server/kv/KvTablet.java +++ b/fluss-server/src/main/java/com/alibaba/fluss/server/kv/KvTablet.java @@ -470,6 +470,15 @@ public List multiGet(List keys) throws IOException { }); } + public List prefixLookup(byte[] prefixKey) throws IOException { + return inReadLock( + kvLock, + () -> { + rocksDBKv.checkIfRocksDBClosed(); + return rocksDBKv.prefixLookup(prefixKey); + }); + } + public List limitScan(int limit) throws IOException { return inReadLock( kvLock, diff --git a/fluss-server/src/main/java/com/alibaba/fluss/server/kv/rocksdb/RocksDBKv.java b/fluss-server/src/main/java/com/alibaba/fluss/server/kv/rocksdb/RocksDBKv.java index aa71749e..2f3fb8c3 100644 --- a/fluss-server/src/main/java/com/alibaba/fluss/server/kv/rocksdb/RocksDBKv.java +++ b/fluss-server/src/main/java/com/alibaba/fluss/server/kv/rocksdb/RocksDBKv.java @@ -56,7 +56,7 @@ public class RocksDBKv implements AutoCloseable { * {@link RocksDB#open(String)} is different from that by {@link * RocksDB#getDefaultColumnFamily()}, probably it's a bug of RocksDB java API. */ - private final ColumnFamilyHandle defaultColumnFamily; + private final ColumnFamilyHandle defaultColumnFamilyHandle; /** Our RocksDB database. Currently, one kv tablet, one RocksDB instance. */ protected final RocksDB db; @@ -73,7 +73,7 @@ public RocksDBKv( this.db = db; this.rocksDBResourceGuard = rocksDBResourceGuard; this.writeOptions = optionsContainer.getWriteOptions(); - this.defaultColumnFamily = defaultColumnFamilyHandle; + this.defaultColumnFamilyHandle = defaultColumnFamilyHandle; } public ResourceGuard getResourceGuard() { @@ -100,17 +100,44 @@ public List multiGet(List keys) throws IOException { } } + public List prefixLookup(byte[] prefixKey) { + List pkList = new ArrayList<>(); + ReadOptions readOptions = new ReadOptions(); + RocksIterator iterator = db.newIterator(defaultColumnFamilyHandle, readOptions); + try { + iterator.seek(prefixKey); + // TODO, This is very inefficient to compare arrays byte by byte. In the future we can + // use JDK9 Arrays.compare(compare(byte[] a, int aFromIndex, int aToIndex, byte[] b, int + // bFromIndex, int bToIndex)) to instead. See issue: + // https://github.com/alibaba/fluss/issues/271 + while (iterator.isValid() && isPrefixEquals(prefixKey, iterator.key())) { + pkList.add(iterator.value()); + iterator.next(); + } + } finally { + readOptions.close(); + iterator.close(); + } + + return pkList; + } + public List limitScan(Integer limit) { List pkList = new ArrayList<>(); ReadOptions readOptions = new ReadOptions(); - RocksIterator iterator = db.newIterator(defaultColumnFamily, readOptions); + RocksIterator iterator = db.newIterator(defaultColumnFamilyHandle, readOptions); int count = 0; - iterator.seekToFirst(); - while (iterator.isValid() && count < limit) { - pkList.add(iterator.value()); - iterator.next(); - count++; + try { + iterator.seekToFirst(); + while (iterator.isValid() && count < limit) { + pkList.add(iterator.value()); + iterator.next(); + count++; + } + } finally { + readOptions.close(); + iterator.close(); } return pkList; @@ -165,8 +192,8 @@ public void close() throws Exception { // Start with default CF ... List columnFamilyOptions = new ArrayList<>(); RocksDBOperationUtils.addColumnFamilyOptionsToCloseLater( - columnFamilyOptions, defaultColumnFamily); - IOUtils.closeQuietly(defaultColumnFamily); + columnFamilyOptions, defaultColumnFamilyHandle); + IOUtils.closeQuietly(defaultColumnFamilyHandle); // ... and finally close the DB instance ... IOUtils.closeQuietly(db); @@ -181,4 +208,25 @@ public void close() throws Exception { public RocksDB getDb() { return db; } + + /** + * Check if the given two byte arrays have the same prefix. If bytes2 is shorter than bytes1, + * return false. Otherwise, compare bytes1 and bytes2 from the start until the end of bytes2. If + * all bytes are equal, return true. + * + * @param bytes1 The first byte array + * @param bytes2 The second byte array + * @return true if the given two byte arrays have the same prefix, false otherwise + */ + public static boolean isPrefixEquals(byte[] bytes1, byte[] bytes2) { + if (bytes1.length > bytes2.length) { + return false; + } + for (int i = 0; i < bytes1.length; i++) { + if (bytes1[i] != bytes2[i]) { + return false; + } + } + return true; + } } diff --git a/fluss-server/src/main/java/com/alibaba/fluss/server/metrics/group/PhysicalTableMetricGroup.java b/fluss-server/src/main/java/com/alibaba/fluss/server/metrics/group/PhysicalTableMetricGroup.java index 18b2ee6c..ec3ba43d 100644 --- a/fluss-server/src/main/java/com/alibaba/fluss/server/metrics/group/PhysicalTableMetricGroup.java +++ b/fluss-server/src/main/java/com/alibaba/fluss/server/metrics/group/PhysicalTableMetricGroup.java @@ -201,6 +201,22 @@ public Counter failedLimitScanRequests() { } } + public Counter totalPrefixLookupRequests() { + if (kvMetrics == null) { + return NoOpCounter.INSTANCE; + } else { + return kvMetrics.totalPrefixLookupRequests; + } + } + + public Counter failedPrefixLookupRequests() { + if (kvMetrics == null) { + return NoOpCounter.INSTANCE; + } else { + return kvMetrics.failedPrefixLookupRequests; + } + } + // ------------------------------------------------------------------------ // bucket groups // ------------------------------------------------------------------------ @@ -326,6 +342,8 @@ private static class KvMetricGroup extends TabletMetricGroup { private final Counter failedPutKvRequests; private final Counter totalLimitScanRequests; private final Counter failedLimitScanRequests; + private final Counter totalPrefixLookupRequests; + private final Counter failedPrefixLookupRequests; public KvMetricGroup(PhysicalTableMetricGroup physicalTableMetricGroup) { super(physicalTableMetricGroup, TabletType.KV); @@ -349,6 +367,16 @@ public KvMetricGroup(PhysicalTableMetricGroup physicalTableMetricGroup) { meter( MetricNames.FAILED_LIMIT_SCAN_REQUESTS_RATE, new MeterView(failedLimitScanRequests)); + + // for index lookup request + totalPrefixLookupRequests = new ThreadSafeSimpleCounter(); + meter( + MetricNames.TOTAL_PREFIX_LOOKUP_REQUESTS_RATE, + new MeterView(totalPrefixLookupRequests)); + failedPrefixLookupRequests = new ThreadSafeSimpleCounter(); + meter( + MetricNames.FAILED_PREFIX_LOOKUP_REQUESTS_RATE, + new MeterView(failedPrefixLookupRequests)); } @Override diff --git a/fluss-server/src/main/java/com/alibaba/fluss/server/replica/Replica.java b/fluss-server/src/main/java/com/alibaba/fluss/server/replica/Replica.java index 337b3f50..08c2aaa1 100644 --- a/fluss-server/src/main/java/com/alibaba/fluss/server/replica/Replica.java +++ b/fluss-server/src/main/java/com/alibaba/fluss/server/replica/Replica.java @@ -166,6 +166,7 @@ public final class Replica { private final KvFormat kvFormat; private final long logTTLMs; private final boolean dataLakeEnabled; + private final boolean supportPrefixLookup; private final int tieredLogLocalSegments; private final AtomicReference leaderReplicaIdOpt = new AtomicReference<>(); private final ReadWriteLock leaderIsrUpdateLock = new ReentrantReadWriteLock(); @@ -231,6 +232,7 @@ public Replica( this.snapshotContext = snapshotContext; // create a closeable registry for the replica this.closeableRegistry = new CloseableRegistry(); + this.supportPrefixLookup = supportPrefixLookup(tableDescriptor); this.logTablet = createLog(lazyHighWatermarkCheckpoint); registerMetrics(); @@ -307,6 +309,10 @@ public long getLogTTLMs() { return logTTLMs; } + public boolean supportPrefixLookup() { + return supportPrefixLookup; + } + public int writerIdCount() { return logTablet.getWriterIdCount(); } @@ -1028,6 +1034,36 @@ public List lookups(List keys) { }); } + public List prefixLookup(byte[] prefixKey) { + if (!isKvTable()) { + throw new NonPrimaryKeyTableException( + "Try to do prefix lookup on a non primary key table: " + getTablePath()); + } + + return inReadLock( + leaderIsrUpdateLock, + () -> { + try { + if (!isLeader()) { + throw new NotLeaderOrFollowerException( + String.format( + "Leader not local for bucket %s on tabletServer %d", + tableBucket, localTabletServerId)); + } + checkNotNull( + kvTablet, "KvTablet for the replica to get key shouldn't be null."); + return kvTablet.prefixLookup(prefixKey); + } catch (IOException e) { + String errorMsg = + String.format( + "Failed to do prefix lookup from local kv for table bucket %s, the cause is: %s", + tableBucket, e.getMessage()); + LOG.error(errorMsg, e); + throw new KvStorageException(errorMsg, e); + } + }); + } + public DefaultValueRecordBatch limitKvScan(int limit) { if (!isKvTable()) { throw new NonPrimaryKeyTableException( @@ -1699,6 +1735,11 @@ private void traceAckInfo(List curMaximalIsr, long requiredOffset) { .collect(Collectors.toList())); } + private boolean supportPrefixLookup(TableDescriptor tableDescriptor) { + return TableDescriptor.bucketKeysMatchPrefixLookupPattern( + tableDescriptor.getSchema(), tableDescriptor.getBucketKey()); + } + @VisibleForTesting public int getBucketEpoch() { return bucketEpoch; diff --git a/fluss-server/src/main/java/com/alibaba/fluss/server/replica/ReplicaManager.java b/fluss-server/src/main/java/com/alibaba/fluss/server/replica/ReplicaManager.java index d86bdfd8..6d31b703 100644 --- a/fluss-server/src/main/java/com/alibaba/fluss/server/replica/ReplicaManager.java +++ b/fluss-server/src/main/java/com/alibaba/fluss/server/replica/ReplicaManager.java @@ -24,6 +24,7 @@ import com.alibaba.fluss.exception.FlussRuntimeException; import com.alibaba.fluss.exception.InvalidCoordinatorException; import com.alibaba.fluss.exception.InvalidRequiredAcksException; +import com.alibaba.fluss.exception.KvStorageException; import com.alibaba.fluss.exception.LogOffsetOutOfRangeException; import com.alibaba.fluss.exception.LogStorageException; import com.alibaba.fluss.exception.NotLeaderOrFollowerException; @@ -46,6 +47,7 @@ import com.alibaba.fluss.rpc.entity.LimitScanResultForBucket; import com.alibaba.fluss.rpc.entity.ListOffsetsResultForBucket; import com.alibaba.fluss.rpc.entity.LookupResultForBucket; +import com.alibaba.fluss.rpc.entity.PrefixLookupResultForBucket; import com.alibaba.fluss.rpc.entity.ProduceLogResultForBucket; import com.alibaba.fluss.rpc.entity.PutKvResultForBucket; import com.alibaba.fluss.rpc.entity.WriteResultForBucket; @@ -409,7 +411,7 @@ public void putRecordsToKv( /** Lookup a single key value. */ @VisibleForTesting protected void lookup(TableBucket tableBucket, byte[] key, Consumer responseCallback) { - multiLookupValues( + lookups( Collections.singletonMap(tableBucket, Collections.singletonList(key)), multiLookupResponseCallBack -> { LookupResultForBucket result = multiLookupResponseCallBack.get(tableBucket); @@ -423,8 +425,8 @@ protected void lookup(TableBucket tableBucket, byte[] key, Consumer resp }); } - /** Multi-lookup from leader replica of the buckets. */ - public void multiLookupValues( + /** Lookup with multi key from leader replica of the buckets. */ + public void lookups( Map> entriesPerBucket, Consumer> responseCallback) { Map lookupResultForBucketMap = new HashMap<>(); @@ -456,6 +458,51 @@ public void multiLookupValues( responseCallback.accept(lookupResultForBucketMap); } + /** Lookup multi prefixKeys by prefix scan on kv store. */ + public void prefixLookups( + Map> entriesPerBucket, + Consumer> responseCallback) { + PhysicalTableMetricGroup tableMetrics = null; + Map result = new HashMap<>(); + for (Map.Entry> entry : entriesPerBucket.entrySet()) { + TableBucket tb = entry.getKey(); + List> resultForBucket = new ArrayList<>(); + try { + Replica replica = getReplicaOrException(tb); + if (!replica.supportPrefixLookup()) { + result.put( + tb, + new PrefixLookupResultForBucket( + tb, + ApiError.fromThrowable( + new KvStorageException( + "Table bucket " + + tb + + " does not support prefix lookup")))); + continue; + } + + tableMetrics = replica.tableMetrics(); + tableMetrics.totalPrefixLookupRequests().inc(); + for (byte[] prefixKey : entry.getValue()) { + List resultForPerKey = new ArrayList<>(replica.prefixLookup(prefixKey)); + resultForBucket.add(resultForPerKey); + } + } catch (Exception e) { + if (isUnexpectedException(e)) { + LOG.error("Error processing prefix lookup operation on replica {}", tb, e); + if (tableMetrics != null) { + tableMetrics.failedPrefixLookupRequests().inc(); + } + } + result.put(tb, new PrefixLookupResultForBucket(tb, ApiError.fromThrowable(e))); + } + + result.put(tb, new PrefixLookupResultForBucket(tb, resultForBucket)); + } + responseCallback.accept(result); + } + public void listOffsets( ListOffsetsParam listOffsetsParam, Set tableBuckets, diff --git a/fluss-server/src/main/java/com/alibaba/fluss/server/tablet/TabletService.java b/fluss-server/src/main/java/com/alibaba/fluss/server/tablet/TabletService.java index 085c80b7..cee76589 100644 --- a/fluss-server/src/main/java/com/alibaba/fluss/server/tablet/TabletService.java +++ b/fluss-server/src/main/java/com/alibaba/fluss/server/tablet/TabletService.java @@ -41,6 +41,8 @@ import com.alibaba.fluss.rpc.messages.NotifyLeaderAndIsrResponse; import com.alibaba.fluss.rpc.messages.NotifyRemoteLogOffsetsRequest; import com.alibaba.fluss.rpc.messages.NotifyRemoteLogOffsetsResponse; +import com.alibaba.fluss.rpc.messages.PrefixLookupRequest; +import com.alibaba.fluss.rpc.messages.PrefixLookupResponse; import com.alibaba.fluss.rpc.messages.ProduceLogRequest; import com.alibaba.fluss.rpc.messages.ProduceLogResponse; import com.alibaba.fluss.rpc.messages.PutKvRequest; @@ -62,7 +64,9 @@ import java.util.concurrent.CompletableFuture; import static com.alibaba.fluss.server.utils.RpcMessageUtils.makeLookupResponse; +import static com.alibaba.fluss.server.utils.RpcMessageUtils.makePrefixLookupResponse; import static com.alibaba.fluss.server.utils.RpcMessageUtils.toLookupData; +import static com.alibaba.fluss.server.utils.RpcMessageUtils.toPrefixLookupData; /** An RPC Gateway service for tablet server. */ public final class TabletService extends RpcServiceBase implements TabletServerGateway { @@ -137,8 +141,16 @@ public CompletableFuture putKv(PutKvRequest request) { public CompletableFuture lookup(LookupRequest request) { CompletableFuture response = new CompletableFuture<>(); Map> lookupData = toLookupData(request); - replicaManager.multiLookupValues( - lookupData, value -> response.complete(makeLookupResponse(value))); + replicaManager.lookups(lookupData, value -> response.complete(makeLookupResponse(value))); + return response; + } + + @Override + public CompletableFuture prefixLookup(PrefixLookupRequest request) { + CompletableFuture response = new CompletableFuture<>(); + replicaManager.prefixLookups( + toPrefixLookupData(request), + value -> response.complete(makePrefixLookupResponse(value))); return response; } diff --git a/fluss-server/src/main/java/com/alibaba/fluss/server/utils/RpcMessageUtils.java b/fluss-server/src/main/java/com/alibaba/fluss/server/utils/RpcMessageUtils.java index 69d626a5..6a96f963 100644 --- a/fluss-server/src/main/java/com/alibaba/fluss/server/utils/RpcMessageUtils.java +++ b/fluss-server/src/main/java/com/alibaba/fluss/server/utils/RpcMessageUtils.java @@ -38,6 +38,7 @@ import com.alibaba.fluss.rpc.entity.LimitScanResultForBucket; import com.alibaba.fluss.rpc.entity.ListOffsetsResultForBucket; import com.alibaba.fluss.rpc.entity.LookupResultForBucket; +import com.alibaba.fluss.rpc.entity.PrefixLookupResultForBucket; import com.alibaba.fluss.rpc.entity.ProduceLogResultForBucket; import com.alibaba.fluss.rpc.entity.PutKvResultForBucket; import com.alibaba.fluss.rpc.messages.AdjustIsrRequest; @@ -84,6 +85,8 @@ import com.alibaba.fluss.rpc.messages.PbNotifyLeaderAndIsrReqForBucket; import com.alibaba.fluss.rpc.messages.PbNotifyLeaderAndIsrRespForBucket; import com.alibaba.fluss.rpc.messages.PbPhysicalTablePath; +import com.alibaba.fluss.rpc.messages.PbPrefixLookupReqForBucket; +import com.alibaba.fluss.rpc.messages.PbPrefixLookupRespForBucket; import com.alibaba.fluss.rpc.messages.PbProduceLogReqForBucket; import com.alibaba.fluss.rpc.messages.PbProduceLogRespForBucket; import com.alibaba.fluss.rpc.messages.PbPutKvReqForBucket; @@ -97,6 +100,9 @@ import com.alibaba.fluss.rpc.messages.PbTableBucket; import com.alibaba.fluss.rpc.messages.PbTablePath; import com.alibaba.fluss.rpc.messages.PbValue; +import com.alibaba.fluss.rpc.messages.PbValueList; +import com.alibaba.fluss.rpc.messages.PrefixLookupRequest; +import com.alibaba.fluss.rpc.messages.PrefixLookupResponse; import com.alibaba.fluss.rpc.messages.ProduceLogRequest; import com.alibaba.fluss.rpc.messages.ProduceLogResponse; import com.alibaba.fluss.rpc.messages.PutKvRequest; @@ -602,6 +608,28 @@ public static Map> toLookupData(LookupRequest lookupRe return lookupEntryData; } + public static Map> toPrefixLookupData( + PrefixLookupRequest prefixLookupRequest) { + long tableId = prefixLookupRequest.getTableId(); + Map> lookupEntryData = new HashMap<>(); + for (PbPrefixLookupReqForBucket lookupReqForBucket : + prefixLookupRequest.getBucketsReqsList()) { + TableBucket tb = + new TableBucket( + tableId, + lookupReqForBucket.hasPartitionId() + ? lookupReqForBucket.getPartitionId() + : null, + lookupReqForBucket.getBucketId()); + List keys = new ArrayList<>(lookupReqForBucket.getKeysCount()); + for (int i = 0; i < lookupReqForBucket.getKeysCount(); i++) { + keys.add(lookupReqForBucket.getKeyAt(i)); + } + lookupEntryData.put(tb, keys); + } + return lookupEntryData; + } + public static @Nullable int[] getTargetColumns(PutKvRequest putKvRequest) { int[] targetColumns = putKvRequest.getTargetColumns(); return targetColumns.length == 0 ? null : targetColumns; @@ -706,6 +734,39 @@ public static LookupResponse makeLookupResponse( return lookupResponse; } + public static PrefixLookupResponse makePrefixLookupResponse( + Map prefixLookupResult) { + PrefixLookupResponse prefixLookupResponse = new PrefixLookupResponse(); + List resultForAll = new ArrayList<>(); + for (Map.Entry entry : + prefixLookupResult.entrySet()) { + PbPrefixLookupRespForBucket respForBucket = new PbPrefixLookupRespForBucket(); + TableBucket tb = entry.getKey(); + respForBucket.setBucketId(tb.getBucket()); + if (tb.getPartitionId() != null) { + respForBucket.setPartitionId(tb.getPartitionId()); + } + + PrefixLookupResultForBucket bucketResult = entry.getValue(); + if (bucketResult.failed()) { + respForBucket.setError(bucketResult.getErrorCode(), bucketResult.getErrorMessage()); + } else { + List keyResultList = new ArrayList<>(); + for (List res : bucketResult.prefixLookupValues()) { + PbValueList pbValueList = new PbValueList(); + for (byte[] bytes : res) { + pbValueList.addValue(bytes); + } + keyResultList.add(pbValueList); + } + respForBucket.addAllValueLists(keyResultList); + } + resultForAll.add(respForBucket); + } + prefixLookupResponse.addAllBucketsResps(resultForAll); + return prefixLookupResponse; + } + public static AdjustIsrRequest makeAdjustIsrRequest( int serverId, Map leaderAndIsrMap) { // group by table id. diff --git a/fluss-server/src/test/java/com/alibaba/fluss/server/log/remote/RemoteLogManagerTest.java b/fluss-server/src/test/java/com/alibaba/fluss/server/log/remote/RemoteLogManagerTest.java index 168d9e6e..ff9685bd 100644 --- a/fluss-server/src/test/java/com/alibaba/fluss/server/log/remote/RemoteLogManagerTest.java +++ b/fluss-server/src/test/java/com/alibaba/fluss/server/log/remote/RemoteLogManagerTest.java @@ -16,6 +16,7 @@ package com.alibaba.fluss.server.log.remote; +import com.alibaba.fluss.config.ConfigOptions; import com.alibaba.fluss.fs.FsPath; import com.alibaba.fluss.metadata.TableBucket; import com.alibaba.fluss.remote.RemoteLogFetchInfo; @@ -39,7 +40,10 @@ import java.util.stream.Collectors; import static com.alibaba.fluss.record.TestData.DATA1_PHYSICAL_TABLE_PATH; +import static com.alibaba.fluss.record.TestData.DATA1_SCHEMA; import static com.alibaba.fluss.record.TestData.DATA1_TABLE_ID; +import static com.alibaba.fluss.record.TestData.DATA1_TABLE_PATH; +import static com.alibaba.fluss.record.TestData.DATA1_TABLE_PATH_PK; import static com.alibaba.fluss.server.zk.data.LeaderAndIsr.INITIAL_LEADER_EPOCH; import static com.alibaba.fluss.utils.FlussPaths.remoteLogDir; import static com.alibaba.fluss.utils.FlussPaths.remoteLogTabletDir; @@ -350,7 +354,7 @@ void testFetchRecordsFromRemote(boolean partitionTable) throws Exception { void testCleanupLocalSegments(boolean partitionTable) throws Exception { TableBucket tb = makeTableBucket(partitionTable); // Need to make leader by ReplicaManager. - makeKvTableAsLeader(tb, INITIAL_LEADER_EPOCH, partitionTable); + makeKvTableAsLeader(tb, DATA1_TABLE_PATH_PK, INITIAL_LEADER_EPOCH, partitionTable); LogTablet logTablet = replicaManager.getReplicaOrException(tb).getLogTablet(); // 2. generate 5 segments and trigger upload 4 to remote storage @@ -412,7 +416,15 @@ void testCleanupLocalSegments(boolean partitionTable) throws Exception { @ValueSource(booleans = {true, false}) void testConfigureTieredLogLocalSegments(boolean partitionedTable) throws Exception { int tieredLogLocalSegments = 8; - long tableId = registerTableInZkClient(tieredLogLocalSegments); + long tableId = + registerTableInZkClient( + DATA1_TABLE_PATH, + DATA1_SCHEMA, + 200L, + Collections.emptyList(), + Collections.singletonMap( + ConfigOptions.TABLE_TIERED_LOG_LOCAL_SEGMENTS.key(), + String.valueOf(tieredLogLocalSegments))); TableBucket tb = makeTableBucket(tableId, partitionedTable); // make leader, and then remote log tablet should be created. diff --git a/fluss-server/src/test/java/com/alibaba/fluss/server/replica/ReplicaManagerTest.java b/fluss-server/src/test/java/com/alibaba/fluss/server/replica/ReplicaManagerTest.java index ac0c81a1..b60f8845 100644 --- a/fluss-server/src/test/java/com/alibaba/fluss/server/replica/ReplicaManagerTest.java +++ b/fluss-server/src/test/java/com/alibaba/fluss/server/replica/ReplicaManagerTest.java @@ -18,7 +18,9 @@ import com.alibaba.fluss.exception.InvalidRequiredAcksException; import com.alibaba.fluss.metadata.PhysicalTablePath; +import com.alibaba.fluss.metadata.Schema; import com.alibaba.fluss.metadata.TableBucket; +import com.alibaba.fluss.metadata.TablePath; import com.alibaba.fluss.record.DefaultValueRecordBatch; import com.alibaba.fluss.record.KvRecord; import com.alibaba.fluss.record.KvRecordBatch; @@ -32,6 +34,7 @@ import com.alibaba.fluss.rpc.entity.LimitScanResultForBucket; import com.alibaba.fluss.rpc.entity.ListOffsetsResultForBucket; import com.alibaba.fluss.rpc.entity.LookupResultForBucket; +import com.alibaba.fluss.rpc.entity.PrefixLookupResultForBucket; import com.alibaba.fluss.rpc.entity.ProduceLogResultForBucket; import com.alibaba.fluss.rpc.entity.PutKvResultForBucket; import com.alibaba.fluss.rpc.protocol.ApiError; @@ -47,6 +50,9 @@ import com.alibaba.fluss.server.testutils.KvTestUtils; import com.alibaba.fluss.server.zk.data.LeaderAndIsr; import com.alibaba.fluss.testutils.DataTestUtils; +import com.alibaba.fluss.types.DataField; +import com.alibaba.fluss.types.DataTypes; +import com.alibaba.fluss.types.RowType; import com.alibaba.fluss.utils.types.Tuple2; import org.junit.jupiter.api.Test; @@ -68,9 +74,11 @@ import static com.alibaba.fluss.record.TestData.DATA1; import static com.alibaba.fluss.record.TestData.DATA1_KEY_TYPE; import static com.alibaba.fluss.record.TestData.DATA1_ROW_TYPE; +import static com.alibaba.fluss.record.TestData.DATA1_SCHEMA_PK; import static com.alibaba.fluss.record.TestData.DATA1_TABLE_ID; import static com.alibaba.fluss.record.TestData.DATA1_TABLE_ID_PK; import static com.alibaba.fluss.record.TestData.DATA1_TABLE_PATH; +import static com.alibaba.fluss.record.TestData.DATA1_TABLE_PATH_PK; import static com.alibaba.fluss.record.TestData.DATA_1_WITH_KEY_AND_VALUE; import static com.alibaba.fluss.record.TestData.DEFAULT_SCHEMA_ID; import static com.alibaba.fluss.server.coordinator.CoordinatorContext.INITIAL_COORDINATOR_EPOCH; @@ -377,7 +385,7 @@ void testFetchLogWithMaxBytesLimitForMultiTableBucket() throws Exception { @Test void testPutKv() throws Exception { TableBucket tb = new TableBucket(DATA1_TABLE_ID_PK, 1); - makeKvTableAsLeader(tb.getBucket()); + makeKvTableAsLeader(DATA1_TABLE_ID_PK, DATA1_TABLE_PATH_PK, tb.getBucket()); // put kv records to kv store. CompletableFuture> future = new CompletableFuture<>(); @@ -425,7 +433,7 @@ tb, genKvRecordBatch(DATA_1_WITH_KEY_AND_VALUE)), @Test void testPutKvWithOutOfBatchSequence() throws Exception { TableBucket tb = new TableBucket(DATA1_TABLE_ID_PK, 1); - makeKvTableAsLeader(tb.getBucket()); + makeKvTableAsLeader(DATA1_TABLE_ID_PK, DATA1_TABLE_PATH_PK, tb.getBucket()); // 1. put kv records to kv store. List> data1 = @@ -438,7 +446,10 @@ void testPutKvWithOutOfBatchSequence() throws Exception { replicaManager.putRecordsToKv( 20000, 1, - Collections.singletonMap(tb, genKvRecordBatchWithWriterId(data1, 100L, 0)), + Collections.singletonMap( + tb, + genKvRecordBatchWithWriterId( + data1, DATA1_KEY_TYPE, DATA1_ROW_TYPE, 100L, 0)), null, future::complete); assertThat(future.get()).containsOnly(new PutKvResultForBucket(tb, 5)); @@ -475,7 +486,10 @@ void testPutKvWithOutOfBatchSequence() throws Exception { replicaManager.putRecordsToKv( 20000, 1, - Collections.singletonMap(tb, genKvRecordBatchWithWriterId(data2, 100L, 3)), + Collections.singletonMap( + tb, + genKvRecordBatchWithWriterId( + data2, DATA1_KEY_TYPE, DATA1_ROW_TYPE, 100L, 3)), null, future::complete); PutKvResultForBucket putKvResultForBucket = future.get().get(0); @@ -508,7 +522,10 @@ void testPutKvWithOutOfBatchSequence() throws Exception { replicaManager.putRecordsToKv( 20000, 1, - Collections.singletonMap(tb, genKvRecordBatchWithWriterId(data3, 100L, 1)), + Collections.singletonMap( + tb, + genKvRecordBatchWithWriterId( + data3, DATA1_KEY_TYPE, DATA1_ROW_TYPE, 100L, 1)), null, future::complete); assertThat(future.get()).containsOnly(new PutKvResultForBucket(tb, 8)); @@ -534,7 +551,7 @@ void testPutKvWithOutOfBatchSequence() throws Exception { @Test void testLookup() throws Exception { TableBucket tb = new TableBucket(DATA1_TABLE_ID_PK, 1); - makeKvTableAsLeader(tb.getBucket()); + makeKvTableAsLeader(DATA1_TABLE_ID_PK, DATA1_TABLE_PATH_PK, tb.getBucket()); // first lookup key without in table, key = 1. Object[] key1 = DATA_1_WITH_KEY_AND_VALUE.get(0).f0; @@ -563,10 +580,10 @@ void testLookup() throws Exception { byte[] key3Bytes = keyEncoder.encode(row(DATA1_KEY_TYPE, key3)); verifyLookup(tb, key3Bytes, null); - // Get key from none pk table. + // Lookup from none pk table. TableBucket tb2 = new TableBucket(DATA1_TABLE_ID, 1); makeLogTableAsLeader(tb2.getBucket()); - replicaManager.multiLookupValues( + replicaManager.lookups( Collections.singletonMap(tb2, Collections.singletonList(key1Bytes)), (lookupResultForBuckets) -> { LookupResultForBucket lookupResultForBucket = lookupResultForBuckets.get(tb2); @@ -578,10 +595,113 @@ void testLookup() throws Exception { }); } + @Test + void testPrefixLookup() throws Exception { + TablePath tablePath = TablePath.of("test_db_1", "test_prefix_lookup_t1"); + Schema schema = + Schema.newBuilder() + .column("a", DataTypes.INT()) + .column("b", DataTypes.STRING()) + .column("c", DataTypes.BIGINT()) + .column("d", DataTypes.STRING()) + .primaryKey("a", "b", "c") + .build(); + RowType rowType = schema.toRowType(); + RowType keyType = + DataTypes.ROW( + new DataField("a", DataTypes.INT()), + new DataField("b", DataTypes.STRING()), + new DataField("c", DataTypes.BIGINT())); + RowType prefixKeyType = + DataTypes.ROW( + new DataField("a", DataTypes.INT()), + new DataField("b", DataTypes.STRING())); + long tableId = + registerTableInZkClient( + tablePath, + schema, + 1998232L, + Arrays.asList("a", "b"), // bucket keys equals prefix keys. + Collections.emptyMap()); + TableBucket tb = new TableBucket(tableId, 0); + makeKvTableAsLeader(tableId, tablePath, tb.getBucket()); + + List> data1 = + Arrays.asList( + Tuple2.of(new Object[] {1, "a", 1L}, new Object[] {1, "a", 1L, "value1"}), + Tuple2.of(new Object[] {1, "a", 2L}, new Object[] {1, "a", 2L, "value2"}), + Tuple2.of(new Object[] {1, "a", 3L}, new Object[] {1, "a", 3L, "value3"}), + Tuple2.of(new Object[] {2, "a", 4L}, new Object[] {2, "a", 4L, "value4"})); + // send one batch kv. + CompletableFuture> future = new CompletableFuture<>(); + replicaManager.putRecordsToKv( + 20000, + 1, + Collections.singletonMap(tb, genKvRecordBatch(keyType, rowType, data1)), + null, + future::complete); + assertThat(future.get()).containsOnly(new PutKvResultForBucket(tb, 4)); + // second prefix lookup in table, prefix key = (1, "a"). + Object[] prefixKey1 = new Object[] {1, "a"}; + KeyEncoder keyEncoder = new KeyEncoder(rowType, new int[] {0, 1}); + byte[] prefixKey1Bytes = keyEncoder.encode(row(prefixKeyType, prefixKey1)); + List key1ExpectedValues = + Arrays.asList( + ValueEncoder.encodeValue( + DEFAULT_SCHEMA_ID, + compactedRow(rowType, new Object[] {1, "a", 1L, "value1"})), + ValueEncoder.encodeValue( + DEFAULT_SCHEMA_ID, + compactedRow(rowType, new Object[] {1, "a", 2L, "value2"})), + ValueEncoder.encodeValue( + DEFAULT_SCHEMA_ID, + compactedRow(rowType, new Object[] {1, "a", 3L, "value3"}))); + verifyPrefixLookup( + tb, + Collections.singletonList(prefixKey1Bytes), + Collections.singletonList(key1ExpectedValues)); + + // third prefix lookup in table for multi prefix keys, prefix key = (1, "a") and (2, "a"). + Object[] prefixKey2 = new Object[] {2, "a"}; + byte[] prefixKey2Bytes = keyEncoder.encode(row(prefixKeyType, prefixKey2)); + List key2ExpectedValues = + Collections.singletonList( + ValueEncoder.encodeValue( + DEFAULT_SCHEMA_ID, + compactedRow(rowType, new Object[] {2, "a", 4L, "value4"}))); + verifyPrefixLookup( + tb, + Arrays.asList(prefixKey1Bytes, prefixKey2Bytes), + Arrays.asList(key1ExpectedValues, key2ExpectedValues)); + + // Prefix lookup an unsupported prefixLookup table. + tablePath = TablePath.of("test_db_1", "test_unsupported_prefix_lookup_t1"); + tableId = + registerTableInZkClient( + tablePath, + DATA1_SCHEMA_PK, + 200L, + Collections.emptyList(), + Collections.emptyMap()); + TableBucket tb3 = new TableBucket(tableId, 0); + makeKvTableAsLeader(tableId, tablePath, tb3.getBucket()); + replicaManager.prefixLookups( + Collections.singletonMap(tb3, Collections.singletonList(prefixKey2Bytes)), + (prefixLookupResultForBuckets) -> { + PrefixLookupResultForBucket lookupResultForBucket = + prefixLookupResultForBuckets.get(tb3); + assertThat(lookupResultForBucket.failed()).isTrue(); + ApiError apiError = lookupResultForBucket.getError(); + assertThat(apiError.error()).isEqualTo(Errors.KV_STORAGE_EXCEPTION); + assertThat(apiError.message()) + .isEqualTo("Table bucket " + tb3 + " does not support prefix lookup"); + }); + } + @Test void testLimitScanPrimaryKeyTable() throws Exception { TableBucket tb = new TableBucket(DATA1_TABLE_ID_PK, 1); - makeKvTableAsLeader(tb.getBucket()); + makeKvTableAsLeader(DATA1_TABLE_ID_PK, DATA1_TABLE_PATH_PK, tb.getBucket()); DefaultValueRecordBatch.Builder builder = DefaultValueRecordBatch.builder(); // first limit scan from an empty table. @@ -797,7 +917,7 @@ void testCompleteDelayProduceLog() throws Exception { @Test void testCompleteDelayPutKv() throws Exception { TableBucket tb = new TableBucket(DATA1_TABLE_ID_PK, 1); - makeKvTableAsLeader(tb.getBucket()); + makeKvTableAsLeader(DATA1_TABLE_ID_PK, DATA1_TABLE_PATH_PK, tb.getBucket()); // put kv records to kv store as ack = -1, which will generate delayed write operation. CompletableFuture> future = new CompletableFuture<>(); @@ -943,7 +1063,7 @@ void testSnapshotKvReplicas() throws Exception { // create multiple kv replicas and all do the snapshot operation int nBuckets = 5; List tableBuckets = createTableBuckets(nBuckets); - makeKvTableAsLeader(tableBuckets, 0); + makeKvTableAsLeader(tableBuckets, DATA1_TABLE_PATH_PK, 0); Map entriesPerBucket = new HashMap<>(); for (int i = 0; i < tableBuckets.size(); i++) { TableBucket tableBucket = tableBuckets.get(i); @@ -1022,7 +1142,7 @@ void testKvRestore() throws Exception { // create multiple kv replicas and all do the snapshot operation int nBuckets = 3; List tableBuckets = createTableBuckets(nBuckets); - makeKvTableAsLeader(tableBuckets, 0); + makeKvTableAsLeader(tableBuckets, DATA1_TABLE_PATH_PK, 0); Map> kvRecordsPerBucket = new HashMap<>(); Map kvRecordBatchPerBucket = new HashMap<>(); @@ -1037,7 +1157,7 @@ void testKvRestore() throws Exception { putRecords(kvRecordBatchPerBucket); // make all become leader with a bigger leader epoch, which will cause restore - makeKvTableAsLeader(tableBuckets, 1); + makeKvTableAsLeader(tableBuckets, DATA1_TABLE_PATH_PK, 1); // let's check the result after restore checkKvDataForBuckets(kvRecordsPerBucket); @@ -1060,7 +1180,7 @@ void testKvRestore() throws Exception { checkKvDataForBuckets(kvRecordsPerBucket); // restore and check data again - makeKvTableAsLeader(tableBuckets, 2); + makeKvTableAsLeader(tableBuckets, DATA1_TABLE_PATH_PK, 2); checkKvDataForBuckets(kvRecordsPerBucket); // fetch log, make sure the log is correct after restore @@ -1097,9 +1217,10 @@ private List createTableBuckets(int nBuckets) { return tableBuckets; } - private void makeKvTableAsLeader(List tableBuckets, int leaderEpoch) { + private void makeKvTableAsLeader( + List tableBuckets, TablePath tablePath, int leaderEpoch) { for (TableBucket tableBucket : tableBuckets) { - makeKvTableAsLeader(tableBucket, leaderEpoch, false); + makeKvTableAsLeader(tableBucket, tablePath, leaderEpoch, false); } } @@ -1151,4 +1272,29 @@ private void verifyLookup(TableBucket tb, byte[] keyBytes, @Nullable byte[] expe byte[] lookupValues = future.get(); assertThat(lookupValues).isEqualTo(expectValues); } + + private void verifyPrefixLookup( + TableBucket tb, List prefixKeyBytes, List> expectedValues) + throws Exception { + Map> entriesPerBucket = new HashMap<>(); + entriesPerBucket.put(tb, prefixKeyBytes); + + CompletableFuture> future = + new CompletableFuture<>(); + replicaManager.prefixLookups(entriesPerBucket, future::complete); + Map prefixResult = future.get(); + assertThat(prefixResult.size()).isEqualTo(1); + PrefixLookupResultForBucket resultForBucket = prefixResult.get(tb); + assertThat(resultForBucket).isNotNull(); + List> prefixLookupValues = resultForBucket.prefixLookupValues(); + assertThat(prefixLookupValues.size()).isEqualTo(expectedValues.size()); + for (int i = 0; i < expectedValues.size(); i++) { + List prefixValueList = prefixLookupValues.get(i); + List expectedValueList = expectedValues.get(i); + assertThat(prefixValueList.size()).isEqualTo(expectedValueList.size()); + for (int j = 0; j < expectedValueList.size(); j++) { + assertThat(prefixValueList.get(j)).isEqualTo(expectedValueList.get(j)); + } + } + } } diff --git a/fluss-server/src/test/java/com/alibaba/fluss/server/replica/ReplicaTestBase.java b/fluss-server/src/test/java/com/alibaba/fluss/server/replica/ReplicaTestBase.java index bbb78319..2db85ff7 100644 --- a/fluss-server/src/test/java/com/alibaba/fluss/server/replica/ReplicaTestBase.java +++ b/fluss-server/src/test/java/com/alibaba/fluss/server/replica/ReplicaTestBase.java @@ -23,8 +23,10 @@ import com.alibaba.fluss.config.MemorySize; import com.alibaba.fluss.fs.FsPath; import com.alibaba.fluss.metadata.PhysicalTablePath; +import com.alibaba.fluss.metadata.Schema; import com.alibaba.fluss.metadata.TableBucket; import com.alibaba.fluss.metadata.TableDescriptor; +import com.alibaba.fluss.metadata.TablePath; import com.alibaba.fluss.record.MemoryLogRecords; import com.alibaba.fluss.rpc.RpcClient; import com.alibaba.fluss.rpc.metrics.TestingClientMetricGroup; @@ -75,6 +77,7 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.concurrent.ExecutorService; @@ -86,7 +89,6 @@ import static com.alibaba.fluss.record.TestData.DATA1; import static com.alibaba.fluss.record.TestData.DATA1_PHYSICAL_TABLE_PATH; import static com.alibaba.fluss.record.TestData.DATA1_PHYSICAL_TABLE_PATH_PA_2024; -import static com.alibaba.fluss.record.TestData.DATA1_PHYSICAL_TABLE_PATH_PK; import static com.alibaba.fluss.record.TestData.DATA1_PHYSICAL_TABLE_PATH_PK_PA_2024; import static com.alibaba.fluss.record.TestData.DATA1_SCHEMA; import static com.alibaba.fluss.record.TestData.DATA1_SCHEMA_PK; @@ -229,22 +231,23 @@ private void registerTableInZkClient() throws Exception { zkClient.registerSchema(DATA2_TABLE_PATH, DATA2_SCHEMA); } - protected long registerTableInZkClient(int tieredLogLocalSegment) throws Exception { - long tableId = 200; - TableDescriptor tableDescriptor = - TableDescriptor.builder() - .schema(DATA1_SCHEMA) - .distributedBy(3) - .property( - ConfigOptions.TABLE_TIERED_LOG_LOCAL_SEGMENTS, - tieredLogLocalSegment) - .build(); + protected long registerTableInZkClient( + TablePath tablePath, + Schema schema, + long tableId, + List bucketKeys, + Map properties) + throws Exception { + TableDescriptor.Builder builder = + TableDescriptor.builder().schema(schema).distributedBy(3, bucketKeys); + properties.forEach(builder::property); + TableDescriptor tableDescriptor = builder.build(); // if exists, drop it firstly - if (zkClient.tableExist(DATA1_TABLE_PATH)) { - zkClient.deleteTable(DATA1_TABLE_PATH); + if (zkClient.tableExist(tablePath)) { + zkClient.deleteTable(tablePath); } - zkClient.registerTable(DATA1_TABLE_PATH, TableRegistration.of(tableId, tableDescriptor)); - zkClient.registerSchema(DATA1_TABLE_PATH, DATA1_SCHEMA); + zkClient.registerTable(tablePath, TableRegistration.of(tableId, tableDescriptor)); + zkClient.registerSchema(tablePath, schema); return tableId; } @@ -332,14 +335,16 @@ protected void makeLogTableAsLeader( // TODO this is only for single tablet server unit test. // TODO add more test cases for partition table which make leader by this method. - protected void makeKvTableAsLeader(int bucketId) { + protected void makeKvTableAsLeader(long tableId, TablePath tablePath, int bucketId) { makeKvTableAsLeader( - new TableBucket(DATA1_TABLE_ID_PK, bucketId), INITIAL_LEADER_EPOCH, false); + new TableBucket(tableId, bucketId), tablePath, INITIAL_LEADER_EPOCH, false); } - protected void makeKvTableAsLeader(TableBucket tb, int leaderEpoch, boolean partitionTable) { + protected void makeKvTableAsLeader( + TableBucket tb, TablePath tablePath, int leaderEpoch, boolean partitionTable) { makeKvTableAsLeader( tb, + tablePath, Collections.singletonList(TABLET_SERVER_ID), Collections.singletonList(TABLET_SERVER_ID), leaderEpoch, @@ -348,6 +353,7 @@ protected void makeKvTableAsLeader(TableBucket tb, int leaderEpoch, boolean part protected void makeKvTableAsLeader( TableBucket tb, + TablePath tablePath, List replicas, List isr, int leaderEpoch, @@ -357,7 +363,7 @@ protected void makeKvTableAsLeader( new NotifyLeaderAndIsrData( partitionTable ? DATA1_PHYSICAL_TABLE_PATH_PK_PA_2024 - : DATA1_PHYSICAL_TABLE_PATH_PK, + : PhysicalTablePath.of(tablePath), tb, replicas, new LeaderAndIsr( diff --git a/fluss-server/src/test/java/com/alibaba/fluss/server/tablet/TabletServiceITCase.java b/fluss-server/src/test/java/com/alibaba/fluss/server/tablet/TabletServiceITCase.java index af039c92..717a884e 100644 --- a/fluss-server/src/test/java/com/alibaba/fluss/server/tablet/TabletServiceITCase.java +++ b/fluss-server/src/test/java/com/alibaba/fluss/server/tablet/TabletServiceITCase.java @@ -18,6 +18,7 @@ import com.alibaba.fluss.exception.InvalidRequiredAcksException; import com.alibaba.fluss.metadata.LogFormat; +import com.alibaba.fluss.metadata.Schema; import com.alibaba.fluss.metadata.TableBucket; import com.alibaba.fluss.metadata.TableDescriptor; import com.alibaba.fluss.metadata.TablePath; @@ -31,11 +32,16 @@ import com.alibaba.fluss.rpc.messages.ListOffsetsResponse; import com.alibaba.fluss.rpc.messages.PbListOffsetsRespForBucket; import com.alibaba.fluss.rpc.messages.PbLookupRespForBucket; +import com.alibaba.fluss.rpc.messages.PbPrefixLookupRespForBucket; import com.alibaba.fluss.rpc.messages.PbPutKvRespForBucket; import com.alibaba.fluss.rpc.messages.PutKvResponse; import com.alibaba.fluss.rpc.protocol.Errors; import com.alibaba.fluss.server.log.ListOffsetsParam; import com.alibaba.fluss.server.testutils.FlussClusterExtension; +import com.alibaba.fluss.types.DataField; +import com.alibaba.fluss.types.DataTypes; +import com.alibaba.fluss.types.RowType; +import com.alibaba.fluss.utils.types.Tuple2; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; @@ -43,6 +49,8 @@ import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; import java.util.List; import static com.alibaba.fluss.record.TestData.ANOTHER_DATA1; @@ -57,6 +65,7 @@ import static com.alibaba.fluss.record.TestData.DATA_1_WITH_KEY_AND_VALUE; import static com.alibaba.fluss.record.TestData.DEFAULT_SCHEMA_ID; import static com.alibaba.fluss.server.testutils.KvTestUtils.assertLookupResponse; +import static com.alibaba.fluss.server.testutils.KvTestUtils.assertPrefixLookupResponse; import static com.alibaba.fluss.server.testutils.RpcMessageTestUtils.assertFetchLogResponse; import static com.alibaba.fluss.server.testutils.RpcMessageTestUtils.assertLimitScanResponse; import static com.alibaba.fluss.server.testutils.RpcMessageTestUtils.assertProduceLogResponse; @@ -65,6 +74,7 @@ import static com.alibaba.fluss.server.testutils.RpcMessageTestUtils.newLimitScanRequest; import static com.alibaba.fluss.server.testutils.RpcMessageTestUtils.newListOffsetsRequest; import static com.alibaba.fluss.server.testutils.RpcMessageTestUtils.newLookupRequest; +import static com.alibaba.fluss.server.testutils.RpcMessageTestUtils.newPrefixLookupRequest; import static com.alibaba.fluss.server.testutils.RpcMessageTestUtils.newProduceLogRequest; import static com.alibaba.fluss.server.testutils.RpcMessageTestUtils.newPutKvRequest; import static com.alibaba.fluss.testutils.DataTestUtils.compactedRow; @@ -305,7 +315,7 @@ void testPutKv() throws Exception { } @Test - void testGetKey() throws Exception { + void testLookup() throws Exception { long tableId = createTable( FLUSS_CLUSTER_EXTENSION, @@ -319,7 +329,7 @@ void testGetKey() throws Exception { TabletServerGateway leaderGateWay = FLUSS_CLUSTER_EXTENSION.newTabletServerClientForNode(leader); - // first get key without in table, key = 1. + // first lookup without in table, key = 1. Object[] key1 = DATA_1_WITH_KEY_AND_VALUE.get(0).f0; KeyEncoder keyEncoder = new KeyEncoder(DATA1_ROW_TYPE, new int[] {0}); byte[] key1Bytes = keyEncoder.encode(row(DATA1_KEY_TYPE, key1)); @@ -334,7 +344,7 @@ void testGetKey() throws Exception { tableId, 0, 1, genKvRecordBatch(DATA_1_WITH_KEY_AND_VALUE))) .get()); - // second get key in table, key = 1, value = 1, "a1". + // second lookup in table, key = 1, value = 1, "a1". Object[] value1 = DATA_1_WITH_KEY_AND_VALUE.get(3).f1; byte[] value1Bytes = ValueEncoder.encodeValue(DEFAULT_SCHEMA_ID, compactedRow(DATA1_ROW_TYPE, value1)); @@ -354,12 +364,12 @@ void testGetKey() throws Exception { .get() .getBucketsRespAt(0); - verifyError( + verifyLookupBucketError( pbLookupRespForBucket, Errors.UNKNOWN_TABLE_OR_BUCKET_EXCEPTION, "Unknown table or bucket: TableBucket{tableId=10005, bucket=6}"); - // Get key from a non-pk table. + // Lookup from a non-pk table. long logTableId = createTable( FLUSS_CLUSTER_EXTENSION, @@ -377,12 +387,138 @@ void testGetKey() throws Exception { .lookup(newLookupRequest(logTableId, 0, key3Bytes)) .get() .getBucketsRespAt(0); - verifyError( + verifyLookupBucketError( pbLookupRespForBucket, Errors.NON_PRIMARY_KEY_TABLE_EXCEPTION, "the primary key table not exists for TableBucket"); } + @Test + void testPrefixLookup() throws Exception { + TablePath tablePath = TablePath.of("test_db_1", "test_prefix_lookup_t1"); + Schema schema = + Schema.newBuilder() + .column("a", DataTypes.INT()) + .column("b", DataTypes.STRING()) + .column("c", DataTypes.BIGINT()) + .column("d", DataTypes.STRING()) + .primaryKey("a", "b", "c") + .build(); + RowType rowType = schema.toRowType(); + RowType primaryKeyType = + DataTypes.ROW( + new DataField("a", DataTypes.INT()), + new DataField("b", DataTypes.STRING()), + new DataField("c", DataTypes.BIGINT())); + RowType prefixKeyType = + DataTypes.ROW( + new DataField("a", DataTypes.INT()), + new DataField("b", DataTypes.STRING())); + + TableDescriptor descriptor = + TableDescriptor.builder().schema(schema).distributedBy(3, "a", "b").build(); + long tableId = createTable(FLUSS_CLUSTER_EXTENSION, tablePath, descriptor); + TableBucket tb = new TableBucket(tableId, 0); + + FLUSS_CLUSTER_EXTENSION.waitUtilAllReplicaReady(tb); + + int leader = FLUSS_CLUSTER_EXTENSION.waitAndGetLeader(tb); + TabletServerGateway leaderGateWay = + FLUSS_CLUSTER_EXTENSION.newTabletServerClientForNode(leader); + // first prefix lookup without in table, prefix key = (1, "a"). + Object[] prefixKey1 = new Object[] {1, "a"}; + KeyEncoder keyEncoder = new KeyEncoder(rowType, new int[] {0, 1}); + byte[] prefixKey1Bytes = keyEncoder.encode(row(prefixKeyType, prefixKey1)); + assertPrefixLookupResponse( + leaderGateWay + .prefixLookup( + newPrefixLookupRequest( + tableId, 0, Collections.singletonList(prefixKey1Bytes))) + .get(), + Collections.singletonList(Collections.emptyList())); + + // send one batch kv. + List> data1 = + Arrays.asList( + Tuple2.of(new Object[] {1, "a", 1L}, new Object[] {1, "a", 1L, "value1"}), + Tuple2.of(new Object[] {1, "a", 2L}, new Object[] {1, "a", 2L, "value2"}), + Tuple2.of(new Object[] {1, "a", 3L}, new Object[] {1, "a", 3L, "value3"}), + Tuple2.of(new Object[] {2, "a", 4L}, new Object[] {2, "a", 4L, "value4"})); + assertPutKvResponse( + leaderGateWay + .putKv( + newPutKvRequest( + tableId, + 0, + 1, + genKvRecordBatch(primaryKeyType, rowType, data1))) + .get()); + + // second prefix lookup in table, prefix key = (1, "a"). + List key1ExpectedValues = + Arrays.asList( + ValueEncoder.encodeValue( + DEFAULT_SCHEMA_ID, + compactedRow(rowType, new Object[] {1, "a", 1L, "value1"})), + ValueEncoder.encodeValue( + DEFAULT_SCHEMA_ID, + compactedRow(rowType, new Object[] {1, "a", 2L, "value2"})), + ValueEncoder.encodeValue( + DEFAULT_SCHEMA_ID, + compactedRow(rowType, new Object[] {1, "a", 3L, "value3"}))); + assertPrefixLookupResponse( + leaderGateWay + .prefixLookup( + newPrefixLookupRequest( + tableId, 0, Collections.singletonList(prefixKey1Bytes))) + .get(), + Collections.singletonList(key1ExpectedValues)); + + // third prefix lookup in table for multi prefix keys, prefix key = (1, "a") and (2, "a"). + Object[] prefixKey2 = new Object[] {2, "a"}; + byte[] prefixKey2Bytes = keyEncoder.encode(row(prefixKeyType, prefixKey2)); + List key2ExpectedValues = + Collections.singletonList( + ValueEncoder.encodeValue( + DEFAULT_SCHEMA_ID, + compactedRow(rowType, new Object[] {2, "a", 4L, "value4"}))); + assertPrefixLookupResponse( + leaderGateWay + .prefixLookup( + newPrefixLookupRequest( + tableId, + 0, + Arrays.asList(prefixKey1Bytes, prefixKey2Bytes))) + .get(), + Arrays.asList(key1ExpectedValues, key2ExpectedValues)); + + // Prefix lookup an unsupported prefixLookup table. + TableDescriptor unsupportedDescriptor = TableDescriptor.builder().schema(schema).build(); + long tableId2 = + createTable( + FLUSS_CLUSTER_EXTENSION, + TablePath.of("test_db_1", "test_unsupported_prefix_lookup_t1"), + unsupportedDescriptor); + tb = new TableBucket(tableId2, 0); + FLUSS_CLUSTER_EXTENSION.waitUtilAllReplicaReady(tb); + leader = FLUSS_CLUSTER_EXTENSION.waitAndGetLeader(tb); + TabletServerGateway leaderGateWay2 = + FLUSS_CLUSTER_EXTENSION.newTabletServerClientForNode(leader); + PbPrefixLookupRespForBucket pbPrefixLookupRespForBucket = + leaderGateWay2 + .prefixLookup( + newPrefixLookupRequest( + tableId2, 0, Collections.singletonList(prefixKey1Bytes))) + .get() + .getBucketsRespAt(0); + verifyPrefixLookupBucketError( + pbPrefixLookupRespForBucket, + Errors.KV_STORAGE_EXCEPTION, + "Table bucket TableBucket{tableId=" + + tableId2 + + ", bucket=0} does not support prefix lookup"); + } + @Test void testLimitScanPrimaryKeyTable() throws Exception { long tableId = @@ -564,7 +700,7 @@ private static void assertListOffsetsResponse( } } - private static void verifyError( + private static void verifyLookupBucketError( PbLookupRespForBucket lookupRespForBucket, Errors expectedError, String expectErrMessage) { @@ -572,4 +708,13 @@ private static void verifyError( assertThat(lookupRespForBucket.getErrorCode()).isEqualTo(expectedError.code()); assertThat(lookupRespForBucket.getErrorMessage()).contains(expectErrMessage); } + + private static void verifyPrefixLookupBucketError( + PbPrefixLookupRespForBucket prefixLookupRespForBucket, + Errors expectedError, + String expectErrMessage) { + assertThat(prefixLookupRespForBucket.hasErrorCode()).isTrue(); + assertThat(prefixLookupRespForBucket.getErrorCode()).isEqualTo(expectedError.code()); + assertThat(prefixLookupRespForBucket.getErrorMessage()).contains(expectErrMessage); + } } diff --git a/fluss-server/src/test/java/com/alibaba/fluss/server/tablet/TestTabletServerGateway.java b/fluss-server/src/test/java/com/alibaba/fluss/server/tablet/TestTabletServerGateway.java index cd9df1d5..dc0ad507 100644 --- a/fluss-server/src/test/java/com/alibaba/fluss/server/tablet/TestTabletServerGateway.java +++ b/fluss-server/src/test/java/com/alibaba/fluss/server/tablet/TestTabletServerGateway.java @@ -70,6 +70,8 @@ import com.alibaba.fluss.rpc.messages.PbStopReplicaReqForBucket; import com.alibaba.fluss.rpc.messages.PbStopReplicaRespForBucket; import com.alibaba.fluss.rpc.messages.PbTableBucket; +import com.alibaba.fluss.rpc.messages.PrefixLookupRequest; +import com.alibaba.fluss.rpc.messages.PrefixLookupResponse; import com.alibaba.fluss.rpc.messages.ProduceLogRequest; import com.alibaba.fluss.rpc.messages.ProduceLogResponse; import com.alibaba.fluss.rpc.messages.PutKvRequest; @@ -183,6 +185,11 @@ public CompletableFuture lookup(LookupRequest request) { return null; } + @Override + public CompletableFuture prefixLookup(PrefixLookupRequest request) { + throw new UnsupportedOperationException(); + } + @Override public CompletableFuture limitScan(LimitScanRequest request) { return null; diff --git a/fluss-server/src/test/java/com/alibaba/fluss/server/testutils/KvTestUtils.java b/fluss-server/src/test/java/com/alibaba/fluss/server/testutils/KvTestUtils.java index 4d611e95..f5101e7b 100644 --- a/fluss-server/src/test/java/com/alibaba/fluss/server/testutils/KvTestUtils.java +++ b/fluss-server/src/test/java/com/alibaba/fluss/server/testutils/KvTestUtils.java @@ -21,7 +21,10 @@ import com.alibaba.fluss.metadata.TableBucket; import com.alibaba.fluss.rpc.messages.LookupResponse; import com.alibaba.fluss.rpc.messages.PbLookupRespForBucket; +import com.alibaba.fluss.rpc.messages.PbPrefixLookupRespForBucket; import com.alibaba.fluss.rpc.messages.PbValue; +import com.alibaba.fluss.rpc.messages.PbValueList; +import com.alibaba.fluss.rpc.messages.PrefixLookupResponse; import com.alibaba.fluss.server.kv.rocksdb.RocksDBKv; import com.alibaba.fluss.server.kv.rocksdb.RocksDBKvBuilder; import com.alibaba.fluss.server.kv.rocksdb.RocksDBResourceContainer; @@ -49,7 +52,6 @@ import java.util.concurrent.Executors; import java.util.stream.Collectors; -import static com.alibaba.fluss.utils.Preconditions.checkArgument; import static org.assertj.core.api.Assertions.assertThat; /** Test utils related to Kv. */ @@ -165,11 +167,28 @@ public static RocksDBKv buildFromSnapshotHandle( public static void assertLookupResponse( LookupResponse lookupResponse, @Nullable byte[] expectedValue) { - checkArgument(lookupResponse.getBucketsRespsCount() == 1); + assertThat(lookupResponse.getBucketsRespsCount()).isEqualTo(1); PbLookupRespForBucket pbLookupRespForBucket = lookupResponse.getBucketsRespAt(0); - checkArgument(pbLookupRespForBucket.getValuesCount() == 1); + assertThat(pbLookupRespForBucket.getValuesCount()).isEqualTo(1); PbValue pbValue = pbLookupRespForBucket.getValueAt(0); byte[] lookupValue = pbValue.hasValues() ? pbValue.getValues() : null; assertThat(lookupValue).isEqualTo(expectedValue); } + + public static void assertPrefixLookupResponse( + PrefixLookupResponse prefixLookupResponse, List> expectedValues) { + assertThat(prefixLookupResponse.getBucketsRespsCount()).isEqualTo(1); + PbPrefixLookupRespForBucket pbPrefixLookupRespForBucket = + prefixLookupResponse.getBucketsRespAt(0); + assertThat(pbPrefixLookupRespForBucket.getValueListsCount()) + .isEqualTo(expectedValues.size()); + for (int i = 0; i < expectedValues.size(); i++) { + PbValueList pbValueList = pbPrefixLookupRespForBucket.getValueListAt(i); + List bytesResultForOnePrefixKey = expectedValues.get(i); + assertThat(pbValueList.getValuesCount()).isEqualTo(bytesResultForOnePrefixKey.size()); + for (int j = 0; j < bytesResultForOnePrefixKey.size(); j++) { + assertThat(pbValueList.getValueAt(j)).isEqualTo(bytesResultForOnePrefixKey.get(j)); + } + } + } } diff --git a/fluss-server/src/test/java/com/alibaba/fluss/server/testutils/RpcMessageTestUtils.java b/fluss-server/src/test/java/com/alibaba/fluss/server/testutils/RpcMessageTestUtils.java index 435eefe4..e6716a8f 100644 --- a/fluss-server/src/test/java/com/alibaba/fluss/server/testutils/RpcMessageTestUtils.java +++ b/fluss-server/src/test/java/com/alibaba/fluss/server/testutils/RpcMessageTestUtils.java @@ -45,10 +45,12 @@ import com.alibaba.fluss.rpc.messages.PbFetchLogRespForBucket; import com.alibaba.fluss.rpc.messages.PbFetchLogRespForTable; import com.alibaba.fluss.rpc.messages.PbLookupReqForBucket; +import com.alibaba.fluss.rpc.messages.PbPrefixLookupReqForBucket; import com.alibaba.fluss.rpc.messages.PbProduceLogReqForBucket; import com.alibaba.fluss.rpc.messages.PbProduceLogRespForBucket; import com.alibaba.fluss.rpc.messages.PbPutKvReqForBucket; import com.alibaba.fluss.rpc.messages.PbTablePath; +import com.alibaba.fluss.rpc.messages.PrefixLookupRequest; import com.alibaba.fluss.rpc.messages.ProduceLogRequest; import com.alibaba.fluss.rpc.messages.ProduceLogResponse; import com.alibaba.fluss.rpc.messages.PutKvRequest; @@ -215,6 +217,17 @@ public static LookupRequest newLookupRequest(long tableId, int bucketId, byte[] return lookupRequest; } + public static PrefixLookupRequest newPrefixLookupRequest( + long tableId, int bucketId, List prefixKeys) { + PrefixLookupRequest prefixLookupRequest = new PrefixLookupRequest().setTableId(tableId); + PbPrefixLookupReqForBucket pbPrefixLookupReqForBucket = prefixLookupRequest.addBucketsReq(); + pbPrefixLookupReqForBucket.setBucketId(bucketId); + for (byte[] prefixKey : prefixKeys) { + pbPrefixLookupReqForBucket.addKey(prefixKey); + } + return prefixLookupRequest; + } + public static LimitScanRequest newLimitScanRequest(long tableId, int bucketId, int limit) { return new LimitScanRequest().setTableId(tableId).setBucketId(bucketId).setLimit(limit); } diff --git a/website/docs/maintenance/configuration.md b/website/docs/maintenance/configuration.md index b138f4e4..e82ba62c 100644 --- a/website/docs/maintenance/configuration.md +++ b/website/docs/maintenance/configuration.md @@ -216,9 +216,10 @@ Currently, we don't support alter table configuration by Flink. This will be sup | client.request-timeout | Duration | 30s | The timeout for a request to complete. If user set the write ack to -1, this timeout is the max time that delayed write try to complete. The default setting is 30 seconds. | | client.scanner.log.check-crc | Boolean | true | Automatically check the CRC3 of the read records for LogScanner. This ensures no on-the-wire or on-disk corruption to the messages occurred. This check adds some overhead, so it may be disabled in cases seeking extreme performance. | | client.scanner.log.max-poll-records | Integer | 500 | The maximum number of records returned in a single call to poll() for LogScanner. Note that this config doesn't impact the underlying fetching behavior. The Scanner will cache the records from each fetch request and returns them incrementally from each poll. | -| client.lookup.queue-size | Integer | 256 | The maximum number of pending lookup operations. | +| client.lookup.queue-size | Integer | 25600 | The maximum number of pending lookup operations. | | client.lookup.max-batch-size | Integer | 128 | The maximum batch size of merging lookup operations to one lookup request. | | client.lookup.max-inflight-requests | Integer | 128 | The maximum number of unacknowledged lookup requests for lookup operations. | +| client.lookup.batch-timeout | Duration | 100ms | The maximum time to wait for the lookup batch to full, if this timeout is reached, the lookup batch will be closed to send. | | client.scanner.remote-log.prefetch-num | Integer | 4 | The number of remote log segments to keep in local temp file for LogScanner, which download from remote storage. The default setting is 4. | | client.scanner.io.tmpdir | String | System.getProperty("java.io.tmpdir") + "/fluss" | Local directory that is used by client for storing the data files (like kv snapshot, log segment files) to read temporarily | | client.remote-file.download-thread-num | Integer | 3 | The number of threads the client uses to download remote files. | diff --git a/website/docs/maintenance/monitor-metrics.md b/website/docs/maintenance/monitor-metrics.md index d4ca5f6f..9078c106 100644 --- a/website/docs/maintenance/monitor-metrics.md +++ b/website/docs/maintenance/monitor-metrics.md @@ -434,8 +434,8 @@ Some metrics might not be exposed when using other JVM implementations (e.g. IBM - tabletserver - table + tabletserver + table messagesInPerSecond The number of messages written per second to this table Meter @@ -490,6 +490,26 @@ Some metrics might not be exposed when using other JVM implementations (e.g. IBM The number of failed lookup requests to lookup value by key from this table per second. Meter + + totalLimitScanRequestsPerSecond + The number of limit scan requests to scan records with limit from this table per second. + Meter + + + failedLimitScanRequestsPerSecond + The number of failed limit scan requests to scan records with limit from this table per second. + Meter + + + totalIndexLookupRequestsPerSecond + The number of index lookup requests to index lookup value by key from this table per second. + Meter + + + failedIndexLookupRequestsPerSecond + The number of failed index lookup requests to index lookup value by key from this table per second. + Meter + remoteLogCopyBytesPerSecond The bytes of log data copied to remote per second. @@ -705,10 +725,10 @@ After extracting the archive, the contents of the working directory should be as ``` -2. Next, you need to adapt the `docker-compose.yml` manifest and +2. Next, you need to adapt the `docker-compose.yml` manifest and - add containers for Prometheus and Grafana and mount the corresponding configuration directories, and -- configure Fluss to expose metrics via Prometheus +- configure Fluss to expose metrics via Prometheus ``` metrics.reporters: prometheus metrics.reporter.prometheus.port: 9250 @@ -811,7 +831,7 @@ services: volumes: - ./fluss-quickstart-observability/grafana:/etc/grafana:ro #end - + volumes: shared-tmpfs: driver: local @@ -839,6 +859,6 @@ docker ps ``` 3. Now you are all set! You can visit - + - [Grafana](http://localhost:3002/dashboards) to observe the cluster status of the Fluss and Flink cluster with the provided dashboards, or - the [Prometheus Web UI](http://localhost:9092) to directly query Prometheus with [PromQL](https://prometheus.io/docs/prometheus/2.55/getting_started/). \ No newline at end of file From 0f63302076f5673058628f5ee35eddd56d302f92 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Thu, 2 Jan 2025 18:14:46 +0800 Subject: [PATCH 2/2] [kv] Improve PrefixLookup implementation (#222) In the previous commit, the bucket key is only allowed to be a prefix of primary key, but we should allow it to be a subset of primary key. Besides, this commit fixes various bugs around PrefixLookup. --- .../fluss/client/lookup/AbstractLookup.java | 8 +- .../alibaba/fluss/client/lookup/Lookup.java | 9 +- .../fluss/client/lookup/LookupClient.java | 2 +- .../fluss/client/lookup/LookupQueue.java | 19 +- .../fluss/client/lookup/LookupResult.java | 19 +- .../fluss/client/lookup/LookupSender.java | 39 ++- .../fluss/client/lookup/PrefixLookup.java | 11 +- .../client/lookup/PrefixLookupBatch.java | 2 +- .../client/lookup/PrefixLookupResult.java | 2 +- .../fluss/client/table/FlussTable.java | 15 +- .../com/alibaba/fluss/client/table/Table.java | 7 +- .../client/table/writer/UpsertWriter.java | 23 +- .../fluss/client/lookup/LookupQueueTest.java | 65 +++++ .../fluss/metadata/TableDescriptor.java | 85 +++---- .../com/alibaba/fluss/utils/ArrayUtils.java | 89 +++++++ .../fluss/metadata/TableDescriptorTest.java | 68 ++++-- .../alibaba/fluss/utils/ArrayUtilsTest.java | 83 ++++++- .../flink/catalog/FlinkTableFactory.java | 31 ++- .../flink/source/FlinkTableSource.java | 29 ++- .../lookup/FlinkAsyncLookupFunction.java | 13 +- .../source/lookup/FlinkLookupFunction.java | 6 +- .../flink/source/lookup/LookupNormalizer.java | 222 ++++++++++++------ .../utils/FlinkConnectorOptionsUtils.java | 26 ++ .../connector/flink/utils/PushdownUtils.java | 6 +- .../flink/source/FlinkTableSourceITCase.java | 119 ++++++++-- .../lookup/FlinkLookupFunctionTest.java | 18 +- .../entity/PrefixLookupResultForBucket.java | 2 +- .../rpc/gateway/TabletServerGateway.java | 4 +- .../rpc/netty/server/RequestsMetrics.java | 2 +- .../fluss/server/kv/rocksdb/RocksDBKv.java | 27 +-- .../group/PhysicalTableMetricGroup.java | 2 +- .../alibaba/fluss/server/replica/Replica.java | 11 - .../fluss/server/replica/ReplicaManager.java | 19 +- .../server/replica/ReplicaManagerTest.java | 19 +- .../server/tablet/TabletServiceITCase.java | 17 +- website/docs/maintenance/monitor-metrics.md | 8 +- 36 files changed, 757 insertions(+), 370 deletions(-) create mode 100644 fluss-client/src/test/java/com/alibaba/fluss/client/lookup/LookupQueueTest.java diff --git a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/AbstractLookup.java b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/AbstractLookup.java index b6aaef98..9d1e9a2a 100644 --- a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/AbstractLookup.java +++ b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/AbstractLookup.java @@ -25,9 +25,11 @@ @Internal public abstract class AbstractLookup { + private final TableBucket tableBucket; private final byte[] key; - public AbstractLookup(byte[] key) { + public AbstractLookup(TableBucket tableBucket, byte[] key) { + this.tableBucket = tableBucket; this.key = key; } @@ -35,7 +37,9 @@ public byte[] key() { return key; } - public abstract TableBucket tableBucket(); + public TableBucket tableBucket() { + return tableBucket; + } public abstract LookupType lookupType(); diff --git a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/Lookup.java b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/Lookup.java index f8b18c3f..8d340a9d 100644 --- a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/Lookup.java +++ b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/Lookup.java @@ -28,20 +28,13 @@ @Internal public class Lookup extends AbstractLookup { - private final TableBucket tableBucket; private final CompletableFuture future; Lookup(TableBucket tableBucket, byte[] key) { - super(key); - this.tableBucket = tableBucket; + super(tableBucket, key); this.future = new CompletableFuture<>(); } - @Override - public TableBucket tableBucket() { - return tableBucket; - } - @Override public LookupType lookupType() { return LookupType.LOOKUP; diff --git a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupClient.java b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupClient.java index 7d53e4c7..72927a36 100644 --- a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupClient.java +++ b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupClient.java @@ -84,7 +84,7 @@ public CompletableFuture lookup(TableBucket tableBucket, byte[] keyBytes public CompletableFuture> prefixLookup( long tableId, int bucketId, byte[] keyBytes) { - // TODO index lookup support partition table. + // TODO prefix lookup support partition table (#266) PrefixLookup prefixLookup = new PrefixLookup(new TableBucket(tableId, bucketId), keyBytes); lookupQueue.appendLookup(prefixLookup); return prefixLookup.future(); diff --git a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupQueue.java b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupQueue.java index c1f2a40f..7cc20e39 100644 --- a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupQueue.java +++ b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupQueue.java @@ -29,23 +29,23 @@ /** * A queue that buffers the pending lookup operations and provides a list of {@link Lookup} when - * call method {@link #drain(LookupType)}. + * call method {@link #drain()}. */ @ThreadSafe @Internal class LookupQueue { private volatile boolean closed; - // Buffering both the Lookup and IndexLookup. + // buffering both the Lookup and PrefixLookup. private final ArrayBlockingQueue> lookupQueue; private final int maxBatchSize; - private final long batchTimeoutMs; + private final long batchTimeoutNanos; LookupQueue(Configuration conf) { this.lookupQueue = new ArrayBlockingQueue<>(conf.get(ConfigOptions.CLIENT_LOOKUP_QUEUE_SIZE)); this.maxBatchSize = conf.get(ConfigOptions.CLIENT_LOOKUP_MAX_BATCH_SIZE); - this.batchTimeoutMs = conf.get(ConfigOptions.CLIENT_LOOKUP_BATCH_TIMEOUT).toMillis(); + this.batchTimeoutNanos = conf.get(ConfigOptions.CLIENT_LOOKUP_BATCH_TIMEOUT).toNanos(); this.closed = false; } @@ -68,20 +68,23 @@ boolean hasUnDrained() { /** Drain a batch of {@link Lookup}s from the lookup queue. */ List> drain() throws Exception { - long start = System.currentTimeMillis(); + final long startNanos = System.nanoTime(); List> lookupOperations = new ArrayList<>(maxBatchSize); int count = 0; while (true) { - if (System.currentTimeMillis() - start > batchTimeoutMs) { + long waitNanos = batchTimeoutNanos - (System.nanoTime() - startNanos); + if (waitNanos <= 0) { break; } - AbstractLookup lookup = lookupQueue.poll(300, TimeUnit.MILLISECONDS); + AbstractLookup lookup = lookupQueue.poll(waitNanos, TimeUnit.NANOSECONDS); if (lookup == null) { break; } - count++; lookupOperations.add(lookup); + count++; + int transferred = lookupQueue.drainTo(lookupOperations, maxBatchSize - count); + count += transferred; if (count >= maxBatchSize) { break; } diff --git a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupResult.java b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupResult.java index d11c8d19..d31f7bbe 100644 --- a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupResult.java +++ b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupResult.java @@ -20,6 +20,10 @@ import com.alibaba.fluss.client.table.Table; import com.alibaba.fluss.row.InternalRow; +import javax.annotation.Nullable; + +import java.util.Objects; + /** * The result of {@link Table#lookup(InternalRow)}. * @@ -27,13 +31,13 @@ */ @PublicEvolving public final class LookupResult { - private final InternalRow row; + private final @Nullable InternalRow row; - public LookupResult(InternalRow row) { + public LookupResult(@Nullable InternalRow row) { this.row = row; } - public InternalRow getRow() { + public @Nullable InternalRow getRow() { return row; } @@ -45,17 +49,18 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - LookupResult that = (LookupResult) o; - return row.equals(that.row); + + LookupResult lookupResult = (LookupResult) o; + return Objects.equals(row, lookupResult.row); } @Override public int hashCode() { - return row.hashCode(); + return Objects.hash(row); } @Override public String toString() { - return "LookupResult{" + "row=" + row + '}'; + return "LookupResult{row=" + row + '}'; } } diff --git a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupSender.java b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupSender.java index bd142f14..cd3be7be 100644 --- a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupSender.java +++ b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/LookupSender.java @@ -110,13 +110,12 @@ private void sendLookups(List> lookups) { if (lookups.isEmpty()) { return; } - // group by to lookup batches + // group by to lookup batches Map, List>> lookupBatches = groupByLeaderAndType(lookups); // now, send the batches lookupBatches.forEach( - (destinationAndType, batch) -> - sendLookups(destinationAndType.f0, destinationAndType.f1, batch)); + (destAndType, batch) -> sendLookups(destAndType.f0, destAndType.f1, batch)); } private Map, List>> groupByLeaderAndType( @@ -124,22 +123,21 @@ private Map, List>> groupByLeaderA // -> lookup batches Map, List>> lookupBatchesByLeader = new HashMap<>(); - for (AbstractLookup abstractLookup : lookups) { + for (AbstractLookup lookup : lookups) { int leader; // lookup the leader node - TableBucket tb = abstractLookup.tableBucket(); + TableBucket tb = lookup.tableBucket(); try { // TODO this can be a re-triable operation. We should retry here instead of // throwing exception. leader = metadataUpdater.leaderFor(tb); } catch (Exception e) { - abstractLookup.future().completeExceptionally(e); + lookup.future().completeExceptionally(e); continue; } lookupBatchesByLeader - .computeIfAbsent( - Tuple2.of(leader, abstractLookup.lookupType()), k -> new ArrayList<>()) - .add(abstractLookup); + .computeIfAbsent(Tuple2.of(leader, lookup.lookupType()), k -> new ArrayList<>()) + .add(lookup); } return lookupBatchesByLeader; } @@ -157,11 +155,10 @@ private void sendLookups( } } - private void sendLookupRequest( - TabletServerGateway gateway, List> lookupBatches) { + private void sendLookupRequest(TabletServerGateway gateway, List> lookups) { // table id -> (bucket -> lookups) Map> lookupByTableId = new HashMap<>(); - for (AbstractLookup abstractLookup : lookupBatches) { + for (AbstractLookup abstractLookup : lookups) { Lookup lookup = (Lookup) abstractLookup; TableBucket tb = lookup.tableBucket(); long tableId = tb.getTableId(); @@ -181,10 +178,10 @@ private void sendLookupRequest( } private void sendPrefixLookupRequest( - TabletServerGateway gateway, List> indexLookupBatches) { + TabletServerGateway gateway, List> prefixLookups) { // table id -> (bucket -> lookups) Map> lookupByTableId = new HashMap<>(); - for (AbstractLookup abstractLookup : indexLookupBatches) { + for (AbstractLookup abstractLookup : prefixLookups) { PrefixLookup prefixLookup = (PrefixLookup) abstractLookup; TableBucket tb = prefixLookup.tableBucket(); long tableId = tb.getTableId(); @@ -195,12 +192,12 @@ private void sendPrefixLookupRequest( } lookupByTableId.forEach( - (tableId, indexLookupBatch) -> + (tableId, prefixLookupBatch) -> sendPrefixLookupRequestAndHandleResponse( gateway, - makePrefixLookupRequest(tableId, indexLookupBatch.values()), + makePrefixLookupRequest(tableId, prefixLookupBatch.values()), tableId, - indexLookupBatch)); + prefixLookupBatch)); } private void sendLookupRequestAndHandleResponse( @@ -307,7 +304,7 @@ private void handleLookupResponse( private void handlePrefixLookupResponse( long tableId, PrefixLookupResponse prefixLookupResponse, - Map indexLookupsByBucket) { + Map prefixLookupsByBucket) { for (PbPrefixLookupRespForBucket pbRespForBucket : prefixLookupResponse.getBucketsRespsList()) { TableBucket tableBucket = @@ -318,7 +315,7 @@ private void handlePrefixLookupResponse( : null, pbRespForBucket.getBucketId()); - PrefixLookupBatch prefixLookupBatch = indexLookupsByBucket.get(tableBucket); + 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); @@ -328,10 +325,10 @@ private void handlePrefixLookupResponse( error.formatErrMsg()); prefixLookupBatch.completeExceptionally(error.exception()); } else { - List> result = new ArrayList<>(); + List> result = new ArrayList<>(pbRespForBucket.getValueListsCount()); for (int i = 0; i < pbRespForBucket.getValueListsCount(); i++) { PbValueList pbValueList = pbRespForBucket.getValueListAt(i); - List keyResult = new ArrayList<>(); + List keyResult = new ArrayList<>(pbValueList.getValuesCount()); for (int j = 0; j < pbValueList.getValuesCount(); j++) { keyResult.add(pbValueList.getValueAt(j)); } diff --git a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/PrefixLookup.java b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/PrefixLookup.java index 289623ee..a9f1050d 100644 --- a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/PrefixLookup.java +++ b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/PrefixLookup.java @@ -28,20 +28,13 @@ */ @Internal public class PrefixLookup extends AbstractLookup> { - private final TableBucket tableBucket; private final CompletableFuture> future; - public PrefixLookup(TableBucket tableBucket, byte[] prefixKey) { - super(prefixKey); - this.tableBucket = tableBucket; + PrefixLookup(TableBucket tableBucket, byte[] prefixKey) { + super(tableBucket, prefixKey); this.future = new CompletableFuture<>(); } - @Override - public TableBucket tableBucket() { - return tableBucket; - } - @Override public CompletableFuture> future() { return future; diff --git a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/PrefixLookupBatch.java b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/PrefixLookupBatch.java index 20188ff5..6c973681 100644 --- a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/PrefixLookupBatch.java +++ b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/PrefixLookupBatch.java @@ -58,7 +58,7 @@ public void complete(List> values) { new FlussRuntimeException( String.format( "The number of values return by prefix lookup request is not equal to the number of " - + "index lookups send. Got %d values, but expected %d.", + + "prefix lookups send. Got %d values, but expected %d.", values.size(), prefixLookups.size()))); } else { for (int i = 0; i < values.size(); i++) { diff --git a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/PrefixLookupResult.java b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/PrefixLookupResult.java index a9ff3044..3a6e3186 100644 --- a/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/PrefixLookupResult.java +++ b/fluss-client/src/main/java/com/alibaba/fluss/client/lookup/PrefixLookupResult.java @@ -24,7 +24,7 @@ /** * The result of {@link Table#prefixLookup(InternalRow)}}. * - * @since 0.1 + * @since 0.6 */ public class PrefixLookupResult { private final List rowList; diff --git a/fluss-client/src/main/java/com/alibaba/fluss/client/table/FlussTable.java b/fluss-client/src/main/java/com/alibaba/fluss/client/table/FlussTable.java index 385e20e1..779fb63e 100644 --- a/fluss-client/src/main/java/com/alibaba/fluss/client/table/FlussTable.java +++ b/fluss-client/src/main/java/com/alibaba/fluss/client/table/FlussTable.java @@ -187,14 +187,14 @@ public CompletableFuture lookup(InternalRow key) { } // encoding the key row using a compacted way consisted with how the key is encoded when put // a row - byte[] keyBytes = primaryKeyEncoder.encode(key); - byte[] lookupBucketKeyBytes = bucketKeyEncoder.encode(key); + byte[] pkBytes = primaryKeyEncoder.encode(key); + byte[] bkBytes = bucketKeyEncoder.encode(key); Long partitionId = keyRowPartitionGetter == null ? null : getPartitionId(key); - int bucketId = getBucketId(lookupBucketKeyBytes, key); + int bucketId = getBucketId(bkBytes, key); TableBucket tableBucket = new TableBucket(tableId, partitionId, bucketId); return lookupClientSupplier .get() - .lookup(tableBucket, keyBytes) + .lookup(tableBucket, pkBytes) .thenApply( valueBytes -> { InternalRow row = @@ -206,14 +206,15 @@ public CompletableFuture lookup(InternalRow key) { } @Override - public CompletableFuture prefixLookup(InternalRow prefixKey) { + public CompletableFuture prefixLookup(InternalRow bucketKey) { if (!hasPrimaryKey) { throw new FlussRuntimeException( String.format("None-pk table %s don't support prefix lookup", tablePath)); } + // TODO: add checks the bucket key is prefix of primary key - byte[] prefixKeyBytes = bucketKeyEncoder.encode(prefixKey); - int bucketId = getBucketId(prefixKeyBytes, prefixKey); + byte[] prefixKeyBytes = bucketKeyEncoder.encode(bucketKey); + int bucketId = getBucketId(prefixKeyBytes, bucketKey); return lookupClientSupplier .get() .prefixLookup(tableId, bucketId, prefixKeyBytes) diff --git a/fluss-client/src/main/java/com/alibaba/fluss/client/table/Table.java b/fluss-client/src/main/java/com/alibaba/fluss/client/table/Table.java index 0b4d6547..62fd8ef1 100644 --- a/fluss-client/src/main/java/com/alibaba/fluss/client/table/Table.java +++ b/fluss-client/src/main/java/com/alibaba/fluss/client/table/Table.java @@ -78,10 +78,13 @@ public interface Table extends AutoCloseable { * the prefix schema would also be [a, b]. This pattern can use PrefixLookup to lookup by prefix * scan. * - * @param prefixKey the given prefix key to do prefix lookup. + *

TODO: currently, the interface only support bucket key as the prefix key to lookup. + * Generalize the prefix lookup to support any prefix key including bucket key. + * + * @param bucketKey the given bucket key to do prefix lookup. * @return the result of prefix lookup. */ - CompletableFuture prefixLookup(InternalRow prefixKey); + CompletableFuture prefixLookup(InternalRow bucketKey); /** * Extracts limit number of rows from the given table bucket. diff --git a/fluss-client/src/main/java/com/alibaba/fluss/client/table/writer/UpsertWriter.java b/fluss-client/src/main/java/com/alibaba/fluss/client/table/writer/UpsertWriter.java index 5f8c5aef..fe84f76e 100644 --- a/fluss-client/src/main/java/com/alibaba/fluss/client/table/writer/UpsertWriter.java +++ b/fluss-client/src/main/java/com/alibaba/fluss/client/table/writer/UpsertWriter.java @@ -41,8 +41,9 @@ @PublicEvolving public class UpsertWriter extends TableWriter { - private final KeyEncoder keyEncoder; - private final KeyEncoder bucketKeyEncoder; + private final KeyEncoder primaryKeyEncoder; + // null if the bucket key is the same to the primary key + private final @Nullable KeyEncoder bucketKeyEncoder; private final @Nullable int[] targetColumns; public UpsertWriter( @@ -58,17 +59,17 @@ public UpsertWriter( RowType rowType = schema.toRowType(); this.targetColumns = upsertWrite.getPartialUpdateColumns(); - this.keyEncoder = + this.primaryKeyEncoder = KeyEncoder.createKeyEncoder( rowType, schema.getPrimaryKey().get().getColumnNames(), tableDescriptor.getPartitionKeys()); - int[] bucketKeyIndexes = tableDescriptor.getBucketKeyIndexes(); - if (bucketKeyIndexes.length != 0) { - this.bucketKeyEncoder = new KeyEncoder(rowType, bucketKeyIndexes); + if (tableDescriptor.isDefaultBucketKey()) { + this.bucketKeyEncoder = null; } else { - this.bucketKeyEncoder = keyEncoder; + int[] bucketKeyIndexes = tableDescriptor.getBucketKeyIndexes(); + this.bucketKeyEncoder = new KeyEncoder(rowType, bucketKeyIndexes); } } @@ -119,8 +120,8 @@ private static void sanityCheck(Schema schema, @Nullable int[] targetColumns) { * @return A {@link CompletableFuture} that always returns null when complete normally. */ public CompletableFuture upsert(InternalRow row) { - byte[] key = keyEncoder.encode(row); - byte[] bucketKey = bucketKeyEncoder.encode(row); + byte[] key = primaryKeyEncoder.encode(row); + byte[] bucketKey = bucketKeyEncoder != null ? bucketKeyEncoder.encode(row) : key; return send( new WriteRecord( getPhysicalPath(row), WriteKind.PUT, key, bucketKey, row, targetColumns)); @@ -134,8 +135,8 @@ public CompletableFuture upsert(InternalRow row) { * @return A {@link CompletableFuture} that always returns null when complete normally. */ public CompletableFuture delete(InternalRow row) { - byte[] key = keyEncoder.encode(row); - byte[] bucketKey = bucketKeyEncoder.encode(row); + byte[] key = primaryKeyEncoder.encode(row); + byte[] bucketKey = bucketKeyEncoder != null ? bucketKeyEncoder.encode(row) : key; return send( new WriteRecord( getPhysicalPath(row), diff --git a/fluss-client/src/test/java/com/alibaba/fluss/client/lookup/LookupQueueTest.java b/fluss-client/src/test/java/com/alibaba/fluss/client/lookup/LookupQueueTest.java new file mode 100644 index 00000000..93fc48ee --- /dev/null +++ b/fluss-client/src/test/java/com/alibaba/fluss/client/lookup/LookupQueueTest.java @@ -0,0 +1,65 @@ +/* + * Copyright (c) 2024 Alibaba Group Holding Ltd. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.alibaba.fluss.client.lookup; + +import com.alibaba.fluss.config.Configuration; +import com.alibaba.fluss.metadata.TableBucket; + +import org.junit.jupiter.api.Test; + +import static com.alibaba.fluss.config.ConfigOptions.CLIENT_LOOKUP_BATCH_TIMEOUT; +import static com.alibaba.fluss.config.ConfigOptions.CLIENT_LOOKUP_MAX_BATCH_SIZE; +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link LookupQueue}. */ +class LookupQueueTest { + + @Test + void testDrainMaxBatchSize() throws Exception { + Configuration conf = new Configuration(); + conf.set(CLIENT_LOOKUP_MAX_BATCH_SIZE, 10); + conf.setString(CLIENT_LOOKUP_BATCH_TIMEOUT.key(), "1ms"); + LookupQueue queue = new LookupQueue(conf); + + // drain empty + assertThat(queue.drain()).hasSize(0); + + appendLookups(queue, 1); + assertThat(queue.drain()).hasSize(1); + assertThat(queue.hasUnDrained()).isFalse(); + + appendLookups(queue, 9); + assertThat(queue.drain()).hasSize(9); + assertThat(queue.hasUnDrained()).isFalse(); + + appendLookups(queue, 10); + assertThat(queue.drain()).hasSize(10); + assertThat(queue.hasUnDrained()).isFalse(); + + appendLookups(queue, 20); + assertThat(queue.drain()).hasSize(10); + assertThat(queue.hasUnDrained()).isTrue(); + assertThat(queue.drainAll()).hasSize(10); + assertThat(queue.hasUnDrained()).isFalse(); + } + + private static void appendLookups(LookupQueue queue, int count) { + for (int i = 0; i < count; i++) { + queue.appendLookup(new Lookup(new TableBucket(1, 1), new byte[] {0})); + } + } +} diff --git a/fluss-common/src/main/java/com/alibaba/fluss/metadata/TableDescriptor.java b/fluss-common/src/main/java/com/alibaba/fluss/metadata/TableDescriptor.java index ec6c7a44..7c1df3db 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/metadata/TableDescriptor.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/metadata/TableDescriptor.java @@ -151,12 +151,16 @@ public Schema getSchema() { return schema; } + /** Returns the bucket key of the table, empty if no bucket key is set. */ public List getBucketKey() { return this.getTableDistribution() .map(TableDescriptor.TableDistribution::getBucketKeys) .orElse(Collections.emptyList()); } + /** + * Returns the indexes of the bucket key fields in the schema, empty if no bucket key is set. + */ public int[] getBucketKeyIndexes() { List bucketKey = getBucketKey(); RowType rowType = schema.toRowType(); @@ -167,6 +171,22 @@ public int[] getBucketKeyIndexes() { return bucketKeyIndex; } + /** + * Check if the table is using a default bucket key. A default bucket key is: + * + *

    + *
  • the same as the primary keys excluding the partition keys. + *
  • empty if the table is not a primary key table. + *
+ */ + public boolean isDefaultBucketKey() { + if (schema.getPrimaryKey().isPresent()) { + return getBucketKey().equals(defaultBucketKeyOfPrimaryKeyTable(schema, partitionKeys)); + } else { + return getBucketKey().isEmpty(); + } + } + /** * Check if the table is partitioned or not. * @@ -371,27 +391,16 @@ private static TableDistribution normalizeDistribution( originDistribution.getBucketCount().orElse(null), defaultBucketKeyOfPrimaryKeyTable(schema, partitionKeys)); } else { - // For the primary key table match prefix lookup pattern, we allow the bucket - // keys different from primary keys. - if (!bucketKeysMatchPrefixLookupPattern(schema, bucketKeys)) { - // check the provided bucket key and expected bucket key - List expectedBucketKeys = - defaultBucketKeyOfPrimaryKeyTable(schema, partitionKeys); - List pkColumns = schema.getPrimaryKey().get().getColumnNames(); - - if (expectedBucketKeys.size() != bucketKeys.size() - || !new HashSet<>(expectedBucketKeys).containsAll(bucketKeys)) { - throw new IllegalArgumentException( - String.format( - "Currently, bucket keys must be equal to primary keys excluding partition " - + "keys for primary-key tables. The primary keys are %s, the " - + "partition keys are %s, the expected bucket keys are %s, but " - + "the user-defined bucket keys are %s.", - pkColumns, - partitionKeys, - expectedBucketKeys, - bucketKeys)); - } + // check the provided bucket key + List pkColumns = schema.getPrimaryKey().get().getColumnNames(); + if (!new HashSet<>(pkColumns).containsAll(bucketKeys)) { + throw new IllegalArgumentException( + String.format( + "Bucket keys must be a subset of primary keys excluding partition " + + "keys for primary-key tables. The primary keys are %s, the " + + "partition keys are %s, but " + + "the user-defined bucket keys are %s.", + pkColumns, partitionKeys, bucketKeys)); } return new TableDistribution( originDistribution.getBucketCount().orElse(null), bucketKeys); @@ -411,40 +420,6 @@ private static TableDistribution normalizeDistribution( } } - /** - * Check if the table supports prefix lookup. Currently, only pk-table are supported, and the - * bucket key needs to be part of the primary key and must be a prefix of the primary key. For - * example, if a table has fields [a,b,c,d], and the primary key is set to [a, b, c], with the - * bucket key set to [a, b]. - * - * @return true if the table supports prefix lookup; otherwise, false - */ - public static boolean bucketKeysMatchPrefixLookupPattern( - Schema schema, List bucketKeys) { - if (!schema.getPrimaryKey().isPresent()) { - return false; - } - - RowType rowType = schema.toRowType(); - int[] pkIndexes = schema.getPrimaryKeyIndexes(); - int[] bucketKeyIndexes = new int[bucketKeys.size()]; - for (int i = 0; i < bucketKeys.size(); i++) { - bucketKeyIndexes[i] = rowType.getFieldIndex(bucketKeys.get(i)); - } - - if (bucketKeyIndexes.length >= pkIndexes.length) { - return false; - } else { - for (int i = 0; i < bucketKeyIndexes.length; i++) { - if (bucketKeyIndexes[i] != pkIndexes[i]) { - return false; - } - } - - return true; - } - } - /** The default bucket key of primary key table is the primary key excluding partition keys. */ private static List defaultBucketKeyOfPrimaryKeyTable( Schema schema, List partitionKeys) { diff --git a/fluss-common/src/main/java/com/alibaba/fluss/utils/ArrayUtils.java b/fluss-common/src/main/java/com/alibaba/fluss/utils/ArrayUtils.java index 9f4ab1be..3a65620c 100644 --- a/fluss-common/src/main/java/com/alibaba/fluss/utils/ArrayUtils.java +++ b/fluss-common/src/main/java/com/alibaba/fluss/utils/ArrayUtils.java @@ -34,4 +34,93 @@ public static String[] concat(String[] array1, String[] array2) { System.arraycopy(array2, 0, resultArray, array1.length, array2.length); return resultArray; } + + public static int[] concat(int[] array1, int[] array2) { + if (array1.length == 0) { + return array2; + } + if (array2.length == 0) { + return array1; + } + int[] resultArray = new int[array1.length + array2.length]; + System.arraycopy(array1, 0, resultArray, 0, array1.length); + System.arraycopy(array2, 0, resultArray, array1.length, array2.length); + return resultArray; + } + + /** Check if the second array is a subset of the first array. */ + public static boolean isSubset(int[] a, int[] b) { + // Iterate over each element in the second array + for (int elementB : b) { + boolean found = false; + // Check if the element exists in the first array + for (int elementA : a) { + if (elementB == elementA) { + found = true; + break; + } + } + // If any element is not found, return false + if (!found) { + return false; + } + } + // If all elements are found, return true + return true; + } + + /** + * Remove the elements of the second array from the first array. + * + * @throws IllegalArgumentException if the element of the second array is not found in the first + * array. + */ + public static int[] removeSet(int[] a, int[] b) { + // Iterate over each element in the second array + // and check if the element exists in the first array + if (!isSubset(a, b)) { + throw new IllegalArgumentException("Element not found in the first array"); + } + // Remove the elements of the second array from the first array + int[] resultArray = new int[a.length - b.length]; + int index = 0; + for (int elementA : a) { + boolean found = false; + for (int elementB : b) { + if (elementA == elementB) { + found = true; + break; + } + } + if (!found) { + resultArray[index] = elementA; + index++; + } + } + return resultArray; + } + + /** + * Returns a new array that contains the intersection of the two arrays and in the order of the + * first array. + * + * @throws IllegalArgumentException if the element of the second array is not found in the first + * array. + */ + public static int[] intersection(int[] a, int[] b) { + // Remove the elements from the first array that not exist in the second array + return removeSet(a, removeSet(a, b)); + } + + /** Check if the second array is a prefix of the first array. */ + public static boolean isPrefix(int[] a, int[] b) { + // Iterate over each element in the second array + for (int i = 0; i < b.length; i++) { + // Check if the element exists in the first array + if (a[i] != b[i]) { + return false; + } + } + return true; + } } diff --git a/fluss-common/src/test/java/com/alibaba/fluss/metadata/TableDescriptorTest.java b/fluss-common/src/test/java/com/alibaba/fluss/metadata/TableDescriptorTest.java index fed0d8ce..478d9c51 100644 --- a/fluss-common/src/test/java/com/alibaba/fluss/metadata/TableDescriptorTest.java +++ b/fluss-common/src/test/java/com/alibaba/fluss/metadata/TableDescriptorTest.java @@ -91,15 +91,44 @@ void testProperties() { @Test void testDistribution() { - final TableDescriptor descriptor = + TableDescriptor descriptor = TableDescriptor.builder().schema(SCHEMA_1).distributedBy(10, "f0", "f3").build(); - Optional distribution = descriptor.getTableDistribution(); assertThat(distribution).isPresent(); assertThat(distribution.get().getBucketCount()).hasValue(10); assertThat(distribution.get().getBucketKeys()).hasSize(2); assertThat(distribution.get().getBucketKeys().get(0)).isEqualTo("f0", "f3"); + assertThat(descriptor.isDefaultBucketKey()).isTrue(); + + // a subset of primary key + descriptor = TableDescriptor.builder().schema(SCHEMA_1).distributedBy(10, "f3").build(); + distribution = descriptor.getTableDistribution(); + assertThat(distribution).isPresent(); + assertThat(distribution.get().getBucketCount()).hasValue(10); + assertThat(distribution.get().getBucketKeys()).isEqualTo(Collections.singletonList("f3")); + assertThat(descriptor.isDefaultBucketKey()).isFalse(); + + // default bucket key for partitioned table + descriptor = TableDescriptor.builder().schema(SCHEMA_1).partitionedBy("f0").build(); + distribution = descriptor.getTableDistribution(); + assertThat(distribution).isPresent(); + assertThat(distribution.get().getBucketCount()).isEmpty(); + assertThat(distribution.get().getBucketKeys()).isEqualTo(Collections.singletonList("f3")); + assertThat(descriptor.isDefaultBucketKey()).isTrue(); + + // test subset of primary key for partitioned table + descriptor = + TableDescriptor.builder() + .schema(SCHEMA_1) + .partitionedBy("f0") + .distributedBy(10, "f3") + .build(); + distribution = descriptor.getTableDistribution(); + assertThat(distribution).isPresent(); + assertThat(distribution.get().getBucketCount()).hasValue(10); + assertThat(distribution.get().getBucketKeys()).isEqualTo(Collections.singletonList("f3")); + assertThat(descriptor.isDefaultBucketKey()).isTrue(); } @Test @@ -116,6 +145,7 @@ void testSchemaWithoutPrimaryKeyAndDistributionWithEmptyBucketKeys() { assertThat(distribution).isPresent(); assertThat(distribution.get().getBucketCount()).hasValue(12); assertThat(distribution.get().getBucketKeys()).hasSize(0); + assertThat(descriptor.isDefaultBucketKey()).isTrue(); } @Test @@ -128,9 +158,9 @@ void testPrimaryKeyDifferentWithBucketKeys() { .build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage( - "Currently, bucket keys must be equal to primary keys excluding partition keys for primary-key tables. " + "Bucket keys must be a subset of primary keys excluding partition keys for primary-key tables. " + "The primary keys are [f0, f3], the partition keys are [], " - + "the expected bucket keys are [f0, f3], but the user-defined bucket keys are [f1]."); + + "but the user-defined bucket keys are [f1]."); assertThatThrownBy( () -> @@ -140,25 +170,21 @@ void testPrimaryKeyDifferentWithBucketKeys() { .build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage( - "Currently, bucket keys must be equal to primary keys excluding partition keys for primary-key tables. " + "Bucket keys must be a subset of primary keys excluding partition keys for primary-key tables. " + "The primary keys are [f0, f3], the partition keys are [], " - + "the expected bucket keys are [f0, f3], but the user-defined bucket keys are [f0, f1]."); + + "but the user-defined bucket keys are [f0, f1]."); - // bucket key is the subset of primary key. This pattern can be support for prefixLookup. - Schema schema0 = - Schema.newBuilder() - .column("f0", DataTypes.STRING()) - .column("f1", DataTypes.BIGINT()) - .primaryKey("f0", "f1") - .build(); - TableDescriptor tableDescriptor = - TableDescriptor.builder().schema(schema0).distributedBy(12, "f0").build(); - assertThat(tableDescriptor.getBucketKey()).containsExactlyInAnyOrder("f0"); - assertThat(tableDescriptor.getBucketKeyIndexes()).isEqualTo(new int[] {0}); - assertThat( - TableDescriptor.bucketKeysMatchPrefixLookupPattern( - schema0, Collections.singletonList("f0"))) - .isTrue(); + // bucket key shouldn't include partition key + assertThatThrownBy( + () -> + TableDescriptor.builder() + .schema(SCHEMA_1) + .partitionedBy("f0") + .distributedBy(3, "f0", "f3") + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Bucket key [f0, f3] shouldn't include any column in partition keys [f0]."); } @Test diff --git a/fluss-common/src/test/java/com/alibaba/fluss/utils/ArrayUtilsTest.java b/fluss-common/src/test/java/com/alibaba/fluss/utils/ArrayUtilsTest.java index c48066e6..956884f5 100644 --- a/fluss-common/src/test/java/com/alibaba/fluss/utils/ArrayUtilsTest.java +++ b/fluss-common/src/test/java/com/alibaba/fluss/utils/ArrayUtilsTest.java @@ -19,12 +19,13 @@ import org.junit.jupiter.api.Test; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for the {@link com.alibaba.fluss.utils.ArrayUtils}. */ public class ArrayUtilsTest { @Test - void concatWithEmptyArray() { + void testConcatWithEmptyArray() { String[] emptyArray = new String[] {}; String[] nonEmptyArray = new String[] {"some value"}; @@ -34,7 +35,7 @@ void concatWithEmptyArray() { } @Test - void concatArrays() { + void testConcatArrays() { String[] array1 = new String[] {"A", "B", "C", "D", "E", "F", "G"}; String[] array2 = new String[] {"1", "2", "3"}; @@ -44,4 +45,82 @@ void concatArrays() { assertThat(ArrayUtils.concat(array2, array1)) .isEqualTo(new String[] {"1", "2", "3", "A", "B", "C", "D", "E", "F", "G"}); } + + @Test + void testConcatIntWithEmptyArray() { + int[] emptyArray = new int[] {}; + int[] nonEmptyArray = new int[] {1}; + + assertThat(ArrayUtils.concat(emptyArray, nonEmptyArray)).isSameAs(nonEmptyArray); + + assertThat(ArrayUtils.concat(nonEmptyArray, emptyArray)).isSameAs(nonEmptyArray); + } + + @Test + void testConcatIntArrays() { + int[] array1 = new int[] {1, 2, 3, 4, 5, 6, 7}; + int[] array2 = new int[] {8, 9, 10}; + + assertThat(ArrayUtils.concat(array1, array2)) + .isEqualTo(new int[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 10}); + + assertThat(ArrayUtils.concat(array2, array1)) + .isEqualTo(new int[] {8, 9, 10, 1, 2, 3, 4, 5, 6, 7}); + } + + @Test + void testIsSubset() { + int[] a = new int[] {1, 2, 3, 4, 5}; + int[] b = new int[] {1, 3, 5}; + int[] c = new int[] {1, 3, 6}; + int[] d = new int[] {}; + + assertThat(ArrayUtils.isSubset(a, b)).isTrue(); + assertThat(ArrayUtils.isSubset(a, c)).isFalse(); + assertThat(ArrayUtils.isSubset(a, d)).isTrue(); + } + + @Test + void testRemoveSet() { + int[] a = new int[] {1, 2, 3, 4, 5}; + int[] b = new int[] {1, 3, 5}; + int[] c = new int[] {1, 3, 6}; + int[] d = new int[] {}; + + assertThat(ArrayUtils.removeSet(a, b)).isEqualTo(new int[] {2, 4}); + assertThat(ArrayUtils.removeSet(a, d)).isEqualTo(new int[] {1, 2, 3, 4, 5}); + assertThatThrownBy(() -> ArrayUtils.removeSet(a, c)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Element not found in the first array"); + } + + @Test + void testIntersection() { + int[] a = new int[] {1, 2, 3, 4, 5}; + int[] b = new int[] {1, 3, 5}; + int[] c = new int[] {5, 3, 1}; + int[] d = new int[] {}; + int[] e = new int[] {1, 3, 6}; + + assertThat(ArrayUtils.intersection(a, b)).isEqualTo(new int[] {1, 3, 5}); + assertThat(ArrayUtils.intersection(a, c)).isEqualTo(new int[] {1, 3, 5}); + assertThat(ArrayUtils.intersection(a, d)).isEqualTo(new int[] {}); + assertThatThrownBy(() -> ArrayUtils.intersection(a, e)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Element not found in the first array"); + } + + @Test + void testIsPrefix() { + int[] a = new int[] {1, 2, 3, 4, 5}; + int[] b = new int[] {1, 2, 3}; + int[] c = new int[] {1, 3, 5}; + int[] d = new int[] {2, 3}; + int[] e = new int[] {}; + + assertThat(ArrayUtils.isPrefix(a, b)).isTrue(); + assertThat(ArrayUtils.isPrefix(a, c)).isFalse(); + assertThat(ArrayUtils.isPrefix(a, d)).isFalse(); + assertThat(ArrayUtils.isPrefix(a, e)).isTrue(); + } } diff --git a/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/catalog/FlinkTableFactory.java b/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/catalog/FlinkTableFactory.java index 62baf248..b719918b 100644 --- a/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/catalog/FlinkTableFactory.java +++ b/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/catalog/FlinkTableFactory.java @@ -90,12 +90,16 @@ public DynamicTableSource createDynamicTableSource(Context context) { ResolvedSchema resolvedSchema = context.getCatalogTable().getResolvedSchema(); ResolvedCatalogTable resolvedCatalogTable = context.getCatalogTable(); int[] primaryKeyIndexes = resolvedSchema.getPrimaryKeyIndexes(); + int[] partitionKeyIndexes = + resolvedCatalogTable.getPartitionKeys().stream() + .mapToInt(tableOutputType::getFieldIndex) + .toArray(); + int[] bucketKeyIndexes = + FlinkConnectorOptionsUtils.getBucketKeyIndexes(tableOptions, tableOutputType); // options for lookup LookupCache cache = null; - LookupOptions.LookupCacheType lookupCacheType = tableOptions.get(LookupOptions.CACHE_TYPE); - if (lookupCacheType.equals(LookupOptions.LookupCacheType.PARTIAL)) { cache = DefaultLookupCache.fromConfig(tableOptions); } else if (lookupCacheType.equals(LookupOptions.LookupCacheType.FULL)) { @@ -105,24 +109,31 @@ public DynamicTableSource createDynamicTableSource(Context context) { throw new UnsupportedOperationException("Full lookup caching is not supported yet."); } + // other option values + long partitionDiscoveryIntervalMs = + tableOptions + .get(FlinkConnectorOptions.SCAN_PARTITION_DISCOVERY_INTERVAL) + .toMillis(); + boolean isDatalakeEnabled = + tableOptions.get( + key(ConfigOptions.TABLE_DATALAKE_ENABLED.key()) + .booleanType() + .defaultValue(false)); + return new FlinkTableSource( toFlussTablePath(context.getObjectIdentifier()), toFlussClientConfig(helper.getOptions(), context.getConfiguration()), tableOutputType, primaryKeyIndexes, - resolvedCatalogTable.getPartitionKeys(), + bucketKeyIndexes, + partitionKeyIndexes, isStreamingMode, startupOptions, tableOptions.get(LookupOptions.MAX_RETRIES), tableOptions.get(FlinkConnectorOptions.LOOKUP_ASYNC), cache, - tableOptions - .get(FlinkConnectorOptions.SCAN_PARTITION_DISCOVERY_INTERVAL) - .toMillis(), - tableOptions.get( - key(ConfigOptions.TABLE_DATALAKE_ENABLED.key()) - .booleanType() - .defaultValue(false))); + partitionDiscoveryIntervalMs, + isDatalakeEnabled); } @Override diff --git a/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/FlinkTableSource.java b/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/FlinkTableSource.java index 53250681..939d872c 100644 --- a/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/FlinkTableSource.java +++ b/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/FlinkTableSource.java @@ -88,9 +88,12 @@ public class FlinkTableSource private final Configuration flussConfig; // output type before projection pushdown private final org.apache.flink.table.types.logical.RowType tableOutputType; - // will be empty if no pk + // will be empty if no primary key private final int[] primaryKeyIndexes; - private final List partitionKeys; + // will be empty if no bucket key + private final int[] bucketKeyIndexes; + // will be empty if no partition key + private final int[] partitionKeyIndexes; private final boolean streaming; private final FlinkConnectorOptionsUtils.StartupOptions startupOptions; @@ -123,7 +126,8 @@ public FlinkTableSource( Configuration flussConfig, org.apache.flink.table.types.logical.RowType tableOutputType, int[] primaryKeyIndexes, - List partitionKeys, + int[] bucketKeyIndexes, + int[] partitionKeyIndexes, boolean streaming, FlinkConnectorOptionsUtils.StartupOptions startupOptions, int lookupMaxRetryTimes, @@ -136,7 +140,8 @@ public FlinkTableSource( this.tableOutputType = tableOutputType; this.producedDataType = tableOutputType; this.primaryKeyIndexes = primaryKeyIndexes; - this.partitionKeys = partitionKeys; + this.bucketKeyIndexes = bucketKeyIndexes; + this.partitionKeyIndexes = partitionKeyIndexes; this.streaming = streaming; this.startupOptions = checkNotNull(startupOptions, "startupOptions must not be null"); @@ -167,6 +172,10 @@ private boolean hasPrimaryKey() { return primaryKeyIndexes.length > 0; } + private boolean isPartitioned() { + return partitionKeyIndexes.length > 0; + } + @Override public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) { // handle single row filter scan @@ -239,7 +248,7 @@ public boolean isBounded() { flussConfig, tablePath, hasPrimaryKey(), - !partitionKeys.isEmpty(), + isPartitioned(), flussRowType, projectedFields, offsetsInitializer, @@ -281,16 +290,16 @@ public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext context) { LookupNormalizer.validateAndCreateLookupNormalizer( context.getKeys(), primaryKeyIndexes, + bucketKeyIndexes, + partitionKeyIndexes, tableOutputType, - tablePath, - flussConfig); + projectedFields); if (lookupAsync) { AsyncLookupFunction asyncLookupFunction = new FlinkAsyncLookupFunction( flussConfig, tablePath, tableOutputType, - lookupNormalizer.getLookupKeyIndexes(), lookupMaxRetryTimes, lookupNormalizer, projectedFields); @@ -305,7 +314,6 @@ public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext context) { flussConfig, tablePath, tableOutputType, - lookupNormalizer.getLookupKeyIndexes(), lookupMaxRetryTimes, lookupNormalizer, projectedFields); @@ -325,7 +333,8 @@ public DynamicTableSource copy() { flussConfig, tableOutputType, primaryKeyIndexes, - partitionKeys, + bucketKeyIndexes, + partitionKeyIndexes, streaming, startupOptions, lookupMaxRetryTimes, diff --git a/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/lookup/FlinkAsyncLookupFunction.java b/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/lookup/FlinkAsyncLookupFunction.java index 0c48a799..ae53093d 100644 --- a/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/lookup/FlinkAsyncLookupFunction.java +++ b/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/lookup/FlinkAsyncLookupFunction.java @@ -59,7 +59,6 @@ public class FlinkAsyncLookupFunction extends AsyncLookupFunction { private final TablePath tablePath; private final int maxRetryTimes; private final RowType flinkRowType; - private final int[] lookupKeyIndexes; private final LookupNormalizer lookupNormalizer; @Nullable private final int[] projection; private final LookupType flussLookupType; @@ -73,7 +72,6 @@ public FlinkAsyncLookupFunction( Configuration flussConfig, TablePath tablePath, RowType flinkRowType, - int[] lookupKeyIndexes, int maxRetryTimes, LookupNormalizer lookupNormalizer, @Nullable int[] projection) { @@ -81,10 +79,9 @@ public FlinkAsyncLookupFunction( this.tablePath = tablePath; this.maxRetryTimes = maxRetryTimes; this.flinkRowType = flinkRowType; - this.lookupKeyIndexes = lookupKeyIndexes; this.lookupNormalizer = lookupNormalizer; this.projection = projection; - this.flussLookupType = lookupNormalizer.getFlussLookupType(); + this.flussLookupType = lookupNormalizer.getLookupType(); } @Override @@ -93,6 +90,7 @@ public void open(FunctionContext context) { connection = ConnectionFactory.createConnection(flussConfig); table = connection.getTable(tablePath); // TODO: convert to Fluss GenericRow to avoid unnecessary deserialization + int[] lookupKeyIndexes = lookupNormalizer.getLookupKeyIndexes(); flinkRowToFlussRowConverter = FlinkRowToFlussRowConverter.create( FlinkUtils.projectRowType(flinkRowType, lookupKeyIndexes), @@ -153,7 +151,7 @@ private void fetchResult( handleLookupSuccess(resultFuture, result, remainingFilter); } }); - } else if (flussLookupType == LookupType.PREFIX_LOOKUP) { + } else { table.prefixLookup(keyRow) .whenComplete( (result, throwable) -> { @@ -169,11 +167,6 @@ private void fetchResult( resultFuture, result, remainingFilter); } }); - } else { - resultFuture.completeExceptionally( - new UnsupportedOperationException( - "Unsupported Fluss lookup type. Currently, Fluss only " - + "support lookup by primary keys or prefix lookup by bucket keys.")); } } diff --git a/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/lookup/FlinkLookupFunction.java b/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/lookup/FlinkLookupFunction.java index 99bb3ced..20788e53 100644 --- a/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/lookup/FlinkLookupFunction.java +++ b/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/lookup/FlinkLookupFunction.java @@ -53,7 +53,6 @@ public class FlinkLookupFunction extends LookupFunction { private final TablePath tablePath; private final int maxRetryTimes; private final RowType flinkRowType; - private final int[] lookupKeyIndexes; private final LookupNormalizer lookupNormalizer; @Nullable private final int[] projection; private final LookupType flussLookupType; @@ -68,7 +67,6 @@ public FlinkLookupFunction( Configuration flussConfig, TablePath tablePath, RowType flinkRowType, - int[] lookupKeyIndexes, int maxRetryTimes, LookupNormalizer lookupNormalizer, @Nullable int[] projection) { @@ -76,10 +74,9 @@ public FlinkLookupFunction( this.tablePath = tablePath; this.maxRetryTimes = maxRetryTimes; this.flinkRowType = flinkRowType; - this.lookupKeyIndexes = lookupKeyIndexes; this.lookupNormalizer = lookupNormalizer; this.projection = projection; - this.flussLookupType = lookupNormalizer.getFlussLookupType(); + this.flussLookupType = lookupNormalizer.getLookupType(); } @Override @@ -88,6 +85,7 @@ public void open(FunctionContext context) { connection = ConnectionFactory.createConnection(flussConfig); table = connection.getTable(tablePath); // TODO: convert to Fluss GenericRow to avoid unnecessary deserialization + int[] lookupKeyIndexes = lookupNormalizer.getLookupKeyIndexes(); flinkRowToFlussRowConverter = FlinkRowToFlussRowConverter.create( FlinkUtils.projectRowType(flinkRowType, lookupKeyIndexes), diff --git a/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/lookup/LookupNormalizer.java b/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/lookup/LookupNormalizer.java index 199c684a..e720eaf1 100644 --- a/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/lookup/LookupNormalizer.java +++ b/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/source/lookup/LookupNormalizer.java @@ -16,13 +16,8 @@ package com.alibaba.fluss.connector.flink.source.lookup; -import com.alibaba.fluss.client.Connection; -import com.alibaba.fluss.client.ConnectionFactory; import com.alibaba.fluss.client.lookup.LookupType; -import com.alibaba.fluss.client.table.Table; -import com.alibaba.fluss.config.Configuration; -import com.alibaba.fluss.metadata.TableDescriptor; -import com.alibaba.fluss.metadata.TablePath; +import com.alibaba.fluss.utils.ArrayUtils; import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.GenericRowData; @@ -36,9 +31,9 @@ import java.io.Serializable; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashSet; import java.util.List; import java.util.Objects; -import java.util.Set; import java.util.stream.Collectors; import static com.alibaba.fluss.utils.Preconditions.checkState; @@ -58,9 +53,6 @@ public class LookupNormalizer implements Serializable { private static final long serialVersionUID = 1L; - public static final LookupNormalizer NOOP_NORMALIZER = - new LookupNormalizer(LookupType.LOOKUP, new int[0], null, null, null); - /** Mapping from normalized key index to the lookup key index (in the lookup row). */ @Nullable private final FieldGetter[] normalizedKeyGetters; @@ -70,16 +62,21 @@ public class LookupNormalizer implements Serializable { /** The field getter to get the remaining condition result from the lookup result row. */ @Nullable private final FieldGetter[] resultFieldGetters; - private final LookupType flussLookupType; + /** + * The lookup request type (primary key lookup or prefix lookup) requested from Flink to Fluss. + */ + private final LookupType lookupType; + + /** The indexes of the lookup keys in the lookup key row. */ private final int[] lookupKeyIndexes; private LookupNormalizer( - LookupType flussLookupType, + LookupType lookupType, int[] lookupKeyIndexes, @Nullable FieldGetter[] normalizedKeyGetters, @Nullable FieldGetter[] conditionFieldGetters, @Nullable FieldGetter[] resultFieldGetters) { - this.flussLookupType = flussLookupType; + this.lookupType = lookupType; this.lookupKeyIndexes = lookupKeyIndexes; this.normalizedKeyGetters = normalizedKeyGetters; this.conditionFieldGetters = conditionFieldGetters; @@ -92,14 +89,20 @@ private LookupNormalizer( } } - public LookupType getFlussLookupType() { - return flussLookupType; + /** + * Returns the lookup type (primary key lookup, or prefix key lookup) requested from Flink to + * Fluss. + */ + public LookupType getLookupType() { + return lookupType; } + /** Returns the indexes of the normalized lookup keys. */ public int[] getLookupKeyIndexes() { return lookupKeyIndexes; } + /** Normalize the lookup key row to match the request key and the key fields order. */ public RowData normalizeLookupKey(RowData lookupKey) { if (normalizedKeyGetters == null) { return lookupKey; @@ -163,108 +166,173 @@ public boolean fieldMatches(RowData result) { // -------------------------------------------------------------------------------------------- - /** Validate the lookup key indexes and primary keys, and create a {@link LookupNormalizer}. */ + /** Create a {@link LookupNormalizer} for primary key lookup. */ + public static LookupNormalizer createPrimaryKeyLookupNormalizer( + int[] primaryKeys, RowType schema) { + List primaryKeyNames = fieldNames(primaryKeys, schema); + return createLookupNormalizer( + primaryKeyNames, primaryKeyNames, primaryKeys, schema, LookupType.LOOKUP); + } + + /** + * Validate the lookup key indexes and primary keys, and create a {@link LookupNormalizer}. + * + * @param lookupKeyIndexes the indexes of the lookup keys in the table row + * @param primaryKeys the indexes of the primary keys of the table + * @param bucketKeys the indexes of the bucket keys of the table, must be a part of primary keys + * @param partitionKeys the indexes of the partition keys of the table, maybe empty if the table + * is not partitioned + * @param schema the schema of the table + */ public static LookupNormalizer validateAndCreateLookupNormalizer( int[][] lookupKeyIndexes, int[] primaryKeys, + int[] bucketKeys, + int[] partitionKeys, RowType schema, - TablePath tablePath, - Configuration flussConfig) { - int[] bucketKeys; - boolean supportPrefixLookup; - try (Connection connection = ConnectionFactory.createConnection(flussConfig); - Table table = connection.getTable(tablePath)) { - TableDescriptor descriptor = table.getDescriptor(); - bucketKeys = descriptor.getBucketKeyIndexes(); - supportPrefixLookup = - TableDescriptor.bucketKeysMatchPrefixLookupPattern( - descriptor.getSchema(), descriptor.getBucketKey()); - } catch (Exception e) { - throw new TableException( - "Failed execute validate and create lookup normalizer operation on Fluss table.", - e); - } - - if (primaryKeys.length == 0 || bucketKeys.length == 0) { + @Nullable int[] projectedFields) { + if (primaryKeys.length == 0) { throw new UnsupportedOperationException( - "Fluss lookup function only support lookup table with primary key or prefix lookup with bucket key."); + "Fluss lookup function only support lookup table with primary key."); + } + // bucket keys must not be empty + if (bucketKeys.length == 0 || !ArrayUtils.isSubset(primaryKeys, bucketKeys)) { + throw new IllegalArgumentException( + "Illegal bucket keys: " + + Arrays.toString(bucketKeys) + + ", must be a part of primary keys: " + + Arrays.toString(primaryKeys)); + } + // partition keys can be empty + if (partitionKeys.length != 0 && !ArrayUtils.isSubset(primaryKeys, partitionKeys)) { + throw new IllegalArgumentException( + "Illegal partition keys: " + + Arrays.toString(partitionKeys) + + ", must be a part of primary keys: " + + Arrays.toString(primaryKeys)); } + int[] lookupKeysBeforeProjection = new int[lookupKeyIndexes.length]; int[] lookupKeys = new int[lookupKeyIndexes.length]; - for (int i = 0; i < lookupKeys.length; i++) { + for (int i = 0; i < lookupKeysBeforeProjection.length; i++) { int[] innerKeyArr = lookupKeyIndexes[i]; checkArgument(innerKeyArr.length == 1, "Do not support nested lookup keys"); // lookupKeyIndexes passed by Flink is key indexed after projection pushdown, - // we do remaining condition filter on the projected row, so no remapping needed. + // we restore the lookup key indexes before pushdown to easier compare with primary + // keys. + if (projectedFields != null) { + lookupKeysBeforeProjection[i] = projectedFields[innerKeyArr[0]]; + } else { + lookupKeysBeforeProjection[i] = innerKeyArr[0]; + } lookupKeys[i] = innerKeyArr[0]; } + List lookupKeyNames = fieldNames(lookupKeysBeforeProjection, schema); + List primaryKeyNames = fieldNames(primaryKeys, schema); - if (supportPrefixLookup && lookupKeys.length == bucketKeys.length) { - // bucket key prefix lookup. - return createLookupNormalizer(lookupKeys, bucketKeys, schema, LookupType.PREFIX_LOOKUP); + if (new HashSet<>(lookupKeyNames).containsAll(primaryKeyNames)) { + // primary key lookup. + return createLookupNormalizer( + lookupKeyNames, primaryKeyNames, lookupKeys, schema, LookupType.LOOKUP); } else { - // Primary key lookup. - return createLookupNormalizer(lookupKeys, primaryKeys, schema, LookupType.LOOKUP); + // the encoding primary key is the primary key without partition keys. + int[] encodedPrimaryKeys = ArrayUtils.removeSet(primaryKeys, partitionKeys); + // the table support prefix lookup iff the bucket key is a prefix of the encoding pk + boolean supportPrefixLookup = ArrayUtils.isPrefix(encodedPrimaryKeys, bucketKeys); + if (supportPrefixLookup) { + // try to create prefix lookup normalizer + // TODO: support prefix lookup with arbitrary part of prefix of primary key + int[] expectedLookupKeys = + ArrayUtils.intersection( + primaryKeys, ArrayUtils.concat(bucketKeys, partitionKeys)); + return createLookupNormalizer( + lookupKeyNames, + fieldNames(expectedLookupKeys, schema), + lookupKeys, + schema, + LookupType.PREFIX_LOOKUP); + } else { + // throw exception for tables that doesn't support prefix lookup + throw new TableException( + "The Fluss lookup function supports lookup tables where the lookup keys include all primary keys, the primary keys are " + + primaryKeyNames + + ", but the lookup keys are " + + lookupKeyNames); + } } } - /** create a {@link LookupNormalizer}. */ + /** + * Create a {@link LookupNormalizer}. + * + *

Note: We compare string names rather than int index for better error message and + * readability, the length of lookup key and keys (primary key or index key) shouldn't be large, + * so the overhead is low. + * + * @param originalLookupKeys the original lookup keys that is determined by Flink. + * @param expectedLookupKeys the expected lookup keys to lookup Fluss. + */ private static LookupNormalizer createLookupNormalizer( - int[] lookupKeys, int[] keys, RowType schema, LookupType flussLookupType) { - // we compare string names rather than int index for better error message and readability, - // the length of lookup key and keys (primary key or index key) shouldn't be large, so the - // overhead is low. - String[] columnNames = schema.getFieldNames().toArray(new String[0]); - String[] keyNames = - Arrays.stream(keys).mapToObj(i -> columnNames[i]).toArray(String[]::new); - - // get the lookup keys - String[] lookupKeyNames = new String[lookupKeys.length]; - for (int i = 0; i < lookupKeyNames.length; i++) { - lookupKeyNames[i] = columnNames[lookupKeys[i]]; - } - - if (Arrays.equals(lookupKeys, keys)) { - return new LookupNormalizer(flussLookupType, keys, null, null, null); + List originalLookupKeys, + List expectedLookupKeys, + int[] lookupKeyIndexes, + RowType schema, + LookupType lookupType) { + if (originalLookupKeys.equals(expectedLookupKeys)) { + int[] normalizedLookupKeys = fieldIndexes(expectedLookupKeys, schema); + return new LookupNormalizer(lookupType, normalizedLookupKeys, null, null, null); } - FieldGetter[] normalizedKeyGetters = new FieldGetter[keys.length]; - for (int i = 0; i < keyNames.length; i++) { - LogicalType fieldType = schema.getTypeAt(keys[i]); - int lookupKeyIndex = findIndex(lookupKeyNames, keyNames[i]); - normalizedKeyGetters[i] = RowData.createFieldGetter(fieldType, lookupKeyIndex); + FieldGetter[] normalizedKeyGetters = new FieldGetter[expectedLookupKeys.size()]; + for (int i = 0; i < expectedLookupKeys.size(); i++) { + String expectedKey = expectedLookupKeys.get(i); + LogicalType fieldType = schema.getTypeAt(schema.getFieldIndex(expectedKey)); + int idxInLookupKey = findIndex(originalLookupKeys, expectedKey); + normalizedKeyGetters[i] = RowData.createFieldGetter(fieldType, idxInLookupKey); } - Set keySet = Arrays.stream(keys).boxed().collect(Collectors.toSet()); List conditionFieldGetters = new ArrayList<>(); List resultFieldGetters = new ArrayList<>(); - for (int i = 0; i < lookupKeys.length; i++) { - if (!keySet.contains(i)) { - LogicalType fieldType = schema.getTypeAt(lookupKeys[i]); + for (int i = 0; i < originalLookupKeys.size(); i++) { + String originalKey = originalLookupKeys.get(i); + if (!expectedLookupKeys.contains(originalKey)) { + LogicalType fieldType = schema.getTypeAt(schema.getFieldIndex(originalKey)); + // get the condition field from the original lookup key row conditionFieldGetters.add(RowData.createFieldGetter(fieldType, i)); - resultFieldGetters.add(RowData.createFieldGetter(fieldType, lookupKeys[i])); + // get the result field from the lookup result row (projected) + resultFieldGetters.add(RowData.createFieldGetter(fieldType, lookupKeyIndexes[i])); } } return new LookupNormalizer( - flussLookupType, - keys, + lookupType, + fieldIndexes(expectedLookupKeys, schema), normalizedKeyGetters, conditionFieldGetters.toArray(new FieldGetter[0]), resultFieldGetters.toArray(new FieldGetter[0])); } - private static int findIndex(String[] columnNames, String key) { - for (int i = 0; i < columnNames.length; i++) { - if (columnNames[i].equals(key)) { + private static int findIndex(List columnNames, String key) { + for (int i = 0; i < columnNames.size(); i++) { + if (columnNames.get(i).equals(key)) { return i; } } throw new TableException( - "Fluss lookup function only supports lookup table with lookup keys contain all primary keys or bucket keys." - + " Can't find primary key or bucket key '" + "The Fluss lookup function supports lookup tables where the lookup keys include all primary keys or all bucket keys." + + " Can't find expected key '" + key + "' in lookup keys " - + Arrays.toString(columnNames)); + + columnNames); + } + + private static List fieldNames(int[] fieldIndexes, RowType schema) { + return Arrays.stream(fieldIndexes) + .mapToObj(i -> schema.getFields().get(i).getName()) + .collect(Collectors.toList()); + } + + private static int[] fieldIndexes(List fieldNames, RowType schema) { + return fieldNames.stream().mapToInt(schema::getFieldIndex).toArray(); } } diff --git a/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/utils/FlinkConnectorOptionsUtils.java b/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/utils/FlinkConnectorOptionsUtils.java index 4a17ac9a..37b7a6f2 100644 --- a/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/utils/FlinkConnectorOptionsUtils.java +++ b/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/utils/FlinkConnectorOptionsUtils.java @@ -22,10 +22,12 @@ import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.table.types.logical.RowType; import java.time.LocalDateTime; import java.time.ZoneId; import java.time.format.DateTimeFormatter; +import java.util.Optional; import static com.alibaba.fluss.connector.flink.FlinkConnectorOptions.SCAN_STARTUP_MODE; import static com.alibaba.fluss.connector.flink.FlinkConnectorOptions.SCAN_STARTUP_TIMESTAMP; @@ -61,6 +63,30 @@ public static StartupOptions getStartupOptions(ReadableConfig tableOptions, Zone return options; } + public static int[] getBucketKeyIndexes(ReadableConfig tableOptions, RowType schema) { + Optional bucketKey = tableOptions.getOptional(FlinkConnectorOptions.BUCKET_KEY); + if (!bucketKey.isPresent()) { + // log tables don't have bucket key by default + return new int[0]; + } + + String[] keys = bucketKey.get().split(","); + int[] indexes = new int[keys.length]; + for (int i = 0; i < keys.length; i++) { + int index = schema.getFieldIndex(keys[i].trim()); + if (index < 0) { + throw new ValidationException( + String.format( + "Field '%s' not found in the schema. Available fields are: %s", + keys[i].trim(), schema.getFieldNames())); + } + indexes[i] = index; + } + return indexes; + } + + // ---------------------------------------------------------------------------------------- + private static void validateScanStartupMode(ReadableConfig tableOptions) { ScanStartupMode scanStartupMode = tableOptions.get(SCAN_STARTUP_MODE); if (scanStartupMode == TIMESTAMP) { diff --git a/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/utils/PushdownUtils.java b/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/utils/PushdownUtils.java index 841e551c..805197da 100644 --- a/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/utils/PushdownUtils.java +++ b/fluss-connectors/fluss-connector-flink/src/main/java/com/alibaba/fluss/connector/flink/utils/PushdownUtils.java @@ -68,6 +68,8 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; +import static com.alibaba.fluss.connector.flink.source.lookup.LookupNormalizer.createPrimaryKeyLookupNormalizer; + /** Utilities for pushdown abilities. */ public class PushdownUtils { private static final int LIMIT_PUSH_DOWN_CEIL = 1024; @@ -255,13 +257,13 @@ public static Collection querySingleRow( int[] primaryKeyIndexes, int lookupMaxRetryTimes, @Nullable int[] projectedFields) { - LookupNormalizer lookupNormalizer = LookupNormalizer.NOOP_NORMALIZER; + LookupNormalizer lookupNormalizer = + createPrimaryKeyLookupNormalizer(primaryKeyIndexes, sourceOutputType); LookupFunction lookupFunction = new FlinkLookupFunction( flussConfig, tablePath, sourceOutputType, - primaryKeyIndexes, lookupMaxRetryTimes, lookupNormalizer, projectedFields); diff --git a/fluss-connectors/fluss-connector-flink/src/test/java/com/alibaba/fluss/connector/flink/source/FlinkTableSourceITCase.java b/fluss-connectors/fluss-connector-flink/src/test/java/com/alibaba/fluss/connector/flink/source/FlinkTableSourceITCase.java index 19845bf3..9fcbb692 100644 --- a/fluss-connectors/fluss-connector-flink/src/test/java/com/alibaba/fluss/connector/flink/source/FlinkTableSourceITCase.java +++ b/fluss-connectors/fluss-connector-flink/src/test/java/com/alibaba/fluss/connector/flink/source/FlinkTableSourceITCase.java @@ -647,7 +647,7 @@ private static Stream lookupArgs() { @ParameterizedTest @MethodSource("lookupArgs") void testLookup1PkTable(Caching caching, boolean async) throws Exception { - String dim = prepareDimTableAndSourceTable(caching, async, new String[] {"id"}, null); + String dim = prepareDimTableAndSourceTable(caching, async, new String[] {"id"}, null, null); String dimJoinQuery = String.format( "SELECT a, c, h.name FROM src JOIN %s FOR SYSTEM_TIME AS OF src.proc as h" @@ -660,13 +660,30 @@ void testLookup1PkTable(Caching caching, boolean async) throws Exception { assertResultsIgnoreOrder(collected, expected, true); } + @ParameterizedTest + @MethodSource("lookupArgs") + void testLookupWithProjection(Caching caching, boolean async) throws Exception { + String dim = + prepareDimTableAndSourceTable(caching, async, new String[] {"name"}, null, null); + String dimJoinQuery = + String.format( + "SELECT a, c, h.address FROM src JOIN %s FOR SYSTEM_TIME AS OF src.proc as h" + + " ON src.b = h.name", + dim); + + CloseableIterator collected = tEnv.executeSql(dimJoinQuery).collect(); + List expected = + Arrays.asList("+I[1, 11, address5]", "+I[2, 2, address2]", "+I[10, 44, address4]"); + assertResultsIgnoreOrder(collected, expected, true); + } + /** * lookup table with one pk, two join condition and one of the join condition is constant value. */ @ParameterizedTest @MethodSource("lookupArgs") void testLookup1PkTableWith2Conditions(Caching caching, boolean async) throws Exception { - String dim = prepareDimTableAndSourceTable(caching, async, new String[] {"id"}, null); + String dim = prepareDimTableAndSourceTable(caching, async, new String[] {"id"}, null, null); String dimJoinQuery = String.format( "SELECT a, b, h.name FROM src JOIN %s FOR SYSTEM_TIME AS OF src.proc as h" @@ -696,7 +713,7 @@ void testLookup1PkTableWith2Conditions(Caching caching, boolean async) throws Ex @ParameterizedTest @MethodSource("lookupArgs") void testLookup1PkTableWith3Conditions(Caching caching, boolean async) throws Exception { - String dim = prepareDimTableAndSourceTable(caching, async, new String[] {"id"}, null); + String dim = prepareDimTableAndSourceTable(caching, async, new String[] {"id"}, null, null); String dimJoinQuery = String.format( "SELECT a, b, c, h.address FROM src LEFT JOIN %s FOR SYSTEM_TIME AS OF src.proc as h" @@ -709,7 +726,7 @@ void testLookup1PkTableWith3Conditions(Caching caching, boolean async) throws Ex "+I[1, name1, 11, null]", "+I[2, name2, 2, address2]", "+I[3, name33, 33, null]", - "+I[10, name4, 44, null]"); + "+I[10, name0, 44, null]"); assertResultsIgnoreOrder(collected, expected, true); } @@ -718,7 +735,8 @@ void testLookup1PkTableWith3Conditions(Caching caching, boolean async) throws Ex @MethodSource("lookupArgs") void testLookup2PkTable(Caching caching, boolean async) throws Exception { String dim = - prepareDimTableAndSourceTable(caching, async, new String[] {"id", "name"}, null); + prepareDimTableAndSourceTable( + caching, async, new String[] {"id", "name"}, null, null); String dimJoinQuery = String.format( "SELECT a, b, h.address FROM src JOIN %s FOR SYSTEM_TIME AS OF src.proc as h" @@ -739,7 +757,8 @@ void testLookup2PkTable(Caching caching, boolean async) throws Exception { void testLookup2PkTableWithUnorderedKey(Caching caching, boolean async) throws Exception { // the primary key is (name, id) but the schema order is (id, name) String dim = - prepareDimTableAndSourceTable(caching, async, new String[] {"name", "id"}, null); + prepareDimTableAndSourceTable( + caching, async, new String[] {"name", "id"}, null, null); String dimJoinQuery = String.format( "SELECT a, b, h.address FROM src JOIN %s FOR SYSTEM_TIME AS OF src.proc as h" @@ -759,7 +778,8 @@ void testLookup2PkTableWithUnorderedKey(Caching caching, boolean async) throws E @MethodSource("lookupArgs") void testLookup2PkTableWith1KeyInCondition(Caching caching, boolean async) throws Exception { String dim = - prepareDimTableAndSourceTable(caching, async, new String[] {"id", "name"}, null); + prepareDimTableAndSourceTable( + caching, async, new String[] {"id", "name"}, null, null); String dimJoinQuery = String.format( "SELECT a, b, h.address FROM src JOIN %s FOR SYSTEM_TIME AS OF src.proc as h" @@ -767,9 +787,9 @@ void testLookup2PkTableWith1KeyInCondition(Caching caching, boolean async) throw dim); assertThatThrownBy(() -> tEnv.executeSql(dimJoinQuery)) .hasStackTraceContaining( - "Fluss lookup function only supports lookup table with " - + "lookup keys contain all primary keys or bucket keys. Can't find primary " - + "key or bucket key 'name' in lookup keys [id]"); + "The Fluss lookup function supports lookup tables where" + + " the lookup keys include all primary keys or all bucket keys." + + " Can't find expected key 'name' in lookup keys [id]"); } /** @@ -780,7 +800,8 @@ void testLookup2PkTableWith1KeyInCondition(Caching caching, boolean async) throw @MethodSource("lookupArgs") void testLookup2PkTableWith3Conditions(Caching caching, boolean async) throws Exception { String dim = - prepareDimTableAndSourceTable(caching, async, new String[] {"id", "name"}, null); + prepareDimTableAndSourceTable( + caching, async, new String[] {"id", "name"}, null, null); String dimJoinQuery = String.format( "SELECT a, h.name, h.address FROM src JOIN %s FOR SYSTEM_TIME AS OF src.proc as h" @@ -795,7 +816,8 @@ void testLookup2PkTableWith3Conditions(Caching caching, boolean async) throws Ex @ParameterizedTest @MethodSource("lookupArgs") void testLookupPartitionedTable(Caching caching, boolean async) throws Exception { - String dim = prepareDimTableAndSourceTable(caching, async, new String[] {"id"}, "p_date"); + String dim = + prepareDimTableAndSourceTable(caching, async, new String[] {"id"}, null, "p_date"); String dimJoinQuery = String.format( @@ -808,6 +830,45 @@ void testLookupPartitionedTable(Caching caching, boolean async) throws Exception assertResultsIgnoreOrder(collected, expected, true); } + @ParameterizedTest + @MethodSource("lookupArgs") + void testPrefixLookup(Caching caching, boolean async) throws Exception { + String dim = + prepareDimTableAndSourceTable( + caching, async, new String[] {"name", "id"}, new String[] {"name"}, null); + String dimJoinQuery = + String.format( + "SELECT a, b, h.address FROM src JOIN %s FOR SYSTEM_TIME AS OF src.proc as h" + + " ON src.b = h.name", + dim); + + CloseableIterator collected = tEnv.executeSql(dimJoinQuery).collect(); + List expected = + Arrays.asList( + "+I[1, name1, address1]", + "+I[1, name1, address5]", + "+I[2, name2, address2]", + "+I[10, name0, address4]"); + assertResultsIgnoreOrder(collected, expected, true); + } + + @ParameterizedTest + @MethodSource("lookupArgs") + void testPrefixLookupWithCondition(Caching caching, boolean async) throws Exception { + String dim = + prepareDimTableAndSourceTable( + caching, async, new String[] {"name", "id"}, new String[] {"name"}, null); + String dimJoinQuery = + String.format( + "SELECT a, b, h.address FROM src JOIN %s FOR SYSTEM_TIME AS OF src.proc as h" + + " ON src.b = h.name AND h.address = 'address5'", + dim); + + CloseableIterator collected = tEnv.executeSql(dimJoinQuery).collect(); + List expected = Collections.singletonList("+I[1, name1, address5]"); + assertResultsIgnoreOrder(collected, expected, true); + } + @Test void testLookupFullCacheThrowException() { tEnv.executeSql( @@ -840,7 +901,11 @@ private InternalRow genRow(boolean isPkTable, RowType rowType, Object[] objects) * @return the table name of the dim table */ private String prepareDimTableAndSourceTable( - Caching caching, boolean async, String[] keys, @Nullable String partitionedKey) + Caching caching, + boolean async, + String[] primaryKeys, + @Nullable String[] bucketKeys, + @Nullable String partitionedKey) throws Exception { String options = async ? "'lookup.async' = 'true'" : "'lookup.async' = 'false'"; if (caching == Caching.ENABLE_CACHE) { @@ -849,6 +914,12 @@ private String prepareDimTableAndSourceTable( + ",'lookup.partial-cache.max-rows' = '1000'" + ",'lookup.partial-cache.expire-after-write' = '10min'"; } + String bucketOptions = + bucketKeys == null + ? "" + : ", 'bucket.num' = '1', 'bucket.key' = '" + + String.join(",", bucketKeys) + + "'"; // create dim table String tableName = @@ -856,7 +927,7 @@ private String prepareDimTableAndSourceTable( "lookup_test_%s_%s_pk_%s_%s", caching.name().toLowerCase(), async ? "async" : "sync", - String.join("_", keys), + String.join("_", primaryKeys), RandomUtils.nextInt()); if (partitionedKey == null) { tEnv.executeSql( @@ -865,8 +936,8 @@ private String prepareDimTableAndSourceTable( + " id int not null," + " address varchar," + " name varchar," - + " primary key (%s) NOT ENFORCED) with (%s)", - tableName, String.join(",", keys), options)); + + " primary key (%s) NOT ENFORCED) with (%s %s)", + tableName, String.join(",", primaryKeys), options, bucketOptions)); } else { tEnv.executeSql( String.format( @@ -876,13 +947,15 @@ private String prepareDimTableAndSourceTable( + " name varchar," + " %s varchar , " + " primary key (%s, %s) NOT ENFORCED) partitioned by (%s) with (%s , " - + "'table.auto-partition.enabled' = 'true', 'table.auto-partition.time-unit' = 'year')", + + " 'table.auto-partition.enabled' = 'true', 'table.auto-partition.time-unit' = 'year'" + + " %s)", tableName, partitionedKey, - String.join(",", keys), + String.join(",", primaryKeys), partitionedKey, partitionedKey, - options)); + options, + bucketOptions)); } TablePath tablePath = TablePath.of(DEFAULT_DB, tableName); @@ -904,8 +977,8 @@ private String prepareDimTableAndSourceTable( for (int i = 1; i <= 5; i++) { Object[] values = partition1 == null - ? new Object[] {i, "address" + i, "name" + i} - : new Object[] {i, "address" + i, "name" + i, partition1}; + ? new Object[] {i, "address" + i, "name" + i % 4} + : new Object[] {i, "address" + i, "name" + i % 4, partition1}; upsertWriter.upsert(compactedRow(dimTableRowType, values)); } upsertWriter.flush(); @@ -918,12 +991,12 @@ private String prepareDimTableAndSourceTable( Row.of(1, "name1", 11), Row.of(2, "name2", 2), Row.of(3, "name33", 33), - Row.of(10, "name4", 44)) + Row.of(10, "name0", 44)) : Arrays.asList( Row.of(1, "name1", 11, partition1), Row.of(2, "name2", 2, partition1), Row.of(3, "name33", 33, partition2), - Row.of(10, "name4", 44, partition2)); + Row.of(10, "name0", 44, partition2)); Schema.Builder builder = Schema.newBuilder() .column("a", DataTypes.INT()) diff --git a/fluss-connectors/fluss-connector-flink/src/test/java/com/alibaba/fluss/connector/flink/source/lookup/FlinkLookupFunctionTest.java b/fluss-connectors/fluss-connector-flink/src/test/java/com/alibaba/fluss/connector/flink/source/lookup/FlinkLookupFunctionTest.java index 471f5ea2..4d3e74a0 100644 --- a/fluss-connectors/fluss-connector-flink/src/test/java/com/alibaba/fluss/connector/flink/source/lookup/FlinkLookupFunctionTest.java +++ b/fluss-connectors/fluss-connector-flink/src/test/java/com/alibaba/fluss/connector/flink/source/lookup/FlinkLookupFunctionTest.java @@ -21,11 +21,11 @@ import com.alibaba.fluss.connector.flink.source.testutils.FlinkTestBase; import com.alibaba.fluss.connector.flink.utils.FlinkConversions; import com.alibaba.fluss.metadata.TablePath; -import com.alibaba.fluss.types.RowType; import org.apache.flink.table.connector.source.lookup.LookupOptions; import org.apache.flink.table.data.RowData; import org.apache.flink.table.functions.AsyncLookupFunction; +import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.Collector; import org.junit.jupiter.api.Test; @@ -37,6 +37,7 @@ import java.util.concurrent.CountDownLatch; import java.util.stream.Collectors; +import static com.alibaba.fluss.connector.flink.source.lookup.LookupNormalizer.createPrimaryKeyLookupNormalizer; import static com.alibaba.fluss.testutils.DataTestUtils.compactedRow; import static org.assertj.core.api.Assertions.assertThat; @@ -48,14 +49,14 @@ void testSyncLookupEval() throws Exception { TablePath tablePath = TablePath.of(DEFAULT_DB, "sync-lookup-table"); prepareData(tablePath, 3); + RowType flinkRowType = FlinkConversions.toFlinkRowType(DEFAULT_PK_TABLE_SCHEMA.toRowType()); FlinkLookupFunction lookupFunction = new FlinkLookupFunction( clientConf, tablePath, - FlinkConversions.toFlinkRowType(DEFAULT_PK_TABLE_SCHEMA.toRowType()), - new int[] {0}, + flinkRowType, LookupOptions.MAX_RETRIES.defaultValue(), - LookupNormalizer.NOOP_NORMALIZER, + createPrimaryKeyLookupNormalizer(new int[] {0}, flinkRowType), null); ListOutputCollector collector = new ListOutputCollector(); @@ -84,14 +85,15 @@ void testAsyncLookupEval() throws Exception { TablePath tablePath = TablePath.of(DEFAULT_DB, "async-lookup-table"); int rows = 3; prepareData(tablePath, rows); + + RowType flinkRowType = FlinkConversions.toFlinkRowType(DEFAULT_PK_TABLE_SCHEMA.toRowType()); AsyncLookupFunction asyncLookupFunction = new FlinkAsyncLookupFunction( clientConf, tablePath, - FlinkConversions.toFlinkRowType(DEFAULT_PK_TABLE_SCHEMA.toRowType()), - new int[] {0}, + flinkRowType, LookupOptions.MAX_RETRIES.defaultValue(), - LookupNormalizer.NOOP_NORMALIZER, + createPrimaryKeyLookupNormalizer(new int[] {0}, flinkRowType), null); asyncLookupFunction.open(null); @@ -134,7 +136,7 @@ void testAsyncLookupEval() throws Exception { private void prepareData(TablePath tablePath, int rows) throws Exception { createTable(tablePath, DEFAULT_PK_TABLE_DESCRIPTOR); - RowType rowType = DEFAULT_PK_TABLE_SCHEMA.toRowType(); + com.alibaba.fluss.types.RowType rowType = DEFAULT_PK_TABLE_SCHEMA.toRowType(); // first write some data to the table try (Table table = conn.getTable(tablePath)) { diff --git a/fluss-rpc/src/main/java/com/alibaba/fluss/rpc/entity/PrefixLookupResultForBucket.java b/fluss-rpc/src/main/java/com/alibaba/fluss/rpc/entity/PrefixLookupResultForBucket.java index 508401eb..e520bccf 100644 --- a/fluss-rpc/src/main/java/com/alibaba/fluss/rpc/entity/PrefixLookupResultForBucket.java +++ b/fluss-rpc/src/main/java/com/alibaba/fluss/rpc/entity/PrefixLookupResultForBucket.java @@ -35,7 +35,7 @@ public PrefixLookupResultForBucket(TableBucket tableBucket, ApiError error) { this(tableBucket, null, error); } - public PrefixLookupResultForBucket( + private PrefixLookupResultForBucket( TableBucket tableBucket, List> values, ApiError error) { super(tableBucket, error); this.values = values; diff --git a/fluss-rpc/src/main/java/com/alibaba/fluss/rpc/gateway/TabletServerGateway.java b/fluss-rpc/src/main/java/com/alibaba/fluss/rpc/gateway/TabletServerGateway.java index efc96fde..85e8f362 100644 --- a/fluss-rpc/src/main/java/com/alibaba/fluss/rpc/gateway/TabletServerGateway.java +++ b/fluss-rpc/src/main/java/com/alibaba/fluss/rpc/gateway/TabletServerGateway.java @@ -102,9 +102,9 @@ CompletableFuture notifyLeaderAndIsr( CompletableFuture lookup(LookupRequest request); /** - * Prefix lookup to get value by index key. + * Prefix lookup to get value by prefix key. * - * @return Index lookup response. + * @return Prefix lookup response. */ @RPC(api = ApiKeys.PREFIX_LOOKUP) CompletableFuture prefixLookup(PrefixLookupRequest request); diff --git a/fluss-rpc/src/main/java/com/alibaba/fluss/rpc/netty/server/RequestsMetrics.java b/fluss-rpc/src/main/java/com/alibaba/fluss/rpc/netty/server/RequestsMetrics.java index 74741c49..ea223c8a 100644 --- a/fluss-rpc/src/main/java/com/alibaba/fluss/rpc/netty/server/RequestsMetrics.java +++ b/fluss-rpc/src/main/java/com/alibaba/fluss/rpc/netty/server/RequestsMetrics.java @@ -94,7 +94,7 @@ private static String toRequestName(ApiKeys apiKeys, boolean isFromFollower) { case LOOKUP: return "lookup"; case PREFIX_LOOKUP: - return "indexLookup"; + return "prefixLookup"; case FETCH_LOG: return isFromFollower ? "fetchLogFollower" : "fetchLogClient"; default: diff --git a/fluss-server/src/main/java/com/alibaba/fluss/server/kv/rocksdb/RocksDBKv.java b/fluss-server/src/main/java/com/alibaba/fluss/server/kv/rocksdb/RocksDBKv.java index 2f3fb8c3..b8fb107f 100644 --- a/fluss-server/src/main/java/com/alibaba/fluss/server/kv/rocksdb/RocksDBKv.java +++ b/fluss-server/src/main/java/com/alibaba/fluss/server/kv/rocksdb/RocksDBKv.java @@ -106,10 +106,6 @@ public List prefixLookup(byte[] prefixKey) { RocksIterator iterator = db.newIterator(defaultColumnFamilyHandle, readOptions); try { iterator.seek(prefixKey); - // TODO, This is very inefficient to compare arrays byte by byte. In the future we can - // use JDK9 Arrays.compare(compare(byte[] a, int aFromIndex, int aToIndex, byte[] b, int - // bFromIndex, int bToIndex)) to instead. See issue: - // https://github.com/alibaba/fluss/issues/271 while (iterator.isValid() && isPrefixEquals(prefixKey, iterator.key())) { pkList.add(iterator.value()); iterator.next(); @@ -210,20 +206,23 @@ public RocksDB getDb() { } /** - * Check if the given two byte arrays have the same prefix. If bytes2 is shorter than bytes1, - * return false. Otherwise, compare bytes1 and bytes2 from the start until the end of bytes2. If - * all bytes are equal, return true. + * Check if the given first byte array ({@code prefix}) is a prefix of the second byte array + * ({@code bytes}). * - * @param bytes1 The first byte array - * @param bytes2 The second byte array - * @return true if the given two byte arrays have the same prefix, false otherwise + * @param prefix The prefix byte array + * @param bytes The byte array to check if it has the prefix + * @return true if the given bytes has the given prefix, false otherwise */ - public static boolean isPrefixEquals(byte[] bytes1, byte[] bytes2) { - if (bytes1.length > bytes2.length) { + public static boolean isPrefixEquals(byte[] prefix, byte[] bytes) { + // TODO, This is very inefficient to compare arrays byte by byte. In the future we can + // use JDK9 Arrays.compare(compare(byte[] a, int aFromIndex, int aToIndex, byte[] b, int + // bFromIndex, int bToIndex)) to instead. See issue: + // https://github.com/alibaba/fluss/issues/271 + if (prefix.length > bytes.length) { return false; } - for (int i = 0; i < bytes1.length; i++) { - if (bytes1[i] != bytes2[i]) { + for (int i = 0; i < prefix.length; i++) { + if (prefix[i] != bytes[i]) { return false; } } diff --git a/fluss-server/src/main/java/com/alibaba/fluss/server/metrics/group/PhysicalTableMetricGroup.java b/fluss-server/src/main/java/com/alibaba/fluss/server/metrics/group/PhysicalTableMetricGroup.java index ec3ba43d..5d651d80 100644 --- a/fluss-server/src/main/java/com/alibaba/fluss/server/metrics/group/PhysicalTableMetricGroup.java +++ b/fluss-server/src/main/java/com/alibaba/fluss/server/metrics/group/PhysicalTableMetricGroup.java @@ -368,7 +368,7 @@ public KvMetricGroup(PhysicalTableMetricGroup physicalTableMetricGroup) { MetricNames.FAILED_LIMIT_SCAN_REQUESTS_RATE, new MeterView(failedLimitScanRequests)); - // for index lookup request + // for prefix lookup request totalPrefixLookupRequests = new ThreadSafeSimpleCounter(); meter( MetricNames.TOTAL_PREFIX_LOOKUP_REQUESTS_RATE, diff --git a/fluss-server/src/main/java/com/alibaba/fluss/server/replica/Replica.java b/fluss-server/src/main/java/com/alibaba/fluss/server/replica/Replica.java index 08c2aaa1..77dcba13 100644 --- a/fluss-server/src/main/java/com/alibaba/fluss/server/replica/Replica.java +++ b/fluss-server/src/main/java/com/alibaba/fluss/server/replica/Replica.java @@ -166,7 +166,6 @@ public final class Replica { private final KvFormat kvFormat; private final long logTTLMs; private final boolean dataLakeEnabled; - private final boolean supportPrefixLookup; private final int tieredLogLocalSegments; private final AtomicReference leaderReplicaIdOpt = new AtomicReference<>(); private final ReadWriteLock leaderIsrUpdateLock = new ReentrantReadWriteLock(); @@ -232,7 +231,6 @@ public Replica( this.snapshotContext = snapshotContext; // create a closeable registry for the replica this.closeableRegistry = new CloseableRegistry(); - this.supportPrefixLookup = supportPrefixLookup(tableDescriptor); this.logTablet = createLog(lazyHighWatermarkCheckpoint); registerMetrics(); @@ -309,10 +307,6 @@ public long getLogTTLMs() { return logTTLMs; } - public boolean supportPrefixLookup() { - return supportPrefixLookup; - } - public int writerIdCount() { return logTablet.getWriterIdCount(); } @@ -1735,11 +1729,6 @@ private void traceAckInfo(List curMaximalIsr, long requiredOffset) { .collect(Collectors.toList())); } - private boolean supportPrefixLookup(TableDescriptor tableDescriptor) { - return TableDescriptor.bucketKeysMatchPrefixLookupPattern( - tableDescriptor.getSchema(), tableDescriptor.getBucketKey()); - } - @VisibleForTesting public int getBucketEpoch() { return bucketEpoch; diff --git a/fluss-server/src/main/java/com/alibaba/fluss/server/replica/ReplicaManager.java b/fluss-server/src/main/java/com/alibaba/fluss/server/replica/ReplicaManager.java index 6d31b703..957f549c 100644 --- a/fluss-server/src/main/java/com/alibaba/fluss/server/replica/ReplicaManager.java +++ b/fluss-server/src/main/java/com/alibaba/fluss/server/replica/ReplicaManager.java @@ -24,7 +24,6 @@ import com.alibaba.fluss.exception.FlussRuntimeException; import com.alibaba.fluss.exception.InvalidCoordinatorException; import com.alibaba.fluss.exception.InvalidRequiredAcksException; -import com.alibaba.fluss.exception.KvStorageException; import com.alibaba.fluss.exception.LogOffsetOutOfRangeException; import com.alibaba.fluss.exception.LogStorageException; import com.alibaba.fluss.exception.NotLeaderOrFollowerException; @@ -469,25 +468,13 @@ public void prefixLookups( List> resultForBucket = new ArrayList<>(); try { Replica replica = getReplicaOrException(tb); - if (!replica.supportPrefixLookup()) { - result.put( - tb, - new PrefixLookupResultForBucket( - tb, - ApiError.fromThrowable( - new KvStorageException( - "Table bucket " - + tb - + " does not support prefix lookup")))); - continue; - } - tableMetrics = replica.tableMetrics(); tableMetrics.totalPrefixLookupRequests().inc(); for (byte[] prefixKey : entry.getValue()) { - List resultForPerKey = new ArrayList<>(replica.prefixLookup(prefixKey)); + List resultForPerKey = replica.prefixLookup(prefixKey); resultForBucket.add(resultForPerKey); } + result.put(tb, new PrefixLookupResultForBucket(tb, resultForBucket)); } catch (Exception e) { if (isUnexpectedException(e)) { LOG.error("Error processing prefix lookup operation on replica {}", tb, e); @@ -497,8 +484,6 @@ public void prefixLookups( } result.put(tb, new PrefixLookupResultForBucket(tb, ApiError.fromThrowable(e))); } - - result.put(tb, new PrefixLookupResultForBucket(tb, resultForBucket)); } responseCallback.accept(result); } diff --git a/fluss-server/src/test/java/com/alibaba/fluss/server/replica/ReplicaManagerTest.java b/fluss-server/src/test/java/com/alibaba/fluss/server/replica/ReplicaManagerTest.java index b60f8845..e438ee13 100644 --- a/fluss-server/src/test/java/com/alibaba/fluss/server/replica/ReplicaManagerTest.java +++ b/fluss-server/src/test/java/com/alibaba/fluss/server/replica/ReplicaManagerTest.java @@ -74,7 +74,7 @@ import static com.alibaba.fluss.record.TestData.DATA1; import static com.alibaba.fluss.record.TestData.DATA1_KEY_TYPE; import static com.alibaba.fluss.record.TestData.DATA1_ROW_TYPE; -import static com.alibaba.fluss.record.TestData.DATA1_SCHEMA_PK; +import static com.alibaba.fluss.record.TestData.DATA1_SCHEMA; import static com.alibaba.fluss.record.TestData.DATA1_TABLE_ID; import static com.alibaba.fluss.record.TestData.DATA1_TABLE_ID_PK; import static com.alibaba.fluss.record.TestData.DATA1_TABLE_PATH; @@ -674,17 +674,16 @@ void testPrefixLookup() throws Exception { Arrays.asList(prefixKey1Bytes, prefixKey2Bytes), Arrays.asList(key1ExpectedValues, key2ExpectedValues)); - // Prefix lookup an unsupported prefixLookup table. - tablePath = TablePath.of("test_db_1", "test_unsupported_prefix_lookup_t1"); + // Prefix lookup an unsupported prefixLookup table (a log table). tableId = registerTableInZkClient( - tablePath, - DATA1_SCHEMA_PK, - 200L, + DATA1_TABLE_PATH, + DATA1_SCHEMA, + 2001L, Collections.emptyList(), Collections.emptyMap()); TableBucket tb3 = new TableBucket(tableId, 0); - makeKvTableAsLeader(tableId, tablePath, tb3.getBucket()); + makeLogTableAsLeader(tb3, false); replicaManager.prefixLookups( Collections.singletonMap(tb3, Collections.singletonList(prefixKey2Bytes)), (prefixLookupResultForBuckets) -> { @@ -692,9 +691,11 @@ void testPrefixLookup() throws Exception { prefixLookupResultForBuckets.get(tb3); assertThat(lookupResultForBucket.failed()).isTrue(); ApiError apiError = lookupResultForBucket.getError(); - assertThat(apiError.error()).isEqualTo(Errors.KV_STORAGE_EXCEPTION); + assertThat(apiError.error()).isEqualTo(Errors.NON_PRIMARY_KEY_TABLE_EXCEPTION); assertThat(apiError.message()) - .isEqualTo("Table bucket " + tb3 + " does not support prefix lookup"); + .isEqualTo( + "Try to do prefix lookup on a non primary key table: " + + DATA1_TABLE_PATH); }); } diff --git a/fluss-server/src/test/java/com/alibaba/fluss/server/tablet/TabletServiceITCase.java b/fluss-server/src/test/java/com/alibaba/fluss/server/tablet/TabletServiceITCase.java index 717a884e..24c6e97f 100644 --- a/fluss-server/src/test/java/com/alibaba/fluss/server/tablet/TabletServiceITCase.java +++ b/fluss-server/src/test/java/com/alibaba/fluss/server/tablet/TabletServiceITCase.java @@ -493,13 +493,12 @@ void testPrefixLookup() throws Exception { Arrays.asList(key1ExpectedValues, key2ExpectedValues)); // Prefix lookup an unsupported prefixLookup table. - TableDescriptor unsupportedDescriptor = TableDescriptor.builder().schema(schema).build(); - long tableId2 = + long logTableId = createTable( FLUSS_CLUSTER_EXTENSION, - TablePath.of("test_db_1", "test_unsupported_prefix_lookup_t1"), - unsupportedDescriptor); - tb = new TableBucket(tableId2, 0); + DATA1_TABLE_PATH, + DATA1_TABLE_INFO.getTableDescriptor()); + tb = new TableBucket(logTableId, 0); FLUSS_CLUSTER_EXTENSION.waitUtilAllReplicaReady(tb); leader = FLUSS_CLUSTER_EXTENSION.waitAndGetLeader(tb); TabletServerGateway leaderGateWay2 = @@ -508,15 +507,13 @@ void testPrefixLookup() throws Exception { leaderGateWay2 .prefixLookup( newPrefixLookupRequest( - tableId2, 0, Collections.singletonList(prefixKey1Bytes))) + logTableId, 0, Collections.singletonList(prefixKey1Bytes))) .get() .getBucketsRespAt(0); verifyPrefixLookupBucketError( pbPrefixLookupRespForBucket, - Errors.KV_STORAGE_EXCEPTION, - "Table bucket TableBucket{tableId=" - + tableId2 - + ", bucket=0} does not support prefix lookup"); + Errors.NON_PRIMARY_KEY_TABLE_EXCEPTION, + "Try to do prefix lookup on a non primary key table: " + DATA1_TABLE_PATH); } @Test diff --git a/website/docs/maintenance/monitor-metrics.md b/website/docs/maintenance/monitor-metrics.md index 9078c106..7304a7e2 100644 --- a/website/docs/maintenance/monitor-metrics.md +++ b/website/docs/maintenance/monitor-metrics.md @@ -501,13 +501,13 @@ Some metrics might not be exposed when using other JVM implementations (e.g. IBM Meter - totalIndexLookupRequestsPerSecond - The number of index lookup requests to index lookup value by key from this table per second. + totalPrefixLookupRequestsPerSecond + The number of prefix lookup requests to lookup value by prefix key from this table per second. Meter - failedIndexLookupRequestsPerSecond - The number of failed index lookup requests to index lookup value by key from this table per second. + failedPrefixLookupRequestsPerSecond + The number of failed prefix lookup requests to lookup value by prefix key from this table per second. Meter