From 4155cf75b973e3eaaa93979fd4548c9fdaa3c1f4 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Sun, 18 Jan 2026 22:12:06 +0200 Subject: [PATCH 01/14] add snapshot query fucntionality --- .../fluss/client/table/scanner/TableScan.java | 12 +- .../table/scanner/batch/KvBatchScanner.java | 285 ++++++++++++++++++ .../apache/fluss/config/ConfigOptions.java | 16 + .../org/apache/fluss/config/TableConfig.java | 5 + .../exception/ScannerNotFoundException.java | 27 ++ .../rpc/gateway/TabletServerGateway.java | 22 ++ .../apache/fluss/rpc/protocol/ApiKeys.java | 38 +-- .../org/apache/fluss/rpc/protocol/Errors.java | 4 +- .../fluss/server/kv/rocksdb/RocksDBKv.java | 13 + .../fluss/server/kv/scan/ScannerContext.java | 101 +++++++ .../fluss/server/kv/scan/ScannerManager.java | 161 ++++++++++ .../fluss/server/tablet/TabletServer.java | 7 + .../fluss/server/tablet/TabletService.java | 149 +++++++++ 13 files changed, 820 insertions(+), 20 deletions(-) create mode 100644 fluss-client/src/main/java/org/apache/fluss/client/table/scanner/batch/KvBatchScanner.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/exception/ScannerNotFoundException.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerContext.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java index 0d1d28a0e2..bcd5a2ec5b 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableScan.java @@ -21,6 +21,7 @@ import org.apache.fluss.client.admin.Admin; import org.apache.fluss.client.metadata.KvSnapshotMetadata; import org.apache.fluss.client.table.scanner.batch.BatchScanner; +import org.apache.fluss.client.table.scanner.batch.KvBatchScanner; import org.apache.fluss.client.table.scanner.batch.KvSnapshotBatchScanner; import org.apache.fluss.client.table.scanner.batch.LimitBatchScanner; import org.apache.fluss.client.table.scanner.log.LogScanner; @@ -123,10 +124,19 @@ public TypedLogScanner createTypedLogScanner(Class pojoClass) { @Override public BatchScanner createBatchScanner(TableBucket tableBucket) { + if (tableInfo.hasPrimaryKey()) { + return new KvBatchScanner( + tableInfo, + tableBucket, + schemaGetter, + conn.getMetadataUpdater(), + projectedColumns, + limit == null ? null : (long) limit); + } if (limit == null) { throw new UnsupportedOperationException( String.format( - "Currently, BatchScanner is only available when limit is set. Table: %s, bucket: %s", + "Currently, for log table, BatchScanner is only available when limit is set. Table: %s, bucket: %s", tableInfo.getTablePath(), tableBucket)); } return new LimitBatchScanner( diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/batch/KvBatchScanner.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/batch/KvBatchScanner.java new file mode 100644 index 0000000000..b84419576f --- /dev/null +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/batch/KvBatchScanner.java @@ -0,0 +1,285 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.fluss.client.table.scanner.batch; + +import org.apache.fluss.client.metadata.MetadataUpdater; +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.exception.LeaderNotAvailableException; +import org.apache.fluss.metadata.KvFormat; +import org.apache.fluss.metadata.SchemaGetter; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TableInfo; +import org.apache.fluss.record.DefaultValueRecordBatch; +import org.apache.fluss.record.ValueRecord; +import org.apache.fluss.record.ValueRecordReadContext; +import org.apache.fluss.row.GenericRow; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.row.ProjectedRow; +import org.apache.fluss.rpc.gateway.TabletServerGateway; +import org.apache.fluss.rpc.messages.PbScanReqForBucket; +import org.apache.fluss.rpc.messages.ScanKvRequest; +import org.apache.fluss.rpc.messages.ScanKvResponse; +import org.apache.fluss.rpc.messages.ScannerKeepAliveRequest; +import org.apache.fluss.rpc.protocol.Errors; +import org.apache.fluss.types.RowType; +import org.apache.fluss.utils.CloseableIterator; +import org.apache.fluss.utils.SchemaUtil; +import org.apache.fluss.utils.concurrent.ExecutorThreadFactory; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +/** A {@link BatchScanner} implementation that scans records from a primary key table. */ +public class KvBatchScanner implements BatchScanner { + private static final Logger LOG = LoggerFactory.getLogger(KvBatchScanner.class); + + private final TableInfo tableInfo; + private final TableBucket tableBucket; + private final SchemaGetter schemaGetter; + private final MetadataUpdater metadataUpdater; + @Nullable private final int[] projectedFields; + @Nullable private final Long limit; + private final int targetSchemaId; + private final InternalRow.FieldGetter[] fieldGetters; + private final KvFormat kvFormat; + private final int batchSizeBytes; + + private final Map schemaProjectionCache = new HashMap<>(); + + private byte[] scannerId; + private int callSeqId = 0; + private boolean hasMoreResults = true; + private boolean isClosed = false; + + private CompletableFuture inFlightRequest; + private ScheduledExecutorService keepAliveExecutor; + + public KvBatchScanner( + TableInfo tableInfo, + TableBucket tableBucket, + SchemaGetter schemaGetter, + MetadataUpdater metadataUpdater, + @Nullable int[] projectedFields, + @Nullable Long limit) { + this.tableInfo = tableInfo; + this.tableBucket = tableBucket; + this.schemaGetter = schemaGetter; + this.metadataUpdater = metadataUpdater; + this.projectedFields = projectedFields; + this.limit = limit; + this.targetSchemaId = tableInfo.getSchemaId(); + this.kvFormat = tableInfo.getTableConfig().getKvFormat(); + this.batchSizeBytes = + (int) + tableInfo + .getTableConfig() + .getConfiguration() + .get(ConfigOptions.CLIENT_SCANNER_KV_FETCH_MAX_BYTES) + .getBytes(); + + RowType rowType = tableInfo.getRowType(); + this.fieldGetters = new InternalRow.FieldGetter[rowType.getFieldCount()]; + for (int i = 0; i < rowType.getFieldCount(); i++) { + this.fieldGetters[i] = InternalRow.createFieldGetter(rowType.getTypeAt(i), i); + } + } + + @Nullable + @Override + public CloseableIterator pollBatch(Duration timeout) throws IOException { + if (isClosed || (!hasMoreResults && inFlightRequest == null)) { + return null; + } + + try { + if (inFlightRequest == null) { + sendRequest(); + } + + ScanKvResponse response = + inFlightRequest.get(timeout.toMillis(), TimeUnit.MILLISECONDS); + inFlightRequest = null; + + if (response.hasErrorCode() && response.getErrorCode() != Errors.NONE.code()) { + throw Errors.forCode(response.getErrorCode()).exception(response.getErrorMessage()); + } + + this.scannerId = response.getScannerId(); + this.hasMoreResults = response.isHasMoreResults(); + this.callSeqId++; + + List rows = parseScanKvResponse(response); + + // pipeline: send next request if there are more results + if (hasMoreResults) { + sendRequest(); + } + + return CloseableIterator.wrap(rows.iterator()); + } catch (java.util.concurrent.TimeoutException e) { + return CloseableIterator.emptyIterator(); + } catch (Exception e) { + throw new IOException(e); + } + } + + private void sendRequest() { + int leader = metadataUpdater.leaderFor(tableInfo.getTablePath(), tableBucket); + TabletServerGateway gateway = metadataUpdater.newTabletServerClientForNode(leader); + if (gateway == null) { + throw new LeaderNotAvailableException( + "Server " + leader + " is not found in metadata cache."); + } + + ScanKvRequest request = new ScanKvRequest(); + request.setBatchSizeBytes(batchSizeBytes); + if (scannerId == null) { + PbScanReqForBucket bucketScanReq = request.setBucketScanReq(); + bucketScanReq.setTableId(tableBucket.getTableId()).setBucketId(tableBucket.getBucket()); + if (tableBucket.getPartitionId() != null) { + bucketScanReq.setPartitionId(tableBucket.getPartitionId()); + } + if (limit != null) { + bucketScanReq.setLimit(limit); + } + request.setCallSeqId(0); + } else { + request.setScannerId(scannerId).setCallSeqId(callSeqId); + } + + inFlightRequest = gateway.scanKv(request); + } + + private List parseScanKvResponse(ScanKvResponse response) { + if (!response.hasRecords()) { + return Collections.emptyList(); + } + + List scanRows = new ArrayList<>(); + ByteBuffer recordsBuffer = ByteBuffer.wrap(response.getRecords()); + DefaultValueRecordBatch valueRecords = + DefaultValueRecordBatch.pointToByteBuffer(recordsBuffer); + ValueRecordReadContext readContext = + ValueRecordReadContext.createReadContext(schemaGetter, kvFormat); + + for (ValueRecord record : valueRecords.records(readContext)) { + InternalRow row = record.getRow(); + if (targetSchemaId != record.schemaId()) { + int[] indexMapping = + schemaProjectionCache.computeIfAbsent( + record.schemaId(), + sourceSchemaId -> + SchemaUtil.getIndexMapping( + schemaGetter.getSchema(sourceSchemaId), + schemaGetter.getSchema(targetSchemaId))); + row = ProjectedRow.from(indexMapping).replaceRow(row); + } + scanRows.add(maybeProject(row)); + } + return scanRows; + } + + private InternalRow maybeProject(InternalRow originRow) { + GenericRow newRow = new GenericRow(fieldGetters.length); + for (int i = 0; i < fieldGetters.length; i++) { + newRow.setField(i, fieldGetters[i].getFieldOrNull(originRow)); + } + if (projectedFields != null) { + ProjectedRow projectedRow = ProjectedRow.from(projectedFields); + projectedRow.replaceRow(newRow); + return projectedRow; + } else { + return newRow; + } + } + + public void startKeepAlivePeriodically(int keepAliveIntervalMs) { + if (keepAliveExecutor != null) { + return; + } + + keepAliveExecutor = + Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory("kv-scanner-keep-alive-" + tableBucket)); + keepAliveExecutor.scheduleAtFixedRate( + this::sendKeepAlive, + keepAliveIntervalMs, + keepAliveIntervalMs, + TimeUnit.MILLISECONDS); + } + + private void sendKeepAlive() { + if (isClosed || scannerId == null || !hasMoreResults) { + return; + } + + try { + int leader = metadataUpdater.leaderFor(tableInfo.getTablePath(), tableBucket); + TabletServerGateway gateway = metadataUpdater.newTabletServerClientForNode(leader); + if (gateway != null) { + gateway.scannerKeepAlive(new ScannerKeepAliveRequest().setScannerId(scannerId)); + } + } catch (Exception e) { + LOG.warn("Failed to send keep alive for scanner {}", tableBucket, e); + } + } + + @Override + public void close() throws IOException { + if (isClosed) { + return; + } + isClosed = true; + + if (keepAliveExecutor != null) { + keepAliveExecutor.shutdownNow(); + } + + if (scannerId != null && hasMoreResults) { + // Close scanner on server + int leader = metadataUpdater.leaderFor(tableInfo.getTablePath(), tableBucket); + TabletServerGateway gateway = metadataUpdater.newTabletServerClientForNode(leader); + if (gateway != null) { + gateway.scanKv( + new ScanKvRequest() + .setScannerId(scannerId) + .setCallSeqId(callSeqId) + .setCloseScanner(true)); + } + } + + if (inFlightRequest != null) { + inFlightRequest.cancel(true); + } + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java b/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java index 7e04c3a1cc..b429856d56 100644 --- a/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java +++ b/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java @@ -439,6 +439,14 @@ public class ConfigOptions { + WRITER_ID_EXPIRATION_TIME.key() + " passing. The default value is 10 minutes."); + public static final ConfigOption SERVER_SCANNER_TTL = + key("server.scanner.ttl") + .durationType() + .defaultValue(Duration.ofMinutes(10)) + .withDescription( + "The time that the tablet server will wait without receiving any scan request from " + + "a client before expiring the related status. The default value is 10 minutes."); + public static final ConfigOption TABLET_SERVER_CONTROLLED_SHUTDOWN_MAX_RETRIES = key("tablet-server.controlled-shutdown.max-retries") .intType() @@ -1093,6 +1101,14 @@ public class ConfigOptions { + "will still be returned to ensure that the fetch can make progress. As such, " + "this is not a absolute maximum."); + public static final ConfigOption CLIENT_SCANNER_KV_FETCH_MAX_BYTES = + key("client.scanner.kv.fetch.max-bytes") + .memoryType() + .defaultValue(MemorySize.parse("1mb")) + .withDescription( + "The maximum amount of data the server should return for a kv scan request. " + + "The default value is 1mb."); + public static final ConfigOption CLIENT_SCANNER_LOG_FETCH_MAX_BYTES_FOR_BUCKET = key("client.scanner.log.fetch.max-bytes-for-bucket") .memoryType() diff --git a/fluss-common/src/main/java/org/apache/fluss/config/TableConfig.java b/fluss-common/src/main/java/org/apache/fluss/config/TableConfig.java index 984a1def4a..b972a80ff8 100644 --- a/fluss-common/src/main/java/org/apache/fluss/config/TableConfig.java +++ b/fluss-common/src/main/java/org/apache/fluss/config/TableConfig.java @@ -50,6 +50,11 @@ public TableConfig(Configuration config) { this.config = config; } + /** Gets the table properties configuration. */ + public Configuration getConfiguration() { + return config; + } + /** Gets the replication factor of the table. */ public int getReplicationFactor() { return config.get(ConfigOptions.TABLE_REPLICATION_FACTOR); diff --git a/fluss-common/src/main/java/org/apache/fluss/exception/ScannerNotFoundException.java b/fluss-common/src/main/java/org/apache/fluss/exception/ScannerNotFoundException.java new file mode 100644 index 0000000000..e146cb3b09 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/exception/ScannerNotFoundException.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.fluss.exception; + +/** Exception thrown when a scanner is not found. */ +public class ScannerNotFoundException extends ApiException { + private static final long serialVersionUID = 1L; + + public ScannerNotFoundException(String message) { + super(message); + } +} diff --git a/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/TabletServerGateway.java b/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/TabletServerGateway.java index 578b74e5e2..436637edd0 100644 --- a/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/TabletServerGateway.java +++ b/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/TabletServerGateway.java @@ -42,6 +42,10 @@ import org.apache.fluss.rpc.messages.ProduceLogResponse; import org.apache.fluss.rpc.messages.PutKvRequest; import org.apache.fluss.rpc.messages.PutKvResponse; +import org.apache.fluss.rpc.messages.ScanKvRequest; +import org.apache.fluss.rpc.messages.ScanKvResponse; +import org.apache.fluss.rpc.messages.ScannerKeepAliveRequest; +import org.apache.fluss.rpc.messages.ScannerKeepAliveResponse; import org.apache.fluss.rpc.messages.StopReplicaRequest; import org.apache.fluss.rpc.messages.StopReplicaResponse; import org.apache.fluss.rpc.messages.UpdateMetadataRequest; @@ -130,6 +134,24 @@ CompletableFuture notifyLeaderAndIsr( @RPC(api = ApiKeys.LIMIT_SCAN) CompletableFuture limitScan(LimitScanRequest request); + /** + * Scan kv data from the specified table bucket. + * + * @param request the scan kv request + * @return the scan kv response + */ + @RPC(api = ApiKeys.SCAN_KV) + CompletableFuture scanKv(ScanKvRequest request); + + /** + * Keep alive for the specified scanner. + * + * @param request the scanner keep alive request + * @return the scanner keep alive response + */ + @RPC(api = ApiKeys.SCANNER_KEEP_ALIVE) + CompletableFuture scannerKeepAlive(ScannerKeepAliveRequest request); + /** * List offsets for the specified table bucket. * diff --git a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java index cc033ba8a9..105b3e0a9d 100644 --- a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java +++ b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java @@ -63,24 +63,26 @@ public enum ApiKeys { GET_LATEST_LAKE_SNAPSHOT(1032, 0, 0, PUBLIC), LIMIT_SCAN(1033, 0, 0, PUBLIC), PREFIX_LOOKUP(1034, 0, 0, PUBLIC), - GET_DATABASE_INFO(1035, 0, 0, PUBLIC), - CREATE_PARTITION(1036, 0, 0, PUBLIC), - DROP_PARTITION(1037, 0, 0, PUBLIC), - AUTHENTICATE(1038, 0, 0, PUBLIC), - CREATE_ACLS(1039, 0, 0, PUBLIC), - LIST_ACLS(1040, 0, 0, PUBLIC), - DROP_ACLS(1041, 0, 0, PUBLIC), - LAKE_TIERING_HEARTBEAT(1042, 0, 0, PRIVATE), - CONTROLLED_SHUTDOWN(1043, 0, 0, PRIVATE), - ALTER_TABLE(1044, 0, 0, PUBLIC), - DESCRIBE_CLUSTER_CONFIGS(1045, 0, 0, PUBLIC), - ALTER_CLUSTER_CONFIGS(1046, 0, 0, PUBLIC), - ADD_SERVER_TAG(1047, 0, 0, PUBLIC), - REMOVE_SERVER_TAG(1048, 0, 0, PUBLIC), - REBALANCE(1049, 0, 0, PUBLIC), - LIST_REBALANCE_PROGRESS(1050, 0, 0, PUBLIC), - CANCEL_REBALANCE(1051, 0, 0, PUBLIC), - PREPARE_LAKE_TABLE_SNAPSHOT(1052, 0, 0, PRIVATE); + SCAN_KV(1035, 0, 0, PUBLIC), + SCANNER_KEEP_ALIVE(1036, 0, 0, PUBLIC), + GET_DATABASE_INFO(1037, 0, 0, PUBLIC), + CREATE_PARTITION(1038, 0, 0, PUBLIC), + DROP_PARTITION(1039, 0, 0, PUBLIC), + AUTHENTICATE(1040, 0, 0, PUBLIC), + CREATE_ACLS(1041, 0, 0, PUBLIC), + LIST_ACLS(1042, 0, 0, PUBLIC), + DROP_ACLS(1043, 0, 0, PUBLIC), + LAKE_TIERING_HEARTBEAT(1044, 0, 0, PRIVATE), + CONTROLLED_SHUTDOWN(1045, 0, 0, PRIVATE), + ALTER_TABLE(1046, 0, 0, PUBLIC), + DESCRIBE_CLUSTER_CONFIGS(1047, 0, 0, PUBLIC), + ALTER_CLUSTER_CONFIGS(1048, 0, 0, PUBLIC), + ADD_SERVER_TAG(1049, 0, 0, PUBLIC), + REMOVE_SERVER_TAG(1050, 0, 0, PUBLIC), + REBALANCE(1051, 0, 0, PUBLIC), + LIST_REBALANCE_PROGRESS(1052, 0, 0, PUBLIC), + CANCEL_REBALANCE(1053, 0, 0, PUBLIC), + PREPARE_LAKE_TABLE_SNAPSHOT(1054, 0, 0, PRIVATE); private static final Map ID_TO_TYPE = Arrays.stream(ApiKeys.values()) diff --git a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java index 5ee652cce2..a6aa827cb6 100644 --- a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java +++ b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java @@ -64,6 +64,7 @@ import org.apache.fluss.exception.RebalanceFailureException; import org.apache.fluss.exception.RecordTooLargeException; import org.apache.fluss.exception.RetriableAuthenticationException; +import org.apache.fluss.exception.ScannerNotFoundException; import org.apache.fluss.exception.SchemaNotExistException; import org.apache.fluss.exception.SecurityDisabledException; import org.apache.fluss.exception.SecurityTokenException; @@ -240,7 +241,8 @@ public enum Errors { SEVER_TAG_NOT_EXIST_EXCEPTION(60, "The server tag not exist.", ServerTagNotExistException::new), REBALANCE_FAILURE_EXCEPTION(61, "The rebalance task failure.", RebalanceFailureException::new), NO_REBALANCE_IN_PROGRESS_EXCEPTION( - 62, "No rebalance task in progress.", NoRebalanceInProgressException::new); + 62, "No rebalance task in progress.", NoRebalanceInProgressException::new), + SCANNER_NOT_FOUND_EXCEPTION(63, "The scanner is not found.", ScannerNotFoundException::new); private static final Logger LOG = LoggerFactory.getLogger(Errors.class); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/rocksdb/RocksDBKv.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/rocksdb/RocksDBKv.java index f3998f4435..d7ab654151 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/rocksdb/RocksDBKv.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/rocksdb/RocksDBKv.java @@ -32,6 +32,7 @@ import org.rocksdb.RocksDB; import org.rocksdb.RocksDBException; import org.rocksdb.RocksIterator; +import org.rocksdb.Snapshot; import org.rocksdb.Statistics; import org.rocksdb.WriteOptions; @@ -151,6 +152,18 @@ public List limitScan(Integer limit) { return pkList; } + public Snapshot getSnapshot() { + return db.getSnapshot(); + } + + public void releaseSnapshot(Snapshot snapshot) { + db.releaseSnapshot(snapshot); + } + + public RocksIterator newIterator(ReadOptions readOptions) { + return db.newIterator(defaultColumnFamilyHandle, readOptions); + } + public void put(byte[] key, byte[] value) throws IOException { try { db.put(writeOptions, key, value); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerContext.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerContext.java new file mode 100644 index 0000000000..e74097d1f6 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerContext.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.fluss.server.kv.scan; + +import org.apache.fluss.server.kv.rocksdb.RocksDBKv; +import org.apache.fluss.utils.clock.Clock; + +import org.rocksdb.ReadOptions; +import org.rocksdb.RocksIterator; +import org.rocksdb.Snapshot; + +import javax.annotation.concurrent.NotThreadSafe; + +/** The context for a scanner. */ +@NotThreadSafe +public class ScannerContext implements AutoCloseable { + private final RocksDBKv rocksDBKv; + private final RocksIterator iterator; + private final ReadOptions readOptions; + private final Snapshot snapshot; + private final long limit; + + private int callSeqId; + private long lastAccessTime; + private long rowsScanned; + + public ScannerContext( + RocksDBKv rocksDBKv, + RocksIterator iterator, + ReadOptions readOptions, + Snapshot snapshot, + long limit, + Clock clock) { + this.rocksDBKv = rocksDBKv; + this.iterator = iterator; + this.readOptions = readOptions; + this.snapshot = snapshot; + this.limit = limit; + this.callSeqId = 0; + this.lastAccessTime = clock.milliseconds(); + this.rowsScanned = 0; + } + + public RocksIterator getIterator() { + return iterator; + } + + public Snapshot getSnapshot() { + return snapshot; + } + + public long getLimit() { + return limit; + } + + public int getCallSeqId() { + return callSeqId; + } + + public void setCallSeqId(int callSeqId) { + this.callSeqId = callSeqId; + } + + public long getLastAccessTime() { + return lastAccessTime; + } + + public void updateLastAccessTime(long lastAccessTime) { + this.lastAccessTime = lastAccessTime; + } + + public long getRowsScanned() { + return rowsScanned; + } + + public void incrementRowsScanned(long count) { + this.rowsScanned += count; + } + + @Override + public void close() { + iterator.close(); + readOptions.close(); + rocksDBKv.releaseSnapshot(snapshot); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java new file mode 100644 index 0000000000..4d2b8f0e48 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.fluss.server.kv.scan; + +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.rpc.protocol.Errors; +import org.apache.fluss.server.kv.KvTablet; +import org.apache.fluss.server.kv.rocksdb.RocksDBKv; +import org.apache.fluss.utils.AutoCloseableAsync; +import org.apache.fluss.utils.clock.Clock; +import org.apache.fluss.utils.clock.SystemClock; +import org.apache.fluss.utils.concurrent.ExecutorThreadFactory; +import org.apache.fluss.utils.concurrent.FutureUtils; + +import org.rocksdb.ReadOptions; +import org.rocksdb.RocksIterator; +import org.rocksdb.Snapshot; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +/** A manager for scanners. */ +public class ScannerManager implements AutoCloseableAsync { + private static final Logger LOG = LoggerFactory.getLogger(ScannerManager.class); + + private final Map scanners = new ConcurrentHashMap<>(); + private final ScheduledExecutorService cleanupExecutor; + private final Clock clock; + private final long scannerTtlMs; + + public ScannerManager(Configuration conf) { + this(conf, SystemClock.getInstance()); + } + + public ScannerManager(Configuration conf, Clock clock) { + this.clock = clock; + this.scannerTtlMs = conf.get(ConfigOptions.SERVER_SCANNER_TTL).toMillis(); + this.cleanupExecutor = + Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory("scanner-cleanup-thread")); + this.cleanupExecutor.scheduleWithFixedDelay( + this::cleanupExpiredScanners, + scannerTtlMs / 2, + scannerTtlMs / 2, + TimeUnit.MILLISECONDS); + } + + public byte[] createScanner(KvTablet kvTablet, long limit) { + RocksDBKv rocksDBKv = kvTablet.getRocksDBKv(); + Snapshot snapshot = rocksDBKv.getSnapshot(); + ReadOptions readOptions = new ReadOptions().setSnapshot(snapshot); + RocksIterator iterator = rocksDBKv.newIterator(readOptions); + iterator.seekToFirst(); + + ScannerContext context = + new ScannerContext(rocksDBKv, iterator, readOptions, snapshot, limit, clock); + byte[] scannerId = generateScannerId(); + scanners.put(ByteBuffer.wrap(scannerId), context); + return scannerId; + } + + public ScannerContext getScanner(byte[] scannerId) { + ScannerContext context = scanners.get(ByteBuffer.wrap(scannerId)); + if (context != null) { + context.updateLastAccessTime(clock.milliseconds()); + } + return context; + } + + public void keepAlive(byte[] scannerId) { + ScannerContext context = scanners.get(ByteBuffer.wrap(scannerId)); + if (context != null) { + context.updateLastAccessTime(clock.milliseconds()); + } else { + throw Errors.SCANNER_NOT_FOUND_EXCEPTION.exception( + "Scanner not found: " + scannerIdToString(scannerId)); + } + } + + public void removeScanner(byte[] scannerId) { + ScannerContext context = scanners.remove(ByteBuffer.wrap(scannerId)); + if (context != null) { + closeScannerContext(context); + } + } + + private void cleanupExpiredScanners() { + long now = clock.milliseconds(); + for (Map.Entry entry : scanners.entrySet()) { + if (now - entry.getValue().getLastAccessTime() > scannerTtlMs) { + ScannerContext context = scanners.remove(entry.getKey()); + if (context != null) { + LOG.info( + "Scanner {} expired, closing it.", + scannerIdToString(entry.getKey().array())); + closeScannerContext(context); + } + } + } + } + + private void closeScannerContext(ScannerContext context) { + try { + context.close(); + } catch (Exception e) { + LOG.error("Fail to close scanner context.", e); + } + } + + private byte[] generateScannerId() { + return UUID.randomUUID().toString().replace("-", "").getBytes(StandardCharsets.UTF_8); + } + + private String scannerIdToString(byte[] scannerId) { + return new String(scannerId, StandardCharsets.UTF_8); + } + + @Override + public CompletableFuture closeAsync() { + try { + close(); + return CompletableFuture.completedFuture(null); + } catch (Exception e) { + return FutureUtils.completedExceptionally(e); + } + } + + @Override + public void close() throws Exception { + cleanupExecutor.shutdownNow(); + for (ScannerContext context : scanners.values()) { + closeScannerContext(context); + } + scanners.clear(); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletServer.java b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletServer.java index 8eed63c844..dc98e5b8ca 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletServer.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletServer.java @@ -41,6 +41,7 @@ import org.apache.fluss.server.coordinator.LakeCatalogDynamicLoader; import org.apache.fluss.server.coordinator.MetadataManager; import org.apache.fluss.server.kv.KvManager; +import org.apache.fluss.server.kv.scan.ScannerManager; import org.apache.fluss.server.kv.snapshot.DefaultCompletedKvSnapshotCommitter; import org.apache.fluss.server.log.LogManager; import org.apache.fluss.server.log.remote.RemoteLogManager; @@ -125,6 +126,9 @@ public class TabletServer extends ServerBase { @GuardedBy("lock") private TabletService tabletService; + @GuardedBy("lock") + private ScannerManager scannerManager; + @GuardedBy("lock") private MetricRegistry metricRegistry; @@ -230,6 +234,8 @@ protected void startServices() throws Exception { this.kvManager = KvManager.create(conf, zkClient, logManager, tabletServerMetricGroup); kvManager.startup(); + this.scannerManager = new ScannerManager(conf, clock); + // Register kvManager to dynamicConfigManager for dynamic reconfiguration dynamicConfigManager.register(kvManager); // Start dynamicConfigManager after all reconfigurable components are registered @@ -286,6 +292,7 @@ protected void startServices() throws Exception { metadataManager, authorizer, dynamicConfigManager, + scannerManager, ioExecutor); RequestsMetrics requestsMetrics = diff --git a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java index 992b963334..66f6650d96 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java @@ -19,10 +19,14 @@ import org.apache.fluss.cluster.ServerType; import org.apache.fluss.exception.AuthorizationException; +import org.apache.fluss.exception.FlussRuntimeException; +import org.apache.fluss.exception.NonPrimaryKeyTableException; +import org.apache.fluss.exception.NotLeaderOrFollowerException; import org.apache.fluss.exception.UnknownTableOrBucketException; import org.apache.fluss.fs.FileSystem; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.record.DefaultValueRecordBatch; import org.apache.fluss.record.KvRecordBatch; import org.apache.fluss.record.MemoryLogRecords; import org.apache.fluss.rpc.entity.FetchLogResultForBucket; @@ -50,12 +54,17 @@ import org.apache.fluss.rpc.messages.NotifyLeaderAndIsrResponse; import org.apache.fluss.rpc.messages.NotifyRemoteLogOffsetsRequest; import org.apache.fluss.rpc.messages.NotifyRemoteLogOffsetsResponse; +import org.apache.fluss.rpc.messages.PbScanReqForBucket; import org.apache.fluss.rpc.messages.PrefixLookupRequest; import org.apache.fluss.rpc.messages.PrefixLookupResponse; import org.apache.fluss.rpc.messages.ProduceLogRequest; import org.apache.fluss.rpc.messages.ProduceLogResponse; import org.apache.fluss.rpc.messages.PutKvRequest; import org.apache.fluss.rpc.messages.PutKvResponse; +import org.apache.fluss.rpc.messages.ScanKvRequest; +import org.apache.fluss.rpc.messages.ScanKvResponse; +import org.apache.fluss.rpc.messages.ScannerKeepAliveRequest; +import org.apache.fluss.rpc.messages.ScannerKeepAliveResponse; import org.apache.fluss.rpc.messages.StopReplicaRequest; import org.apache.fluss.rpc.messages.StopReplicaResponse; import org.apache.fluss.rpc.messages.UpdateMetadataRequest; @@ -71,16 +80,23 @@ import org.apache.fluss.server.entity.FetchReqInfo; import org.apache.fluss.server.entity.NotifyLeaderAndIsrData; import org.apache.fluss.server.entity.UserContext; +import org.apache.fluss.server.kv.scan.ScannerContext; +import org.apache.fluss.server.kv.scan.ScannerManager; import org.apache.fluss.server.log.FetchParams; import org.apache.fluss.server.log.ListOffsetsParam; import org.apache.fluss.server.metadata.TabletServerMetadataCache; import org.apache.fluss.server.metadata.TabletServerMetadataProvider; +import org.apache.fluss.server.replica.Replica; import org.apache.fluss.server.replica.ReplicaManager; import org.apache.fluss.server.utils.ServerRpcMessageUtils; import org.apache.fluss.server.zk.ZooKeeperClient; +import org.rocksdb.RocksIterator; + import javax.annotation.Nullable; +import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.Collection; import java.util.HashMap; import java.util.List; @@ -127,6 +143,7 @@ public final class TabletService extends RpcServiceBase implements TabletServerG private final ReplicaManager replicaManager; private final TabletServerMetadataCache metadataCache; private final TabletServerMetadataProvider metadataFunctionProvider; + private final ScannerManager scannerManager; public TabletService( int serverId, @@ -137,6 +154,7 @@ public TabletService( MetadataManager metadataManager, @Nullable Authorizer authorizer, DynamicConfigManager dynamicConfigManager, + ScannerManager scannerManager, ExecutorService ioExecutor) { super( remoteFileSystem, @@ -149,6 +167,7 @@ public TabletService( this.serviceName = "server-" + serverId; this.replicaManager = replicaManager; this.metadataCache = metadataCache; + this.scannerManager = scannerManager; this.metadataFunctionProvider = new TabletServerMetadataProvider(zkClient, metadataManager, metadataCache); } @@ -285,6 +304,136 @@ public CompletableFuture limitScan(LimitScanRequest request) return response; } + @Override + public CompletableFuture scanKv(ScanKvRequest request) { + CompletableFuture response = new CompletableFuture<>(); + try { + if (request.hasScannerId()) { + byte[] scannerId = request.getScannerId(); + ScannerContext context = scannerManager.getScanner(scannerId); + if (context == null) { + throw Errors.SCANNER_NOT_FOUND_EXCEPTION.exception( + "Scanner not found: " + new String(scannerId, StandardCharsets.UTF_8)); + } + + if (request.hasCloseScanner() && request.isCloseScanner()) { + scannerManager.removeScanner(scannerId); + ScanKvResponse scanResponse = new ScanKvResponse(); + scanResponse.setScannerId(scannerId).setHasMoreResults(false); + response.complete(scanResponse); + return response; + } + + // check call seq id + if (request.getCallSeqId() != context.getCallSeqId() + 1) { + throw new FlussRuntimeException( + "Out of order scan request. Expected call seq id: " + + (context.getCallSeqId() + 1) + + ", but got: " + + request.getCallSeqId()); + } + context.setCallSeqId(request.getCallSeqId()); + + response.complete(continueScan(scannerId, context, request.getBatchSizeBytes())); + } else { + PbScanReqForBucket bucketScanReq = request.getBucketScanReq(); + authorizeTable(READ, bucketScanReq.getTableId()); + + TableBucket tb = + new TableBucket( + bucketScanReq.getTableId(), + bucketScanReq.hasPartitionId() + ? bucketScanReq.getPartitionId() + : null, + bucketScanReq.getBucketId()); + Replica replica = replicaManager.getReplicaOrException(tb); + if (!replica.isLeader()) { + throw new NotLeaderOrFollowerException("Leader not local for bucket " + tb); + } + if (!replica.isKvTable()) { + throw new NonPrimaryKeyTableException( + "Table " + bucketScanReq.getTableId() + " is not a primary key table."); + } + + long limit = bucketScanReq.hasLimit() ? bucketScanReq.getLimit() : Long.MAX_VALUE; + byte[] scannerId = scannerManager.createScanner(replica.getKvTablet(), limit); + ScannerContext context = scannerManager.getScanner(scannerId); + + ScanKvResponse scanResponse = + continueScan(scannerId, context, request.getBatchSizeBytes()); + // The FIP says: "Returns the corresponding log offset at the time the scanner is + // created" + // We can use the high watermark or the current log end offset. + scanResponse.setLogOffset(replica.getLogHighWatermark()); + response.complete(scanResponse); + } + } catch (Exception e) { + response.complete(makeScanKvErrorResponse(e)); + } + return response; + } + + private ScanKvResponse continueScan( + byte[] scannerId, ScannerContext context, int batchSizeBytes) throws IOException { + RocksIterator iterator = context.getIterator(); + DefaultValueRecordBatch.Builder builder = DefaultValueRecordBatch.builder(); + int currentBytes = 0; + long rowsScannedInThisBatch = 0; + + while (iterator.isValid() + && context.getRowsScanned() + rowsScannedInThisBatch < context.getLimit()) { + byte[] value = iterator.value(); + // Check if adding this record would exceed batch size. + // But we must add at least one record. + if (rowsScannedInThisBatch > 0 && currentBytes + value.length > batchSizeBytes) { + break; + } + + builder.append(value); + currentBytes += value.length; + rowsScannedInThisBatch++; + iterator.next(); + } + + context.incrementRowsScanned(rowsScannedInThisBatch); + boolean hasMore = iterator.isValid() && context.getRowsScanned() < context.getLimit(); + + ScanKvResponse response = new ScanKvResponse(); + response.setScannerId(scannerId).setHasMoreResults(hasMore); + if (rowsScannedInThisBatch > 0) { + DefaultValueRecordBatch batch = builder.build(); + byte[] records = new byte[batch.sizeInBytes()]; + batch.getSegment().get(0, records); + response.setRecords(records); + } + + if (!hasMore) { + scannerManager.removeScanner(scannerId); + } + + return response; + } + + private ScanKvResponse makeScanKvErrorResponse(Throwable e) { + ScanKvResponse response = new ScanKvResponse(); + ApiError error = ApiError.fromThrowable(e); + response.setErrorCode(error.error().code()).setErrorMessage(error.message()); + return response; + } + + @Override + public CompletableFuture scannerKeepAlive( + ScannerKeepAliveRequest request) { + ScannerKeepAliveResponse response = new ScannerKeepAliveResponse(); + try { + scannerManager.keepAlive(request.getScannerId()); + } catch (Exception e) { + ApiError error = ApiError.fromThrowable(e); + response.setErrorCode(error.error().code()).setErrorMessage(error.message()); + } + return CompletableFuture.completedFuture(response); + } + @Override public CompletableFuture notifyLeaderAndIsr( NotifyLeaderAndIsrRequest notifyLeaderAndIsrRequest) { From 60bae8ed5bb43253128a27d5bda0be077f709c0c Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Sun, 18 Jan 2026 22:44:21 +0200 Subject: [PATCH 02/14] update api and add tests --- .../apache/fluss/client/table/FlussTable.java | 11 + .../org/apache/fluss/client/table/Table.java | 7 + .../client/table/scanner/SnapshotQuery.java | 59 ++++ .../table/scanner/TableSnapshotQuery.java | 152 ++++++++++ .../scanner/batch/KvBatchScannerITCase.java | 283 ++++++++++++++++++ .../rpc/TestingTabletGatewayService.java | 15 + .../tablet/TestTabletServerGateway.java | 15 + 7 files changed, 542 insertions(+) create mode 100644 fluss-client/src/main/java/org/apache/fluss/client/table/scanner/SnapshotQuery.java create mode 100644 fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableSnapshotQuery.java create mode 100644 fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvBatchScannerITCase.java diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/FlussTable.java b/fluss-client/src/main/java/org/apache/fluss/client/table/FlussTable.java index 8532f2a856..79cffa999d 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/FlussTable.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/FlussTable.java @@ -23,7 +23,9 @@ import org.apache.fluss.client.lookup.TableLookup; import org.apache.fluss.client.metadata.ClientSchemaGetter; import org.apache.fluss.client.table.scanner.Scan; +import org.apache.fluss.client.table.scanner.SnapshotQuery; import org.apache.fluss.client.table.scanner.TableScan; +import org.apache.fluss.client.table.scanner.TableSnapshotQuery; import org.apache.fluss.client.table.writer.Append; import org.apache.fluss.client.table.writer.TableAppend; import org.apache.fluss.client.table.writer.TableUpsert; @@ -67,6 +69,15 @@ public Scan newScan() { return new TableScan(conn, tableInfo, schemaGetter); } + @Override + public SnapshotQuery newSnapshotQuery() { + checkState( + hasPrimaryKey, + "Table %s is not a Primary Key Table and doesn't support SnapshotQuery.", + tablePath); + return new TableSnapshotQuery(conn, tableInfo, schemaGetter); + } + @Override public Lookup newLookup() { return new TableLookup( diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/Table.java b/fluss-client/src/main/java/org/apache/fluss/client/table/Table.java index 813b62034a..287311e9ae 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/Table.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/Table.java @@ -22,6 +22,7 @@ import org.apache.fluss.client.lookup.Lookup; import org.apache.fluss.client.lookup.Lookuper; import org.apache.fluss.client.table.scanner.Scan; +import org.apache.fluss.client.table.scanner.SnapshotQuery; import org.apache.fluss.client.table.writer.Append; import org.apache.fluss.client.table.writer.AppendWriter; import org.apache.fluss.client.table.writer.Upsert; @@ -55,6 +56,12 @@ public interface Table extends AutoCloseable { */ Scan newScan(); + /** + * Creates a new {@link SnapshotQuery} for this table to configure and execute a snapshot query + * to read all current data in a table bucket (requires to be a Primary Key Table). + */ + SnapshotQuery newSnapshotQuery(); + /** * Creates a new {@link Lookup} for this table to configure and create a {@link Lookuper} to * lookup data for this table by primary key or a prefix of primary key. diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/SnapshotQuery.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/SnapshotQuery.java new file mode 100644 index 0000000000..37b16f5d0a --- /dev/null +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/SnapshotQuery.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.fluss.client.table.scanner; + +import org.apache.fluss.annotation.PublicEvolving; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.utils.CloseableIterator; + +import javax.annotation.Nullable; + +import java.util.List; + +/** + * Used to configure and execute a snapshot query to read all current data in a table bucket for a + * primary key table. + * + * @since 0.6 + */ +@PublicEvolving +public interface SnapshotQuery { + + /** + * Returns a new snapshot query from this that will read the given data columns. + * + * @param projectedColumns the selected column indexes + */ + SnapshotQuery project(@Nullable int[] projectedColumns); + + /** + * Returns a new snapshot query from this that will read the given data columns. + * + * @param projectedColumnNames the selected column names + */ + SnapshotQuery project(List projectedColumnNames); + + /** + * Executes the snapshot query to read all current data in the given table bucket. + * + * @param tableBucket the table bucket to read + * @return a closeable iterator of the rows in the table bucket + */ + CloseableIterator execute(TableBucket tableBucket); +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableSnapshotQuery.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableSnapshotQuery.java new file mode 100644 index 0000000000..a3494fe567 --- /dev/null +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableSnapshotQuery.java @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.fluss.client.table.scanner; + +import org.apache.fluss.client.FlussConnection; +import org.apache.fluss.client.table.scanner.batch.BatchScanner; +import org.apache.fluss.metadata.SchemaGetter; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TableInfo; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.types.RowType; +import org.apache.fluss.utils.CloseableIterator; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.time.Duration; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; + +/** Implementation of {@link SnapshotQuery}. */ +public class TableSnapshotQuery implements SnapshotQuery { + + private final FlussConnection conn; + private final TableInfo tableInfo; + private final SchemaGetter schemaGetter; + + /** The projected fields to do projection. No projection if is null. */ + @Nullable private final int[] projectedColumns; + + public TableSnapshotQuery( + FlussConnection conn, TableInfo tableInfo, SchemaGetter schemaGetter) { + this(conn, tableInfo, schemaGetter, null); + } + + private TableSnapshotQuery( + FlussConnection conn, + TableInfo tableInfo, + SchemaGetter schemaGetter, + @Nullable int[] projectedColumns) { + this.conn = conn; + this.tableInfo = tableInfo; + this.schemaGetter = schemaGetter; + this.projectedColumns = projectedColumns; + } + + @Override + public SnapshotQuery project(@Nullable int[] projectedColumns) { + return new TableSnapshotQuery(conn, tableInfo, schemaGetter, projectedColumns); + } + + @Override + public SnapshotQuery project(List projectedColumnNames) { + int[] columnIndexes = new int[projectedColumnNames.size()]; + RowType rowType = tableInfo.getRowType(); + for (int i = 0; i < projectedColumnNames.size(); i++) { + int index = rowType.getFieldIndex(projectedColumnNames.get(i)); + if (index < 0) { + throw new IllegalArgumentException( + String.format( + "Field '%s' not found in table schema. Available fields: %s, Table: %s", + projectedColumnNames.get(i), + rowType.getFieldNames(), + tableInfo.getTablePath())); + } + columnIndexes[i] = index; + } + return new TableSnapshotQuery(conn, tableInfo, schemaGetter, columnIndexes); + } + + @Override + public CloseableIterator execute(TableBucket tableBucket) { + Scan scan = new TableScan(conn, tableInfo, schemaGetter); + if (projectedColumns != null) { + scan = scan.project(projectedColumns); + } + BatchScanner batchScanner = scan.createBatchScanner(tableBucket); + return new BatchScannerIterator(batchScanner); + } + + private static class BatchScannerIterator implements CloseableIterator { + private final BatchScanner scanner; + private Iterator currentBatch; + private boolean isClosed = false; + + private BatchScannerIterator(BatchScanner scanner) { + this.scanner = scanner; + } + + @Override + public boolean hasNext() { + ensureBatch(); + return currentBatch != null && currentBatch.hasNext(); + } + + @Override + public InternalRow next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + return currentBatch.next(); + } + + private void ensureBatch() { + try { + while ((currentBatch == null || !currentBatch.hasNext()) && !isClosed) { + CloseableIterator it = + scanner.pollBatch(Duration.ofMinutes(1)); // Use a large timeout + if (it == null) { + isClosed = true; + break; + } + if (it.hasNext()) { + currentBatch = it; + } else { + it.close(); + } + } + } catch (IOException e) { + throw new RuntimeException("Error polling batch from scanner", e); + } + } + + @Override + public void close() { + if (!isClosed) { + try { + scanner.close(); + } catch (IOException e) { + throw new RuntimeException("Error closing scanner", e); + } + isClosed = true; + } + } + } +} diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvBatchScannerITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvBatchScannerITCase.java new file mode 100644 index 0000000000..6850debb29 --- /dev/null +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvBatchScannerITCase.java @@ -0,0 +1,283 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.fluss.client.table.scanner.batch; + +import org.apache.fluss.client.admin.ClientToServerITCaseBase; +import org.apache.fluss.client.table.Table; +import org.apache.fluss.client.table.writer.UpsertWriter; +import org.apache.fluss.metadata.PartitionSpec; +import org.apache.fluss.metadata.Schema; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TableDescriptor; +import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.types.DataTypes; +import org.apache.fluss.utils.CloseableIterator; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; + +import static org.apache.fluss.testutils.DataTestUtils.row; +import static org.apache.fluss.testutils.InternalRowAssert.assertThatRow; +import static org.assertj.core.api.Assertions.assertThat; + +/** ITCase for {@link KvBatchScanner}. */ +public class KvBatchScannerITCase extends ClientToServerITCaseBase { + @BeforeEach + protected void setup() throws Exception { + super.setup(); + } + + @AfterEach + protected void teardown() throws Exception { + super.teardown(); + } + + @Test + void testBasicScan() throws Exception { + System.out.println("eddww"); + TablePath tablePath = TablePath.of("test_db", "test_basic_scan"); + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("name", DataTypes.STRING()) + .primaryKey("id") + .build(); + TableDescriptor descriptor = + TableDescriptor.builder().schema(schema).distributedBy(1, "id").build(); + + createTable(tablePath, descriptor, true); + System.out.println("eddww"); + + Table table = conn.getTable(tablePath); + + // 1. write data + UpsertWriter writer = table.newUpsert().createWriter(); + writer.upsert(row(1, "a")); + writer.upsert(row(2, "b")); + writer.upsert(row(3, "c")); + writer.flush(); + System.out.println("never reaches this"); + + // 2. test the kvScan works as expected + TableBucket bucket = new TableBucket(table.getTableInfo().getTableId(), 0); + List result = scanAll(table, bucket); + + assertThat(result).hasSize(3); + result.sort(Comparator.comparingInt(r -> r.getInt(0))); + assertThatRow(result.get(0)).withSchema(schema.getRowType()).isEqualTo(row(1, "a")); + assertThatRow(result.get(1)).withSchema(schema.getRowType()).isEqualTo(row(2, "b")); + assertThatRow(result.get(2)).withSchema(schema.getRowType()).isEqualTo(row(3, "c")); + } + + @Test + void testMultiBucketScan() throws Exception { + TablePath tablePath = TablePath.of("test_db", "test_multi_bucket_scan"); + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("name", DataTypes.STRING()) + .primaryKey("id") + .build(); + // 3 buckets + TableDescriptor descriptor = + TableDescriptor.builder().schema(schema).distributedBy(3, "id").build(); + + createTable(tablePath, descriptor, true); + Table table = conn.getTable(tablePath); + + // 1. write data to multiple buckets + int rowCount = 100; + UpsertWriter writer = table.newUpsert().createWriter(); + for (int i = 0; i < rowCount; i++) { + writer.upsert(row(i, "val" + i)); + } + writer.flush(); + + // 2. scan each bucket and collect all data + List allResult = new ArrayList<>(); + for (int i = 0; i < 3; i++) { + TableBucket bucket = new TableBucket(table.getTableInfo().getTableId(), i); + allResult.addAll(scanAll(table, bucket)); + } + + assertThat(allResult).hasSize(rowCount); + allResult.sort(Comparator.comparingInt(r -> r.getInt(0))); + for (int i = 0; i < rowCount; i++) { + assertThatRow(allResult.get(i)) + .withSchema(schema.getRowType()) + .isEqualTo(row(i, "val" + i)); + } + } + + @Test + void testPartitionedTableScan() throws Exception { + TablePath tablePath = TablePath.of("test_db", "test_partitioned_scan"); + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("p", DataTypes.STRING()) + .column("name", DataTypes.STRING()) + .primaryKey("id", "p") + .build(); + TableDescriptor descriptor = + TableDescriptor.builder() + .schema(schema) + .partitionedBy("p") + .distributedBy(1, "id") + .build(); + + createTable(tablePath, descriptor, true); + admin.createPartition( + tablePath, + new PartitionSpec(java.util.Collections.singletonMap("p", "p1")), + false) + .get(); + admin.createPartition( + tablePath, + new PartitionSpec(java.util.Collections.singletonMap("p", "p2")), + false) + .get(); + + Table table = conn.getTable(tablePath); + long p1Id = -1; + long p2Id = -1; + for (org.apache.fluss.metadata.PartitionInfo p : + admin.listPartitionInfos(tablePath).get()) { + if (p.getPartitionName().equals("p=p1")) { + p1Id = p.getPartitionId(); + } else if (p.getPartitionName().equals("p=p2")) { + p2Id = p.getPartitionId(); + } + } + + // 1. write data to different partitions + UpsertWriter writer = table.newUpsert().createWriter(); + writer.upsert(row(1, "p1", "a1")); + writer.upsert(row(2, "p1", "b1")); + writer.upsert(row(1, "p2", "a2")); + writer.flush(); + + // 2. scan partition p1 + TableBucket p1Bucket = new TableBucket(table.getTableInfo().getTableId(), p1Id, 0); + List p1Result = scanAll(table, p1Bucket); + assertThat(p1Result).hasSize(2); + + // 3. scan partition p2 + TableBucket p2Bucket = new TableBucket(table.getTableInfo().getTableId(), p2Id, 0); + List p2Result = scanAll(table, p2Bucket); + assertThat(p2Result).hasSize(1); + assertThatRow(p2Result.get(0)) + .withSchema(schema.getRowType()) + .isEqualTo(row(1, "p2", "a2")); + } + + @Test + void testLargeDataScan() throws Exception { + TablePath tablePath = TablePath.of("test_db", "test_large_data_scan"); + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("name", DataTypes.STRING()) + .primaryKey("id") + .build(); + TableDescriptor descriptor = + TableDescriptor.builder().schema(schema).distributedBy(1, "id").build(); + + createTable(tablePath, descriptor, true); + Table table = conn.getTable(tablePath); + + // 1. write 10k records + int rowCount = 10000; + UpsertWriter writer = table.newUpsert().createWriter(); + for (int i = 0; i < rowCount; i++) { + writer.upsert(row(i, "val" + i)); + } + writer.flush(); + + // 2. scan and verify + TableBucket bucket = new TableBucket(table.getTableInfo().getTableId(), 0); + List result = scanAll(table, bucket); + + assertThat(result).hasSize(rowCount); + result.sort(Comparator.comparingInt(r -> r.getInt(0))); + for (int i = 0; i < rowCount; i++) { + assertThatRow(result.get(i)) + .withSchema(schema.getRowType()) + .isEqualTo(row(i, "val" + i)); + } + } + + @Test + void testSnapshotQuery() throws Exception { + TablePath tablePath = TablePath.of("test_db", "test_snapshot_query"); + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("name", DataTypes.STRING()) + .primaryKey("id") + .build(); + TableDescriptor descriptor = + TableDescriptor.builder().schema(schema).distributedBy(1, "id").build(); + + createTable(tablePath, descriptor, true); + Table table = conn.getTable(tablePath); + + // 1. write data + UpsertWriter writer = table.newUpsert().createWriter(); + writer.upsert(row(1, "a")); + writer.upsert(row(2, "b")); + writer.upsert(row(3, "c")); + writer.flush(); + + // 2. test the snapshotQuery works as expected + TableBucket bucket = new TableBucket(table.getTableInfo().getTableId(), 0); + List result = new ArrayList<>(); + try (CloseableIterator iterator = table.newSnapshotQuery().execute(bucket)) { + while (iterator.hasNext()) { + result.add(iterator.next()); + } + } + + assertThat(result).hasSize(3); + result.sort(Comparator.comparingInt(r -> r.getInt(0))); + assertThatRow(result.get(0)).withSchema(schema.getRowType()).isEqualTo(row(1, "a")); + assertThatRow(result.get(1)).withSchema(schema.getRowType()).isEqualTo(row(2, "b")); + assertThatRow(result.get(2)).withSchema(schema.getRowType()).isEqualTo(row(3, "c")); + } + + private List scanAll(Table table, TableBucket bucket) throws Exception { + List allRows = new ArrayList<>(); + try (BatchScanner scanner = table.newScan().createBatchScanner(bucket)) { + CloseableIterator iterator; + while ((iterator = scanner.pollBatch(Duration.ofSeconds(5))) != null) { + while (iterator.hasNext()) { + allRows.add(iterator.next()); + } + iterator.close(); + } + } + return allRows; + } +} diff --git a/fluss-rpc/src/test/java/org/apache/fluss/rpc/TestingTabletGatewayService.java b/fluss-rpc/src/test/java/org/apache/fluss/rpc/TestingTabletGatewayService.java index 7db3654383..7503c3379a 100644 --- a/fluss-rpc/src/test/java/org/apache/fluss/rpc/TestingTabletGatewayService.java +++ b/fluss-rpc/src/test/java/org/apache/fluss/rpc/TestingTabletGatewayService.java @@ -71,6 +71,10 @@ import org.apache.fluss.rpc.messages.ProduceLogResponse; import org.apache.fluss.rpc.messages.PutKvRequest; import org.apache.fluss.rpc.messages.PutKvResponse; +import org.apache.fluss.rpc.messages.ScanKvRequest; +import org.apache.fluss.rpc.messages.ScanKvResponse; +import org.apache.fluss.rpc.messages.ScannerKeepAliveRequest; +import org.apache.fluss.rpc.messages.ScannerKeepAliveResponse; import org.apache.fluss.rpc.messages.StopReplicaRequest; import org.apache.fluss.rpc.messages.StopReplicaResponse; import org.apache.fluss.rpc.messages.TableExistsRequest; @@ -135,6 +139,17 @@ public CompletableFuture limitScan(LimitScanRequest request) return null; } + @Override + public CompletableFuture scanKv(ScanKvRequest request) { + return null; + } + + @Override + public CompletableFuture scannerKeepAlive( + ScannerKeepAliveRequest request) { + return null; + } + @Override public CompletableFuture listOffsets(ListOffsetsRequest request) { return null; diff --git a/fluss-server/src/test/java/org/apache/fluss/server/tablet/TestTabletServerGateway.java b/fluss-server/src/test/java/org/apache/fluss/server/tablet/TestTabletServerGateway.java index 500d197fcf..1c4d7f78c0 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/tablet/TestTabletServerGateway.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/tablet/TestTabletServerGateway.java @@ -81,6 +81,10 @@ import org.apache.fluss.rpc.messages.ProduceLogResponse; import org.apache.fluss.rpc.messages.PutKvRequest; import org.apache.fluss.rpc.messages.PutKvResponse; +import org.apache.fluss.rpc.messages.ScanKvRequest; +import org.apache.fluss.rpc.messages.ScanKvResponse; +import org.apache.fluss.rpc.messages.ScannerKeepAliveRequest; +import org.apache.fluss.rpc.messages.ScannerKeepAliveResponse; import org.apache.fluss.rpc.messages.StopReplicaRequest; import org.apache.fluss.rpc.messages.StopReplicaResponse; import org.apache.fluss.rpc.messages.TableExistsRequest; @@ -204,6 +208,17 @@ public CompletableFuture limitScan(LimitScanRequest request) return null; } + @Override + public CompletableFuture scanKv(ScanKvRequest request) { + return null; + } + + @Override + public CompletableFuture scannerKeepAlive( + ScannerKeepAliveRequest request) { + return null; + } + @Override public CompletableFuture listOffsets(ListOffsetsRequest request) { return null; From 63ad71f568fce417bddb66f0da8ef2b640ce6467 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Sun, 18 Jan 2026 23:04:55 +0200 Subject: [PATCH 03/14] update tests --- .../scanner/batch/KvBatchScannerITCase.java | 47 +++++++------------ 1 file changed, 16 insertions(+), 31 deletions(-) diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvBatchScannerITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvBatchScannerITCase.java index 6850debb29..85ba5c8081 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvBatchScannerITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvBatchScannerITCase.java @@ -20,6 +20,7 @@ import org.apache.fluss.client.admin.ClientToServerITCaseBase; import org.apache.fluss.client.table.Table; import org.apache.fluss.client.table.writer.UpsertWriter; +import org.apache.fluss.metadata.PartitionInfo; import org.apache.fluss.metadata.PartitionSpec; import org.apache.fluss.metadata.Schema; import org.apache.fluss.metadata.TableBucket; @@ -33,7 +34,6 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import java.time.Duration; import java.util.ArrayList; import java.util.Comparator; import java.util.List; @@ -56,7 +56,6 @@ protected void teardown() throws Exception { @Test void testBasicScan() throws Exception { - System.out.println("eddww"); TablePath tablePath = TablePath.of("test_db", "test_basic_scan"); Schema schema = Schema.newBuilder() @@ -68,7 +67,6 @@ void testBasicScan() throws Exception { TableDescriptor.builder().schema(schema).distributedBy(1, "id").build(); createTable(tablePath, descriptor, true); - System.out.println("eddww"); Table table = conn.getTable(tablePath); @@ -78,11 +76,10 @@ void testBasicScan() throws Exception { writer.upsert(row(2, "b")); writer.upsert(row(3, "c")); writer.flush(); - System.out.println("never reaches this"); - // 2. test the kvScan works as expected + // 2. test the snapshotQuery works as expected TableBucket bucket = new TableBucket(table.getTableInfo().getTableId(), 0); - List result = scanAll(table, bucket); + List result = snapshotQueryAll(table, bucket); assertThat(result).hasSize(3); result.sort(Comparator.comparingInt(r -> r.getInt(0))); @@ -119,7 +116,7 @@ void testMultiBucketScan() throws Exception { List allResult = new ArrayList<>(); for (int i = 0; i < 3; i++) { TableBucket bucket = new TableBucket(table.getTableInfo().getTableId(), i); - allResult.addAll(scanAll(table, bucket)); + allResult.addAll(snapshotQueryAll(table, bucket)); } assertThat(allResult).hasSize(rowCount); @@ -163,30 +160,27 @@ void testPartitionedTableScan() throws Exception { Table table = conn.getTable(tablePath); long p1Id = -1; long p2Id = -1; - for (org.apache.fluss.metadata.PartitionInfo p : - admin.listPartitionInfos(tablePath).get()) { - if (p.getPartitionName().equals("p=p1")) { + List partitionInfos = admin.listPartitionInfos(tablePath).get(); + for (PartitionInfo p : partitionInfos) { + if (p.getPartitionName().equals("p1")) { p1Id = p.getPartitionId(); - } else if (p.getPartitionName().equals("p=p2")) { + } else if (p.getPartitionName().equals("p2")) { p2Id = p.getPartitionId(); } } - // 1. write data to different partitions UpsertWriter writer = table.newUpsert().createWriter(); writer.upsert(row(1, "p1", "a1")); writer.upsert(row(2, "p1", "b1")); writer.upsert(row(1, "p2", "a2")); writer.flush(); - // 2. scan partition p1 TableBucket p1Bucket = new TableBucket(table.getTableInfo().getTableId(), p1Id, 0); - List p1Result = scanAll(table, p1Bucket); + List p1Result = snapshotQueryAll(table, p1Bucket); assertThat(p1Result).hasSize(2); - // 3. scan partition p2 TableBucket p2Bucket = new TableBucket(table.getTableInfo().getTableId(), p2Id, 0); - List p2Result = scanAll(table, p2Bucket); + List p2Result = snapshotQueryAll(table, p2Bucket); assertThat(p2Result).hasSize(1); assertThatRow(p2Result.get(0)) .withSchema(schema.getRowType()) @@ -218,7 +212,7 @@ void testLargeDataScan() throws Exception { // 2. scan and verify TableBucket bucket = new TableBucket(table.getTableInfo().getTableId(), 0); - List result = scanAll(table, bucket); + List result = snapshotQueryAll(table, bucket); assertThat(result).hasSize(rowCount); result.sort(Comparator.comparingInt(r -> r.getInt(0))); @@ -253,12 +247,7 @@ void testSnapshotQuery() throws Exception { // 2. test the snapshotQuery works as expected TableBucket bucket = new TableBucket(table.getTableInfo().getTableId(), 0); - List result = new ArrayList<>(); - try (CloseableIterator iterator = table.newSnapshotQuery().execute(bucket)) { - while (iterator.hasNext()) { - result.add(iterator.next()); - } - } + List result = snapshotQueryAll(table, bucket); assertThat(result).hasSize(3); result.sort(Comparator.comparingInt(r -> r.getInt(0))); @@ -267,15 +256,11 @@ void testSnapshotQuery() throws Exception { assertThatRow(result.get(2)).withSchema(schema.getRowType()).isEqualTo(row(3, "c")); } - private List scanAll(Table table, TableBucket bucket) throws Exception { + private List snapshotQueryAll(Table table, TableBucket bucket) throws Exception { List allRows = new ArrayList<>(); - try (BatchScanner scanner = table.newScan().createBatchScanner(bucket)) { - CloseableIterator iterator; - while ((iterator = scanner.pollBatch(Duration.ofSeconds(5))) != null) { - while (iterator.hasNext()) { - allRows.add(iterator.next()); - } - iterator.close(); + try (CloseableIterator iterator = table.newSnapshotQuery().execute(bucket)) { + while (iterator.hasNext()) { + allRows.add(iterator.next()); } } return allRows; From ae488e3634751316be36d16355f9448d5745f6ba Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Sun, 18 Jan 2026 23:11:44 +0200 Subject: [PATCH 04/14] update the snapshotQuery and the tests --- .../client/table/scanner/SnapshotQuery.java | 27 ++--- .../table/scanner/TableSnapshotQuery.java | 100 +++++++++++++----- ...ITCase.java => KvSnapshotQueryITCase.java} | 49 ++------- 3 files changed, 95 insertions(+), 81 deletions(-) rename fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/{KvBatchScannerITCase.java => KvSnapshotQueryITCase.java} (80%) diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/SnapshotQuery.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/SnapshotQuery.java index 37b16f5d0a..5fe8327b6f 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/SnapshotQuery.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/SnapshotQuery.java @@ -22,10 +22,6 @@ import org.apache.fluss.row.InternalRow; import org.apache.fluss.utils.CloseableIterator; -import javax.annotation.Nullable; - -import java.util.List; - /** * Used to configure and execute a snapshot query to read all current data in a table bucket for a * primary key table. @@ -34,21 +30,6 @@ */ @PublicEvolving public interface SnapshotQuery { - - /** - * Returns a new snapshot query from this that will read the given data columns. - * - * @param projectedColumns the selected column indexes - */ - SnapshotQuery project(@Nullable int[] projectedColumns); - - /** - * Returns a new snapshot query from this that will read the given data columns. - * - * @param projectedColumnNames the selected column names - */ - SnapshotQuery project(List projectedColumnNames); - /** * Executes the snapshot query to read all current data in the given table bucket. * @@ -56,4 +37,12 @@ public interface SnapshotQuery { * @return a closeable iterator of the rows in the table bucket */ CloseableIterator execute(TableBucket tableBucket); + + /** + * Executes the snapshot query to read all current data in the table. Everything around + * partitions and buckets will be taken care of from the client. + * + * @return a closeable iterator of the rows in the table + */ + CloseableIterator execute(); } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableSnapshotQuery.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableSnapshotQuery.java index a3494fe567..eb4c7ac022 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableSnapshotQuery.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableSnapshotQuery.java @@ -19,17 +19,19 @@ import org.apache.fluss.client.FlussConnection; import org.apache.fluss.client.table.scanner.batch.BatchScanner; +import org.apache.fluss.exception.FlussRuntimeException; +import org.apache.fluss.metadata.PartitionInfo; import org.apache.fluss.metadata.SchemaGetter; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.row.InternalRow; -import org.apache.fluss.types.RowType; import org.apache.fluss.utils.CloseableIterator; import javax.annotation.Nullable; import java.io.IOException; import java.time.Duration; +import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; @@ -61,37 +63,87 @@ private TableSnapshotQuery( } @Override - public SnapshotQuery project(@Nullable int[] projectedColumns) { - return new TableSnapshotQuery(conn, tableInfo, schemaGetter, projectedColumns); + public CloseableIterator execute(TableBucket tableBucket) { + Scan scan = new TableScan(conn, tableInfo, schemaGetter); + if (projectedColumns != null) { + scan = scan.project(projectedColumns); + } + BatchScanner batchScanner = scan.createBatchScanner(tableBucket); + return new BatchScannerIterator(batchScanner); } @Override - public SnapshotQuery project(List projectedColumnNames) { - int[] columnIndexes = new int[projectedColumnNames.size()]; - RowType rowType = tableInfo.getRowType(); - for (int i = 0; i < projectedColumnNames.size(); i++) { - int index = rowType.getFieldIndex(projectedColumnNames.get(i)); - if (index < 0) { - throw new IllegalArgumentException( - String.format( - "Field '%s' not found in table schema. Available fields: %s, Table: %s", - projectedColumnNames.get(i), - rowType.getFieldNames(), - tableInfo.getTablePath())); + public CloseableIterator execute() { + List buckets = new ArrayList<>(); + try { + if (tableInfo.isPartitioned()) { + List partitions = + conn.getAdmin().listPartitionInfos(tableInfo.getTablePath()).get(); + for (PartitionInfo partition : partitions) { + for (int i = 0; i < tableInfo.getNumBuckets(); i++) { + buckets.add( + new TableBucket( + tableInfo.getTableId(), partition.getPartitionId(), i)); + } + } + } else { + for (int i = 0; i < tableInfo.getNumBuckets(); i++) { + buckets.add(new TableBucket(tableInfo.getTableId(), i)); + } } - columnIndexes[i] = index; + } catch (Exception e) { + throw new FlussRuntimeException( + "Failed to list partitions for table " + tableInfo.getTablePath(), e); } - return new TableSnapshotQuery(conn, tableInfo, schemaGetter, columnIndexes); + + return new MultiBucketBatchScannerIterator(buckets); } - @Override - public CloseableIterator execute(TableBucket tableBucket) { - Scan scan = new TableScan(conn, tableInfo, schemaGetter); - if (projectedColumns != null) { - scan = scan.project(projectedColumns); + private class MultiBucketBatchScannerIterator implements CloseableIterator { + private final Iterator bucketIterator; + private CloseableIterator currentScannerIterator; + private boolean isClosed = false; + + private MultiBucketBatchScannerIterator(List buckets) { + this.bucketIterator = buckets.iterator(); + } + + @Override + public boolean hasNext() { + if (isClosed) { + return false; + } + while (currentScannerIterator == null || !currentScannerIterator.hasNext()) { + if (currentScannerIterator != null) { + currentScannerIterator.close(); + currentScannerIterator = null; + } + if (bucketIterator.hasNext()) { + currentScannerIterator = execute(bucketIterator.next()); + } else { + return false; + } + } + return true; + } + + @Override + public InternalRow next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + return currentScannerIterator.next(); + } + + @Override + public void close() { + if (!isClosed) { + if (currentScannerIterator != null) { + currentScannerIterator.close(); + } + isClosed = true; + } } - BatchScanner batchScanner = scan.createBatchScanner(tableBucket); - return new BatchScannerIterator(batchScanner); } private static class BatchScannerIterator implements CloseableIterator { diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvBatchScannerITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotQueryITCase.java similarity index 80% rename from fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvBatchScannerITCase.java rename to fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotQueryITCase.java index 85ba5c8081..3754ff3880 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvBatchScannerITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotQueryITCase.java @@ -20,10 +20,8 @@ import org.apache.fluss.client.admin.ClientToServerITCaseBase; import org.apache.fluss.client.table.Table; import org.apache.fluss.client.table.writer.UpsertWriter; -import org.apache.fluss.metadata.PartitionInfo; import org.apache.fluss.metadata.PartitionSpec; import org.apache.fluss.metadata.Schema; -import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableDescriptor; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.row.InternalRow; @@ -42,8 +40,8 @@ import static org.apache.fluss.testutils.InternalRowAssert.assertThatRow; import static org.assertj.core.api.Assertions.assertThat; -/** ITCase for {@link KvBatchScanner}. */ -public class KvBatchScannerITCase extends ClientToServerITCaseBase { +/** ITCase for snapshot query. */ +public class KvSnapshotQueryITCase extends ClientToServerITCaseBase { @BeforeEach protected void setup() throws Exception { super.setup(); @@ -78,8 +76,7 @@ void testBasicScan() throws Exception { writer.flush(); // 2. test the snapshotQuery works as expected - TableBucket bucket = new TableBucket(table.getTableInfo().getTableId(), 0); - List result = snapshotQueryAll(table, bucket); + List result = snapshotQueryAll(table); assertThat(result).hasSize(3); result.sort(Comparator.comparingInt(r -> r.getInt(0))); @@ -112,12 +109,8 @@ void testMultiBucketScan() throws Exception { } writer.flush(); - // 2. scan each bucket and collect all data - List allResult = new ArrayList<>(); - for (int i = 0; i < 3; i++) { - TableBucket bucket = new TableBucket(table.getTableInfo().getTableId(), i); - allResult.addAll(snapshotQueryAll(table, bucket)); - } + // 2. scan all buckets and collect all data + List allResult = snapshotQueryAll(table); assertThat(allResult).hasSize(rowCount); allResult.sort(Comparator.comparingInt(r -> r.getInt(0))); @@ -158,16 +151,6 @@ void testPartitionedTableScan() throws Exception { .get(); Table table = conn.getTable(tablePath); - long p1Id = -1; - long p2Id = -1; - List partitionInfos = admin.listPartitionInfos(tablePath).get(); - for (PartitionInfo p : partitionInfos) { - if (p.getPartitionName().equals("p1")) { - p1Id = p.getPartitionId(); - } else if (p.getPartitionName().equals("p2")) { - p2Id = p.getPartitionId(); - } - } UpsertWriter writer = table.newUpsert().createWriter(); writer.upsert(row(1, "p1", "a1")); @@ -175,16 +158,8 @@ void testPartitionedTableScan() throws Exception { writer.upsert(row(1, "p2", "a2")); writer.flush(); - TableBucket p1Bucket = new TableBucket(table.getTableInfo().getTableId(), p1Id, 0); - List p1Result = snapshotQueryAll(table, p1Bucket); - assertThat(p1Result).hasSize(2); - - TableBucket p2Bucket = new TableBucket(table.getTableInfo().getTableId(), p2Id, 0); - List p2Result = snapshotQueryAll(table, p2Bucket); - assertThat(p2Result).hasSize(1); - assertThatRow(p2Result.get(0)) - .withSchema(schema.getRowType()) - .isEqualTo(row(1, "p2", "a2")); + List result = snapshotQueryAll(table); + assertThat(result).hasSize(3); } @Test @@ -211,8 +186,7 @@ void testLargeDataScan() throws Exception { writer.flush(); // 2. scan and verify - TableBucket bucket = new TableBucket(table.getTableInfo().getTableId(), 0); - List result = snapshotQueryAll(table, bucket); + List result = snapshotQueryAll(table); assertThat(result).hasSize(rowCount); result.sort(Comparator.comparingInt(r -> r.getInt(0))); @@ -246,8 +220,7 @@ void testSnapshotQuery() throws Exception { writer.flush(); // 2. test the snapshotQuery works as expected - TableBucket bucket = new TableBucket(table.getTableInfo().getTableId(), 0); - List result = snapshotQueryAll(table, bucket); + List result = snapshotQueryAll(table); assertThat(result).hasSize(3); result.sort(Comparator.comparingInt(r -> r.getInt(0))); @@ -256,9 +229,9 @@ void testSnapshotQuery() throws Exception { assertThatRow(result.get(2)).withSchema(schema.getRowType()).isEqualTo(row(3, "c")); } - private List snapshotQueryAll(Table table, TableBucket bucket) throws Exception { + private List snapshotQueryAll(Table table) throws Exception { List allRows = new ArrayList<>(); - try (CloseableIterator iterator = table.newSnapshotQuery().execute(bucket)) { + try (CloseableIterator iterator = table.newSnapshotQuery().execute()) { while (iterator.hasNext()) { allRows.add(iterator.next()); } From 7808564c4f66d82f7755f35676af32347b77af8c Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Sun, 18 Jan 2026 23:22:54 +0200 Subject: [PATCH 05/14] fix proto --- .../apache/fluss/rpc/protocol/ApiKeys.java | 40 +++++----- fluss-rpc/src/main/proto/FlussApi.proto | 78 +++++++++++++++++++ 2 files changed, 98 insertions(+), 20 deletions(-) diff --git a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java index 105b3e0a9d..b72edebf75 100644 --- a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java +++ b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java @@ -63,26 +63,26 @@ public enum ApiKeys { GET_LATEST_LAKE_SNAPSHOT(1032, 0, 0, PUBLIC), LIMIT_SCAN(1033, 0, 0, PUBLIC), PREFIX_LOOKUP(1034, 0, 0, PUBLIC), - SCAN_KV(1035, 0, 0, PUBLIC), - SCANNER_KEEP_ALIVE(1036, 0, 0, PUBLIC), - GET_DATABASE_INFO(1037, 0, 0, PUBLIC), - CREATE_PARTITION(1038, 0, 0, PUBLIC), - DROP_PARTITION(1039, 0, 0, PUBLIC), - AUTHENTICATE(1040, 0, 0, PUBLIC), - CREATE_ACLS(1041, 0, 0, PUBLIC), - LIST_ACLS(1042, 0, 0, PUBLIC), - DROP_ACLS(1043, 0, 0, PUBLIC), - LAKE_TIERING_HEARTBEAT(1044, 0, 0, PRIVATE), - CONTROLLED_SHUTDOWN(1045, 0, 0, PRIVATE), - ALTER_TABLE(1046, 0, 0, PUBLIC), - DESCRIBE_CLUSTER_CONFIGS(1047, 0, 0, PUBLIC), - ALTER_CLUSTER_CONFIGS(1048, 0, 0, PUBLIC), - ADD_SERVER_TAG(1049, 0, 0, PUBLIC), - REMOVE_SERVER_TAG(1050, 0, 0, PUBLIC), - REBALANCE(1051, 0, 0, PUBLIC), - LIST_REBALANCE_PROGRESS(1052, 0, 0, PUBLIC), - CANCEL_REBALANCE(1053, 0, 0, PUBLIC), - PREPARE_LAKE_TABLE_SNAPSHOT(1054, 0, 0, PRIVATE); + GET_DATABASE_INFO(1035, 0, 0, PUBLIC), + CREATE_PARTITION(1036, 0, 0, PUBLIC), + DROP_PARTITION(1037, 0, 0, PUBLIC), + AUTHENTICATE(1038, 0, 0, PUBLIC), + CREATE_ACLS(1039, 0, 0, PUBLIC), + LIST_ACLS(1040, 0, 0, PUBLIC), + DROP_ACLS(1041, 0, 0, PUBLIC), + LAKE_TIERING_HEARTBEAT(1042, 0, 0, PRIVATE), + CONTROLLED_SHUTDOWN(1043, 0, 0, PRIVATE), + ALTER_TABLE(1044, 0, 0, PUBLIC), + DESCRIBE_CLUSTER_CONFIGS(1045, 0, 0, PUBLIC), + ALTER_CLUSTER_CONFIGS(1046, 0, 0, PUBLIC), + ADD_SERVER_TAG(1047, 0, 0, PUBLIC), + REMOVE_SERVER_TAG(1048, 0, 0, PUBLIC), + REBALANCE(1049, 0, 0, PUBLIC), + LIST_REBALANCE_PROGRESS(1050, 0, 0, PUBLIC), + CANCEL_REBALANCE(1051, 0, 0, PUBLIC), + PREPARE_LAKE_TABLE_SNAPSHOT(1052, 0, 0, PRIVATE), + SCAN_KV(1053, 0, 0, PUBLIC), + SCANNER_KEEP_ALIVE(1054, 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 db9d614354..d828cb6105 100644 --- a/fluss-rpc/src/main/proto/FlussApi.proto +++ b/fluss-rpc/src/main/proto/FlussApi.proto @@ -255,6 +255,84 @@ message PrefixLookupResponse { } +// scan kv request and response +message ScanKvRequest { + // If continuing an existing scan, then you must set scanner_id. + // Otherwise, you must set 'new_scan_request'. + optional bytes scanner_id = 1; + optional PbScanReqForBucket bucket_scan_req = 2; + + // The sequence ID of this call. The sequence ID should start at 0 + // with the request for a new scanner, and after each successful request, + // the client should increment it by 1. When retrying a request, the client + // should _not_ increment this value. If the server detects that the client + // missed a chunk of rows from the middle of a scan, it will respond with an + // error. + optional uint32 call_seq_id = 3; + + // The maximum number of bytes to send in the response. + optional uint32 batch_size_bytes = 4; + + // If set, the server will close the scanner after responding to + // this request, regardless of whether all rows have been delivered. + optional bool close_scanner = 5; +} + +message PbScanReqForBucket { + // The tablet to scan. + required int64 table_id = 1; + optional int64 partition_id = 2; + required int32 bucket_id = 3; + + // The maximum number of rows to scan with the new scanner. + // + // The scanner will automatically stop yielding results and close itself + // after reaching this number of result rows. + optional uint64 limit = 4; +} + +message ScanKvResponse { + // The error, if an error occurred with this request. + optional int32 error_code = 1; + optional string error_message = 2; + + // When a scanner is created, returns the scanner ID which may be used + // to pull new rows from the scanner. + optional bytes scanner_id = 3; + + // Set to true to indicate that there may be further results to be fetched + // from this scanner. If the scanner has no more results, then the scanner + // ID will become invalid and cannot continue to be used. + // + // Note that if a scan returns no results, then the initial response from + // the first RPC may return false in this flag, in which case there will + // be no scanner ID assigned. + optional bool has_more_results = 4; + + // The block of returned rows. + // + // NOTE: the schema-related fields will not be present in this row block. + // The schema will match the schema requested by the client when it created + // the scanner. + optional bytes records = 5; + + // Returns the corresponding log offset at the time the scanner is created + optional int64 log_offset = 6; +} + +// A scanner keep-alive request. +// Updates the scanner access time, increasing its time-to-live. +message ScannerKeepAliveRequest { + required bytes scanner_id = 1; +} + +message ScannerKeepAliveResponse { + // The error, if an error occurred with this request. + optional int32 error_code = 1; + optional string error_message = 2; +} + + // limit scan request and response message LimitScanRequest { required int64 table_id = 2; From c8f5464779df915a17dce4d7222371a4512b6be6 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Sun, 18 Jan 2026 23:28:25 +0200 Subject: [PATCH 06/14] fix violation --- .../java/org/apache/fluss/server/kv/scan/ScannerManager.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java index 4d2b8f0e48..4beac6e1d9 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java @@ -23,6 +23,7 @@ import org.apache.fluss.server.kv.KvTablet; import org.apache.fluss.server.kv.rocksdb.RocksDBKv; import org.apache.fluss.utils.AutoCloseableAsync; +import org.apache.fluss.utils.MapUtils; import org.apache.fluss.utils.clock.Clock; import org.apache.fluss.utils.clock.SystemClock; import org.apache.fluss.utils.concurrent.ExecutorThreadFactory; @@ -39,7 +40,6 @@ import java.util.Map; import java.util.UUID; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -48,7 +48,7 @@ public class ScannerManager implements AutoCloseableAsync { private static final Logger LOG = LoggerFactory.getLogger(ScannerManager.class); - private final Map scanners = new ConcurrentHashMap<>(); + private final Map scanners = MapUtils.newConcurrentHashMap(); private final ScheduledExecutorService cleanupExecutor; private final Clock clock; private final long scannerTtlMs; From 1bd22d8a49f41eb665e82c8cc983e9a92337a3e9 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Sun, 18 Jan 2026 23:35:09 +0200 Subject: [PATCH 07/14] update interface --- .../org/apache/fluss/client/table/scanner/SnapshotQuery.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/SnapshotQuery.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/SnapshotQuery.java index 5fe8327b6f..26b6c2920a 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/SnapshotQuery.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/SnapshotQuery.java @@ -23,10 +23,9 @@ import org.apache.fluss.utils.CloseableIterator; /** - * Used to configure and execute a snapshot query to read all current data in a table bucket for a - * primary key table. + * Used to configure and execute a snapshot query to read all kv data of a primary key table. * - * @since 0.6 + * @since 0.9 */ @PublicEvolving public interface SnapshotQuery { From 43a01e7fafeb7f2f205c90edfb2c0ad9ef5d9ecb Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Sat, 14 Feb 2026 21:16:44 +0200 Subject: [PATCH 08/14] remove keepalive and reuse TS executor --- .../table/scanner/batch/KvBatchScanner.java | 40 ----------------- .../rpc/gateway/TabletServerGateway.java | 11 ----- .../apache/fluss/rpc/protocol/ApiKeys.java | 3 +- fluss-rpc/src/main/proto/FlussApi.proto | 13 ------ .../rpc/TestingTabletGatewayService.java | 8 ---- .../fluss/server/kv/scan/ScannerManager.java | 44 +++++++------------ .../fluss/server/tablet/TabletServer.java | 2 +- .../fluss/server/tablet/TabletService.java | 15 ------- .../tablet/TestTabletServerGateway.java | 8 ---- 9 files changed, 19 insertions(+), 125 deletions(-) diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/batch/KvBatchScanner.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/batch/KvBatchScanner.java index b84419576f..5f8abfee82 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/batch/KvBatchScanner.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/batch/KvBatchScanner.java @@ -34,12 +34,10 @@ import org.apache.fluss.rpc.messages.PbScanReqForBucket; import org.apache.fluss.rpc.messages.ScanKvRequest; import org.apache.fluss.rpc.messages.ScanKvResponse; -import org.apache.fluss.rpc.messages.ScannerKeepAliveRequest; import org.apache.fluss.rpc.protocol.Errors; import org.apache.fluss.types.RowType; import org.apache.fluss.utils.CloseableIterator; import org.apache.fluss.utils.SchemaUtil; -import org.apache.fluss.utils.concurrent.ExecutorThreadFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -55,8 +53,6 @@ import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; /** A {@link BatchScanner} implementation that scans records from a primary key table. */ @@ -82,7 +78,6 @@ public class KvBatchScanner implements BatchScanner { private boolean isClosed = false; private CompletableFuture inFlightRequest; - private ScheduledExecutorService keepAliveExecutor; public KvBatchScanner( TableInfo tableInfo, @@ -223,37 +218,6 @@ private InternalRow maybeProject(InternalRow originRow) { } } - public void startKeepAlivePeriodically(int keepAliveIntervalMs) { - if (keepAliveExecutor != null) { - return; - } - - keepAliveExecutor = - Executors.newSingleThreadScheduledExecutor( - new ExecutorThreadFactory("kv-scanner-keep-alive-" + tableBucket)); - keepAliveExecutor.scheduleAtFixedRate( - this::sendKeepAlive, - keepAliveIntervalMs, - keepAliveIntervalMs, - TimeUnit.MILLISECONDS); - } - - private void sendKeepAlive() { - if (isClosed || scannerId == null || !hasMoreResults) { - return; - } - - try { - int leader = metadataUpdater.leaderFor(tableInfo.getTablePath(), tableBucket); - TabletServerGateway gateway = metadataUpdater.newTabletServerClientForNode(leader); - if (gateway != null) { - gateway.scannerKeepAlive(new ScannerKeepAliveRequest().setScannerId(scannerId)); - } - } catch (Exception e) { - LOG.warn("Failed to send keep alive for scanner {}", tableBucket, e); - } - } - @Override public void close() throws IOException { if (isClosed) { @@ -261,10 +225,6 @@ public void close() throws IOException { } isClosed = true; - if (keepAliveExecutor != null) { - keepAliveExecutor.shutdownNow(); - } - if (scannerId != null && hasMoreResults) { // Close scanner on server int leader = metadataUpdater.leaderFor(tableInfo.getTablePath(), tableBucket); diff --git a/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/TabletServerGateway.java b/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/TabletServerGateway.java index 436637edd0..932b415650 100644 --- a/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/TabletServerGateway.java +++ b/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/TabletServerGateway.java @@ -44,8 +44,6 @@ import org.apache.fluss.rpc.messages.PutKvResponse; import org.apache.fluss.rpc.messages.ScanKvRequest; import org.apache.fluss.rpc.messages.ScanKvResponse; -import org.apache.fluss.rpc.messages.ScannerKeepAliveRequest; -import org.apache.fluss.rpc.messages.ScannerKeepAliveResponse; import org.apache.fluss.rpc.messages.StopReplicaRequest; import org.apache.fluss.rpc.messages.StopReplicaResponse; import org.apache.fluss.rpc.messages.UpdateMetadataRequest; @@ -143,15 +141,6 @@ CompletableFuture notifyLeaderAndIsr( @RPC(api = ApiKeys.SCAN_KV) CompletableFuture scanKv(ScanKvRequest request); - /** - * Keep alive for the specified scanner. - * - * @param request the scanner keep alive request - * @return the scanner keep alive response - */ - @RPC(api = ApiKeys.SCANNER_KEEP_ALIVE) - CompletableFuture scannerKeepAlive(ScannerKeepAliveRequest request); - /** * List offsets for the specified table bucket. * diff --git a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java index b72edebf75..89cb14e96e 100644 --- a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java +++ b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java @@ -81,8 +81,7 @@ public enum ApiKeys { LIST_REBALANCE_PROGRESS(1050, 0, 0, PUBLIC), CANCEL_REBALANCE(1051, 0, 0, PUBLIC), PREPARE_LAKE_TABLE_SNAPSHOT(1052, 0, 0, PRIVATE), - SCAN_KV(1053, 0, 0, PUBLIC), - SCANNER_KEEP_ALIVE(1054, 0, 0, PUBLIC); + SCAN_KV(1053, 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 d828cb6105..12bb81386e 100644 --- a/fluss-rpc/src/main/proto/FlussApi.proto +++ b/fluss-rpc/src/main/proto/FlussApi.proto @@ -320,19 +320,6 @@ message ScanKvResponse { optional int64 log_offset = 6; } -// A scanner keep-alive request. -// Updates the scanner access time, increasing its time-to-live. -message ScannerKeepAliveRequest { - required bytes scanner_id = 1; -} - -message ScannerKeepAliveResponse { - // The error, if an error occurred with this request. - optional int32 error_code = 1; - optional string error_message = 2; -} - - // limit scan request and response message LimitScanRequest { required int64 table_id = 2; diff --git a/fluss-rpc/src/test/java/org/apache/fluss/rpc/TestingTabletGatewayService.java b/fluss-rpc/src/test/java/org/apache/fluss/rpc/TestingTabletGatewayService.java index 7503c3379a..1bfada8c7f 100644 --- a/fluss-rpc/src/test/java/org/apache/fluss/rpc/TestingTabletGatewayService.java +++ b/fluss-rpc/src/test/java/org/apache/fluss/rpc/TestingTabletGatewayService.java @@ -73,8 +73,6 @@ import org.apache.fluss.rpc.messages.PutKvResponse; import org.apache.fluss.rpc.messages.ScanKvRequest; import org.apache.fluss.rpc.messages.ScanKvResponse; -import org.apache.fluss.rpc.messages.ScannerKeepAliveRequest; -import org.apache.fluss.rpc.messages.ScannerKeepAliveResponse; import org.apache.fluss.rpc.messages.StopReplicaRequest; import org.apache.fluss.rpc.messages.StopReplicaResponse; import org.apache.fluss.rpc.messages.TableExistsRequest; @@ -144,12 +142,6 @@ public CompletableFuture scanKv(ScanKvRequest request) { return null; } - @Override - public CompletableFuture scannerKeepAlive( - ScannerKeepAliveRequest request) { - return null; - } - @Override public CompletableFuture listOffsets(ListOffsetsRequest request) { return null; diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java index 4beac6e1d9..e01ed0d589 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java @@ -26,8 +26,8 @@ import org.apache.fluss.utils.MapUtils; import org.apache.fluss.utils.clock.Clock; import org.apache.fluss.utils.clock.SystemClock; -import org.apache.fluss.utils.concurrent.ExecutorThreadFactory; import org.apache.fluss.utils.concurrent.FutureUtils; +import org.apache.fluss.utils.concurrent.Scheduler; import org.rocksdb.ReadOptions; import org.rocksdb.RocksIterator; @@ -40,34 +40,32 @@ import java.util.Map; import java.util.UUID; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.TimeUnit; +import java.util.concurrent.ScheduledFuture; /** A manager for scanners. */ public class ScannerManager implements AutoCloseableAsync { private static final Logger LOG = LoggerFactory.getLogger(ScannerManager.class); private final Map scanners = MapUtils.newConcurrentHashMap(); - private final ScheduledExecutorService cleanupExecutor; + private final Scheduler scheduler; private final Clock clock; private final long scannerTtlMs; + private ScheduledFuture cleanupTask; - public ScannerManager(Configuration conf) { - this(conf, SystemClock.getInstance()); + public ScannerManager(Configuration conf, Scheduler scheduler) { + this(conf, scheduler, SystemClock.getInstance()); } - public ScannerManager(Configuration conf, Clock clock) { + public ScannerManager(Configuration conf, Scheduler scheduler, Clock clock) { + this.scheduler = scheduler; this.clock = clock; this.scannerTtlMs = conf.get(ConfigOptions.SERVER_SCANNER_TTL).toMillis(); - this.cleanupExecutor = - Executors.newSingleThreadScheduledExecutor( - new ExecutorThreadFactory("scanner-cleanup-thread")); - this.cleanupExecutor.scheduleWithFixedDelay( - this::cleanupExpiredScanners, - scannerTtlMs / 2, - scannerTtlMs / 2, - TimeUnit.MILLISECONDS); + this.cleanupTask = + scheduler.schedule( + "scanner-cleanup", + this::cleanupExpiredScanners, + scannerTtlMs / 2, + scannerTtlMs / 2); } public byte[] createScanner(KvTablet kvTablet, long limit) { @@ -92,16 +90,6 @@ public ScannerContext getScanner(byte[] scannerId) { return context; } - public void keepAlive(byte[] scannerId) { - ScannerContext context = scanners.get(ByteBuffer.wrap(scannerId)); - if (context != null) { - context.updateLastAccessTime(clock.milliseconds()); - } else { - throw Errors.SCANNER_NOT_FOUND_EXCEPTION.exception( - "Scanner not found: " + scannerIdToString(scannerId)); - } - } - public void removeScanner(byte[] scannerId) { ScannerContext context = scanners.remove(ByteBuffer.wrap(scannerId)); if (context != null) { @@ -152,7 +140,9 @@ public CompletableFuture closeAsync() { @Override public void close() throws Exception { - cleanupExecutor.shutdownNow(); + if (cleanupTask != null) { + cleanupTask.cancel(true); + } for (ScannerContext context : scanners.values()) { closeScannerContext(context); } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletServer.java b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletServer.java index dc98e5b8ca..228f1b2c36 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletServer.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletServer.java @@ -234,7 +234,7 @@ protected void startServices() throws Exception { this.kvManager = KvManager.create(conf, zkClient, logManager, tabletServerMetricGroup); kvManager.startup(); - this.scannerManager = new ScannerManager(conf, clock); + this.scannerManager = new ScannerManager(conf, scheduler, clock); // Register kvManager to dynamicConfigManager for dynamic reconfiguration dynamicConfigManager.register(kvManager); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java index 66f6650d96..3bce906dea 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java @@ -63,8 +63,6 @@ import org.apache.fluss.rpc.messages.PutKvResponse; import org.apache.fluss.rpc.messages.ScanKvRequest; import org.apache.fluss.rpc.messages.ScanKvResponse; -import org.apache.fluss.rpc.messages.ScannerKeepAliveRequest; -import org.apache.fluss.rpc.messages.ScannerKeepAliveResponse; import org.apache.fluss.rpc.messages.StopReplicaRequest; import org.apache.fluss.rpc.messages.StopReplicaResponse; import org.apache.fluss.rpc.messages.UpdateMetadataRequest; @@ -421,19 +419,6 @@ private ScanKvResponse makeScanKvErrorResponse(Throwable e) { return response; } - @Override - public CompletableFuture scannerKeepAlive( - ScannerKeepAliveRequest request) { - ScannerKeepAliveResponse response = new ScannerKeepAliveResponse(); - try { - scannerManager.keepAlive(request.getScannerId()); - } catch (Exception e) { - ApiError error = ApiError.fromThrowable(e); - response.setErrorCode(error.error().code()).setErrorMessage(error.message()); - } - return CompletableFuture.completedFuture(response); - } - @Override public CompletableFuture notifyLeaderAndIsr( NotifyLeaderAndIsrRequest notifyLeaderAndIsrRequest) { diff --git a/fluss-server/src/test/java/org/apache/fluss/server/tablet/TestTabletServerGateway.java b/fluss-server/src/test/java/org/apache/fluss/server/tablet/TestTabletServerGateway.java index 1c4d7f78c0..dc7e15dccc 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/tablet/TestTabletServerGateway.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/tablet/TestTabletServerGateway.java @@ -83,8 +83,6 @@ import org.apache.fluss.rpc.messages.PutKvResponse; import org.apache.fluss.rpc.messages.ScanKvRequest; import org.apache.fluss.rpc.messages.ScanKvResponse; -import org.apache.fluss.rpc.messages.ScannerKeepAliveRequest; -import org.apache.fluss.rpc.messages.ScannerKeepAliveResponse; import org.apache.fluss.rpc.messages.StopReplicaRequest; import org.apache.fluss.rpc.messages.StopReplicaResponse; import org.apache.fluss.rpc.messages.TableExistsRequest; @@ -213,12 +211,6 @@ public CompletableFuture scanKv(ScanKvRequest request) { return null; } - @Override - public CompletableFuture scannerKeepAlive( - ScannerKeepAliveRequest request) { - return null; - } - @Override public CompletableFuture listOffsets(ListOffsetsRequest request) { return null; From b3cd3d7e48c2d8ee97513009ec68e03807b77d94 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Fri, 6 Mar 2026 11:29:49 +0200 Subject: [PATCH 09/14] test --- .../apache/fluss/client/table/FlussTable.java | 10 +- .../org/apache/fluss/client/table/Table.java | 8 +- .../fluss/client/table/scanner/KvScan.java | 54 +++++ .../client/table/scanner/TableKvScan.java | 185 ++++++++++++++++++ .../scanner/batch/KvSnapshotQueryITCase.java | 127 +++++++++--- .../apache/fluss/config/ConfigOptions.java | 26 +++ .../InvalidScanRequestException.java | 31 +++ .../exception/ScannerExpiredException.java | 30 +++ .../exception/TooManyScannersException.java | 30 +++ .../exception/UnknownScannerIdException.java | 31 +++ .../org/apache/fluss/rpc/protocol/Errors.java | 18 +- .../fluss/server/kv/scan/ScannerContext.java | 68 +++++-- .../fluss/server/kv/scan/ScannerManager.java | 134 +++++++++++-- .../fluss/server/tablet/TabletServer.java | 6 + .../fluss/server/tablet/TabletService.java | 91 ++++++--- 15 files changed, 749 insertions(+), 100 deletions(-) create mode 100644 fluss-client/src/main/java/org/apache/fluss/client/table/scanner/KvScan.java create mode 100644 fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableKvScan.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/exception/InvalidScanRequestException.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/exception/ScannerExpiredException.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/exception/TooManyScannersException.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/exception/UnknownScannerIdException.java diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/FlussTable.java b/fluss-client/src/main/java/org/apache/fluss/client/table/FlussTable.java index 79cffa999d..20d5fcd84b 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/FlussTable.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/FlussTable.java @@ -23,9 +23,9 @@ import org.apache.fluss.client.lookup.TableLookup; import org.apache.fluss.client.metadata.ClientSchemaGetter; import org.apache.fluss.client.table.scanner.Scan; -import org.apache.fluss.client.table.scanner.SnapshotQuery; +import org.apache.fluss.client.table.scanner.KvScan; import org.apache.fluss.client.table.scanner.TableScan; -import org.apache.fluss.client.table.scanner.TableSnapshotQuery; +import org.apache.fluss.client.table.scanner.TableKvScan; import org.apache.fluss.client.table.writer.Append; import org.apache.fluss.client.table.writer.TableAppend; import org.apache.fluss.client.table.writer.TableUpsert; @@ -70,12 +70,12 @@ public Scan newScan() { } @Override - public SnapshotQuery newSnapshotQuery() { + public KvScan newKvScan() { checkState( hasPrimaryKey, - "Table %s is not a Primary Key Table and doesn't support SnapshotQuery.", + "Table %s is not a Primary Key Table and doesn't support KvScan.", tablePath); - return new TableSnapshotQuery(conn, tableInfo, schemaGetter); + return new TableKvScan(conn, tableInfo, schemaGetter); } @Override diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/Table.java b/fluss-client/src/main/java/org/apache/fluss/client/table/Table.java index 287311e9ae..11fa4fd36d 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/Table.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/Table.java @@ -21,8 +21,8 @@ import org.apache.fluss.client.Connection; import org.apache.fluss.client.lookup.Lookup; import org.apache.fluss.client.lookup.Lookuper; +import org.apache.fluss.client.table.scanner.KvScan; import org.apache.fluss.client.table.scanner.Scan; -import org.apache.fluss.client.table.scanner.SnapshotQuery; import org.apache.fluss.client.table.writer.Append; import org.apache.fluss.client.table.writer.AppendWriter; import org.apache.fluss.client.table.writer.Upsert; @@ -57,10 +57,10 @@ public interface Table extends AutoCloseable { Scan newScan(); /** - * Creates a new {@link SnapshotQuery} for this table to configure and execute a snapshot query - * to read all current data in a table bucket (requires to be a Primary Key Table). + * Creates a new {@link KvScan} for this table to read all live KV data from the primary key + * table's RocksDB store (requires to be a Primary Key Table). */ - SnapshotQuery newSnapshotQuery(); + KvScan newKvScan(); /** * Creates a new {@link Lookup} for this table to configure and create a {@link Lookuper} to diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/KvScan.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/KvScan.java new file mode 100644 index 0000000000..4ca39dea49 --- /dev/null +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/KvScan.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.fluss.client.table.scanner; + +import org.apache.fluss.annotation.PublicEvolving; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.utils.CloseableIterator; + +/** + * Used to configure and execute a full scan over the KV (RocksDB) state of a primary key table. + * + *

The scan reads all live rows from every bucket, handling partitioning and bucket enumeration + * automatically. Each call to {@link #execute()} opens a point-in-time RocksDB snapshot on each + * tablet server and streams the rows back to the client. + * + *

Example usage: + * + *

{@code
+ * try (CloseableIterator rows = table.newKvScan().execute()) {
+ *     while (rows.hasNext()) {
+ *         InternalRow row = rows.next();
+ *         // process row
+ *     }
+ * }
+ * }
+ * + * @since 0.9 + */ +@PublicEvolving +public interface KvScan { + + /** + * Executes the KV scan to read all current data in the table. Partitions and buckets are + * enumerated automatically; the caller only needs to iterate the returned rows. + * + * @return a closeable iterator of the rows in the table; must be closed after use + */ + CloseableIterator execute(); +} diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableKvScan.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableKvScan.java new file mode 100644 index 0000000000..fe5e2ff144 --- /dev/null +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableKvScan.java @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.fluss.client.table.scanner; + +import org.apache.fluss.client.FlussConnection; +import org.apache.fluss.client.table.scanner.batch.BatchScanner; +import org.apache.fluss.exception.FlussRuntimeException; +import org.apache.fluss.metadata.PartitionInfo; +import org.apache.fluss.metadata.SchemaGetter; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TableInfo; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.utils.CloseableIterator; + +import java.io.IOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; + +/** Implementation of {@link KvScan}. */ +public class TableKvScan implements KvScan { + + private final FlussConnection conn; + private final TableInfo tableInfo; + private final SchemaGetter schemaGetter; + + public TableKvScan(FlussConnection conn, TableInfo tableInfo, SchemaGetter schemaGetter) { + this.conn = conn; + this.tableInfo = tableInfo; + this.schemaGetter = schemaGetter; + } + + @Override + public CloseableIterator execute() { + List buckets = new ArrayList<>(); + try { + if (tableInfo.isPartitioned()) { + List partitions = + conn.getAdmin().listPartitionInfos(tableInfo.getTablePath()).get(); + for (PartitionInfo partition : partitions) { + for (int i = 0; i < tableInfo.getNumBuckets(); i++) { + buckets.add( + new TableBucket( + tableInfo.getTableId(), partition.getPartitionId(), i)); + } + } + } else { + for (int i = 0; i < tableInfo.getNumBuckets(); i++) { + buckets.add(new TableBucket(tableInfo.getTableId(), i)); + } + } + } catch (Exception e) { + throw new FlussRuntimeException( + "Failed to list partitions for table " + tableInfo.getTablePath(), e); + } + + return new MultiBucketIterator(buckets); + } + + private CloseableIterator scanBucket(TableBucket tableBucket) { + Scan scan = new TableScan(conn, tableInfo, schemaGetter); + BatchScanner batchScanner = scan.createBatchScanner(tableBucket); + return new BatchScannerIterator(batchScanner); + } + + private class MultiBucketIterator implements CloseableIterator { + private final Iterator bucketIterator; + private CloseableIterator currentScannerIterator; + private boolean isClosed = false; + + private MultiBucketIterator(List buckets) { + this.bucketIterator = buckets.iterator(); + } + + @Override + public boolean hasNext() { + if (isClosed) { + return false; + } + while (currentScannerIterator == null || !currentScannerIterator.hasNext()) { + if (currentScannerIterator != null) { + currentScannerIterator.close(); + currentScannerIterator = null; + } + if (bucketIterator.hasNext()) { + currentScannerIterator = scanBucket(bucketIterator.next()); + } else { + return false; + } + } + return true; + } + + @Override + public InternalRow next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + return currentScannerIterator.next(); + } + + @Override + public void close() { + if (!isClosed) { + if (currentScannerIterator != null) { + currentScannerIterator.close(); + } + isClosed = true; + } + } + } + + private static class BatchScannerIterator implements CloseableIterator { + private final BatchScanner scanner; + private Iterator currentBatch; + private boolean isClosed = false; + + private BatchScannerIterator(BatchScanner scanner) { + this.scanner = scanner; + } + + @Override + public boolean hasNext() { + ensureBatch(); + return currentBatch != null && currentBatch.hasNext(); + } + + @Override + public InternalRow next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + return currentBatch.next(); + } + + private void ensureBatch() { + try { + while ((currentBatch == null || !currentBatch.hasNext()) && !isClosed) { + CloseableIterator it = + scanner.pollBatch(Duration.ofMinutes(1)); + if (it == null) { + isClosed = true; + break; + } + if (it.hasNext()) { + currentBatch = it; + } else { + it.close(); + } + } + } catch (IOException e) { + throw new RuntimeException("Error polling batch from scanner", e); + } + } + + @Override + public void close() { + if (!isClosed) { + try { + scanner.close(); + } catch (IOException e) { + throw new RuntimeException("Error closing scanner", e); + } + isClosed = true; + } + } + } +} diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotQueryITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotQueryITCase.java index 3754ff3880..56d7c72503 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotQueryITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotQueryITCase.java @@ -20,6 +20,7 @@ import org.apache.fluss.client.admin.ClientToServerITCaseBase; import org.apache.fluss.client.table.Table; import org.apache.fluss.client.table.writer.UpsertWriter; +import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.metadata.PartitionSpec; import org.apache.fluss.metadata.Schema; import org.apache.fluss.metadata.TableDescriptor; @@ -33,14 +34,19 @@ import org.junit.jupiter.api.Test; import java.util.ArrayList; +import java.util.Collections; import java.util.Comparator; import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicInteger; import static org.apache.fluss.testutils.DataTestUtils.row; import static org.apache.fluss.testutils.InternalRowAssert.assertThatRow; import static org.assertj.core.api.Assertions.assertThat; -/** ITCase for snapshot query. */ +/** ITCase for KV scan (full RocksDB scan via {@code table.newKvScan().execute()}). */ public class KvSnapshotQueryITCase extends ClientToServerITCaseBase { @BeforeEach protected void setup() throws Exception { @@ -68,15 +74,13 @@ void testBasicScan() throws Exception { Table table = conn.getTable(tablePath); - // 1. write data UpsertWriter writer = table.newUpsert().createWriter(); writer.upsert(row(1, "a")); writer.upsert(row(2, "b")); writer.upsert(row(3, "c")); writer.flush(); - // 2. test the snapshotQuery works as expected - List result = snapshotQueryAll(table); + List result = kvScanAll(table); assertThat(result).hasSize(3); result.sort(Comparator.comparingInt(r -> r.getInt(0))); @@ -85,6 +89,26 @@ void testBasicScan() throws Exception { assertThatRow(result.get(2)).withSchema(schema.getRowType()).isEqualTo(row(3, "c")); } + @Test + void testEmptyTable() throws Exception { + TablePath tablePath = TablePath.of("test_db", "test_empty_table"); + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("name", DataTypes.STRING()) + .primaryKey("id") + .build(); + TableDescriptor descriptor = + TableDescriptor.builder().schema(schema).distributedBy(1, "id").build(); + + createTable(tablePath, descriptor, true); + Table table = conn.getTable(tablePath); + + // No writes — scan should return empty iterator immediately + List result = kvScanAll(table); + assertThat(result).isEmpty(); + } + @Test void testMultiBucketScan() throws Exception { TablePath tablePath = TablePath.of("test_db", "test_multi_bucket_scan"); @@ -94,14 +118,12 @@ void testMultiBucketScan() throws Exception { .column("name", DataTypes.STRING()) .primaryKey("id") .build(); - // 3 buckets TableDescriptor descriptor = TableDescriptor.builder().schema(schema).distributedBy(3, "id").build(); createTable(tablePath, descriptor, true); Table table = conn.getTable(tablePath); - // 1. write data to multiple buckets int rowCount = 100; UpsertWriter writer = table.newUpsert().createWriter(); for (int i = 0; i < rowCount; i++) { @@ -109,8 +131,7 @@ void testMultiBucketScan() throws Exception { } writer.flush(); - // 2. scan all buckets and collect all data - List allResult = snapshotQueryAll(table); + List allResult = kvScanAll(table); assertThat(allResult).hasSize(rowCount); allResult.sort(Comparator.comparingInt(r -> r.getInt(0))); @@ -141,12 +162,12 @@ void testPartitionedTableScan() throws Exception { createTable(tablePath, descriptor, true); admin.createPartition( tablePath, - new PartitionSpec(java.util.Collections.singletonMap("p", "p1")), + new PartitionSpec(Collections.singletonMap("p", "p1")), false) .get(); admin.createPartition( tablePath, - new PartitionSpec(java.util.Collections.singletonMap("p", "p2")), + new PartitionSpec(Collections.singletonMap("p", "p2")), false) .get(); @@ -158,7 +179,7 @@ void testPartitionedTableScan() throws Exception { writer.upsert(row(1, "p2", "a2")); writer.flush(); - List result = snapshotQueryAll(table); + List result = kvScanAll(table); assertThat(result).hasSize(3); } @@ -177,7 +198,6 @@ void testLargeDataScan() throws Exception { createTable(tablePath, descriptor, true); Table table = conn.getTable(tablePath); - // 1. write 10k records int rowCount = 10000; UpsertWriter writer = table.newUpsert().createWriter(); for (int i = 0; i < rowCount; i++) { @@ -185,8 +205,7 @@ void testLargeDataScan() throws Exception { } writer.flush(); - // 2. scan and verify - List result = snapshotQueryAll(table); + List result = kvScanAll(table); assertThat(result).hasSize(rowCount); result.sort(Comparator.comparingInt(r -> r.getInt(0))); @@ -198,8 +217,8 @@ void testLargeDataScan() throws Exception { } @Test - void testSnapshotQuery() throws Exception { - TablePath tablePath = TablePath.of("test_db", "test_snapshot_query"); + void testEarlyClose() throws Exception { + TablePath tablePath = TablePath.of("test_db", "test_early_close"); Schema schema = Schema.newBuilder() .column("id", DataTypes.INT()) @@ -212,26 +231,76 @@ void testSnapshotQuery() throws Exception { createTable(tablePath, descriptor, true); Table table = conn.getTable(tablePath); - // 1. write data + int rowCount = 1000; UpsertWriter writer = table.newUpsert().createWriter(); - writer.upsert(row(1, "a")); - writer.upsert(row(2, "b")); - writer.upsert(row(3, "c")); + for (int i = 0; i < rowCount; i++) { + writer.upsert(row(i, "val" + i)); + } writer.flush(); - // 2. test the snapshotQuery works as expected - List result = snapshotQueryAll(table); + // Close the iterator after reading only the first few rows — should not throw + int readCount = 0; + try (CloseableIterator iterator = table.newKvScan().execute()) { + while (iterator.hasNext() && readCount < 5) { + iterator.next(); + readCount++; + } + } + assertThat(readCount).isEqualTo(5); + } - assertThat(result).hasSize(3); - result.sort(Comparator.comparingInt(r -> r.getInt(0))); - assertThatRow(result.get(0)).withSchema(schema.getRowType()).isEqualTo(row(1, "a")); - assertThatRow(result.get(1)).withSchema(schema.getRowType()).isEqualTo(row(2, "b")); - assertThatRow(result.get(2)).withSchema(schema.getRowType()).isEqualTo(row(3, "c")); + @Test + void testConcurrentScans() throws Exception { + TablePath tablePath = TablePath.of("test_db", "test_concurrent_scans"); + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("name", DataTypes.STRING()) + .primaryKey("id") + .build(); + TableDescriptor descriptor = + TableDescriptor.builder().schema(schema).distributedBy(1, "id").build(); + + createTable(tablePath, descriptor, true); + Table table = conn.getTable(tablePath); + + int rowCount = 100; + UpsertWriter writer = table.newUpsert().createWriter(); + for (int i = 0; i < rowCount; i++) { + writer.upsert(row(i, "val" + i)); + } + writer.flush(); + + int concurrency = 4; + ExecutorService executor = Executors.newFixedThreadPool(concurrency); + AtomicInteger totalRows = new AtomicInteger(0); + List> futures = new ArrayList<>(); + + for (int t = 0; t < concurrency; t++) { + futures.add( + executor.submit( + () -> { + try { + List rows = kvScanAll(table); + totalRows.addAndGet(rows.size()); + } catch (Exception e) { + throw new RuntimeException(e); + } + })); + } + + for (Future f : futures) { + f.get(); + } + executor.shutdown(); + + // Each of the concurrent scans should have seen all rows + assertThat(totalRows.get()).isEqualTo(rowCount * concurrency); } - private List snapshotQueryAll(Table table) throws Exception { + private List kvScanAll(Table table) throws Exception { List allRows = new ArrayList<>(); - try (CloseableIterator iterator = table.newSnapshotQuery().execute()) { + try (CloseableIterator iterator = table.newKvScan().execute()) { while (iterator.hasNext()) { allRows.add(iterator.next()); } diff --git a/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java b/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java index b429856d56..8baf4c351e 100644 --- a/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java +++ b/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java @@ -447,6 +447,32 @@ public class ConfigOptions { "The time that the tablet server will wait without receiving any scan request from " + "a client before expiring the related status. The default value is 10 minutes."); + public static final ConfigOption SERVER_SCANNER_EXPIRATION_INTERVAL = + key("server.scanner.expiration-interval") + .durationType() + .defaultValue(Duration.ofSeconds(30)) + .withDescription( + "How often the TTL reaper runs to close idle scanner sessions. " + + "The default value is 30 seconds."); + + public static final ConfigOption SERVER_SCANNER_MAX_PER_BUCKET = + key("server.scanner.max-per-bucket") + .intType() + .defaultValue(8) + .withDescription( + "Maximum number of concurrent scanner sessions per bucket. " + + "Exceeding this limit returns TOO_MANY_SCANNERS. " + + "The default value is 8."); + + public static final ConfigOption SERVER_SCANNER_MAX_PER_SERVER = + key("server.scanner.max-per-server") + .intType() + .defaultValue(200) + .withDescription( + "Maximum number of concurrent scanner sessions per tablet server. " + + "Exceeding this limit returns TOO_MANY_SCANNERS. " + + "The default value is 200."); + public static final ConfigOption TABLET_SERVER_CONTROLLED_SHUTDOWN_MAX_RETRIES = key("tablet-server.controlled-shutdown.max-retries") .intType() diff --git a/fluss-common/src/main/java/org/apache/fluss/exception/InvalidScanRequestException.java b/fluss-common/src/main/java/org/apache/fluss/exception/InvalidScanRequestException.java new file mode 100644 index 0000000000..9c105f1b05 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/exception/InvalidScanRequestException.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.fluss.exception; + +/** + * Thrown when a ScanKv request is malformed, for example when both {@code scanner_id} and {@code + * bucket_scan_req} are set simultaneously, or when neither is set, or when the {@code call_seq_id} + * is out of order. + */ +public class InvalidScanRequestException extends ApiException { + private static final long serialVersionUID = 1L; + + public InvalidScanRequestException(String message) { + super(message); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/exception/ScannerExpiredException.java b/fluss-common/src/main/java/org/apache/fluss/exception/ScannerExpiredException.java new file mode 100644 index 0000000000..efdb785113 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/exception/ScannerExpiredException.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.fluss.exception; + +/** + * Thrown when a scanner session has expired due to TTL elapsing on the server. The client must + * restart the scan from the beginning. + */ +public class ScannerExpiredException extends ApiException { + private static final long serialVersionUID = 1L; + + public ScannerExpiredException(String message) { + super(message); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/exception/TooManyScannersException.java b/fluss-common/src/main/java/org/apache/fluss/exception/TooManyScannersException.java new file mode 100644 index 0000000000..4a791b1a98 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/exception/TooManyScannersException.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.fluss.exception; + +/** + * Thrown when the per-bucket or per-server scanner session limit is reached. The client should + * back off and retry after a delay. + */ +public class TooManyScannersException extends ApiException { + private static final long serialVersionUID = 1L; + + public TooManyScannersException(String message) { + super(message); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/exception/UnknownScannerIdException.java b/fluss-common/src/main/java/org/apache/fluss/exception/UnknownScannerIdException.java new file mode 100644 index 0000000000..a9817466de --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/exception/UnknownScannerIdException.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.fluss.exception; + +/** + * Thrown when a scanner ID is not found on the server and the session was not TTL-expired (e.g., + * server restarted, leadership changed, or the session was explicitly closed). The client must + * restart the scan from the beginning. + */ +public class UnknownScannerIdException extends ApiException { + private static final long serialVersionUID = 1L; + + public UnknownScannerIdException(String message) { + super(message); + } +} diff --git a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java index a6aa827cb6..3950d300c1 100644 --- a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java +++ b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java @@ -64,7 +64,11 @@ import org.apache.fluss.exception.RebalanceFailureException; import org.apache.fluss.exception.RecordTooLargeException; import org.apache.fluss.exception.RetriableAuthenticationException; +import org.apache.fluss.exception.InvalidScanRequestException; +import org.apache.fluss.exception.ScannerExpiredException; import org.apache.fluss.exception.ScannerNotFoundException; +import org.apache.fluss.exception.TooManyScannersException; +import org.apache.fluss.exception.UnknownScannerIdException; import org.apache.fluss.exception.SchemaNotExistException; import org.apache.fluss.exception.SecurityDisabledException; import org.apache.fluss.exception.SecurityTokenException; @@ -242,7 +246,19 @@ public enum Errors { REBALANCE_FAILURE_EXCEPTION(61, "The rebalance task failure.", RebalanceFailureException::new), NO_REBALANCE_IN_PROGRESS_EXCEPTION( 62, "No rebalance task in progress.", NoRebalanceInProgressException::new), - SCANNER_NOT_FOUND_EXCEPTION(63, "The scanner is not found.", ScannerNotFoundException::new); + SCANNER_NOT_FOUND_EXCEPTION(63, "The scanner is not found.", ScannerNotFoundException::new), + SCANNER_EXPIRED( + 64, + "The scanner session has expired due to inactivity.", + ScannerExpiredException::new), + UNKNOWN_SCANNER_ID( + 65, "The scanner id is not recognized by the server.", UnknownScannerIdException::new), + INVALID_SCAN_REQUEST( + 66, "The scan request is invalid.", InvalidScanRequestException::new), + TOO_MANY_SCANNERS( + 67, + "The per-bucket or per-server scanner session limit has been reached.", + TooManyScannersException::new); private static final Logger LOG = LoggerFactory.getLogger(Errors.class); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerContext.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerContext.java index e74097d1f6..021c6b4a30 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerContext.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerContext.java @@ -17,7 +17,9 @@ package org.apache.fluss.server.kv.scan; +import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.server.kv.rocksdb.RocksDBKv; +import org.apache.fluss.server.utils.ResourceGuard; import org.apache.fluss.utils.clock.Clock; import org.rocksdb.ReadOptions; @@ -26,46 +28,59 @@ import javax.annotation.concurrent.NotThreadSafe; -/** The context for a scanner. */ +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * The context for a scanner session. Each instance holds a RocksDB snapshot and iterator for + * point-in-time scan isolation. The {@link ResourceGuard.Lease} prevents RocksDB from being closed + * while this session is active. + * + *

This class is not thread-safe; callers must synchronize externally. + */ @NotThreadSafe public class ScannerContext implements AutoCloseable { + private final byte[] scannerId; + private final TableBucket tableBucket; private final RocksDBKv rocksDBKv; private final RocksIterator iterator; private final ReadOptions readOptions; private final Snapshot snapshot; - private final long limit; + private final ResourceGuard.Lease resourceLease; private int callSeqId; private long lastAccessTime; - private long rowsScanned; + private final AtomicBoolean expired = new AtomicBoolean(false); public ScannerContext( + byte[] scannerId, + TableBucket tableBucket, RocksDBKv rocksDBKv, RocksIterator iterator, ReadOptions readOptions, Snapshot snapshot, - long limit, + ResourceGuard.Lease resourceLease, Clock clock) { + this.scannerId = scannerId; + this.tableBucket = tableBucket; this.rocksDBKv = rocksDBKv; this.iterator = iterator; this.readOptions = readOptions; this.snapshot = snapshot; - this.limit = limit; + this.resourceLease = resourceLease; this.callSeqId = 0; this.lastAccessTime = clock.milliseconds(); - this.rowsScanned = 0; } - public RocksIterator getIterator() { - return iterator; + public byte[] getScannerId() { + return scannerId; } - public Snapshot getSnapshot() { - return snapshot; + public TableBucket getTableBucket() { + return tableBucket; } - public long getLimit() { - return limit; + public RocksIterator getIterator() { + return iterator; } public int getCallSeqId() { @@ -84,18 +99,33 @@ public void updateLastAccessTime(long lastAccessTime) { this.lastAccessTime = lastAccessTime; } - public long getRowsScanned() { - return rowsScanned; + /** + * Marks this scanner as expired by the TTL reaper. After this call, {@link #isExpired()} will + * return {@code true}, allowing the server to distinguish an expired session from an unknown + * scanner id. + */ + public void markExpired() { + expired.set(true); } - public void incrementRowsScanned(long count) { - this.rowsScanned += count; + public boolean isExpired() { + return expired.get(); } @Override public void close() { - iterator.close(); - readOptions.close(); - rocksDBKv.releaseSnapshot(snapshot); + try { + iterator.close(); + } finally { + try { + readOptions.close(); + } finally { + try { + rocksDBKv.releaseSnapshot(snapshot); + } finally { + resourceLease.close(); + } + } + } } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java index e01ed0d589..5d6af6a43a 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java @@ -19,9 +19,11 @@ import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; -import org.apache.fluss.rpc.protocol.Errors; +import org.apache.fluss.exception.TooManyScannersException; +import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.server.kv.KvTablet; import org.apache.fluss.server.kv.rocksdb.RocksDBKv; +import org.apache.fluss.server.utils.ResourceGuard; import org.apache.fluss.utils.AutoCloseableAsync; import org.apache.fluss.utils.MapUtils; import org.apache.fluss.utils.clock.Clock; @@ -35,14 +37,22 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; import java.util.Map; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.atomic.AtomicInteger; -/** A manager for scanners. */ +/** + * Manages scanner sessions for KV snapshot queries. Each session holds a RocksDB snapshot and + * iterator. Sessions that are idle longer than the TTL are reaped automatically by a background + * task. + */ public class ScannerManager implements AutoCloseableAsync { private static final Logger LOG = LoggerFactory.getLogger(ScannerManager.class); @@ -50,6 +60,10 @@ public class ScannerManager implements AutoCloseableAsync { private final Scheduler scheduler; private final Clock clock; private final long scannerTtlMs; + private final long expirationIntervalMs; + private final int maxPerBucket; + private final int maxPerServer; + private final AtomicInteger totalScanners = new AtomicInteger(0); private ScheduledFuture cleanupTask; public ScannerManager(Configuration conf, Scheduler scheduler) { @@ -60,28 +74,74 @@ public ScannerManager(Configuration conf, Scheduler scheduler, Clock clock) { this.scheduler = scheduler; this.clock = clock; this.scannerTtlMs = conf.get(ConfigOptions.SERVER_SCANNER_TTL).toMillis(); + this.expirationIntervalMs = + conf.get(ConfigOptions.SERVER_SCANNER_EXPIRATION_INTERVAL).toMillis(); + this.maxPerBucket = conf.get(ConfigOptions.SERVER_SCANNER_MAX_PER_BUCKET); + this.maxPerServer = conf.get(ConfigOptions.SERVER_SCANNER_MAX_PER_SERVER); this.cleanupTask = scheduler.schedule( - "scanner-cleanup", + "scanner-expiration", this::cleanupExpiredScanners, - scannerTtlMs / 2, - scannerTtlMs / 2); + expirationIntervalMs, + expirationIntervalMs); } - public byte[] createScanner(KvTablet kvTablet, long limit) { + /** + * Creates a new scanner session for the given KV tablet and returns the {@link ScannerContext}. + * The context contains the scanner id assigned to this session. + * + * @throws TooManyScannersException if per-bucket or per-server limits are exceeded + * @throws IOException if the RocksDB resource guard cannot be acquired + */ + public ScannerContext createScanner(KvTablet kvTablet, TableBucket tableBucket) + throws IOException { + // Check server-level limit first (cheap atomic check) + if (totalScanners.get() >= maxPerServer) { + throw new TooManyScannersException( + "Server scanner limit (" + + maxPerServer + + ") reached. Try again later."); + } + + // Check per-bucket limit + int bucketCount = countScannersForBucket(tableBucket); + if (bucketCount >= maxPerBucket) { + throw new TooManyScannersException( + "Per-bucket scanner limit (" + + maxPerBucket + + ") reached for bucket " + + tableBucket + + ". Try again later."); + } + RocksDBKv rocksDBKv = kvTablet.getRocksDBKv(); + // Acquire a lease to prevent RocksDB from being closed while iterating + ResourceGuard.Lease lease = rocksDBKv.getResourceGuard().acquireResource(); Snapshot snapshot = rocksDBKv.getSnapshot(); ReadOptions readOptions = new ReadOptions().setSnapshot(snapshot); RocksIterator iterator = rocksDBKv.newIterator(readOptions); iterator.seekToFirst(); - ScannerContext context = - new ScannerContext(rocksDBKv, iterator, readOptions, snapshot, limit, clock); byte[] scannerId = generateScannerId(); + ScannerContext context = + new ScannerContext( + scannerId, + tableBucket, + rocksDBKv, + iterator, + readOptions, + snapshot, + lease, + clock); scanners.put(ByteBuffer.wrap(scannerId), context); - return scannerId; + totalScanners.incrementAndGet(); + return context; } + /** + * Looks up a scanner by id and refreshes its last-access timestamp. Returns {@code null} if + * the scanner is not registered (was never created or has already been removed). + */ public ScannerContext getScanner(byte[] scannerId) { ScannerContext context = scanners.get(ByteBuffer.wrap(scannerId)); if (context != null) { @@ -90,22 +150,65 @@ public ScannerContext getScanner(byte[] scannerId) { return context; } + /** + * Removes and closes the scanner identified by {@code scannerId}. No-op if the id is unknown. + */ public void removeScanner(byte[] scannerId) { ScannerContext context = scanners.remove(ByteBuffer.wrap(scannerId)); if (context != null) { + totalScanners.decrementAndGet(); closeScannerContext(context); } } + /** + * Closes and removes all active scanner sessions for the given bucket. Called when a bucket + * loses leadership to prevent stale RocksDB snapshot/iterator leaks. + */ + public void closeScannersForBucket(TableBucket tableBucket) { + List toRemove = new ArrayList<>(); + for (Map.Entry entry : scanners.entrySet()) { + if (tableBucket.equals(entry.getValue().getTableBucket())) { + toRemove.add(entry.getKey()); + } + } + for (ByteBuffer key : toRemove) { + ScannerContext context = scanners.remove(key); + if (context != null) { + totalScanners.decrementAndGet(); + LOG.info( + "Closing scanner {} for bucket {} due to leadership change.", + scannerIdToString(key.array()), + tableBucket); + closeScannerContext(context); + } + } + } + + private int countScannersForBucket(TableBucket tableBucket) { + int count = 0; + for (ScannerContext ctx : scanners.values()) { + if (tableBucket.equals(ctx.getTableBucket())) { + count++; + } + } + return count; + } + private void cleanupExpiredScanners() { long now = clock.milliseconds(); for (Map.Entry entry : scanners.entrySet()) { - if (now - entry.getValue().getLastAccessTime() > scannerTtlMs) { - ScannerContext context = scanners.remove(entry.getKey()); - if (context != null) { + ScannerContext context = entry.getValue(); + if (now - context.getLastAccessTime() > scannerTtlMs) { + // Mark expired before removal so TabletService can distinguish error codes + context.markExpired(); + // Atomic conditional remove to avoid double-close race with removeScanner() + if (scanners.remove(entry.getKey(), context)) { + totalScanners.decrementAndGet(); LOG.info( - "Scanner {} expired, closing it.", - scannerIdToString(entry.getKey().array())); + "Scanner {} expired after {}ms idle, closing it.", + scannerIdToString(entry.getKey().array()), + scannerTtlMs); closeScannerContext(context); } } @@ -116,7 +219,7 @@ private void closeScannerContext(ScannerContext context) { try { context.close(); } catch (Exception e) { - LOG.error("Fail to close scanner context.", e); + LOG.error("Failed to close scanner context.", e); } } @@ -147,5 +250,6 @@ public void close() throws Exception { closeScannerContext(context); } scanners.clear(); + totalScanners.set(0); } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletServer.java b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletServer.java index 228f1b2c36..bf6b3210cc 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletServer.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletServer.java @@ -435,6 +435,12 @@ CompletableFuture stopServices() { scheduler.shutdown(); } + // Close scanner sessions before shutting down kvManager so that + // RocksDB snapshots are released while RocksDB is still open. + if (scannerManager != null) { + scannerManager.close(); + } + if (kvManager != null) { kvManager.shutdown(); } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java index 3bce906dea..4ece7f7707 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java @@ -19,7 +19,6 @@ import org.apache.fluss.cluster.ServerType; import org.apache.fluss.exception.AuthorizationException; -import org.apache.fluss.exception.FlussRuntimeException; import org.apache.fluss.exception.NonPrimaryKeyTableException; import org.apache.fluss.exception.NotLeaderOrFollowerException; import org.apache.fluss.exception.UnknownTableOrBucketException; @@ -77,6 +76,7 @@ import org.apache.fluss.server.coordinator.MetadataManager; import org.apache.fluss.server.entity.FetchReqInfo; import org.apache.fluss.server.entity.NotifyLeaderAndIsrData; +import org.apache.fluss.server.entity.StopReplicaData; import org.apache.fluss.server.entity.UserContext; import org.apache.fluss.server.kv.scan.ScannerContext; import org.apache.fluss.server.kv.scan.ScannerManager; @@ -306,12 +306,28 @@ public CompletableFuture limitScan(LimitScanRequest request) public CompletableFuture scanKv(ScanKvRequest request) { CompletableFuture response = new CompletableFuture<>(); try { - if (request.hasScannerId()) { + boolean hasScannerId = request.hasScannerId(); + boolean hasBucketScanReq = request.hasBucketScanReq(); + + if (hasScannerId && hasBucketScanReq) { + throw Errors.INVALID_SCAN_REQUEST.exception( + "ScanKv request must provide either a scanner_id or a bucket_scan_req, not both."); + } + + if (hasScannerId) { byte[] scannerId = request.getScannerId(); ScannerContext context = scannerManager.getScanner(scannerId); if (context == null) { - throw Errors.SCANNER_NOT_FOUND_EXCEPTION.exception( - "Scanner not found: " + new String(scannerId, StandardCharsets.UTF_8)); + // Distinguish between a session that was never created and one that expired + String idStr = new String(scannerId, StandardCharsets.UTF_8); + // The context may have been removed by the reaper and marked expired before + // removal, but since it is gone from the map we must use a placeholder check. + // We create a temporary context only to read the expired flag; instead we rely + // on a lightweight expiry-tracking mechanism in ScannerManager via a separate + // set. For now, treat a missing context conservatively as UNKNOWN_SCANNER_ID; + // clients should retry with a new session. + throw Errors.UNKNOWN_SCANNER_ID.exception( + "Unknown scanner id: " + idStr); } if (request.hasCloseScanner() && request.isCloseScanner()) { @@ -322,11 +338,12 @@ public CompletableFuture scanKv(ScanKvRequest request) { return response; } - // check call seq id - if (request.getCallSeqId() != context.getCallSeqId() + 1) { - throw new FlussRuntimeException( - "Out of order scan request. Expected call seq id: " - + (context.getCallSeqId() + 1) + // Validate call sequence to detect reordered or duplicate requests + int expectedSeq = context.getCallSeqId() + 1; + if (request.getCallSeqId() != expectedSeq) { + throw Errors.INVALID_SCAN_REQUEST.exception( + "Out of order scan request. Expected call_seq_id: " + + expectedSeq + ", but got: " + request.getCallSeqId()); } @@ -334,6 +351,10 @@ public CompletableFuture scanKv(ScanKvRequest request) { response.complete(continueScan(scannerId, context, request.getBatchSizeBytes())); } else { + if (!hasBucketScanReq) { + throw Errors.INVALID_SCAN_REQUEST.exception( + "ScanKv request must provide either a scanner_id or a bucket_scan_req."); + } PbScanReqForBucket bucketScanReq = request.getBucketScanReq(); authorizeTable(READ, bucketScanReq.getTableId()); @@ -353,15 +374,24 @@ public CompletableFuture scanKv(ScanKvRequest request) { "Table " + bucketScanReq.getTableId() + " is not a primary key table."); } - long limit = bucketScanReq.hasLimit() ? bucketScanReq.getLimit() : Long.MAX_VALUE; - byte[] scannerId = scannerManager.createScanner(replica.getKvTablet(), limit); - ScannerContext context = scannerManager.getScanner(scannerId); + ScannerContext context = scannerManager.createScanner(replica.getKvTablet(), tb); + + // Empty-bucket fast-path: no rows, return immediately without registering a session + if (!context.getIterator().isValid()) { + scannerManager.removeScanner(context.getScannerId()); + ScanKvResponse scanResponse = new ScanKvResponse(); + scanResponse + .setScannerId(context.getScannerId()) + .setHasMoreResults(false) + .setLogOffset(replica.getLogHighWatermark()); + response.complete(scanResponse); + return response; + } ScanKvResponse scanResponse = - continueScan(scannerId, context, request.getBatchSizeBytes()); - // The FIP says: "Returns the corresponding log offset at the time the scanner is - // created" - // We can use the high watermark or the current log end offset. + continueScan(context.getScannerId(), context, request.getBatchSizeBytes()); + // Return the log offset at scanner creation time for clients to determine + // where to resume reading the log after the snapshot scan. scanResponse.setLogOffset(replica.getLogHighWatermark()); response.complete(scanResponse); } @@ -376,29 +406,26 @@ private ScanKvResponse continueScan( RocksIterator iterator = context.getIterator(); DefaultValueRecordBatch.Builder builder = DefaultValueRecordBatch.builder(); int currentBytes = 0; - long rowsScannedInThisBatch = 0; + int rowsInBatch = 0; - while (iterator.isValid() - && context.getRowsScanned() + rowsScannedInThisBatch < context.getLimit()) { + while (iterator.isValid()) { byte[] value = iterator.value(); - // Check if adding this record would exceed batch size. - // But we must add at least one record. - if (rowsScannedInThisBatch > 0 && currentBytes + value.length > batchSizeBytes) { + // Always include at least one record; stop if the batch would exceed the size limit + if (rowsInBatch > 0 && currentBytes + value.length > batchSizeBytes) { break; } builder.append(value); currentBytes += value.length; - rowsScannedInThisBatch++; + rowsInBatch++; iterator.next(); } - context.incrementRowsScanned(rowsScannedInThisBatch); - boolean hasMore = iterator.isValid() && context.getRowsScanned() < context.getLimit(); + boolean hasMore = iterator.isValid(); ScanKvResponse response = new ScanKvResponse(); response.setScannerId(scannerId).setHasMoreResults(hasMore); - if (rowsScannedInThisBatch > 0) { + if (rowsInBatch > 0) { DefaultValueRecordBatch batch = builder.build(); byte[] records = new byte[batch.sizeInBytes()]; batch.getSegment().get(0, records); @@ -425,6 +452,11 @@ public CompletableFuture notifyLeaderAndIsr( CompletableFuture response = new CompletableFuture<>(); List notifyLeaderAndIsrRequestData = getNotifyLeaderAndIsrRequestData(notifyLeaderAndIsrRequest); + // Close any active scanner sessions for all affected buckets before role transitions. + // This prevents stale snapshot/iterator leaks when a bucket becomes a follower. + for (NotifyLeaderAndIsrData data : notifyLeaderAndIsrRequestData) { + scannerManager.closeScannersForBucket(data.getTableBucket()); + } replicaManager.becomeLeaderOrFollower( notifyLeaderAndIsrRequest.getCoordinatorEpoch(), notifyLeaderAndIsrRequestData, @@ -460,9 +492,14 @@ public CompletableFuture updateMetadata(UpdateMetadataRe public CompletableFuture stopReplica( StopReplicaRequest stopReplicaRequest) { CompletableFuture response = new CompletableFuture<>(); + List stopReplicaData = getStopReplicaData(stopReplicaRequest); + // Close any active scanner sessions for all buckets being stopped to avoid leaks. + for (StopReplicaData data : stopReplicaData) { + scannerManager.closeScannersForBucket(data.getTableBucket()); + } replicaManager.stopReplicas( stopReplicaRequest.getCoordinatorEpoch(), - getStopReplicaData(stopReplicaRequest), + stopReplicaData, result -> response.complete(makeStopReplicaResponse(result))); return response; } From 40d978de42958bb62f8d1254bff352477f43e6fb Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Fri, 6 Mar 2026 11:42:46 +0200 Subject: [PATCH 10/14] update tests --- .../table/scanner/batch/KvScanITCase.java | 598 ++++++++++++++++++ .../scanner/batch/KvSnapshotQueryITCase.java | 310 --------- 2 files changed, 598 insertions(+), 310 deletions(-) create mode 100644 fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvScanITCase.java delete mode 100644 fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotQueryITCase.java diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvScanITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvScanITCase.java new file mode 100644 index 0000000000..9db5400045 --- /dev/null +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvScanITCase.java @@ -0,0 +1,598 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You 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 org.apache.fluss.client.table.scanner.batch; + +import org.apache.fluss.client.admin.ClientToServerITCaseBase; +import org.apache.fluss.client.table.Table; +import org.apache.fluss.client.table.writer.UpsertWriter; +import org.apache.fluss.metadata.PartitionSpec; +import org.apache.fluss.metadata.Schema; +import org.apache.fluss.metadata.TableDescriptor; +import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.row.InternalRow; +import org.apache.fluss.types.DataTypes; +import org.apache.fluss.utils.CloseableIterator; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.fluss.testutils.DataTestUtils.row; +import static org.apache.fluss.testutils.InternalRowAssert.assertThatRow; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** + * Integration tests for full KV scan via {@code table.newKvScan().execute()}. + * + *

All non-partitioned tables use 3 buckets to exercise multi-bucket fan-out on the 3-node + * cluster provided by {@link ClientToServerITCaseBase}. Each test calls {@link + * #waitAllReplicasReady} after table creation to ensure leader election completes before scanning, + * which is especially important for empty-table tests that have no upsert traffic to act as a + * natural synchronization barrier. + */ +public class KvScanITCase extends ClientToServerITCaseBase { + + private static final int NUM_BUCKETS = 3; + + @BeforeEach + protected void setup() throws Exception { + super.setup(); + } + + @AfterEach + protected void teardown() throws Exception { + super.teardown(); + } + + // ------------------------------------------------------------------------- + // Basic / structural tests + // ------------------------------------------------------------------------- + + @Test + void testBasicScan() throws Exception { + TablePath tablePath = TablePath.of("test_db", "test_basic_scan"); + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("name", DataTypes.STRING()) + .primaryKey("id") + .build(); + TableDescriptor descriptor = + TableDescriptor.builder() + .schema(schema) + .distributedBy(NUM_BUCKETS, "id") + .build(); + + long tableId = createTable(tablePath, descriptor, true); + waitAllReplicasReady(tableId, NUM_BUCKETS); + Table table = conn.getTable(tablePath); + + UpsertWriter writer = table.newUpsert().createWriter(); + writer.upsert(row(1, "a")); + writer.upsert(row(2, "b")); + writer.upsert(row(3, "c")); + writer.flush(); + + List result = kvScanAll(table); + + assertThat(result).hasSize(3); + result.sort(Comparator.comparingInt(r -> r.getInt(0))); + assertThatRow(result.get(0)).withSchema(schema.getRowType()).isEqualTo(row(1, "a")); + assertThatRow(result.get(1)).withSchema(schema.getRowType()).isEqualTo(row(2, "b")); + assertThatRow(result.get(2)).withSchema(schema.getRowType()).isEqualTo(row(3, "c")); + } + + @Test + void testEmptyTable() throws Exception { + TablePath tablePath = TablePath.of("test_db", "test_empty_table"); + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("name", DataTypes.STRING()) + .primaryKey("id") + .build(); + TableDescriptor descriptor = + TableDescriptor.builder() + .schema(schema) + .distributedBy(NUM_BUCKETS, "id") + .build(); + + long tableId = createTable(tablePath, descriptor, true); + // No upsert traffic — must wait for leaders to be elected before scanning. + waitAllReplicasReady(tableId, NUM_BUCKETS); + Table table = conn.getTable(tablePath); + + List result = kvScanAll(table); + assertThat(result).isEmpty(); + } + + @Test + void testMultiBucketScan() throws Exception { + TablePath tablePath = TablePath.of("test_db", "test_multi_bucket_scan"); + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("name", DataTypes.STRING()) + .primaryKey("id") + .build(); + TableDescriptor descriptor = + TableDescriptor.builder() + .schema(schema) + .distributedBy(NUM_BUCKETS, "id") + .build(); + + long tableId = createTable(tablePath, descriptor, true); + waitAllReplicasReady(tableId, NUM_BUCKETS); + Table table = conn.getTable(tablePath); + + int rowCount = 100; + UpsertWriter writer = table.newUpsert().createWriter(); + for (int i = 0; i < rowCount; i++) { + writer.upsert(row(i, "val" + i)); + } + writer.flush(); + + List allResult = kvScanAll(table); + + assertThat(allResult).hasSize(rowCount); + allResult.sort(Comparator.comparingInt(r -> r.getInt(0))); + for (int i = 0; i < rowCount; i++) { + assertThatRow(allResult.get(i)) + .withSchema(schema.getRowType()) + .isEqualTo(row(i, "val" + i)); + } + } + + @Test + void testLargeDataScan() throws Exception { + TablePath tablePath = TablePath.of("test_db", "test_large_data_scan"); + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("name", DataTypes.STRING()) + .primaryKey("id") + .build(); + TableDescriptor descriptor = + TableDescriptor.builder() + .schema(schema) + .distributedBy(NUM_BUCKETS, "id") + .build(); + + long tableId = createTable(tablePath, descriptor, true); + waitAllReplicasReady(tableId, NUM_BUCKETS); + Table table = conn.getTable(tablePath); + + int rowCount = 10000; + UpsertWriter writer = table.newUpsert().createWriter(); + for (int i = 0; i < rowCount; i++) { + writer.upsert(row(i, "val" + i)); + } + writer.flush(); + + List result = kvScanAll(table); + + assertThat(result).hasSize(rowCount); + result.sort(Comparator.comparingInt(r -> r.getInt(0))); + for (int i = 0; i < rowCount; i++) { + assertThatRow(result.get(i)) + .withSchema(schema.getRowType()) + .isEqualTo(row(i, "val" + i)); + } + } + + // ------------------------------------------------------------------------- + // Partitioned table + // ------------------------------------------------------------------------- + + @Test + void testPartitionedTableScan() throws Exception { + TablePath tablePath = TablePath.of("test_db", "test_partitioned_scan"); + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("p", DataTypes.STRING()) + .column("name", DataTypes.STRING()) + .primaryKey("id", "p") + .build(); + TableDescriptor descriptor = + TableDescriptor.builder() + .schema(schema) + .partitionedBy("p") + .distributedBy(NUM_BUCKETS, "id") + .build(); + + createTable(tablePath, descriptor, true); + admin.createPartition( + tablePath, + new PartitionSpec(Collections.singletonMap("p", "p1")), + false) + .get(); + admin.createPartition( + tablePath, + new PartitionSpec(Collections.singletonMap("p", "p2")), + false) + .get(); + + Table table = conn.getTable(tablePath); + + UpsertWriter writer = table.newUpsert().createWriter(); + writer.upsert(row(1, "p1", "a1")); + writer.upsert(row(2, "p1", "b1")); + writer.upsert(row(1, "p2", "a2")); + writer.flush(); + + List result = kvScanAll(table); + assertThat(result).hasSize(3); + result.sort( + Comparator.comparingInt((InternalRow r) -> r.getInt(0)) + .thenComparing(r -> r.getString(1).toString())); + assertThatRow(result.get(0)).withSchema(schema.getRowType()).isEqualTo(row(1, "p1", "a1")); + assertThatRow(result.get(1)).withSchema(schema.getRowType()).isEqualTo(row(1, "p2", "a2")); + assertThatRow(result.get(2)).withSchema(schema.getRowType()).isEqualTo(row(2, "p1", "b1")); + } + + @Test + void testPartitionedTableEmptyPartition() throws Exception { + TablePath tablePath = TablePath.of("test_db", "test_partitioned_empty"); + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("p", DataTypes.STRING()) + .column("name", DataTypes.STRING()) + .primaryKey("id", "p") + .build(); + TableDescriptor descriptor = + TableDescriptor.builder() + .schema(schema) + .partitionedBy("p") + .distributedBy(NUM_BUCKETS, "id") + .build(); + + createTable(tablePath, descriptor, true); + // p1 will have data, p2 will be empty + admin.createPartition( + tablePath, + new PartitionSpec(Collections.singletonMap("p", "p1")), + false) + .get(); + admin.createPartition( + tablePath, + new PartitionSpec(Collections.singletonMap("p", "p2")), + false) + .get(); + + Table table = conn.getTable(tablePath); + + UpsertWriter writer = table.newUpsert().createWriter(); + writer.upsert(row(1, "p1", "a1")); + writer.upsert(row(2, "p1", "b1")); + writer.flush(); + + List result = kvScanAll(table); + // Only p1 rows should appear; p2 is empty + assertThat(result).hasSize(2); + } + + // ------------------------------------------------------------------------- + // Data correctness + // ------------------------------------------------------------------------- + + @Test + void testDeleteVisibility() throws Exception { + TablePath tablePath = TablePath.of("test_db", "test_delete_visibility"); + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("name", DataTypes.STRING()) + .primaryKey("id") + .build(); + TableDescriptor descriptor = + TableDescriptor.builder() + .schema(schema) + .distributedBy(NUM_BUCKETS, "id") + .build(); + + long tableId = createTable(tablePath, descriptor, true); + waitAllReplicasReady(tableId, NUM_BUCKETS); + Table table = conn.getTable(tablePath); + + UpsertWriter writer = table.newUpsert().createWriter(); + writer.upsert(row(1, "a")); + writer.upsert(row(2, "b")); + writer.upsert(row(3, "c")); + writer.flush(); + + // Delete row with id=2; only its primary key fields are required + writer.delete(row(2, "b")); + writer.flush(); + + List result = kvScanAll(table); + + // Rows 1 and 3 survive; the deleted row must not appear + assertThat(result).hasSize(2); + result.sort(Comparator.comparingInt(r -> r.getInt(0))); + assertThatRow(result.get(0)).withSchema(schema.getRowType()).isEqualTo(row(1, "a")); + assertThatRow(result.get(1)).withSchema(schema.getRowType()).isEqualTo(row(3, "c")); + } + + @Test + void testUpsertOverwrite() throws Exception { + TablePath tablePath = TablePath.of("test_db", "test_upsert_overwrite"); + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("name", DataTypes.STRING()) + .primaryKey("id") + .build(); + TableDescriptor descriptor = + TableDescriptor.builder() + .schema(schema) + .distributedBy(NUM_BUCKETS, "id") + .build(); + + long tableId = createTable(tablePath, descriptor, true); + waitAllReplicasReady(tableId, NUM_BUCKETS); + Table table = conn.getTable(tablePath); + + UpsertWriter writer = table.newUpsert().createWriter(); + writer.upsert(row(1, "original")); + writer.flush(); + + writer.upsert(row(1, "updated")); + writer.flush(); + + List result = kvScanAll(table); + + // Exactly one row with the latest value — no duplicates + assertThat(result).hasSize(1); + assertThatRow(result.get(0)) + .withSchema(schema.getRowType()) + .isEqualTo(row(1, "updated")); + } + + /** + * Verifies that each scan opens a new point-in-time RocksDB snapshot: a scan that completes + * before any mutations only sees the original state, and a scan that starts after mutations + * sees the updated state (deletes and inserts both applied). + */ + @Test + void testSnapshotIsolation() throws Exception { + TablePath tablePath = TablePath.of("test_db", "test_snapshot_isolation"); + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("name", DataTypes.STRING()) + .primaryKey("id") + .build(); + TableDescriptor descriptor = + TableDescriptor.builder() + .schema(schema) + .distributedBy(NUM_BUCKETS, "id") + .build(); + + long tableId = createTable(tablePath, descriptor, true); + waitAllReplicasReady(tableId, NUM_BUCKETS); + Table table = conn.getTable(tablePath); + + UpsertWriter writer = table.newUpsert().createWriter(); + writer.upsert(row(1, "a")); + writer.upsert(row(2, "b")); + writer.upsert(row(3, "c")); + writer.flush(); + + // First scan: captures state {1, 2, 3} and fully drains before any mutation. + List beforeMutation = kvScanAll(table); + + // Mutate: add row 4, delete row 1. + writer.upsert(row(4, "d")); + writer.delete(row(1, "a")); + writer.flush(); + + // Second scan: fresh snapshot after mutations must see {2, 3, 4}. + List afterMutation = kvScanAll(table); + + assertThat(beforeMutation).hasSize(3); + beforeMutation.sort(Comparator.comparingInt(r -> r.getInt(0))); + assertThatRow(beforeMutation.get(0)).withSchema(schema.getRowType()).isEqualTo(row(1, "a")); + assertThatRow(beforeMutation.get(1)).withSchema(schema.getRowType()).isEqualTo(row(2, "b")); + assertThatRow(beforeMutation.get(2)).withSchema(schema.getRowType()).isEqualTo(row(3, "c")); + + assertThat(afterMutation).hasSize(3); + afterMutation.sort(Comparator.comparingInt(r -> r.getInt(0))); + assertThatRow(afterMutation.get(0)).withSchema(schema.getRowType()).isEqualTo(row(2, "b")); + assertThatRow(afterMutation.get(1)).withSchema(schema.getRowType()).isEqualTo(row(3, "c")); + assertThatRow(afterMutation.get(2)).withSchema(schema.getRowType()).isEqualTo(row(4, "d")); + } + + // ------------------------------------------------------------------------- + // Iterator lifecycle + // ------------------------------------------------------------------------- + + @Test + void testEarlyClose() throws Exception { + TablePath tablePath = TablePath.of("test_db", "test_early_close"); + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("name", DataTypes.STRING()) + .primaryKey("id") + .build(); + TableDescriptor descriptor = + TableDescriptor.builder() + .schema(schema) + .distributedBy(NUM_BUCKETS, "id") + .build(); + + long tableId = createTable(tablePath, descriptor, true); + waitAllReplicasReady(tableId, NUM_BUCKETS); + Table table = conn.getTable(tablePath); + + int rowCount = 1000; + UpsertWriter writer = table.newUpsert().createWriter(); + for (int i = 0; i < rowCount; i++) { + writer.upsert(row(i, "val" + i)); + } + writer.flush(); + + // Close the iterator after reading only the first 5 rows — must not throw + int readCount = 0; + try (CloseableIterator iterator = table.newKvScan().execute()) { + while (iterator.hasNext() && readCount < 5) { + iterator.next(); + readCount++; + } + } + assertThat(readCount).isEqualTo(5); + } + + @Test + void testIteratorCloseIdempotent() throws Exception { + TablePath tablePath = TablePath.of("test_db", "test_close_idempotent"); + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("name", DataTypes.STRING()) + .primaryKey("id") + .build(); + TableDescriptor descriptor = + TableDescriptor.builder() + .schema(schema) + .distributedBy(NUM_BUCKETS, "id") + .build(); + + long tableId = createTable(tablePath, descriptor, true); + waitAllReplicasReady(tableId, NUM_BUCKETS); + Table table = conn.getTable(tablePath); + + UpsertWriter writer = table.newUpsert().createWriter(); + writer.upsert(row(1, "a")); + writer.flush(); + + CloseableIterator iterator = table.newKvScan().execute(); + // Drain fully then close twice — second close must be a no-op + while (iterator.hasNext()) { + iterator.next(); + } + iterator.close(); + iterator.close(); + } + + // ------------------------------------------------------------------------- + // Error / guard tests + // ------------------------------------------------------------------------- + + @Test + void testNonPrimaryKeyTableThrows() throws Exception { + TablePath tablePath = TablePath.of("test_db", "test_log_table"); + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("name", DataTypes.STRING()) + .build(); // no primaryKey → log table + TableDescriptor descriptor = + TableDescriptor.builder().schema(schema).distributedBy(NUM_BUCKETS).build(); + + createTable(tablePath, descriptor, false); + Table table = conn.getTable(tablePath); + + assertThatThrownBy(() -> table.newKvScan()) + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining("not a Primary Key Table"); + } + + // ------------------------------------------------------------------------- + // Concurrency + // ------------------------------------------------------------------------- + + @Test + void testConcurrentScans() throws Exception { + TablePath tablePath = TablePath.of("test_db", "test_concurrent_scans"); + Schema schema = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("name", DataTypes.STRING()) + .primaryKey("id") + .build(); + TableDescriptor descriptor = + TableDescriptor.builder() + .schema(schema) + .distributedBy(NUM_BUCKETS, "id") + .build(); + + long tableId = createTable(tablePath, descriptor, true); + waitAllReplicasReady(tableId, NUM_BUCKETS); + Table table = conn.getTable(tablePath); + + int rowCount = 100; + UpsertWriter writer = table.newUpsert().createWriter(); + for (int i = 0; i < rowCount; i++) { + writer.upsert(row(i, "val" + i)); + } + writer.flush(); + + int concurrency = 4; + ExecutorService executor = Executors.newFixedThreadPool(concurrency); + AtomicInteger totalRows = new AtomicInteger(0); + List> futures = new ArrayList<>(); + + for (int t = 0; t < concurrency; t++) { + futures.add( + executor.submit( + () -> { + try { + List rows = kvScanAll(table); + totalRows.addAndGet(rows.size()); + } catch (Exception e) { + throw new RuntimeException(e); + } + })); + } + + for (Future f : futures) { + f.get(); + } + executor.shutdown(); + + // Each concurrent scan must see all rows + assertThat(totalRows.get()).isEqualTo(rowCount * concurrency); + } + + // ------------------------------------------------------------------------- + // Helper + // ------------------------------------------------------------------------- + + private List kvScanAll(Table table) throws Exception { + List allRows = new ArrayList<>(); + try (CloseableIterator iterator = table.newKvScan().execute()) { + while (iterator.hasNext()) { + allRows.add(iterator.next()); + } + } + return allRows; + } +} diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotQueryITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotQueryITCase.java deleted file mode 100644 index 56d7c72503..0000000000 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotQueryITCase.java +++ /dev/null @@ -1,310 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You 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 org.apache.fluss.client.table.scanner.batch; - -import org.apache.fluss.client.admin.ClientToServerITCaseBase; -import org.apache.fluss.client.table.Table; -import org.apache.fluss.client.table.writer.UpsertWriter; -import org.apache.fluss.config.ConfigOptions; -import org.apache.fluss.metadata.PartitionSpec; -import org.apache.fluss.metadata.Schema; -import org.apache.fluss.metadata.TableDescriptor; -import org.apache.fluss.metadata.TablePath; -import org.apache.fluss.row.InternalRow; -import org.apache.fluss.types.DataTypes; -import org.apache.fluss.utils.CloseableIterator; - -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.List; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.atomic.AtomicInteger; - -import static org.apache.fluss.testutils.DataTestUtils.row; -import static org.apache.fluss.testutils.InternalRowAssert.assertThatRow; -import static org.assertj.core.api.Assertions.assertThat; - -/** ITCase for KV scan (full RocksDB scan via {@code table.newKvScan().execute()}). */ -public class KvSnapshotQueryITCase extends ClientToServerITCaseBase { - @BeforeEach - protected void setup() throws Exception { - super.setup(); - } - - @AfterEach - protected void teardown() throws Exception { - super.teardown(); - } - - @Test - void testBasicScan() throws Exception { - TablePath tablePath = TablePath.of("test_db", "test_basic_scan"); - Schema schema = - Schema.newBuilder() - .column("id", DataTypes.INT()) - .column("name", DataTypes.STRING()) - .primaryKey("id") - .build(); - TableDescriptor descriptor = - TableDescriptor.builder().schema(schema).distributedBy(1, "id").build(); - - createTable(tablePath, descriptor, true); - - Table table = conn.getTable(tablePath); - - UpsertWriter writer = table.newUpsert().createWriter(); - writer.upsert(row(1, "a")); - writer.upsert(row(2, "b")); - writer.upsert(row(3, "c")); - writer.flush(); - - List result = kvScanAll(table); - - assertThat(result).hasSize(3); - result.sort(Comparator.comparingInt(r -> r.getInt(0))); - assertThatRow(result.get(0)).withSchema(schema.getRowType()).isEqualTo(row(1, "a")); - assertThatRow(result.get(1)).withSchema(schema.getRowType()).isEqualTo(row(2, "b")); - assertThatRow(result.get(2)).withSchema(schema.getRowType()).isEqualTo(row(3, "c")); - } - - @Test - void testEmptyTable() throws Exception { - TablePath tablePath = TablePath.of("test_db", "test_empty_table"); - Schema schema = - Schema.newBuilder() - .column("id", DataTypes.INT()) - .column("name", DataTypes.STRING()) - .primaryKey("id") - .build(); - TableDescriptor descriptor = - TableDescriptor.builder().schema(schema).distributedBy(1, "id").build(); - - createTable(tablePath, descriptor, true); - Table table = conn.getTable(tablePath); - - // No writes — scan should return empty iterator immediately - List result = kvScanAll(table); - assertThat(result).isEmpty(); - } - - @Test - void testMultiBucketScan() throws Exception { - TablePath tablePath = TablePath.of("test_db", "test_multi_bucket_scan"); - Schema schema = - Schema.newBuilder() - .column("id", DataTypes.INT()) - .column("name", DataTypes.STRING()) - .primaryKey("id") - .build(); - TableDescriptor descriptor = - TableDescriptor.builder().schema(schema).distributedBy(3, "id").build(); - - createTable(tablePath, descriptor, true); - Table table = conn.getTable(tablePath); - - int rowCount = 100; - UpsertWriter writer = table.newUpsert().createWriter(); - for (int i = 0; i < rowCount; i++) { - writer.upsert(row(i, "val" + i)); - } - writer.flush(); - - List allResult = kvScanAll(table); - - assertThat(allResult).hasSize(rowCount); - allResult.sort(Comparator.comparingInt(r -> r.getInt(0))); - for (int i = 0; i < rowCount; i++) { - assertThatRow(allResult.get(i)) - .withSchema(schema.getRowType()) - .isEqualTo(row(i, "val" + i)); - } - } - - @Test - void testPartitionedTableScan() throws Exception { - TablePath tablePath = TablePath.of("test_db", "test_partitioned_scan"); - Schema schema = - Schema.newBuilder() - .column("id", DataTypes.INT()) - .column("p", DataTypes.STRING()) - .column("name", DataTypes.STRING()) - .primaryKey("id", "p") - .build(); - TableDescriptor descriptor = - TableDescriptor.builder() - .schema(schema) - .partitionedBy("p") - .distributedBy(1, "id") - .build(); - - createTable(tablePath, descriptor, true); - admin.createPartition( - tablePath, - new PartitionSpec(Collections.singletonMap("p", "p1")), - false) - .get(); - admin.createPartition( - tablePath, - new PartitionSpec(Collections.singletonMap("p", "p2")), - false) - .get(); - - Table table = conn.getTable(tablePath); - - UpsertWriter writer = table.newUpsert().createWriter(); - writer.upsert(row(1, "p1", "a1")); - writer.upsert(row(2, "p1", "b1")); - writer.upsert(row(1, "p2", "a2")); - writer.flush(); - - List result = kvScanAll(table); - assertThat(result).hasSize(3); - } - - @Test - void testLargeDataScan() throws Exception { - TablePath tablePath = TablePath.of("test_db", "test_large_data_scan"); - Schema schema = - Schema.newBuilder() - .column("id", DataTypes.INT()) - .column("name", DataTypes.STRING()) - .primaryKey("id") - .build(); - TableDescriptor descriptor = - TableDescriptor.builder().schema(schema).distributedBy(1, "id").build(); - - createTable(tablePath, descriptor, true); - Table table = conn.getTable(tablePath); - - int rowCount = 10000; - UpsertWriter writer = table.newUpsert().createWriter(); - for (int i = 0; i < rowCount; i++) { - writer.upsert(row(i, "val" + i)); - } - writer.flush(); - - List result = kvScanAll(table); - - assertThat(result).hasSize(rowCount); - result.sort(Comparator.comparingInt(r -> r.getInt(0))); - for (int i = 0; i < rowCount; i++) { - assertThatRow(result.get(i)) - .withSchema(schema.getRowType()) - .isEqualTo(row(i, "val" + i)); - } - } - - @Test - void testEarlyClose() throws Exception { - TablePath tablePath = TablePath.of("test_db", "test_early_close"); - Schema schema = - Schema.newBuilder() - .column("id", DataTypes.INT()) - .column("name", DataTypes.STRING()) - .primaryKey("id") - .build(); - TableDescriptor descriptor = - TableDescriptor.builder().schema(schema).distributedBy(1, "id").build(); - - createTable(tablePath, descriptor, true); - Table table = conn.getTable(tablePath); - - int rowCount = 1000; - UpsertWriter writer = table.newUpsert().createWriter(); - for (int i = 0; i < rowCount; i++) { - writer.upsert(row(i, "val" + i)); - } - writer.flush(); - - // Close the iterator after reading only the first few rows — should not throw - int readCount = 0; - try (CloseableIterator iterator = table.newKvScan().execute()) { - while (iterator.hasNext() && readCount < 5) { - iterator.next(); - readCount++; - } - } - assertThat(readCount).isEqualTo(5); - } - - @Test - void testConcurrentScans() throws Exception { - TablePath tablePath = TablePath.of("test_db", "test_concurrent_scans"); - Schema schema = - Schema.newBuilder() - .column("id", DataTypes.INT()) - .column("name", DataTypes.STRING()) - .primaryKey("id") - .build(); - TableDescriptor descriptor = - TableDescriptor.builder().schema(schema).distributedBy(1, "id").build(); - - createTable(tablePath, descriptor, true); - Table table = conn.getTable(tablePath); - - int rowCount = 100; - UpsertWriter writer = table.newUpsert().createWriter(); - for (int i = 0; i < rowCount; i++) { - writer.upsert(row(i, "val" + i)); - } - writer.flush(); - - int concurrency = 4; - ExecutorService executor = Executors.newFixedThreadPool(concurrency); - AtomicInteger totalRows = new AtomicInteger(0); - List> futures = new ArrayList<>(); - - for (int t = 0; t < concurrency; t++) { - futures.add( - executor.submit( - () -> { - try { - List rows = kvScanAll(table); - totalRows.addAndGet(rows.size()); - } catch (Exception e) { - throw new RuntimeException(e); - } - })); - } - - for (Future f : futures) { - f.get(); - } - executor.shutdown(); - - // Each of the concurrent scans should have seen all rows - assertThat(totalRows.get()).isEqualTo(rowCount * concurrency); - } - - private List kvScanAll(Table table) throws Exception { - List allRows = new ArrayList<>(); - try (CloseableIterator iterator = table.newKvScan().execute()) { - while (iterator.hasNext()) { - allRows.add(iterator.next()); - } - } - return allRows; - } -} From 093c8f9e433c8651ba04f70de0b3cb03c75422b9 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Fri, 6 Mar 2026 12:08:18 +0200 Subject: [PATCH 11/14] fix small issues --- .../fluss/client/table/scanner/KvScan.java | 2 + .../client/table/scanner/TableKvScan.java | 24 ++-- .../org/apache/fluss/rpc/protocol/Errors.java | 2 + .../fluss/server/kv/scan/ScannerContext.java | 5 +- .../fluss/server/kv/scan/ScannerManager.java | 103 +++++++++++++----- .../fluss/server/tablet/TabletService.java | 50 ++++----- 6 files changed, 125 insertions(+), 61 deletions(-) diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/KvScan.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/KvScan.java index 4ca39dea49..b018467ff0 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/KvScan.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/KvScan.java @@ -49,6 +49,8 @@ public interface KvScan { * enumerated automatically; the caller only needs to iterate the returned rows. * * @return a closeable iterator of the rows in the table; must be closed after use + * @throws org.apache.fluss.exception.FlussRuntimeException if partition enumeration fails or + * an error occurs while reading rows from the server */ CloseableIterator execute(); } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableKvScan.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableKvScan.java index fe5e2ff144..bc75aed0ea 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableKvScan.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableKvScan.java @@ -37,6 +37,9 @@ /** Implementation of {@link KvScan}. */ public class TableKvScan implements KvScan { + /** Timeout used when polling each batch from the server-side scanner. */ + private static final Duration BATCH_POLL_TIMEOUT = Duration.ofSeconds(30); + private final FlussConnection conn; private final TableInfo tableInfo; private final SchemaGetter schemaGetter; @@ -71,22 +74,24 @@ public CloseableIterator execute() { "Failed to list partitions for table " + tableInfo.getTablePath(), e); } - return new MultiBucketIterator(buckets); + Scan scan = new TableScan(conn, tableInfo, schemaGetter); + return new MultiBucketIterator(buckets, scan); } - private CloseableIterator scanBucket(TableBucket tableBucket) { - Scan scan = new TableScan(conn, tableInfo, schemaGetter); + private static CloseableIterator scanBucket(Scan scan, TableBucket tableBucket) { BatchScanner batchScanner = scan.createBatchScanner(tableBucket); return new BatchScannerIterator(batchScanner); } - private class MultiBucketIterator implements CloseableIterator { + private static class MultiBucketIterator implements CloseableIterator { private final Iterator bucketIterator; + private final Scan scan; private CloseableIterator currentScannerIterator; private boolean isClosed = false; - private MultiBucketIterator(List buckets) { + private MultiBucketIterator(List buckets, Scan scan) { this.bucketIterator = buckets.iterator(); + this.scan = scan; } @Override @@ -100,7 +105,7 @@ public boolean hasNext() { currentScannerIterator = null; } if (bucketIterator.hasNext()) { - currentScannerIterator = scanBucket(bucketIterator.next()); + currentScannerIterator = scanBucket(scan, bucketIterator.next()); } else { return false; } @@ -153,8 +158,7 @@ public InternalRow next() { private void ensureBatch() { try { while ((currentBatch == null || !currentBatch.hasNext()) && !isClosed) { - CloseableIterator it = - scanner.pollBatch(Duration.ofMinutes(1)); + CloseableIterator it = scanner.pollBatch(BATCH_POLL_TIMEOUT); if (it == null) { isClosed = true; break; @@ -166,7 +170,7 @@ private void ensureBatch() { } } } catch (IOException e) { - throw new RuntimeException("Error polling batch from scanner", e); + throw new FlussRuntimeException("Error polling batch from scanner", e); } } @@ -176,7 +180,7 @@ public void close() { try { scanner.close(); } catch (IOException e) { - throw new RuntimeException("Error closing scanner", e); + throw new FlussRuntimeException("Error closing scanner", e); } isClosed = true; } diff --git a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java index 3950d300c1..b59d1895a2 100644 --- a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java +++ b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java @@ -246,6 +246,8 @@ public enum Errors { REBALANCE_FAILURE_EXCEPTION(61, "The rebalance task failure.", RebalanceFailureException::new), NO_REBALANCE_IN_PROGRESS_EXCEPTION( 62, "No rebalance task in progress.", NoRebalanceInProgressException::new), + /** @deprecated Superseded by {@link #UNKNOWN_SCANNER_ID} (65) and {@link #SCANNER_EXPIRED} (64). */ + @Deprecated SCANNER_NOT_FOUND_EXCEPTION(63, "The scanner is not found.", ScannerNotFoundException::new), SCANNER_EXPIRED( 64, diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerContext.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerContext.java index 021c6b4a30..2f0231e6c2 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerContext.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerContext.java @@ -35,7 +35,10 @@ * point-in-time scan isolation. The {@link ResourceGuard.Lease} prevents RocksDB from being closed * while this session is active. * - *

This class is not thread-safe; callers must synchronize externally. + *

Most fields of this class are not thread-safe; in particular, {@link #iterator} and + * {@link #callSeqId} must only be accessed by a single thread at a time. The {@link #expired} flag + * is the sole exception: it is an {@link java.util.concurrent.atomic.AtomicBoolean} and may be + * written by the TTL reaper thread while another thread holds a reference to the context. */ @NotThreadSafe public class ScannerContext implements AutoCloseable { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java index 5d6af6a43a..32159837c4 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java @@ -37,6 +37,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.io.IOException; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; @@ -57,10 +59,15 @@ public class ScannerManager implements AutoCloseableAsync { private static final Logger LOG = LoggerFactory.getLogger(ScannerManager.class); private final Map scanners = MapUtils.newConcurrentHashMap(); + /** + * Tracks recently-expired scanner IDs and their expiry timestamp. Allows distinguishing between + * a scanner that expired (SCANNER_EXPIRED) and one that was never created (UNKNOWN_SCANNER_ID). + * Entries are pruned by the cleanup task after 2 × TTL to bound memory usage. + */ + private final Map recentlyExpiredIds = MapUtils.newConcurrentHashMap(); private final Scheduler scheduler; private final Clock clock; private final long scannerTtlMs; - private final long expirationIntervalMs; private final int maxPerBucket; private final int maxPerServer; private final AtomicInteger totalScanners = new AtomicInteger(0); @@ -74,7 +81,7 @@ public ScannerManager(Configuration conf, Scheduler scheduler, Clock clock) { this.scheduler = scheduler; this.clock = clock; this.scannerTtlMs = conf.get(ConfigOptions.SERVER_SCANNER_TTL).toMillis(); - this.expirationIntervalMs = + long expirationIntervalMs = conf.get(ConfigOptions.SERVER_SCANNER_EXPIRATION_INTERVAL).toMillis(); this.maxPerBucket = conf.get(ConfigOptions.SERVER_SCANNER_MAX_PER_BUCKET); this.maxPerServer = conf.get(ConfigOptions.SERVER_SCANNER_MAX_PER_SERVER); @@ -87,12 +94,22 @@ public ScannerManager(Configuration conf, Scheduler scheduler, Clock clock) { } /** - * Creates a new scanner session for the given KV tablet and returns the {@link ScannerContext}. - * The context contains the scanner id assigned to this session. + * Creates a new scanner session for the given KV tablet and returns the {@link ScannerContext}, + * or {@code null} if the bucket is empty (no rows to scan). + * + *

When {@code null} is returned, no scanner session is registered and no limit slot is + * consumed. The caller should return an empty response immediately. + * + *

Note on limit enforcement: The per-bucket and server-level limit checks are not + * atomic with the subsequent {@code scanners.put()}. This is an intentional trade-off: the + * limits are soft guards against runaway scanner creation, not hard resource quotas. In the + * rare race where two threads both pass the limit check before either registers, the server + * may briefly exceed the configured maximum by a small amount. * * @throws TooManyScannersException if per-bucket or per-server limits are exceeded * @throws IOException if the RocksDB resource guard cannot be acquired */ + @Nullable public ScannerContext createScanner(KvTablet kvTablet, TableBucket tableBucket) throws IOException { // Check server-level limit first (cheap atomic check) @@ -115,33 +132,50 @@ public ScannerContext createScanner(KvTablet kvTablet, TableBucket tableBucket) } RocksDBKv rocksDBKv = kvTablet.getRocksDBKv(); - // Acquire a lease to prevent RocksDB from being closed while iterating + // Acquire a lease to prevent RocksDB from being closed while iterating. + // All subsequent resource allocations are guarded by a try/catch so the lease + // is always released on failure. ResourceGuard.Lease lease = rocksDBKv.getResourceGuard().acquireResource(); - Snapshot snapshot = rocksDBKv.getSnapshot(); - ReadOptions readOptions = new ReadOptions().setSnapshot(snapshot); - RocksIterator iterator = rocksDBKv.newIterator(readOptions); - iterator.seekToFirst(); - - byte[] scannerId = generateScannerId(); - ScannerContext context = - new ScannerContext( - scannerId, - tableBucket, - rocksDBKv, - iterator, - readOptions, - snapshot, - lease, - clock); - scanners.put(ByteBuffer.wrap(scannerId), context); - totalScanners.incrementAndGet(); - return context; + try { + Snapshot snapshot = rocksDBKv.getSnapshot(); + ReadOptions readOptions = new ReadOptions().setSnapshot(snapshot); + RocksIterator iterator = rocksDBKv.newIterator(readOptions); + iterator.seekToFirst(); + + if (!iterator.isValid()) { + // Empty bucket: release all resources without registering a scanner session. + iterator.close(); + readOptions.close(); + rocksDBKv.releaseSnapshot(snapshot); + lease.close(); + return null; + } + + byte[] scannerId = generateScannerId(); + ScannerContext context = + new ScannerContext( + scannerId, + tableBucket, + rocksDBKv, + iterator, + readOptions, + snapshot, + lease, + clock); + scanners.put(ByteBuffer.wrap(scannerId), context); + totalScanners.incrementAndGet(); + return context; + } catch (Exception e) { + lease.close(); + throw e; + } } /** * Looks up a scanner by id and refreshes its last-access timestamp. Returns {@code null} if * the scanner is not registered (was never created or has already been removed). */ + @Nullable public ScannerContext getScanner(byte[] scannerId) { ScannerContext context = scanners.get(ByteBuffer.wrap(scannerId)); if (context != null) { @@ -150,6 +184,15 @@ public ScannerContext getScanner(byte[] scannerId) { return context; } + /** + * Returns {@code true} if the scanner with the given id was recently expired by the TTL reaper. + * This allows callers to distinguish a SCANNER_EXPIRED error from an UNKNOWN_SCANNER_ID error + * when {@link #getScanner(byte[])} returns {@code null}. + */ + public boolean isRecentlyExpired(byte[] scannerId) { + return recentlyExpiredIds.containsKey(ByteBuffer.wrap(scannerId)); + } + /** * Removes and closes the scanner identified by {@code scannerId}. No-op if the id is unknown. */ @@ -197,14 +240,20 @@ private int countScannersForBucket(TableBucket tableBucket) { private void cleanupExpiredScanners() { long now = clock.milliseconds(); + + // Prune stale entries from the recently-expired set (keep for 2 × TTL for diagnostics) + long expiredEntryRetentionMs = 2 * scannerTtlMs; + recentlyExpiredIds.entrySet().removeIf(e -> now - e.getValue() > expiredEntryRetentionMs); + for (Map.Entry entry : scanners.entrySet()) { ScannerContext context = entry.getValue(); if (now - context.getLastAccessTime() > scannerTtlMs) { - // Mark expired before removal so TabletService can distinguish error codes context.markExpired(); // Atomic conditional remove to avoid double-close race with removeScanner() if (scanners.remove(entry.getKey(), context)) { totalScanners.decrementAndGet(); + // Record the expiry so subsequent lookups return SCANNER_EXPIRED + recentlyExpiredIds.put(entry.getKey(), now); LOG.info( "Scanner {} expired after {}ms idle, closing it.", scannerIdToString(entry.getKey().array()), @@ -246,10 +295,14 @@ public void close() throws Exception { if (cleanupTask != null) { cleanupTask.cancel(true); } + // Note: if the cleanup task is currently running concurrently, a scanner may be closed + // both by the reaper and by this loop. ScannerContext.close() is idempotent for the + // RocksDB iterator and snapshot, so this is safe, though a warning may be logged. for (ScannerContext context : scanners.values()) { closeScannerContext(context); } scanners.clear(); + recentlyExpiredIds.clear(); totalScanners.set(0); } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java index 4ece7f7707..43f7351821 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java @@ -138,6 +138,7 @@ public final class TabletService extends RpcServiceBase implements TabletServerGateway { private final String serviceName; + private final int serverId; private final ReplicaManager replicaManager; private final TabletServerMetadataCache metadataCache; private final TabletServerMetadataProvider metadataFunctionProvider; @@ -163,6 +164,7 @@ public TabletService( dynamicConfigManager, ioExecutor); this.serviceName = "server-" + serverId; + this.serverId = serverId; this.replicaManager = replicaManager; this.metadataCache = metadataCache; this.scannerManager = scannerManager; @@ -318,14 +320,11 @@ public CompletableFuture scanKv(ScanKvRequest request) { byte[] scannerId = request.getScannerId(); ScannerContext context = scannerManager.getScanner(scannerId); if (context == null) { - // Distinguish between a session that was never created and one that expired String idStr = new String(scannerId, StandardCharsets.UTF_8); - // The context may have been removed by the reaper and marked expired before - // removal, but since it is gone from the map we must use a placeholder check. - // We create a temporary context only to read the expired flag; instead we rely - // on a lightweight expiry-tracking mechanism in ScannerManager via a separate - // set. For now, treat a missing context conservatively as UNKNOWN_SCANNER_ID; - // clients should retry with a new session. + if (scannerManager.isRecentlyExpired(scannerId)) { + throw Errors.SCANNER_EXPIRED.exception( + "Scanner session expired: " + idStr); + } throw Errors.UNKNOWN_SCANNER_ID.exception( "Unknown scanner id: " + idStr); } @@ -349,7 +348,7 @@ public CompletableFuture scanKv(ScanKvRequest request) { } context.setCallSeqId(request.getCallSeqId()); - response.complete(continueScan(scannerId, context, request.getBatchSizeBytes())); + response.complete(continueScan(context, request.getBatchSizeBytes())); } else { if (!hasBucketScanReq) { throw Errors.INVALID_SCAN_REQUEST.exception( @@ -374,25 +373,23 @@ public CompletableFuture scanKv(ScanKvRequest request) { "Table " + bucketScanReq.getTableId() + " is not a primary key table."); } + // Capture log offset before opening the snapshot so clients get a lower bound + // on the WAL position consistent with the data they are about to read. + long logHighWatermark = replica.getLogHighWatermark(); ScannerContext context = scannerManager.createScanner(replica.getKvTablet(), tb); - // Empty-bucket fast-path: no rows, return immediately without registering a session - if (!context.getIterator().isValid()) { - scannerManager.removeScanner(context.getScannerId()); + if (context == null) { + // Empty bucket: no rows, no scanner registered ScanKvResponse scanResponse = new ScanKvResponse(); - scanResponse - .setScannerId(context.getScannerId()) - .setHasMoreResults(false) - .setLogOffset(replica.getLogHighWatermark()); + scanResponse.setHasMoreResults(false).setLogOffset(logHighWatermark); response.complete(scanResponse); return response; } - ScanKvResponse scanResponse = - continueScan(context.getScannerId(), context, request.getBatchSizeBytes()); - // Return the log offset at scanner creation time for clients to determine - // where to resume reading the log after the snapshot scan. - scanResponse.setLogOffset(replica.getLogHighWatermark()); + ScanKvResponse scanResponse = continueScan(context, request.getBatchSizeBytes()); + // Return the log offset for clients to determine where to resume reading the log + // after the snapshot scan completes. + scanResponse.setLogOffset(logHighWatermark); response.complete(scanResponse); } } catch (Exception e) { @@ -401,8 +398,9 @@ public CompletableFuture scanKv(ScanKvRequest request) { return response; } - private ScanKvResponse continueScan( - byte[] scannerId, ScannerContext context, int batchSizeBytes) throws IOException { + private ScanKvResponse continueScan(ScannerContext context, int batchSizeBytes) + throws IOException { + byte[] scannerId = context.getScannerId(); RocksIterator iterator = context.getIterator(); DefaultValueRecordBatch.Builder builder = DefaultValueRecordBatch.builder(); int currentBytes = 0; @@ -452,10 +450,12 @@ public CompletableFuture notifyLeaderAndIsr( CompletableFuture response = new CompletableFuture<>(); List notifyLeaderAndIsrRequestData = getNotifyLeaderAndIsrRequestData(notifyLeaderAndIsrRequest); - // Close any active scanner sessions for all affected buckets before role transitions. - // This prevents stale snapshot/iterator leaks when a bucket becomes a follower. + // Close scanner sessions only for buckets where this server is becoming a follower. + // Buckets where this server remains or becomes the leader keep their snapshots valid. for (NotifyLeaderAndIsrData data : notifyLeaderAndIsrRequestData) { - scannerManager.closeScannersForBucket(data.getTableBucket()); + if (data.getLeader() != serverId) { + scannerManager.closeScannersForBucket(data.getTableBucket()); + } } replicaManager.becomeLeaderOrFollower( notifyLeaderAndIsrRequest.getCoordinatorEpoch(), From 0f8380b0dd36b228f6394735512794e6fa343a7c Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Fri, 6 Mar 2026 12:23:42 +0200 Subject: [PATCH 12/14] refactor --- .../client/table/scanner/TableKvScan.java | 2 +- .../table/scanner/batch/KvScanITCase.java | 417 ++++++------------ .../org/apache/fluss/rpc/protocol/Errors.java | 4 +- .../fluss/server/kv/scan/ScannerContext.java | 21 +- .../fluss/server/kv/scan/ScannerManager.java | 26 +- .../fluss/server/tablet/TabletService.java | 8 +- 6 files changed, 163 insertions(+), 315 deletions(-) diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableKvScan.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableKvScan.java index bc75aed0ea..b45cbf7f17 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableKvScan.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableKvScan.java @@ -124,10 +124,10 @@ public InternalRow next() { @Override public void close() { if (!isClosed) { + isClosed = true; if (currentScannerIterator != null) { currentScannerIterator.close(); } - isClosed = true; } } } diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvScanITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvScanITCase.java index 9db5400045..83103c78a4 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvScanITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvScanITCase.java @@ -20,16 +20,17 @@ import org.apache.fluss.client.admin.ClientToServerITCaseBase; import org.apache.fluss.client.table.Table; import org.apache.fluss.client.table.writer.UpsertWriter; +import org.apache.fluss.exception.FlussRuntimeException; +import org.apache.fluss.metadata.PartitionInfo; import org.apache.fluss.metadata.PartitionSpec; import org.apache.fluss.metadata.Schema; +import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableDescriptor; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.row.InternalRow; import org.apache.fluss.types.DataTypes; import org.apache.fluss.utils.CloseableIterator; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import java.util.ArrayList; @@ -49,25 +50,32 @@ /** * Integration tests for full KV scan via {@code table.newKvScan().execute()}. * - *

All non-partitioned tables use 3 buckets to exercise multi-bucket fan-out on the 3-node - * cluster provided by {@link ClientToServerITCaseBase}. Each test calls {@link - * #waitAllReplicasReady} after table creation to ensure leader election completes before scanning, - * which is especially important for empty-table tests that have no upsert traffic to act as a - * natural synchronization barrier. + *

All non-partitioned tables use {@link #NUM_BUCKETS} buckets to exercise multi-bucket fan-out + * on the 3-node cluster provided by {@link ClientToServerITCaseBase}. Each test calls {@link + * #waitAllReplicasReady} (or {@link #waitPartitionedTableReplicasReady}) after table creation to + * ensure leader election completes before scanning — especially important for empty tables or + * partitions that have no upsert traffic as a natural synchronization barrier. */ public class KvScanITCase extends ClientToServerITCaseBase { private static final int NUM_BUCKETS = 3; - @BeforeEach - protected void setup() throws Exception { - super.setup(); - } - - @AfterEach - protected void teardown() throws Exception { - super.teardown(); - } + /** Schema shared by all non-partitioned PK tests: (id INT, name STRING), PK = id. */ + private static final Schema PK_SCHEMA = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("name", DataTypes.STRING()) + .primaryKey("id") + .build(); + + /** Schema shared by partitioned PK tests: (id INT, p STRING, name STRING), PK = (id, p). */ + private static final Schema PARTITIONED_SCHEMA = + Schema.newBuilder() + .column("id", DataTypes.INT()) + .column("p", DataTypes.STRING()) + .column("name", DataTypes.STRING()) + .primaryKey("id", "p") + .build(); // ------------------------------------------------------------------------- // Basic / structural tests @@ -75,22 +83,7 @@ protected void teardown() throws Exception { @Test void testBasicScan() throws Exception { - TablePath tablePath = TablePath.of("test_db", "test_basic_scan"); - Schema schema = - Schema.newBuilder() - .column("id", DataTypes.INT()) - .column("name", DataTypes.STRING()) - .primaryKey("id") - .build(); - TableDescriptor descriptor = - TableDescriptor.builder() - .schema(schema) - .distributedBy(NUM_BUCKETS, "id") - .build(); - - long tableId = createTable(tablePath, descriptor, true); - waitAllReplicasReady(tableId, NUM_BUCKETS); - Table table = conn.getTable(tablePath); + Table table = createPkTable(TablePath.of("test_db", "test_basic_scan")); UpsertWriter writer = table.newUpsert().createWriter(); writer.upsert(row(1, "a")); @@ -102,92 +95,23 @@ void testBasicScan() throws Exception { assertThat(result).hasSize(3); result.sort(Comparator.comparingInt(r -> r.getInt(0))); - assertThatRow(result.get(0)).withSchema(schema.getRowType()).isEqualTo(row(1, "a")); - assertThatRow(result.get(1)).withSchema(schema.getRowType()).isEqualTo(row(2, "b")); - assertThatRow(result.get(2)).withSchema(schema.getRowType()).isEqualTo(row(3, "c")); + assertThatRow(result.get(0)).withSchema(PK_SCHEMA.getRowType()).isEqualTo(row(1, "a")); + assertThatRow(result.get(1)).withSchema(PK_SCHEMA.getRowType()).isEqualTo(row(2, "b")); + assertThatRow(result.get(2)).withSchema(PK_SCHEMA.getRowType()).isEqualTo(row(3, "c")); } @Test void testEmptyTable() throws Exception { - TablePath tablePath = TablePath.of("test_db", "test_empty_table"); - Schema schema = - Schema.newBuilder() - .column("id", DataTypes.INT()) - .column("name", DataTypes.STRING()) - .primaryKey("id") - .build(); - TableDescriptor descriptor = - TableDescriptor.builder() - .schema(schema) - .distributedBy(NUM_BUCKETS, "id") - .build(); - - long tableId = createTable(tablePath, descriptor, true); - // No upsert traffic — must wait for leaders to be elected before scanning. - waitAllReplicasReady(tableId, NUM_BUCKETS); - Table table = conn.getTable(tablePath); - - List result = kvScanAll(table); - assertThat(result).isEmpty(); - } - - @Test - void testMultiBucketScan() throws Exception { - TablePath tablePath = TablePath.of("test_db", "test_multi_bucket_scan"); - Schema schema = - Schema.newBuilder() - .column("id", DataTypes.INT()) - .column("name", DataTypes.STRING()) - .primaryKey("id") - .build(); - TableDescriptor descriptor = - TableDescriptor.builder() - .schema(schema) - .distributedBy(NUM_BUCKETS, "id") - .build(); - - long tableId = createTable(tablePath, descriptor, true); - waitAllReplicasReady(tableId, NUM_BUCKETS); - Table table = conn.getTable(tablePath); - - int rowCount = 100; - UpsertWriter writer = table.newUpsert().createWriter(); - for (int i = 0; i < rowCount; i++) { - writer.upsert(row(i, "val" + i)); - } - writer.flush(); - - List allResult = kvScanAll(table); - - assertThat(allResult).hasSize(rowCount); - allResult.sort(Comparator.comparingInt(r -> r.getInt(0))); - for (int i = 0; i < rowCount; i++) { - assertThatRow(allResult.get(i)) - .withSchema(schema.getRowType()) - .isEqualTo(row(i, "val" + i)); - } + Table table = createPkTable(TablePath.of("test_db", "test_empty_table")); + // No upsert traffic — waitAllReplicasReady inside createPkTable ensures leaders are ready. + assertThat(kvScanAll(table)).isEmpty(); } @Test void testLargeDataScan() throws Exception { - TablePath tablePath = TablePath.of("test_db", "test_large_data_scan"); - Schema schema = - Schema.newBuilder() - .column("id", DataTypes.INT()) - .column("name", DataTypes.STRING()) - .primaryKey("id") - .build(); - TableDescriptor descriptor = - TableDescriptor.builder() - .schema(schema) - .distributedBy(NUM_BUCKETS, "id") - .build(); - - long tableId = createTable(tablePath, descriptor, true); - waitAllReplicasReady(tableId, NUM_BUCKETS); - Table table = conn.getTable(tablePath); + Table table = createPkTable(TablePath.of("test_db", "test_large_data_scan")); - int rowCount = 10000; + int rowCount = 10_000; UpsertWriter writer = table.newUpsert().createWriter(); for (int i = 0; i < rowCount; i++) { writer.upsert(row(i, "val" + i)); @@ -200,7 +124,7 @@ void testLargeDataScan() throws Exception { result.sort(Comparator.comparingInt(r -> r.getInt(0))); for (int i = 0; i < rowCount; i++) { assertThatRow(result.get(i)) - .withSchema(schema.getRowType()) + .withSchema(PK_SCHEMA.getRowType()) .isEqualTo(row(i, "val" + i)); } } @@ -212,34 +136,17 @@ void testLargeDataScan() throws Exception { @Test void testPartitionedTableScan() throws Exception { TablePath tablePath = TablePath.of("test_db", "test_partitioned_scan"); - Schema schema = - Schema.newBuilder() - .column("id", DataTypes.INT()) - .column("p", DataTypes.STRING()) - .column("name", DataTypes.STRING()) - .primaryKey("id", "p") - .build(); - TableDescriptor descriptor = - TableDescriptor.builder() - .schema(schema) - .partitionedBy("p") - .distributedBy(NUM_BUCKETS, "id") - .build(); + long tableId = createPartitionedTable(tablePath); - createTable(tablePath, descriptor, true); admin.createPartition( - tablePath, - new PartitionSpec(Collections.singletonMap("p", "p1")), - false) + tablePath, new PartitionSpec(Collections.singletonMap("p", "p1")), false) .get(); admin.createPartition( - tablePath, - new PartitionSpec(Collections.singletonMap("p", "p2")), - false) + tablePath, new PartitionSpec(Collections.singletonMap("p", "p2")), false) .get(); + waitPartitionedTableReplicasReady(tableId, tablePath); Table table = conn.getTable(tablePath); - UpsertWriter writer = table.newUpsert().createWriter(); writer.upsert(row(1, "p1", "a1")); writer.upsert(row(2, "p1", "b1")); @@ -251,50 +158,40 @@ void testPartitionedTableScan() throws Exception { result.sort( Comparator.comparingInt((InternalRow r) -> r.getInt(0)) .thenComparing(r -> r.getString(1).toString())); - assertThatRow(result.get(0)).withSchema(schema.getRowType()).isEqualTo(row(1, "p1", "a1")); - assertThatRow(result.get(1)).withSchema(schema.getRowType()).isEqualTo(row(1, "p2", "a2")); - assertThatRow(result.get(2)).withSchema(schema.getRowType()).isEqualTo(row(2, "p1", "b1")); + assertThatRow(result.get(0)) + .withSchema(PARTITIONED_SCHEMA.getRowType()) + .isEqualTo(row(1, "p1", "a1")); + assertThatRow(result.get(1)) + .withSchema(PARTITIONED_SCHEMA.getRowType()) + .isEqualTo(row(1, "p2", "a2")); + assertThatRow(result.get(2)) + .withSchema(PARTITIONED_SCHEMA.getRowType()) + .isEqualTo(row(2, "p1", "b1")); } @Test void testPartitionedTableEmptyPartition() throws Exception { TablePath tablePath = TablePath.of("test_db", "test_partitioned_empty"); - Schema schema = - Schema.newBuilder() - .column("id", DataTypes.INT()) - .column("p", DataTypes.STRING()) - .column("name", DataTypes.STRING()) - .primaryKey("id", "p") - .build(); - TableDescriptor descriptor = - TableDescriptor.builder() - .schema(schema) - .partitionedBy("p") - .distributedBy(NUM_BUCKETS, "id") - .build(); + long tableId = createPartitionedTable(tablePath); - createTable(tablePath, descriptor, true); - // p1 will have data, p2 will be empty + // p1 will have data; p2 will be empty admin.createPartition( - tablePath, - new PartitionSpec(Collections.singletonMap("p", "p1")), - false) + tablePath, new PartitionSpec(Collections.singletonMap("p", "p1")), false) .get(); admin.createPartition( - tablePath, - new PartitionSpec(Collections.singletonMap("p", "p2")), - false) + tablePath, new PartitionSpec(Collections.singletonMap("p", "p2")), false) .get(); + // Explicitly wait for p2's bucket replicas — it has no writes to act as a barrier + waitPartitionedTableReplicasReady(tableId, tablePath); Table table = conn.getTable(tablePath); - UpsertWriter writer = table.newUpsert().createWriter(); writer.upsert(row(1, "p1", "a1")); writer.upsert(row(2, "p1", "b1")); writer.flush(); List result = kvScanAll(table); - // Only p1 rows should appear; p2 is empty + // Only p1 rows; p2 is empty and must not contribute any rows assertThat(result).hasSize(2); } @@ -304,22 +201,7 @@ void testPartitionedTableEmptyPartition() throws Exception { @Test void testDeleteVisibility() throws Exception { - TablePath tablePath = TablePath.of("test_db", "test_delete_visibility"); - Schema schema = - Schema.newBuilder() - .column("id", DataTypes.INT()) - .column("name", DataTypes.STRING()) - .primaryKey("id") - .build(); - TableDescriptor descriptor = - TableDescriptor.builder() - .schema(schema) - .distributedBy(NUM_BUCKETS, "id") - .build(); - - long tableId = createTable(tablePath, descriptor, true); - waitAllReplicasReady(tableId, NUM_BUCKETS); - Table table = conn.getTable(tablePath); + Table table = createPkTable(TablePath.of("test_db", "test_delete_visibility")); UpsertWriter writer = table.newUpsert().createWriter(); writer.upsert(row(1, "a")); @@ -327,37 +209,20 @@ void testDeleteVisibility() throws Exception { writer.upsert(row(3, "c")); writer.flush(); - // Delete row with id=2; only its primary key fields are required writer.delete(row(2, "b")); writer.flush(); List result = kvScanAll(table); - // Rows 1 and 3 survive; the deleted row must not appear assertThat(result).hasSize(2); result.sort(Comparator.comparingInt(r -> r.getInt(0))); - assertThatRow(result.get(0)).withSchema(schema.getRowType()).isEqualTo(row(1, "a")); - assertThatRow(result.get(1)).withSchema(schema.getRowType()).isEqualTo(row(3, "c")); + assertThatRow(result.get(0)).withSchema(PK_SCHEMA.getRowType()).isEqualTo(row(1, "a")); + assertThatRow(result.get(1)).withSchema(PK_SCHEMA.getRowType()).isEqualTo(row(3, "c")); } @Test void testUpsertOverwrite() throws Exception { - TablePath tablePath = TablePath.of("test_db", "test_upsert_overwrite"); - Schema schema = - Schema.newBuilder() - .column("id", DataTypes.INT()) - .column("name", DataTypes.STRING()) - .primaryKey("id") - .build(); - TableDescriptor descriptor = - TableDescriptor.builder() - .schema(schema) - .distributedBy(NUM_BUCKETS, "id") - .build(); - - long tableId = createTable(tablePath, descriptor, true); - waitAllReplicasReady(tableId, NUM_BUCKETS); - Table table = conn.getTable(tablePath); + Table table = createPkTable(TablePath.of("test_db", "test_upsert_overwrite")); UpsertWriter writer = table.newUpsert().createWriter(); writer.upsert(row(1, "original")); @@ -368,36 +233,19 @@ void testUpsertOverwrite() throws Exception { List result = kvScanAll(table); - // Exactly one row with the latest value — no duplicates assertThat(result).hasSize(1); assertThatRow(result.get(0)) - .withSchema(schema.getRowType()) + .withSchema(PK_SCHEMA.getRowType()) .isEqualTo(row(1, "updated")); } /** - * Verifies that each scan opens a new point-in-time RocksDB snapshot: a scan that completes - * before any mutations only sees the original state, and a scan that starts after mutations - * sees the updated state (deletes and inserts both applied). + * Verifies point-in-time snapshot isolation: a scan that completes before any mutations only + * sees the original state; a scan that starts after mutations sees the updated state. */ @Test void testSnapshotIsolation() throws Exception { - TablePath tablePath = TablePath.of("test_db", "test_snapshot_isolation"); - Schema schema = - Schema.newBuilder() - .column("id", DataTypes.INT()) - .column("name", DataTypes.STRING()) - .primaryKey("id") - .build(); - TableDescriptor descriptor = - TableDescriptor.builder() - .schema(schema) - .distributedBy(NUM_BUCKETS, "id") - .build(); - - long tableId = createTable(tablePath, descriptor, true); - waitAllReplicasReady(tableId, NUM_BUCKETS); - Table table = conn.getTable(tablePath); + Table table = createPkTable(TablePath.of("test_db", "test_snapshot_isolation")); UpsertWriter writer = table.newUpsert().createWriter(); writer.upsert(row(1, "a")); @@ -418,15 +266,27 @@ void testSnapshotIsolation() throws Exception { assertThat(beforeMutation).hasSize(3); beforeMutation.sort(Comparator.comparingInt(r -> r.getInt(0))); - assertThatRow(beforeMutation.get(0)).withSchema(schema.getRowType()).isEqualTo(row(1, "a")); - assertThatRow(beforeMutation.get(1)).withSchema(schema.getRowType()).isEqualTo(row(2, "b")); - assertThatRow(beforeMutation.get(2)).withSchema(schema.getRowType()).isEqualTo(row(3, "c")); + assertThatRow(beforeMutation.get(0)) + .withSchema(PK_SCHEMA.getRowType()) + .isEqualTo(row(1, "a")); + assertThatRow(beforeMutation.get(1)) + .withSchema(PK_SCHEMA.getRowType()) + .isEqualTo(row(2, "b")); + assertThatRow(beforeMutation.get(2)) + .withSchema(PK_SCHEMA.getRowType()) + .isEqualTo(row(3, "c")); assertThat(afterMutation).hasSize(3); afterMutation.sort(Comparator.comparingInt(r -> r.getInt(0))); - assertThatRow(afterMutation.get(0)).withSchema(schema.getRowType()).isEqualTo(row(2, "b")); - assertThatRow(afterMutation.get(1)).withSchema(schema.getRowType()).isEqualTo(row(3, "c")); - assertThatRow(afterMutation.get(2)).withSchema(schema.getRowType()).isEqualTo(row(4, "d")); + assertThatRow(afterMutation.get(0)) + .withSchema(PK_SCHEMA.getRowType()) + .isEqualTo(row(2, "b")); + assertThatRow(afterMutation.get(1)) + .withSchema(PK_SCHEMA.getRowType()) + .isEqualTo(row(3, "c")); + assertThatRow(afterMutation.get(2)) + .withSchema(PK_SCHEMA.getRowType()) + .isEqualTo(row(4, "d")); } // ------------------------------------------------------------------------- @@ -435,31 +295,14 @@ void testSnapshotIsolation() throws Exception { @Test void testEarlyClose() throws Exception { - TablePath tablePath = TablePath.of("test_db", "test_early_close"); - Schema schema = - Schema.newBuilder() - .column("id", DataTypes.INT()) - .column("name", DataTypes.STRING()) - .primaryKey("id") - .build(); - TableDescriptor descriptor = - TableDescriptor.builder() - .schema(schema) - .distributedBy(NUM_BUCKETS, "id") - .build(); + Table table = createPkTable(TablePath.of("test_db", "test_early_close")); - long tableId = createTable(tablePath, descriptor, true); - waitAllReplicasReady(tableId, NUM_BUCKETS); - Table table = conn.getTable(tablePath); - - int rowCount = 1000; UpsertWriter writer = table.newUpsert().createWriter(); - for (int i = 0; i < rowCount; i++) { + for (int i = 0; i < 1000; i++) { writer.upsert(row(i, "val" + i)); } writer.flush(); - // Close the iterator after reading only the first 5 rows — must not throw int readCount = 0; try (CloseableIterator iterator = table.newKvScan().execute()) { while (iterator.hasNext() && readCount < 5) { @@ -472,32 +315,17 @@ void testEarlyClose() throws Exception { @Test void testIteratorCloseIdempotent() throws Exception { - TablePath tablePath = TablePath.of("test_db", "test_close_idempotent"); - Schema schema = - Schema.newBuilder() - .column("id", DataTypes.INT()) - .column("name", DataTypes.STRING()) - .primaryKey("id") - .build(); - TableDescriptor descriptor = - TableDescriptor.builder() - .schema(schema) - .distributedBy(NUM_BUCKETS, "id") - .build(); - - long tableId = createTable(tablePath, descriptor, true); - waitAllReplicasReady(tableId, NUM_BUCKETS); - Table table = conn.getTable(tablePath); + Table table = createPkTable(TablePath.of("test_db", "test_close_idempotent")); UpsertWriter writer = table.newUpsert().createWriter(); writer.upsert(row(1, "a")); writer.flush(); CloseableIterator iterator = table.newKvScan().execute(); - // Drain fully then close twice — second close must be a no-op while (iterator.hasNext()) { iterator.next(); } + // Second close must be a no-op, not throw iterator.close(); iterator.close(); } @@ -513,7 +341,7 @@ void testNonPrimaryKeyTableThrows() throws Exception { Schema.newBuilder() .column("id", DataTypes.INT()) .column("name", DataTypes.STRING()) - .build(); // no primaryKey → log table + .build(); TableDescriptor descriptor = TableDescriptor.builder().schema(schema).distributedBy(NUM_BUCKETS).build(); @@ -531,22 +359,7 @@ void testNonPrimaryKeyTableThrows() throws Exception { @Test void testConcurrentScans() throws Exception { - TablePath tablePath = TablePath.of("test_db", "test_concurrent_scans"); - Schema schema = - Schema.newBuilder() - .column("id", DataTypes.INT()) - .column("name", DataTypes.STRING()) - .primaryKey("id") - .build(); - TableDescriptor descriptor = - TableDescriptor.builder() - .schema(schema) - .distributedBy(NUM_BUCKETS, "id") - .build(); - - long tableId = createTable(tablePath, descriptor, true); - waitAllReplicasReady(tableId, NUM_BUCKETS); - Table table = conn.getTable(tablePath); + Table table = createPkTable(TablePath.of("test_db", "test_concurrent_scans")); int rowCount = 100; UpsertWriter writer = table.newUpsert().createWriter(); @@ -565,10 +378,9 @@ void testConcurrentScans() throws Exception { executor.submit( () -> { try { - List rows = kvScanAll(table); - totalRows.addAndGet(rows.size()); + totalRows.addAndGet(kvScanAll(table).size()); } catch (Exception e) { - throw new RuntimeException(e); + throw new FlussRuntimeException(e); } })); } @@ -583,16 +395,61 @@ void testConcurrentScans() throws Exception { } // ------------------------------------------------------------------------- - // Helper + // Helpers // ------------------------------------------------------------------------- + /** + * Creates a non-partitioned primary key table using {@link #PK_SCHEMA} with {@link + * #NUM_BUCKETS} buckets, waits for all replicas to be ready, and returns an open {@link Table}. + */ + private Table createPkTable(TablePath path) throws Exception { + TableDescriptor descriptor = + TableDescriptor.builder() + .schema(PK_SCHEMA) + .distributedBy(NUM_BUCKETS, "id") + .build(); + long tableId = createTable(path, descriptor, true); + waitAllReplicasReady(tableId, NUM_BUCKETS); + return conn.getTable(path); + } + + /** + * Creates a partitioned primary key table using {@link #PARTITIONED_SCHEMA} with {@link + * #NUM_BUCKETS} buckets and returns the table ID. Partitions must be created separately before + * scanning. + */ + private long createPartitionedTable(TablePath path) throws Exception { + TableDescriptor descriptor = + TableDescriptor.builder() + .schema(PARTITIONED_SCHEMA) + .partitionedBy("p") + .distributedBy(NUM_BUCKETS, "id") + .build(); + return createTable(path, descriptor, true); + } + + /** + * Waits until all bucket replicas are ready for every partition of a partitioned table. This is + * critical for empty partitions that have no write traffic to act as a natural barrier. + */ + private void waitPartitionedTableReplicasReady(long tableId, TablePath tablePath) + throws Exception { + List partitions = admin.listPartitionInfos(tablePath).get(); + for (PartitionInfo partition : partitions) { + for (int i = 0; i < NUM_BUCKETS; i++) { + FLUSS_CLUSTER_EXTENSION.waitUntilAllReplicaReady( + new TableBucket(tableId, partition.getPartitionId(), i)); + } + } + } + private List kvScanAll(Table table) throws Exception { - List allRows = new ArrayList<>(); + List rows = new ArrayList<>(); try (CloseableIterator iterator = table.newKvScan().execute()) { while (iterator.hasNext()) { - allRows.add(iterator.next()); + rows.add(iterator.next()); } } - return allRows; + return rows; } } diff --git a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java index b59d1895a2..3fb86dc3b5 100644 --- a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java +++ b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java @@ -61,15 +61,15 @@ import org.apache.fluss.exception.OutOfOrderSequenceException; import org.apache.fluss.exception.PartitionAlreadyExistsException; import org.apache.fluss.exception.PartitionNotExistException; +import org.apache.fluss.exception.InvalidScanRequestException; import org.apache.fluss.exception.RebalanceFailureException; import org.apache.fluss.exception.RecordTooLargeException; import org.apache.fluss.exception.RetriableAuthenticationException; -import org.apache.fluss.exception.InvalidScanRequestException; import org.apache.fluss.exception.ScannerExpiredException; import org.apache.fluss.exception.ScannerNotFoundException; +import org.apache.fluss.exception.SchemaNotExistException; import org.apache.fluss.exception.TooManyScannersException; import org.apache.fluss.exception.UnknownScannerIdException; -import org.apache.fluss.exception.SchemaNotExistException; import org.apache.fluss.exception.SecurityDisabledException; import org.apache.fluss.exception.SecurityTokenException; import org.apache.fluss.exception.ServerNotExistException; diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerContext.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerContext.java index 2f0231e6c2..e87fc742ab 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerContext.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerContext.java @@ -28,17 +28,12 @@ import javax.annotation.concurrent.NotThreadSafe; -import java.util.concurrent.atomic.AtomicBoolean; - /** * The context for a scanner session. Each instance holds a RocksDB snapshot and iterator for * point-in-time scan isolation. The {@link ResourceGuard.Lease} prevents RocksDB from being closed * while this session is active. * - *

Most fields of this class are not thread-safe; in particular, {@link #iterator} and - * {@link #callSeqId} must only be accessed by a single thread at a time. The {@link #expired} flag - * is the sole exception: it is an {@link java.util.concurrent.atomic.AtomicBoolean} and may be - * written by the TTL reaper thread while another thread holds a reference to the context. + *

This class is not thread-safe; all fields must only be accessed by a single thread at a time. */ @NotThreadSafe public class ScannerContext implements AutoCloseable { @@ -52,7 +47,6 @@ public class ScannerContext implements AutoCloseable { private int callSeqId; private long lastAccessTime; - private final AtomicBoolean expired = new AtomicBoolean(false); public ScannerContext( byte[] scannerId, @@ -102,19 +96,6 @@ public void updateLastAccessTime(long lastAccessTime) { this.lastAccessTime = lastAccessTime; } - /** - * Marks this scanner as expired by the TTL reaper. After this call, {@link #isExpired()} will - * return {@code true}, allowing the server to distinguish an expired session from an unknown - * scanner id. - */ - public void markExpired() { - expired.set(true); - } - - public boolean isExpired() { - return expired.get(); - } - @Override public void close() { try { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java index 32159837c4..6a85a351a5 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java @@ -62,12 +62,15 @@ public class ScannerManager implements AutoCloseableAsync { /** * Tracks recently-expired scanner IDs and their expiry timestamp. Allows distinguishing between * a scanner that expired (SCANNER_EXPIRED) and one that was never created (UNKNOWN_SCANNER_ID). - * Entries are pruned by the cleanup task after 2 × TTL to bound memory usage. + * Entries are pruned by the cleanup task after {@link #recentlyExpiredRetentionMs} to bound + * memory usage. */ private final Map recentlyExpiredIds = MapUtils.newConcurrentHashMap(); private final Scheduler scheduler; private final Clock clock; private final long scannerTtlMs; + /** How long recently-expired IDs are retained for diagnostic error reporting (2 × TTL). */ + private final long recentlyExpiredRetentionMs; private final int maxPerBucket; private final int maxPerServer; private final AtomicInteger totalScanners = new AtomicInteger(0); @@ -81,6 +84,7 @@ public ScannerManager(Configuration conf, Scheduler scheduler, Clock clock) { this.scheduler = scheduler; this.clock = clock; this.scannerTtlMs = conf.get(ConfigOptions.SERVER_SCANNER_TTL).toMillis(); + this.recentlyExpiredRetentionMs = 2 * scannerTtlMs; long expirationIntervalMs = conf.get(ConfigOptions.SERVER_SCANNER_EXPIRATION_INTERVAL).toMillis(); this.maxPerBucket = conf.get(ConfigOptions.SERVER_SCANNER_MAX_PER_BUCKET); @@ -228,6 +232,10 @@ public void closeScannersForBucket(TableBucket tableBucket) { } } + /** + * Counts active scanners for a given bucket. O(n) over total scanners — acceptable because + * {@code maxPerServer} is small (default 200) and this path is only on scanner creation. + */ private int countScannersForBucket(TableBucket tableBucket) { int count = 0; for (ScannerContext ctx : scanners.values()) { @@ -241,18 +249,16 @@ private int countScannersForBucket(TableBucket tableBucket) { private void cleanupExpiredScanners() { long now = clock.milliseconds(); - // Prune stale entries from the recently-expired set (keep for 2 × TTL for diagnostics) - long expiredEntryRetentionMs = 2 * scannerTtlMs; - recentlyExpiredIds.entrySet().removeIf(e -> now - e.getValue() > expiredEntryRetentionMs); + // Prune stale entries from the recently-expired set to bound memory usage + recentlyExpiredIds.entrySet().removeIf(e -> now - e.getValue() > recentlyExpiredRetentionMs); for (Map.Entry entry : scanners.entrySet()) { ScannerContext context = entry.getValue(); if (now - context.getLastAccessTime() > scannerTtlMs) { - context.markExpired(); // Atomic conditional remove to avoid double-close race with removeScanner() if (scanners.remove(entry.getKey(), context)) { totalScanners.decrementAndGet(); - // Record the expiry so subsequent lookups return SCANNER_EXPIRED + // Record the expiry so subsequent lookups can return SCANNER_EXPIRED recentlyExpiredIds.put(entry.getKey(), now); LOG.info( "Scanner {} expired after {}ms idle, closing it.", @@ -295,9 +301,11 @@ public void close() throws Exception { if (cleanupTask != null) { cleanupTask.cancel(true); } - // Note: if the cleanup task is currently running concurrently, a scanner may be closed - // both by the reaper and by this loop. ScannerContext.close() is idempotent for the - // RocksDB iterator and snapshot, so this is safe, though a warning may be logged. + // If the cleanup task races with this loop, a scanner could be closed by both. RocksDB's + // NativeReference.close() is safe to call twice, but ResourceGuard.Lease.close() is not — + // it decrements a reference count. In practice this window is tiny (cancel + loop are on + // the same thread in tests; production uses a scheduled executor where the task is + // already done or is skipped by the cancel). This is an acceptable trade-off at shutdown. for (ScannerContext context : scanners.values()) { closeScannerContext(context); } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java index 43f7351821..3556215f88 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java @@ -337,7 +337,9 @@ public CompletableFuture scanKv(ScanKvRequest request) { return response; } - // Validate call sequence to detect reordered or duplicate requests + // Validate call sequence to detect reordered or duplicate requests. + // Update callSeqId only after continueScan() succeeds so that clients can safely + // retry with the same callSeqId if the server returns an error response. int expectedSeq = context.getCallSeqId() + 1; if (request.getCallSeqId() != expectedSeq) { throw Errors.INVALID_SCAN_REQUEST.exception( @@ -346,9 +348,9 @@ public CompletableFuture scanKv(ScanKvRequest request) { + ", but got: " + request.getCallSeqId()); } + ScanKvResponse scanResponse = continueScan(context, request.getBatchSizeBytes()); context.setCallSeqId(request.getCallSeqId()); - - response.complete(continueScan(context, request.getBatchSizeBytes())); + response.complete(scanResponse); } else { if (!hasBucketScanReq) { throw Errors.INVALID_SCAN_REQUEST.exception( From 730b2aed84e254a5fd888e300a18eef5f6094bcd Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Fri, 6 Mar 2026 12:41:16 +0200 Subject: [PATCH 13/14] add some optimizations --- .../client/table/scanner/TableKvScan.java | 50 +++++++++++++++---- .../table/scanner/batch/KvBatchScanner.java | 35 ++++++++----- .../fluss/server/kv/scan/ScannerContext.java | 9 ++++ .../fluss/server/kv/scan/ScannerManager.java | 13 ++++- 4 files changed, 84 insertions(+), 23 deletions(-) diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableKvScan.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableKvScan.java index b45cbf7f17..780350980f 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableKvScan.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableKvScan.java @@ -19,6 +19,7 @@ import org.apache.fluss.client.FlussConnection; import org.apache.fluss.client.table.scanner.batch.BatchScanner; +import org.apache.fluss.client.table.scanner.batch.KvBatchScanner; import org.apache.fluss.exception.FlussRuntimeException; import org.apache.fluss.metadata.PartitionInfo; import org.apache.fluss.metadata.SchemaGetter; @@ -27,6 +28,8 @@ import org.apache.fluss.row.InternalRow; import org.apache.fluss.utils.CloseableIterator; +import javax.annotation.Nullable; + import java.io.IOException; import java.time.Duration; import java.util.ArrayList; @@ -78,20 +81,31 @@ public CloseableIterator execute() { return new MultiBucketIterator(buckets, scan); } - private static CloseableIterator scanBucket(Scan scan, TableBucket tableBucket) { - BatchScanner batchScanner = scan.createBatchScanner(tableBucket); - return new BatchScannerIterator(batchScanner); - } - private static class MultiBucketIterator implements CloseableIterator { - private final Iterator bucketIterator; + private final List buckets; private final Scan scan; + private int nextBucketIndex; + @Nullable private BatchScanner prefetchedScanner; private CloseableIterator currentScannerIterator; private boolean isClosed = false; private MultiBucketIterator(List buckets, Scan scan) { - this.bucketIterator = buckets.iterator(); + this.buckets = buckets; this.scan = scan; + this.nextBucketIndex = 0; + // Eagerly open and prefetch the first bucket scanner so its first RPC is in-flight + // while the caller is setting up iteration. + if (!buckets.isEmpty()) { + prefetchedScanner = openAndPrefetch(nextBucketIndex++); + } + } + + private BatchScanner openAndPrefetch(int index) { + BatchScanner scanner = scan.createBatchScanner(buckets.get(index)); + if (scanner instanceof KvBatchScanner) { + ((KvBatchScanner) scanner).prefetch(); + } + return scanner; } @Override @@ -104,11 +118,21 @@ public boolean hasNext() { currentScannerIterator.close(); currentScannerIterator = null; } - if (bucketIterator.hasNext()) { - currentScannerIterator = scanBucket(scan, bucketIterator.next()); + // Take the prefetched scanner (first RPC already in-flight), or open a new one. + BatchScanner nextScanner; + if (prefetchedScanner != null) { + nextScanner = prefetchedScanner; + prefetchedScanner = null; + } else if (nextBucketIndex < buckets.size()) { + nextScanner = scan.createBatchScanner(buckets.get(nextBucketIndex++)); } else { return false; } + // Eagerly open and prefetch the scanner for the bucket after this one. + if (nextBucketIndex < buckets.size()) { + prefetchedScanner = openAndPrefetch(nextBucketIndex++); + } + currentScannerIterator = new BatchScannerIterator(nextScanner); } return true; } @@ -125,6 +149,14 @@ public InternalRow next() { public void close() { if (!isClosed) { isClosed = true; + if (prefetchedScanner != null) { + try { + prefetchedScanner.close(); + } catch (IOException e) { + throw new FlussRuntimeException("Error closing prefetched scanner", e); + } + prefetchedScanner = null; + } if (currentScannerIterator != null) { currentScannerIterator.close(); } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/batch/KvBatchScanner.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/batch/KvBatchScanner.java index 5f8abfee82..57cb9d5c62 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/batch/KvBatchScanner.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/batch/KvBatchScanner.java @@ -71,6 +71,7 @@ public class KvBatchScanner implements BatchScanner { private final int batchSizeBytes; private final Map schemaProjectionCache = new HashMap<>(); + private final ValueRecordReadContext readContext; private byte[] scannerId; private int callSeqId = 0; @@ -107,6 +108,7 @@ public KvBatchScanner( for (int i = 0; i < rowType.getFieldCount(); i++) { this.fieldGetters[i] = InternalRow.createFieldGetter(rowType.getTypeAt(i), i); } + this.readContext = ValueRecordReadContext.createReadContext(schemaGetter, kvFormat); } @Nullable @@ -148,6 +150,16 @@ public CloseableIterator pollBatch(Duration timeout) throws IOExcep } } + /** + * Fires the first RPC eagerly so the network round-trip overlaps with draining the previous + * bucket. No-op if a request is already in-flight or the scanner is closed/exhausted. + */ + public void prefetch() { + if (!isClosed && hasMoreResults && inFlightRequest == null) { + sendRequest(); + } + } + private void sendRequest() { int leader = metadataUpdater.leaderFor(tableInfo.getTablePath(), tableBucket); TabletServerGateway gateway = metadataUpdater.newTabletServerClientForNode(leader); @@ -180,12 +192,10 @@ private List parseScanKvResponse(ScanKvResponse response) { return Collections.emptyList(); } - List scanRows = new ArrayList<>(); + List scanRows = new ArrayList<>(64); ByteBuffer recordsBuffer = ByteBuffer.wrap(response.getRecords()); DefaultValueRecordBatch valueRecords = DefaultValueRecordBatch.pointToByteBuffer(recordsBuffer); - ValueRecordReadContext readContext = - ValueRecordReadContext.createReadContext(schemaGetter, kvFormat); for (ValueRecord record : valueRecords.records(readContext)) { InternalRow row = record.getRow(); @@ -199,23 +209,22 @@ private List parseScanKvResponse(ScanKvResponse response) { schemaGetter.getSchema(targetSchemaId))); row = ProjectedRow.from(indexMapping).replaceRow(row); } - scanRows.add(maybeProject(row)); + if (projectedFields != null) { + row = applyColumnProjection(row); + } + scanRows.add(row); } return scanRows; } - private InternalRow maybeProject(InternalRow originRow) { + private InternalRow applyColumnProjection(InternalRow row) { GenericRow newRow = new GenericRow(fieldGetters.length); for (int i = 0; i < fieldGetters.length; i++) { - newRow.setField(i, fieldGetters[i].getFieldOrNull(originRow)); - } - if (projectedFields != null) { - ProjectedRow projectedRow = ProjectedRow.from(projectedFields); - projectedRow.replaceRow(newRow); - return projectedRow; - } else { - return newRow; + newRow.setField(i, fieldGetters[i].getFieldOrNull(row)); } + ProjectedRow projectedRow = ProjectedRow.from(projectedFields); + projectedRow.replaceRow(newRow); + return projectedRow; } @Override diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerContext.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerContext.java index e87fc742ab..01def75dff 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerContext.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerContext.java @@ -28,6 +28,8 @@ import javax.annotation.concurrent.NotThreadSafe; +import java.nio.ByteBuffer; + /** * The context for a scanner session. Each instance holds a RocksDB snapshot and iterator for * point-in-time scan isolation. The {@link ResourceGuard.Lease} prevents RocksDB from being closed @@ -38,6 +40,8 @@ @NotThreadSafe public class ScannerContext implements AutoCloseable { private final byte[] scannerId; + /** Pre-computed map key to avoid per-RPC {@code ByteBuffer.wrap(scannerId)} allocation. */ + private final ByteBuffer scannerIdKey; private final TableBucket tableBucket; private final RocksDBKv rocksDBKv; private final RocksIterator iterator; @@ -58,6 +62,7 @@ public ScannerContext( ResourceGuard.Lease resourceLease, Clock clock) { this.scannerId = scannerId; + this.scannerIdKey = ByteBuffer.wrap(scannerId); this.tableBucket = tableBucket; this.rocksDBKv = rocksDBKv; this.iterator = iterator; @@ -72,6 +77,10 @@ public byte[] getScannerId() { return scannerId; } + public ByteBuffer getScannerIdKey() { + return scannerIdKey; + } + public TableBucket getTableBucket() { return tableBucket; } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java index 6a85a351a5..602e48b713 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java @@ -166,7 +166,7 @@ public ScannerContext createScanner(KvTablet kvTablet, TableBucket tableBucket) snapshot, lease, clock); - scanners.put(ByteBuffer.wrap(scannerId), context); + scanners.put(context.getScannerIdKey(), context); totalScanners.incrementAndGet(); return context; } catch (Exception e) { @@ -208,6 +208,17 @@ public void removeScanner(byte[] scannerId) { } } + /** + * Removes and closes a known scanner context directly, avoiding a map lookup. Prefer this over + * {@link #removeScanner(byte[])} when the caller already holds the {@link ScannerContext}. + */ + public void removeScanner(ScannerContext context) { + if (scanners.remove(context.getScannerIdKey(), context)) { + totalScanners.decrementAndGet(); + closeScannerContext(context); + } + } + /** * Closes and removes all active scanner sessions for the given bucket. Called when a bucket * loses leadership to prevent stale RocksDB snapshot/iterator leaks. From b3d5b7af0e5d8a74c0fe2569d380262b6023f7fb Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Fri, 6 Mar 2026 12:58:27 +0200 Subject: [PATCH 14/14] update --- .../fluss/client/table/scanner/TableKvScan.java | 8 +++++--- .../table/scanner/batch/KvBatchScanner.java | 16 +--------------- .../org/apache/fluss/rpc/protocol/Errors.java | 6 +++--- .../fluss/server/kv/scan/ScannerContext.java | 2 +- .../fluss/server/kv/scan/ScannerManager.java | 17 ++++++++--------- .../fluss/server/tablet/TabletService.java | 6 +++--- 6 files changed, 21 insertions(+), 34 deletions(-) diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableKvScan.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableKvScan.java index 780350980f..9f6e214b12 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableKvScan.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/TableKvScan.java @@ -96,7 +96,8 @@ private MultiBucketIterator(List buckets, Scan scan) { // Eagerly open and prefetch the first bucket scanner so its first RPC is in-flight // while the caller is setting up iteration. if (!buckets.isEmpty()) { - prefetchedScanner = openAndPrefetch(nextBucketIndex++); + prefetchedScanner = openAndPrefetch(nextBucketIndex); + nextBucketIndex++; } } @@ -130,7 +131,8 @@ public boolean hasNext() { } // Eagerly open and prefetch the scanner for the bucket after this one. if (nextBucketIndex < buckets.size()) { - prefetchedScanner = openAndPrefetch(nextBucketIndex++); + prefetchedScanner = openAndPrefetch(nextBucketIndex); + nextBucketIndex++; } currentScannerIterator = new BatchScannerIterator(nextScanner); } @@ -209,12 +211,12 @@ private void ensureBatch() { @Override public void close() { if (!isClosed) { + isClosed = true; try { scanner.close(); } catch (IOException e) { throw new FlussRuntimeException("Error closing scanner", e); } - isClosed = true; } } } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/batch/KvBatchScanner.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/batch/KvBatchScanner.java index 57cb9d5c62..427d1a22d3 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/batch/KvBatchScanner.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/batch/KvBatchScanner.java @@ -27,7 +27,6 @@ import org.apache.fluss.record.DefaultValueRecordBatch; import org.apache.fluss.record.ValueRecord; import org.apache.fluss.record.ValueRecordReadContext; -import org.apache.fluss.row.GenericRow; import org.apache.fluss.row.InternalRow; import org.apache.fluss.row.ProjectedRow; import org.apache.fluss.rpc.gateway.TabletServerGateway; @@ -35,7 +34,6 @@ import org.apache.fluss.rpc.messages.ScanKvRequest; import org.apache.fluss.rpc.messages.ScanKvResponse; import org.apache.fluss.rpc.protocol.Errors; -import org.apache.fluss.types.RowType; import org.apache.fluss.utils.CloseableIterator; import org.apache.fluss.utils.SchemaUtil; @@ -66,7 +64,6 @@ public class KvBatchScanner implements BatchScanner { @Nullable private final int[] projectedFields; @Nullable private final Long limit; private final int targetSchemaId; - private final InternalRow.FieldGetter[] fieldGetters; private final KvFormat kvFormat; private final int batchSizeBytes; @@ -103,11 +100,6 @@ public KvBatchScanner( .get(ConfigOptions.CLIENT_SCANNER_KV_FETCH_MAX_BYTES) .getBytes(); - RowType rowType = tableInfo.getRowType(); - this.fieldGetters = new InternalRow.FieldGetter[rowType.getFieldCount()]; - for (int i = 0; i < rowType.getFieldCount(); i++) { - this.fieldGetters[i] = InternalRow.createFieldGetter(rowType.getTypeAt(i), i); - } this.readContext = ValueRecordReadContext.createReadContext(schemaGetter, kvFormat); } @@ -218,13 +210,7 @@ private List parseScanKvResponse(ScanKvResponse response) { } private InternalRow applyColumnProjection(InternalRow row) { - GenericRow newRow = new GenericRow(fieldGetters.length); - for (int i = 0; i < fieldGetters.length; i++) { - newRow.setField(i, fieldGetters[i].getFieldOrNull(row)); - } - ProjectedRow projectedRow = ProjectedRow.from(projectedFields); - projectedRow.replaceRow(newRow); - return projectedRow; + return ProjectedRow.from(projectedFields).replaceRow(row); } @Override diff --git a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java index 3fb86dc3b5..f43e6cfa7a 100644 --- a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java +++ b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java @@ -38,6 +38,7 @@ import org.apache.fluss.exception.InvalidPartitionException; import org.apache.fluss.exception.InvalidReplicationFactorException; import org.apache.fluss.exception.InvalidRequiredAcksException; +import org.apache.fluss.exception.InvalidScanRequestException; import org.apache.fluss.exception.InvalidServerRackInfoException; import org.apache.fluss.exception.InvalidTableException; import org.apache.fluss.exception.InvalidTargetColumnException; @@ -61,15 +62,12 @@ import org.apache.fluss.exception.OutOfOrderSequenceException; import org.apache.fluss.exception.PartitionAlreadyExistsException; import org.apache.fluss.exception.PartitionNotExistException; -import org.apache.fluss.exception.InvalidScanRequestException; import org.apache.fluss.exception.RebalanceFailureException; import org.apache.fluss.exception.RecordTooLargeException; import org.apache.fluss.exception.RetriableAuthenticationException; import org.apache.fluss.exception.ScannerExpiredException; import org.apache.fluss.exception.ScannerNotFoundException; import org.apache.fluss.exception.SchemaNotExistException; -import org.apache.fluss.exception.TooManyScannersException; -import org.apache.fluss.exception.UnknownScannerIdException; import org.apache.fluss.exception.SecurityDisabledException; import org.apache.fluss.exception.SecurityTokenException; import org.apache.fluss.exception.ServerNotExistException; @@ -82,6 +80,8 @@ import org.apache.fluss.exception.TimeoutException; import org.apache.fluss.exception.TooManyBucketsException; import org.apache.fluss.exception.TooManyPartitionsException; +import org.apache.fluss.exception.TooManyScannersException; +import org.apache.fluss.exception.UnknownScannerIdException; import org.apache.fluss.exception.UnknownServerException; import org.apache.fluss.exception.UnknownTableOrBucketException; import org.apache.fluss.exception.UnknownWriterIdException; diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerContext.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerContext.java index 01def75dff..914a8faeef 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerContext.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerContext.java @@ -78,7 +78,7 @@ public byte[] getScannerId() { } public ByteBuffer getScannerIdKey() { - return scannerIdKey; + return scannerIdKey.duplicate(); } public TableBucket getTableBucket() { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java index 602e48b713..36fb5f93cd 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java @@ -310,17 +310,16 @@ public CompletableFuture closeAsync() { @Override public void close() throws Exception { if (cleanupTask != null) { - cleanupTask.cancel(true); + cleanupTask.cancel(false); } - // If the cleanup task races with this loop, a scanner could be closed by both. RocksDB's - // NativeReference.close() is safe to call twice, but ResourceGuard.Lease.close() is not — - // it decrements a reference count. In practice this window is tiny (cancel + loop are on - // the same thread in tests; production uses a scheduled executor where the task is - // already done or is skipped by the cancel). This is an acceptable trade-off at shutdown. - for (ScannerContext context : scanners.values()) { - closeScannerContext(context); + // Use the same conditional-remove pattern as the cleanup task so that each ScannerContext + // is closed by exactly one caller even if the cleanup task is concurrently running. + for (Map.Entry entry : scanners.entrySet()) { + if (scanners.remove(entry.getKey(), entry.getValue())) { + totalScanners.decrementAndGet(); + closeScannerContext(entry.getValue()); + } } - scanners.clear(); recentlyExpiredIds.clear(); totalScanners.set(0); } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java index 3556215f88..507abd34ba 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java @@ -330,7 +330,7 @@ public CompletableFuture scanKv(ScanKvRequest request) { } if (request.hasCloseScanner() && request.isCloseScanner()) { - scannerManager.removeScanner(scannerId); + scannerManager.removeScanner(context); ScanKvResponse scanResponse = new ScanKvResponse(); scanResponse.setScannerId(scannerId).setHasMoreResults(false); response.complete(scanResponse); @@ -402,7 +402,7 @@ public CompletableFuture scanKv(ScanKvRequest request) { private ScanKvResponse continueScan(ScannerContext context, int batchSizeBytes) throws IOException { - byte[] scannerId = context.getScannerId(); + final byte[] scannerId = context.getScannerId(); RocksIterator iterator = context.getIterator(); DefaultValueRecordBatch.Builder builder = DefaultValueRecordBatch.builder(); int currentBytes = 0; @@ -433,7 +433,7 @@ private ScanKvResponse continueScan(ScannerContext context, int batchSizeBytes) } if (!hasMore) { - scannerManager.removeScanner(scannerId); + scannerManager.removeScanner(context); } return response;