From 39a3cc58d7fb400a3f896077044d9943cbdd0f79 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Tue, 21 Apr 2026 12:28:15 +0300 Subject: [PATCH 01/17] add configoptions --- .../apache/fluss/config/ConfigOptions.java | 34 +++++++++++++++++++ 1 file changed, 34 insertions(+) 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 df21129db6..4e3713f0c3 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 @@ -515,6 +515,40 @@ public class ConfigOptions { + WRITER_ID_EXPIRATION_TIME.key() + " passing. The default value is 10 minutes."); + public static final ConfigOption KV_SCANNER_TTL = + key("kv.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 KV_SCANNER_EXPIRATION_INTERVAL = + key("kv.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 KV_SCANNER_MAX_PER_BUCKET = + key("kv.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 KV_SCANNER_MAX_PER_SERVER = + key("kv.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() From 245c5fbfd474b9ae5712edf334de892913bc6dcf Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Tue, 21 Apr 2026 12:28:37 +0300 Subject: [PATCH 02/17] add server side impl --- .../org/apache/fluss/server/kv/KvTablet.java | 80 ++++ .../fluss/server/kv/scan/ScannerContext.java | 172 ++++++++ .../fluss/server/kv/scan/ScannerManager.java | 413 ++++++++++++++++++ .../apache/fluss/server/replica/Replica.java | 24 + .../fluss/server/replica/ReplicaManager.java | 46 ++ .../fluss/server/tablet/TabletServer.java | 14 +- .../fluss/server/tablet/TabletService.java | 127 +++++- .../apache/fluss/server/kv/KvTabletTest.java | 139 ++++++ .../server/kv/scan/ScannerManagerTest.java | 359 +++++++++++++++ 9 files changed, 1371 insertions(+), 3 deletions(-) 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 create mode 100644 fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/KvTablet.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvTablet.java index 85c192c922..db763c7a9d 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/KvTablet.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvTablet.java @@ -58,6 +58,7 @@ import org.apache.fluss.server.kv.rocksdb.RocksDBStatistics; import org.apache.fluss.server.kv.rowmerger.DefaultRowMerger; import org.apache.fluss.server.kv.rowmerger.RowMerger; +import org.apache.fluss.server.kv.scan.ScannerContext; import org.apache.fluss.server.kv.snapshot.KvFileHandleAndLocalPath; import org.apache.fluss.server.kv.snapshot.KvSnapshotDataUploader; import org.apache.fluss.server.kv.snapshot.RocksIncrementalSnapshot; @@ -70,12 +71,16 @@ import org.apache.fluss.server.log.LogTablet; import org.apache.fluss.server.metrics.group.TabletServerMetricGroup; import org.apache.fluss.server.utils.FatalErrorHandler; +import org.apache.fluss.server.utils.ResourceGuard; import org.apache.fluss.shaded.arrow.org.apache.arrow.memory.BufferAllocator; import org.apache.fluss.types.RowType; import org.apache.fluss.utils.BytesUtils; import org.apache.fluss.utils.FileUtils; import org.rocksdb.RateLimiter; +import org.rocksdb.ReadOptions; +import org.rocksdb.RocksIterator; +import org.rocksdb.Snapshot; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -757,6 +762,81 @@ public List limitScan(int limit) throws IOException { }); } + /** + * Opens a new full-scan session, taking a point-in-time RocksDB snapshot under the {@code + * kvLock} read lock. + * + *

Returns {@code null} if the bucket contains no rows at the time of the call. In that case + * all acquired resources are released internally and no session is registered. + * + *

The returned {@link ScannerContext} is unregistered — the caller ({@link + * org.apache.fluss.server.kv.scan.ScannerManager}) is responsible for registering it and for + * closing it when the scan is complete. + * + * @param scannerId the server-assigned scanner ID bytes + * @param limit maximum number of rows to return across all batches ({@code ≤ 0} = unlimited) + * @param initialAccessTimeMs wall-clock time (ms) to use as the initial last-access timestamp + * @return a newly created, cursor-positioned {@link ScannerContext}, or {@code null} if the + * bucket is empty + * @throws IOException if the ResourceGuard is already closed (RocksDB is shutting down) + */ + @Nullable + public ScannerContext openScan(String scannerId, long limit, long initialAccessTimeMs) + throws IOException { + return inReadLock( + kvLock, + () -> { + rocksDBKv.checkIfRocksDBClosed(); + ResourceGuard.Lease lease = rocksDBKv.getResourceGuard().acquireResource(); + Snapshot snapshot = null; + ReadOptions readOptions = null; + RocksIterator iterator = null; + try { + snapshot = rocksDBKv.getDb().getSnapshot(); + readOptions = new ReadOptions().setSnapshot(snapshot); + iterator = + rocksDBKv + .getDb() + .newIterator( + rocksDBKv.getDefaultColumnFamilyHandle(), + readOptions); + iterator.seekToFirst(); + if (!iterator.isValid()) { + // Empty bucket: release all resources without creating a session. + iterator.close(); + readOptions.close(); + rocksDBKv.getDb().releaseSnapshot(snapshot); + snapshot.close(); + lease.close(); + return null; + } + return new ScannerContext( + scannerId, + tableBucket, + rocksDBKv, + iterator, + readOptions, + snapshot, + lease, + limit, + initialAccessTimeMs); + } catch (Exception e) { + if (iterator != null) { + iterator.close(); + } + if (readOptions != null) { + readOptions.close(); + } + if (snapshot != null) { + rocksDBKv.getDb().releaseSnapshot(snapshot); + snapshot.close(); + } + lease.close(); + throw e; + } + }); + } + public KvBatchWriter createKvBatchWriter() { return rocksDBKv.newWriteBatch( writeBatchSize, 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..baa0a90426 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerContext.java @@ -0,0 +1,172 @@ +/* + * 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.metadata.TableBucket; +import org.apache.fluss.server.kv.rocksdb.RocksDBKv; +import org.apache.fluss.server.utils.ResourceGuard; + +import org.rocksdb.ReadOptions; +import org.rocksdb.RocksIterator; +import org.rocksdb.Snapshot; + +import javax.annotation.concurrent.NotThreadSafe; + +import java.io.Closeable; +import java.nio.charset.StandardCharsets; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * Server-side state for a single KV full-scan session. + * + *

A {@code ScannerContext} holds a point-in-time RocksDB {@link Snapshot}, the {@link + * ReadOptions} pinning it, and a cursor ({@link RocksIterator}) that persists across multiple + * batched-fetch RPCs from the same client. It also holds a {@link ResourceGuard.Lease} that + * prevents the underlying RocksDB instance from being closed while the scan is in progress. + * + *

Instances are created by {@link org.apache.fluss.server.kv.KvTablet#openScan} and registered + * by {@link ScannerManager}. They must be closed when the scan completes, the client requests an + * explicit close, or the session expires due to inactivity. + * + *

Thread safety: The iterator cursor ({@link #advance()}, {@link #isValid()}, {@link + * #currentValue()}) must be driven by only one thread at a time. {@link #close()} is thread-safe. + */ +@NotThreadSafe +public class ScannerContext implements Closeable { + private final String scannerId; + private final byte[] scannerIdBytes; + private final TableBucket tableBucket; + private final RocksDBKv rocksDBKv; + private final RocksIterator iterator; + private final ReadOptions readOptions; + private final Snapshot snapshot; + private final ResourceGuard.Lease resourceLease; + private long remainingLimit; + // Initial value -1 so that the first client call_seq_id of 0 satisfies the server's + // in-order check: expectedSeqId = callSeqId + 1 = -1 + 1 = 0. + // callSeqId validation is only performed for continuation requests (those carrying a + // scanner_id), never for the initial open request (those carrying a bucket_scan_req). + private int callSeqId = -1; + + /** + * Wall-clock timestamp (ms) of the most recent request that touched this session. Used by + * {@link ScannerManager} for TTL-based eviction. + */ + private long lastAccessTime; + + private final AtomicBoolean closed = new AtomicBoolean(false); + + public ScannerContext( + String scannerId, + TableBucket tableBucket, + RocksDBKv rocksDBKv, + RocksIterator iterator, + ReadOptions readOptions, + Snapshot snapshot, + ResourceGuard.Lease resourceLease, + long limit, + long initialAccessTimeMs) { + this.scannerId = scannerId; + this.scannerIdBytes = scannerId.getBytes(StandardCharsets.UTF_8); + this.tableBucket = tableBucket; + this.rocksDBKv = rocksDBKv; + this.iterator = iterator; + this.readOptions = readOptions; + this.snapshot = snapshot; + this.resourceLease = resourceLease; + this.remainingLimit = limit <= 0 ? -1L : limit; + this.lastAccessTime = initialAccessTimeMs; + } + + public byte[] getScannerId() { + return scannerIdBytes; + } + + String getId() { + return scannerId; + } + + public TableBucket getTableBucket() { + return tableBucket; + } + + public boolean isValid() { + return iterator.isValid() && remainingLimit != 0; + } + + public byte[] currentValue() { + return iterator.value(); + } + + /** + * Advances the cursor by one entry and decrements the remaining-rows limit if applicable. Must + * only be called when {@link #isValid()} returns {@code true}. + */ + public void advance() { + iterator.next(); + if (remainingLimit > 0) { + remainingLimit--; + } + } + + /** Returns the call-sequence ID of the last successfully served request, or {@code -1}. */ + public int getCallSeqId() { + return callSeqId; + } + + /** + * Updates the call-sequence ID. Must be called after the response payload has been + * fully prepared, so that a client retry with the same {@code callSeqId} can be detected. + */ + public void setCallSeqId(int callSeqId) { + this.callSeqId = callSeqId; + } + + /** Resets the idle-TTL timer to the given wall-clock time. */ + public void updateLastAccessTime(long nowMs) { + this.lastAccessTime = nowMs; + } + + /** + * Returns {@code true} if the session has been idle for longer than {@code ttlMs}, based on the + * provided current time. + */ + public boolean isExpired(long ttlMs, long nowMs) { + return nowMs - lastAccessTime > ttlMs; + } + + @Override + public void close() { + if (closed.compareAndSet(false, true)) { + try { + iterator.close(); + } finally { + try { + readOptions.close(); + } finally { + try { + rocksDBKv.getDb().releaseSnapshot(snapshot); + snapshot.close(); + } 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 new file mode 100644 index 0000000000..aee3d641f0 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/ScannerManager.java @@ -0,0 +1,413 @@ +/* + * 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.annotation.VisibleForTesting; +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.exception.TooManyScannersException; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.server.kv.KvTablet; +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.FutureUtils; +import org.apache.fluss.utils.concurrent.Scheduler; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +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.ConcurrentHashMap; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Manages server-side KV full-scan sessions ({@link ScannerContext}). + * + *

Each KV full scan opens a persistent server-side session that holds a point-in-time RocksDB + * snapshot and a cursor. Sessions are keyed by a server-assigned UUID-based scanner ID and persist + * across multiple batched-fetch RPCs from the same client. + * + *

Concurrency limits

+ * + *
    + *
  • Per-bucket: at most {@code maxPerBucket} concurrent sessions on any single bucket. + *
  • Per-server: at most {@code maxPerServer} concurrent sessions across all buckets. + *
+ * + *

Limit enforcement is two-phase: a fast pre-check guards the common case; the subsequent atomic + * increment with re-check and rollback prevents the TOCTOU race from permanently breaching the + * configured limits. Exceeding either limit causes {@link TooManyScannersException}. + * + *

Empty bucket handling

+ * + *

If the target bucket contains no rows at the time the scan is opened, {@link + * #createScanner(KvTablet, TableBucket, Long)} returns {@code null} without consuming a limit slot. + * The caller should return an empty response immediately. + * + *

TTL eviction

+ * + *

A background reaper task runs every {@code kv.scanner.expiration-interval} and evicts sessions + * idle longer than {@code kv.scanner.ttl}. Recently evicted IDs are retained for {@code 2 × ttl} so + * callers can distinguish "expired" from "never existed." + * + *

Leadership change

+ * + * {@link #closeScannersForBucket(TableBucket)} must be called when a bucket loses leadership to + * release all RocksDB snapshot/iterator resources for that bucket promptly. + */ +public class ScannerManager implements AutoCloseableAsync { + + private static final Logger LOG = LoggerFactory.getLogger(ScannerManager.class); + + private final Map scanners = new ConcurrentHashMap<>(); + private final Map recentlyExpiredIds = new ConcurrentHashMap<>(); + + /** Per-bucket active scanner count, used for O(1) per-bucket limit enforcement. */ + private final Map perBucketCount = new ConcurrentHashMap<>(); + + /** Total active scanner count across all buckets on this tablet server. */ + private final AtomicInteger totalScanners = new AtomicInteger(0); + + private final Clock clock; + private final long scannerTtlMs; + private final long recentlyExpiredRetentionMs; + private final int maxPerBucket; + private final int maxPerServer; + + @Nullable private ScheduledFuture cleanupTask; + + public ScannerManager(Configuration conf, Scheduler scheduler) { + this(conf, scheduler, SystemClock.getInstance()); + } + + @VisibleForTesting + ScannerManager(Configuration conf, Scheduler scheduler, Clock clock) { + this.clock = clock; + this.scannerTtlMs = conf.get(ConfigOptions.KV_SCANNER_TTL).toMillis(); + this.recentlyExpiredRetentionMs = 2 * scannerTtlMs; + this.maxPerBucket = conf.get(ConfigOptions.KV_SCANNER_MAX_PER_BUCKET); + this.maxPerServer = conf.get(ConfigOptions.KV_SCANNER_MAX_PER_SERVER); + + long expirationIntervalMs = + conf.get(ConfigOptions.KV_SCANNER_EXPIRATION_INTERVAL).toMillis(); + this.cleanupTask = + scheduler.schedule( + "scanner-expiration", + this::cleanupExpiredScanners, + expirationIntervalMs, + expirationIntervalMs); + + LOG.info( + "Started ScannerManager: ttl={}ms, expirationInterval={}ms, " + + "maxPerBucket={}, maxPerServer={}", + scannerTtlMs, + expirationIntervalMs, + maxPerBucket, + maxPerServer); + } + + /** + * Creates a new scan session for the given bucket, taking a point-in-time RocksDB snapshot. + * + *

Returns {@code null} if the bucket is empty (no rows to scan). In that case no session + * slot is consumed and the caller should return an empty response immediately. + * + *

Limit enforcement is two-phase: a fast pre-check guards the common case; the + * subsequent atomic increment + re-check prevents the TOCTOU race from permanently breaching + * configured limits. If registration fails after the snapshot is already opened, the context is + * closed and the exception is re-thrown to avoid leaking resources. + * + * @param kvTablet the {@link KvTablet} for the bucket; used to open the snapshot + * @param tableBucket the bucket being scanned + * @param limit optional row-count limit ({@code null} or ≤ 0 means unlimited) + * @return the newly registered {@link ScannerContext}, or {@code null} if the bucket is empty + * @throws TooManyScannersException if the per-bucket or per-server limit is exceeded + * @throws IOException if the underlying {@link org.apache.fluss.server.utils.ResourceGuard} is + * already closed (the bucket is shutting down) + */ + @Nullable + public ScannerContext createScanner( + KvTablet kvTablet, TableBucket tableBucket, @Nullable Long limit) throws IOException { + checkLimits(tableBucket); + + String scannerId = generateScannerId(); + ScannerContext context = + kvTablet.openScan(scannerId, limit != null ? limit : -1L, clock.milliseconds()); + if (context == null) { + // Bucket is empty — no session slot consumed. + return null; + } + + try { + registerContext(context, tableBucket); + } catch (TooManyScannersException e) { + // Limit was exceeded between the initial check and registration (race window). + // Close the already-opened context to avoid leaking the snapshot and lease. + closeScannerContext(context); + throw e; + } + return context; + } + + /** + * Looks up an existing scanner session by its raw ID bytes and refreshes its last-access + * timestamp. + * + * @return the {@link ScannerContext}, or {@code null} if not found (may have expired or never + * existed) + */ + @Nullable + public ScannerContext getScanner(byte[] scannerId) { + ScannerContext context = scanners.get(new String(scannerId, StandardCharsets.UTF_8)); + if (context != null) { + context.updateLastAccessTime(clock.milliseconds()); + } + return context; + } + + /** + * Returns {@code true} if the given scanner ID belongs to a session that was recently evicted + * by the TTL reaper (within the last {@code 2 × ttlMs}). + * + *

Callers can use this to distinguish "scanner expired" from "unknown scanner ID." + */ + public boolean isRecentlyExpired(byte[] scannerId) { + return recentlyExpiredIds.containsKey(new String(scannerId, StandardCharsets.UTF_8)); + } + + /** + * Removes and closes a known scanner context directly, avoiding a map lookup. + * + *

Uses a conditional remove ({@link java.util.concurrent.ConcurrentHashMap#remove(Object, + * Object)}) so that concurrent calls — e.g. from the TTL reaper and a close-scanner RPC + * arriving simultaneously — result in exactly one winner closing the context, preventing + * double-release of the non-idempotent {@link + * org.apache.fluss.server.utils.ResourceGuard.Lease}. + */ + public void removeScanner(ScannerContext context) { + if (scanners.remove(context.getId(), context)) { + decrementCounts(context.getTableBucket()); + closeScannerContext(context); + } + } + + /** + * Looks up and removes a scanner session by its raw ID bytes. + * + *

Delegates to {@link #removeScanner(ScannerContext)} to ensure a conditional {@link + * java.util.concurrent.ConcurrentHashMap#remove(Object, Object)} is used, which prevents a + * double-decrement of {@code perBucketCount} when the TTL reaper races with an explicit close + * request for the same scanner. + * + *

No-op if the ID is not found (already removed or expired). + */ + public void removeScanner(byte[] scannerId) { + String key = new String(scannerId, StandardCharsets.UTF_8); + ScannerContext context = scanners.get(key); + if (context != null) { + removeScanner(context); + } + } + + /** Returns the total number of active scanner sessions on this tablet server. */ + @VisibleForTesting + public int activeScannerCount() { + return totalScanners.get(); + } + + /** Returns the number of active scanner sessions for the given bucket. */ + @VisibleForTesting + public int activeScannerCountForBucket(TableBucket tableBucket) { + AtomicInteger count = perBucketCount.get(tableBucket); + return count == null ? 0 : count.get(); + } + + /** + * Closes and removes all active scanner sessions for the given bucket. Must be 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 (String key : toRemove) { + ScannerContext context = scanners.get(key); + if (context != null) { + LOG.info( + "Closing scanner {} for bucket {} due to leadership change.", + key, + tableBucket); + removeScanner(context); + } + } + } + + /** + * Fast pre-check of per-server and per-bucket limits before opening the snapshot. This is a + * best-effort check; a small race window exists between the check and {@link #registerContext}. + * The race is handled by the atomic re-check inside {@link #registerContext}. + */ + private void checkLimits(TableBucket tableBucket) { + if (totalScanners.get() >= maxPerServer) { + throw new TooManyScannersException( + String.format( + "Cannot create scanner for bucket %s: server-wide limit of %d reached.", + tableBucket, maxPerServer)); + } + AtomicInteger bucketCount = + perBucketCount.computeIfAbsent(tableBucket, k -> new AtomicInteger(0)); + if (bucketCount.get() >= maxPerBucket) { + throw new TooManyScannersException( + String.format( + "Cannot create scanner for bucket %s: per-bucket limit of %d reached.", + tableBucket, maxPerBucket)); + } + } + + /** + * Atomically increments the counters and puts the context in the map. Throws {@link + * TooManyScannersException} and rolls back the increments if a concurrent create caused either + * limit to be exceeded between the initial check and this call. + */ + private void registerContext(ScannerContext context, TableBucket tableBucket) { + AtomicInteger bucketCount = + perBucketCount.computeIfAbsent(tableBucket, k -> new AtomicInteger(0)); + + int newTotal = totalScanners.incrementAndGet(); + if (newTotal > maxPerServer) { + totalScanners.decrementAndGet(); + throw new TooManyScannersException( + String.format( + "Cannot create scanner for bucket %s: server-wide limit of %d reached.", + tableBucket, maxPerServer)); + } + + int newBucketCount = bucketCount.incrementAndGet(); + if (newBucketCount > maxPerBucket) { + bucketCount.decrementAndGet(); + totalScanners.decrementAndGet(); + throw new TooManyScannersException( + String.format( + "Cannot create scanner for bucket %s: per-bucket limit of %d reached.", + tableBucket, maxPerBucket)); + } + + scanners.put(context.getId(), context); + + LOG.debug( + "Registered scanner {} for bucket {} (total={}, perBucket={})", + context.getId(), + tableBucket, + newTotal, + newBucketCount); + } + + /** TTL reaper — invoked periodically by the background scheduler. */ + private void cleanupExpiredScanners() { + long now = clock.milliseconds(); + + // Prune stale entries from the recently-expired cache to bound memory usage. + recentlyExpiredIds + .entrySet() + .removeIf(e -> now - e.getValue() > recentlyExpiredRetentionMs); + + for (Map.Entry entry : scanners.entrySet()) { + ScannerContext context = entry.getValue(); + if (context.isExpired(scannerTtlMs, now)) { + // Conditional remove prevents double-close if removeScanner() fires concurrently. + if (scanners.remove(entry.getKey(), context)) { + LOG.info( + "Evicted idle scanner {} for bucket {} (idle > {}ms).", + entry.getKey(), + context.getTableBucket(), + scannerTtlMs); + recentlyExpiredIds.put(entry.getKey(), now); + decrementCounts(context.getTableBucket()); + closeScannerContext(context); + } + } + } + } + + private void decrementCounts(TableBucket bucket) { + totalScanners.decrementAndGet(); + perBucketCount.computeIfPresent( + bucket, + (k, count) -> { + int remaining = count.decrementAndGet(); + return remaining <= 0 ? null : count; + }); + } + + private void closeScannerContext(ScannerContext context) { + try { + context.close(); + } catch (Exception e) { + LOG.warn( + "Error closing scanner {} for bucket {}.", + context.getId(), + context.getTableBucket(), + e); + } + } + + private static String generateScannerId() { + return UUID.randomUUID().toString().replace("-", ""); + } + + @Override + public CompletableFuture closeAsync() { + try { + close(); + return CompletableFuture.completedFuture(null); + } catch (Exception e) { + return FutureUtils.completedExceptionally(e); + } + } + + @Override + public void close() { + if (cleanupTask != null) { + cleanupTask.cancel(false); + cleanupTask = null; + } + + for (Map.Entry entry : scanners.entrySet()) { + if (scanners.remove(entry.getKey(), entry.getValue())) { + decrementCounts(entry.getValue().getTableBucket()); + closeScannerContext(entry.getValue()); + } + } + + recentlyExpiredIds.clear(); + totalScanners.set(0); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java index 2259ec4c32..abe6b8bfb5 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java @@ -61,6 +61,7 @@ import org.apache.fluss.server.kv.RemoteLogFetcher; import org.apache.fluss.server.kv.autoinc.AutoIncIDRange; import org.apache.fluss.server.kv.rocksdb.RocksDBKvBuilder; +import org.apache.fluss.server.kv.scan.ScannerManager; import org.apache.fluss.server.kv.snapshot.CompletedKvSnapshotCommitter; import org.apache.fluss.server.kv.snapshot.CompletedSnapshot; import org.apache.fluss.server.kv.snapshot.KvFileHandleAndLocalPath; @@ -211,6 +212,15 @@ public final class Replica { private volatile @Nullable CloseableRegistry closeableRegistryForKv; private @Nullable PeriodicSnapshotManager kvSnapshotManager; + /** + * Optional reference to the server-wide {@link ScannerManager}. When set, active scanner + * sessions for this bucket are closed eagerly in {@link #dropKv()} as a safety net, even on + * code paths that do not go through {@link + * org.apache.fluss.server.replica.ReplicaManager#makeFollowers} or {@link + * org.apache.fluss.server.replica.ReplicaManager#stopReplicas}. + */ + @Nullable private volatile ScannerManager scannerManager; + // ------- metrics private Counter isrShrinks; private Counter isrExpands; @@ -378,6 +388,11 @@ public Path getTabletParentDir() { return kvTablet; } + /** Injects the {@link ScannerManager} so that {@link #dropKv()} can close active scanners. */ + public void setScannerManager(@Nullable ScannerManager scannerManager) { + this.scannerManager = scannerManager; + } + public TablePath getTablePath() { return physicalPath.getTablePath(); } @@ -702,6 +717,15 @@ private void createKv() { } private void dropKv() { + // Safety net: close any lingering scanner sessions for this bucket before tearing down + // the KV tablet. The main cleanup paths (makeFollowers, stopReplica) call + // ScannerManager.closeScannersForBucket directly on ReplicaManager, but this guard + // ensures ResourceGuard leases are released even on unexpected code paths, preventing + // KvTablet.close() from blocking indefinitely on resourceGuard.close(). + ScannerManager sm = this.scannerManager; + if (sm != null) { + sm.closeScannersForBucket(tableBucket); + } // close any closeable registry for kv if (closeableRegistry.unregisterCloseable(closeableRegistryForKv)) { IOUtils.closeQuietly(closeableRegistryForKv); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java index 3b847266eb..817572381d 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java @@ -27,6 +27,7 @@ import org.apache.fluss.exception.InvalidColumnProjectionException; import org.apache.fluss.exception.InvalidCoordinatorException; import org.apache.fluss.exception.InvalidRequiredAcksException; +import org.apache.fluss.exception.InvalidTableException; import org.apache.fluss.exception.LogOffsetOutOfRangeException; import org.apache.fluss.exception.LogStorageException; import org.apache.fluss.exception.NotLeaderOrFollowerException; @@ -79,6 +80,8 @@ import org.apache.fluss.server.entity.UserContext; import org.apache.fluss.server.kv.KvManager; import org.apache.fluss.server.kv.KvSnapshotResource; +import org.apache.fluss.server.kv.KvTablet; +import org.apache.fluss.server.kv.scan.ScannerManager; import org.apache.fluss.server.kv.snapshot.CompletedKvSnapshotCommitter; import org.apache.fluss.server.kv.snapshot.DefaultSnapshotContext; import org.apache.fluss.server.log.FetchDataInfo; @@ -209,6 +212,8 @@ public class ReplicaManager implements ServerReconfigurable { private final Clock clock; + @Nullable private ScannerManager scannerManager; + public ReplicaManager( Configuration conf, Scheduler scheduler, @@ -324,6 +329,10 @@ public void startup() { conf.get(ConfigOptions.LOG_REPLICA_MAX_LAG_TIME).toMillis() / 2); } + public void setScannerManager(ScannerManager scannerManager) { + this.scannerManager = scannerManager; + } + public RemoteLogManager getRemoteLogManager() { return remoteLogManager; } @@ -1155,6 +1164,9 @@ private void makeFollowers( Replica replica = getReplicaOrException(data.getTableBucket()); if (replica.makeFollower(data)) { replicasBecomeFollower.add(replica); + if (scannerManager != null) { + scannerManager.closeScannersForBucket(tb); + } } // stop the remote log tiering tasks for followers remoteLogManager.stopLogTiering(replica); @@ -1833,6 +1845,10 @@ private StopReplicaResultForBucket stopReplica( // First stop fetchers for this table bucket. replicaFetcherManager.removeFetcherForBuckets(Collections.singleton(tb)); + if (scannerManager != null) { + scannerManager.closeScannersForBucket(tb); + } + HostedReplica replica = getReplica(tb); if (replica instanceof OnlineReplica) { Replica replicaToDelete = ((OnlineReplica) replica).getReplica(); @@ -1946,6 +1962,9 @@ protected Optional maybeCreateReplica(NotifyLeaderAndIsrData data) { tableInfo, clock, remoteLogManager); + // Inject the ScannerManager so that Replica.dropKv() can eagerly close scanner + // sessions as a safety net on unexpected shutdown paths. + replica.setScannerManager(scannerManager); allReplicas.put(tb, new OnlineReplica(replica)); replicaOpt = Optional.of(replica); } else if (hostedReplica instanceof OnlineReplica) { @@ -1978,6 +1997,33 @@ public HostedReplica getReplica(TableBucket tableBucket) { return allReplicas.getOrDefault(tableBucket, new NoneReplica()); } + /** + * Returns the {@link KvTablet} for the local leader replica of the given bucket. + * + * @throws NotLeaderOrFollowerException if this server is not the leader for the bucket + * @throws InvalidTableException if the bucket does not have KV storage (not a primary-key + * table) + * @throws UnknownTableOrBucketException if the bucket is not known to this server + */ + public KvTablet getLeaderKvTablet(TableBucket tableBucket) { + Replica replica = getReplicaOrException(tableBucket); + if (!replica.isLeader()) { + throw new NotLeaderOrFollowerException( + String.format( + "Leader not local for bucket %s on tablet server %d", + tableBucket, serverId)); + } + KvTablet kvTablet = replica.getKvTablet(); + if (kvTablet == null) { + throw new InvalidTableException( + String.format( + "Bucket %s does not have KV storage. " + + "Full KV scan is only supported on primary-key tables.", + tableBucket)); + } + return kvTablet; + } + private boolean isRequiredAcksInvalid(int requiredAcks) { return requiredAcks != 0 && requiredAcks != 1 && requiredAcks != -1; } 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 988f7565ac..247d270ba7 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 @@ -40,6 +40,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; @@ -145,6 +146,9 @@ public class TabletServer extends ServerBase { @GuardedBy("lock") private ReplicaManager replicaManager; + @GuardedBy("lock") + private ScannerManager scannerManager; + @GuardedBy("lock") private @Nullable RemoteLogManager remoteLogManager = null; @@ -281,6 +285,9 @@ protected void startServices() throws Exception { // Start dynamicConfigManager after all reconfigurable components are registered dynamicConfigManager.startup(); + this.scannerManager = new ScannerManager(conf, scheduler); + replicaManager.setScannerManager(scannerManager); + this.tabletService = new TabletService( serverId, @@ -291,7 +298,8 @@ protected void startServices() throws Exception { metadataManager, authorizer, dynamicConfigManager, - ioExecutor); + ioExecutor, + scannerManager); RequestsMetrics requestsMetrics = RequestsMetrics.createTabletServerRequestMetrics(tabletServerMetricGroup); @@ -433,6 +441,10 @@ CompletableFuture stopServices() { scheduler.shutdown(); } + 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 08536adfd9..44429d9f4e 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.InvalidScanRequestException; +import org.apache.fluss.exception.ScannerExpiredException; +import org.apache.fluss.exception.UnknownScannerIdException; 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; @@ -52,6 +56,7 @@ 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; @@ -76,6 +81,8 @@ 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.FetchParamsBuilder; import org.apache.fluss.server.log.FilterInfo; @@ -137,6 +144,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, @@ -147,7 +155,8 @@ public TabletService( MetadataManager metadataManager, @Nullable Authorizer authorizer, DynamicConfigManager dynamicConfigManager, - ExecutorService ioExecutor) { + ExecutorService ioExecutor, + ScannerManager scannerManager) { super( remoteFileSystem, ServerType.TABLET_SERVER, @@ -161,6 +170,7 @@ public TabletService( this.metadataCache = metadataCache; this.metadataFunctionProvider = new TabletServerMetadataProvider(zkClient, metadataManager, metadataCache); + this.scannerManager = scannerManager; } @Override @@ -435,7 +445,120 @@ public CompletableFuture notifyLakeTableOffset( @Override public CompletableFuture scanKv(ScanKvRequest request) { - return null; + ScanKvResponse response = new ScanKvResponse(); + try { + ScannerContext context; + + if (request.hasBucketScanReq() && request.hasScannerId()) { + throw new InvalidScanRequestException( + "ScanKvRequest must not set both bucket_scan_req and scanner_id."); + } + + if (request.hasBucketScanReq()) { + // New scan: open a fresh scanner session + PbScanReqForBucket bucketReq = request.getBucketScanReq(); + long tableId = bucketReq.getTableId(); + authorizeTable(READ, tableId); + + TableBucket tableBucket = + new TableBucket( + tableId, + bucketReq.hasPartitionId() ? bucketReq.getPartitionId() : null, + bucketReq.getBucketId()); + Long limit = bucketReq.hasLimit() ? bucketReq.getLimit() : null; + + context = + scannerManager.createScanner( + replicaManager.getLeaderKvTablet(tableBucket), tableBucket, limit); + + if (context == null) { + // Bucket is empty — return an empty response immediately without registering a + // session. + response.setHasMoreResults(false); + return CompletableFuture.completedFuture(response); + } + } else { + if (!request.hasScannerId()) { + throw new InvalidScanRequestException( + "ScanKvRequest must have either bucket_scan_req (new scan) " + + "or scanner_id (continuation)."); + } + byte[] scannerId = request.getScannerId(); + context = scannerManager.getScanner(scannerId); + if (context == null) { + if (scannerManager.isRecentlyExpired(scannerId)) { + throw new ScannerExpiredException( + "Scanner session has expired due to inactivity. " + + "Please start a new scan."); + } else { + throw new UnknownScannerIdException( + "Unknown scanner ID. The session may have expired or " + + "never existed."); + } + } + // Validate call-sequence ordering to detect duplicate or out-of-order requests. + // getScanner() already refreshed the last-access timestamp. + if (request.hasCallSeqId()) { + int expectedSeqId = context.getCallSeqId() + 1; + int requestSeqId = request.getCallSeqId(); + if (requestSeqId != expectedSeqId) { + throw new InvalidScanRequestException( + String.format( + "Out-of-order scan request: expected callSeqId=%d but got %d.", + expectedSeqId, requestSeqId)); + } + } + } + + // Handle explicit close request + if (request.hasCloseScanner() && request.isCloseScanner()) { + scannerManager.removeScanner(context); + response.setScannerId(context.getScannerId()); + response.setHasMoreResults(false); + return CompletableFuture.completedFuture(response); + } + + // Build the next batch + int batchSizeBytes = request.getBatchSizeBytes(); + if (batchSizeBytes <= 0) { + throw new InvalidScanRequestException("batch_size_bytes must be greater than 0."); + } + DefaultValueRecordBatch.Builder builder = DefaultValueRecordBatch.builder(); + int totalBytes = 0; + + while (context.isValid() && totalBytes < batchSizeBytes) { + byte[] value = context.currentValue(); + builder.append(value); + totalBytes += value.length; + context.advance(); + } + + boolean hasMore = context.isValid(); + DefaultValueRecordBatch batch = builder.build(); + + response.setScannerId(context.getScannerId()); + response.setHasMoreResults(hasMore); + if (batch.sizeInBytes() > 0) { + response.setRecords(batch.getSegment(), batch.getPosition(), batch.sizeInBytes()); + } + + // Update callSeqId AFTER the response is prepared so that a client retry with the + // same callSeqId (due to a transient failure) can be detected and rejected. + if (request.hasCallSeqId()) { + context.setCallSeqId(request.getCallSeqId()); + } + + // Auto-close the session when all data has been drained. + if (!hasMore) { + scannerManager.removeScanner(context); + } + + } catch (Exception e) { + response.setErrorCode(Errors.forException(e).code()); + response.setErrorMessage(e.getMessage() != null ? e.getMessage() : ""); + } + + return CompletableFuture.completedFuture(response); } @Override diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/KvTabletTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/KvTabletTest.java index fd08eeb4aa..652b5d1239 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/kv/KvTabletTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/KvTabletTest.java @@ -57,6 +57,7 @@ import org.apache.fluss.server.kv.prewrite.KvPreWriteBuffer.Value; import org.apache.fluss.server.kv.rocksdb.RocksDBStatistics; import org.apache.fluss.server.kv.rowmerger.RowMerger; +import org.apache.fluss.server.kv.scan.ScannerContext; import org.apache.fluss.server.log.FetchIsolation; import org.apache.fluss.server.log.LogAppendInfo; import org.apache.fluss.server.log.LogTablet; @@ -1857,4 +1858,142 @@ void testRowCountWithMixedOperations() throws Exception { kvTablet.close(); } + + @Test + void testOpenScan_emptyBucket_returnsNull() throws Exception { + initLogTabletAndKvTablet(DATA1_SCHEMA_PK, new HashMap<>()); + // No data has been written — openScan must return null. + ScannerContext context = kvTablet.openScan("scanner-empty", -1L, 0L); + assertThat(context).isNull(); + } + + @Test + void testOpenScan_returnsAllRows() throws Exception { + initLogTabletAndKvTablet(DATA1_SCHEMA_PK, new HashMap<>()); + + int numRows = 5; + List rows = new ArrayList<>(); + for (int i = 0; i < numRows; i++) { + rows.add( + kvRecordFactory.ofRecord( + String.valueOf(i).getBytes(), new Object[] {i, "v" + i})); + } + kvTablet.putAsLeader(kvRecordBatchFactory.ofRecords(rows), null); + kvTablet.flush(Long.MAX_VALUE, NOPErrorHandler.INSTANCE); + + ScannerContext context = kvTablet.openScan("scanner-all", -1L, 0L); + assertThat(context).isNotNull(); + + int count = 0; + while (context.isValid()) { + assertThat(context.currentValue()).isNotNull(); + context.advance(); + count++; + } + context.close(); + + assertThat(count).isEqualTo(numRows); + } + + @Test + void testOpenScan_snapshotIsolation() throws Exception { + initLogTabletAndKvTablet(DATA1_SCHEMA_PK, new HashMap<>()); + + // Write and flush 3 rows before opening the scan. + List initialRows = + Arrays.asList( + kvRecordFactory.ofRecord("0".getBytes(), new Object[] {0, "v0"}), + kvRecordFactory.ofRecord("1".getBytes(), new Object[] {1, "v1"}), + kvRecordFactory.ofRecord("2".getBytes(), new Object[] {2, "v2"})); + kvTablet.putAsLeader(kvRecordBatchFactory.ofRecords(initialRows), null); + kvTablet.flush(Long.MAX_VALUE, NOPErrorHandler.INSTANCE); + + ScannerContext context = kvTablet.openScan("scanner-snap", -1L, 0L); + assertThat(context).isNotNull(); + + // Write 2 more rows AFTER opening the scan, then flush. + List lateRows = + Arrays.asList( + kvRecordFactory.ofRecord("3".getBytes(), new Object[] {3, "v3"}), + kvRecordFactory.ofRecord("4".getBytes(), new Object[] {4, "v4"})); + kvTablet.putAsLeader(kvRecordBatchFactory.ofRecords(lateRows), null); + kvTablet.flush(Long.MAX_VALUE, NOPErrorHandler.INSTANCE); + + // The scan must still see only the 3 rows that existed at snapshot time. + int count = 0; + while (context.isValid()) { + context.advance(); + count++; + } + context.close(); + + assertThat(count).isEqualTo(3); + } + + @Test + void testOpenScan_withLimit() throws Exception { + initLogTabletAndKvTablet(DATA1_SCHEMA_PK, new HashMap<>()); + + int numRows = 5; + List rows = new ArrayList<>(); + for (int i = 0; i < numRows; i++) { + rows.add( + kvRecordFactory.ofRecord( + String.valueOf(i).getBytes(), new Object[] {i, "v" + i})); + } + kvTablet.putAsLeader(kvRecordBatchFactory.ofRecords(rows), null); + kvTablet.flush(Long.MAX_VALUE, NOPErrorHandler.INSTANCE); + + long limit = 3L; + ScannerContext context = kvTablet.openScan("scanner-limit", limit, 0L); + assertThat(context).isNotNull(); + + int count = 0; + while (context.isValid()) { + context.advance(); + count++; + } + context.close(); + + // The scan must stop after exactly `limit` rows. + assertThat(count).isEqualTo((int) limit); + } + + @Test + void testOpenScan_multipleSessionsIndependent() throws Exception { + initLogTabletAndKvTablet(DATA1_SCHEMA_PK, new HashMap<>()); + + List rows = + Arrays.asList( + kvRecordFactory.ofRecord("0".getBytes(), new Object[] {0, "v0"}), + kvRecordFactory.ofRecord("1".getBytes(), new Object[] {1, "v1"}), + kvRecordFactory.ofRecord("2".getBytes(), new Object[] {2, "v2"})); + kvTablet.putAsLeader(kvRecordBatchFactory.ofRecords(rows), null); + kvTablet.flush(Long.MAX_VALUE, NOPErrorHandler.INSTANCE); + + // Open two independent scans. + ScannerContext ctx1 = kvTablet.openScan("scanner-a", -1L, 0L); + ScannerContext ctx2 = kvTablet.openScan("scanner-b", -1L, 0L); + assertThat(ctx1).isNotNull(); + assertThat(ctx2).isNotNull(); + + // Drain ctx1 fully. + int count1 = 0; + while (ctx1.isValid()) { + ctx1.advance(); + count1++; + } + ctx1.close(); + + // ctx2 cursor must be unaffected; it should still see all 3 rows. + int count2 = 0; + while (ctx2.isValid()) { + ctx2.advance(); + count2++; + } + ctx2.close(); + + assertThat(count1).isEqualTo(3); + assertThat(count2).isEqualTo(3); + } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java new file mode 100644 index 0000000000..7cd18d2ddb --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java @@ -0,0 +1,359 @@ +/* + * 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.config.TableConfig; +import org.apache.fluss.exception.TooManyScannersException; +import org.apache.fluss.memory.TestingMemorySegmentPool; +import org.apache.fluss.metadata.KvFormat; +import org.apache.fluss.metadata.LogFormat; +import org.apache.fluss.metadata.PhysicalTablePath; +import org.apache.fluss.metadata.SchemaInfo; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.record.KvRecord; +import org.apache.fluss.record.KvRecordTestUtils; +import org.apache.fluss.record.TestData; +import org.apache.fluss.record.TestingSchemaGetter; +import org.apache.fluss.server.kv.KvManager; +import org.apache.fluss.server.kv.KvTablet; +import org.apache.fluss.server.kv.autoinc.AutoIncrementManager; +import org.apache.fluss.server.kv.autoinc.TestingSequenceGeneratorFactory; +import org.apache.fluss.server.kv.rowmerger.RowMerger; +import org.apache.fluss.server.log.LogTablet; +import org.apache.fluss.server.log.LogTestUtils; +import org.apache.fluss.server.metrics.group.TestingMetricGroups; +import org.apache.fluss.server.zk.NOPErrorHandler; +import org.apache.fluss.shaded.arrow.org.apache.arrow.memory.RootAllocator; +import org.apache.fluss.utils.clock.ManualClock; +import org.apache.fluss.utils.concurrent.FlussScheduler; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import static org.apache.fluss.compression.ArrowCompressionInfo.DEFAULT_COMPRESSION; +import static org.apache.fluss.record.TestData.DATA1_SCHEMA_PK; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Tests for {@link ScannerManager}. */ +class ScannerManagerTest { + + private static final short SCHEMA_ID = 1; + + @TempDir File tempLogDir; + @TempDir File tmpKvDir; + + private final Configuration conf = new Configuration(); + private final KvRecordTestUtils.KvRecordBatchFactory kvRecordBatchFactory = + KvRecordTestUtils.KvRecordBatchFactory.of(SCHEMA_ID); + private final KvRecordTestUtils.KvRecordFactory kvRecordFactory = + KvRecordTestUtils.KvRecordFactory.of(TestData.DATA1_ROW_TYPE); + + private ManualClock clock; + private FlussScheduler scheduler; + private LogTablet logTablet; + private KvTablet kvTablet; + + @BeforeEach + void setUp() throws Exception { + clock = new ManualClock(0); + scheduler = new FlussScheduler(1); + scheduler.startup(); + + PhysicalTablePath physicalTablePath = PhysicalTablePath.of(TablePath.of("testDb", "t1")); + TestingSchemaGetter schemaGetter = + new TestingSchemaGetter(new SchemaInfo(DATA1_SCHEMA_PK, SCHEMA_ID)); + + File logTabletDir = + LogTestUtils.makeRandomLogTabletDir( + tempLogDir, + physicalTablePath.getDatabaseName(), + 0L, + physicalTablePath.getTableName()); + logTablet = + LogTablet.create( + physicalTablePath, + logTabletDir, + conf, + TestingMetricGroups.TABLET_SERVER_METRICS, + 0, + new FlussScheduler(1), + LogFormat.ARROW, + 1, + true, + org.apache.fluss.utils.clock.SystemClock.getInstance(), + true); + + TableBucket tableBucket = logTablet.getTableBucket(); + TableConfig tableConf = new TableConfig(new Configuration()); + RowMerger rowMerger = RowMerger.create(tableConf, KvFormat.COMPACTED, schemaGetter); + AutoIncrementManager autoIncrementManager = + new AutoIncrementManager( + schemaGetter, + physicalTablePath.getTablePath(), + tableConf, + new TestingSequenceGeneratorFactory()); + + kvTablet = + KvTablet.create( + physicalTablePath, + tableBucket, + logTablet, + tmpKvDir, + conf, + TestingMetricGroups.TABLET_SERVER_METRICS, + new RootAllocator(Long.MAX_VALUE), + new TestingMemorySegmentPool(10 * 1024), + KvFormat.COMPACTED, + rowMerger, + DEFAULT_COMPRESSION, + schemaGetter, + tableConf.getChangelogImage(), + KvManager.getDefaultRateLimiter(), + autoIncrementManager); + } + + @AfterEach + void tearDown() throws Exception { + if (kvTablet != null) { + kvTablet.close(); + } + if (logTablet != null) { + logTablet.close(); + } + if (scheduler != null) { + scheduler.shutdown(); + } + } + + // ------------------------------------------------------------------------- + // Helpers + // ------------------------------------------------------------------------- + + /** Creates a {@link ScannerManager} with a long TTL so the reaper never fires during tests. */ + private ScannerManager createManager() { + Configuration c = new Configuration(); + c.set(ConfigOptions.KV_SCANNER_TTL, Duration.ofHours(1)); + c.set(ConfigOptions.KV_SCANNER_EXPIRATION_INTERVAL, Duration.ofHours(1)); + c.set(ConfigOptions.KV_SCANNER_MAX_PER_BUCKET, 8); + c.set(ConfigOptions.KV_SCANNER_MAX_PER_SERVER, 200); + return new ScannerManager(c, scheduler, clock); + } + + /** Creates a {@link ScannerManager} with configurable limits and a long reaper interval. */ + private ScannerManager createManager(int maxPerBucket, int maxPerServer) { + Configuration c = new Configuration(); + c.set(ConfigOptions.KV_SCANNER_TTL, Duration.ofHours(1)); + c.set(ConfigOptions.KV_SCANNER_EXPIRATION_INTERVAL, Duration.ofHours(1)); + c.set(ConfigOptions.KV_SCANNER_MAX_PER_BUCKET, maxPerBucket); + c.set(ConfigOptions.KV_SCANNER_MAX_PER_SERVER, maxPerServer); + return new ScannerManager(c, scheduler, clock); + } + + /** Creates a {@link ScannerManager} with a short TTL and reaper interval for eviction tests. */ + private ScannerManager createManagerWithShortTtl(long ttlMs, long expirationIntervalMs) { + Configuration c = new Configuration(); + c.set(ConfigOptions.KV_SCANNER_TTL, Duration.ofMillis(ttlMs)); + c.set( + ConfigOptions.KV_SCANNER_EXPIRATION_INTERVAL, + Duration.ofMillis(expirationIntervalMs)); + c.set(ConfigOptions.KV_SCANNER_MAX_PER_BUCKET, 8); + c.set(ConfigOptions.KV_SCANNER_MAX_PER_SERVER, 200); + return new ScannerManager(c, scheduler, clock); + } + + /** Writes {@code count} rows into the KvTablet and flushes to RocksDB. */ + private void putAndFlush(int count) throws Exception { + List rows = new ArrayList<>(); + for (int i = 0; i < count; i++) { + rows.add( + kvRecordFactory.ofRecord( + String.valueOf(i).getBytes(), new Object[] {i, "v" + i})); + } + kvTablet.putAsLeader(kvRecordBatchFactory.ofRecords(rows), null); + kvTablet.flush(Long.MAX_VALUE, NOPErrorHandler.INSTANCE); + } + + // ------------------------------------------------------------------------- + // Tests + // ------------------------------------------------------------------------- + + @Test + void testCreateScanner_emptyBucket_returnsNull() throws Exception { + try (ScannerManager manager = createManager()) { + TableBucket tableBucket = kvTablet.getTableBucket(); + // Bucket has no data — openScan must return null; no slot consumed. + ScannerContext context = manager.createScanner(kvTablet, tableBucket, null); + assertThat(context).isNull(); + assertThat(manager.activeScannerCount()).isEqualTo(0); + } + } + + @Test + void testCreateAndRemoveScanner() throws Exception { + putAndFlush(3); + try (ScannerManager manager = createManager()) { + TableBucket tableBucket = kvTablet.getTableBucket(); + + ScannerContext context = manager.createScanner(kvTablet, tableBucket, null); + assertThat(context).isNotNull(); + assertThat(manager.activeScannerCount()).isEqualTo(1); + assertThat(manager.activeScannerCountForBucket(tableBucket)).isEqualTo(1); + + manager.removeScanner(context); + assertThat(manager.activeScannerCount()).isEqualTo(0); + assertThat(manager.activeScannerCountForBucket(tableBucket)).isEqualTo(0); + } + } + + @Test + void testGetScanner_refreshesLastAccessTime() throws Exception { + putAndFlush(3); + try (ScannerManager manager = createManager()) { + TableBucket tableBucket = kvTablet.getTableBucket(); + + // Create scanner at t=0. + ScannerContext context = manager.createScanner(kvTablet, tableBucket, null); + assertThat(context).isNotNull(); + byte[] scannerId = context.getScannerId(); + + // Advance clock far past any TTL, then getScanner to refresh. + clock.advanceTime(5000, TimeUnit.MILLISECONDS); + ScannerContext fetched = manager.getScanner(scannerId); + assertThat(fetched).isSameAs(context); + + // With a 1-hour TTL, isExpired must be false right after the refresh. + assertThat(context.isExpired(3_600_000L, clock.milliseconds())).isFalse(); + + manager.removeScanner(context); + } + } + + @Test + void testTtlEviction() throws Exception { + putAndFlush(3); + // TTL = 200 ms, reaper every 200 ms — wide enough for slow CI schedulers. + ScannerManager manager = createManagerWithShortTtl(200, 200); + try { + TableBucket tableBucket = kvTablet.getTableBucket(); + + ScannerContext context = manager.createScanner(kvTablet, tableBucket, null); + assertThat(context).isNotNull(); + byte[] scannerId = context.getScannerId(); + + // Advance ManualClock past TTL so the reaper considers the session idle. + clock.advanceTime(500, TimeUnit.MILLISECONDS); + + // Wait for the real scheduler to invoke the cleanup task. + long deadline = System.currentTimeMillis() + 10_000; + while (manager.activeScannerCount() > 0 && System.currentTimeMillis() < deadline) { + Thread.sleep(50); + } + + assertThat(manager.activeScannerCount()).isEqualTo(0); + assertThat(manager.getScanner(scannerId)).isNull(); + assertThat(manager.isRecentlyExpired(scannerId)).isTrue(); + } finally { + manager.close(); + } + } + + @Test + void testPerBucketLimit() throws Exception { + putAndFlush(3); + try (ScannerManager manager = createManager(2, 200)) { + TableBucket tableBucket = kvTablet.getTableBucket(); + + ScannerContext ctx1 = manager.createScanner(kvTablet, tableBucket, null); + ScannerContext ctx2 = manager.createScanner(kvTablet, tableBucket, null); + assertThat(manager.activeScannerCountForBucket(tableBucket)).isEqualTo(2); + + assertThatThrownBy(() -> manager.createScanner(kvTablet, tableBucket, null)) + .isInstanceOf(TooManyScannersException.class); + + // Count must not have changed after the failed attempt. + assertThat(manager.activeScannerCountForBucket(tableBucket)).isEqualTo(2); + + manager.removeScanner(ctx1); + manager.removeScanner(ctx2); + } + } + + @Test + void testPerServerLimit() throws Exception { + putAndFlush(3); + try (ScannerManager manager = createManager(8, 2)) { + TableBucket tableBucket = kvTablet.getTableBucket(); + + ScannerContext ctx1 = manager.createScanner(kvTablet, tableBucket, null); + ScannerContext ctx2 = manager.createScanner(kvTablet, tableBucket, null); + assertThat(manager.activeScannerCount()).isEqualTo(2); + + assertThatThrownBy(() -> manager.createScanner(kvTablet, tableBucket, null)) + .isInstanceOf(TooManyScannersException.class); + + assertThat(manager.activeScannerCount()).isEqualTo(2); + + manager.removeScanner(ctx1); + manager.removeScanner(ctx2); + } + } + + @Test + void testCloseScannersForBucket() throws Exception { + putAndFlush(3); + try (ScannerManager manager = createManager()) { + TableBucket tableBucket = kvTablet.getTableBucket(); + + manager.createScanner(kvTablet, tableBucket, null); + manager.createScanner(kvTablet, tableBucket, null); + assertThat(manager.activeScannerCount()).isEqualTo(2); + + manager.closeScannersForBucket(tableBucket); + + assertThat(manager.activeScannerCount()).isEqualTo(0); + assertThat(manager.activeScannerCountForBucket(tableBucket)).isEqualTo(0); + } + } + + @Test + void testShutdown_closesAllScanners() throws Exception { + putAndFlush(3); + ScannerManager manager = createManager(); + TableBucket tableBucket = kvTablet.getTableBucket(); + + manager.createScanner(kvTablet, tableBucket, null); + manager.createScanner(kvTablet, tableBucket, null); + assertThat(manager.activeScannerCount()).isEqualTo(2); + + manager.close(); + + assertThat(manager.activeScannerCount()).isEqualTo(0); + } +} From e289f1a68dbc498465a4d7bf7b107b32353bac72 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Tue, 21 Apr 2026 12:31:00 +0300 Subject: [PATCH 03/17] add documentation for configs --- website/docs/maintenance/configuration.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/website/docs/maintenance/configuration.md b/website/docs/maintenance/configuration.md index 48fc393b80..3c5d45e098 100644 --- a/website/docs/maintenance/configuration.md +++ b/website/docs/maintenance/configuration.md @@ -171,6 +171,10 @@ during the Fluss cluster working. | kv.rocksdb.bloom-filter.block-based-mode | Boolean | false | If true, RocksDB will use block-based filter instead of full filter, this only take effect when bloom filter is used. The default value is `false`. | | kv.rocksdb.shared-rate-limiter-bytes-per-sec | MemorySize | Long.MAX_VALUE | The bytes per second rate limit for RocksDB flush and compaction operations shared across all RocksDB instances on the TabletServer. The rate limiter is always enabled. The default value is Long.MAX_VALUE (effectively unlimited). Set to a lower value (e.g., 100MB) to limit the rate. This configuration can be updated dynamically without server restart. See [Updating Configs](operations/updating-configs.md) for more details. | | kv.recover.log-record-batch.max-size | MemorySize | 16mb | The max fetch size for fetching log to apply to kv during recovering kv. | +| kv.scanner.ttl | Duration | 10min | The time-to-live for an idle KV scanner session on the server. A scanner that has not received a request within this duration will be automatically expired and its resources released. The default value is 10 minutes. | +| kv.scanner.expiration-interval | Duration | 30s | The interval at which the server checks for and removes expired KV scanner sessions. The default value is 30 seconds. | +| kv.scanner.max-per-bucket | Integer | 8 | The maximum number of concurrent KV scanner sessions allowed per bucket. New scan requests that exceed this limit will be rejected with an error. The default value is 8. | +| kv.scanner.max-per-server | Integer | 200 | The maximum total number of concurrent KV scanner sessions allowed across all buckets on a single tablet server. New scan requests that exceed this limit will be rejected with an error. The default value is 200. | ## Metrics From 62c82a2a833ce98ff310b32afdc514346b491867 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Tue, 21 Apr 2026 13:10:02 +0300 Subject: [PATCH 04/17] make some improvements --- .../apache/fluss/config/ConfigOptions.java | 2 +- .../fluss/server/kv/scan/ScannerContext.java | 7 ++- .../fluss/server/kv/scan/ScannerManager.java | 40 +++++++------ .../fluss/server/replica/ReplicaManager.java | 6 ++ .../fluss/server/tablet/TabletService.java | 32 ++++++++++- .../server/kv/scan/ScannerManagerTest.java | 12 ++-- .../server/replica/ReplicaManagerTest.java | 57 +++++++++++++++++++ 7 files changed, 128 insertions(+), 28 deletions(-) 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 4e3713f0c3..923c74ccd8 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 @@ -528,7 +528,7 @@ public class ConfigOptions { .durationType() .defaultValue(Duration.ofSeconds(30)) .withDescription( - "How often the TTL reaper runs to close idle scanner sessions. " + "How often the TTL evictor runs to close idle scanner sessions. " + "The default value is 30 seconds."); public static final ConfigOption KV_SCANNER_MAX_PER_BUCKET = 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 baa0a90426..ba0c91fe66 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 @@ -97,7 +97,12 @@ public byte[] getScannerId() { return scannerIdBytes; } - String getId() { + /** + * Returns the scanner ID as a UTF-8 {@link String}. Package-private: used by {@link + * ScannerManager} as the key in its internal {@code scanners} map. The wire-format + * representation is always {@link #getScannerId()} (raw bytes). + */ + String getIdString() { return scannerId; } 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 aee3d641f0..223226e42b 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 @@ -71,9 +71,9 @@ * *

TTL eviction

* - *

A background reaper task runs every {@code kv.scanner.expiration-interval} and evicts sessions - * idle longer than {@code kv.scanner.ttl}. Recently evicted IDs are retained for {@code 2 × ttl} so - * callers can distinguish "expired" from "never existed." + *

A background evictor task runs every {@code kv.scanner.expiration-interval} and removes + * sessions idle longer than {@code kv.scanner.ttl}. Recently evicted IDs are retained for {@code 2 + * × ttl} so callers can distinguish "expired" from "never existed." * *

Leadership change

* @@ -99,7 +99,7 @@ public class ScannerManager implements AutoCloseableAsync { private final int maxPerBucket; private final int maxPerServer; - @Nullable private ScheduledFuture cleanupTask; + @Nullable private ScheduledFuture evictorTask; public ScannerManager(Configuration conf, Scheduler scheduler) { this(conf, scheduler, SystemClock.getInstance()); @@ -115,10 +115,10 @@ public ScannerManager(Configuration conf, Scheduler scheduler) { long expirationIntervalMs = conf.get(ConfigOptions.KV_SCANNER_EXPIRATION_INTERVAL).toMillis(); - this.cleanupTask = + this.evictorTask = scheduler.schedule( "scanner-expiration", - this::cleanupExpiredScanners, + this::evictExpiredScanners, expirationIntervalMs, expirationIntervalMs); @@ -192,7 +192,7 @@ public ScannerContext getScanner(byte[] scannerId) { /** * Returns {@code true} if the given scanner ID belongs to a session that was recently evicted - * by the TTL reaper (within the last {@code 2 × ttlMs}). + * by the TTL evictor (within the last {@code 2 × ttlMs}). * *

Callers can use this to distinguish "scanner expired" from "unknown scanner ID." */ @@ -204,13 +204,13 @@ public boolean isRecentlyExpired(byte[] scannerId) { * Removes and closes a known scanner context directly, avoiding a map lookup. * *

Uses a conditional remove ({@link java.util.concurrent.ConcurrentHashMap#remove(Object, - * Object)}) so that concurrent calls — e.g. from the TTL reaper and a close-scanner RPC + * Object)}) so that concurrent calls — e.g. from the TTL evictor and a close-scanner RPC * arriving simultaneously — result in exactly one winner closing the context, preventing * double-release of the non-idempotent {@link * org.apache.fluss.server.utils.ResourceGuard.Lease}. */ public void removeScanner(ScannerContext context) { - if (scanners.remove(context.getId(), context)) { + if (scanners.remove(context.getIdString(), context)) { decrementCounts(context.getTableBucket()); closeScannerContext(context); } @@ -221,7 +221,7 @@ public void removeScanner(ScannerContext context) { * *

Delegates to {@link #removeScanner(ScannerContext)} to ensure a conditional {@link * java.util.concurrent.ConcurrentHashMap#remove(Object, Object)} is used, which prevents a - * double-decrement of {@code perBucketCount} when the TTL reaper races with an explicit close + * double-decrement of {@code perBucketCount} when the TTL evictor races with an explicit close * request for the same scanner. * *

No-op if the ID is not found (already removed or expired). @@ -320,18 +320,18 @@ private void registerContext(ScannerContext context, TableBucket tableBucket) { tableBucket, maxPerBucket)); } - scanners.put(context.getId(), context); + scanners.put(context.getIdString(), context); LOG.debug( "Registered scanner {} for bucket {} (total={}, perBucket={})", - context.getId(), + context.getIdString(), tableBucket, newTotal, newBucketCount); } - /** TTL reaper — invoked periodically by the background scheduler. */ - private void cleanupExpiredScanners() { + /** TTL evictor — invoked periodically by the background scheduler. */ + private void evictExpiredScanners() { long now = clock.milliseconds(); // Prune stale entries from the recently-expired cache to bound memory usage. @@ -373,7 +373,7 @@ private void closeScannerContext(ScannerContext context) { } catch (Exception e) { LOG.warn( "Error closing scanner {} for bucket {}.", - context.getId(), + context.getIdString(), context.getTableBucket(), e); } @@ -395,9 +395,13 @@ public CompletableFuture closeAsync() { @Override public void close() { - if (cleanupTask != null) { - cleanupTask.cancel(false); - cleanupTask = null; + // Note: we cancel but do not join the evictor. The evictor may still be mid-iteration + // when close() begins. This is safe because (a) scanners is a ConcurrentHashMap, and + // (b) both shutdown and the evictor use conditional remove(key, value) to mutate it, + // so at most one side ever closes a given ScannerContext. + if (evictorTask != null) { + evictorTask.cancel(false); + evictorTask = null; } for (Map.Entry entry : scanners.entrySet()) { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java index 817572381d..b34cea21f9 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java @@ -1845,6 +1845,12 @@ private StopReplicaResultForBucket stopReplica( // First stop fetchers for this table bucket. replicaFetcherManager.removeFetcherForBuckets(Collections.singleton(tb)); + // Close active scanner sessions for this bucket before tearing down the KV tablet. + // A concurrent scanKv RPC in flight can still reach getLeaderKvTablet(tb) and race + // with createScanner. Both KvTablet.openScan and ResourceGuard.acquireResource() + // synchronise with the subsequent replica/RocksDB close, so the worst case is an + // IOException surfaced cleanly as an RPC error — never cursor corruption or a + // post-shutdown snapshot. if (scannerManager != null) { scannerManager.closeScannersForBucket(tb); } 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 44429d9f4e..62222c0bf5 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 @@ -446,6 +446,12 @@ public CompletableFuture notifyLakeTableOffset( @Override public CompletableFuture scanKv(ScanKvRequest request) { ScanKvResponse response = new ScanKvResponse(); + // Tracks a live session that must be force-closed if anything after it throws. + // A scan that produced a partial but un-returned batch is not resumable: the RocksDB + // cursor has already advanced past rows whose data never reached the client, so we + // tear the session down and force the client to restart. This matches the non-resumable + // contract documented on ScannerExpiredException. + ScannerContext openedContext = null; try { ScannerContext context; @@ -477,6 +483,7 @@ public CompletableFuture scanKv(ScanKvRequest request) { response.setHasMoreResults(false); return CompletableFuture.completedFuture(response); } + openedContext = context; } else { if (!request.hasScannerId()) { throw new InvalidScanRequestException( @@ -497,22 +504,25 @@ public CompletableFuture scanKv(ScanKvRequest request) { } } // Validate call-sequence ordering to detect duplicate or out-of-order requests. - // getScanner() already refreshed the last-access timestamp. + // getScanner() already refreshed the last-access timestamp. Use long arithmetic to + // avoid a silent 32-bit overflow at Integer.MAX_VALUE continuations. if (request.hasCallSeqId()) { - int expectedSeqId = context.getCallSeqId() + 1; + long expectedSeqId = (long) context.getCallSeqId() + 1L; int requestSeqId = request.getCallSeqId(); - if (requestSeqId != expectedSeqId) { + if ((long) requestSeqId != expectedSeqId) { throw new InvalidScanRequestException( String.format( "Out-of-order scan request: expected callSeqId=%d but got %d.", expectedSeqId, requestSeqId)); } } + openedContext = context; } // Handle explicit close request if (request.hasCloseScanner() && request.isCloseScanner()) { scannerManager.removeScanner(context); + openedContext = null; response.setScannerId(context.getScannerId()); response.setHasMoreResults(false); return CompletableFuture.completedFuture(response); @@ -552,10 +562,26 @@ public CompletableFuture scanKv(ScanKvRequest request) { if (!hasMore) { scannerManager.removeScanner(context); } + // Response successfully prepared — session state is consistent; do not force-close. + openedContext = null; } catch (Exception e) { + // Restore the interrupt flag if a lower-level call wrapped an InterruptedException. + // No method in the try block declares `throws InterruptedException` directly, but a + // future refactor or a rethrown wrapper should not silently lose the signal. + if (e instanceof InterruptedException || e.getCause() instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } response.setErrorCode(Errors.forException(e).code()); response.setErrorMessage(e.getMessage() != null ? e.getMessage() : ""); + } finally { + // If we made it past createScanner/getScanner but failed to deliver a complete + // response, close the session rather than leaking it to TTL. The cursor has + // already advanced past rows whose values were never sent; resuming would drop + // data. Forcing a restart is the safe option. + if (openedContext != null) { + scannerManager.removeScanner(openedContext); + } } return CompletableFuture.completedFuture(response); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java index 7cd18d2ddb..b054d30960 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java @@ -156,7 +156,7 @@ void tearDown() throws Exception { // Helpers // ------------------------------------------------------------------------- - /** Creates a {@link ScannerManager} with a long TTL so the reaper never fires during tests. */ + /** Creates a {@link ScannerManager} with a long TTL so the evictor never fires during tests. */ private ScannerManager createManager() { Configuration c = new Configuration(); c.set(ConfigOptions.KV_SCANNER_TTL, Duration.ofHours(1)); @@ -166,7 +166,7 @@ private ScannerManager createManager() { return new ScannerManager(c, scheduler, clock); } - /** Creates a {@link ScannerManager} with configurable limits and a long reaper interval. */ + /** Creates a {@link ScannerManager} with configurable limits and a long evictor interval. */ private ScannerManager createManager(int maxPerBucket, int maxPerServer) { Configuration c = new Configuration(); c.set(ConfigOptions.KV_SCANNER_TTL, Duration.ofHours(1)); @@ -176,7 +176,9 @@ private ScannerManager createManager(int maxPerBucket, int maxPerServer) { return new ScannerManager(c, scheduler, clock); } - /** Creates a {@link ScannerManager} with a short TTL and reaper interval for eviction tests. */ + /** + * Creates a {@link ScannerManager} with a short TTL and evictor interval for eviction tests. + */ private ScannerManager createManagerWithShortTtl(long ttlMs, long expirationIntervalMs) { Configuration c = new Configuration(); c.set(ConfigOptions.KV_SCANNER_TTL, Duration.ofMillis(ttlMs)); @@ -258,7 +260,7 @@ void testGetScanner_refreshesLastAccessTime() throws Exception { @Test void testTtlEviction() throws Exception { putAndFlush(3); - // TTL = 200 ms, reaper every 200 ms — wide enough for slow CI schedulers. + // TTL = 200 ms, evictor every 200 ms — wide enough for slow CI schedulers. ScannerManager manager = createManagerWithShortTtl(200, 200); try { TableBucket tableBucket = kvTablet.getTableBucket(); @@ -267,7 +269,7 @@ void testTtlEviction() throws Exception { assertThat(context).isNotNull(); byte[] scannerId = context.getScannerId(); - // Advance ManualClock past TTL so the reaper considers the session idle. + // Advance ManualClock past TTL so the evictor considers the session idle. clock.advanceTime(500, TimeUnit.MILLISECONDS); // Wait for the real scheduler to invoke the cleanup task. diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java index 5120bf6a6d..3eec7ac61d 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java @@ -37,6 +37,7 @@ import org.apache.fluss.record.DefaultValueRecordBatch; import org.apache.fluss.record.KvRecord; import org.apache.fluss.record.KvRecordBatch; +import org.apache.fluss.record.KvRecordTestUtils; import org.apache.fluss.record.LogRecord; import org.apache.fluss.record.LogRecordBatch; import org.apache.fluss.record.LogRecordReadContext; @@ -62,7 +63,9 @@ import org.apache.fluss.server.entity.NotifyLeaderAndIsrResultForBucket; import org.apache.fluss.server.entity.StopReplicaData; import org.apache.fluss.server.entity.StopReplicaResultForBucket; +import org.apache.fluss.server.kv.KvTablet; import org.apache.fluss.server.kv.rocksdb.RocksDBKv; +import org.apache.fluss.server.kv.scan.ScannerManager; import org.apache.fluss.server.kv.snapshot.CompletedSnapshot; import org.apache.fluss.server.log.FetchParams; import org.apache.fluss.server.log.ListOffsetsParam; @@ -72,6 +75,7 @@ import org.apache.fluss.server.metadata.ServerInfo; import org.apache.fluss.server.metadata.TableMetadata; import org.apache.fluss.server.testutils.KvTestUtils; +import org.apache.fluss.server.zk.NOPErrorHandler; import org.apache.fluss.server.zk.data.LeaderAndIsr; import org.apache.fluss.server.zk.data.TableRegistration; import org.apache.fluss.testutils.DataTestUtils; @@ -79,6 +83,7 @@ import org.apache.fluss.types.DataTypes; import org.apache.fluss.types.RowType; import org.apache.fluss.utils.CloseableIterator; +import org.apache.fluss.utils.concurrent.FlussScheduler; import org.apache.fluss.utils.types.Tuple2; import org.junit.jupiter.api.Test; @@ -2374,4 +2379,56 @@ private void assertUpdateMetadataEquals( } }); } + + /** + * When a replica is stopped, {@link ScannerManager#closeScannersForBucket} must be called so + * that open scanner sessions are released before the KV store is destroyed. + */ + @Test + void testStopReplicas_closesScanners() throws Exception { + TableBucket tb = new TableBucket(DATA1_TABLE_ID_PK, 0); + makeKvTableAsLeader(DATA1_TABLE_ID_PK, DATA1_TABLE_PATH_PK, tb.getBucket()); + + KvTablet kvTablet = + kvManager + .getKv(tb) + .orElseThrow(() -> new IllegalStateException("KvTablet not found")); + KvRecordTestUtils.KvRecordBatchFactory batchFactory = + KvRecordTestUtils.KvRecordBatchFactory.of(DEFAULT_SCHEMA_ID); + KvRecordTestUtils.KvRecordFactory recordFactory = + KvRecordTestUtils.KvRecordFactory.of(DATA1_ROW_TYPE); + kvTablet.putAsLeader( + batchFactory.ofRecords( + Collections.singletonList( + recordFactory.ofRecord("k1".getBytes(), new Object[] {1, "v1"}))), + null); + kvTablet.flush(Long.MAX_VALUE, NOPErrorHandler.INSTANCE); + + FlussScheduler testScheduler = new FlussScheduler(1); + testScheduler.startup(); + try (ScannerManager scannerManager = new ScannerManager(conf, testScheduler)) { + + replicaManager.setScannerManager(scannerManager); + scannerManager.createScanner(kvTablet, tb, null); + assertThat(scannerManager.activeScannerCount()).isEqualTo(1); + + CompletableFuture> future = new CompletableFuture<>(); + replicaManager.stopReplicas( + INITIAL_COORDINATOR_EPOCH, + Collections.singletonList( + new StopReplicaData( + tb, + false, + false, + INITIAL_COORDINATOR_EPOCH, + INITIAL_LEADER_EPOCH)), + future::complete); + future.get(); + + assertThat(scannerManager.activeScannerCount()).isEqualTo(0); + } finally { + testScheduler.shutdown(); + replicaManager.setScannerManager(null); + } + } } From 2f628a7cccf03c1623d580b2629802772801c934 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Tue, 21 Apr 2026 17:13:45 +0300 Subject: [PATCH 05/17] address copilot suggestions --- .../apache/fluss/config/ConfigOptions.java | 5 +- .../org/apache/fluss/server/kv/KvTablet.java | 61 ++++++++++--------- .../fluss/server/kv/scan/ScannerManager.java | 5 +- .../fluss/server/tablet/TabletService.java | 2 +- 4 files changed, 40 insertions(+), 33 deletions(-) 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 923c74ccd8..fe93dd328d 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 @@ -520,8 +520,9 @@ public class ConfigOptions { .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."); + "The time-to-live for an idle KV scanner session on the server. A scanner that has not " + + "received a request within this duration will be automatically expired and its " + + "resources released. The default value is 10 minutes."); public static final ConfigOption KV_SCANNER_EXPIRATION_INTERVAL = key("kv.scanner.expiration-interval") diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/KvTablet.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvTablet.java index db763c7a9d..e27d7ff323 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/KvTablet.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvTablet.java @@ -76,6 +76,7 @@ import org.apache.fluss.types.RowType; import org.apache.fluss.utils.BytesUtils; import org.apache.fluss.utils.FileUtils; +import org.apache.fluss.utils.IOUtils; import org.rocksdb.RateLimiter; import org.rocksdb.ReadOptions; @@ -773,7 +774,7 @@ public List limitScan(int limit) throws IOException { * org.apache.fluss.server.kv.scan.ScannerManager}) is responsible for registering it and for * closing it when the scan is complete. * - * @param scannerId the server-assigned scanner ID bytes + * @param scannerId the server-assigned scanner ID * @param limit maximum number of rows to return across all batches ({@code ≤ 0} = unlimited) * @param initialAccessTimeMs wall-clock time (ms) to use as the initial last-access timestamp * @return a newly created, cursor-positioned {@link ScannerContext}, or {@code null} if the @@ -791,6 +792,7 @@ public ScannerContext openScan(String scannerId, long limit, long initialAccessT Snapshot snapshot = null; ReadOptions readOptions = null; RocksIterator iterator = null; + boolean success = false; try { snapshot = rocksDBKv.getDb().getSnapshot(); readOptions = new ReadOptions().setSnapshot(snapshot); @@ -802,37 +804,38 @@ public ScannerContext openScan(String scannerId, long limit, long initialAccessT readOptions); iterator.seekToFirst(); if (!iterator.isValid()) { - // Empty bucket: release all resources without creating a session. - iterator.close(); - readOptions.close(); - rocksDBKv.getDb().releaseSnapshot(snapshot); - snapshot.close(); - lease.close(); + // Empty bucket: no session will be registered; cleanup in finally. return null; } - return new ScannerContext( - scannerId, - tableBucket, - rocksDBKv, - iterator, - readOptions, - snapshot, - lease, - limit, - initialAccessTimeMs); - } catch (Exception e) { - if (iterator != null) { - iterator.close(); - } - if (readOptions != null) { - readOptions.close(); - } - if (snapshot != null) { - rocksDBKv.getDb().releaseSnapshot(snapshot); - snapshot.close(); + ScannerContext context = + new ScannerContext( + scannerId, + tableBucket, + rocksDBKv, + iterator, + readOptions, + snapshot, + lease, + limit, + initialAccessTimeMs); + success = true; + return context; + } finally { + if (!success) { + // Release in reverse allocation order. Each close is independent, + // so a failure in one must not prevent the others from running. + IOUtils.closeQuietly(iterator); + IOUtils.closeQuietly(readOptions); + if (snapshot != null) { + try { + rocksDBKv.getDb().releaseSnapshot(snapshot); + } catch (Throwable t) { + LOG.warn("Error releasing RocksDB snapshot.", t); + } + IOUtils.closeQuietly(snapshot); + } + IOUtils.closeQuietly(lease); } - lease.close(); - throw e; } }); } 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 223226e42b..e0401f8c9d 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 @@ -411,7 +411,10 @@ public void close() { } } + // Note: totalScanners and perBucketCount are not forcibly reset here. Because both + // shutdown and the evictor use conditional remove(key, value), each scanner is + // decremented exactly once, so the counters naturally reach zero. A forced reset + // would risk driving counters negative if the evictor wins a remove during close(). 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 62222c0bf5..1a56f3e193 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 @@ -534,7 +534,7 @@ public CompletableFuture scanKv(ScanKvRequest request) { throw new InvalidScanRequestException("batch_size_bytes must be greater than 0."); } DefaultValueRecordBatch.Builder builder = DefaultValueRecordBatch.builder(); - int totalBytes = 0; + long totalBytes = 0L; while (context.isValid() && totalBytes < batchSizeBytes) { byte[] value = context.currentValue(); From 3f8685a6aac131e0c1444f434844f08df3d7ebbe Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Wed, 6 May 2026 11:08:46 +0300 Subject: [PATCH 06/17] address immutability + leadership-race comments --- .../fluss/server/kv/scan/ScannerManager.java | 54 ++++++------ .../apache/fluss/server/replica/Replica.java | 88 +++++++++++++++---- .../fluss/server/replica/ReplicaManager.java | 62 +++---------- .../fluss/server/tablet/TabletServer.java | 8 +- .../fluss/server/tablet/TabletService.java | 2 +- .../server/kv/scan/ScannerManagerTest.java | 62 ++++++++----- .../server/replica/ReplicaManagerTest.java | 43 ++++----- .../fluss/server/replica/ReplicaTestBase.java | 13 ++- .../fetcher/ReplicaFetcherThreadTest.java | 2 + 9 files changed, 190 insertions(+), 144 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 e0401f8c9d..60062df16b 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 @@ -22,7 +22,7 @@ import org.apache.fluss.config.Configuration; import org.apache.fluss.exception.TooManyScannersException; import org.apache.fluss.metadata.TableBucket; -import org.apache.fluss.server.kv.KvTablet; +import org.apache.fluss.server.replica.Replica; import org.apache.fluss.utils.AutoCloseableAsync; import org.apache.fluss.utils.clock.Clock; import org.apache.fluss.utils.clock.SystemClock; @@ -132,18 +132,24 @@ public ScannerManager(Configuration conf, Scheduler scheduler) { } /** - * Creates a new scan session for the given bucket, taking a point-in-time RocksDB snapshot. + * Creates a new scan session for the given replica, taking a point-in-time RocksDB snapshot + * under the replica's leader-ISR read lock. + * + *

Snapshot opening, leadership validation, and session registration all happen while the + * replica's leader-ISR lock is held, so a concurrent leadership change cannot race with this + * operation: either the scanner is registered before the leader flips (and is then evicted by + * {@link #closeScannersForBucket(TableBucket)} during {@code makeFollowers} / {@code + * stopReplica}), or the leadership check fails and no scanner is ever created. * *

Returns {@code null} if the bucket is empty (no rows to scan). In that case no session * slot is consumed and the caller should return an empty response immediately. * *

Limit enforcement is two-phase: a fast pre-check guards the common case; the * subsequent atomic increment + re-check prevents the TOCTOU race from permanently breaching - * configured limits. If registration fails after the snapshot is already opened, the context is - * closed and the exception is re-thrown to avoid leaking resources. + * configured limits. If registration fails after the snapshot is already opened, the context + * is closed and the exception is re-thrown to avoid leaking resources. * - * @param kvTablet the {@link KvTablet} for the bucket; used to open the snapshot - * @param tableBucket the bucket being scanned + * @param replica the leader {@link Replica} for the bucket being scanned * @param limit optional row-count limit ({@code null} or ≤ 0 means unlimited) * @return the newly registered {@link ScannerContext}, or {@code null} if the bucket is empty * @throws TooManyScannersException if the per-bucket or per-server limit is exceeded @@ -151,27 +157,22 @@ public ScannerManager(Configuration conf, Scheduler scheduler) { * already closed (the bucket is shutting down) */ @Nullable - public ScannerContext createScanner( - KvTablet kvTablet, TableBucket tableBucket, @Nullable Long limit) throws IOException { - checkLimits(tableBucket); - + public ScannerContext createScanner(Replica replica, @Nullable Long limit) throws IOException { + checkLimits(replica.getTableBucket()); String scannerId = generateScannerId(); - ScannerContext context = - kvTablet.openScan(scannerId, limit != null ? limit : -1L, clock.milliseconds()); - if (context == null) { - // Bucket is empty — no session slot consumed. - return null; - } + return replica.openScan(this, scannerId, limit != null ? limit : -1L, clock.milliseconds()); + } - try { - registerContext(context, tableBucket); - } catch (TooManyScannersException e) { - // Limit was exceeded between the initial check and registration (race window). - // Close the already-opened context to avoid leaking the snapshot and lease. - closeScannerContext(context); - throw e; - } - return context; + /** + * Atomically registers an already-opened {@link ScannerContext}, enforcing the per-bucket and + * per-server limits. On {@link TooManyScannersException} the caller is responsible for closing + * the context to release the underlying RocksDB resources. + * + *

Called by {@link Replica#openScan} while the leader-ISR read lock is held so that + * registration cannot race with a leadership change. + */ + public void register(ScannerContext context) { + registerContext(context); } /** @@ -297,7 +298,8 @@ private void checkLimits(TableBucket tableBucket) { * TooManyScannersException} and rolls back the increments if a concurrent create caused either * limit to be exceeded between the initial check and this call. */ - private void registerContext(ScannerContext context, TableBucket tableBucket) { + private void registerContext(ScannerContext context) { + TableBucket tableBucket = context.getTableBucket(); AtomicInteger bucketCount = perBucketCount.computeIfAbsent(tableBucket, k -> new AtomicInteger(0)); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java index abe6b8bfb5..ea08444f01 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java @@ -31,6 +31,7 @@ import org.apache.fluss.exception.NonPrimaryKeyTableException; import org.apache.fluss.exception.NotEnoughReplicasException; import org.apache.fluss.exception.NotLeaderOrFollowerException; +import org.apache.fluss.exception.TooManyScannersException; import org.apache.fluss.fs.FsPath; import org.apache.fluss.metadata.ChangelogImage; import org.apache.fluss.metadata.LogFormat; @@ -61,6 +62,7 @@ import org.apache.fluss.server.kv.RemoteLogFetcher; import org.apache.fluss.server.kv.autoinc.AutoIncIDRange; import org.apache.fluss.server.kv.rocksdb.RocksDBKvBuilder; +import org.apache.fluss.server.kv.scan.ScannerContext; import org.apache.fluss.server.kv.scan.ScannerManager; import org.apache.fluss.server.kv.snapshot.CompletedKvSnapshotCommitter; import org.apache.fluss.server.kv.snapshot.CompletedSnapshot; @@ -213,13 +215,12 @@ public final class Replica { private @Nullable PeriodicSnapshotManager kvSnapshotManager; /** - * Optional reference to the server-wide {@link ScannerManager}. When set, active scanner - * sessions for this bucket are closed eagerly in {@link #dropKv()} as a safety net, even on - * code paths that do not go through {@link - * org.apache.fluss.server.replica.ReplicaManager#makeFollowers} or {@link + * Reference to the server-wide {@link ScannerManager}. Active scanner sessions for this bucket + * are closed eagerly in {@link #dropKv()} as a safety net, even on code paths that do not go + * through {@link org.apache.fluss.server.replica.ReplicaManager#makeFollowers} or {@link * org.apache.fluss.server.replica.ReplicaManager#stopReplicas}. */ - @Nullable private volatile ScannerManager scannerManager; + private final ScannerManager scannerManager; // ------- metrics private Counter isrShrinks; @@ -246,7 +247,8 @@ public Replica( BucketMetricGroup bucketMetricGroup, TableInfo tableInfo, Clock clock, - RemoteLogManager remoteLogManager) + RemoteLogManager remoteLogManager, + ScannerManager scannerManager) throws Exception { this.physicalPath = physicalPath; this.tableBucket = tableBucket; @@ -279,6 +281,7 @@ public Replica( this.logTablet.updateIsDataLakeEnabled(tableConfig.isDataLakeEnabled()); this.clock = clock; this.remoteLogManager = remoteLogManager; + this.scannerManager = checkNotNull(scannerManager, "scannerManager"); registerMetrics(); } @@ -388,11 +391,6 @@ public Path getTabletParentDir() { return kvTablet; } - /** Injects the {@link ScannerManager} so that {@link #dropKv()} can close active scanners. */ - public void setScannerManager(@Nullable ScannerManager scannerManager) { - this.scannerManager = scannerManager; - } - public TablePath getTablePath() { return physicalPath.getTablePath(); } @@ -722,10 +720,7 @@ private void dropKv() { // ScannerManager.closeScannersForBucket directly on ReplicaManager, but this guard // ensures ResourceGuard leases are released even on unexpected code paths, preventing // KvTablet.close() from blocking indefinitely on resourceGuard.close(). - ScannerManager sm = this.scannerManager; - if (sm != null) { - sm.closeScannersForBucket(tableBucket); - } + scannerManager.closeScannersForBucket(tableBucket); // close any closeable registry for kv if (closeableRegistry.unregisterCloseable(closeableRegistryForKv)) { IOUtils.closeQuietly(closeableRegistryForKv); @@ -1401,6 +1396,69 @@ public DefaultValueRecordBatch limitKvScan(int limit) { }); } + /** + * Opens a new full-scan session against this replica's KV store, taking a point-in-time + * RocksDB snapshot under the {@code leaderIsrUpdateLock} read lock. + * + *

The lock is held for the entire flow — leadership check, KV-tablet acquisition, snapshot + * creation, and registration with the {@link ScannerManager} — so a concurrent leadership + * change cannot leave a scanner registered for a follower bucket. Once a leadership change + * acquires the write lock, {@link + * org.apache.fluss.server.replica.ReplicaManager#makeFollowers} / {@code stopReplicas} will + * call {@link ScannerManager#closeScannersForBucket(TableBucket)} and any scanner registered + * before the flip is released eagerly. + * + *

Returns {@code null} if the bucket is empty at snapshot time; in that case no session + * slot is consumed. + * + * @param scannerManager the manager to register the new context with + * @param scannerId the server-assigned scanner ID + * @param limit maximum number of rows to return ({@code ≤ 0} = unlimited) + * @param initialAccessTimeMs initial last-access timestamp for TTL accounting + * @throws NonPrimaryKeyTableException if this replica is not a primary-key (KV) table + * @throws NotLeaderOrFollowerException if this replica is not currently the leader + * @throws TooManyScannersException if registering would breach the configured scanner limits + * @throws IOException if RocksDB is shutting down + */ + @Nullable + public ScannerContext openScan( + ScannerManager scannerManager, String scannerId, long limit, long initialAccessTimeMs) + throws IOException { + if (!isKvTable()) { + throw new NonPrimaryKeyTableException( + "the primary key table not exists for " + tableBucket); + } + + return inReadLock( + leaderIsrUpdateLock, + () -> { + if (!isLeader()) { + throw new NotLeaderOrFollowerException( + String.format( + "Leader not local for bucket %s on tabletServer %d", + tableBucket, localTabletServerId)); + } + checkNotNull( + kvTablet, + "KvTablet for the replica to open scan shouldn't be null."); + ScannerContext context = + kvTablet.openScan(scannerId, limit, initialAccessTimeMs); + if (context == null) { + // Empty bucket — no session is registered. + return null; + } + try { + scannerManager.register(context); + } catch (TooManyScannersException e) { + // Limit was breached between the pre-check and registration. + // Close the already-opened context to release the snapshot and lease. + IOUtils.closeQuietly(context); + throw e; + } + return context; + }); + } + public LogRecords limitLogScan(int limit) { return inReadLock( leaderIsrUpdateLock, diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java index b34cea21f9..6bda1e4f6b 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java @@ -27,7 +27,6 @@ import org.apache.fluss.exception.InvalidColumnProjectionException; import org.apache.fluss.exception.InvalidCoordinatorException; import org.apache.fluss.exception.InvalidRequiredAcksException; -import org.apache.fluss.exception.InvalidTableException; import org.apache.fluss.exception.LogOffsetOutOfRangeException; import org.apache.fluss.exception.LogStorageException; import org.apache.fluss.exception.NotLeaderOrFollowerException; @@ -80,7 +79,6 @@ import org.apache.fluss.server.entity.UserContext; import org.apache.fluss.server.kv.KvManager; import org.apache.fluss.server.kv.KvSnapshotResource; -import org.apache.fluss.server.kv.KvTablet; import org.apache.fluss.server.kv.scan.ScannerManager; import org.apache.fluss.server.kv.snapshot.CompletedKvSnapshotCommitter; import org.apache.fluss.server.kv.snapshot.DefaultSnapshotContext; @@ -212,7 +210,7 @@ public class ReplicaManager implements ServerReconfigurable { private final Clock clock; - @Nullable private ScannerManager scannerManager; + private final ScannerManager scannerManager; public ReplicaManager( Configuration conf, @@ -228,6 +226,7 @@ public ReplicaManager( FatalErrorHandler fatalErrorHandler, TabletServerMetricGroup serverMetricGroup, UserMetrics userMetrics, + ScannerManager scannerManager, Clock clock, ExecutorService ioExecutor) throws IOException { @@ -246,6 +245,7 @@ public ReplicaManager( serverMetricGroup, userMetrics, new RemoteLogManager(conf, zkClient, coordinatorGateway, clock, ioExecutor), + scannerManager, clock, ioExecutor); } @@ -266,6 +266,7 @@ public ReplicaManager( TabletServerMetricGroup serverMetricGroup, UserMetrics userMetrics, RemoteLogManager remoteLogManager, + ScannerManager scannerManager, Clock clock, ExecutorService ioExecutor) throws IOException { @@ -315,6 +316,7 @@ public ReplicaManager( this.clock = clock; this.ioExecutor = ioExecutor; this.minInSyncReplicas = conf.get(ConfigOptions.LOG_REPLICA_MIN_IN_SYNC_REPLICAS_NUMBER); + this.scannerManager = checkNotNull(scannerManager, "scannerManager"); registerMetrics(); } @@ -329,10 +331,6 @@ public void startup() { conf.get(ConfigOptions.LOG_REPLICA_MAX_LAG_TIME).toMillis() / 2); } - public void setScannerManager(ScannerManager scannerManager) { - this.scannerManager = scannerManager; - } - public RemoteLogManager getRemoteLogManager() { return remoteLogManager; } @@ -1164,9 +1162,7 @@ private void makeFollowers( Replica replica = getReplicaOrException(data.getTableBucket()); if (replica.makeFollower(data)) { replicasBecomeFollower.add(replica); - if (scannerManager != null) { - scannerManager.closeScannersForBucket(tb); - } + scannerManager.closeScannersForBucket(tb); } // stop the remote log tiering tasks for followers remoteLogManager.stopLogTiering(replica); @@ -1846,14 +1842,11 @@ private StopReplicaResultForBucket stopReplica( replicaFetcherManager.removeFetcherForBuckets(Collections.singleton(tb)); // Close active scanner sessions for this bucket before tearing down the KV tablet. - // A concurrent scanKv RPC in flight can still reach getLeaderKvTablet(tb) and race - // with createScanner. Both KvTablet.openScan and ResourceGuard.acquireResource() - // synchronise with the subsequent replica/RocksDB close, so the worst case is an - // IOException surfaced cleanly as an RPC error — never cursor corruption or a - // post-shutdown snapshot. - if (scannerManager != null) { - scannerManager.closeScannersForBucket(tb); - } + // Replica#openScan registers scanners under the leaderIsrUpdateLock read lock and the + // make-follower / stop-replica flow flips leadership under the write lock, so any + // scanner that races with this stop is either registered before the leader flips (and + // released here) or rejected by the isLeader() check inside openScan. + scannerManager.closeScannersForBucket(tb); HostedReplica replica = getReplica(tb); if (replica instanceof OnlineReplica) { @@ -1967,10 +1960,8 @@ protected Optional maybeCreateReplica(NotifyLeaderAndIsrData data) { bucketMetricGroup, tableInfo, clock, - remoteLogManager); - // Inject the ScannerManager so that Replica.dropKv() can eagerly close scanner - // sessions as a safety net on unexpected shutdown paths. - replica.setScannerManager(scannerManager); + remoteLogManager, + scannerManager); allReplicas.put(tb, new OnlineReplica(replica)); replicaOpt = Optional.of(replica); } else if (hostedReplica instanceof OnlineReplica) { @@ -2003,33 +1994,6 @@ public HostedReplica getReplica(TableBucket tableBucket) { return allReplicas.getOrDefault(tableBucket, new NoneReplica()); } - /** - * Returns the {@link KvTablet} for the local leader replica of the given bucket. - * - * @throws NotLeaderOrFollowerException if this server is not the leader for the bucket - * @throws InvalidTableException if the bucket does not have KV storage (not a primary-key - * table) - * @throws UnknownTableOrBucketException if the bucket is not known to this server - */ - public KvTablet getLeaderKvTablet(TableBucket tableBucket) { - Replica replica = getReplicaOrException(tableBucket); - if (!replica.isLeader()) { - throw new NotLeaderOrFollowerException( - String.format( - "Leader not local for bucket %s on tablet server %d", - tableBucket, serverId)); - } - KvTablet kvTablet = replica.getKvTablet(); - if (kvTablet == null) { - throw new InvalidTableException( - String.format( - "Bucket %s does not have KV storage. " - + "Full KV scan is only supported on primary-key tables.", - tableBucket)); - } - return kvTablet; - } - private boolean isRequiredAcksInvalid(int requiredAcks) { return requiredAcks != 0 && requiredAcks != 1 && requiredAcks != -1; } 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 247d270ba7..aa43b7410a 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 @@ -258,6 +258,10 @@ protected void startServices() throws Exception { conf.get(ConfigOptions.SERVER_IO_POOL_SIZE), new ExecutorThreadFactory("tablet-server-io")); + // Create the ScannerManager before the ReplicaManager so that it can be wired in + // through the ReplicaManager's constructor (and reach Replica via constructor too). + this.scannerManager = new ScannerManager(conf, scheduler); + this.replicaManager = new ReplicaManager( conf, @@ -274,6 +278,7 @@ protected void startServices() throws Exception { this, tabletServerMetricGroup, userMetrics, + scannerManager, clock, ioExecutor); replicaManager.startup(); @@ -285,9 +290,6 @@ protected void startServices() throws Exception { // Start dynamicConfigManager after all reconfigurable components are registered dynamicConfigManager.startup(); - this.scannerManager = new ScannerManager(conf, scheduler); - replicaManager.setScannerManager(scannerManager); - this.tabletService = new TabletService( serverId, 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 1a56f3e193..dadc9ef488 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 @@ -475,7 +475,7 @@ public CompletableFuture scanKv(ScanKvRequest request) { context = scannerManager.createScanner( - replicaManager.getLeaderKvTablet(tableBucket), tableBucket, limit); + replicaManager.getReplicaOrException(tableBucket), limit); if (context == null) { // Bucket is empty — return an empty response immediately without registering a diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java index b054d30960..e5d8939944 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java @@ -190,6 +190,31 @@ private ScannerManager createManagerWithShortTtl(long ttlMs, long expirationInte return new ScannerManager(c, scheduler, clock); } + /** + * Helper used by tests to open a scanner context directly against the KvTablet (bypassing + * Replica) and register it with the manager. The Replica-mediated flow is covered by + * integration tests; here we want to exercise ScannerManager in isolation. + */ + private ScannerContext openAndRegister(ScannerManager manager) throws Exception { + ScannerContext ctx = + kvTablet.openScan( + java.util.UUID.randomUUID().toString(), -1L, clock.milliseconds()); + if (ctx == null) { + return null; + } + try { + manager.register(ctx); + } catch (RuntimeException e) { + // Mirror the production behaviour in Replica#openScan: if registration fails + // (e.g. TooManyScannersException), close the already-opened context to release + // the underlying RocksDB snapshot/iterator/lease so the test KvTablet can shut + // down cleanly. + ctx.close(); + throw e; + } + return ctx; + } + /** Writes {@code count} rows into the KvTablet and flushes to RocksDB. */ private void putAndFlush(int count) throws Exception { List rows = new ArrayList<>(); @@ -209,9 +234,8 @@ private void putAndFlush(int count) throws Exception { @Test void testCreateScanner_emptyBucket_returnsNull() throws Exception { try (ScannerManager manager = createManager()) { - TableBucket tableBucket = kvTablet.getTableBucket(); // Bucket has no data — openScan must return null; no slot consumed. - ScannerContext context = manager.createScanner(kvTablet, tableBucket, null); + ScannerContext context = openAndRegister(manager); assertThat(context).isNull(); assertThat(manager.activeScannerCount()).isEqualTo(0); } @@ -223,7 +247,7 @@ void testCreateAndRemoveScanner() throws Exception { try (ScannerManager manager = createManager()) { TableBucket tableBucket = kvTablet.getTableBucket(); - ScannerContext context = manager.createScanner(kvTablet, tableBucket, null); + ScannerContext context = openAndRegister(manager); assertThat(context).isNotNull(); assertThat(manager.activeScannerCount()).isEqualTo(1); assertThat(manager.activeScannerCountForBucket(tableBucket)).isEqualTo(1); @@ -238,10 +262,8 @@ void testCreateAndRemoveScanner() throws Exception { void testGetScanner_refreshesLastAccessTime() throws Exception { putAndFlush(3); try (ScannerManager manager = createManager()) { - TableBucket tableBucket = kvTablet.getTableBucket(); - // Create scanner at t=0. - ScannerContext context = manager.createScanner(kvTablet, tableBucket, null); + ScannerContext context = openAndRegister(manager); assertThat(context).isNotNull(); byte[] scannerId = context.getScannerId(); @@ -263,9 +285,7 @@ void testTtlEviction() throws Exception { // TTL = 200 ms, evictor every 200 ms — wide enough for slow CI schedulers. ScannerManager manager = createManagerWithShortTtl(200, 200); try { - TableBucket tableBucket = kvTablet.getTableBucket(); - - ScannerContext context = manager.createScanner(kvTablet, tableBucket, null); + ScannerContext context = openAndRegister(manager); assertThat(context).isNotNull(); byte[] scannerId = context.getScannerId(); @@ -292,11 +312,11 @@ void testPerBucketLimit() throws Exception { try (ScannerManager manager = createManager(2, 200)) { TableBucket tableBucket = kvTablet.getTableBucket(); - ScannerContext ctx1 = manager.createScanner(kvTablet, tableBucket, null); - ScannerContext ctx2 = manager.createScanner(kvTablet, tableBucket, null); + ScannerContext ctx1 = openAndRegister(manager); + ScannerContext ctx2 = openAndRegister(manager); assertThat(manager.activeScannerCountForBucket(tableBucket)).isEqualTo(2); - assertThatThrownBy(() -> manager.createScanner(kvTablet, tableBucket, null)) + assertThatThrownBy(() -> openAndRegister(manager)) .isInstanceOf(TooManyScannersException.class); // Count must not have changed after the failed attempt. @@ -311,13 +331,11 @@ void testPerBucketLimit() throws Exception { void testPerServerLimit() throws Exception { putAndFlush(3); try (ScannerManager manager = createManager(8, 2)) { - TableBucket tableBucket = kvTablet.getTableBucket(); - - ScannerContext ctx1 = manager.createScanner(kvTablet, tableBucket, null); - ScannerContext ctx2 = manager.createScanner(kvTablet, tableBucket, null); + ScannerContext ctx1 = openAndRegister(manager); + ScannerContext ctx2 = openAndRegister(manager); assertThat(manager.activeScannerCount()).isEqualTo(2); - assertThatThrownBy(() -> manager.createScanner(kvTablet, tableBucket, null)) + assertThatThrownBy(() -> openAndRegister(manager)) .isInstanceOf(TooManyScannersException.class); assertThat(manager.activeScannerCount()).isEqualTo(2); @@ -333,8 +351,8 @@ void testCloseScannersForBucket() throws Exception { try (ScannerManager manager = createManager()) { TableBucket tableBucket = kvTablet.getTableBucket(); - manager.createScanner(kvTablet, tableBucket, null); - manager.createScanner(kvTablet, tableBucket, null); + openAndRegister(manager); + openAndRegister(manager); assertThat(manager.activeScannerCount()).isEqualTo(2); manager.closeScannersForBucket(tableBucket); @@ -348,10 +366,8 @@ void testCloseScannersForBucket() throws Exception { void testShutdown_closesAllScanners() throws Exception { putAndFlush(3); ScannerManager manager = createManager(); - TableBucket tableBucket = kvTablet.getTableBucket(); - - manager.createScanner(kvTablet, tableBucket, null); - manager.createScanner(kvTablet, tableBucket, null); + openAndRegister(manager); + openAndRegister(manager); assertThat(manager.activeScannerCount()).isEqualTo(2); manager.close(); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java index 3eec7ac61d..49c8d2ec05 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java @@ -83,7 +83,6 @@ import org.apache.fluss.types.DataTypes; import org.apache.fluss.types.RowType; import org.apache.fluss.utils.CloseableIterator; -import org.apache.fluss.utils.concurrent.FlussScheduler; import org.apache.fluss.utils.types.Tuple2; import org.junit.jupiter.api.Test; @@ -2404,31 +2403,23 @@ void testStopReplicas_closesScanners() throws Exception { null); kvTablet.flush(Long.MAX_VALUE, NOPErrorHandler.INSTANCE); - FlussScheduler testScheduler = new FlussScheduler(1); - testScheduler.startup(); - try (ScannerManager scannerManager = new ScannerManager(conf, testScheduler)) { - - replicaManager.setScannerManager(scannerManager); - scannerManager.createScanner(kvTablet, tb, null); - assertThat(scannerManager.activeScannerCount()).isEqualTo(1); - - CompletableFuture> future = new CompletableFuture<>(); - replicaManager.stopReplicas( - INITIAL_COORDINATOR_EPOCH, - Collections.singletonList( - new StopReplicaData( - tb, - false, - false, - INITIAL_COORDINATOR_EPOCH, - INITIAL_LEADER_EPOCH)), - future::complete); - future.get(); + Replica replica = replicaManager.getReplicaOrException(tb); + scannerManager.createScanner(replica, null); + assertThat(scannerManager.activeScannerCount()).isEqualTo(1); - assertThat(scannerManager.activeScannerCount()).isEqualTo(0); - } finally { - testScheduler.shutdown(); - replicaManager.setScannerManager(null); - } + CompletableFuture> future = new CompletableFuture<>(); + replicaManager.stopReplicas( + INITIAL_COORDINATOR_EPOCH, + Collections.singletonList( + new StopReplicaData( + tb, + false, + false, + INITIAL_COORDINATOR_EPOCH, + INITIAL_LEADER_EPOCH)), + future::complete); + future.get(); + + assertThat(scannerManager.activeScannerCount()).isEqualTo(0); } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTestBase.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTestBase.java index f6b2e01c87..a9041acbda 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTestBase.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTestBase.java @@ -37,6 +37,7 @@ import org.apache.fluss.server.coordinator.TestCoordinatorGateway; import org.apache.fluss.server.entity.NotifyLeaderAndIsrData; import org.apache.fluss.server.kv.KvManager; +import org.apache.fluss.server.kv.scan.ScannerManager; import org.apache.fluss.server.kv.snapshot.CompletedKvSnapshotCommitter; import org.apache.fluss.server.kv.snapshot.CompletedSnapshot; import org.apache.fluss.server.kv.snapshot.KvSnapshotDataDownloader; @@ -140,6 +141,7 @@ public class ReplicaTestBase { protected LogManager logManager; protected KvManager kvManager; protected ReplicaManager replicaManager; + protected ScannerManager scannerManager; protected RpcClient rpcClient; protected Configuration conf; protected TabletServerMetadataCache serverMetadataCache; @@ -308,6 +310,9 @@ protected long registerTableInZkClient( protected ReplicaManager buildReplicaManager(CoordinatorGateway coordinatorGateway) throws Exception { + if (scannerManager == null) { + scannerManager = new ScannerManager(conf, scheduler); + } return new ReplicaManager( conf, scheduler, @@ -323,6 +328,7 @@ protected ReplicaManager buildReplicaManager(CoordinatorGateway coordinatorGatew TestingMetricGroups.TABLET_SERVER_METRICS, TestingMetricGroups.USER_METRICS, remoteLogManager, + scannerManager, manualClock, ioExecutor); } @@ -351,6 +357,10 @@ void tearDown() throws Exception { replicaManager.shutdown(); } + if (scannerManager != null) { + scannerManager.close(); + } + if (rpcClient != null) { rpcClient.close(); } @@ -502,7 +512,8 @@ private Replica makeReplica( metricGroup, DATA1_TABLE_INFO, manualClock, - remoteLogManager); + remoteLogManager, + scannerManager); } private void initRemoteLogEnv() throws Exception { diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherThreadTest.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherThreadTest.java index ff089cb4a9..b68cbe6b76 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherThreadTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherThreadTest.java @@ -33,6 +33,7 @@ import org.apache.fluss.server.entity.NotifyLeaderAndIsrData; import org.apache.fluss.server.entity.NotifyLeaderAndIsrResultForBucket; import org.apache.fluss.server.kv.KvManager; +import org.apache.fluss.server.kv.scan.ScannerManager; import org.apache.fluss.server.kv.snapshot.TestingCompletedKvSnapshotCommitter; import org.apache.fluss.server.log.LogManager; import org.apache.fluss.server.metadata.TabletServerMetadataCache; @@ -566,6 +567,7 @@ public TestingReplicaManager( NOPErrorHandler.INSTANCE, serverMetricGroup, USER_METRICS, + new ScannerManager(conf, scheduler), clock, ioExecutor); } From 58bfa1f8c92f7922ea1855fb8c318e25f626f90a Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Wed, 6 May 2026 11:22:54 +0300 Subject: [PATCH 07/17] address ScannerManager / ScannerContext comments --- .../fluss/server/kv/scan/ScannerContext.java | 9 ++- .../fluss/server/kv/scan/ScannerManager.java | 80 +++++++++++++------ .../apache/fluss/server/replica/Replica.java | 18 ++--- .../server/kv/scan/ScannerManagerTest.java | 10 +-- .../server/replica/ReplicaManagerTest.java | 6 +- 5 files changed, 73 insertions(+), 50 deletions(-) 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 ba0c91fe66..18a425065e 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 @@ -61,13 +61,16 @@ public class ScannerContext implements Closeable { // in-order check: expectedSeqId = callSeqId + 1 = -1 + 1 = 0. // callSeqId validation is only performed for continuation requests (those carrying a // scanner_id), never for the initial open request (those carrying a bucket_scan_req). - private int callSeqId = -1; + // volatile because a continuation may be served by a different RPC worker thread than the + // one that last advanced this counter. + private volatile int callSeqId = -1; /** * Wall-clock timestamp (ms) of the most recent request that touched this session. Used by - * {@link ScannerManager} for TTL-based eviction. + * {@link ScannerManager} for TTL-based eviction. {@code volatile} so the evictor thread cannot + * observe a stale timestamp written by the most recent RPC worker. */ - private long lastAccessTime; + private volatile long lastAccessTime; private final AtomicBoolean closed = new AtomicBoolean(false); 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 60062df16b..607286a10e 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 @@ -33,6 +33,7 @@ import org.slf4j.LoggerFactory; import javax.annotation.Nullable; +import javax.annotation.concurrent.ThreadSafe; import java.io.IOException; import java.nio.charset.StandardCharsets; @@ -43,6 +44,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; /** @@ -80,10 +82,18 @@ * {@link #closeScannersForBucket(TableBucket)} must be called when a bucket loses leadership to * release all RocksDB snapshot/iterator resources for that bucket promptly. */ +@ThreadSafe public class ScannerManager implements AutoCloseableAsync { private static final Logger LOG = LoggerFactory.getLogger(ScannerManager.class); + /** + * Sentinel zero counter returned by {@link #activeScannerCountForBucket(TableBucket)} and + * {@link #checkLimits(TableBucket)} when a bucket has no entry in {@link #perBucketCount}, to + * avoid creating a fresh {@link AtomicInteger} for purely-read paths. + */ + private static final AtomicInteger ZERO = new AtomicInteger(0); + private final Map scanners = new ConcurrentHashMap<>(); private final Map recentlyExpiredIds = new ConcurrentHashMap<>(); @@ -93,6 +103,12 @@ public class ScannerManager implements AutoCloseableAsync { /** Total active scanner count across all buckets on this tablet server. */ private final AtomicInteger totalScanners = new AtomicInteger(0); + /** + * Set to {@code true} on entry to {@link #close()} so the background TTL evictor can short- + * circuit and avoid mutating counters concurrently with shutdown. + */ + private final AtomicBoolean closed = new AtomicBoolean(false); + private final Clock clock; private final long scannerTtlMs; private final long recentlyExpiredRetentionMs; @@ -146,8 +162,8 @@ public ScannerManager(Configuration conf, Scheduler scheduler) { * *

Limit enforcement is two-phase: a fast pre-check guards the common case; the * subsequent atomic increment + re-check prevents the TOCTOU race from permanently breaching - * configured limits. If registration fails after the snapshot is already opened, the context - * is closed and the exception is re-thrown to avoid leaking resources. + * configured limits. If registration fails after the snapshot is already opened, the context is + * closed and the exception is re-thrown to avoid leaking resources. * * @param replica the leader {@link Replica} for the bucket being scanned * @param limit optional row-count limit ({@code null} or ≤ 0 means unlimited) @@ -244,8 +260,7 @@ public int activeScannerCount() { /** Returns the number of active scanner sessions for the given bucket. */ @VisibleForTesting public int activeScannerCountForBucket(TableBucket tableBucket) { - AtomicInteger count = perBucketCount.get(tableBucket); - return count == null ? 0 : count.get(); + return perBucketCount.getOrDefault(tableBucket, ZERO).get(); } /** @@ -253,22 +268,23 @@ public int activeScannerCountForBucket(TableBucket tableBucket) { * bucket loses leadership to prevent stale RocksDB snapshot/iterator leaks. */ public void closeScannersForBucket(TableBucket tableBucket) { - List toRemove = new ArrayList<>(); + List toRemove = new ArrayList<>(); for (Map.Entry entry : scanners.entrySet()) { if (tableBucket.equals(entry.getValue().getTableBucket())) { - toRemove.add(entry.getKey()); + toRemove.add(entry.getValue()); } } - for (String key : toRemove) { - ScannerContext context = scanners.get(key); - if (context != null) { - LOG.info( - "Closing scanner {} for bucket {} due to leadership change.", - key, - tableBucket); - removeScanner(context); - } + for (ScannerContext context : toRemove) { + LOG.info( + "Closing scanner {} for bucket {} due to leadership change.", + context.getIdString(), + tableBucket); + removeScanner(context); } + // Drop any leftover per-bucket counter so we don't leak an empty AtomicInteger after + // the bucket has lost leadership. decrementCounts() also tears down empty entries + // opportunistically; this is the belt-and-suspenders cleanup at the end. + perBucketCount.remove(tableBucket); } /** @@ -283,9 +299,9 @@ private void checkLimits(TableBucket tableBucket) { "Cannot create scanner for bucket %s: server-wide limit of %d reached.", tableBucket, maxPerServer)); } - AtomicInteger bucketCount = - perBucketCount.computeIfAbsent(tableBucket, k -> new AtomicInteger(0)); - if (bucketCount.get() >= maxPerBucket) { + // Read via ZERO sentinel so a failed pre-check does not leak an empty AtomicInteger + // into perBucketCount; the entry is created lazily by registerContext() on success. + if (perBucketCount.getOrDefault(tableBucket, ZERO).get() >= maxPerBucket) { throw new TooManyScannersException( String.format( "Cannot create scanner for bucket %s: per-bucket limit of %d reached.", @@ -300,8 +316,6 @@ private void checkLimits(TableBucket tableBucket) { */ private void registerContext(ScannerContext context) { TableBucket tableBucket = context.getTableBucket(); - AtomicInteger bucketCount = - perBucketCount.computeIfAbsent(tableBucket, k -> new AtomicInteger(0)); int newTotal = totalScanners.incrementAndGet(); if (newTotal > maxPerServer) { @@ -312,6 +326,10 @@ private void registerContext(ScannerContext context) { tableBucket, maxPerServer)); } + // Lazily allocate the per-bucket counter only after the server-wide check passes, + // so a server-overload reject path doesn't pollute perBucketCount. + AtomicInteger bucketCount = + perBucketCount.computeIfAbsent(tableBucket, k -> new AtomicInteger(0)); int newBucketCount = bucketCount.incrementAndGet(); if (newBucketCount > maxPerBucket) { bucketCount.decrementAndGet(); @@ -334,6 +352,11 @@ private void registerContext(ScannerContext context) { /** TTL evictor — invoked periodically by the background scheduler. */ private void evictExpiredScanners() { + // If close() has begun, skip eviction entirely so we cannot race with shutdown's + // own teardown of the scanners map and counters. + if (closed.get()) { + return; + } long now = clock.milliseconds(); // Prune stale entries from the recently-expired cache to bound memory usage. @@ -397,10 +420,14 @@ public CompletableFuture closeAsync() { @Override public void close() { + // Idempotent: a second close() is a no-op. + if (!closed.compareAndSet(false, true)) { + return; + } // Note: we cancel but do not join the evictor. The evictor may still be mid-iteration - // when close() begins. This is safe because (a) scanners is a ConcurrentHashMap, and - // (b) both shutdown and the evictor use conditional remove(key, value) to mutate it, - // so at most one side ever closes a given ScannerContext. + // when close() begins, but it checks the `closed` flag at the top and short-circuits + // before mutating any counters; combined with conditional remove(key, value) on + // `scanners`, each ScannerContext is closed by exactly one side. if (evictorTask != null) { evictorTask.cancel(false); evictorTask = null; @@ -414,9 +441,10 @@ public void close() { } // Note: totalScanners and perBucketCount are not forcibly reset here. Because both - // shutdown and the evictor use conditional remove(key, value), each scanner is - // decremented exactly once, so the counters naturally reach zero. A forced reset - // would risk driving counters negative if the evictor wins a remove during close(). + // shutdown and the evictor use conditional remove(key, value) — and the evictor + // bails out via the `closed` flag — each scanner is decremented exactly once, so + // the counters naturally reach zero. A forced reset would risk driving counters + // negative if a stray decrement still completes after close(). recentlyExpiredIds.clear(); } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java index ea08444f01..8381e9de9a 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java @@ -1397,19 +1397,18 @@ public DefaultValueRecordBatch limitKvScan(int limit) { } /** - * Opens a new full-scan session against this replica's KV store, taking a point-in-time - * RocksDB snapshot under the {@code leaderIsrUpdateLock} read lock. + * Opens a new full-scan session against this replica's KV store, taking a point-in-time RocksDB + * snapshot under the {@code leaderIsrUpdateLock} read lock. * *

The lock is held for the entire flow — leadership check, KV-tablet acquisition, snapshot * creation, and registration with the {@link ScannerManager} — so a concurrent leadership * change cannot leave a scanner registered for a follower bucket. Once a leadership change - * acquires the write lock, {@link - * org.apache.fluss.server.replica.ReplicaManager#makeFollowers} / {@code stopReplicas} will - * call {@link ScannerManager#closeScannersForBucket(TableBucket)} and any scanner registered - * before the flip is released eagerly. + * acquires the write lock, {@link org.apache.fluss.server.replica.ReplicaManager#makeFollowers} + * / {@code stopReplicas} will call {@link ScannerManager#closeScannersForBucket(TableBucket)} + * and any scanner registered before the flip is released eagerly. * - *

Returns {@code null} if the bucket is empty at snapshot time; in that case no session - * slot is consumed. + *

Returns {@code null} if the bucket is empty at snapshot time; in that case no session slot + * is consumed. * * @param scannerManager the manager to register the new context with * @param scannerId the server-assigned scanner ID @@ -1439,8 +1438,7 @@ public ScannerContext openScan( tableBucket, localTabletServerId)); } checkNotNull( - kvTablet, - "KvTablet for the replica to open scan shouldn't be null."); + kvTablet, "KvTablet for the replica to open scan shouldn't be null."); ScannerContext context = kvTablet.openScan(scannerId, limit, initialAccessTimeMs); if (context == null) { diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java index e5d8939944..529e2c9527 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java @@ -58,6 +58,7 @@ import static org.apache.fluss.compression.ArrowCompressionInfo.DEFAULT_COMPRESSION; import static org.apache.fluss.record.TestData.DATA1_SCHEMA_PK; +import static org.apache.fluss.testutils.common.CommonTestUtils.retry; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -293,12 +294,9 @@ void testTtlEviction() throws Exception { clock.advanceTime(500, TimeUnit.MILLISECONDS); // Wait for the real scheduler to invoke the cleanup task. - long deadline = System.currentTimeMillis() + 10_000; - while (manager.activeScannerCount() > 0 && System.currentTimeMillis() < deadline) { - Thread.sleep(50); - } - - assertThat(manager.activeScannerCount()).isEqualTo(0); + retry( + Duration.ofSeconds(10), + () -> assertThat(manager.activeScannerCount()).isEqualTo(0)); assertThat(manager.getScanner(scannerId)).isNull(); assertThat(manager.isRecentlyExpired(scannerId)).isTrue(); } finally { diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java index 49c8d2ec05..49480b4ab4 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java @@ -2412,11 +2412,7 @@ void testStopReplicas_closesScanners() throws Exception { INITIAL_COORDINATOR_EPOCH, Collections.singletonList( new StopReplicaData( - tb, - false, - false, - INITIAL_COORDINATOR_EPOCH, - INITIAL_LEADER_EPOCH)), + tb, false, false, INITIAL_COORDINATOR_EPOCH, INITIAL_LEADER_EPOCH)), future::complete); future.get(); From d5e250f0d522b523b82276144818f921c7aaa7c3 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Wed, 6 May 2026 11:58:29 +0300 Subject: [PATCH 08/17] address TabletService correctness comments --- .../apache/fluss/config/ConfigOptions.java | 10 +++ .../fluss/record/DefaultValueRecordBatch.java | 5 ++ .../org/apache/fluss/server/kv/KvTablet.java | 24 ++++--- .../fluss/server/kv/scan/OpenScanResult.java | 60 +++++++++++++++++ .../fluss/server/kv/scan/ScannerContext.java | 18 +++++ .../fluss/server/kv/scan/ScannerManager.java | 12 ++-- .../apache/fluss/server/replica/Replica.java | 20 +++--- .../fluss/server/tablet/TabletServer.java | 9 ++- .../fluss/server/tablet/TabletService.java | 67 +++++++++++++++---- .../apache/fluss/server/kv/KvTabletTest.java | 24 ++++--- .../server/kv/scan/ScannerManagerTest.java | 4 +- website/docs/maintenance/configuration.md | 1 + 12 files changed, 207 insertions(+), 47 deletions(-) create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/kv/scan/OpenScanResult.java 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 fe93dd328d..02e016852f 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 @@ -550,6 +550,16 @@ public class ConfigOptions { + "Exceeding this limit returns TOO_MANY_SCANNERS. " + "The default value is 200."); + public static final ConfigOption KV_SCANNER_MAX_BATCH_SIZE = + key("kv.scanner.max-batch-size") + .memoryType() + .defaultValue(MemorySize.parse("10mb")) + .withDescription( + "Server-side cap on the per-batch payload size for KV full-scan responses. " + + "The effective batch size is min(client-requested batch_size_bytes, " + + "this value). Protects the tablet server from out-of-memory if a " + + "client passes an excessively large batch size. The default value is 10mb."); + 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/record/DefaultValueRecordBatch.java b/fluss-common/src/main/java/org/apache/fluss/record/DefaultValueRecordBatch.java index 8d41ec4c86..3bb7539f61 100644 --- a/fluss-common/src/main/java/org/apache/fluss/record/DefaultValueRecordBatch.java +++ b/fluss-common/src/main/java/org/apache/fluss/record/DefaultValueRecordBatch.java @@ -278,6 +278,11 @@ public DefaultValueRecordBatch build() throws IOException { return DefaultValueRecordBatch.pointToMemory(segment, 0); } + /** Returns the current serialized size of the batch, including the record-batch header. */ + public int sizeInBytes() { + return sizeInBytes; + } + private void writeBatchHeader() throws IOException { outputView.setPosition(0); // update header diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/KvTablet.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvTablet.java index e27d7ff323..d2da93a696 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/KvTablet.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvTablet.java @@ -58,6 +58,7 @@ import org.apache.fluss.server.kv.rocksdb.RocksDBStatistics; import org.apache.fluss.server.kv.rowmerger.DefaultRowMerger; import org.apache.fluss.server.kv.rowmerger.RowMerger; +import org.apache.fluss.server.kv.scan.OpenScanResult; import org.apache.fluss.server.kv.scan.ScannerContext; import org.apache.fluss.server.kv.snapshot.KvFileHandleAndLocalPath; import org.apache.fluss.server.kv.snapshot.KvSnapshotDataUploader; @@ -767,8 +768,10 @@ public List limitScan(int limit) throws IOException { * Opens a new full-scan session, taking a point-in-time RocksDB snapshot under the {@code * kvLock} read lock. * - *

Returns {@code null} if the bucket contains no rows at the time of the call. In that case - * all acquired resources are released internally and no session is registered. + *

The returned {@link OpenScanResult} always carries the {@link OpenScanResult#getLogOffset + * log offset} captured at snapshot time (the latest record flushed into the KV store). On the + * empty-bucket fast path the result's {@link OpenScanResult#getContext context} is {@code null} + * and all RocksDB resources have been released internally; no session is registered. * *

The returned {@link ScannerContext} is unregistered — the caller ({@link * org.apache.fluss.server.kv.scan.ScannerManager}) is responsible for registering it and for @@ -777,12 +780,11 @@ public List limitScan(int limit) throws IOException { * @param scannerId the server-assigned scanner ID * @param limit maximum number of rows to return across all batches ({@code ≤ 0} = unlimited) * @param initialAccessTimeMs wall-clock time (ms) to use as the initial last-access timestamp - * @return a newly created, cursor-positioned {@link ScannerContext}, or {@code null} if the - * bucket is empty + * @return an {@link OpenScanResult} carrying the captured log offset and (for non-empty + * buckets) a cursor-positioned {@link ScannerContext} * @throws IOException if the ResourceGuard is already closed (RocksDB is shutting down) */ - @Nullable - public ScannerContext openScan(String scannerId, long limit, long initialAccessTimeMs) + public OpenScanResult openScan(String scannerId, long limit, long initialAccessTimeMs) throws IOException { return inReadLock( kvLock, @@ -795,6 +797,9 @@ public ScannerContext openScan(String scannerId, long limit, long initialAccessT boolean success = false; try { snapshot = rocksDBKv.getDb().getSnapshot(); + // Capture the flushed log offset under the same lock that gates flushes, + // so the value reflects exactly the data visible through the snapshot. + long capturedLogOffset = flushedLogOffset; readOptions = new ReadOptions().setSnapshot(snapshot); iterator = rocksDBKv @@ -805,7 +810,9 @@ public ScannerContext openScan(String scannerId, long limit, long initialAccessT iterator.seekToFirst(); if (!iterator.isValid()) { // Empty bucket: no session will be registered; cleanup in finally. - return null; + // Return the offset so the empty-bucket fast path can still hand it + // to the client for snapshot-to-log handoff. + return new OpenScanResult(null, capturedLogOffset); } ScannerContext context = new ScannerContext( @@ -817,9 +824,10 @@ public ScannerContext openScan(String scannerId, long limit, long initialAccessT snapshot, lease, limit, + capturedLogOffset, initialAccessTimeMs); success = true; - return context; + return new OpenScanResult(context, capturedLogOffset); } finally { if (!success) { // Release in reverse allocation order. Each close is independent, diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/OpenScanResult.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/OpenScanResult.java new file mode 100644 index 0000000000..83e6328427 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/OpenScanResult.java @@ -0,0 +1,60 @@ +/* + * 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.annotation.Internal; + +import javax.annotation.Nullable; + +/** + * Result of opening a KV full-scan session, returned by {@link + * org.apache.fluss.server.kv.KvTablet#openScan}, {@link + * org.apache.fluss.server.replica.Replica#openScan}, and {@link ScannerManager#createScanner}. + * + *

Carries both the (optional) {@link ScannerContext} and the log offset captured under the same + * lock as the RocksDB snapshot. The log offset must reach the client even on the empty-bucket fast + * path so that a snapshot-to-log handoff can be performed without missing or duplicating records. + */ +@Internal +public final class OpenScanResult { + + @Nullable private final ScannerContext context; + private final long logOffset; + + public OpenScanResult(@Nullable ScannerContext context, long logOffset) { + this.context = context; + this.logOffset = logOffset; + } + + /** + * Returns the registered {@link ScannerContext}, or {@code null} if the bucket was empty at + * snapshot time (no session is created in that case). + */ + @Nullable + public ScannerContext getContext() { + return context; + } + + /** + * Returns the log offset of the latest record that was flushed into the KV store at the moment + * the RocksDB snapshot was opened. Always non-negative. + */ + public long getLogOffset() { + return logOffset; + } +} 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 18a425065e..de1a6eb3ab 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 @@ -56,6 +56,14 @@ public class ScannerContext implements Closeable { private final ReadOptions readOptions; private final Snapshot snapshot; private final ResourceGuard.Lease resourceLease; + + /** + * Log offset of the latest record flushed to the KV store at the moment this scanner's RocksDB + * snapshot was opened. Sent to the client on the first response so that downstream consumers + * can perform a consistent snapshot-to-log handoff. + */ + private final long logOffset; + private long remainingLimit; // Initial value -1 so that the first client call_seq_id of 0 satisfies the server's // in-order check: expectedSeqId = callSeqId + 1 = -1 + 1 = 0. @@ -83,6 +91,7 @@ public ScannerContext( Snapshot snapshot, ResourceGuard.Lease resourceLease, long limit, + long logOffset, long initialAccessTimeMs) { this.scannerId = scannerId; this.scannerIdBytes = scannerId.getBytes(StandardCharsets.UTF_8); @@ -93,9 +102,18 @@ public ScannerContext( this.snapshot = snapshot; this.resourceLease = resourceLease; this.remainingLimit = limit <= 0 ? -1L : limit; + this.logOffset = logOffset; this.lastAccessTime = initialAccessTimeMs; } + /** + * Returns the log offset captured at the moment this scanner's RocksDB snapshot was opened. See + * {@link #logOffset}. + */ + public long getLogOffset() { + return logOffset; + } + public byte[] getScannerId() { return scannerIdBytes; } 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 607286a10e..faf6521711 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 @@ -157,8 +157,10 @@ public ScannerManager(Configuration conf, Scheduler scheduler) { * {@link #closeScannersForBucket(TableBucket)} during {@code makeFollowers} / {@code * stopReplica}), or the leadership check fails and no scanner is ever created. * - *

Returns {@code null} if the bucket is empty (no rows to scan). In that case no session - * slot is consumed and the caller should return an empty response immediately. + *

The returned {@link OpenScanResult} always carries the log offset captured at snapshot + * time. If the bucket is empty (no rows to scan), the result's {@link OpenScanResult#getContext + * context} is {@code null} and no session slot is consumed; the caller should still relay the + * offset on the response. * *

Limit enforcement is two-phase: a fast pre-check guards the common case; the * subsequent atomic increment + re-check prevents the TOCTOU race from permanently breaching @@ -167,13 +169,13 @@ public ScannerManager(Configuration conf, Scheduler scheduler) { * * @param replica the leader {@link Replica} for the bucket being scanned * @param limit optional row-count limit ({@code null} or ≤ 0 means unlimited) - * @return the newly registered {@link ScannerContext}, or {@code null} if the bucket is empty + * @return an {@link OpenScanResult} with the captured log offset and (for non-empty buckets) + * the newly registered {@link ScannerContext} * @throws TooManyScannersException if the per-bucket or per-server limit is exceeded * @throws IOException if the underlying {@link org.apache.fluss.server.utils.ResourceGuard} is * already closed (the bucket is shutting down) */ - @Nullable - public ScannerContext createScanner(Replica replica, @Nullable Long limit) throws IOException { + public OpenScanResult createScanner(Replica replica, @Nullable Long limit) throws IOException { checkLimits(replica.getTableBucket()); String scannerId = generateScannerId(); return replica.openScan(this, scannerId, limit != null ? limit : -1L, clock.milliseconds()); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java index 8381e9de9a..bce8085cae 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java @@ -62,6 +62,7 @@ import org.apache.fluss.server.kv.RemoteLogFetcher; import org.apache.fluss.server.kv.autoinc.AutoIncIDRange; import org.apache.fluss.server.kv.rocksdb.RocksDBKvBuilder; +import org.apache.fluss.server.kv.scan.OpenScanResult; import org.apache.fluss.server.kv.scan.ScannerContext; import org.apache.fluss.server.kv.scan.ScannerManager; import org.apache.fluss.server.kv.snapshot.CompletedKvSnapshotCommitter; @@ -1407,8 +1408,10 @@ public DefaultValueRecordBatch limitKvScan(int limit) { * / {@code stopReplicas} will call {@link ScannerManager#closeScannersForBucket(TableBucket)} * and any scanner registered before the flip is released eagerly. * - *

Returns {@code null} if the bucket is empty at snapshot time; in that case no session slot - * is consumed. + *

The returned {@link OpenScanResult} always carries the log offset captured at snapshot + * time. If the bucket is empty, the result's {@link OpenScanResult#getContext context} is + * {@code null} and no session slot is consumed; the caller should still relay the offset on the + * response. * * @param scannerManager the manager to register the new context with * @param scannerId the server-assigned scanner ID @@ -1419,8 +1422,7 @@ public DefaultValueRecordBatch limitKvScan(int limit) { * @throws TooManyScannersException if registering would breach the configured scanner limits * @throws IOException if RocksDB is shutting down */ - @Nullable - public ScannerContext openScan( + public OpenScanResult openScan( ScannerManager scannerManager, String scannerId, long limit, long initialAccessTimeMs) throws IOException { if (!isKvTable()) { @@ -1439,11 +1441,13 @@ public ScannerContext openScan( } checkNotNull( kvTablet, "KvTablet for the replica to open scan shouldn't be null."); - ScannerContext context = + OpenScanResult result = kvTablet.openScan(scannerId, limit, initialAccessTimeMs); + ScannerContext context = result.getContext(); if (context == null) { - // Empty bucket — no session is registered. - return null; + // Empty bucket — no session is registered, but propagate the captured + // log offset back to the caller. + return result; } try { scannerManager.register(context); @@ -1453,7 +1457,7 @@ public ScannerContext openScan( IOUtils.closeQuietly(context); throw e; } - return context; + return result; }); } 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 aa43b7410a..1f8a10bcf0 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 @@ -290,6 +290,12 @@ protected void startServices() throws Exception { // Start dynamicConfigManager after all reconfigurable components are registered dynamicConfigManager.startup(); + // Server-side cap on the per-batch payload size; clamped to int range because the + // ScanKvRequest.batch_size_bytes wire-field is int32. + long configuredMaxBatch = conf.get(ConfigOptions.KV_SCANNER_MAX_BATCH_SIZE).getBytes(); + int kvScanMaxBatchSizeBytes = + (int) Math.min((long) Integer.MAX_VALUE, configuredMaxBatch); + this.tabletService = new TabletService( serverId, @@ -301,7 +307,8 @@ protected void startServices() throws Exception { authorizer, dynamicConfigManager, ioExecutor, - scannerManager); + scannerManager, + kvScanMaxBatchSizeBytes); RequestsMetrics requestsMetrics = RequestsMetrics.createTabletServerRequestMetrics(tabletServerMetricGroup); 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 dadc9ef488..84e1532831 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 @@ -81,6 +81,7 @@ 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.OpenScanResult; import org.apache.fluss.server.kv.scan.ScannerContext; import org.apache.fluss.server.kv.scan.ScannerManager; import org.apache.fluss.server.log.FetchParams; @@ -146,6 +147,13 @@ public final class TabletService extends RpcServiceBase implements TabletServerG private final TabletServerMetadataProvider metadataFunctionProvider; private final ScannerManager scannerManager; + /** + * Server-side cap on per-batch payload size for KV full-scan responses. The effective batch + * size used in {@link #scanKv} is {@code min(client-requested batch_size_bytes, this value)}, + * which protects the server against OOM if a client passes an excessively large value. + */ + private final int kvScanMaxBatchSizeBytes; + public TabletService( int serverId, FileSystem remoteFileSystem, @@ -156,7 +164,8 @@ public TabletService( @Nullable Authorizer authorizer, DynamicConfigManager dynamicConfigManager, ExecutorService ioExecutor, - ScannerManager scannerManager) { + ScannerManager scannerManager, + int kvScanMaxBatchSizeBytes) { super( remoteFileSystem, ServerType.TABLET_SERVER, @@ -171,6 +180,7 @@ public TabletService( this.metadataFunctionProvider = new TabletServerMetadataProvider(zkClient, metadataManager, metadataCache); this.scannerManager = scannerManager; + this.kvScanMaxBatchSizeBytes = kvScanMaxBatchSizeBytes; } @Override @@ -454,6 +464,10 @@ public CompletableFuture scanKv(ScanKvRequest request) { ScannerContext openedContext = null; try { ScannerContext context; + // True only on the initial (bucket_scan_req) path, where we must echo the snapshot + // log offset back to the client. On continuations the field stays absent. + boolean isNewScan = false; + long initialLogOffset = 0L; if (request.hasBucketScanReq() && request.hasScannerId()) { throw new InvalidScanRequestException( @@ -473,14 +487,19 @@ public CompletableFuture scanKv(ScanKvRequest request) { bucketReq.getBucketId()); Long limit = bucketReq.hasLimit() ? bucketReq.getLimit() : null; - context = + OpenScanResult openResult = scannerManager.createScanner( replicaManager.getReplicaOrException(tableBucket), limit); + isNewScan = true; + initialLogOffset = openResult.getLogOffset(); + context = openResult.getContext(); if (context == null) { // Bucket is empty — return an empty response immediately without registering a - // session. + // session, but echo the captured log offset so the client can perform a + // consistent snapshot-to-log handoff. response.setHasMoreResults(false); + response.setLogOffset(initialLogOffset); return CompletableFuture.completedFuture(response); } openedContext = context; @@ -493,6 +512,15 @@ public CompletableFuture scanKv(ScanKvRequest request) { byte[] scannerId = request.getScannerId(); context = scannerManager.getScanner(scannerId); if (context == null) { + // If the client is sending a close request and the scanner is already gone + // (auto-closed when fully drained, or evicted by TTL), this is a benign + // no-op — return a finished response instead of surfacing an error that + // the client cannot act on. + if (request.hasCloseScanner() && request.isCloseScanner()) { + response.setScannerId(scannerId); + response.setHasMoreResults(false); + return CompletableFuture.completedFuture(response); + } if (scannerManager.isRecentlyExpired(scannerId)) { throw new ScannerExpiredException( "Scanner session has expired due to inactivity. " @@ -528,18 +556,25 @@ public CompletableFuture scanKv(ScanKvRequest request) { return CompletableFuture.completedFuture(response); } - // Build the next batch - int batchSizeBytes = request.getBatchSizeBytes(); - if (batchSizeBytes <= 0) { + // batch_size_bytes is optional in proto; require it for data-fetching requests and + // clamp to the server-side cap so a malicious or buggy client cannot trigger an OOM + // by passing Integer.MAX_VALUE. + if (!request.hasBatchSizeBytes()) { + throw new InvalidScanRequestException( + "batch_size_bytes is required for data-fetching scan requests."); + } + int requestedBatchSize = request.getBatchSizeBytes(); + if (requestedBatchSize <= 0) { throw new InvalidScanRequestException("batch_size_bytes must be greater than 0."); } - DefaultValueRecordBatch.Builder builder = DefaultValueRecordBatch.builder(); - long totalBytes = 0L; + int effectiveBatchSize = Math.min(requestedBatchSize, kvScanMaxBatchSizeBytes); - while (context.isValid() && totalBytes < batchSizeBytes) { - byte[] value = context.currentValue(); - builder.append(value); - totalBytes += value.length; + // Build the next batch using the builder's own running size as the gating signal so + // the threshold reflects the actual serialised batch (header + per-record framing), + // not just the raw value bytes. + DefaultValueRecordBatch.Builder builder = DefaultValueRecordBatch.builder(); + while (context.isValid() && builder.sizeInBytes() < effectiveBatchSize) { + builder.append(context.currentValue()); context.advance(); } @@ -551,6 +586,9 @@ public CompletableFuture scanKv(ScanKvRequest request) { if (batch.sizeInBytes() > 0) { response.setRecords(batch.getSegment(), batch.getPosition(), batch.sizeInBytes()); } + if (isNewScan) { + response.setLogOffset(initialLogOffset); + } // Update callSeqId AFTER the response is prepared so that a client retry with the // same callSeqId (due to a transient failure) can be detected and rejected. @@ -572,8 +610,9 @@ public CompletableFuture scanKv(ScanKvRequest request) { if (e instanceof InterruptedException || e.getCause() instanceof InterruptedException) { Thread.currentThread().interrupt(); } - response.setErrorCode(Errors.forException(e).code()); - response.setErrorMessage(e.getMessage() != null ? e.getMessage() : ""); + ApiError apiError = ApiError.fromThrowable(e); + response.setErrorCode(apiError.error().code()); + response.setErrorMessage(apiError.message() != null ? apiError.message() : ""); } finally { // If we made it past createScanner/getScanner but failed to deliver a complete // response, close the session rather than leaking it to TTL. The cursor has diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/KvTabletTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/KvTabletTest.java index 652b5d1239..7c3b4bb2c3 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/kv/KvTabletTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/KvTabletTest.java @@ -57,6 +57,7 @@ import org.apache.fluss.server.kv.prewrite.KvPreWriteBuffer.Value; import org.apache.fluss.server.kv.rocksdb.RocksDBStatistics; import org.apache.fluss.server.kv.rowmerger.RowMerger; +import org.apache.fluss.server.kv.scan.OpenScanResult; import org.apache.fluss.server.kv.scan.ScannerContext; import org.apache.fluss.server.log.FetchIsolation; import org.apache.fluss.server.log.LogAppendInfo; @@ -1860,11 +1861,14 @@ void testRowCountWithMixedOperations() throws Exception { } @Test - void testOpenScan_emptyBucket_returnsNull() throws Exception { + void testOpenScan_emptyBucket_returnsNullContext() throws Exception { initLogTabletAndKvTablet(DATA1_SCHEMA_PK, new HashMap<>()); - // No data has been written — openScan must return null. - ScannerContext context = kvTablet.openScan("scanner-empty", -1L, 0L); - assertThat(context).isNull(); + // No data has been written — the result wraps a null context but still carries the + // captured log offset for the empty-bucket fast path. + OpenScanResult result = kvTablet.openScan("scanner-empty", -1L, 0L); + assertThat(result).isNotNull(); + assertThat(result.getContext()).isNull(); + assertThat(result.getLogOffset()).isGreaterThanOrEqualTo(0L); } @Test @@ -1881,8 +1885,10 @@ void testOpenScan_returnsAllRows() throws Exception { kvTablet.putAsLeader(kvRecordBatchFactory.ofRecords(rows), null); kvTablet.flush(Long.MAX_VALUE, NOPErrorHandler.INSTANCE); - ScannerContext context = kvTablet.openScan("scanner-all", -1L, 0L); + OpenScanResult result = kvTablet.openScan("scanner-all", -1L, 0L); + ScannerContext context = result.getContext(); assertThat(context).isNotNull(); + assertThat(result.getLogOffset()).isEqualTo(context.getLogOffset()); int count = 0; while (context.isValid()) { @@ -1908,7 +1914,7 @@ void testOpenScan_snapshotIsolation() throws Exception { kvTablet.putAsLeader(kvRecordBatchFactory.ofRecords(initialRows), null); kvTablet.flush(Long.MAX_VALUE, NOPErrorHandler.INSTANCE); - ScannerContext context = kvTablet.openScan("scanner-snap", -1L, 0L); + ScannerContext context = kvTablet.openScan("scanner-snap", -1L, 0L).getContext(); assertThat(context).isNotNull(); // Write 2 more rows AFTER opening the scan, then flush. @@ -1945,7 +1951,7 @@ void testOpenScan_withLimit() throws Exception { kvTablet.flush(Long.MAX_VALUE, NOPErrorHandler.INSTANCE); long limit = 3L; - ScannerContext context = kvTablet.openScan("scanner-limit", limit, 0L); + ScannerContext context = kvTablet.openScan("scanner-limit", limit, 0L).getContext(); assertThat(context).isNotNull(); int count = 0; @@ -1972,8 +1978,8 @@ void testOpenScan_multipleSessionsIndependent() throws Exception { kvTablet.flush(Long.MAX_VALUE, NOPErrorHandler.INSTANCE); // Open two independent scans. - ScannerContext ctx1 = kvTablet.openScan("scanner-a", -1L, 0L); - ScannerContext ctx2 = kvTablet.openScan("scanner-b", -1L, 0L); + ScannerContext ctx1 = kvTablet.openScan("scanner-a", -1L, 0L).getContext(); + ScannerContext ctx2 = kvTablet.openScan("scanner-b", -1L, 0L).getContext(); assertThat(ctx1).isNotNull(); assertThat(ctx2).isNotNull(); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java index 529e2c9527..009db421b4 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java @@ -198,8 +198,8 @@ private ScannerManager createManagerWithShortTtl(long ttlMs, long expirationInte */ private ScannerContext openAndRegister(ScannerManager manager) throws Exception { ScannerContext ctx = - kvTablet.openScan( - java.util.UUID.randomUUID().toString(), -1L, clock.milliseconds()); + kvTablet.openScan(java.util.UUID.randomUUID().toString(), -1L, clock.milliseconds()) + .getContext(); if (ctx == null) { return null; } diff --git a/website/docs/maintenance/configuration.md b/website/docs/maintenance/configuration.md index 3c5d45e098..2f77fb5676 100644 --- a/website/docs/maintenance/configuration.md +++ b/website/docs/maintenance/configuration.md @@ -175,6 +175,7 @@ during the Fluss cluster working. | kv.scanner.expiration-interval | Duration | 30s | The interval at which the server checks for and removes expired KV scanner sessions. The default value is 30 seconds. | | kv.scanner.max-per-bucket | Integer | 8 | The maximum number of concurrent KV scanner sessions allowed per bucket. New scan requests that exceed this limit will be rejected with an error. The default value is 8. | | kv.scanner.max-per-server | Integer | 200 | The maximum total number of concurrent KV scanner sessions allowed across all buckets on a single tablet server. New scan requests that exceed this limit will be rejected with an error. The default value is 200. | +| kv.scanner.max-batch-size | MemorySize | 10mb | Server-side cap on the per-batch payload size for KV full-scan responses. The effective batch size is min(client-requested batch_size_bytes, this value). Protects the tablet server from out-of-memory if a client passes an excessively large batch size. The default value is 10mb. | ## Metrics From f249f07a959d1291a9c5901c26a239885ee28107 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Wed, 6 May 2026 12:41:13 +0300 Subject: [PATCH 09/17] get ConcurrentHashMap from MapUtils and a few other small improvements --- .../java/org/apache/fluss/utils/MapUtils.java | 43 +++++++++++++++++ .../fluss/server/kv/scan/ScannerManager.java | 22 ++++++--- .../apache/fluss/server/replica/Replica.java | 19 ++++---- .../fluss/server/replica/ReplicaManager.java | 9 ++-- .../fluss/server/tablet/TabletService.java | 46 +++++++++++++++++-- .../server/kv/scan/ScannerManagerTest.java | 23 ++++++++++ 6 files changed, 140 insertions(+), 22 deletions(-) create mode 100644 fluss-common/src/main/java/org/apache/fluss/utils/MapUtils.java diff --git a/fluss-common/src/main/java/org/apache/fluss/utils/MapUtils.java b/fluss-common/src/main/java/org/apache/fluss/utils/MapUtils.java new file mode 100644 index 0000000000..d9e20855f1 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/utils/MapUtils.java @@ -0,0 +1,43 @@ +/* + * 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.utils; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +/** + * Simple utility to work with Java maps. + * + *

Prefer {@link #newConcurrentMap()} over {@code new ConcurrentHashMap<>()} so that the codebase + * has a single, swappable construction point — see apache/fluss#375. + */ +public class MapUtils { + + /** Returns a new empty {@link ConcurrentHashMap}. */ + public static Map newConcurrentMap() { + return new ConcurrentHashMap<>(); + } + + /** Returns a new empty {@link ConcurrentHashMap} with the given initial capacity. */ + public static Map newConcurrentMap(int initialCapacity) { + return new ConcurrentHashMap<>(initialCapacity); + } + + private MapUtils() {} +} 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 faf6521711..c641aeb0ce 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 @@ -24,6 +24,7 @@ import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.server.replica.Replica; 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.FutureUtils; @@ -42,7 +43,6 @@ import java.util.Map; import java.util.UUID; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -68,8 +68,9 @@ *

Empty bucket handling

* *

If the target bucket contains no rows at the time the scan is opened, {@link - * #createScanner(KvTablet, TableBucket, Long)} returns {@code null} without consuming a limit slot. - * The caller should return an empty response immediately. + * #createScanner(Replica, Long)} returns an {@link OpenScanResult} whose {@link + * OpenScanResult#getContext context} is {@code null} without consuming a limit slot. The caller + * should return an empty response immediately, echoing back {@link OpenScanResult#getLogOffset}. * *

TTL eviction

* @@ -94,11 +95,11 @@ public class ScannerManager implements AutoCloseableAsync { */ private static final AtomicInteger ZERO = new AtomicInteger(0); - private final Map scanners = new ConcurrentHashMap<>(); - private final Map recentlyExpiredIds = new ConcurrentHashMap<>(); + private final Map scanners = MapUtils.newConcurrentMap(); + private final Map recentlyExpiredIds = MapUtils.newConcurrentMap(); /** Per-bucket active scanner count, used for O(1) per-bucket limit enforcement. */ - private final Map perBucketCount = new ConcurrentHashMap<>(); + private final Map perBucketCount = MapUtils.newConcurrentMap(); /** Total active scanner count across all buckets on this tablet server. */ private final AtomicInteger totalScanners = new AtomicInteger(0); @@ -268,6 +269,11 @@ public int activeScannerCountForBucket(TableBucket tableBucket) { /** * Closes and removes all active scanner sessions for the given bucket. Must be called when a * bucket loses leadership to prevent stale RocksDB snapshot/iterator leaks. + * + *

Closed scanner IDs are recorded in {@link #recentlyExpiredIds} so that a continuation RPC + * arriving after the close surfaces as {@code SCANNER_EXPIRED} (recoverable: client should + * restart against the new leader) rather than {@code UNKNOWN_SCANNER_ID}, which leaves the + * client unable to disambiguate "leadership moved" from "I made up an ID". */ public void closeScannersForBucket(TableBucket tableBucket) { List toRemove = new ArrayList<>(); @@ -276,11 +282,15 @@ public void closeScannersForBucket(TableBucket tableBucket) { toRemove.add(entry.getValue()); } } + long now = clock.milliseconds(); for (ScannerContext context : toRemove) { LOG.info( "Closing scanner {} for bucket {} due to leadership change.", context.getIdString(), tableBucket); + // Record before removal so a continuation racing with the close cannot observe a + // window where the ID is neither in `scanners` nor in `recentlyExpiredIds`. + recentlyExpiredIds.put(context.getIdString(), now); removeScanner(context); } // Drop any leftover per-bucket counter so we don't leak an empty AtomicInteger after diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java index bce8085cae..e208c88610 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java @@ -217,9 +217,12 @@ public final class Replica { /** * Reference to the server-wide {@link ScannerManager}. Active scanner sessions for this bucket - * are closed eagerly in {@link #dropKv()} as a safety net, even on code paths that do not go - * through {@link org.apache.fluss.server.replica.ReplicaManager#makeFollowers} or {@link - * org.apache.fluss.server.replica.ReplicaManager#stopReplicas}. + * are closed in {@link #dropKv()} under the {@code leaderIsrUpdateLock} write lock so + * that no new scanner can register between the leadership flip and the KV tablet teardown. This + * is the authoritative cleanup path; the additional {@code closeScannersForBucket} calls in + * {@link org.apache.fluss.server.replica.ReplicaManager#stopReplicas} (before {@code delete()}) + * and {@link org.apache.fluss.server.replica.ReplicaManager#makeFollowers} (after {@code + * makeFollower(...)}) cover narrower windows but do not replace this guard. */ private final ScannerManager scannerManager; @@ -716,11 +719,11 @@ private void createKv() { } private void dropKv() { - // Safety net: close any lingering scanner sessions for this bucket before tearing down - // the KV tablet. The main cleanup paths (makeFollowers, stopReplica) call - // ScannerManager.closeScannersForBucket directly on ReplicaManager, but this guard - // ensures ResourceGuard leases are released even on unexpected code paths, preventing - // KvTablet.close() from blocking indefinitely on resourceGuard.close(). + // Close any active scanner sessions for this bucket BEFORE tearing down the KV tablet + // — otherwise outstanding ResourceGuard leases would block kvTablet.close() indefinitely + // on resourceGuard.close(). This call runs under leaderIsrUpdateLock(W) (held by every + // caller of dropKv: delete(), onBecomeNewLeader(), onBecomeNewFollower()), so no new + // scanner can register concurrently — Replica#openScan registers under the read lock. scannerManager.closeScannersForBucket(tableBucket); // close any closeable registry for kv if (closeableRegistry.unregisterCloseable(closeableRegistryForKv)) { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java index 6bda1e4f6b..ea3ed59a74 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java @@ -1841,11 +1841,10 @@ private StopReplicaResultForBucket stopReplica( // First stop fetchers for this table bucket. replicaFetcherManager.removeFetcherForBuckets(Collections.singleton(tb)); - // Close active scanner sessions for this bucket before tearing down the KV tablet. - // Replica#openScan registers scanners under the leaderIsrUpdateLock read lock and the - // make-follower / stop-replica flow flips leadership under the write lock, so any - // scanner that races with this stop is either registered before the leader flips (and - // released here) or rejected by the isLeader() check inside openScan. + // Close active scanner sessions for this bucket as a first cut before the replica's + // write-lock teardown. The authoritative cleanup happens inside Replica#dropKv (called + // from delete() under the write lock); this outer call narrows the window during which + // a concurrent scanKv could observe stale state, and is a no-op once dropKv has run. scannerManager.closeScannersForBucket(tb); HostedReplica replica = getReplica(tb); 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 84e1532831..c52fc60b6e 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 @@ -20,6 +20,7 @@ import org.apache.fluss.cluster.ServerType; import org.apache.fluss.exception.AuthorizationException; import org.apache.fluss.exception.InvalidScanRequestException; +import org.apache.fluss.exception.NotLeaderOrFollowerException; import org.apache.fluss.exception.ScannerExpiredException; import org.apache.fluss.exception.UnknownScannerIdException; import org.apache.fluss.exception.UnknownTableOrBucketException; @@ -90,6 +91,7 @@ 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; @@ -473,6 +475,15 @@ public CompletableFuture scanKv(ScanKvRequest request) { throw new InvalidScanRequestException( "ScanKvRequest must not set both bucket_scan_req and scanner_id."); } + // close_scanner only makes sense on a continuation (paired with scanner_id); + // pairing it with a fresh bucket_scan_req would open and immediately tear down a + // session, wasting a slot and confusing the client contract. + if (request.hasBucketScanReq() + && request.hasCloseScanner() + && request.isCloseScanner()) { + throw new InvalidScanRequestException( + "ScanKvRequest must not set close_scanner together with bucket_scan_req."); + } if (request.hasBucketScanReq()) { // New scan: open a fresh scanner session @@ -547,7 +558,9 @@ public CompletableFuture scanKv(ScanKvRequest request) { openedContext = context; } - // Handle explicit close request + // Handle explicit close request. Honour the close even on a non-leader: the local + // session resources are still ours to release, and we don't want a leadership flip + // racing with a close to leak a snapshot. if (request.hasCloseScanner() && request.isCloseScanner()) { scannerManager.removeScanner(context); openedContext = null; @@ -556,6 +569,27 @@ public CompletableFuture scanKv(ScanKvRequest request) { return CompletableFuture.completedFuture(response); } + // Continuation: re-verify that this server is still the leader for the bucket + // before serving more data. closeScannersForBucket() will eventually evict the + // scanner on a leadership flip (and surface SCANNER_EXPIRED on later RPCs), but + // there is a small window between the leader flip and that callback during which + // the scanner remains in the map. Catching the flip here lets the client redirect + // to the new leader instead of silently consuming a stale snapshot. + if (!request.hasBucketScanReq()) { + Replica replica = replicaManager.getReplicaOrException(context.getTableBucket()); + if (!replica.isLeader()) { + // Drop the local session so resources aren't held while the client + // redirects; closeScannersForBucket() will be a no-op when it runs. + scannerManager.removeScanner(context); + openedContext = null; + throw new NotLeaderOrFollowerException( + String.format( + "Leader is no longer local for bucket %s; client should " + + "restart the scan against the new leader.", + context.getTableBucket())); + } + } + // batch_size_bytes is optional in proto; require it for data-fetching requests and // clamp to the server-side cap so a malicious or buggy client cannot trigger an OOM // by passing Integer.MAX_VALUE. @@ -571,11 +605,17 @@ public CompletableFuture scanKv(ScanKvRequest request) { // Build the next batch using the builder's own running size as the gating signal so // the threshold reflects the actual serialised batch (header + per-record framing), - // not just the raw value bytes. + // not just the raw value bytes. We always append at least one record (when data is + // available) so an unusually small effectiveBatchSize — e.g. one smaller than the + // empty-builder header — cannot produce an empty has_more=true response that would + // make the client loop indefinitely. DefaultValueRecordBatch.Builder builder = DefaultValueRecordBatch.builder(); - while (context.isValid() && builder.sizeInBytes() < effectiveBatchSize) { + boolean appendedAny = false; + while (context.isValid() + && (!appendedAny || builder.sizeInBytes() < effectiveBatchSize)) { builder.append(context.currentValue()); context.advance(); + appendedAny = true; } boolean hasMore = context.isValid(); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java index 009db421b4..aff7dd5b62 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java @@ -360,6 +360,29 @@ void testCloseScannersForBucket() throws Exception { } } + /** + * Leadership-induced closure must record the scanner ID in {@code recentlyExpiredIds} so a + * continuation RPC arriving after the close surfaces SCANNER_EXPIRED (recoverable: client + * restarts against the new leader) rather than UNKNOWN_SCANNER_ID, which would leave the client + * unable to disambiguate "leadership moved" from "I made up an ID". + */ + @Test + void testCloseScannersForBucket_marksRecentlyExpired() throws Exception { + putAndFlush(3); + try (ScannerManager manager = createManager()) { + TableBucket tableBucket = kvTablet.getTableBucket(); + + ScannerContext ctx = openAndRegister(manager); + assertThat(ctx).isNotNull(); + byte[] scannerId = ctx.getScannerId(); + + manager.closeScannersForBucket(tableBucket); + + assertThat(manager.getScanner(scannerId)).isNull(); + assertThat(manager.isRecentlyExpired(scannerId)).isTrue(); + } + } + @Test void testShutdown_closesAllScanners() throws Exception { putAndFlush(3); From 952a9cc55a3fc9b3d9268641e16c10844f8b32cd Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Wed, 6 May 2026 14:51:05 +0300 Subject: [PATCH 10/17] add more tests to tabletservice --- .../server/tablet/TabletServiceITCase.java | 405 ++++++++++++++++++ 1 file changed, 405 insertions(+) diff --git a/fluss-server/src/test/java/org/apache/fluss/server/tablet/TabletServiceITCase.java b/fluss-server/src/test/java/org/apache/fluss/server/tablet/TabletServiceITCase.java index 17b436e1ac..bd55349698 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/tablet/TabletServiceITCase.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/tablet/TabletServiceITCase.java @@ -51,6 +51,9 @@ import org.apache.fluss.rpc.messages.PbPutKvRespForBucket; import org.apache.fluss.rpc.messages.ProduceLogResponse; import org.apache.fluss.rpc.messages.PutKvResponse; +import org.apache.fluss.rpc.messages.StopReplicaRequest; +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.server.entity.NotifyLeaderAndIsrData; import org.apache.fluss.server.entity.NotifyLeaderAndIsrResultForBucket; @@ -113,6 +116,7 @@ import static org.apache.fluss.server.testutils.RpcMessageTestUtils.newPutKvRequest; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.getNotifyLeaderAndIsrResponseData; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.makeNotifyBucketLeaderAndIsr; +import static org.apache.fluss.server.utils.ServerRpcMessageUtils.makeStopBucketReplica; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.makeUpdateMetadataRequest; import static org.apache.fluss.testutils.DataTestUtils.compactedRow; import static org.apache.fluss.testutils.DataTestUtils.genKvRecordBatch; @@ -1075,4 +1079,405 @@ void testLookupWithInsertIfNotExistsAutoIncrement() throws Exception { assertThat(existingRow.getLong(1)).isEqualTo(1L); assertThat(newRow.getLong(1)).isEqualTo(3L); } + + // ------------------------------------------------------------------------- + // scanKv RPC tests (FIP-17 KV full-scan sessions) + // + // After applying DATA_1_WITH_KEY_AND_VALUE the merged final RocksDB state + // contains exactly two rows (keys 1 and 2; key 3 is tombstoned). Each test + // triggers a snapshot before scanning so the prewrite buffer is flushed + // into RocksDB and visible to the scanner's snapshot iterator. + // ------------------------------------------------------------------------- + + @Test + void testScanKv_newScan_happyPath() throws Exception { + long tableId = + createTable( + FLUSS_CLUSTER_EXTENSION, DATA1_TABLE_PATH_PK, DATA1_TABLE_DESCRIPTOR_PK); + TableBucket tb = new TableBucket(tableId, 0); + FLUSS_CLUSTER_EXTENSION.waitUntilAllReplicaReady(tb); + int leader = FLUSS_CLUSTER_EXTENSION.waitAndGetLeader(tb); + TabletServerGateway leaderGateWay = + FLUSS_CLUSTER_EXTENSION.newTabletServerClientForNode(leader); + + assertPutKvResponse( + leaderGateWay + .putKv( + newPutKvRequest( + tableId, 0, 1, genKvRecordBatch(DATA_1_WITH_KEY_AND_VALUE))) + .get()); + FLUSS_CLUSTER_EXTENSION.triggerAndWaitSnapshot(tb); + + // 1 MiB easily fits the merged state (two rows) into a single batch. + ScanKvResponse response = + leaderGateWay.scanKv(newScanKvOpenRequest(tableId, 0, 1024 * 1024)).get(); + + assertThat(response.hasErrorCode()).isFalse(); + assertThat(response.getScannerId()).isNotEmpty(); + assertThat(response.isHasMoreResults()).isFalse(); + // The captured log offset must always be carried back on the first response. + assertThat(response.hasLogOffset()).isTrue(); + assertThat(response.getLogOffset()).isGreaterThanOrEqualTo(0L); + assertThat(response.hasRecords()).isTrue(); + DefaultValueRecordBatch batch = + DefaultValueRecordBatch.pointToBytes(response.getRecords()); + assertThat(batch.getRecordCount()).isEqualTo(2); + } + + @Test + void testScanKv_multiBatchContinuation() throws Exception { + long tableId = + createTable( + FLUSS_CLUSTER_EXTENSION, DATA1_TABLE_PATH_PK, DATA1_TABLE_DESCRIPTOR_PK); + TableBucket tb = new TableBucket(tableId, 0); + FLUSS_CLUSTER_EXTENSION.waitUntilAllReplicaReady(tb); + int leader = FLUSS_CLUSTER_EXTENSION.waitAndGetLeader(tb); + TabletServerGateway leaderGateWay = + FLUSS_CLUSTER_EXTENSION.newTabletServerClientForNode(leader); + + assertPutKvResponse( + leaderGateWay + .putKv( + newPutKvRequest( + tableId, 0, 1, genKvRecordBatch(DATA_1_WITH_KEY_AND_VALUE))) + .get()); + FLUSS_CLUSTER_EXTENSION.triggerAndWaitSnapshot(tb); + + // batch_size_bytes=1 forces one record per batch via the appendedAny progress guard. + ScanKvResponse first = leaderGateWay.scanKv(newScanKvOpenRequest(tableId, 0, 1)).get(); + assertThat(first.hasErrorCode()).isFalse(); + byte[] scannerId = first.getScannerId(); + long firstLogOffset = first.getLogOffset(); + int totalRecords = + DefaultValueRecordBatch.pointToBytes(first.getRecords()).getRecordCount(); + + ScanKvResponse current = first; + int seq = 0; + while (current.isHasMoreResults()) { + current = + leaderGateWay + .scanKv(newScanKvContinueRequest(scannerId, seq++, /*batch=*/ 1)) + .get(); + assertThat(current.hasErrorCode()).isFalse(); + assertThat(current.getScannerId()).isEqualTo(scannerId); + // The log offset is only carried on the first (open) response; continuations + // must not re-set it. + assertThat(current.hasLogOffset()).isFalse(); + if (current.hasRecords()) { + totalRecords += + DefaultValueRecordBatch.pointToBytes(current.getRecords()) + .getRecordCount(); + } + } + assertThat(totalRecords).isEqualTo(2); + // After draining, the session is auto-closed: a continuation should now be unknown. + ScanKvResponse afterDrain = + leaderGateWay.scanKv(newScanKvContinueRequest(scannerId, seq, 1024)).get(); + assertThat(afterDrain.getErrorCode()).isEqualTo(Errors.UNKNOWN_SCANNER_ID.code()); + assertThat(firstLogOffset).isGreaterThanOrEqualTo(0L); + } + + @Test + void testScanKv_explicitClose() throws Exception { + long tableId = + createTable( + FLUSS_CLUSTER_EXTENSION, DATA1_TABLE_PATH_PK, DATA1_TABLE_DESCRIPTOR_PK); + TableBucket tb = new TableBucket(tableId, 0); + FLUSS_CLUSTER_EXTENSION.waitUntilAllReplicaReady(tb); + int leader = FLUSS_CLUSTER_EXTENSION.waitAndGetLeader(tb); + TabletServerGateway leaderGateWay = + FLUSS_CLUSTER_EXTENSION.newTabletServerClientForNode(leader); + + assertPutKvResponse( + leaderGateWay + .putKv( + newPutKvRequest( + tableId, 0, 1, genKvRecordBatch(DATA_1_WITH_KEY_AND_VALUE))) + .get()); + FLUSS_CLUSTER_EXTENSION.triggerAndWaitSnapshot(tb); + + // Use a tiny batch so the session stays open after the first response. + ScanKvResponse open = leaderGateWay.scanKv(newScanKvOpenRequest(tableId, 0, 1)).get(); + assertThat(open.hasErrorCode()).isFalse(); + assertThat(open.isHasMoreResults()).isTrue(); + byte[] scannerId = open.getScannerId(); + + ScanKvResponse close = leaderGateWay.scanKv(newScanKvCloseRequest(scannerId)).get(); + assertThat(close.hasErrorCode()).isFalse(); + assertThat(close.getScannerId()).isEqualTo(scannerId); + assertThat(close.isHasMoreResults()).isFalse(); + // No records on a close response. + assertThat(close.hasRecords()).isFalse(); + } + + @Test + void testScanKv_closeOnAlreadyGoneScannerIsNoOp() throws Exception { + long tableId = + createTable( + FLUSS_CLUSTER_EXTENSION, DATA1_TABLE_PATH_PK, DATA1_TABLE_DESCRIPTOR_PK); + TableBucket tb = new TableBucket(tableId, 0); + FLUSS_CLUSTER_EXTENSION.waitUntilAllReplicaReady(tb); + int leader = FLUSS_CLUSTER_EXTENSION.waitAndGetLeader(tb); + TabletServerGateway leaderGateWay = + FLUSS_CLUSTER_EXTENSION.newTabletServerClientForNode(leader); + + assertPutKvResponse( + leaderGateWay + .putKv( + newPutKvRequest( + tableId, 0, 1, genKvRecordBatch(DATA_1_WITH_KEY_AND_VALUE))) + .get()); + FLUSS_CLUSTER_EXTENSION.triggerAndWaitSnapshot(tb); + + ScanKvResponse open = leaderGateWay.scanKv(newScanKvOpenRequest(tableId, 0, 1)).get(); + byte[] scannerId = open.getScannerId(); + + // First close removes the session. + leaderGateWay.scanKv(newScanKvCloseRequest(scannerId)).get(); + + // Second close on the already-gone session must be a benign no-op (not an error). + ScanKvResponse second = leaderGateWay.scanKv(newScanKvCloseRequest(scannerId)).get(); + assertThat(second.hasErrorCode()).isFalse(); + assertThat(second.getScannerId()).isEqualTo(scannerId); + assertThat(second.isHasMoreResults()).isFalse(); + } + + @Test + void testScanKv_unknownScannerId() throws Exception { + long tableId = + createTable( + FLUSS_CLUSTER_EXTENSION, DATA1_TABLE_PATH_PK, DATA1_TABLE_DESCRIPTOR_PK); + TableBucket tb = new TableBucket(tableId, 0); + FLUSS_CLUSTER_EXTENSION.waitUntilAllReplicaReady(tb); + int leader = FLUSS_CLUSTER_EXTENSION.waitAndGetLeader(tb); + TabletServerGateway leaderGateWay = + FLUSS_CLUSTER_EXTENSION.newTabletServerClientForNode(leader); + + // No openScan was ever issued — this id is fabricated. + byte[] fakeId = "not-a-real-scanner-id".getBytes(); + ScanKvResponse response = + leaderGateWay.scanKv(newScanKvContinueRequest(fakeId, 0, 1024)).get(); + assertThat(response.getErrorCode()).isEqualTo(Errors.UNKNOWN_SCANNER_ID.code()); + assertThat(response.getErrorMessage()).contains("Unknown scanner ID"); + } + + @Test + void testScanKv_expiredOnRecentlyEvicted() throws Exception { + long tableId = + createTable( + FLUSS_CLUSTER_EXTENSION, DATA1_TABLE_PATH_PK, DATA1_TABLE_DESCRIPTOR_PK); + TableBucket tb = new TableBucket(tableId, 0); + FLUSS_CLUSTER_EXTENSION.waitUntilAllReplicaReady(tb); + int leader = FLUSS_CLUSTER_EXTENSION.waitAndGetLeader(tb); + TabletServerGateway leaderGateWay = + FLUSS_CLUSTER_EXTENSION.newTabletServerClientForNode(leader); + + assertPutKvResponse( + leaderGateWay + .putKv( + newPutKvRequest( + tableId, 0, 1, genKvRecordBatch(DATA_1_WITH_KEY_AND_VALUE))) + .get()); + FLUSS_CLUSTER_EXTENSION.triggerAndWaitSnapshot(tb); + + // Open a long-lived session. + ScanKvResponse open = leaderGateWay.scanKv(newScanKvOpenRequest(tableId, 0, 1)).get(); + assertThat(open.hasErrorCode()).isFalse(); + assertThat(open.isHasMoreResults()).isTrue(); + byte[] scannerId = open.getScannerId(); + + // Evict the session by stopping the leader replica (delete=false). Internally this + // calls scannerManager.closeScannersForBucket, which both removes the session and + // records the id in recentlyExpiredIds. The bucket itself remains hosted. + LeaderAndIsr la = FLUSS_CLUSTER_EXTENSION.waitLeaderAndIsrReady(tb); + stopReplicaWithLatestEpoch(leaderGateWay, tb, la); + + // Continuation against the (still-live) tablet server now sees SCANNER_EXPIRED, not + // UNKNOWN_SCANNER_ID — the recently-expired cache lets the client distinguish + // "session was reaped" from "you fabricated an id". + ScanKvResponse cont = + leaderGateWay.scanKv(newScanKvContinueRequest(scannerId, 0, 1024)).get(); + assertThat(cont.getErrorCode()).isEqualTo(Errors.SCANNER_EXPIRED.code()); + assertThat(cont.getErrorMessage()).contains("expired"); + } + + /** + * Models a mid-scan leadership flip via {@code stopReplica(delete=false)}, which is the + * RPC path the coordinator drives when transitioning a leader away. Internally this + * calls {@code scannerManager.closeScannersForBucket}, the same chain a real leader + * -> follower transition takes via {@code makeFollower -> dropKv}, so the + * client-observable signal is identical. + */ + @Test + void testScanKv_leadershipFlipMidScan() throws Exception { + long tableId = + createTable( + FLUSS_CLUSTER_EXTENSION, DATA1_TABLE_PATH_PK, DATA1_TABLE_DESCRIPTOR_PK); + TableBucket tb = new TableBucket(tableId, 0); + FLUSS_CLUSTER_EXTENSION.waitUntilAllReplicaReady(tb); + int leader = FLUSS_CLUSTER_EXTENSION.waitAndGetLeader(tb); + TabletServerGateway leaderGateWay = + FLUSS_CLUSTER_EXTENSION.newTabletServerClientForNode(leader); + + assertPutKvResponse( + leaderGateWay + .putKv( + newPutKvRequest( + tableId, 0, 1, genKvRecordBatch(DATA_1_WITH_KEY_AND_VALUE))) + .get()); + FLUSS_CLUSTER_EXTENSION.triggerAndWaitSnapshot(tb); + + ScanKvResponse open = leaderGateWay.scanKv(newScanKvOpenRequest(tableId, 0, 1)).get(); + assertThat(open.hasErrorCode()).isFalse(); + assertThat(open.isHasMoreResults()).isTrue(); + byte[] scannerId = open.getScannerId(); + + // Drive the leader-side teardown. closeScannersForBucket runs synchronously inside + // ReplicaManager.stopReplicas and records the scanner id in recentlyExpiredIds + // before stopReplica returns, so by the time the next continuation arrives the + // session is gone but addressable as expired. + LeaderAndIsr la = FLUSS_CLUSTER_EXTENSION.waitLeaderAndIsrReady(tb); + stopReplicaWithLatestEpoch(leaderGateWay, tb, la); + + // The continuation now sees a redirect-style signal so the client knows to find + // the new leader rather than retrying blindly. SCANNER_EXPIRED is the canonical + // response from the recently-expired cache; NOT_LEADER_OR_FOLLOWER is acceptable + // if the in-handler leadership re-check ever wins the race. + ScanKvResponse cont = + leaderGateWay.scanKv(newScanKvContinueRequest(scannerId, 0, 1024)).get(); + assertThat(cont.getErrorCode()) + .isIn(Errors.SCANNER_EXPIRED.code(), Errors.NOT_LEADER_OR_FOLLOWER.code()); + } + + @Test + void testScanKv_callSeqIdMismatch() throws Exception { + long tableId = + createTable( + FLUSS_CLUSTER_EXTENSION, DATA1_TABLE_PATH_PK, DATA1_TABLE_DESCRIPTOR_PK); + TableBucket tb = new TableBucket(tableId, 0); + FLUSS_CLUSTER_EXTENSION.waitUntilAllReplicaReady(tb); + int leader = FLUSS_CLUSTER_EXTENSION.waitAndGetLeader(tb); + TabletServerGateway leaderGateWay = + FLUSS_CLUSTER_EXTENSION.newTabletServerClientForNode(leader); + + assertPutKvResponse( + leaderGateWay + .putKv( + newPutKvRequest( + tableId, 0, 1, genKvRecordBatch(DATA_1_WITH_KEY_AND_VALUE))) + .get()); + FLUSS_CLUSTER_EXTENSION.triggerAndWaitSnapshot(tb); + + ScanKvResponse open = leaderGateWay.scanKv(newScanKvOpenRequest(tableId, 0, 1)).get(); + byte[] scannerId = open.getScannerId(); + + // Initial server callSeqId is -1, so the first continuation MUST send 0. Sending 5 + // is detected as out-of-order and rejected. + ScanKvResponse bad = + leaderGateWay.scanKv(newScanKvContinueRequest(scannerId, 5, 1024)).get(); + assertThat(bad.getErrorCode()).isEqualTo(Errors.INVALID_SCAN_REQUEST.code()); + assertThat(bad.getErrorMessage()).contains("Out-of-order"); + } + + @Test + void testScanKv_oversizeBatchSizeBytesIsClamped() throws Exception { + long tableId = + createTable( + FLUSS_CLUSTER_EXTENSION, DATA1_TABLE_PATH_PK, DATA1_TABLE_DESCRIPTOR_PK); + TableBucket tb = new TableBucket(tableId, 0); + FLUSS_CLUSTER_EXTENSION.waitUntilAllReplicaReady(tb); + int leader = FLUSS_CLUSTER_EXTENSION.waitAndGetLeader(tb); + TabletServerGateway leaderGateWay = + FLUSS_CLUSTER_EXTENSION.newTabletServerClientForNode(leader); + + assertPutKvResponse( + leaderGateWay + .putKv( + newPutKvRequest( + tableId, 0, 1, genKvRecordBatch(DATA_1_WITH_KEY_AND_VALUE))) + .get()); + FLUSS_CLUSTER_EXTENSION.triggerAndWaitSnapshot(tb); + + // Integer.MAX_VALUE >> kv.scanner.max-batch-size (10 MiB by default). The server + // must silently clamp and serve the request — not reject it and not OOM. + ScanKvResponse response = + leaderGateWay + .scanKv(newScanKvOpenRequest(tableId, 0, Integer.MAX_VALUE)) + .get(); + assertThat(response.hasErrorCode()).isFalse(); + assertThat(response.hasRecords()).isTrue(); + assertThat(DefaultValueRecordBatch.pointToBytes(response.getRecords()).getRecordCount()) + .isEqualTo(2); + } + + @Test + void testScanKv_bucketScanReqAndScannerIdRejected() throws Exception { + long tableId = + createTable( + FLUSS_CLUSTER_EXTENSION, DATA1_TABLE_PATH_PK, DATA1_TABLE_DESCRIPTOR_PK); + TableBucket tb = new TableBucket(tableId, 0); + FLUSS_CLUSTER_EXTENSION.waitUntilAllReplicaReady(tb); + int leader = FLUSS_CLUSTER_EXTENSION.waitAndGetLeader(tb); + TabletServerGateway leaderGateWay = + FLUSS_CLUSTER_EXTENSION.newTabletServerClientForNode(leader); + + // Setting both bucket_scan_req (new scan) AND scanner_id (continuation) is an + // ambiguous request and must be rejected at the head of scanKv before any state + // is mutated. + ScanKvRequest bad = new ScanKvRequest(); + bad.setBucketScanReq().setTableId(tableId).setBucketId(0); + bad.setScannerId("ambiguous".getBytes()); + bad.setBatchSizeBytes(1024); + ScanKvResponse response = leaderGateWay.scanKv(bad).get(); + assertThat(response.getErrorCode()).isEqualTo(Errors.INVALID_SCAN_REQUEST.code()); + assertThat(response.getErrorMessage()) + .contains("must not set both bucket_scan_req and scanner_id"); + } + + // ------------------------------------------------------------------------- + // scanKv request helpers + // ------------------------------------------------------------------------- + + private static ScanKvRequest newScanKvOpenRequest(long tableId, int bucketId, int batchSize) { + ScanKvRequest req = new ScanKvRequest(); + req.setBucketScanReq().setTableId(tableId).setBucketId(bucketId); + req.setBatchSizeBytes(batchSize); + return req; + } + + private static ScanKvRequest newScanKvContinueRequest( + byte[] scannerId, int callSeqId, int batchSize) { + ScanKvRequest req = new ScanKvRequest(); + req.setScannerId(scannerId); + req.setCallSeqId(callSeqId); + req.setBatchSizeBytes(batchSize); + return req; + } + + private static ScanKvRequest newScanKvCloseRequest(byte[] scannerId) { + ScanKvRequest req = new ScanKvRequest(); + req.setScannerId(scannerId); + req.setCloseScanner(true); + return req; + } + + /** + * Sends a stopReplica RPC for the given bucket using the coordinator epoch carried in + * {@code la}. The {@code FLUSS_CLUSTER_EXTENSION.stopReplica} helper hardcodes + * {@code coordinatorEpoch=0}, which is rejected by tablet servers once any earlier + * test in the same JVM has bumped the coordinator epoch above zero. + */ + private static void stopReplicaWithLatestEpoch( + TabletServerGateway leaderGateWay, TableBucket tb, LeaderAndIsr la) throws Exception { + leaderGateWay + .stopReplica( + new StopReplicaRequest() + .setCoordinatorEpoch(la.coordinatorEpoch()) + .addAllStopReplicasReqs( + Collections.singleton( + makeStopBucketReplica( + tb, false, false, la.leaderEpoch())))) + .get(); + } } + From bcf1357cacfba76d7ffc1bd206c865554e58b847 Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Wed, 6 May 2026 14:57:13 +0300 Subject: [PATCH 11/17] fix spotless --- .../server/tablet/TabletServiceITCase.java | 30 ++++++++----------- 1 file changed, 12 insertions(+), 18 deletions(-) diff --git a/fluss-server/src/test/java/org/apache/fluss/server/tablet/TabletServiceITCase.java b/fluss-server/src/test/java/org/apache/fluss/server/tablet/TabletServiceITCase.java index bd55349698..6479843857 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/tablet/TabletServiceITCase.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/tablet/TabletServiceITCase.java @@ -51,9 +51,9 @@ import org.apache.fluss.rpc.messages.PbPutKvRespForBucket; import org.apache.fluss.rpc.messages.ProduceLogResponse; import org.apache.fluss.rpc.messages.PutKvResponse; -import org.apache.fluss.rpc.messages.StopReplicaRequest; import org.apache.fluss.rpc.messages.ScanKvRequest; import org.apache.fluss.rpc.messages.ScanKvResponse; +import org.apache.fluss.rpc.messages.StopReplicaRequest; import org.apache.fluss.rpc.protocol.Errors; import org.apache.fluss.server.entity.NotifyLeaderAndIsrData; import org.apache.fluss.server.entity.NotifyLeaderAndIsrResultForBucket; @@ -1119,8 +1119,7 @@ void testScanKv_newScan_happyPath() throws Exception { assertThat(response.hasLogOffset()).isTrue(); assertThat(response.getLogOffset()).isGreaterThanOrEqualTo(0L); assertThat(response.hasRecords()).isTrue(); - DefaultValueRecordBatch batch = - DefaultValueRecordBatch.pointToBytes(response.getRecords()); + DefaultValueRecordBatch batch = DefaultValueRecordBatch.pointToBytes(response.getRecords()); assertThat(batch.getRecordCount()).isEqualTo(2); } @@ -1165,8 +1164,7 @@ void testScanKv_multiBatchContinuation() throws Exception { assertThat(current.hasLogOffset()).isFalse(); if (current.hasRecords()) { totalRecords += - DefaultValueRecordBatch.pointToBytes(current.getRecords()) - .getRecordCount(); + DefaultValueRecordBatch.pointToBytes(current.getRecords()).getRecordCount(); } } assertThat(totalRecords).isEqualTo(2); @@ -1302,11 +1300,10 @@ void testScanKv_expiredOnRecentlyEvicted() throws Exception { } /** - * Models a mid-scan leadership flip via {@code stopReplica(delete=false)}, which is the - * RPC path the coordinator drives when transitioning a leader away. Internally this - * calls {@code scannerManager.closeScannersForBucket}, the same chain a real leader - * -> follower transition takes via {@code makeFollower -> dropKv}, so the - * client-observable signal is identical. + * Models a mid-scan leadership flip via {@code stopReplica(delete=false)}, which is the RPC + * path the coordinator drives when transitioning a leader away. Internally this calls {@code + * scannerManager.closeScannersForBucket}, the same chain a real leader -> follower transition + * takes via {@code makeFollower -> dropKv}, so the client-observable signal is identical. */ @Test void testScanKv_leadershipFlipMidScan() throws Exception { @@ -1401,9 +1398,7 @@ void testScanKv_oversizeBatchSizeBytesIsClamped() throws Exception { // Integer.MAX_VALUE >> kv.scanner.max-batch-size (10 MiB by default). The server // must silently clamp and serve the request — not reject it and not OOM. ScanKvResponse response = - leaderGateWay - .scanKv(newScanKvOpenRequest(tableId, 0, Integer.MAX_VALUE)) - .get(); + leaderGateWay.scanKv(newScanKvOpenRequest(tableId, 0, Integer.MAX_VALUE)).get(); assertThat(response.hasErrorCode()).isFalse(); assertThat(response.hasRecords()).isTrue(); assertThat(DefaultValueRecordBatch.pointToBytes(response.getRecords()).getRecordCount()) @@ -1462,10 +1457,10 @@ private static ScanKvRequest newScanKvCloseRequest(byte[] scannerId) { } /** - * Sends a stopReplica RPC for the given bucket using the coordinator epoch carried in - * {@code la}. The {@code FLUSS_CLUSTER_EXTENSION.stopReplica} helper hardcodes - * {@code coordinatorEpoch=0}, which is rejected by tablet servers once any earlier - * test in the same JVM has bumped the coordinator epoch above zero. + * Sends a stopReplica RPC for the given bucket using the coordinator epoch carried in {@code + * la}. The {@code FLUSS_CLUSTER_EXTENSION.stopReplica} helper hardcodes {@code + * coordinatorEpoch=0}, which is rejected by tablet servers once any earlier test in the same + * JVM has bumped the coordinator epoch above zero. */ private static void stopReplicaWithLatestEpoch( TabletServerGateway leaderGateWay, TableBucket tb, LeaderAndIsr la) throws Exception { @@ -1480,4 +1475,3 @@ private static void stopReplicaWithLatestEpoch( .get(); } } - From 3e04fbed7a4fd279eb28f7e9945e4d3939c7731d Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Wed, 6 May 2026 15:50:29 +0300 Subject: [PATCH 12/17] Serialise scanKv per scannerId; defer TTL refresh until validation; fix snapshot leak --- .../fluss/server/kv/scan/ScannerContext.java | 48 +++++++++- .../fluss/server/kv/scan/ScannerManager.java | 22 +++-- .../fluss/server/tablet/TabletService.java | 57 +++++++++--- .../server/kv/scan/ScannerManagerTest.java | 93 ++++++++++++++++++- 4 files changed, 195 insertions(+), 25 deletions(-) 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 de1a6eb3ab..eb535f9b52 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 @@ -44,7 +44,11 @@ * explicit close, or the session expires due to inactivity. * *

Thread safety: The iterator cursor ({@link #advance()}, {@link #isValid()}, {@link - * #currentValue()}) must be driven by only one thread at a time. {@link #close()} is thread-safe. + * #currentValue()}) must be driven by only one thread at a time. Callers that mutate cursor or + * sequence-id state must first acquire exclusive use via {@link #tryAcquireForUse()} and release it + * with {@link #releaseAfterUse()} when done; this prevents two concurrent client RPCs sharing a + * scanner ID from racing the iterator (which would corrupt RocksDB state at the JNI boundary). + * {@link #close()} is thread-safe. */ @NotThreadSafe public class ScannerContext implements Closeable { @@ -82,6 +86,15 @@ public class ScannerContext implements Closeable { private final AtomicBoolean closed = new AtomicBoolean(false); + /** + * Cursor-exclusion flag. Two concurrent {@code scanKv} RPCs sharing a scanner ID would both + * pass the {@code callSeqId} check (each sees the same pre-mutation value) and then race on + * {@link RocksIterator#next()}, which is unsafe at the JNI boundary. Acquiring this flag at the + * top of the handler and releasing it in {@code finally} serialises the cursor without blocking + * — the loser of the CAS receives an explicit "concurrent scan" error. + */ + private final AtomicBoolean inUse = new AtomicBoolean(false); + public ScannerContext( String scannerId, TableBucket tableBucket, @@ -176,6 +189,28 @@ public boolean isExpired(long ttlMs, long nowMs) { return nowMs - lastAccessTime > ttlMs; } + /** + * Atomically tries to claim exclusive use of this context's cursor and sequence-id state. + * Returns {@code true} if the caller now holds the exclusive-use flag and may safely advance + * the iterator and update {@link #setCallSeqId(int)}; {@code false} if another thread is + * already inside a {@code scanKv} call for this scanner ID. + * + *

Callers MUST pair every successful acquire with a {@link #releaseAfterUse()} in a {@code + * finally} block. + */ + public boolean tryAcquireForUse() { + return inUse.compareAndSet(false, true); + } + + /** + * Releases the exclusive-use flag obtained via {@link #tryAcquireForUse()}. Calling this on a + * context that has already been {@link #close()}d is harmless: the context is no longer + * reachable through {@link ScannerManager}, so no other thread will observe the flag. + */ + public void releaseAfterUse() { + inUse.set(false); + } + @Override public void close() { if (closed.compareAndSet(false, true)) { @@ -186,8 +221,15 @@ public void close() { readOptions.close(); } finally { try { - rocksDBKv.getDb().releaseSnapshot(snapshot); - snapshot.close(); + // Wrap releaseSnapshot in its own try/finally so that snapshot.close() + // always runs even if releaseSnapshot throws — otherwise the native + // snapshot handle would leak while the ResourceGuard.Lease is still + // released, masking the leak from later kvTablet.close() shutdown checks. + try { + rocksDBKv.getDb().releaseSnapshot(snapshot); + } finally { + snapshot.close(); + } } 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 c641aeb0ce..aca4a82b6a 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 @@ -195,19 +195,27 @@ public void register(ScannerContext context) { } /** - * Looks up an existing scanner session by its raw ID bytes and refreshes its last-access - * timestamp. + * Looks up an existing scanner session by its raw ID bytes. Does not refresh the + * last-access timestamp — callers must invoke {@link #markAccessed(ScannerContext)} only after + * the request has been validated, otherwise an invalid request (bad call-sequence id, missing + * batch size, leadership lost, etc.) would silently extend the session TTL and let an orphan + * scanner survive past its idle deadline. * * @return the {@link ScannerContext}, or {@code null} if not found (may have expired or never * existed) */ @Nullable public ScannerContext getScanner(byte[] scannerId) { - ScannerContext context = scanners.get(new String(scannerId, StandardCharsets.UTF_8)); - if (context != null) { - context.updateLastAccessTime(clock.milliseconds()); - } - return context; + return scanners.get(new String(scannerId, StandardCharsets.UTF_8)); + } + + /** + * Refreshes the last-access timestamp on the given context. Must be called only when the caller + * has decided the request is well-formed and is about to do real work, so that rejected + * requests do not extend the idle TTL. + */ + public void markAccessed(ScannerContext context) { + context.updateLastAccessTime(clock.milliseconds()); } /** 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 c52fc60b6e..f8b07fffbc 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 @@ -464,6 +464,9 @@ public CompletableFuture scanKv(ScanKvRequest request) { // tear the session down and force the client to restart. This matches the non-resumable // contract documented on ScannerExpiredException. ScannerContext openedContext = null; + // Tracks the context whose `inUse` flag we own so that finally{} can release it. May + // diverge from openedContext (which is nulled out on success) so we keep it separate. + ScannerContext acquiredContext = null; try { ScannerContext context; // True only on the initial (bucket_scan_req) path, where we must echo the snapshot @@ -542,22 +545,37 @@ public CompletableFuture scanKv(ScanKvRequest request) { + "never existed."); } } - // Validate call-sequence ordering to detect duplicate or out-of-order requests. - // getScanner() already refreshed the last-access timestamp. Use long arithmetic to - // avoid a silent 32-bit overflow at Integer.MAX_VALUE continuations. - if (request.hasCallSeqId()) { - long expectedSeqId = (long) context.getCallSeqId() + 1L; - int requestSeqId = request.getCallSeqId(); - if ((long) requestSeqId != expectedSeqId) { - throw new InvalidScanRequestException( - String.format( - "Out-of-order scan request: expected callSeqId=%d but got %d.", - expectedSeqId, requestSeqId)); - } - } openedContext = context; } + // Acquire single-thread access to the cursor before any state mutation. Without + // this, two concurrent scanKv RPCs sharing a scannerId would both observe the same + // pre-mutation callSeqId, both pass the in-order check, and both race + // iterator.next() - corrupting RocksDB state at the JNI boundary. The loser of the + // CAS gets a clear error so the client can retry sequentially. + if (!context.tryAcquireForUse()) { + throw new InvalidScanRequestException( + String.format( + "Concurrent scan request on scanner ID for bucket %s; only one " + + "in-flight scanKv RPC per scanner is allowed.", + context.getTableBucket())); + } + acquiredContext = context; + + // Validate call-sequence ordering to detect duplicate or out-of-order requests. + // Use long arithmetic to avoid a silent 32-bit overflow at Integer.MAX_VALUE + // continuations. Skipped on the new-scan path (no scannerId on the wire). + if (!request.hasBucketScanReq() && request.hasCallSeqId()) { + long expectedSeqId = (long) context.getCallSeqId() + 1L; + int requestSeqId = request.getCallSeqId(); + if ((long) requestSeqId != expectedSeqId) { + throw new InvalidScanRequestException( + String.format( + "Out-of-order scan request: expected callSeqId=%d but got %d.", + expectedSeqId, requestSeqId)); + } + } + // Handle explicit close request. Honour the close even on a non-leader: the local // session resources are still ours to release, and we don't want a leadership flip // racing with a close to leak a snapshot. @@ -603,6 +621,12 @@ public CompletableFuture scanKv(ScanKvRequest request) { } int effectiveBatchSize = Math.min(requestedBatchSize, kvScanMaxBatchSizeBytes); + // Refresh the idle-TTL deadline only now that the request is fully validated and we + // are about to do real work. Earlier refreshes (during getScanner) would let any + // malformed request - bad callSeqId, missing batch size, lost leadership - keep an + // orphan session alive past its idle deadline. + scannerManager.markAccessed(context); + // Build the next batch using the builder's own running size as the gating signal so // the threshold reflects the actual serialised batch (header + per-record framing), // not just the raw value bytes. We always append at least one record (when data is @@ -654,6 +678,13 @@ public CompletableFuture scanKv(ScanKvRequest request) { response.setErrorCode(apiError.error().code()); response.setErrorMessage(apiError.message() != null ? apiError.message() : ""); } finally { + // Release the cursor-exclusion flag before any force-close: the close path is + // CAS-guarded on its own `closed` flag, so the order is purely cosmetic, but + // releasing first keeps the invariant "inUse=true means a thread is mid-handler" + // tight. Calling releaseAfterUse on an already-closed context is a no-op. + if (acquiredContext != null) { + acquiredContext.releaseAfterUse(); + } // If we made it past createScanner/getScanner but failed to deliver a complete // response, close the session rather than leaking it to TTL. The cursor has // already advanced past rows whose values were never sent; resuming would drop diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java index aff7dd5b62..a61762718f 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java @@ -260,7 +260,7 @@ void testCreateAndRemoveScanner() throws Exception { } @Test - void testGetScanner_refreshesLastAccessTime() throws Exception { + void testGetScanner_doesNotRefreshLastAccessTime() throws Exception { putAndFlush(3); try (ScannerManager manager = createManager()) { // Create scanner at t=0. @@ -268,11 +268,31 @@ void testGetScanner_refreshesLastAccessTime() throws Exception { assertThat(context).isNotNull(); byte[] scannerId = context.getScannerId(); - // Advance clock far past any TTL, then getScanner to refresh. + // Advance clock far past the initial access time and look up the scanner. clock.advanceTime(5000, TimeUnit.MILLISECONDS); ScannerContext fetched = manager.getScanner(scannerId); assertThat(fetched).isSameAs(context); + // getScanner alone must NOT refresh the last-access time: an invalid request + // (bad callSeqId, missing batch size, lost leadership) would otherwise extend + // the idle TTL and let an orphan session survive past its deadline. + assertThat(context.isExpired(1000L, clock.milliseconds())).isTrue(); + + manager.removeScanner(context); + } + } + + @Test + void testMarkAccessed_refreshesLastAccessTime() throws Exception { + putAndFlush(3); + try (ScannerManager manager = createManager()) { + ScannerContext context = openAndRegister(manager); + assertThat(context).isNotNull(); + + clock.advanceTime(5000, TimeUnit.MILLISECONDS); + // Must explicitly mark the session as accessed; lookup is non-mutating. + manager.markAccessed(context); + // With a 1-hour TTL, isExpired must be false right after the refresh. assertThat(context.isExpired(3_600_000L, clock.milliseconds())).isFalse(); @@ -280,6 +300,75 @@ void testGetScanner_refreshesLastAccessTime() throws Exception { } } + @Test + void testTryAcquireForUse_serialisesConcurrentClaims() throws Exception { + putAndFlush(3); + try (ScannerManager manager = createManager()) { + ScannerContext context = openAndRegister(manager); + assertThat(context).isNotNull(); + + // First claim wins. + assertThat(context.tryAcquireForUse()).isTrue(); + // A second concurrent claim must fail until the first releases. + assertThat(context.tryAcquireForUse()).isFalse(); + + context.releaseAfterUse(); + + // After release, a fresh claim succeeds again. + assertThat(context.tryAcquireForUse()).isTrue(); + context.releaseAfterUse(); + + manager.removeScanner(context); + } + } + + /** + * Stress-test the cursor-exclusion CAS: many threads race to acquire a single context, and + * exactly one must win. This is the invariant that protects {@link + * org.apache.fluss.server.tablet.TabletService#scanKv} from concurrent same-scannerId RPCs + * racing the RocksDB iterator at the JNI boundary. + */ + @Test + void testTryAcquireForUse_exactlyOneWinnerUnderContention() throws Exception { + putAndFlush(3); + try (ScannerManager manager = createManager()) { + ScannerContext context = openAndRegister(manager); + assertThat(context).isNotNull(); + + int threadCount = 16; + java.util.concurrent.CountDownLatch start = new java.util.concurrent.CountDownLatch(1); + java.util.concurrent.atomic.AtomicInteger winners = + new java.util.concurrent.atomic.AtomicInteger(0); + java.util.concurrent.ExecutorService pool = + java.util.concurrent.Executors.newFixedThreadPool(threadCount); + try { + for (int i = 0; i < threadCount; i++) { + pool.submit( + () -> { + try { + start.await(); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + return; + } + if (context.tryAcquireForUse()) { + winners.incrementAndGet(); + } + }); + } + start.countDown(); + pool.shutdown(); + assertThat(pool.awaitTermination(10, TimeUnit.SECONDS)).isTrue(); + } finally { + pool.shutdownNow(); + } + + assertThat(winners.get()).isEqualTo(1); + context.releaseAfterUse(); + manager.removeScanner(context); + } + } + @Test void testTtlEviction() throws Exception { putAndFlush(3); From 25d630c76c74e8ae3dad93b465373085c3313c4e Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Wed, 6 May 2026 18:05:10 +0300 Subject: [PATCH 13/17] remove the MapUtils as it was intentionally removed in #3113 --- .../java/org/apache/fluss/utils/MapUtils.java | 43 ------------------- .../fluss/server/kv/scan/ScannerContext.java | 21 +++++++++ .../fluss/server/kv/scan/ScannerManager.java | 8 ++-- .../fluss/server/tablet/TabletService.java | 8 ++++ .../server/kv/scan/ScannerManagerTest.java | 27 ++++++++++++ 5 files changed, 60 insertions(+), 47 deletions(-) delete mode 100644 fluss-common/src/main/java/org/apache/fluss/utils/MapUtils.java diff --git a/fluss-common/src/main/java/org/apache/fluss/utils/MapUtils.java b/fluss-common/src/main/java/org/apache/fluss/utils/MapUtils.java deleted file mode 100644 index d9e20855f1..0000000000 --- a/fluss-common/src/main/java/org/apache/fluss/utils/MapUtils.java +++ /dev/null @@ -1,43 +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.utils; - -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; - -/** - * Simple utility to work with Java maps. - * - *

Prefer {@link #newConcurrentMap()} over {@code new ConcurrentHashMap<>()} so that the codebase - * has a single, swappable construction point — see apache/fluss#375. - */ -public class MapUtils { - - /** Returns a new empty {@link ConcurrentHashMap}. */ - public static Map newConcurrentMap() { - return new ConcurrentHashMap<>(); - } - - /** Returns a new empty {@link ConcurrentHashMap} with the given initial capacity. */ - public static Map newConcurrentMap(int initialCapacity) { - return new ConcurrentHashMap<>(initialCapacity); - } - - private MapUtils() {} -} 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 eb535f9b52..abd0c736ff 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,11 +17,13 @@ package org.apache.fluss.server.kv.scan; +import org.apache.fluss.exception.KvStorageException; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.server.kv.rocksdb.RocksDBKv; import org.apache.fluss.server.utils.ResourceGuard; import org.rocksdb.ReadOptions; +import org.rocksdb.RocksDBException; import org.rocksdb.RocksIterator; import org.rocksdb.Snapshot; @@ -163,6 +165,25 @@ public void advance() { } } + /** + * Validates the underlying RocksDB iterator's status. {@link RocksIterator#next()} does not + * throw on RocksDB-internal errors — the iterator silently transitions to invalid and the error + * is carried by {@link RocksIterator#status()}. Callers MUST invoke this once iteration has + * stopped (i.e. {@link #isValid()} returned {@code false}) so that a clean end-of-range is + * distinguishable from an error path; otherwise an internal failure would silently truncate the + * scan and the client would conclude the scan completed when in fact rows were dropped. + * + * @throws KvStorageException if the iterator reports an internal error + */ + public void checkIteratorStatus() { + try { + iterator.status(); + } catch (RocksDBException e) { + throw new KvStorageException( + "RocksDB iterator error during scan for bucket " + tableBucket, e); + } + } + /** Returns the call-sequence ID of the last successfully served request, or {@code -1}. */ public int getCallSeqId() { return callSeqId; 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 aca4a82b6a..b72d06a047 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 @@ -24,7 +24,6 @@ import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.server.replica.Replica; 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.FutureUtils; @@ -43,6 +42,7 @@ import java.util.Map; import java.util.UUID; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -95,11 +95,11 @@ public class ScannerManager implements AutoCloseableAsync { */ private static final AtomicInteger ZERO = new AtomicInteger(0); - private final Map scanners = MapUtils.newConcurrentMap(); - private final Map recentlyExpiredIds = MapUtils.newConcurrentMap(); + private final Map scanners = new ConcurrentHashMap<>(); + private final Map recentlyExpiredIds = new ConcurrentHashMap<>(); /** Per-bucket active scanner count, used for O(1) per-bucket limit enforcement. */ - private final Map perBucketCount = MapUtils.newConcurrentMap(); + private final Map perBucketCount = new ConcurrentHashMap<>(); /** Total active scanner count across all buckets on this tablet server. */ private final AtomicInteger totalScanners = new AtomicInteger(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 f8b07fffbc..2f331d3f3e 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 @@ -643,6 +643,14 @@ public CompletableFuture scanKv(ScanKvRequest request) { } boolean hasMore = context.isValid(); + if (!hasMore) { + // RocksIterator.next() does not throw on internal errors; an unchecked status + // here would silently turn an iterator-internal failure into has_more=false, + // dropping every row past the failure point and letting the client conclude + // the scan completed cleanly. Surface the error so the catch block maps it to + // KV_STORAGE_EXCEPTION and the finally block force-closes the session. + context.checkIteratorStatus(); + } DefaultValueRecordBatch batch = builder.build(); response.setScannerId(context.getScannerId()); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java index a61762718f..91831fcce6 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java @@ -369,6 +369,33 @@ void testTryAcquireForUse_exactlyOneWinnerUnderContention() throws Exception { } } + /** + * On a healthy iterator, {@link ScannerContext#checkIteratorStatus()} must be a no-op so the + * post-loop check in {@link org.apache.fluss.server.tablet.TabletService#scanKv} does not + * spuriously fail clean end-of-range scans. The error path (RocksDB-internal failure) is + * exercised at the integration level — fabricating a JNI-level error is impractical in a unit + * test, so this test pins the no-error contract. + */ + @Test + void testCheckIteratorStatus_healthyIteratorIsNoOp() throws Exception { + putAndFlush(3); + try (ScannerManager manager = createManager()) { + ScannerContext context = openAndRegister(manager); + assertThat(context).isNotNull(); + + // Drain the cursor so isValid() flips to false through the natural end-of-range. + while (context.isValid()) { + context.advance(); + } + + // status() must report ok for an iterator that ended cleanly — otherwise every + // successful scan would surface KV_STORAGE_EXCEPTION at the post-loop guard. + context.checkIteratorStatus(); + + manager.removeScanner(context); + } + } + @Test void testTtlEviction() throws Exception { putAndFlush(3); From 6748c7f41ed02d279ff3bd88bf1f6f0bcf5381eb Mon Sep 17 00:00:00 2001 From: ipolyzos Date: Wed, 6 May 2026 19:16:04 +0300 Subject: [PATCH 14/17] small improvements and cleanup --- .../org/apache/fluss/server/kv/KvTablet.java | 32 +-- .../fluss/server/kv/scan/OpenScanResult.java | 19 +- .../fluss/server/kv/scan/ScannerContext.java | 124 +++++------- .../fluss/server/kv/scan/ScannerManager.java | 187 ++++-------------- .../apache/fluss/server/replica/Replica.java | 54 ++--- .../fluss/server/replica/ReplicaManager.java | 4 - .../fluss/server/tablet/TabletServer.java | 7 +- .../fluss/server/tablet/TabletService.java | 155 +++++---------- .../apache/fluss/server/kv/KvTabletTest.java | 10 +- .../server/kv/scan/ScannerManagerTest.java | 109 +++++----- .../server/replica/ReplicaManagerTest.java | 5 - .../server/tablet/TabletServiceITCase.java | 64 +----- 12 files changed, 211 insertions(+), 559 deletions(-) diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/KvTablet.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvTablet.java index d2da93a696..f380193997 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/KvTablet.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvTablet.java @@ -765,24 +765,13 @@ public List limitScan(int limit) throws IOException { } /** - * Opens a new full-scan session, taking a point-in-time RocksDB snapshot under the {@code - * kvLock} read lock. + * Opens a new full-scan session under the {@code kvLock} read lock. Returns an empty-bucket + * result (context = {@code null}, all RocksDB resources released internally) when the bucket + * has no rows. The returned {@link ScannerContext} is unregistered; the caller owns + * registration and close. * - *

The returned {@link OpenScanResult} always carries the {@link OpenScanResult#getLogOffset - * log offset} captured at snapshot time (the latest record flushed into the KV store). On the - * empty-bucket fast path the result's {@link OpenScanResult#getContext context} is {@code null} - * and all RocksDB resources have been released internally; no session is registered. - * - *

The returned {@link ScannerContext} is unregistered — the caller ({@link - * org.apache.fluss.server.kv.scan.ScannerManager}) is responsible for registering it and for - * closing it when the scan is complete. - * - * @param scannerId the server-assigned scanner ID - * @param limit maximum number of rows to return across all batches ({@code ≤ 0} = unlimited) - * @param initialAccessTimeMs wall-clock time (ms) to use as the initial last-access timestamp - * @return an {@link OpenScanResult} carrying the captured log offset and (for non-empty - * buckets) a cursor-positioned {@link ScannerContext} - * @throws IOException if the ResourceGuard is already closed (RocksDB is shutting down) + * @param limit row-count cap across all batches ({@code ≤ 0} means unlimited) + * @throws IOException if RocksDB is shutting down */ public OpenScanResult openScan(String scannerId, long limit, long initialAccessTimeMs) throws IOException { @@ -797,8 +786,8 @@ public OpenScanResult openScan(String scannerId, long limit, long initialAccessT boolean success = false; try { snapshot = rocksDBKv.getDb().getSnapshot(); - // Capture the flushed log offset under the same lock that gates flushes, - // so the value reflects exactly the data visible through the snapshot. + // Capture under kvLock so the offset matches the data visible through + // the snapshot. long capturedLogOffset = flushedLogOffset; readOptions = new ReadOptions().setSnapshot(snapshot); iterator = @@ -809,9 +798,6 @@ public OpenScanResult openScan(String scannerId, long limit, long initialAccessT readOptions); iterator.seekToFirst(); if (!iterator.isValid()) { - // Empty bucket: no session will be registered; cleanup in finally. - // Return the offset so the empty-bucket fast path can still hand it - // to the client for snapshot-to-log handoff. return new OpenScanResult(null, capturedLogOffset); } ScannerContext context = @@ -830,8 +816,6 @@ public OpenScanResult openScan(String scannerId, long limit, long initialAccessT return new OpenScanResult(context, capturedLogOffset); } finally { if (!success) { - // Release in reverse allocation order. Each close is independent, - // so a failure in one must not prevent the others from running. IOUtils.closeQuietly(iterator); IOUtils.closeQuietly(readOptions); if (snapshot != null) { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/OpenScanResult.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/OpenScanResult.java index 83e6328427..5331a7a1cf 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/OpenScanResult.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/scan/OpenScanResult.java @@ -22,13 +22,9 @@ import javax.annotation.Nullable; /** - * Result of opening a KV full-scan session, returned by {@link - * org.apache.fluss.server.kv.KvTablet#openScan}, {@link - * org.apache.fluss.server.replica.Replica#openScan}, and {@link ScannerManager#createScanner}. - * - *

Carries both the (optional) {@link ScannerContext} and the log offset captured under the same - * lock as the RocksDB snapshot. The log offset must reach the client even on the empty-bucket fast - * path so that a snapshot-to-log handoff can be performed without missing or duplicating records. + * Result of opening a KV full-scan session: an optional {@link ScannerContext} plus the log offset + * captured at snapshot time. The offset must reach the client even on the empty-bucket fast path so + * a snapshot-to-log handoff cannot miss or duplicate records. */ @Internal public final class OpenScanResult { @@ -41,19 +37,12 @@ public OpenScanResult(@Nullable ScannerContext context, long logOffset) { this.logOffset = logOffset; } - /** - * Returns the registered {@link ScannerContext}, or {@code null} if the bucket was empty at - * snapshot time (no session is created in that case). - */ + /** {@code null} if the bucket was empty at snapshot time. */ @Nullable public ScannerContext getContext() { return context; } - /** - * Returns the log offset of the latest record that was flushed into the KV store at the moment - * the RocksDB snapshot was opened. Always non-negative. - */ public long getLogOffset() { return logOffset; } 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 abd0c736ff..112c76d5bd 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 @@ -34,23 +34,13 @@ import java.util.concurrent.atomic.AtomicBoolean; /** - * Server-side state for a single KV full-scan session. + * Server-side state for a single KV full-scan session: a pinned RocksDB {@link Snapshot}, a {@link + * RocksIterator} cursor, and a {@link ResourceGuard.Lease} that keeps the underlying RocksDB alive + * for the lifetime of the scan. * - *

A {@code ScannerContext} holds a point-in-time RocksDB {@link Snapshot}, the {@link - * ReadOptions} pinning it, and a cursor ({@link RocksIterator}) that persists across multiple - * batched-fetch RPCs from the same client. It also holds a {@link ResourceGuard.Lease} that - * prevents the underlying RocksDB instance from being closed while the scan is in progress. - * - *

Instances are created by {@link org.apache.fluss.server.kv.KvTablet#openScan} and registered - * by {@link ScannerManager}. They must be closed when the scan completes, the client requests an - * explicit close, or the session expires due to inactivity. - * - *

Thread safety: The iterator cursor ({@link #advance()}, {@link #isValid()}, {@link - * #currentValue()}) must be driven by only one thread at a time. Callers that mutate cursor or - * sequence-id state must first acquire exclusive use via {@link #tryAcquireForUse()} and release it - * with {@link #releaseAfterUse()} when done; this prevents two concurrent client RPCs sharing a - * scanner ID from racing the iterator (which would corrupt RocksDB state at the JNI boundary). - * {@link #close()} is thread-safe. + *

Cursor methods ({@link #advance()}, {@link #isValid()}, {@link #currentValue()}) are + * single-threaded; callers must hold the {@link #tryAcquireForUse()} flag while mutating cursor or + * sequence-id state. {@link #close()} is thread-safe and fences on that flag. */ @NotThreadSafe public class ScannerContext implements Closeable { @@ -63,38 +53,20 @@ public class ScannerContext implements Closeable { private final Snapshot snapshot; private final ResourceGuard.Lease resourceLease; - /** - * Log offset of the latest record flushed to the KV store at the moment this scanner's RocksDB - * snapshot was opened. Sent to the client on the first response so that downstream consumers - * can perform a consistent snapshot-to-log handoff. - */ + /** Log offset captured when the snapshot was opened. */ private final long logOffset; private long remainingLimit; - // Initial value -1 so that the first client call_seq_id of 0 satisfies the server's - // in-order check: expectedSeqId = callSeqId + 1 = -1 + 1 = 0. - // callSeqId validation is only performed for continuation requests (those carrying a - // scanner_id), never for the initial open request (those carrying a bucket_scan_req). - // volatile because a continuation may be served by a different RPC worker thread than the - // one that last advanced this counter. + + // -1 so the first client call_seq_id of 0 satisfies expectedSeqId = callSeqId + 1. + // volatile: a continuation may run on a different RPC worker than the previous one. private volatile int callSeqId = -1; - /** - * Wall-clock timestamp (ms) of the most recent request that touched this session. Used by - * {@link ScannerManager} for TTL-based eviction. {@code volatile} so the evictor thread cannot - * observe a stale timestamp written by the most recent RPC worker. - */ + /** Last-access wall-clock (ms); volatile for the TTL evictor. */ private volatile long lastAccessTime; private final AtomicBoolean closed = new AtomicBoolean(false); - /** - * Cursor-exclusion flag. Two concurrent {@code scanKv} RPCs sharing a scanner ID would both - * pass the {@code callSeqId} check (each sees the same pre-mutation value) and then race on - * {@link RocksIterator#next()}, which is unsafe at the JNI boundary. Acquiring this flag at the - * top of the handler and releasing it in {@code finally} serialises the cursor without blocking - * — the loser of the CAS receives an explicit "concurrent scan" error. - */ private final AtomicBoolean inUse = new AtomicBoolean(false); public ScannerContext( @@ -121,10 +93,7 @@ public ScannerContext( this.lastAccessTime = initialAccessTimeMs; } - /** - * Returns the log offset captured at the moment this scanner's RocksDB snapshot was opened. See - * {@link #logOffset}. - */ + /** Log offset captured when the snapshot was opened. */ public long getLogOffset() { return logOffset; } @@ -133,11 +102,7 @@ public byte[] getScannerId() { return scannerIdBytes; } - /** - * Returns the scanner ID as a UTF-8 {@link String}. Package-private: used by {@link - * ScannerManager} as the key in its internal {@code scanners} map. The wire-format - * representation is always {@link #getScannerId()} (raw bytes). - */ + /** Scanner ID as a UTF-8 string; used as the key in {@link ScannerManager#scanners}. */ String getIdString() { return scannerId; } @@ -154,10 +119,7 @@ public byte[] currentValue() { return iterator.value(); } - /** - * Advances the cursor by one entry and decrements the remaining-rows limit if applicable. Must - * only be called when {@link #isValid()} returns {@code true}. - */ + /** Advances the cursor; must be called only when {@link #isValid()} is {@code true}. */ public void advance() { iterator.next(); if (remainingLimit > 0) { @@ -166,12 +128,10 @@ public void advance() { } /** - * Validates the underlying RocksDB iterator's status. {@link RocksIterator#next()} does not - * throw on RocksDB-internal errors — the iterator silently transitions to invalid and the error - * is carried by {@link RocksIterator#status()}. Callers MUST invoke this once iteration has - * stopped (i.e. {@link #isValid()} returned {@code false}) so that a clean end-of-range is - * distinguishable from an error path; otherwise an internal failure would silently truncate the - * scan and the client would conclude the scan completed when in fact rows were dropped. + * Surfaces RocksDB-internal iterator errors. {@link RocksIterator#next()} does not throw on + * such errors; instead the iterator silently transitions to invalid. Callers MUST invoke this + * once iteration has stopped, otherwise an internal failure would be indistinguishable from a + * clean end-of-range and the client would think the scan completed when rows were dropped. * * @throws KvStorageException if the iterator reports an internal error */ @@ -190,8 +150,8 @@ public int getCallSeqId() { } /** - * Updates the call-sequence ID. Must be called after the response payload has been - * fully prepared, so that a client retry with the same {@code callSeqId} can be detected. + * Updates the call-sequence ID. Must be called after the response payload is fully + * prepared so a client retry with the same id can be detected. */ public void setCallSeqId(int callSeqId) { this.callSeqId = callSeqId; @@ -202,31 +162,35 @@ public void updateLastAccessTime(long nowMs) { this.lastAccessTime = nowMs; } - /** - * Returns {@code true} if the session has been idle for longer than {@code ttlMs}, based on the - * provided current time. - */ + /** Returns {@code true} if idle for longer than {@code ttlMs}. */ public boolean isExpired(long ttlMs, long nowMs) { return nowMs - lastAccessTime > ttlMs; } /** - * Atomically tries to claim exclusive use of this context's cursor and sequence-id state. - * Returns {@code true} if the caller now holds the exclusive-use flag and may safely advance - * the iterator and update {@link #setCallSeqId(int)}; {@code false} if another thread is - * already inside a {@code scanKv} call for this scanner ID. - * - *

Callers MUST pair every successful acquire with a {@link #releaseAfterUse()} in a {@code - * finally} block. + * Tries to claim exclusive use of the cursor. Returns {@code false} if another thread already + * holds it, or if {@link #close()} has been initiated. Every successful acquire MUST be paired + * with {@link #releaseAfterUse()} in a {@code finally}, otherwise {@code close()} blocks. */ public boolean tryAcquireForUse() { - return inUse.compareAndSet(false, true); + if (closed.get()) { + return false; + } + if (!inUse.compareAndSet(false, true)) { + return false; + } + // Re-check after the CAS in case close() flipped `closed` in between; let close() proceed. + if (closed.get()) { + inUse.set(false); + return false; + } + return true; } /** - * Releases the exclusive-use flag obtained via {@link #tryAcquireForUse()}. Calling this on a - * context that has already been {@link #close()}d is harmless: the context is no longer - * reachable through {@link ScannerManager}, so no other thread will observe the flag. + * Releases the flag obtained via {@link #tryAcquireForUse()}. MUST be called BEFORE any path + * that may trigger {@link #close()} on the same thread, otherwise {@code close()} self- + * deadlocks on its inUse fence. */ public void releaseAfterUse() { inUse.set(false); @@ -235,6 +199,12 @@ public void releaseAfterUse() { @Override public void close() { if (closed.compareAndSet(false, true)) { + // Fence: closing the iterator while another thread is mid iterator.next() is + // undefined at the RocksDB JNI boundary. tryAcquireForUse() re-checks `closed` + // after winning its CAS, so any racing waiter releases inUse and lets us through. + while (inUse.get()) { + Thread.yield(); + } try { iterator.close(); } finally { @@ -242,10 +212,6 @@ public void close() { readOptions.close(); } finally { try { - // Wrap releaseSnapshot in its own try/finally so that snapshot.close() - // always runs even if releaseSnapshot throws — otherwise the native - // snapshot handle would leak while the ResourceGuard.Lease is still - // released, masking the leak from later kvTablet.close() shutdown checks. try { rocksDBKv.getDb().releaseSnapshot(snapshot); } finally { 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 b72d06a047..145fb307a8 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 @@ -48,66 +48,31 @@ import java.util.concurrent.atomic.AtomicInteger; /** - * Manages server-side KV full-scan sessions ({@link ScannerContext}). + * Manages server-side KV full-scan sessions ({@link ScannerContext}). Sessions are keyed by a + * server-assigned UUID and persist across multiple batched-fetch RPCs. * - *

Each KV full scan opens a persistent server-side session that holds a point-in-time RocksDB - * snapshot and a cursor. Sessions are keyed by a server-assigned UUID-based scanner ID and persist - * across multiple batched-fetch RPCs from the same client. + *

Concurrency limits are enforced both per-bucket and per-server, with a pre-check followed by + * an atomic increment + re-check (rolled back on contention) so the configured caps cannot be + * permanently breached. Sessions idle longer than {@code kv.scanner.ttl} are evicted by a + * background task; their IDs are retained for {@code 2 × ttl} so that {@link + * #isRecentlyExpired(byte[])} can distinguish expired sessions from unknown ones. * - *

Concurrency limits

- * - *
    - *
  • Per-bucket: at most {@code maxPerBucket} concurrent sessions on any single bucket. - *
  • Per-server: at most {@code maxPerServer} concurrent sessions across all buckets. - *
- * - *

Limit enforcement is two-phase: a fast pre-check guards the common case; the subsequent atomic - * increment with re-check and rollback prevents the TOCTOU race from permanently breaching the - * configured limits. Exceeding either limit causes {@link TooManyScannersException}. - * - *

Empty bucket handling

- * - *

If the target bucket contains no rows at the time the scan is opened, {@link - * #createScanner(Replica, Long)} returns an {@link OpenScanResult} whose {@link - * OpenScanResult#getContext context} is {@code null} without consuming a limit slot. The caller - * should return an empty response immediately, echoing back {@link OpenScanResult#getLogOffset}. - * - *

TTL eviction

- * - *

A background evictor task runs every {@code kv.scanner.expiration-interval} and removes - * sessions idle longer than {@code kv.scanner.ttl}. Recently evicted IDs are retained for {@code 2 - * × ttl} so callers can distinguish "expired" from "never existed." - * - *

Leadership change

- * - * {@link #closeScannersForBucket(TableBucket)} must be called when a bucket loses leadership to - * release all RocksDB snapshot/iterator resources for that bucket promptly. + *

{@link #closeScannersForBucket(TableBucket)} must be called when a bucket loses leadership to + * release RocksDB resources promptly. */ @ThreadSafe public class ScannerManager implements AutoCloseableAsync { private static final Logger LOG = LoggerFactory.getLogger(ScannerManager.class); - /** - * Sentinel zero counter returned by {@link #activeScannerCountForBucket(TableBucket)} and - * {@link #checkLimits(TableBucket)} when a bucket has no entry in {@link #perBucketCount}, to - * avoid creating a fresh {@link AtomicInteger} for purely-read paths. - */ + /** Sentinel for read paths so a missing bucket entry does not allocate a counter. */ private static final AtomicInteger ZERO = new AtomicInteger(0); private final Map scanners = new ConcurrentHashMap<>(); private final Map recentlyExpiredIds = new ConcurrentHashMap<>(); - - /** Per-bucket active scanner count, used for O(1) per-bucket limit enforcement. */ private final Map perBucketCount = new ConcurrentHashMap<>(); - - /** Total active scanner count across all buckets on this tablet server. */ private final AtomicInteger totalScanners = new AtomicInteger(0); - /** - * Set to {@code true} on entry to {@link #close()} so the background TTL evictor can short- - * circuit and avoid mutating counters concurrently with shutdown. - */ private final AtomicBoolean closed = new AtomicBoolean(false); private final Clock clock; @@ -149,32 +114,14 @@ public ScannerManager(Configuration conf, Scheduler scheduler) { } /** - * Creates a new scan session for the given replica, taking a point-in-time RocksDB snapshot - * under the replica's leader-ISR read lock. - * - *

Snapshot opening, leadership validation, and session registration all happen while the - * replica's leader-ISR lock is held, so a concurrent leadership change cannot race with this - * operation: either the scanner is registered before the leader flips (and is then evicted by - * {@link #closeScannersForBucket(TableBucket)} during {@code makeFollowers} / {@code - * stopReplica}), or the leadership check fails and no scanner is ever created. - * - *

The returned {@link OpenScanResult} always carries the log offset captured at snapshot - * time. If the bucket is empty (no rows to scan), the result's {@link OpenScanResult#getContext - * context} is {@code null} and no session slot is consumed; the caller should still relay the - * offset on the response. - * - *

Limit enforcement is two-phase: a fast pre-check guards the common case; the - * subsequent atomic increment + re-check prevents the TOCTOU race from permanently breaching - * configured limits. If registration fails after the snapshot is already opened, the context is - * closed and the exception is re-thrown to avoid leaking resources. + * Opens a new scan session against the given leader replica. The snapshot is taken under the + * replica's leader-ISR read lock; on {@link TooManyScannersException} the just-opened context + * is closed by {@link Replica#openScan} before the exception propagates. Returns an empty- + * bucket result (context = {@code null}) without consuming a slot when the bucket has no rows. * - * @param replica the leader {@link Replica} for the bucket being scanned - * @param limit optional row-count limit ({@code null} or ≤ 0 means unlimited) - * @return an {@link OpenScanResult} with the captured log offset and (for non-empty buckets) - * the newly registered {@link ScannerContext} + * @param limit optional row-count cap ({@code null} or ≤ 0 means unlimited) * @throws TooManyScannersException if the per-bucket or per-server limit is exceeded - * @throws IOException if the underlying {@link org.apache.fluss.server.utils.ResourceGuard} is - * already closed (the bucket is shutting down) + * @throws IOException if RocksDB is shutting down */ public OpenScanResult createScanner(Replica replica, @Nullable Long limit) throws IOException { checkLimits(replica.getTableBucket()); @@ -183,59 +130,39 @@ public OpenScanResult createScanner(Replica replica, @Nullable Long limit) throw } /** - * Atomically registers an already-opened {@link ScannerContext}, enforcing the per-bucket and - * per-server limits. On {@link TooManyScannersException} the caller is responsible for closing - * the context to release the underlying RocksDB resources. - * - *

Called by {@link Replica#openScan} while the leader-ISR read lock is held so that - * registration cannot race with a leadership change. + * Registers an already-opened {@link ScannerContext}, enforcing the per-bucket and per-server + * limits. The caller must close the context if {@link TooManyScannersException} is thrown. */ public void register(ScannerContext context) { registerContext(context); } /** - * Looks up an existing scanner session by its raw ID bytes. Does not refresh the - * last-access timestamp — callers must invoke {@link #markAccessed(ScannerContext)} only after - * the request has been validated, otherwise an invalid request (bad call-sequence id, missing - * batch size, leadership lost, etc.) would silently extend the session TTL and let an orphan - * scanner survive past its idle deadline. - * - * @return the {@link ScannerContext}, or {@code null} if not found (may have expired or never - * existed) + * Looks up a scanner session. Does not refresh the last-access timestamp — call {@link + * #markAccessed(ScannerContext)} only after the request has been fully validated, so malformed + * requests cannot extend the idle TTL of an orphan session. */ @Nullable public ScannerContext getScanner(byte[] scannerId) { return scanners.get(new String(scannerId, StandardCharsets.UTF_8)); } - /** - * Refreshes the last-access timestamp on the given context. Must be called only when the caller - * has decided the request is well-formed and is about to do real work, so that rejected - * requests do not extend the idle TTL. - */ + /** Refreshes the last-access timestamp; call only when about to do real work. */ public void markAccessed(ScannerContext context) { context.updateLastAccessTime(clock.milliseconds()); } /** - * Returns {@code true} if the given scanner ID belongs to a session that was recently evicted - * by the TTL evictor (within the last {@code 2 × ttlMs}). - * - *

Callers can use this to distinguish "scanner expired" from "unknown scanner ID." + * Returns {@code true} if the ID belongs to a session evicted within the last {@code 2 × + * ttlMs}; lets callers distinguish "expired" from "unknown". */ public boolean isRecentlyExpired(byte[] scannerId) { return recentlyExpiredIds.containsKey(new String(scannerId, StandardCharsets.UTF_8)); } /** - * Removes and closes a known scanner context directly, avoiding a map lookup. - * - *

Uses a conditional remove ({@link java.util.concurrent.ConcurrentHashMap#remove(Object, - * Object)}) so that concurrent calls — e.g. from the TTL evictor and a close-scanner RPC - * arriving simultaneously — result in exactly one winner closing the context, preventing - * double-release of the non-idempotent {@link - * org.apache.fluss.server.utils.ResourceGuard.Lease}. + * Removes and closes the given scanner. Uses a conditional remove so the TTL evictor and an + * explicit close cannot both run the close path. */ public void removeScanner(ScannerContext context) { if (scanners.remove(context.getIdString(), context)) { @@ -244,16 +171,7 @@ public void removeScanner(ScannerContext context) { } } - /** - * Looks up and removes a scanner session by its raw ID bytes. - * - *

Delegates to {@link #removeScanner(ScannerContext)} to ensure a conditional {@link - * java.util.concurrent.ConcurrentHashMap#remove(Object, Object)} is used, which prevents a - * double-decrement of {@code perBucketCount} when the TTL evictor races with an explicit close - * request for the same scanner. - * - *

No-op if the ID is not found (already removed or expired). - */ + /** Looks up and removes a scanner by raw ID bytes; no-op if not found. */ public void removeScanner(byte[] scannerId) { String key = new String(scannerId, StandardCharsets.UTF_8); ScannerContext context = scanners.get(key); @@ -262,26 +180,20 @@ public void removeScanner(byte[] scannerId) { } } - /** Returns the total number of active scanner sessions on this tablet server. */ @VisibleForTesting public int activeScannerCount() { return totalScanners.get(); } - /** Returns the number of active scanner sessions for the given bucket. */ @VisibleForTesting public int activeScannerCountForBucket(TableBucket tableBucket) { return perBucketCount.getOrDefault(tableBucket, ZERO).get(); } /** - * Closes and removes all active scanner sessions for the given bucket. Must be called when a - * bucket loses leadership to prevent stale RocksDB snapshot/iterator leaks. - * - *

Closed scanner IDs are recorded in {@link #recentlyExpiredIds} so that a continuation RPC - * arriving after the close surfaces as {@code SCANNER_EXPIRED} (recoverable: client should - * restart against the new leader) rather than {@code UNKNOWN_SCANNER_ID}, which leaves the - * client unable to disambiguate "leadership moved" from "I made up an ID". + * Closes all active scanner sessions for the bucket on a leadership change. Records the IDs in + * {@link #recentlyExpiredIds} so continuation RPCs after the close surface SCANNER_EXPIRED + * (recoverable) instead of UNKNOWN_SCANNER_ID. */ public void closeScannersForBucket(TableBucket tableBucket) { List toRemove = new ArrayList<>(); @@ -296,22 +208,15 @@ public void closeScannersForBucket(TableBucket tableBucket) { "Closing scanner {} for bucket {} due to leadership change.", context.getIdString(), tableBucket); - // Record before removal so a continuation racing with the close cannot observe a - // window where the ID is neither in `scanners` nor in `recentlyExpiredIds`. + // Record before removal so a racing continuation never sees the ID as both unknown + // and not-recently-expired. recentlyExpiredIds.put(context.getIdString(), now); removeScanner(context); } - // Drop any leftover per-bucket counter so we don't leak an empty AtomicInteger after - // the bucket has lost leadership. decrementCounts() also tears down empty entries - // opportunistically; this is the belt-and-suspenders cleanup at the end. perBucketCount.remove(tableBucket); } - /** - * Fast pre-check of per-server and per-bucket limits before opening the snapshot. This is a - * best-effort check; a small race window exists between the check and {@link #registerContext}. - * The race is handled by the atomic re-check inside {@link #registerContext}. - */ + /** Fast best-effort pre-check; the atomic re-check in {@link #registerContext} is the gate. */ private void checkLimits(TableBucket tableBucket) { if (totalScanners.get() >= maxPerServer) { throw new TooManyScannersException( @@ -319,8 +224,6 @@ private void checkLimits(TableBucket tableBucket) { "Cannot create scanner for bucket %s: server-wide limit of %d reached.", tableBucket, maxPerServer)); } - // Read via ZERO sentinel so a failed pre-check does not leak an empty AtomicInteger - // into perBucketCount; the entry is created lazily by registerContext() on success. if (perBucketCount.getOrDefault(tableBucket, ZERO).get() >= maxPerBucket) { throw new TooManyScannersException( String.format( @@ -329,11 +232,6 @@ private void checkLimits(TableBucket tableBucket) { } } - /** - * Atomically increments the counters and puts the context in the map. Throws {@link - * TooManyScannersException} and rolls back the increments if a concurrent create caused either - * limit to be exceeded between the initial check and this call. - */ private void registerContext(ScannerContext context) { TableBucket tableBucket = context.getTableBucket(); @@ -346,8 +244,6 @@ private void registerContext(ScannerContext context) { tableBucket, maxPerServer)); } - // Lazily allocate the per-bucket counter only after the server-wide check passes, - // so a server-overload reject path doesn't pollute perBucketCount. AtomicInteger bucketCount = perBucketCount.computeIfAbsent(tableBucket, k -> new AtomicInteger(0)); int newBucketCount = bucketCount.incrementAndGet(); @@ -370,16 +266,12 @@ private void registerContext(ScannerContext context) { newBucketCount); } - /** TTL evictor — invoked periodically by the background scheduler. */ private void evictExpiredScanners() { - // If close() has begun, skip eviction entirely so we cannot race with shutdown's - // own teardown of the scanners map and counters. if (closed.get()) { return; } long now = clock.milliseconds(); - // Prune stale entries from the recently-expired cache to bound memory usage. recentlyExpiredIds .entrySet() .removeIf(e -> now - e.getValue() > recentlyExpiredRetentionMs); @@ -387,7 +279,6 @@ private void evictExpiredScanners() { for (Map.Entry entry : scanners.entrySet()) { ScannerContext context = entry.getValue(); if (context.isExpired(scannerTtlMs, now)) { - // Conditional remove prevents double-close if removeScanner() fires concurrently. if (scanners.remove(entry.getKey(), context)) { LOG.info( "Evicted idle scanner {} for bucket {} (idle > {}ms).", @@ -440,14 +331,10 @@ public CompletableFuture closeAsync() { @Override public void close() { - // Idempotent: a second close() is a no-op. if (!closed.compareAndSet(false, true)) { return; } - // Note: we cancel but do not join the evictor. The evictor may still be mid-iteration - // when close() begins, but it checks the `closed` flag at the top and short-circuits - // before mutating any counters; combined with conditional remove(key, value) on - // `scanners`, each ScannerContext is closed by exactly one side. + if (evictorTask != null) { evictorTask.cancel(false); evictorTask = null; @@ -459,12 +346,6 @@ public void close() { closeScannerContext(entry.getValue()); } } - - // Note: totalScanners and perBucketCount are not forcibly reset here. Because both - // shutdown and the evictor use conditional remove(key, value) — and the evictor - // bails out via the `closed` flag — each scanner is decremented exactly once, so - // the counters naturally reach zero. A forced reset would risk driving counters - // negative if a stray decrement still completes after close(). recentlyExpiredIds.clear(); } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java index e208c88610..6a84316c39 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java @@ -216,13 +216,9 @@ public final class Replica { private @Nullable PeriodicSnapshotManager kvSnapshotManager; /** - * Reference to the server-wide {@link ScannerManager}. Active scanner sessions for this bucket - * are closed in {@link #dropKv()} under the {@code leaderIsrUpdateLock} write lock so - * that no new scanner can register between the leadership flip and the KV tablet teardown. This - * is the authoritative cleanup path; the additional {@code closeScannersForBucket} calls in - * {@link org.apache.fluss.server.replica.ReplicaManager#stopReplicas} (before {@code delete()}) - * and {@link org.apache.fluss.server.replica.ReplicaManager#makeFollowers} (after {@code - * makeFollower(...)}) cover narrower windows but do not replace this guard. + * Server-wide {@link ScannerManager}. Active sessions for this bucket are closed in {@link + * #dropKv()} under the {@code leaderIsrUpdateLock} write lock; {@link #openScan} registers + * under the read lock, so registration cannot race a leadership flip. */ private final ScannerManager scannerManager; @@ -719,22 +715,16 @@ private void createKv() { } private void dropKv() { - // Close any active scanner sessions for this bucket BEFORE tearing down the KV tablet - // — otherwise outstanding ResourceGuard leases would block kvTablet.close() indefinitely - // on resourceGuard.close(). This call runs under leaderIsrUpdateLock(W) (held by every - // caller of dropKv: delete(), onBecomeNewLeader(), onBecomeNewFollower()), so no new - // scanner can register concurrently — Replica#openScan registers under the read lock. + // Release scanner leases first; otherwise resourceGuard.close() inside kvTablet.close() + // blocks waiting for them. Runs under leaderIsrUpdateLock(W), so no concurrent register. scannerManager.closeScannersForBucket(tableBucket); - // close any closeable registry for kv + if (closeableRegistry.unregisterCloseable(closeableRegistryForKv)) { IOUtils.closeQuietly(closeableRegistryForKv); } if (kvTablet != null) { - // Unregister RocksDB statistics before dropping KvTablet - // This ensures statistics are cleaned up when KvTablet is destroyed bucketMetricGroup.unregisterRocksDBStatistics(); - // drop the kv tablet checkNotNull(kvManager); kvManager.dropKv(tableBucket); kvTablet = null; @@ -1401,28 +1391,14 @@ public DefaultValueRecordBatch limitKvScan(int limit) { } /** - * Opens a new full-scan session against this replica's KV store, taking a point-in-time RocksDB - * snapshot under the {@code leaderIsrUpdateLock} read lock. - * - *

The lock is held for the entire flow — leadership check, KV-tablet acquisition, snapshot - * creation, and registration with the {@link ScannerManager} — so a concurrent leadership - * change cannot leave a scanner registered for a follower bucket. Once a leadership change - * acquires the write lock, {@link org.apache.fluss.server.replica.ReplicaManager#makeFollowers} - * / {@code stopReplicas} will call {@link ScannerManager#closeScannersForBucket(TableBucket)} - * and any scanner registered before the flip is released eagerly. - * - *

The returned {@link OpenScanResult} always carries the log offset captured at snapshot - * time. If the bucket is empty, the result's {@link OpenScanResult#getContext context} is - * {@code null} and no session slot is consumed; the caller should still relay the offset on the - * response. + * Opens a new full-scan session against this replica's KV store. The leader check, snapshot + * open, and {@link ScannerManager#register} all happen under {@code leaderIsrUpdateLock(R)}, so + * a leadership flip cannot leave a scanner registered for a non-leader bucket. * - * @param scannerManager the manager to register the new context with - * @param scannerId the server-assigned scanner ID - * @param limit maximum number of rows to return ({@code ≤ 0} = unlimited) - * @param initialAccessTimeMs initial last-access timestamp for TTL accounting - * @throws NonPrimaryKeyTableException if this replica is not a primary-key (KV) table - * @throws NotLeaderOrFollowerException if this replica is not currently the leader - * @throws TooManyScannersException if registering would breach the configured scanner limits + * @param limit row-count cap ({@code ≤ 0} means unlimited) + * @throws NonPrimaryKeyTableException if this replica is not a primary-key table + * @throws NotLeaderOrFollowerException if this replica is not the leader + * @throws TooManyScannersException if scanner limits are exceeded * @throws IOException if RocksDB is shutting down */ public OpenScanResult openScan( @@ -1448,15 +1424,11 @@ public OpenScanResult openScan( kvTablet.openScan(scannerId, limit, initialAccessTimeMs); ScannerContext context = result.getContext(); if (context == null) { - // Empty bucket — no session is registered, but propagate the captured - // log offset back to the caller. return result; } try { scannerManager.register(context); } catch (TooManyScannersException e) { - // Limit was breached between the pre-check and registration. - // Close the already-opened context to release the snapshot and lease. IOUtils.closeQuietly(context); throw e; } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java index ea3ed59a74..f2a01cbe95 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java @@ -1841,10 +1841,6 @@ private StopReplicaResultForBucket stopReplica( // First stop fetchers for this table bucket. replicaFetcherManager.removeFetcherForBuckets(Collections.singleton(tb)); - // Close active scanner sessions for this bucket as a first cut before the replica's - // write-lock teardown. The authoritative cleanup happens inside Replica#dropKv (called - // from delete() under the write lock); this outer call narrows the window during which - // a concurrent scanKv could observe stale state, and is a no-op once dropKv has run. scannerManager.closeScannersForBucket(tb); HostedReplica replica = getReplica(tb); 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 1f8a10bcf0..b0bbed2290 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 @@ -258,8 +258,6 @@ protected void startServices() throws Exception { conf.get(ConfigOptions.SERVER_IO_POOL_SIZE), new ExecutorThreadFactory("tablet-server-io")); - // Create the ScannerManager before the ReplicaManager so that it can be wired in - // through the ReplicaManager's constructor (and reach Replica via constructor too). this.scannerManager = new ScannerManager(conf, scheduler); this.replicaManager = @@ -290,11 +288,8 @@ protected void startServices() throws Exception { // Start dynamicConfigManager after all reconfigurable components are registered dynamicConfigManager.startup(); - // Server-side cap on the per-batch payload size; clamped to int range because the - // ScanKvRequest.batch_size_bytes wire-field is int32. long configuredMaxBatch = conf.get(ConfigOptions.KV_SCANNER_MAX_BATCH_SIZE).getBytes(); - int kvScanMaxBatchSizeBytes = - (int) Math.min((long) Integer.MAX_VALUE, configuredMaxBatch); + int kvScanMaxBatchSizeBytes = (int) Math.min(Integer.MAX_VALUE, configuredMaxBatch); this.tabletService = new TabletService( 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 2f331d3f3e..50219d2abc 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 @@ -149,11 +149,6 @@ public final class TabletService extends RpcServiceBase implements TabletServerG private final TabletServerMetadataProvider metadataFunctionProvider; private final ScannerManager scannerManager; - /** - * Server-side cap on per-batch payload size for KV full-scan responses. The effective batch - * size used in {@link #scanKv} is {@code min(client-requested batch_size_bytes, this value)}, - * which protects the server against OOM if a client passes an excessively large value. - */ private final int kvScanMaxBatchSizeBytes; public TabletService( @@ -458,38 +453,48 @@ public CompletableFuture notifyLakeTableOffset( @Override public CompletableFuture scanKv(ScanKvRequest request) { ScanKvResponse response = new ScanKvResponse(); - // Tracks a live session that must be force-closed if anything after it throws. - // A scan that produced a partial but un-returned batch is not resumable: the RocksDB - // cursor has already advanced past rows whose data never reached the client, so we - // tear the session down and force the client to restart. This matches the non-resumable - // contract documented on ScannerExpiredException. ScannerContext openedContext = null; - // Tracks the context whose `inUse` flag we own so that finally{} can release it. May - // diverge from openedContext (which is nulled out on success) so we keep it separate. ScannerContext acquiredContext = null; try { - ScannerContext context; - // True only on the initial (bucket_scan_req) path, where we must echo the snapshot - // log offset back to the client. On continuations the field stays absent. - boolean isNewScan = false; - long initialLogOffset = 0L; - if (request.hasBucketScanReq() && request.hasScannerId()) { throw new InvalidScanRequestException( "ScanKvRequest must not set both bucket_scan_req and scanner_id."); } - // close_scanner only makes sense on a continuation (paired with scanner_id); - // pairing it with a fresh bucket_scan_req would open and immediately tear down a - // session, wasting a slot and confusing the client contract. if (request.hasBucketScanReq() && request.hasCloseScanner() && request.isCloseScanner()) { throw new InvalidScanRequestException( "ScanKvRequest must not set close_scanner together with bucket_scan_req."); } + if (!request.hasBucketScanReq() && !request.hasScannerId()) { + throw new InvalidScanRequestException( + "ScanKvRequest must have either bucket_scan_req (new scan) " + + "or scanner_id (continuation)."); + } + + boolean isCloseRequest = request.hasCloseScanner() && request.isCloseScanner(); + + // Validate batch_size_bytes up-front so malformed requests never open a snapshot. + int effectiveBatchSize = 0; + if (!isCloseRequest) { + if (!request.hasBatchSizeBytes()) { + throw new InvalidScanRequestException( + "batch_size_bytes is required for data-fetching scan requests."); + } + int requestedBatchSize = request.getBatchSizeBytes(); + if (requestedBatchSize <= 0) { + throw new InvalidScanRequestException( + "batch_size_bytes must be greater than 0."); + } + effectiveBatchSize = Math.min(requestedBatchSize, kvScanMaxBatchSizeBytes); + } + + ScannerContext context; + + boolean isNewScan = false; + long initialLogOffset = 0L; if (request.hasBucketScanReq()) { - // New scan: open a fresh scanner session PbScanReqForBucket bucketReq = request.getBucketScanReq(); long tableId = bucketReq.getTableId(); authorizeTable(READ, tableId); @@ -509,28 +514,17 @@ public CompletableFuture scanKv(ScanKvRequest request) { context = openResult.getContext(); if (context == null) { - // Bucket is empty — return an empty response immediately without registering a - // session, but echo the captured log offset so the client can perform a - // consistent snapshot-to-log handoff. + // Empty bucket: no session registered; still return the captured offset. response.setHasMoreResults(false); response.setLogOffset(initialLogOffset); return CompletableFuture.completedFuture(response); } openedContext = context; } else { - if (!request.hasScannerId()) { - throw new InvalidScanRequestException( - "ScanKvRequest must have either bucket_scan_req (new scan) " - + "or scanner_id (continuation)."); - } byte[] scannerId = request.getScannerId(); context = scannerManager.getScanner(scannerId); if (context == null) { - // If the client is sending a close request and the scanner is already gone - // (auto-closed when fully drained, or evicted by TTL), this is a benign - // no-op — return a finished response instead of surfacing an error that - // the client cannot act on. - if (request.hasCloseScanner() && request.isCloseScanner()) { + if (isCloseRequest) { response.setScannerId(scannerId); response.setHasMoreResults(false); return CompletableFuture.completedFuture(response); @@ -548,23 +542,18 @@ public CompletableFuture scanKv(ScanKvRequest request) { openedContext = context; } - // Acquire single-thread access to the cursor before any state mutation. Without - // this, two concurrent scanKv RPCs sharing a scannerId would both observe the same - // pre-mutation callSeqId, both pass the in-order check, and both race - // iterator.next() - corrupting RocksDB state at the JNI boundary. The loser of the - // CAS gets a clear error so the client can retry sequentially. + // Cursor-exclusion CAS: serialises concurrent same-scannerId RPCs and rejects if + // close() has begun. if (!context.tryAcquireForUse()) { throw new InvalidScanRequestException( String.format( - "Concurrent scan request on scanner ID for bucket %s; only one " - + "in-flight scanKv RPC per scanner is allowed.", + "Concurrent scan request on scanner ID for bucket %s, or session " + + "is closing; only one in-flight scanKv RPC per scanner " + + "is allowed.", context.getTableBucket())); } acquiredContext = context; - // Validate call-sequence ordering to detect duplicate or out-of-order requests. - // Use long arithmetic to avoid a silent 32-bit overflow at Integer.MAX_VALUE - // continuations. Skipped on the new-scan path (no scannerId on the wire). if (!request.hasBucketScanReq() && request.hasCallSeqId()) { long expectedSeqId = (long) context.getCallSeqId() + 1L; int requestSeqId = request.getCallSeqId(); @@ -576,30 +565,18 @@ public CompletableFuture scanKv(ScanKvRequest request) { } } - // Handle explicit close request. Honour the close even on a non-leader: the local - // session resources are still ours to release, and we don't want a leadership flip - // racing with a close to leak a snapshot. - if (request.hasCloseScanner() && request.isCloseScanner()) { - scannerManager.removeScanner(context); - openedContext = null; + // Honour close even on a non-leader: the local session is still ours to release. + if (isCloseRequest) { response.setScannerId(context.getScannerId()); response.setHasMoreResults(false); return CompletableFuture.completedFuture(response); } - // Continuation: re-verify that this server is still the leader for the bucket - // before serving more data. closeScannersForBucket() will eventually evict the - // scanner on a leadership flip (and surface SCANNER_EXPIRED on later RPCs), but - // there is a small window between the leader flip and that callback during which - // the scanner remains in the map. Catching the flip here lets the client redirect - // to the new leader instead of silently consuming a stale snapshot. + // Catch a leadership flip ahead of the eventual closeScannersForBucket callback so + // the client can redirect rather than consume a stale snapshot. if (!request.hasBucketScanReq()) { Replica replica = replicaManager.getReplicaOrException(context.getTableBucket()); if (!replica.isLeader()) { - // Drop the local session so resources aren't held while the client - // redirects; closeScannersForBucket() will be a no-op when it runs. - scannerManager.removeScanner(context); - openedContext = null; throw new NotLeaderOrFollowerException( String.format( "Leader is no longer local for bucket %s; client should " @@ -608,31 +585,12 @@ public CompletableFuture scanKv(ScanKvRequest request) { } } - // batch_size_bytes is optional in proto; require it for data-fetching requests and - // clamp to the server-side cap so a malicious or buggy client cannot trigger an OOM - // by passing Integer.MAX_VALUE. - if (!request.hasBatchSizeBytes()) { - throw new InvalidScanRequestException( - "batch_size_bytes is required for data-fetching scan requests."); - } - int requestedBatchSize = request.getBatchSizeBytes(); - if (requestedBatchSize <= 0) { - throw new InvalidScanRequestException("batch_size_bytes must be greater than 0."); - } - int effectiveBatchSize = Math.min(requestedBatchSize, kvScanMaxBatchSizeBytes); - - // Refresh the idle-TTL deadline only now that the request is fully validated and we - // are about to do real work. Earlier refreshes (during getScanner) would let any - // malformed request - bad callSeqId, missing batch size, lost leadership - keep an - // orphan session alive past its idle deadline. + // Refresh TTL only now that the request is fully validated. scannerManager.markAccessed(context); - // Build the next batch using the builder's own running size as the gating signal so - // the threshold reflects the actual serialised batch (header + per-record framing), - // not just the raw value bytes. We always append at least one record (when data is - // available) so an unusually small effectiveBatchSize — e.g. one smaller than the - // empty-builder header — cannot produce an empty has_more=true response that would - // make the client loop indefinitely. + // Gate on builder.sizeInBytes() (not raw bytes) so the threshold reflects the + // serialised batch. Always append at least one record so a tiny effectiveBatchSize + // cannot produce an empty has_more=true response. DefaultValueRecordBatch.Builder builder = DefaultValueRecordBatch.builder(); boolean appendedAny = false; while (context.isValid() @@ -645,10 +603,7 @@ public CompletableFuture scanKv(ScanKvRequest request) { boolean hasMore = context.isValid(); if (!hasMore) { // RocksIterator.next() does not throw on internal errors; an unchecked status - // here would silently turn an iterator-internal failure into has_more=false, - // dropping every row past the failure point and letting the client conclude - // the scan completed cleanly. Surface the error so the catch block maps it to - // KV_STORAGE_EXCEPTION and the finally block force-closes the session. + // would silently truncate the scan and report has_more=false to the client. context.checkIteratorStatus(); } DefaultValueRecordBatch batch = builder.build(); @@ -662,23 +617,19 @@ public CompletableFuture scanKv(ScanKvRequest request) { response.setLogOffset(initialLogOffset); } - // Update callSeqId AFTER the response is prepared so that a client retry with the - // same callSeqId (due to a transient failure) can be detected and rejected. + // Update callSeqId AFTER the response is prepared so a duplicate retry can be + // detected via the in-order check. if (request.hasCallSeqId()) { context.setCallSeqId(request.getCallSeqId()); } - // Auto-close the session when all data has been drained. - if (!hasMore) { - scannerManager.removeScanner(context); + // Keep the session alive only if there's more to read; otherwise leave openedContext + // set so finally drains it. + if (hasMore) { + openedContext = null; } - // Response successfully prepared — session state is consistent; do not force-close. - openedContext = null; } catch (Exception e) { - // Restore the interrupt flag if a lower-level call wrapped an InterruptedException. - // No method in the try block declares `throws InterruptedException` directly, but a - // future refactor or a rethrown wrapper should not silently lose the signal. if (e instanceof InterruptedException || e.getCause() instanceof InterruptedException) { Thread.currentThread().interrupt(); } @@ -686,17 +637,9 @@ public CompletableFuture scanKv(ScanKvRequest request) { response.setErrorCode(apiError.error().code()); response.setErrorMessage(apiError.message() != null ? apiError.message() : ""); } finally { - // Release the cursor-exclusion flag before any force-close: the close path is - // CAS-guarded on its own `closed` flag, so the order is purely cosmetic, but - // releasing first keeps the invariant "inUse=true means a thread is mid-handler" - // tight. Calling releaseAfterUse on an already-closed context is a no-op. if (acquiredContext != null) { acquiredContext.releaseAfterUse(); } - // If we made it past createScanner/getScanner but failed to deliver a complete - // response, close the session rather than leaking it to TTL. The cursor has - // already advanced past rows whose values were never sent; resuming would drop - // data. Forcing a restart is the safe option. if (openedContext != null) { scannerManager.removeScanner(openedContext); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/KvTabletTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/KvTabletTest.java index 7c3b4bb2c3..1f2d5a89fa 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/kv/KvTabletTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/KvTabletTest.java @@ -1863,8 +1863,6 @@ void testRowCountWithMixedOperations() throws Exception { @Test void testOpenScan_emptyBucket_returnsNullContext() throws Exception { initLogTabletAndKvTablet(DATA1_SCHEMA_PK, new HashMap<>()); - // No data has been written — the result wraps a null context but still carries the - // captured log offset for the empty-bucket fast path. OpenScanResult result = kvTablet.openScan("scanner-empty", -1L, 0L); assertThat(result).isNotNull(); assertThat(result.getContext()).isNull(); @@ -1901,11 +1899,11 @@ void testOpenScan_returnsAllRows() throws Exception { assertThat(count).isEqualTo(numRows); } + /** Snapshot isolation: rows written after openScan must not appear in the scan. */ @Test void testOpenScan_snapshotIsolation() throws Exception { initLogTabletAndKvTablet(DATA1_SCHEMA_PK, new HashMap<>()); - // Write and flush 3 rows before opening the scan. List initialRows = Arrays.asList( kvRecordFactory.ofRecord("0".getBytes(), new Object[] {0, "v0"}), @@ -1917,7 +1915,6 @@ void testOpenScan_snapshotIsolation() throws Exception { ScannerContext context = kvTablet.openScan("scanner-snap", -1L, 0L).getContext(); assertThat(context).isNotNull(); - // Write 2 more rows AFTER opening the scan, then flush. List lateRows = Arrays.asList( kvRecordFactory.ofRecord("3".getBytes(), new Object[] {3, "v3"}), @@ -1925,7 +1922,6 @@ void testOpenScan_snapshotIsolation() throws Exception { kvTablet.putAsLeader(kvRecordBatchFactory.ofRecords(lateRows), null); kvTablet.flush(Long.MAX_VALUE, NOPErrorHandler.INSTANCE); - // The scan must still see only the 3 rows that existed at snapshot time. int count = 0; while (context.isValid()) { context.advance(); @@ -1961,7 +1957,6 @@ void testOpenScan_withLimit() throws Exception { } context.close(); - // The scan must stop after exactly `limit` rows. assertThat(count).isEqualTo((int) limit); } @@ -1977,13 +1972,11 @@ void testOpenScan_multipleSessionsIndependent() throws Exception { kvTablet.putAsLeader(kvRecordBatchFactory.ofRecords(rows), null); kvTablet.flush(Long.MAX_VALUE, NOPErrorHandler.INSTANCE); - // Open two independent scans. ScannerContext ctx1 = kvTablet.openScan("scanner-a", -1L, 0L).getContext(); ScannerContext ctx2 = kvTablet.openScan("scanner-b", -1L, 0L).getContext(); assertThat(ctx1).isNotNull(); assertThat(ctx2).isNotNull(); - // Drain ctx1 fully. int count1 = 0; while (ctx1.isValid()) { ctx1.advance(); @@ -1991,7 +1984,6 @@ void testOpenScan_multipleSessionsIndependent() throws Exception { } ctx1.close(); - // ctx2 cursor must be unaffected; it should still see all 3 rows. int count2 = 0; while (ctx2.isValid()) { ctx2.advance(); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java index 91831fcce6..71f757bfcc 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java @@ -54,6 +54,7 @@ import java.time.Duration; import java.util.ArrayList; import java.util.List; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import static org.apache.fluss.compression.ArrowCompressionInfo.DEFAULT_COMPRESSION; @@ -153,11 +154,6 @@ void tearDown() throws Exception { } } - // ------------------------------------------------------------------------- - // Helpers - // ------------------------------------------------------------------------- - - /** Creates a {@link ScannerManager} with a long TTL so the evictor never fires during tests. */ private ScannerManager createManager() { Configuration c = new Configuration(); c.set(ConfigOptions.KV_SCANNER_TTL, Duration.ofHours(1)); @@ -167,7 +163,6 @@ private ScannerManager createManager() { return new ScannerManager(c, scheduler, clock); } - /** Creates a {@link ScannerManager} with configurable limits and a long evictor interval. */ private ScannerManager createManager(int maxPerBucket, int maxPerServer) { Configuration c = new Configuration(); c.set(ConfigOptions.KV_SCANNER_TTL, Duration.ofHours(1)); @@ -177,9 +172,6 @@ private ScannerManager createManager(int maxPerBucket, int maxPerServer) { return new ScannerManager(c, scheduler, clock); } - /** - * Creates a {@link ScannerManager} with a short TTL and evictor interval for eviction tests. - */ private ScannerManager createManagerWithShortTtl(long ttlMs, long expirationIntervalMs) { Configuration c = new Configuration(); c.set(ConfigOptions.KV_SCANNER_TTL, Duration.ofMillis(ttlMs)); @@ -191,11 +183,7 @@ private ScannerManager createManagerWithShortTtl(long ttlMs, long expirationInte return new ScannerManager(c, scheduler, clock); } - /** - * Helper used by tests to open a scanner context directly against the KvTablet (bypassing - * Replica) and register it with the manager. The Replica-mediated flow is covered by - * integration tests; here we want to exercise ScannerManager in isolation. - */ + /** Opens a scanner directly via KvTablet and registers it; mirrors Replica#openScan. */ private ScannerContext openAndRegister(ScannerManager manager) throws Exception { ScannerContext ctx = kvTablet.openScan(java.util.UUID.randomUUID().toString(), -1L, clock.milliseconds()) @@ -206,17 +194,12 @@ private ScannerContext openAndRegister(ScannerManager manager) throws Exception try { manager.register(ctx); } catch (RuntimeException e) { - // Mirror the production behaviour in Replica#openScan: if registration fails - // (e.g. TooManyScannersException), close the already-opened context to release - // the underlying RocksDB snapshot/iterator/lease so the test KvTablet can shut - // down cleanly. ctx.close(); throw e; } return ctx; } - /** Writes {@code count} rows into the KvTablet and flushes to RocksDB. */ private void putAndFlush(int count) throws Exception { List rows = new ArrayList<>(); for (int i = 0; i < count; i++) { @@ -228,14 +211,9 @@ private void putAndFlush(int count) throws Exception { kvTablet.flush(Long.MAX_VALUE, NOPErrorHandler.INSTANCE); } - // ------------------------------------------------------------------------- - // Tests - // ------------------------------------------------------------------------- - @Test void testCreateScanner_emptyBucket_returnsNull() throws Exception { try (ScannerManager manager = createManager()) { - // Bucket has no data — openScan must return null; no slot consumed. ScannerContext context = openAndRegister(manager); assertThat(context).isNull(); assertThat(manager.activeScannerCount()).isEqualTo(0); @@ -263,19 +241,14 @@ void testCreateAndRemoveScanner() throws Exception { void testGetScanner_doesNotRefreshLastAccessTime() throws Exception { putAndFlush(3); try (ScannerManager manager = createManager()) { - // Create scanner at t=0. ScannerContext context = openAndRegister(manager); assertThat(context).isNotNull(); byte[] scannerId = context.getScannerId(); - // Advance clock far past the initial access time and look up the scanner. clock.advanceTime(5000, TimeUnit.MILLISECONDS); ScannerContext fetched = manager.getScanner(scannerId); assertThat(fetched).isSameAs(context); - // getScanner alone must NOT refresh the last-access time: an invalid request - // (bad callSeqId, missing batch size, lost leadership) would otherwise extend - // the idle TTL and let an orphan session survive past its deadline. assertThat(context.isExpired(1000L, clock.milliseconds())).isTrue(); manager.removeScanner(context); @@ -290,10 +263,8 @@ void testMarkAccessed_refreshesLastAccessTime() throws Exception { assertThat(context).isNotNull(); clock.advanceTime(5000, TimeUnit.MILLISECONDS); - // Must explicitly mark the session as accessed; lookup is non-mutating. manager.markAccessed(context); - // With a 1-hour TTL, isExpired must be false right after the refresh. assertThat(context.isExpired(3_600_000L, clock.milliseconds())).isFalse(); manager.removeScanner(context); @@ -307,14 +278,11 @@ void testTryAcquireForUse_serialisesConcurrentClaims() throws Exception { ScannerContext context = openAndRegister(manager); assertThat(context).isNotNull(); - // First claim wins. assertThat(context.tryAcquireForUse()).isTrue(); - // A second concurrent claim must fail until the first releases. assertThat(context.tryAcquireForUse()).isFalse(); context.releaseAfterUse(); - // After release, a fresh claim succeeds again. assertThat(context.tryAcquireForUse()).isTrue(); context.releaseAfterUse(); @@ -322,12 +290,6 @@ void testTryAcquireForUse_serialisesConcurrentClaims() throws Exception { } } - /** - * Stress-test the cursor-exclusion CAS: many threads race to acquire a single context, and - * exactly one must win. This is the invariant that protects {@link - * org.apache.fluss.server.tablet.TabletService#scanKv} from concurrent same-scannerId RPCs - * racing the RocksDB iterator at the JNI boundary. - */ @Test void testTryAcquireForUse_exactlyOneWinnerUnderContention() throws Exception { putAndFlush(3); @@ -369,13 +331,6 @@ void testTryAcquireForUse_exactlyOneWinnerUnderContention() throws Exception { } } - /** - * On a healthy iterator, {@link ScannerContext#checkIteratorStatus()} must be a no-op so the - * post-loop check in {@link org.apache.fluss.server.tablet.TabletService#scanKv} does not - * spuriously fail clean end-of-range scans. The error path (RocksDB-internal failure) is - * exercised at the integration level — fabricating a JNI-level error is impractical in a unit - * test, so this test pins the no-error contract. - */ @Test void testCheckIteratorStatus_healthyIteratorIsNoOp() throws Exception { putAndFlush(3); @@ -383,13 +338,10 @@ void testCheckIteratorStatus_healthyIteratorIsNoOp() throws Exception { ScannerContext context = openAndRegister(manager); assertThat(context).isNotNull(); - // Drain the cursor so isValid() flips to false through the natural end-of-range. while (context.isValid()) { context.advance(); } - // status() must report ok for an iterator that ended cleanly — otherwise every - // successful scan would surface KV_STORAGE_EXCEPTION at the post-loop guard. context.checkIteratorStatus(); manager.removeScanner(context); @@ -399,17 +351,14 @@ void testCheckIteratorStatus_healthyIteratorIsNoOp() throws Exception { @Test void testTtlEviction() throws Exception { putAndFlush(3); - // TTL = 200 ms, evictor every 200 ms — wide enough for slow CI schedulers. ScannerManager manager = createManagerWithShortTtl(200, 200); try { ScannerContext context = openAndRegister(manager); assertThat(context).isNotNull(); byte[] scannerId = context.getScannerId(); - // Advance ManualClock past TTL so the evictor considers the session idle. clock.advanceTime(500, TimeUnit.MILLISECONDS); - // Wait for the real scheduler to invoke the cleanup task. retry( Duration.ofSeconds(10), () -> assertThat(manager.activeScannerCount()).isEqualTo(0)); @@ -433,7 +382,6 @@ void testPerBucketLimit() throws Exception { assertThatThrownBy(() -> openAndRegister(manager)) .isInstanceOf(TooManyScannersException.class); - // Count must not have changed after the failed attempt. assertThat(manager.activeScannerCountForBucket(tableBucket)).isEqualTo(2); manager.removeScanner(ctx1); @@ -476,12 +424,6 @@ void testCloseScannersForBucket() throws Exception { } } - /** - * Leadership-induced closure must record the scanner ID in {@code recentlyExpiredIds} so a - * continuation RPC arriving after the close surfaces SCANNER_EXPIRED (recoverable: client - * restarts against the new leader) rather than UNKNOWN_SCANNER_ID, which would leave the client - * unable to disambiguate "leadership moved" from "I made up an ID". - */ @Test void testCloseScannersForBucket_marksRecentlyExpired() throws Exception { putAndFlush(3); @@ -511,4 +453,51 @@ void testShutdown_closesAllScanners() throws Exception { assertThat(manager.activeScannerCount()).isEqualTo(0); } + + @Test + void testTryAcquireForUse_rejectsAfterClose() throws Exception { + putAndFlush(3); + try (ScannerManager manager = createManager()) { + ScannerContext ctx = openAndRegister(manager); + assertThat(ctx).isNotNull(); + + manager.removeScanner(ctx); + + assertThat(ctx.tryAcquireForUse()).isFalse(); + } + } + + /** close() must spin until inUse=false; iterating after close is undefined at the JNI. */ + @Test + void testClose_waitsForInUse() throws Exception { + putAndFlush(3); + try (ScannerManager manager = createManager()) { + ScannerContext ctx = openAndRegister(manager); + assertThat(ctx).isNotNull(); + + assertThat(ctx.tryAcquireForUse()).isTrue(); + + CountDownLatch closeStarted = new CountDownLatch(1); + Thread closer = + new Thread( + () -> { + closeStarted.countDown(); + ctx.close(); + }, + "scanner-context-closer"); + closer.start(); + + assertThat(closeStarted.await(5, TimeUnit.SECONDS)).isTrue(); + Thread.sleep(50); + assertThat(closer.isAlive()).as("close() must spin while inUse=true").isTrue(); + + ctx.releaseAfterUse(); + closer.join(TimeUnit.SECONDS.toMillis(5)); + assertThat(closer.isAlive()).isFalse(); + + assertThat(ctx.tryAcquireForUse()).isFalse(); + + manager.removeScanner(ctx); + } + } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java index 49480b4ab4..fa502f928f 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java @@ -65,7 +65,6 @@ import org.apache.fluss.server.entity.StopReplicaResultForBucket; import org.apache.fluss.server.kv.KvTablet; import org.apache.fluss.server.kv.rocksdb.RocksDBKv; -import org.apache.fluss.server.kv.scan.ScannerManager; import org.apache.fluss.server.kv.snapshot.CompletedSnapshot; import org.apache.fluss.server.log.FetchParams; import org.apache.fluss.server.log.ListOffsetsParam; @@ -2379,10 +2378,6 @@ private void assertUpdateMetadataEquals( }); } - /** - * When a replica is stopped, {@link ScannerManager#closeScannersForBucket} must be called so - * that open scanner sessions are released before the KV store is destroyed. - */ @Test void testStopReplicas_closesScanners() throws Exception { TableBucket tb = new TableBucket(DATA1_TABLE_ID_PK, 0); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/tablet/TabletServiceITCase.java b/fluss-server/src/test/java/org/apache/fluss/server/tablet/TabletServiceITCase.java index 6479843857..8adb942daf 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/tablet/TabletServiceITCase.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/tablet/TabletServiceITCase.java @@ -1080,15 +1080,6 @@ void testLookupWithInsertIfNotExistsAutoIncrement() throws Exception { assertThat(newRow.getLong(1)).isEqualTo(3L); } - // ------------------------------------------------------------------------- - // scanKv RPC tests (FIP-17 KV full-scan sessions) - // - // After applying DATA_1_WITH_KEY_AND_VALUE the merged final RocksDB state - // contains exactly two rows (keys 1 and 2; key 3 is tombstoned). Each test - // triggers a snapshot before scanning so the prewrite buffer is flushed - // into RocksDB and visible to the scanner's snapshot iterator. - // ------------------------------------------------------------------------- - @Test void testScanKv_newScan_happyPath() throws Exception { long tableId = @@ -1108,14 +1099,12 @@ void testScanKv_newScan_happyPath() throws Exception { .get()); FLUSS_CLUSTER_EXTENSION.triggerAndWaitSnapshot(tb); - // 1 MiB easily fits the merged state (two rows) into a single batch. ScanKvResponse response = leaderGateWay.scanKv(newScanKvOpenRequest(tableId, 0, 1024 * 1024)).get(); assertThat(response.hasErrorCode()).isFalse(); assertThat(response.getScannerId()).isNotEmpty(); assertThat(response.isHasMoreResults()).isFalse(); - // The captured log offset must always be carried back on the first response. assertThat(response.hasLogOffset()).isTrue(); assertThat(response.getLogOffset()).isGreaterThanOrEqualTo(0L); assertThat(response.hasRecords()).isTrue(); @@ -1159,8 +1148,7 @@ void testScanKv_multiBatchContinuation() throws Exception { .get(); assertThat(current.hasErrorCode()).isFalse(); assertThat(current.getScannerId()).isEqualTo(scannerId); - // The log offset is only carried on the first (open) response; continuations - // must not re-set it. + // log_offset is carried only on the open response. assertThat(current.hasLogOffset()).isFalse(); if (current.hasRecords()) { totalRecords += @@ -1168,7 +1156,7 @@ void testScanKv_multiBatchContinuation() throws Exception { } } assertThat(totalRecords).isEqualTo(2); - // After draining, the session is auto-closed: a continuation should now be unknown. + // After draining, the session is auto-closed. ScanKvResponse afterDrain = leaderGateWay.scanKv(newScanKvContinueRequest(scannerId, seq, 1024)).get(); assertThat(afterDrain.getErrorCode()).isEqualTo(Errors.UNKNOWN_SCANNER_ID.code()); @@ -1194,7 +1182,7 @@ void testScanKv_explicitClose() throws Exception { .get()); FLUSS_CLUSTER_EXTENSION.triggerAndWaitSnapshot(tb); - // Use a tiny batch so the session stays open after the first response. + // Tiny batch keeps the session open after the first response. ScanKvResponse open = leaderGateWay.scanKv(newScanKvOpenRequest(tableId, 0, 1)).get(); assertThat(open.hasErrorCode()).isFalse(); assertThat(open.isHasMoreResults()).isTrue(); @@ -1204,7 +1192,6 @@ void testScanKv_explicitClose() throws Exception { assertThat(close.hasErrorCode()).isFalse(); assertThat(close.getScannerId()).isEqualTo(scannerId); assertThat(close.isHasMoreResults()).isFalse(); - // No records on a close response. assertThat(close.hasRecords()).isFalse(); } @@ -1230,10 +1217,9 @@ void testScanKv_closeOnAlreadyGoneScannerIsNoOp() throws Exception { ScanKvResponse open = leaderGateWay.scanKv(newScanKvOpenRequest(tableId, 0, 1)).get(); byte[] scannerId = open.getScannerId(); - // First close removes the session. leaderGateWay.scanKv(newScanKvCloseRequest(scannerId)).get(); - // Second close on the already-gone session must be a benign no-op (not an error). + // Second close on the already-gone session must be a benign no-op. ScanKvResponse second = leaderGateWay.scanKv(newScanKvCloseRequest(scannerId)).get(); assertThat(second.hasErrorCode()).isFalse(); assertThat(second.getScannerId()).isEqualTo(scannerId); @@ -1251,7 +1237,6 @@ void testScanKv_unknownScannerId() throws Exception { TabletServerGateway leaderGateWay = FLUSS_CLUSTER_EXTENSION.newTabletServerClientForNode(leader); - // No openScan was ever issued — this id is fabricated. byte[] fakeId = "not-a-real-scanner-id".getBytes(); ScanKvResponse response = leaderGateWay.scanKv(newScanKvContinueRequest(fakeId, 0, 1024)).get(); @@ -1278,33 +1263,22 @@ void testScanKv_expiredOnRecentlyEvicted() throws Exception { .get()); FLUSS_CLUSTER_EXTENSION.triggerAndWaitSnapshot(tb); - // Open a long-lived session. ScanKvResponse open = leaderGateWay.scanKv(newScanKvOpenRequest(tableId, 0, 1)).get(); assertThat(open.hasErrorCode()).isFalse(); assertThat(open.isHasMoreResults()).isTrue(); byte[] scannerId = open.getScannerId(); - // Evict the session by stopping the leader replica (delete=false). Internally this - // calls scannerManager.closeScannersForBucket, which both removes the session and - // records the id in recentlyExpiredIds. The bucket itself remains hosted. + // stopReplica(delete=false) routes through closeScannersForBucket, which records + // the id in recentlyExpiredIds. LeaderAndIsr la = FLUSS_CLUSTER_EXTENSION.waitLeaderAndIsrReady(tb); stopReplicaWithLatestEpoch(leaderGateWay, tb, la); - // Continuation against the (still-live) tablet server now sees SCANNER_EXPIRED, not - // UNKNOWN_SCANNER_ID — the recently-expired cache lets the client distinguish - // "session was reaped" from "you fabricated an id". ScanKvResponse cont = leaderGateWay.scanKv(newScanKvContinueRequest(scannerId, 0, 1024)).get(); assertThat(cont.getErrorCode()).isEqualTo(Errors.SCANNER_EXPIRED.code()); assertThat(cont.getErrorMessage()).contains("expired"); } - /** - * Models a mid-scan leadership flip via {@code stopReplica(delete=false)}, which is the RPC - * path the coordinator drives when transitioning a leader away. Internally this calls {@code - * scannerManager.closeScannersForBucket}, the same chain a real leader -> follower transition - * takes via {@code makeFollower -> dropKv}, so the client-observable signal is identical. - */ @Test void testScanKv_leadershipFlipMidScan() throws Exception { long tableId = @@ -1329,17 +1303,9 @@ void testScanKv_leadershipFlipMidScan() throws Exception { assertThat(open.isHasMoreResults()).isTrue(); byte[] scannerId = open.getScannerId(); - // Drive the leader-side teardown. closeScannersForBucket runs synchronously inside - // ReplicaManager.stopReplicas and records the scanner id in recentlyExpiredIds - // before stopReplica returns, so by the time the next continuation arrives the - // session is gone but addressable as expired. LeaderAndIsr la = FLUSS_CLUSTER_EXTENSION.waitLeaderAndIsrReady(tb); stopReplicaWithLatestEpoch(leaderGateWay, tb, la); - // The continuation now sees a redirect-style signal so the client knows to find - // the new leader rather than retrying blindly. SCANNER_EXPIRED is the canonical - // response from the recently-expired cache; NOT_LEADER_OR_FOLLOWER is acceptable - // if the in-handler leadership re-check ever wins the race. ScanKvResponse cont = leaderGateWay.scanKv(newScanKvContinueRequest(scannerId, 0, 1024)).get(); assertThat(cont.getErrorCode()) @@ -1368,8 +1334,6 @@ void testScanKv_callSeqIdMismatch() throws Exception { ScanKvResponse open = leaderGateWay.scanKv(newScanKvOpenRequest(tableId, 0, 1)).get(); byte[] scannerId = open.getScannerId(); - // Initial server callSeqId is -1, so the first continuation MUST send 0. Sending 5 - // is detected as out-of-order and rejected. ScanKvResponse bad = leaderGateWay.scanKv(newScanKvContinueRequest(scannerId, 5, 1024)).get(); assertThat(bad.getErrorCode()).isEqualTo(Errors.INVALID_SCAN_REQUEST.code()); @@ -1395,8 +1359,6 @@ void testScanKv_oversizeBatchSizeBytesIsClamped() throws Exception { .get()); FLUSS_CLUSTER_EXTENSION.triggerAndWaitSnapshot(tb); - // Integer.MAX_VALUE >> kv.scanner.max-batch-size (10 MiB by default). The server - // must silently clamp and serve the request — not reject it and not OOM. ScanKvResponse response = leaderGateWay.scanKv(newScanKvOpenRequest(tableId, 0, Integer.MAX_VALUE)).get(); assertThat(response.hasErrorCode()).isFalse(); @@ -1416,9 +1378,6 @@ void testScanKv_bucketScanReqAndScannerIdRejected() throws Exception { TabletServerGateway leaderGateWay = FLUSS_CLUSTER_EXTENSION.newTabletServerClientForNode(leader); - // Setting both bucket_scan_req (new scan) AND scanner_id (continuation) is an - // ambiguous request and must be rejected at the head of scanKv before any state - // is mutated. ScanKvRequest bad = new ScanKvRequest(); bad.setBucketScanReq().setTableId(tableId).setBucketId(0); bad.setScannerId("ambiguous".getBytes()); @@ -1429,10 +1388,6 @@ void testScanKv_bucketScanReqAndScannerIdRejected() throws Exception { .contains("must not set both bucket_scan_req and scanner_id"); } - // ------------------------------------------------------------------------- - // scanKv request helpers - // ------------------------------------------------------------------------- - private static ScanKvRequest newScanKvOpenRequest(long tableId, int bucketId, int batchSize) { ScanKvRequest req = new ScanKvRequest(); req.setBucketScanReq().setTableId(tableId).setBucketId(bucketId); @@ -1456,12 +1411,7 @@ private static ScanKvRequest newScanKvCloseRequest(byte[] scannerId) { return req; } - /** - * Sends a stopReplica RPC for the given bucket using the coordinator epoch carried in {@code - * la}. The {@code FLUSS_CLUSTER_EXTENSION.stopReplica} helper hardcodes {@code - * coordinatorEpoch=0}, which is rejected by tablet servers once any earlier test in the same - * JVM has bumped the coordinator epoch above zero. - */ + /** Sends stopReplica using the current coordinator epoch (vs. the helper's hardcoded 0). */ private static void stopReplicaWithLatestEpoch( TabletServerGateway leaderGateWay, TableBucket tb, LeaderAndIsr la) throws Exception { leaderGateWay From af4e58ee19fdfef9886e3959146f98d67e35a9a4 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Sat, 9 May 2026 16:15:57 +0800 Subject: [PATCH 15/17] Refactor ScannerManager to manage max batch size for KV scans --- .../org/apache/fluss/server/kv/scan/ScannerContext.java | 2 +- .../org/apache/fluss/server/kv/scan/ScannerManager.java | 8 ++++++++ .../org/apache/fluss/server/tablet/TabletServer.java | 6 +----- .../org/apache/fluss/server/tablet/TabletService.java | 9 +++------ 4 files changed, 13 insertions(+), 12 deletions(-) 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 112c76d5bd..37846c1a68 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 @@ -102,7 +102,7 @@ public byte[] getScannerId() { return scannerIdBytes; } - /** Scanner ID as a UTF-8 string; used as the key in {@link ScannerManager#scanners}. */ + /** Scanner ID as a UTF-8 string; used as the key in {@code ScannerManager#scanners}. */ String getIdString() { return scannerId; } 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 145fb307a8..c6e834c0d6 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 @@ -80,6 +80,7 @@ public class ScannerManager implements AutoCloseableAsync { private final long recentlyExpiredRetentionMs; private final int maxPerBucket; private final int maxPerServer; + private final int maxBatchSizeBytes; @Nullable private ScheduledFuture evictorTask; @@ -94,6 +95,8 @@ public ScannerManager(Configuration conf, Scheduler scheduler) { this.recentlyExpiredRetentionMs = 2 * scannerTtlMs; this.maxPerBucket = conf.get(ConfigOptions.KV_SCANNER_MAX_PER_BUCKET); this.maxPerServer = conf.get(ConfigOptions.KV_SCANNER_MAX_PER_SERVER); + long configuredMaxBatch = conf.get(ConfigOptions.KV_SCANNER_MAX_BATCH_SIZE).getBytes(); + this.maxBatchSizeBytes = (int) Math.min(Integer.MAX_VALUE, configuredMaxBatch); long expirationIntervalMs = conf.get(ConfigOptions.KV_SCANNER_EXPIRATION_INTERVAL).toMillis(); @@ -180,6 +183,11 @@ public void removeScanner(byte[] scannerId) { } } + /** Returns the max batch size in bytes allowed for a kv scan response. */ + public int getMaxBatchSizeBytes() { + return maxBatchSizeBytes; + } + @VisibleForTesting public int activeScannerCount() { return totalScanners.get(); 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 b0bbed2290..d0f57db640 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 @@ -288,9 +288,6 @@ protected void startServices() throws Exception { // Start dynamicConfigManager after all reconfigurable components are registered dynamicConfigManager.startup(); - long configuredMaxBatch = conf.get(ConfigOptions.KV_SCANNER_MAX_BATCH_SIZE).getBytes(); - int kvScanMaxBatchSizeBytes = (int) Math.min(Integer.MAX_VALUE, configuredMaxBatch); - this.tabletService = new TabletService( serverId, @@ -302,8 +299,7 @@ protected void startServices() throws Exception { authorizer, dynamicConfigManager, ioExecutor, - scannerManager, - kvScanMaxBatchSizeBytes); + scannerManager); RequestsMetrics requestsMetrics = RequestsMetrics.createTabletServerRequestMetrics(tabletServerMetricGroup); 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 50219d2abc..88731daaba 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 @@ -149,8 +149,6 @@ public final class TabletService extends RpcServiceBase implements TabletServerG private final TabletServerMetadataProvider metadataFunctionProvider; private final ScannerManager scannerManager; - private final int kvScanMaxBatchSizeBytes; - public TabletService( int serverId, FileSystem remoteFileSystem, @@ -161,8 +159,7 @@ public TabletService( @Nullable Authorizer authorizer, DynamicConfigManager dynamicConfigManager, ExecutorService ioExecutor, - ScannerManager scannerManager, - int kvScanMaxBatchSizeBytes) { + ScannerManager scannerManager) { super( remoteFileSystem, ServerType.TABLET_SERVER, @@ -177,7 +174,6 @@ public TabletService( this.metadataFunctionProvider = new TabletServerMetadataProvider(zkClient, metadataManager, metadataCache); this.scannerManager = scannerManager; - this.kvScanMaxBatchSizeBytes = kvScanMaxBatchSizeBytes; } @Override @@ -486,7 +482,8 @@ public CompletableFuture scanKv(ScanKvRequest request) { throw new InvalidScanRequestException( "batch_size_bytes must be greater than 0."); } - effectiveBatchSize = Math.min(requestedBatchSize, kvScanMaxBatchSizeBytes); + effectiveBatchSize = + Math.min(requestedBatchSize, scannerManager.getMaxBatchSizeBytes()); } ScannerContext context; From fb0e133719cfdc7622150cb20354b705646e380f Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Sat, 9 May 2026 16:23:00 +0800 Subject: [PATCH 16/17] Refactor close method in ScannerContext to use IOUtils for resource management --- .../fluss/server/kv/scan/ScannerContext.java | 31 +++++-------------- 1 file changed, 8 insertions(+), 23 deletions(-) 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 37846c1a68..6347a3d439 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 @@ -21,6 +21,7 @@ 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.IOUtils; import org.rocksdb.ReadOptions; import org.rocksdb.RocksDBException; @@ -199,29 +200,13 @@ public void releaseAfterUse() { @Override public void close() { if (closed.compareAndSet(false, true)) { - // Fence: closing the iterator while another thread is mid iterator.next() is - // undefined at the RocksDB JNI boundary. tryAcquireForUse() re-checks `closed` - // after winning its CAS, so any racing waiter releases inUse and lets us through. - while (inUse.get()) { - Thread.yield(); - } - try { - iterator.close(); - } finally { - try { - readOptions.close(); - } finally { - try { - try { - rocksDBKv.getDb().releaseSnapshot(snapshot); - } finally { - snapshot.close(); - } - } finally { - resourceLease.close(); - } - } - } + // force close the inUse fence so any racing tryAcquireForUse() calls + inUse.set(false); + IOUtils.closeQuietly(iterator); + IOUtils.closeQuietly(readOptions); + IOUtils.closeQuietly(() -> rocksDBKv.getDb().releaseSnapshot(snapshot)); + IOUtils.closeQuietly(snapshot); + IOUtils.closeQuietly(resourceLease); } } } From 6b611fb70f346e3d98a735ca43bd5d9627439939 Mon Sep 17 00:00:00 2001 From: Jark Wu Date: Sat, 9 May 2026 17:00:02 +0800 Subject: [PATCH 17/17] fix test --- .../server/kv/scan/ScannerManagerTest.java | 25 ++++--------------- 1 file changed, 5 insertions(+), 20 deletions(-) diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java index 71f757bfcc..c04a907b32 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/scan/ScannerManagerTest.java @@ -54,7 +54,6 @@ import java.time.Duration; import java.util.ArrayList; import java.util.List; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import static org.apache.fluss.compression.ArrowCompressionInfo.DEFAULT_COMPRESSION; @@ -467,9 +466,9 @@ void testTryAcquireForUse_rejectsAfterClose() throws Exception { } } - /** close() must spin until inUse=false; iterating after close is undefined at the JNI. */ + /** close() force-clears inUse and completes immediately even if inUse=true. */ @Test - void testClose_waitsForInUse() throws Exception { + void testClose_notWaitForInUse() throws Exception { putAndFlush(3); try (ScannerManager manager = createManager()) { ScannerContext ctx = openAndRegister(manager); @@ -477,24 +476,10 @@ void testClose_waitsForInUse() throws Exception { assertThat(ctx.tryAcquireForUse()).isTrue(); - CountDownLatch closeStarted = new CountDownLatch(1); - Thread closer = - new Thread( - () -> { - closeStarted.countDown(); - ctx.close(); - }, - "scanner-context-closer"); - closer.start(); - - assertThat(closeStarted.await(5, TimeUnit.SECONDS)).isTrue(); - Thread.sleep(50); - assertThat(closer.isAlive()).as("close() must spin while inUse=true").isTrue(); - - ctx.releaseAfterUse(); - closer.join(TimeUnit.SECONDS.toMillis(5)); - assertThat(closer.isAlive()).isFalse(); + // close() should complete immediately despite inUse=true + ctx.close(); + // after close, tryAcquireForUse must be rejected assertThat(ctx.tryAcquireForUse()).isFalse(); manager.removeScanner(ctx);