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 8da2246b0a..a514886df3 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 @@ -1919,6 +1919,23 @@ public class ConfigOptions { .withDescription( "The max fetch size for fetching log to apply to kv during recovering kv."); + // ------------------------------------------------------------------------ + // ConfigOptions for KV lazy open + // ------------------------------------------------------------------------ + + public static final ConfigOption KV_LAZY_OPEN_ENABLED = + key("kv.lazy-open.enabled") + .booleanType() + .defaultValue(false) + .withDescription("Whether to enable KvTablet lazy open."); + + public static final ConfigOption KV_LAZY_OPEN_IDLE_TIMEOUT = + key("kv.lazy-open.idle-timeout") + .durationType() + .defaultValue(Duration.ofHours(24)) + .withDescription( + "Idle time before an open KvTablet is eligible for release back to lazy state."); + // ------------------------------------------------------------------------ // ConfigOptions for metrics // ------------------------------------------------------------------------ diff --git a/fluss-common/src/main/java/org/apache/fluss/metrics/MetricNames.java b/fluss-common/src/main/java/org/apache/fluss/metrics/MetricNames.java index b11c3b92fc..6b96ab2f01 100644 --- a/fluss-common/src/main/java/org/apache/fluss/metrics/MetricNames.java +++ b/fluss-common/src/main/java/org/apache/fluss/metrics/MetricNames.java @@ -147,6 +147,9 @@ public class MetricNames { "preWriteBufferTruncateAsDuplicatedPerSecond"; public static final String KV_PRE_WRITE_BUFFER_TRUNCATE_AS_ERROR_RATE = "preWriteBufferTruncateAsErrorPerSecond"; + public static final String KV_TABLET_OPEN_COUNT = "kvTabletOpenCount"; + public static final String KV_TABLET_LAZY_COUNT = "kvTabletLazyCount"; + public static final String KV_TABLET_FAILED_COUNT = "kvTabletFailedCount"; // -------------------------------------------------------------------------------------------- // RocksDB metrics diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/KvIdleReleaseController.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvIdleReleaseController.java new file mode 100644 index 0000000000..e290696f8d --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvIdleReleaseController.java @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.kv; + +import org.apache.fluss.annotation.VisibleForTesting; +import org.apache.fluss.utils.clock.Clock; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.util.Collection; +import java.util.Comparator; +import java.util.List; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +/** + * Periodically checks OPEN KvTablets and releases idle ones back to LAZY state. Tablets are sorted + * by last access time (LRU). Operates directly on {@link KvTablet} — no dependency on Replica + * layer. + */ +public class KvIdleReleaseController implements Closeable { + + private static final Logger LOG = LoggerFactory.getLogger(KvIdleReleaseController.class); + + private final ScheduledExecutorService scheduler; + private final Supplier> tabletSupplier; + private final Clock clock; + + private final long checkIntervalMs; + private final long idleIntervalMs; + + private volatile ScheduledFuture scheduledTask; + + public KvIdleReleaseController( + ScheduledExecutorService scheduler, + Supplier> tabletSupplier, + Clock clock, + long checkIntervalMs, + long idleIntervalMs) { + this.scheduler = scheduler; + this.tabletSupplier = tabletSupplier; + this.clock = clock; + this.checkIntervalMs = checkIntervalMs; + this.idleIntervalMs = idleIntervalMs; + } + + public void start() { + scheduledTask = + scheduler.scheduleWithFixedDelay( + this::checkAndRelease, + checkIntervalMs, + checkIntervalMs, + TimeUnit.MILLISECONDS); + LOG.info( + "KvIdleReleaseController started: checkInterval={}ms, idleInterval={}ms", + checkIntervalMs, + idleIntervalMs); + } + + /** + * Scans all open tablets, selects those idle beyond the configured threshold, and releases them + * in LRU order (coldest first). + */ + @VisibleForTesting + void checkAndRelease() { + try { + Collection tablets = tabletSupplier.get(); + long now = clock.milliseconds(); + + // Snapshot timestamp once per tablet to avoid double-read of volatile field + List idleCandidates = + tablets.stream() + .map(t -> new ReleaseCandidate(t, t.getLastAccessTimestamp())) + .filter(c -> now - c.lastAccessTimestamp > idleIntervalMs) + .sorted(Comparator.comparingLong(c -> c.lastAccessTimestamp)) + .collect(Collectors.toList()); + + if (idleCandidates.isEmpty()) { + LOG.debug("Idle release round: no idle tablets found"); + return; + } + + LOG.info("Idle release round: open={}, idle={}", tablets.size(), idleCandidates.size()); + + int released = 0; + for (ReleaseCandidate candidate : idleCandidates) { + KvTablet tablet = candidate.tablet; + + if (tablet.canRelease(idleIntervalMs, now)) { + try { + boolean success = tablet.releaseKv(); + if (success) { + released++; + LOG.debug( + "Released KvTablet for {} (idle {}ms)", + tablet.getTableBucket(), + now - candidate.lastAccessTimestamp); + } + } catch (Exception e) { + LOG.warn("Failed to release KvTablet for {}", tablet.getTableBucket(), e); + } + } + } + + if (released > 0) { + LOG.info( + "Idle release round complete: released={}/{}", + released, + idleCandidates.size()); + } + } catch (Exception e) { + LOG.error("Error during idle release check", e); + } + } + + @Override + public void close() { + if (scheduledTask != null) { + scheduledTask.cancel(false); + } + } + + private static class ReleaseCandidate { + final KvTablet tablet; + final long lastAccessTimestamp; + + ReleaseCandidate(KvTablet tablet, long lastAccessTimestamp) { + this.tablet = tablet; + this.lastAccessTimestamp = lastAccessTimestamp; + } + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/KvManager.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvManager.java index 1f0c9c2689..f885a3e2bc 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/KvManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvManager.java @@ -56,6 +56,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; import javax.annotation.concurrent.ThreadSafe; import java.io.File; @@ -64,6 +65,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.concurrent.Semaphore; import static org.apache.fluss.utils.concurrent.LockUtils.inLock; @@ -109,6 +111,18 @@ public static RateLimiter getDefaultRateLimiter() { return DEFAULT_RATE_LIMITER; } + // ---- KV lazy open internal constants ---- + private static final int LAZY_OPEN_MAX_CONCURRENT_OPENS = 10; + private static final long LAZY_OPEN_TIMEOUT_MS = 300_000; + private static final long LAZY_OPEN_FAILED_BACKOFF_BASE_MS = 5_000; + private static final long LAZY_OPEN_FAILED_BACKOFF_MAX_MS = 300_000; + + /** + * Timeout for draining pins during release in milliseconds. Public because {@link + * org.apache.fluss.server.replica.Replica} (different package) needs this value. + */ + public static final long RELEASE_DRAIN_TIMEOUT_MS = 5_000; + private final LogManager logManager; private final TabletServerMetricGroup serverMetricGroup; @@ -138,6 +152,13 @@ public static RateLimiter getDefaultRateLimiter() { /** Current shared rate limiter configuration in bytes per second. */ private volatile long currentSharedRateLimitBytesPerSec; + // ---- KV lazy open configuration ---- + private final boolean lazyOpenEnabled; + private final @Nullable Semaphore openSemaphore; + private final long openTimeoutMs; + private final long failedBackoffBaseMs; + private final long failedBackoffMaxMs; + private volatile boolean isShutdown = false; private KvManager( @@ -159,6 +180,20 @@ private KvManager( this.sharedRocksDBRateLimiter = createSharedRateLimiter(conf); this.currentSharedRateLimitBytesPerSec = conf.get(ConfigOptions.KV_SHARED_RATE_LIMITER_BYTES_PER_SEC).getBytes(); + + // KV lazy open configuration + this.lazyOpenEnabled = conf.get(ConfigOptions.KV_LAZY_OPEN_ENABLED); + if (lazyOpenEnabled) { + this.openSemaphore = new Semaphore(LAZY_OPEN_MAX_CONCURRENT_OPENS); + this.openTimeoutMs = LAZY_OPEN_TIMEOUT_MS; + this.failedBackoffBaseMs = LAZY_OPEN_FAILED_BACKOFF_BASE_MS; + this.failedBackoffMaxMs = LAZY_OPEN_FAILED_BACKOFF_MAX_MS; + } else { + this.openSemaphore = null; + this.openTimeoutMs = 0; + this.failedBackoffBaseMs = 0; + this.failedBackoffMaxMs = 0; + } } private static RateLimiter createSharedRateLimiter(Configuration conf) { @@ -196,6 +231,26 @@ public static KvManager create( tabletServerMetricGroup); } + public boolean isLazyOpenEnabled() { + return lazyOpenEnabled; + } + + public @Nullable Semaphore getOpenSemaphore() { + return openSemaphore; + } + + public long getOpenTimeoutMs() { + return openTimeoutMs; + } + + public long getFailedBackoffBaseMs() { + return failedBackoffBaseMs; + } + + public long getFailedBackoffMaxMs() { + return failedBackoffMaxMs; + } + public void startup() { // should do nothing now } @@ -221,7 +276,7 @@ public void shutdown() { /** * If the kv already exists, just return a copy of the existing kv. Otherwise, create a kv for - * the given table and the given bucket. + * the given table and the given bucket, and register it in the registry. * *

Note: if the parameter {@code partitionName} is null, the log dir path is: * /{database}/{table-name}-{table_id}/kv-{bucket-id}. Otherwise, the log dir path is: @@ -248,6 +303,36 @@ public KvTablet getOrCreateKv( return currentKvs.get(tableBucket); } + KvTablet tablet = + createKvTabletUnregistered( + tablePath, + tableBucket, + logTablet, + kvFormat, + schemaGetter, + tableConfig, + arrowCompressionInfo); + currentKvs.put(tableBucket, tablet); + return tablet; + }); + } + + /** + * Create a new KvTablet without registering it in {@code currentKvs}. The caller is responsible + * for registration via {@link #registerKv(TableBucket, KvTablet)} if needed. + */ + public KvTablet createKvTabletUnregistered( + PhysicalTablePath tablePath, + TableBucket tableBucket, + LogTablet logTablet, + KvFormat kvFormat, + SchemaGetter schemaGetter, + TableConfig tableConfig, + ArrowCompressionInfo arrowCompressionInfo) + throws Exception { + return inLock( + tabletCreationOrDeletionLock, + () -> { File tabletDir = getOrCreateTabletDir(tablePath, tableBucket); RowMerger merger = RowMerger.create(tableConfig, kvFormat, schemaGetter); AutoIncrementManager autoIncrementManager = @@ -274,7 +359,6 @@ public KvTablet getOrCreateKv( tableConfig.getChangelogImage(), sharedRocksDBRateLimiter, autoIncrementManager); - currentKvs.put(tableBucket, tablet); LOG.info( "Created kv tablet for bucket {} in dir {}.", @@ -344,6 +428,30 @@ public void dropKv(TableBucket tableBucket) { } public KvTablet loadKv(File tabletDir, SchemaGetter schemaGetter) throws Exception { + KvTablet kvTablet = loadKvUnregistered(tabletDir, schemaGetter); + TableBucket tableBucket = kvTablet.getTableBucket(); + if (this.currentKvs.containsKey(tableBucket)) { + throw new IllegalStateException( + String.format( + "Duplicate kv tablet directories for bucket %s are found in both %s and %s. " + + "Recover server from this " + + "failure by manually deleting one of the two kv directories for this bucket. " + + "It is recommended to delete the bucket in the kv tablet directory that is " + + "known to have failed recently.", + tableBucket, + tabletDir.getAbsolutePath(), + currentKvs.get(tableBucket).getKvTabletDir().getAbsolutePath())); + } + this.currentKvs.put(tableBucket, kvTablet); + return kvTablet; + } + + /** + * Load a KvTablet from an existing tablet directory without registering it in {@code + * currentKvs}. The caller is responsible for registration via {@link #registerKv(TableBucket, + * KvTablet)} if needed. + */ + public KvTablet loadKvUnregistered(File tabletDir, SchemaGetter schemaGetter) throws Exception { Tuple2 pathAndBucket = FlussPaths.parseTabletDir(tabletDir); PhysicalTablePath physicalTablePath = pathAndBucket.f0; TableBucket tableBucket = pathAndBucket.f1; @@ -374,38 +482,48 @@ public KvTablet loadKv(File tabletDir, SchemaGetter schemaGetter) throws Excepti tablePath, tableConfig, new ZkSequenceGeneratorFactory(zkClient)); - KvTablet kvTablet = - KvTablet.create( - physicalTablePath, - tableBucket, - logTablet, - tabletDir, - conf, - serverMetricGroup, - arrowBufferAllocator, - memorySegmentPool, - tableConfig.getKvFormat(), - rowMerger, - tableConfig.getArrowCompressionInfo(), - schemaGetter, - tableConfig.getChangelogImage(), - sharedRocksDBRateLimiter, - autoIncrementManager); - if (this.currentKvs.containsKey(tableBucket)) { - throw new IllegalStateException( - String.format( - "Duplicate kv tablet directories for bucket %s are found in both %s and %s. " - + "Recover server from this " - + "failure by manually deleting one of the two kv directories for this bucket. " - + "It is recommended to delete the bucket in the kv tablet directory that is " - + "known to have failed recently.", - tableBucket, - tabletDir.getAbsolutePath(), - currentKvs.get(tableBucket).getKvTabletDir().getAbsolutePath())); - } - this.currentKvs.put(tableBucket, kvTablet); + return KvTablet.create( + physicalTablePath, + tableBucket, + logTablet, + tabletDir, + conf, + serverMetricGroup, + arrowBufferAllocator, + memorySegmentPool, + tableConfig.getKvFormat(), + rowMerger, + tableConfig.getArrowCompressionInfo(), + schemaGetter, + tableConfig.getChangelogImage(), + sharedRocksDBRateLimiter, + autoIncrementManager); + } - return kvTablet; + /** Register a KvTablet (e.g. a lazy sentinel) into the {@code currentKvs} registry. */ + public void registerKv(TableBucket tableBucket, KvTablet kvTablet) { + inLock( + tabletCreationOrDeletionLock, + () -> { + if (currentKvs.containsKey(tableBucket)) { + throw new IllegalStateException( + "KvTablet already registered for " + tableBucket); + } + currentKvs.put(tableBucket, kvTablet); + }); + } + + /** Remove a KvTablet from the {@code currentKvs} registry (e.g. on drop). */ + public void unregisterKv(TableBucket tableBucket) { + inLock(tabletCreationOrDeletionLock, () -> currentKvs.remove(tableBucket)); + } + + /** + * Get the tablet directory path for the given table path and table bucket. This is a read-only + * accessor — it does not create the directory. + */ + public File getTabletDirPath(PhysicalTablePath tablePath, TableBucket tableBucket) { + return getTabletDir(tablePath, tableBucket); } public void deleteRemoteKvSnapshot( 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 e72428a02f..617012fb68 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 @@ -74,6 +74,7 @@ import org.apache.fluss.types.RowType; import org.apache.fluss.utils.BytesUtils; import org.apache.fluss.utils.FileUtils; +import org.apache.fluss.utils.clock.Clock; import org.rocksdb.RateLimiter; import org.slf4j.Logger; @@ -99,52 +100,69 @@ @ThreadSafe public final class KvTablet { private static final Logger LOG = LoggerFactory.getLogger(KvTablet.class); - private static final long ROW_COUNT_DISABLED = -1; + public static final long ROW_COUNT_DISABLED = -1; + + /** + * A guard that prevents the underlying RocksDB from being released while held. Use with + * try-with-resources. In eager mode, this is a no-op. In lazy mode, it pins the RocksDB + * instance and ensures it is open before returning. + */ + public static final class Guard implements AutoCloseable { + private final @Nullable KvTablet tablet; + private boolean released; + + /** No-op guard for eager mode. */ + static final Guard NOOP = new Guard(null); + + Guard(@Nullable KvTablet tablet) { + this.tablet = tablet; + } + + @Override + public void close() { + if (!released && tablet != null) { + released = true; + tablet.releasePin(); + } + } + } private final PhysicalTablePath physicalPath; private final TableBucket tableBucket; - private final LogTablet logTablet; - private final ArrowWriterProvider arrowWriterProvider; - private final MemorySegmentPool memorySegmentPool; + /** Package-private for {@link KvTabletLazyLifecycle} access (log end offset during reopen). */ + final LogTablet logTablet; - private final File kvTabletDir; - private final long writeBatchSize; - private final RocksDBKv rocksDBKv; - private final KvPreWriteBuffer kvPreWriteBuffer; - private final TabletServerMetricGroup serverMetricGroup; + /** Package-private for {@link KvTabletLazyLifecycle} access (gauge updates). */ + final @Nullable TabletServerMetricGroup serverMetricGroup; // A lock that guards all modifications to the kv. private final ReadWriteLock kvLock = new ReentrantReadWriteLock(); - private final LogFormat logFormat; - private final KvFormat kvFormat; - // defines how to merge rows on the same primary key - private final RowMerger rowMerger; - // Pre-created DefaultRowMerger for OVERWRITE mode (undo recovery scenarios) - // This avoids creating a new instance on every putAsLeader call - private final RowMerger overwriteRowMerger; - private final ArrowCompressionInfo arrowCompressionInfo; - private final AutoIncrementManager autoIncrementManager; - - private final SchemaGetter schemaGetter; - // the changelog image mode for this tablet - private final ChangelogImage changelogImage; + // ---- Immutable holder for all RocksDB-related fields ---- + // In EAGER mode all fields are final (JIT can cache in registers). + // In LAZY mode a single volatile write publishes the entire state atomically. - // RocksDB statistics accessor for this tablet - @Nullable private final RocksDBStatistics rocksDBStatistics; + private volatile @Nullable RocksDBState rocksDBState; /** * The kv data in pre-write buffer whose log offset is less than the flushedLogOffset has been - * flushed into kv. + * flushed into kv. Package-private volatile for cross-thread access by {@link + * KvTabletLazyLifecycle} during release (caching metadata). */ - private volatile long flushedLogOffset = 0; + volatile long flushedLogOffset = 0; - private volatile long rowCount; + /** Current row count. Package-private volatile for {@link KvTabletLazyLifecycle} access. */ + volatile long rowCount; @GuardedBy("kvLock") private volatile boolean isClosed = false; + // ---- Lazy lifecycle (null in EAGER mode) ---- + + private @Nullable KvTabletLazyLifecycle lifecycle; + + /** Full constructor for eager mode (RocksDB is immediately available). */ private KvTablet( PhysicalTablePath physicalPath, TableBucket tableBucket, @@ -166,28 +184,101 @@ private KvTablet( this.physicalPath = physicalPath; this.tableBucket = tableBucket; this.logTablet = logTablet; - this.kvTabletDir = kvTabletDir; - this.rocksDBKv = rocksDBKv; - this.writeBatchSize = writeBatchSize; this.serverMetricGroup = serverMetricGroup; - this.kvPreWriteBuffer = new KvPreWriteBuffer(createKvBatchWriter(), serverMetricGroup); - this.logFormat = logFormat; - this.arrowWriterProvider = new ArrowWriterPool(arrowBufferAllocator); - this.memorySegmentPool = memorySegmentPool; - this.kvFormat = kvFormat; - this.rowMerger = rowMerger; // Pre-create DefaultRowMerger for OVERWRITE mode to avoid creating new instances // on every putAsLeader call. Used for undo recovery scenarios. - this.overwriteRowMerger = new DefaultRowMerger(kvFormat, DeleteBehavior.ALLOW); - this.arrowCompressionInfo = arrowCompressionInfo; - this.schemaGetter = schemaGetter; - this.changelogImage = changelogImage; - this.rocksDBStatistics = rocksDBStatistics; - this.autoIncrementManager = autoIncrementManager; + RowMerger overwriteRowMerger = new DefaultRowMerger(kvFormat, DeleteBehavior.ALLOW); + ArrowWriterProvider arrowWriterProvider = new ArrowWriterPool(arrowBufferAllocator); + KvPreWriteBuffer kvPreWriteBuffer = + new KvPreWriteBuffer( + rocksDBKv.newWriteBatch( + writeBatchSize, + serverMetricGroup.kvFlushCount(), + serverMetricGroup.kvFlushLatencyHistogram()), + serverMetricGroup); + this.rocksDBState = + new RocksDBState( + kvTabletDir, + rocksDBKv, + writeBatchSize, + kvPreWriteBuffer, + logFormat, + kvFormat, + arrowWriterProvider, + memorySegmentPool, + rowMerger, + overwriteRowMerger, + arrowCompressionInfo, + schemaGetter, + changelogImage, + rocksDBStatistics, + autoIncrementManager); // disable row count for WAL image mode. this.rowCount = changelogImage == ChangelogImage.WAL ? ROW_COUNT_DISABLED : 0L; } + /** + * Lazy sentinel constructor: creates a KvTablet in LAZY state without RocksDB. The RocksDB + * fields are populated later via {@link #installRocksDB(KvTablet)} when ensureOpen() triggers. + */ + private KvTablet( + PhysicalTablePath physicalPath, + TableBucket tableBucket, + LogTablet logTablet, + @Nullable TabletServerMetricGroup serverMetricGroup) { + this.physicalPath = physicalPath; + this.tableBucket = tableBucket; + this.logTablet = logTablet; + this.serverMetricGroup = serverMetricGroup; + this.lifecycle = new KvTabletLazyLifecycle(this); + } + + /** + * Create a lazy sentinel KvTablet that starts in LAZY state without RocksDB. Use {@link + * KvTabletLazyLifecycle#configureLazyOpen} and callback setters on the lifecycle to configure + * before use. + */ + public static KvTablet createLazySentinel( + PhysicalTablePath physicalPath, + TableBucket tableBucket, + LogTablet logTablet, + @Nullable TabletServerMetricGroup serverMetricGroup) { + return new KvTablet(physicalPath, tableBucket, logTablet, serverMetricGroup); + } + + /** Returns the lifecycle manager, or null if this tablet is in EAGER mode. */ + @Nullable + public KvTabletLazyLifecycle getLifecycle() { + return lifecycle; + } + + /** + * Install RocksDB state from a fully-constructed KvTablet into this lazy sentinel. Called by + * commitOpenResult() after a successful open. The source tablet is consumed (its RocksDB + * reference is transferred, not copied). + */ + void installRocksDB(KvTablet source) { + this.rocksDBState = source.rocksDBState; + this.rowCount = source.rowCount; + this.flushedLogOffset = source.flushedLogOffset; + this.isClosed = false; + source.rocksDBState = null; + source.isClosed = true; + } + + /** + * Detach RocksDB state from this sentinel (during release). Nulls out RocksDB-related fields so + * the sentinel returns to a lightweight state. + */ + void detachRocksDB() { + RocksDBState s = this.rocksDBState; + if (s != null) { + closeRocksDBState(s, "detach"); + } + this.rocksDBState = null; + this.isClosed = true; + } + public static KvTablet create( PhysicalTablePath tablePath, TableBucket tableBucket, @@ -262,11 +353,15 @@ public TablePath getTablePath() { } public long getAutoIncrementCacheSize() { - return autoIncrementManager.getAutoIncrementCacheSize(); + RocksDBState s = this.rocksDBState; + return s != null ? s.autoIncrementManager.getAutoIncrementCacheSize() : 0; } public void updateAutoIncrementIDRange(AutoIncIDRange newRange) { - autoIncrementManager.updateIDRange(newRange); + RocksDBState s = this.rocksDBState; + if (s != null) { + s.autoIncrementManager.updateIDRange(newRange); + } } @Nullable @@ -275,7 +370,8 @@ public String getPartitionName() { } public File getKvTabletDir() { - return kvTabletDir; + RocksDBState s = this.rocksDBState; + return s != null ? s.kvTabletDir : null; } /** @@ -285,7 +381,13 @@ public File getKvTabletDir() { */ @Nullable public RocksDBStatistics getRocksDBStatistics() { - return rocksDBStatistics; + RocksDBState s = this.rocksDBState; + return s != null ? s.rocksDBStatistics : null; + } + + /** Returns the log offset up to which data has been flushed from pre-write buffer into KV. */ + public long getFlushedLogOffset() { + return flushedLogOffset; } void setFlushedLogOffset(long flushedLogOffset) { @@ -298,6 +400,19 @@ void setRowCount(long rowCount) { // row_count is volatile, so it's safe to read without lock public long getRowCount() { + // In LAZY state, RocksDB is not open — return cached value + if (lifecycle != null && lifecycle.needsCachedRowCount()) { + long cached = lifecycle.getCachedRowCount(); + if (cached == ROW_COUNT_DISABLED) { + throw new InvalidTableException( + String.format( + "Row count is disabled for this table '%s'. " + + "This usually happens when the table is " + + "created before v0.9 or the changelog image is set to WAL.", + getTablePath())); + } + return cached; + } if (rowCount == ROW_COUNT_DISABLED) { throw new InvalidTableException( String.format( @@ -320,10 +435,11 @@ public long getRowCount() { */ @GuardedBy("kvLock") public TabletState getTabletState() { + RocksDBState s = this.rocksDBState; return new TabletState( flushedLogOffset, rowCount == ROW_COUNT_DISABLED ? null : rowCount, - autoIncrementManager.getCurrentIDRanges()); + s.autoIncrementManager.getCurrentIDRanges()); } /** @@ -364,18 +480,19 @@ public LogAppendInfo putAsLeader(KvRecordBatch kvRecords, @Nullable int[] target public LogAppendInfo putAsLeader( KvRecordBatch kvRecords, @Nullable int[] targetColumns, MergeMode mergeMode) throws Exception { + RocksDBState s = this.rocksDBState; return inWriteLock( kvLock, () -> { - rocksDBKv.checkIfRocksDBClosed(); + s.rocksDBKv.checkIfRocksDBClosed(); - SchemaInfo schemaInfo = schemaGetter.getLatestSchemaInfo(); + SchemaInfo schemaInfo = s.schemaGetter.getLatestSchemaInfo(); Schema latestSchema = schemaInfo.getSchema(); short latestSchemaId = (short) schemaInfo.getSchemaId(); validateSchemaId(kvRecords.schemaId(), latestSchemaId); AutoIncrementUpdater currentAutoIncrementUpdater = - autoIncrementManager.getUpdaterForSchema(kvFormat, latestSchemaId); + s.autoIncrementManager.getUpdaterForSchema(s.kvFormat, latestSchemaId); // Validate targetColumns doesn't contain auto-increment column currentAutoIncrementUpdater.validateTargetColumns(targetColumns); @@ -387,13 +504,13 @@ public LogAppendInfo putAsLeader( // We only support ADD COLUMN, so targetColumns is fine to be used directly. RowMerger currentMerger = (mergeMode == MergeMode.OVERWRITE) - ? overwriteRowMerger.configureTargetColumns( + ? s.overwriteRowMerger.configureTargetColumns( targetColumns, latestSchemaId, latestSchema) - : rowMerger.configureTargetColumns( + : s.rowMerger.configureTargetColumns( targetColumns, latestSchemaId, latestSchema); RowType latestRowType = latestSchema.getRowType(); - WalBuilder walBuilder = createWalBuilder(latestSchemaId, latestRowType); + WalBuilder walBuilder = createWalBuilder(s, latestSchemaId, latestRowType); walBuilder.setWriterState(kvRecords.writerId(), kvRecords.batchSequence()); // we only support ADD COLUMN LAST, so the BinaryRow after RowMerger is // only has fewer ending columns than latest schema, so we pad nulls to @@ -404,6 +521,7 @@ public LogAppendInfo putAsLeader( try { processKvRecords( + s, kvRecords, kvRecords.schemaId(), currentMerger, @@ -427,7 +545,7 @@ public LogAppendInfo putAsLeader( // if the batch is duplicated, we should truncate the kvPreWriteBuffer // already written. if (logAppendInfo.duplicated()) { - kvPreWriteBuffer.truncateTo( + s.kvPreWriteBuffer.truncateTo( logEndOffsetOfPrevBatch, TruncateReason.DUPLICATED); } return logAppendInfo; @@ -438,7 +556,8 @@ public LogAppendInfo putAsLeader( // retry-send batch will produce incorrect CDC logs. // TODO for some errors, the cdc logs may already be written to disk, for // those errors, we should not truncate the kvPreWriteBuffer. - kvPreWriteBuffer.truncateTo(logEndOffsetOfPrevBatch, TruncateReason.ERROR); + s.kvPreWriteBuffer.truncateTo( + logEndOffsetOfPrevBatch, TruncateReason.ERROR); throw t; } finally { // deallocate the memory and arrow writer used by the wal builder @@ -458,6 +577,7 @@ private void validateSchemaId(short schemaIdOfNewData, short latestSchemaId) { } private void processKvRecords( + RocksDBState s, KvRecordBatch kvRecords, short schemaIdOfNewData, RowMerger currentMerger, @@ -470,8 +590,8 @@ private void processKvRecords( // TODO: reuse the read context and decoder KvRecordBatch.ReadContext readContext = - KvRecordReadContext.createReadContext(kvFormat, schemaGetter); - ValueDecoder valueDecoder = new ValueDecoder(schemaGetter, kvFormat); + KvRecordReadContext.createReadContext(s.kvFormat, s.schemaGetter); + ValueDecoder valueDecoder = new ValueDecoder(s.schemaGetter, s.kvFormat); for (KvRecord kvRecord : kvRecords.records(readContext)) { byte[] keyBytes = BytesUtils.toArray(kvRecord.getKey()); @@ -482,6 +602,7 @@ private void processKvRecords( if (currentValue == null) { logOffset = processDeletion( + s, key, currentMerger, valueDecoder, @@ -491,6 +612,7 @@ private void processKvRecords( } else { logOffset = processUpsert( + s, key, currentValue, currentMerger, @@ -504,6 +626,7 @@ private void processKvRecords( } private long processDeletion( + RocksDBState s, KvPreWriteBuffer.Key key, RowMerger currentMerger, ValueDecoder valueDecoder, @@ -521,7 +644,7 @@ private long processDeletion( + "The table.delete.behavior is set to 'disable'."); } - byte[] oldValueBytes = getFromBufferOrKv(key); + byte[] oldValueBytes = getFromBufferOrKv(s, key); if (oldValueBytes == null) { LOG.debug( "The specific key can't be found in kv tablet although the kv record is for deletion, " @@ -534,13 +657,14 @@ private long processDeletion( // if newValue is null, it means the row should be deleted if (newValue == null) { - return applyDelete(key, oldValue, walBuilder, latestSchemaRow, logOffset); + return applyDelete(s, key, oldValue, walBuilder, latestSchemaRow, logOffset); } else { - return applyUpdate(key, oldValue, newValue, walBuilder, latestSchemaRow, logOffset); + return applyUpdate(s, key, oldValue, newValue, walBuilder, latestSchemaRow, logOffset); } } private long processUpsert( + RocksDBState s, KvPreWriteBuffer.Key key, BinaryValue currentValue, RowMerger currentMerger, @@ -554,15 +678,16 @@ private long processUpsert( // and there is no auto-increment column, we can skip fetching old value for better // performance since the result always reflects the new value. In this case, both INSERT and // UPDATE will produce UPDATE_AFTER. - if (changelogImage == ChangelogImage.WAL + if (s.changelogImage == ChangelogImage.WAL && !autoIncrementUpdater.hasAutoIncrement() && currentMerger instanceof DefaultRowMerger) { - return applyUpdate(key, null, currentValue, walBuilder, latestSchemaRow, logOffset); + return applyUpdate(s, key, null, currentValue, walBuilder, latestSchemaRow, logOffset); } - byte[] oldValueBytes = getFromBufferOrKv(key); + byte[] oldValueBytes = getFromBufferOrKv(s, key); if (oldValueBytes == null) { return applyInsert( + s, key, currentValue, walBuilder, @@ -579,10 +704,11 @@ private long processUpsert( return logOffset; } - return applyUpdate(key, oldValue, newValue, walBuilder, latestSchemaRow, logOffset); + return applyUpdate(s, key, oldValue, newValue, walBuilder, latestSchemaRow, logOffset); } private long applyDelete( + RocksDBState s, KvPreWriteBuffer.Key key, BinaryValue oldValue, WalBuilder walBuilder, @@ -590,11 +716,12 @@ private long applyDelete( long logOffset) throws Exception { walBuilder.append(ChangeType.DELETE, latestSchemaRow.replaceRow(oldValue.row)); - kvPreWriteBuffer.delete(key, logOffset); + s.kvPreWriteBuffer.delete(key, logOffset); return logOffset + 1; } private long applyInsert( + RocksDBState s, KvPreWriteBuffer.Key key, BinaryValue currentValue, WalBuilder walBuilder, @@ -604,11 +731,12 @@ private long applyInsert( throws Exception { BinaryValue newValue = autoIncrementUpdater.updateAutoIncrementColumns(currentValue); walBuilder.append(ChangeType.INSERT, latestSchemaRow.replaceRow(newValue.row)); - kvPreWriteBuffer.insert(key, newValue.encodeValue(), logOffset); + s.kvPreWriteBuffer.insert(key, newValue.encodeValue(), logOffset); return logOffset + 1; } private long applyUpdate( + RocksDBState s, KvPreWriteBuffer.Key key, BinaryValue oldValue, BinaryValue newValue, @@ -616,22 +744,23 @@ private long applyUpdate( PaddingRow latestSchemaRow, long logOffset) throws Exception { - if (changelogImage == ChangelogImage.WAL) { + if (s.changelogImage == ChangelogImage.WAL) { walBuilder.append(ChangeType.UPDATE_AFTER, latestSchemaRow.replaceRow(newValue.row)); - kvPreWriteBuffer.update(key, newValue.encodeValue(), logOffset); + s.kvPreWriteBuffer.update(key, newValue.encodeValue(), logOffset); return logOffset + 1; } else { walBuilder.append(ChangeType.UPDATE_BEFORE, latestSchemaRow.replaceRow(oldValue.row)); walBuilder.append(ChangeType.UPDATE_AFTER, latestSchemaRow.replaceRow(newValue.row)); - kvPreWriteBuffer.update(key, newValue.encodeValue(), logOffset + 1); + s.kvPreWriteBuffer.update(key, newValue.encodeValue(), logOffset + 1); return logOffset + 2; } } - private WalBuilder createWalBuilder(int schemaId, RowType rowType) throws Exception { - switch (logFormat) { + private WalBuilder createWalBuilder(RocksDBState s, int schemaId, RowType rowType) + throws Exception { + switch (s.logFormat) { case INDEXED: - if (kvFormat == KvFormat.COMPACTED) { + if (s.kvFormat == KvFormat.COMPACTED) { // convert from compacted row to indexed row is time cost, and gain // less benefits, currently we won't support compacted as kv format and // indexed as cdc log format. @@ -639,70 +768,82 @@ private WalBuilder createWalBuilder(int schemaId, RowType rowType) throws Except throw new IllegalArgumentException( "Primary Key Table with COMPACTED kv format doesn't support INDEXED cdc log format."); } - return new IndexWalBuilder(schemaId, memorySegmentPool); + return new IndexWalBuilder(schemaId, s.memorySegmentPool); case COMPACTED: - return new CompactedWalBuilder(schemaId, rowType, memorySegmentPool); + return new CompactedWalBuilder(schemaId, rowType, s.memorySegmentPool); case ARROW: return new ArrowWalBuilder( schemaId, - arrowWriterProvider.getOrCreateWriter( + s.arrowWriterProvider.getOrCreateWriter( tableBucket.getTableId(), schemaId, // we don't limit size of the arrow batch, because all the // changelogs should be in a single batch Integer.MAX_VALUE, rowType, - arrowCompressionInfo), - memorySegmentPool); + s.arrowCompressionInfo), + s.memorySegmentPool); default: - throw new IllegalArgumentException("Unsupported log format: " + logFormat); + throw new IllegalArgumentException("Unsupported log format: " + s.logFormat); } } public void flush(long exclusiveUpToLogOffset, FatalErrorHandler fatalErrorHandler) { - // todo: need to introduce a backpressure mechanism - // to avoid too much records in kvPreWriteBuffer - inWriteLock( - kvLock, - () -> { - // when kv manager is closed which means kv tablet is already closed, - // but the tablet server may still handle fetch log request from follower - // as the tablet rpc service is closed asynchronously, then update the watermark - // and then flush the pre-write buffer. - - // In such case, if the tablet is already closed, we won't flush pre-write - // buffer, just warning it. - if (isClosed) { - LOG.warn( - "The kv tablet for {} is already closed, ignore flushing kv pre-write buffer.", - tableBucket); - } else { - try { - int rowCountDiff = kvPreWriteBuffer.flush(exclusiveUpToLogOffset); - flushedLogOffset = exclusiveUpToLogOffset; - if (rowCount != ROW_COUNT_DISABLED) { - // row count is enabled, we update the row count after flush. - long currentRowCount = rowCount; - rowCount = currentRowCount + rowCountDiff; + Guard guard = tryAcquireExistingGuard(); + if (guard == null) { + return; + } + try (Guard ignored = guard) { + RocksDBState s = this.rocksDBState; + if (s == null) { + return; + } + // todo: need to introduce a backpressure mechanism + // to avoid too much records in kvPreWriteBuffer + inWriteLock( + kvLock, + () -> { + // when kv manager is closed which means kv tablet is already closed, + // but the tablet server may still handle fetch log request from follower + // as the tablet rpc service is closed asynchronously, then update the + // watermark and then flush the pre-write buffer. + + // In such case, if the tablet is already closed, we won't flush pre-write + // buffer, just warning it. + if (isClosed) { + LOG.warn( + "The kv tablet for {} is already closed, ignore flushing kv pre-write buffer.", + tableBucket); + } else { + try { + int rowCountDiff = s.kvPreWriteBuffer.flush(exclusiveUpToLogOffset); + flushedLogOffset = exclusiveUpToLogOffset; + if (rowCount != ROW_COUNT_DISABLED) { + // row count is enabled, we update the row count after flush. + long currentRowCount = rowCount; + rowCount = currentRowCount + rowCountDiff; + } + } catch (Throwable t) { + fatalErrorHandler.onFatalError( + new KvStorageException( + "Failed to flush kv pre-write buffer.")); } - } catch (Throwable t) { - fatalErrorHandler.onFatalError( - new KvStorageException("Failed to flush kv pre-write buffer.")); } - } - }); + }); + } } /** put key,value,logOffset into pre-write buffer directly. */ void putToPreWriteBuffer( ChangeType changeType, byte[] key, @Nullable byte[] value, long logOffset) { + RocksDBState s = this.rocksDBState; KvPreWriteBuffer.Key wrapKey = KvPreWriteBuffer.Key.of(key); if (changeType == ChangeType.DELETE && value == null) { - kvPreWriteBuffer.delete(wrapKey, logOffset); + s.kvPreWriteBuffer.delete(wrapKey, logOffset); } else if (changeType == ChangeType.INSERT) { - kvPreWriteBuffer.insert(wrapKey, value, logOffset); + s.kvPreWriteBuffer.insert(wrapKey, value, logOffset); } else if (changeType == ChangeType.UPDATE_AFTER) { - kvPreWriteBuffer.update(wrapKey, value, logOffset); + s.kvPreWriteBuffer.update(wrapKey, value, logOffset); } else { throw new IllegalArgumentException( "Unsupported change type for putToPreWriteBuffer: " + changeType); @@ -717,54 +858,70 @@ void putToPreWriteBuffer( * tablet. */ public Executor getGuardedExecutor() { - return runnable -> inWriteLock(kvLock, runnable::run); + return runnable -> { + Guard guard = tryAcquireExistingGuard(); + if (guard == null) { + return; + } + try (Guard ignored = guard) { + inWriteLock(kvLock, runnable::run); + } + }; } // get from kv pre-write buffer first, if can't find, get from rocksdb - private byte[] getFromBufferOrKv(KvPreWriteBuffer.Key key) throws IOException { - KvPreWriteBuffer.Value value = kvPreWriteBuffer.get(key); + private byte[] getFromBufferOrKv(RocksDBState s, KvPreWriteBuffer.Key key) throws IOException { + KvPreWriteBuffer.Value value = s.kvPreWriteBuffer.get(key); if (value == null) { - return rocksDBKv.get(key.get()); + return s.rocksDBKv.get(key.get()); } return value.get(); } public List multiGet(List keys) throws IOException { + RocksDBState s = this.rocksDBState; return inReadLock( kvLock, () -> { - rocksDBKv.checkIfRocksDBClosed(); - return rocksDBKv.multiGet(keys); + s.rocksDBKv.checkIfRocksDBClosed(); + return s.rocksDBKv.multiGet(keys); }); } public List prefixLookup(byte[] prefixKey) throws IOException { + RocksDBState s = this.rocksDBState; return inReadLock( kvLock, () -> { - rocksDBKv.checkIfRocksDBClosed(); - return rocksDBKv.prefixLookup(prefixKey); + s.rocksDBKv.checkIfRocksDBClosed(); + return s.rocksDBKv.prefixLookup(prefixKey); }); } public List limitScan(int limit) throws IOException { + RocksDBState s = this.rocksDBState; return inReadLock( kvLock, () -> { - rocksDBKv.checkIfRocksDBClosed(); - return rocksDBKv.limitScan(limit); + s.rocksDBKv.checkIfRocksDBClosed(); + return s.rocksDBKv.limitScan(limit); }); } public KvBatchWriter createKvBatchWriter() { - return rocksDBKv.newWriteBatch( - writeBatchSize, + RocksDBState s = this.rocksDBState; + return s.rocksDBKv.newWriteBatch( + s.writeBatchSize, serverMetricGroup.kvFlushCount(), serverMetricGroup.kvFlushLatencyHistogram()); } public void close() throws Exception { LOG.info("close kv tablet {} for table {}.", tableBucket, physicalPath); + if (lifecycle != null) { + lifecycle.closeKvLazy(); + return; + } inWriteLock( kvLock, () -> { @@ -773,8 +930,9 @@ public void close() throws Exception { } // Note: RocksDB metrics lifecycle is managed by TableMetricGroup // No need to close it here - if (rocksDBKv != null) { - rocksDBKv.close(); + RocksDBState s = this.rocksDBState; + if (s != null) { + closeRocksDBState(s, "close"); } isClosed = true; }); @@ -788,32 +946,258 @@ public void drop() throws Exception { // first close the kv. close(); // then delete the directory. - FileUtils.deleteDirectory(kvTabletDir); + RocksDBState s = this.rocksDBState; + if (s != null) { + FileUtils.deleteDirectory(s.kvTabletDir); + } }); } + /** + * Delete the local tablet directory. Prefers {@code kvTabletDir} if available (OPEN state), + * falls back to lifecycle's tabletDirSupplier (LAZY/FAILED state). + */ + void deleteLocalDirectory() { + RocksDBState s = this.rocksDBState; + File dir = s != null ? s.kvTabletDir : null; + if (dir == null && lifecycle != null) { + dir = lifecycle.getTabletDir(); + } + if (dir != null) { + FileUtils.deleteDirectoryQuietly(dir); + LOG.info( + "Deleted local data directory for bucket {} at {}.", + tableBucket, + dir.getAbsolutePath()); + } + } + public RocksIncrementalSnapshot createIncrementalSnapshot( Map> uploadedSstFiles, KvSnapshotDataUploader kvSnapshotDataUploader, long lastCompletedSnapshotId) { + RocksDBState s = this.rocksDBState; + if (s == null) { + throw new IllegalStateException( + "Cannot create incremental snapshot for lazy tablet " + + tableBucket + + " while closed."); + } return new RocksIncrementalSnapshot( uploadedSstFiles, - rocksDBKv.getDb(), - rocksDBKv.getResourceGuard(), + s.rocksDBKv.getDb(), + s.rocksDBKv.getResourceGuard(), kvSnapshotDataUploader, - kvTabletDir, + s.kvTabletDir, lastCompletedSnapshotId); } // only for testing. @VisibleForTesting KvPreWriteBuffer getKvPreWriteBuffer() { - return kvPreWriteBuffer; + RocksDBState s = this.rocksDBState; + return s != null ? s.kvPreWriteBuffer : null; } // only for testing. @VisibleForTesting public RocksDBKv getRocksDBKv() { - return rocksDBKv; + RocksDBState s = this.rocksDBState; + return s != null ? s.rocksDBKv : null; + } + + // ======================================================================== + // Lazy lifecycle delegation + // ======================================================================== + + /** Returns true if this tablet is in lazy mode (as opposed to eager/traditional mode). */ + public boolean isLazyMode() { + return lifecycle != null; + } + + /** Returns true if this tablet is in lazy mode and currently OPEN (RocksDB loaded). */ + public boolean isLazyOpen() { + return lifecycle != null && lifecycle.isOpen(); + } + + /** Returns the current lazy state. */ + public KvTabletLazyLifecycle.LazyState getLazyState() { + return lifecycle != null ? lifecycle.getLazyState() : null; + } + + /** + * Acquire a guard that prevents RocksDB from being released while held. In eager mode, returns + * a no-op guard immediately. In lazy mode, ensures RocksDB is open (blocking if necessary) and + * pins it. + * + *

Must be called OUTSIDE any Replica-level locks (e.g. leaderIsrUpdateLock) to avoid + * blocking leader transitions during slow opens. + */ + public Guard acquireGuard() { + return lifecycle != null ? lifecycle.acquireGuard() : Guard.NOOP; + } + + /** + * Try to acquire a guard only if RocksDB is already open. + * + *

Returns {@code null} for a lazy tablet that is currently LAZY/FAILED/RELEASING/CLOSED. + */ + @Nullable + Guard tryAcquireExistingGuard() { + return lifecycle != null ? lifecycle.tryAcquireExistingGuard() : Guard.NOOP; + } + + /** Release a pin. Called by Guard.close(). */ + void releasePin() { + if (lifecycle != null) { + lifecycle.releasePin(); + } + } + + /** Pre-check for idle release eligibility. */ + public boolean canRelease(long closeIdleIntervalMs, long nowMs) { + return lifecycle != null && lifecycle.canRelease(closeIdleIntervalMs, nowMs); + } + + public boolean releaseKv() { + return lifecycle != null && lifecycle.releaseKv(); + } + + public void dropKvLazy() { + if (lifecycle != null) { + lifecycle.dropKvLazy(); + } + } + + // ---- Lazy state queries ---- + + public long getCachedRowCount() { + return lifecycle != null ? lifecycle.getCachedRowCount() : 0; + } + + public long getCachedFlushedLogOffset() { + return lifecycle != null ? lifecycle.getCachedFlushedLogOffset() : 0; + } + + public long getLastAccessTimestamp() { + return lifecycle != null ? lifecycle.getLastAccessTimestamp() : 0; + } + + /** + * Returns the number of active pins preventing RocksDB release. + * + *

This is primarily for testing and monitoring purposes. Exposing internal concurrency state + * may lead to misuse in production code. + */ + @VisibleForTesting + public int getActivePins() { + return lifecycle != null ? lifecycle.getActivePins() : 0; + } + + // ---- Test helpers ---- + + @VisibleForTesting + void setLazyStateForTesting(KvTabletLazyLifecycle.LazyState state) { + if (lifecycle != null) { + lifecycle.setLazyStateForTesting(state); + } + } + + @VisibleForTesting + void setLastAccessTimestampForTesting(long timestamp) { + if (lifecycle != null) { + lifecycle.setLastAccessTimestampForTesting(timestamp); + } + } + + @VisibleForTesting + void setClockForTesting(Clock clock) { + if (lifecycle != null) { + lifecycle.setClockForTesting(clock); + } + } + + private void closeRocksDBState(RocksDBState state, String operation) { + try { + state.kvPreWriteBuffer.close(); + } catch (Exception e) { + LOG.warn( + "Failed to close kv pre-write buffer during {} for {}", + operation, + tableBucket, + e); + } + try { + state.arrowWriterProvider.close(); + } catch (Exception e) { + LOG.warn( + "Failed to close arrow writer pool during {} for {}", + operation, + tableBucket, + e); + } + try { + state.rocksDBKv.close(); + } catch (Exception e) { + LOG.warn("Failed to close RocksDB during {} for {}", operation, tableBucket, e); + } + } + + // ---- Immutable RocksDB state holder ---- + + /** + * Immutable holder for all RocksDB-related fields. All fields are {@code final}, so in EAGER + * mode the JIT can cache them in registers. In LAZY mode, a single volatile write of the holder + * reference publishes all fields atomically (instead of 15+ individual volatile writes). + */ + static final class RocksDBState { + final File kvTabletDir; + final RocksDBKv rocksDBKv; + final long writeBatchSize; + final KvPreWriteBuffer kvPreWriteBuffer; + final LogFormat logFormat; + final KvFormat kvFormat; + final ArrowWriterProvider arrowWriterProvider; + final MemorySegmentPool memorySegmentPool; + final RowMerger rowMerger; + final RowMerger overwriteRowMerger; + final ArrowCompressionInfo arrowCompressionInfo; + final SchemaGetter schemaGetter; + final ChangelogImage changelogImage; + final @Nullable RocksDBStatistics rocksDBStatistics; + final AutoIncrementManager autoIncrementManager; + + RocksDBState( + File kvTabletDir, + RocksDBKv rocksDBKv, + long writeBatchSize, + KvPreWriteBuffer kvPreWriteBuffer, + LogFormat logFormat, + KvFormat kvFormat, + ArrowWriterProvider arrowWriterProvider, + MemorySegmentPool memorySegmentPool, + RowMerger rowMerger, + RowMerger overwriteRowMerger, + ArrowCompressionInfo arrowCompressionInfo, + SchemaGetter schemaGetter, + ChangelogImage changelogImage, + @Nullable RocksDBStatistics rocksDBStatistics, + AutoIncrementManager autoIncrementManager) { + this.kvTabletDir = kvTabletDir; + this.rocksDBKv = rocksDBKv; + this.writeBatchSize = writeBatchSize; + this.kvPreWriteBuffer = kvPreWriteBuffer; + this.logFormat = logFormat; + this.kvFormat = kvFormat; + this.arrowWriterProvider = arrowWriterProvider; + this.memorySegmentPool = memorySegmentPool; + this.rowMerger = rowMerger; + this.overwriteRowMerger = overwriteRowMerger; + this.arrowCompressionInfo = arrowCompressionInfo; + this.schemaGetter = schemaGetter; + this.changelogImage = changelogImage; + this.rocksDBStatistics = rocksDBStatistics; + this.autoIncrementManager = autoIncrementManager; + } } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/KvTabletLazyLifecycle.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvTabletLazyLifecycle.java new file mode 100644 index 0000000000..bcc311bf61 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvTabletLazyLifecycle.java @@ -0,0 +1,1050 @@ +/* + * 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; + +import org.apache.fluss.annotation.VisibleForTesting; +import org.apache.fluss.exception.KvStorageException; +import org.apache.fluss.server.metrics.group.TabletServerMetricGroup; +import org.apache.fluss.utils.ExponentialBackoff; +import org.apache.fluss.utils.FileUtils; +import org.apache.fluss.utils.clock.Clock; +import org.apache.fluss.utils.clock.SystemClock; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import javax.annotation.concurrent.GuardedBy; + +import java.io.File; +import java.util.concurrent.CancellationException; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.ReentrantLock; +import java.util.function.IntSupplier; +import java.util.function.Supplier; + +/** + * Manages the lazy lifecycle state machine for a {@link KvTablet}. Encapsulates all state + * transitions (LAZY → OPENING → OPEN → RELEASING → LAZY), pin management, open/release/drop logic, + * and callback orchestration. + * + *

This class is only instantiated for lazy-mode tablets. EAGER-mode tablets do not use this + * class. + */ +public final class KvTabletLazyLifecycle { + private static final Logger LOG = LoggerFactory.getLogger(KvTabletLazyLifecycle.class); + + /** Coarsen access timestamp updates to reduce volatile writes on hot path. */ + private static final long ACCESS_TIMESTAMP_GRANULARITY_MS = 1000; + + // ---- Lazy lifecycle states ---- + + /** Internal RocksDB lazy lifecycle states. */ + enum LazyState { + /** Lazy mode: RocksDB not yet opened. */ + LAZY, + /** Lazy mode: RocksDB is being opened by another thread. */ + OPENING, + /** Lazy mode: RocksDB is open and serving requests. */ + OPEN, + /** Lazy mode: RocksDB is being released (closing without deleting data). */ + RELEASING, + /** Lazy mode: last open attempt failed, in backoff cooldown. */ + FAILED, + /** + * Terminal state: tablet is closed and will not be reopened. Local data may or may not have + * been deleted depending on whether close or drop was called. + */ + CLOSED + } + + // ---- Callback interfaces ---- + + /** + * Callback for the actual RocksDB open work. Provided by Replica since it requires access to + * snapshot context, log recovery, etc. + */ + @FunctionalInterface + public interface OpenCallback { + /** Performs the open and returns the opened RocksDB KvTablet instance. */ + KvTablet doOpen(boolean hasLocalData) throws Exception; + } + + /** + * Callback invoked after a successful open commit. Used by Replica to install the tablet + * reference and start periodic snapshot. + */ + @FunctionalInterface + public interface OpenCommitCallback { + /** Called after RocksDB state has been installed into the sentinel. */ + void onOpenCommitted(KvTablet kvTablet); + } + + /** Callback for releasing (closing) RocksDB without destroying local data. */ + @FunctionalInterface + public interface ReleaseCallback { + /** Performs release-time cleanup (e.g. stop snapshots, unregister metrics). */ + void doRelease(KvTablet kvTablet); + } + + /** Callback for dropping (destroying) a KvTablet and cleaning up associated resources. */ + @FunctionalInterface + public interface DropCallback { + /** Performs drop-time cleanup (e.g. unregister metrics, stop snapshots). */ + void doDrop(KvTablet kvTablet); + } + + // ---- The owning tablet ---- + + private final KvTablet tablet; + + // ---- State machine fields ---- + + private volatile LazyState lazyState = LazyState.LAZY; + + /** Lock guarding lazy state transitions. */ + private final ReentrantLock lazyStateLock = new ReentrantLock(); + + private final Condition lazyStateChanged = lazyStateLock.newCondition(); + + /** Active pin count — prevents release while operations are in-flight. */ + private final AtomicInteger activePins = new AtomicInteger(0); + + private volatile boolean rejectNewPins; + + /** Generation counter for fencing stale open results. */ + private long openGeneration; + + /** Semaphore for throttling concurrent open operations across all tablets. */ + private @Nullable Semaphore openSemaphore; + + private long openTimeoutMs; + private long releaseDrainTimeoutMs; + private @Nullable ExponentialBackoff failedBackoff; + private Clock clock = SystemClock.getInstance(); + + /** FAILED state tracking. */ + private long failedTimestamp; + + private int failureCount; + private @Nullable Throwable lastFailureCause; + + /** Cached values served when RocksDB is not open. */ + private volatile long cachedRowCount; + + private volatile long cachedFlushedLogOffset; + + /** Whether local RocksDB data directory exists from a previous open. */ + @GuardedBy("lazyStateLock") + private boolean hasLocalData; + + /** Access timestamp for idle release decisions (coarsened to reduce volatile writes). */ + private volatile long lastAccessTimestamp; + + /** Epoch suppliers for fencing stale open results. */ + private @Nullable IntSupplier leaderEpochSupplier; + + private @Nullable IntSupplier bucketEpochSupplier; + + /** Supplier for tablet directory path — used by sentinel in LAZY/FAILED state. */ + private @Nullable Supplier tabletDirSupplier; + + /** Callbacks (set by Replica after construction). */ + private @Nullable OpenCallback openCallback; + + private @Nullable OpenCommitCallback commitCallback; + private @Nullable DropCallback dropCallback; + private @Nullable ReleaseCallback releaseCallback; + + /** Tracks post-open finalization that runs after OPEN becomes externally visible. */ + private boolean commitCallbackInFlight; + + KvTabletLazyLifecycle(KvTablet tablet) { + this.tablet = tablet; + } + + // ---- Configuration (called by Replica after sentinel creation) ---- + + /** + * Configure lazy open parameters. Must be called before any {@link #acquireGuard()} calls. + * + * @param clock the clock for time-based operations + * @param openSemaphore semaphore to limit concurrent open operations (can be null) + * @param openTimeoutMs timeout for open operations in milliseconds + * @param failedBackoffBaseMs base backoff time for failed opens in milliseconds + * @param failedBackoffMaxMs max backoff time for failed opens in milliseconds + * @param releaseDrainTimeoutMs timeout for draining pins during release in milliseconds + */ + public void configureLazyOpen( + Clock clock, + Semaphore openSemaphore, + long openTimeoutMs, + long failedBackoffBaseMs, + long failedBackoffMaxMs, + long releaseDrainTimeoutMs) { + this.clock = clock; + this.openSemaphore = openSemaphore; + this.openTimeoutMs = openTimeoutMs; + this.failedBackoff = new ExponentialBackoff(failedBackoffBaseMs, 2, failedBackoffMaxMs, 0); + this.releaseDrainTimeoutMs = releaseDrainTimeoutMs; + // Register initial LAZY state in metrics + updateStateGauges(null, LazyState.LAZY); + } + + /** + * Sets the callback responsible for performing the actual RocksDB open operation. This is + * provided by Replica since it requires access to snapshot context, log recovery, etc. + * + * @param callback the open callback to invoke when transitioning from LAZY to OPEN + */ + public void setOpenCallback(OpenCallback callback) { + this.openCallback = callback; + } + + /** + * Sets the callback invoked after a successful open commit. Used by Replica to install the + * tablet reference and start periodic snapshot. + * + * @param callback the commit callback to invoke after RocksDB state is installed + */ + public void setCommitCallback(OpenCommitCallback callback) { + this.commitCallback = callback; + } + + /** + * Sets the callback for full-state cleanup when the tablet is dropped. Handles all states, + * waits for in-progress operations, then transitions to CLOSED. + * + * @param callback the drop callback to invoke during tablet deletion + */ + public void setDropCallback(DropCallback callback) { + this.dropCallback = callback; + } + + /** + * Sets the callback for releasing (closing) RocksDB without destroying local data. Called by + * the idle release controller when transitioning from OPEN to LAZY. + * + * @param callback the release callback to invoke during idle release + */ + public void setReleaseCallback(ReleaseCallback callback) { + this.releaseCallback = callback; + } + + /** + * Sets the supplier for the tablet directory path. Used by sentinel in LAZY/FAILED state for + * cleanup operations. + * + * @param supplier the directory supplier + */ + public void setTabletDirSupplier(Supplier supplier) { + this.tabletDirSupplier = supplier; + } + + /** Returns the tablet directory from the supplier, or null if not configured. */ + @Nullable + File getTabletDir() { + return tabletDirSupplier != null ? tabletDirSupplier.get() : null; + } + + /** + * Sets the supplier for the leader epoch used in fencing stale open operations. + * + * @param supplier the leader epoch supplier + */ + public void setLeaderEpochSupplier(IntSupplier supplier) { + this.leaderEpochSupplier = supplier; + } + + /** + * Sets the supplier for the bucket epoch used in fencing stale open operations. + * + * @param supplier the bucket epoch supplier + */ + public void setBucketEpochSupplier(IntSupplier supplier) { + this.bucketEpochSupplier = supplier; + } + + /** + * Initializes the cached row count from snapshot metadata. Called during sentinel creation to + * serve read queries while in LAZY state before the first open. Also sets the sentinel's + * rowCount field so that OPENING state reads return the correct value. + * + * @param rowCount the initial row count (-1 if unknown/disabled) + */ + public void initCachedRowCount(long rowCount) { + this.cachedRowCount = rowCount; + tablet.setRowCount(rowCount); + } + + // ---- State queries ---- + + /** Returns whether the tablet is currently in OPEN state with RocksDB loaded and serving. */ + boolean isOpen() { + return lazyState == LazyState.OPEN; + } + + /** Returns the current lifecycle state of this lazy tablet. */ + LazyState getLazyState() { + return lazyState; + } + + /** + * Returns the cached row count from the last flush or release operation. This value may be + * stale but is sufficient for serving read queries while in LAZY state. + */ + long getCachedRowCount() { + return cachedRowCount; + } + + /** + * Returns the flushed log offset cached from the last flush or release operation. Used to + * determine how much incremental log needs to be replayed when reopening from local data. + */ + long getCachedFlushedLogOffset() { + return cachedFlushedLogOffset; + } + + /** + * Returns the timestamp (in milliseconds) of the last access to this tablet. Used by the idle + * release controller to determine eligibility for release. + */ + long getLastAccessTimestamp() { + return lastAccessTimestamp; + } + + /** + * Returns the number of active pins that are currently preventing RocksDB from being released. + * Each pin corresponds to a {@link KvTablet.Guard} held by an in-flight operation. + */ + int getActivePins() { + return activePins.get(); + } + + /** + * Returns true if this tablet is in LAZY or FAILED state (RocksDB not open), meaning + * KvTablet.getRowCount() should return the cached value instead of querying RocksDB. + */ + boolean needsCachedRowCount() { + LazyState s = lazyState; + return s == LazyState.LAZY || s == LazyState.FAILED; + } + + // ---- Guard / Pin management ---- + + /** + * Acquire a guard that prevents RocksDB from being released while held. Ensures RocksDB is open + * (blocking if necessary) and pins it. + * + *

Must be called OUTSIDE any Replica-level locks (e.g. leaderIsrUpdateLock) to avoid + * blocking leader transitions during slow opens. + * + * @throws IllegalStateException if the lifecycle has not been fully configured via {@link + * #configureLazyOpen} and callback setters + */ + KvTablet.Guard acquireGuard() { + // Defensive check: ensure all required callbacks are set before allowing access + if (openCallback == null) { + throw new IllegalStateException( + "KvTabletLazyLifecycle not fully configured: openCallback is null. " + + "Ensure configureLazyOpen() and setOpenCallback() are called before use."); + } + + // Fast path: try to pin without blocking + KvTablet.Guard fast = tryAcquirePinAsGuard(); + if (fast != null) { + return fast; + } + + // Slow path: ensure open, then pin + ensureOpen(); + // Between ensureOpen() returning and pin, a concurrent release could happen. + // Retry up to 3 times. + for (int attempt = 0; ; attempt++) { + lazyStateLock.lock(); + try { + if (lazyState == LazyState.OPEN && !rejectNewPins) { + activePins.incrementAndGet(); + touchAccessTimestamp(); + return new KvTablet.Guard(tablet); + } + if (lazyState == LazyState.CLOSED) { + throw new KvStorageException( + "KvTablet is closed for " + tablet.getTableBucket()); + } + } finally { + lazyStateLock.unlock(); + } + if (attempt >= 2) { + throw new KvStorageException( + "Failed to pin KvTablet after open for " + tablet.getTableBucket()); + } + // Re-open and retry + ensureOpen(); + } + } + + /** + * Try to acquire a guard only if RocksDB is already OPEN. + * + *

Unlike {@link #acquireGuard()}, this method never triggers a lazy open. It is intended for + * background maintenance paths (flush, snapshot init) that must coordinate with idle release + * but should not reopen a lazy tablet on their own. + */ + @Nullable + KvTablet.Guard tryAcquireExistingGuard() { + return tryAcquirePinAsGuard(); + } + + /** + * Fast-path pin attempt. Returns Guard if OPEN and accepting pins; null otherwise. + * + *

Safety relies on the double-check pattern: we optimistically increment the pin count, then + * re-verify the state. If the state changed concurrently (e.g., release started), the re-check + * fails and we undo the increment. This is safe on all architectures (x86, ARM) regardless of + * volatile read ordering between independent variables. + */ + @Nullable + private KvTablet.Guard tryAcquirePinAsGuard() { + if (lazyState == LazyState.OPEN && !rejectNewPins) { + activePins.incrementAndGet(); + if (lazyState == LazyState.OPEN && !rejectNewPins) { + touchAccessTimestamp(); + return new KvTablet.Guard(tablet); + } + decrementAndMaybeSignal(); + } + return null; + } + + /** Release a pin. Called by Guard.close(). */ + void releasePin() { + decrementAndMaybeSignal(); + } + + private void decrementAndMaybeSignal() { + if (activePins.decrementAndGet() == 0) { + if (rejectNewPins) { + lazyStateLock.lock(); + try { + lazyStateChanged.signalAll(); + } finally { + lazyStateLock.unlock(); + } + } + } + } + + private void touchAccessTimestamp() { + long now = clock.milliseconds(); + if (now - lastAccessTimestamp > ACCESS_TIMESTAMP_GRANULARITY_MS) { + lastAccessTimestamp = now; + } + } + + /** + * Update state gauge metrics on state transitions. Only LAZY, OPEN, FAILED are tracked as + * "stable" states; OPENING, RELEASING, CLOSED fall into default (no-op). + */ + private void updateStateGauges(@Nullable LazyState from, LazyState to) { + TabletServerMetricGroup metricGroup = tablet.serverMetricGroup; + if (metricGroup == null) { + return; + } + if (from != null) { + switch (from) { + case LAZY: + metricGroup.kvTabletLazyCount().decrementAndGet(); + break; + case OPEN: + metricGroup.kvTabletOpenCount().decrementAndGet(); + break; + case FAILED: + metricGroup.kvTabletFailedCount().decrementAndGet(); + break; + default: + break; + } + } + switch (to) { + case LAZY: + metricGroup.kvTabletLazyCount().incrementAndGet(); + break; + case OPEN: + metricGroup.kvTabletOpenCount().incrementAndGet(); + break; + case FAILED: + metricGroup.kvTabletFailedCount().incrementAndGet(); + break; + default: + break; + } + } + + // ---- Open management ---- + + /** + * Ensures RocksDB is open. If already OPEN, returns immediately. If LAZY or FAILED (past + * cooldown), triggers a slow-path open. If OPENING or RELEASING, blocks until state changes. + */ + private void ensureOpen() { + lazyStateLock.lock(); + try { + while (true) { + switch (lazyState) { + case OPEN: + return; + + case OPENING: + if (!lazyStateChanged.await(openTimeoutMs, TimeUnit.MILLISECONDS)) { + throw new KvStorageException( + "KvTablet open timed out for " + tablet.getTableBucket()); + } + continue; + + case RELEASING: + if (!lazyStateChanged.await(openTimeoutMs, TimeUnit.MILLISECONDS)) { + throw new KvStorageException( + "KvTablet open timed out waiting for release: " + + tablet.getTableBucket()); + } + continue; + + case FAILED: + long elapsed = clock.milliseconds() - failedTimestamp; + long cooldown = failedBackoff.backoff(failureCount); + if (elapsed < cooldown) { + throw new KvStorageException( + "KvTablet open failed for " + + tablet.getTableBucket() + + ", remaining cooldown: " + + (cooldown - elapsed) + + " ms", + lastFailureCause); + } + // Fall through to LAZY — cooldown expired, retry open. + + case LAZY: + updateStateGauges(lazyState, LazyState.OPENING); + lazyState = LazyState.OPENING; + openGeneration++; + long myGeneration = openGeneration; + int myLeaderEpoch = + leaderEpochSupplier != null ? leaderEpochSupplier.getAsInt() : -1; + int myBucketEpoch = + bucketEpochSupplier != null ? bucketEpochSupplier.getAsInt() : -1; + boolean localDataExists = hasLocalData; + lazyStateLock.unlock(); + try { + doSlowOpen(myGeneration, myLeaderEpoch, myBucketEpoch, localDataExists); + } catch (Throwable t) { + // doSlowOpen already called commitOpenResult() which transitions + // to FAILED, so no need to handle state transition here. + if (t instanceof RuntimeException) { + throw (RuntimeException) t; + } + throw new KvStorageException( + "KvTablet open failed for " + tablet.getTableBucket(), t); + } + return; + + case CLOSED: + throw new KvStorageException( + "KvTablet is closed for " + tablet.getTableBucket()); + + default: + throw new IllegalStateException("Unexpected state: " + lazyState); + } + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new KvStorageException( + "Interrupted waiting for KvTablet open: " + tablet.getTableBucket(), e); + } finally { + if (lazyStateLock.isHeldByCurrentThread()) { + lazyStateLock.unlock(); + } + } + } + + private void doSlowOpen( + long myGeneration, int myLeaderEpoch, int myBucketEpoch, boolean localDataExists) + throws Exception { + KvTablet newTablet = null; + boolean committed = false; + boolean semaphoreAcquired = false; + + try { + if (openSemaphore != null) { + semaphoreAcquired = openSemaphore.tryAcquire(openTimeoutMs, TimeUnit.MILLISECONDS); + if (!semaphoreAcquired) { + throw new KvStorageException( + "KvTablet open semaphore timeout for " + tablet.getTableBucket()); + } + } + + newTablet = openCallback.doOpen(localDataExists); + committed = + commitOpenResult(myGeneration, myLeaderEpoch, myBucketEpoch, newTablet, null); + if (!committed) { + throw new CancellationException( + "open result fenced by concurrent epoch/generation change"); + } + } catch (Exception e) { + commitOpenResult(myGeneration, myLeaderEpoch, myBucketEpoch, null, e); + throw e; + } finally { + if (!committed && newTablet != null) { + try { + newTablet.close(); + } catch (Exception ignored) { + } + // Clean up the directory created by the failed open attempt BEFORE + // releasing the semaphore, so that a subsequent open cannot start + // writing to the same canonical path while deletion is in progress. + File failedDir = newTablet.getKvTabletDir(); + if (failedDir != null) { + FileUtils.deleteDirectoryQuietly(failedDir); + } + } + if (semaphoreAcquired) { + openSemaphore.release(); + } + } + } + + private boolean commitOpenResult( + long myGeneration, + int myLeaderEpoch, + int myBucketEpoch, + @Nullable KvTablet newTablet, + @Nullable Exception error) { + lazyStateLock.lock(); + try { + if (openGeneration != myGeneration) { + if (lazyState == LazyState.OPENING) { + transitionToFailed( + new CancellationException( + "open superseded by generation " + openGeneration), + true); + } + return false; + } + + if (error != null) { + transitionToFailed(error, false); + return false; + } + + // Epoch fencing + int currentLeaderEpoch = + leaderEpochSupplier != null ? leaderEpochSupplier.getAsInt() : -1; + int currentBucketEpoch = + bucketEpochSupplier != null ? bucketEpochSupplier.getAsInt() : -1; + if (currentLeaderEpoch != myLeaderEpoch || currentBucketEpoch != myBucketEpoch) { + transitionToFailed(new CancellationException("epoch changed during open"), true); + return false; + } + + // Success — absorb RocksDB state from the newly opened tablet. + // Note: we set OPEN and signal waiters before running commitCallback (which starts + // the snapshot manager). Writes during this brief window are safe — data is in + // RocksDB and the WAL. The commitCallbackInFlight flag prevents release until the + // callback completes. + tablet.installRocksDB(newTablet); + lazyState = LazyState.OPEN; + failureCount = 0; + lastAccessTimestamp = clock.milliseconds(); + rejectNewPins = false; + commitCallbackInFlight = true; + lazyStateChanged.signalAll(); + } finally { + lazyStateLock.unlock(); + } + + try { + updateStateGauges(LazyState.OPENING, LazyState.OPEN); + + // Invoke commit callback outside the lock to avoid blocking acquireGuard() callers. + if (commitCallback != null) { + commitCallback.onOpenCommitted(tablet); + } + } catch (Exception e) { + LOG.warn( + "Post-open commit callback failed for {}, keeping tablet OPEN", + tablet.getTableBucket(), + e); + } finally { + lazyStateLock.lock(); + try { + commitCallbackInFlight = false; + lazyStateChanged.signalAll(); + } finally { + lazyStateLock.unlock(); + } + } + return true; + } + + /** Must be called while holding {@code lazyStateLock}. */ + private void transitionToFailed(Throwable cause, boolean resetCount) { + updateStateGauges(LazyState.OPENING, LazyState.FAILED); + lazyState = LazyState.FAILED; + failedTimestamp = clock.milliseconds(); + if (resetCount) { + failureCount = 0; + } else { + failureCount++; + } + lastFailureCause = cause; + lazyStateChanged.signalAll(); + } + + // ---- Release logic (idle release by KvManager) ---- + + /** Pre-check for idle release eligibility. */ + boolean canRelease(long closeIdleIntervalMs, long nowMs) { + if (lazyState != LazyState.OPEN) { + return false; + } + if (activePins.get() > 0) { + return false; + } + return nowMs - lastAccessTimestamp >= closeIdleIntervalMs; + } + + /** + * Release RocksDB resources: OPEN -> RELEASING -> LAZY. Caches metadata before close for + * serving queries while in LAZY state. + * + * @return true if release succeeded, false if aborted + */ + boolean releaseKv() { + // Pre-check state outside lock as a fast-path rejection + if (lazyState != LazyState.OPEN) { + return false; + } + + lazyStateLock.lock(); + try { + if (lazyState != LazyState.OPEN) { + return false; + } + + // Transition to RELEASING — write rejectNewPins before lazyState so that + // tryAcquirePinAsGuard() sees the reject flag if it reads OPEN. + rejectNewPins = true; + lazyState = LazyState.RELEASING; + + if (!drainPins()) { + LOG.warn("{} release drain timeout, aborting release", tablet.getTableBucket()); + transitionBackToOpen(); + return false; + } + + // Re-check flush progress only after blocking new pins and draining + // all in-flight operations. This closes the race where a writer acquired + // a guard before RELEASING but appended after an earlier LEO check. + long logEndOffset = tablet.logTablet.localLogEndOffset(); + if (tablet.flushedLogOffset < logEndOffset) { + transitionBackToOpen(); + return false; + } + + if (!awaitCommitCallbackCompletion()) { + LOG.warn( + "{} release waited too long for open commit finalization, aborting release", + tablet.getTableBucket()); + transitionBackToOpen(); + return false; + } + + // Cache metadata AFTER drain — all pinned operations have completed, + // so rowCount reflects the latest flushed state. + cachedRowCount = tablet.rowCount; + cachedFlushedLogOffset = tablet.flushedLogOffset; + } finally { + lazyStateLock.unlock(); + } + + // All pins drained, safe to close + boolean releaseSucceeded = false; + try { + if (releaseCallback != null) { + releaseCallback.doRelease(tablet); + } + tablet.detachRocksDB(); + releaseSucceeded = true; + } catch (Exception e) { + LOG.warn( + "{} release callback failed, rolling back to OPEN", tablet.getTableBucket(), e); + } finally { + lazyStateLock.lock(); + try { + if (releaseSucceeded) { + lazyState = LazyState.LAZY; + hasLocalData = true; + updateStateGauges(LazyState.OPEN, LazyState.LAZY); + rejectNewPins = false; + lazyStateChanged.signalAll(); + } else { + transitionBackToOpen(); + } + } finally { + lazyStateLock.unlock(); + } + } + return releaseSucceeded; + } + + // ---- Drop logic ---- + + /** + * Full-state cleanup for lazy mode: handles all states, waits for in-progress operations, then + * transitions to CLOSED. Called by Replica.dropKv(). + */ + void dropKvLazy() { + lazyStateLock.lock(); + try { + switch (lazyState) { + case CLOSED: + break; + + case LAZY: + case FAILED: + tablet.deleteLocalDirectory(); + break; + + case OPENING: + openGeneration++; + awaitStateExit(LazyState.OPENING, openTimeoutMs * 2); + if (lazyState == LazyState.OPEN) { + rejectNewPins = true; + awaitCommitCallbackCompletion(); + drainPinsForDrop(); + dropOpenTablet(); + } else if (lazyState != LazyState.OPENING) { + tablet.deleteLocalDirectory(); + } + break; + + case OPEN: + rejectNewPins = true; + awaitCommitCallbackCompletion(); + drainPinsForDrop(); + dropOpenTablet(); + break; + + case RELEASING: + awaitStateExit(LazyState.RELEASING, releaseDrainTimeoutMs * 2); + if (lazyState == LazyState.OPEN) { + rejectNewPins = true; + awaitCommitCallbackCompletion(); + drainPinsForDrop(); + dropOpenTablet(); + } else if (lazyState != LazyState.RELEASING) { + tablet.deleteLocalDirectory(); + } + break; + + default: + throw new IllegalStateException("Unexpected state in dropKvLazy: " + lazyState); + } + + LazyState stateBeforeClose = lazyState; + if (stateBeforeClose != LazyState.CLOSED) { + updateStateGauges(stateBeforeClose, LazyState.CLOSED); + } + lazyState = LazyState.CLOSED; + hasLocalData = false; + rejectNewPins = false; + failureCount = 0; + failedTimestamp = 0; + lastFailureCause = null; + lazyStateChanged.signalAll(); + } finally { + lazyStateLock.unlock(); + } + } + + private void drainPinsForDrop() { + if (!drainPins()) { + LOG.warn( + "{} drop drain timeout with {} active pins, proceeding with close", + tablet.getTableBucket(), + activePins.get()); + } + } + + private boolean drainPins() { + long deadline = clock.milliseconds() + releaseDrainTimeoutMs; + while (activePins.get() > 0) { + long remaining = deadline - clock.milliseconds(); + if (remaining <= 0) { + return false; + } + try { + lazyStateChanged.await(remaining, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + return false; + } + } + return true; + } + + /** Waits under {@code lazyStateLock} for post-open finalization to complete. */ + private boolean awaitCommitCallbackCompletion() { + long deadline = clock.milliseconds() + releaseDrainTimeoutMs; + while (commitCallbackInFlight) { + long remaining = deadline - clock.milliseconds(); + if (remaining <= 0) { + return false; + } + try { + if (!lazyStateChanged.await(remaining, TimeUnit.MILLISECONDS)) { + return false; + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + return false; + } + } + return true; + } + + /** + * Wait (under lazyStateLock) until {@code lazyState} is no longer {@code state}, or timeout. + */ + private void awaitStateExit(LazyState state, long timeoutMs) { + long deadlineNanos = clock.nanoseconds() + TimeUnit.MILLISECONDS.toNanos(timeoutMs); + while (lazyState == state) { + try { + long remainNanos = deadlineNanos - clock.nanoseconds(); + if (remainNanos <= 0 + || !lazyStateChanged.await(remainNanos, TimeUnit.NANOSECONDS)) { + break; + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + break; + } + } + } + + private void doDropViaCallback() { + if (dropCallback != null) { + dropCallback.doDrop(tablet); + } + } + + void closeKvLazy() { + lazyStateLock.lock(); + try { + switch (lazyState) { + case CLOSED: + return; + case LAZY: + case FAILED: + break; + case OPENING: + openGeneration++; + awaitStateExit(LazyState.OPENING, openTimeoutMs * 2); + if (lazyState == LazyState.OPEN) { + rejectNewPins = true; + awaitCommitCallbackCompletion(); + drainPins(); + tablet.detachRocksDB(); + } + break; + case OPEN: + rejectNewPins = true; + awaitCommitCallbackCompletion(); + drainPins(); + tablet.detachRocksDB(); + break; + case RELEASING: + awaitStateExit(LazyState.RELEASING, releaseDrainTimeoutMs * 2); + if (lazyState == LazyState.OPEN) { + rejectNewPins = true; + awaitCommitCallbackCompletion(); + drainPins(); + tablet.detachRocksDB(); + } + break; + default: + throw new IllegalStateException( + "Unexpected state in closeKvLazy: " + lazyState); + } + + LazyState stateBeforeClose = lazyState; + if (stateBeforeClose != LazyState.CLOSED) { + updateStateGauges(stateBeforeClose, LazyState.CLOSED); + } + lazyState = LazyState.CLOSED; + hasLocalData = false; + rejectNewPins = false; + failureCount = 0; + failedTimestamp = 0; + lastFailureCause = null; + lazyStateChanged.signalAll(); + } finally { + lazyStateLock.unlock(); + } + } + + /** Must be called while holding {@code lazyStateLock} and after pins have drained. */ + private void dropOpenTablet() { + doDropViaCallback(); + tablet.detachRocksDB(); + tablet.deleteLocalDirectory(); + } + + /** Must be called while holding {@code lazyStateLock}. */ + private void transitionBackToOpen() { + lazyState = LazyState.OPEN; + rejectNewPins = false; + lazyStateChanged.signalAll(); + } + + // ---- Test helpers ---- + + @VisibleForTesting + void setLazyStateForTesting(LazyState state) { + lazyStateLock.lock(); + try { + this.lazyState = state; + lazyStateChanged.signalAll(); + } finally { + lazyStateLock.unlock(); + } + } + + @VisibleForTesting + void setLastAccessTimestampForTesting(long timestamp) { + this.lastAccessTimestamp = timestamp; + } + + @VisibleForTesting + void setClockForTesting(Clock clock) { + this.clock = clock; + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/metrics/group/TabletServerMetricGroup.java b/fluss-server/src/main/java/org/apache/fluss/server/metrics/group/TabletServerMetricGroup.java index f4b2c6b730..adbdfc2ca0 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/metrics/group/TabletServerMetricGroup.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/metrics/group/TabletServerMetricGroup.java @@ -34,6 +34,7 @@ import org.apache.fluss.utils.MapUtils; import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; /** The metric group for tablet server. */ public class TabletServerMetricGroup extends AbstractMetricGroup { @@ -70,6 +71,9 @@ public class TabletServerMetricGroup extends AbstractMetricGroup { private final Histogram kvFlushLatencyHistogram; private final Counter kvTruncateAsDuplicatedCount; private final Counter kvTruncateAsErrorCount; + private final AtomicInteger kvTabletOpenCount = new AtomicInteger(0); + private final AtomicInteger kvTabletLazyCount = new AtomicInteger(0); + private final AtomicInteger kvTabletFailedCount = new AtomicInteger(0); // aggregated replica metrics private final Counter isrShrinks; @@ -126,6 +130,9 @@ public TabletServerMetricGroup( meter( MetricNames.KV_PRE_WRITE_BUFFER_TRUNCATE_AS_ERROR_RATE, new MeterView(kvTruncateAsErrorCount)); + gauge(MetricNames.KV_TABLET_OPEN_COUNT, kvTabletOpenCount::get); + gauge(MetricNames.KV_TABLET_LAZY_COUNT, kvTabletLazyCount::get); + gauge(MetricNames.KV_TABLET_FAILED_COUNT, kvTabletFailedCount::get); // replica metrics isrExpands = new SimpleCounter(); @@ -228,6 +235,21 @@ public Counter kvTruncateAsErrorCount() { return kvTruncateAsErrorCount; } + /** Returns the gauge-backing counter for currently open lazy KvTablets. */ + public AtomicInteger kvTabletOpenCount() { + return kvTabletOpenCount; + } + + /** Returns the gauge-backing counter for KvTablets in LAZY (released) state. */ + public AtomicInteger kvTabletLazyCount() { + return kvTabletLazyCount; + } + + /** Returns the gauge-backing counter for KvTablets in FAILED state. */ + public AtomicInteger kvTabletFailedCount() { + return kvTabletFailedCount; + } + public Counter isrShrinks() { return isrShrinks; } 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 dd83c58b1d..369fa084c1 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 @@ -53,6 +53,7 @@ import org.apache.fluss.server.kv.KvManager; import org.apache.fluss.server.kv.KvRecoverHelper; import org.apache.fluss.server.kv.KvTablet; +import org.apache.fluss.server.kv.KvTabletLazyLifecycle; import org.apache.fluss.server.kv.autoinc.AutoIncIDRange; import org.apache.fluss.server.kv.rocksdb.RocksDBKvBuilder; import org.apache.fluss.server.kv.snapshot.CompletedKvSnapshotCommitter; @@ -91,6 +92,7 @@ import org.apache.fluss.server.zk.data.LeaderAndIsr; import org.apache.fluss.server.zk.data.ZkData; import org.apache.fluss.utils.CloseableRegistry; +import org.apache.fluss.utils.FileUtils; import org.apache.fluss.utils.FlussPaths; import org.apache.fluss.utils.IOUtils; import org.apache.fluss.utils.MapUtils; @@ -144,6 +146,12 @@ @ThreadSafe public final class Replica { + /** Operation executed under a {@link KvTablet.Guard} in {@link #withGuardedLeaderKv}. */ + @FunctionalInterface + private interface GuardedKvOperation { + T apply(KvTablet kvTablet) throws Exception; + } + private static final Logger LOG = LoggerFactory.getLogger(Replica.class); private final PhysicalTablePath physicalPath; @@ -198,7 +206,7 @@ public final class Replica { // null if table without pk or haven't become leader private volatile @Nullable KvTablet kvTablet; private volatile @Nullable CloseableRegistry closeableRegistryForKv; - private @Nullable PeriodicSnapshotManager kvSnapshotManager; + private volatile @Nullable PeriodicSnapshotManager kvSnapshotManager; // ------- metrics private Counter isrShrinks; @@ -256,9 +264,19 @@ public Replica( this.logTablet = createLog(lazyHighWatermarkCheckpoint); this.logTablet.updateIsDataLakeEnabled(tableConfig.isDataLakeEnabled()); this.clock = clock; + registerMetrics(); } + /** Stop the periodic snapshot manager (does NOT clear kvTablet — sentinel stays). */ + private void stopSnapshotManager() { + PeriodicSnapshotManager mgr = this.kvSnapshotManager; + if (mgr != null) { + this.kvSnapshotManager = null; + mgr.close(); + } + } + private void registerMetrics() { // get root metric in the reference: bucket -> table -> tabletServer TabletServerMetricGroup serverMetrics = @@ -285,12 +303,21 @@ public long logicalStorageLogSize() { public long logicalStorageKvSize() { if (isLeader() && isKvTable()) { - checkNotNull(kvSnapshotManager, "kvSnapshotManager is null"); - return kvSnapshotManager.getSnapshotSize(); - } else { - // follower doesn't need to report the logical storage size. + PeriodicSnapshotManager mgr = this.kvSnapshotManager; + if (mgr != null) { + return mgr.getSnapshotSize(); + } + // In lazy mode, snapshot manager may be null when tablet is released. + // Fall back to committed snapshot size. + KvTablet kv = this.kvTablet; + if (kv != null && kv.isLazyMode()) { + return getLatestKvSnapshotSize(); + } + // Eager mode: snapshot manager not yet initialized (during startup). + // Return 0 to avoid blocking metrics thread with ZK calls. return 0L; } + return 0L; } public boolean isKvTable() { @@ -530,11 +557,9 @@ private void onBecomeNewLeader() { } if (isKvTable()) { - // if it's become new leader, we must - // first destroy the old kv tablet - // if exist. Otherwise, it'll use still the old kv tablet which will cause data loss + // Destroy the old kv tablet if it exists — reusing a stale tablet from a previous + // leader epoch would cause data loss. dropKv(); - // now, we can create a new kv tablet createKv(); } } @@ -621,7 +646,7 @@ private void createKv() { try { // create a closeable registry for the closable related to kv closeableRegistryForKv = new CloseableRegistry(); - // resister the closeable registry for kv + // register the closeable registry for kv closeableRegistry.registerCloseable(closeableRegistryForKv); } catch (IOException e) { LOG.warn( @@ -630,11 +655,117 @@ private void createKv() { e); } + checkNotNull(kvManager); + + if (kvManager.isLazyOpenEnabled()) { + createKvLazy(); + } else { + createKvEager(); + } + } + + private void createKvLazy() { + TabletServerMetricGroup serverMetrics = + bucketMetricGroup.getTableMetricGroup().getServerMetricGroup(); + + // Create a lazy sentinel — no RocksDB yet + KvTablet sentinel = + KvTablet.createLazySentinel(physicalPath, tableBucket, logTablet, serverMetrics); + + // Configure lazy open parameters via lifecycle + KvTabletLazyLifecycle lifecycle = sentinel.getLifecycle(); + lifecycle.configureLazyOpen( + clock, + kvManager.getOpenSemaphore(), + kvManager.getOpenTimeoutMs(), + kvManager.getFailedBackoffBaseMs(), + kvManager.getFailedBackoffMaxMs(), + KvManager.RELEASE_DRAIN_TIMEOUT_MS); + lifecycle.setLeaderEpochSupplier(() -> leaderEpoch); + lifecycle.setBucketEpochSupplier(() -> bucketEpoch); + + // Tablet directory supplier for LAZY/FAILED state cleanup + lifecycle.setTabletDirSupplier(() -> kvManager.getTabletDirPath(physicalPath, tableBucket)); + + // Shared state between open and commit callbacks to pass snapshot reference + AtomicReference snapshotRef = new AtomicReference<>(); + + // Open callback: creates a real KvTablet + // If local data exists (reopen after idle release), load from local directory + // and replay only incremental logs. Otherwise, do full init (download snapshot). + lifecycle.setOpenCallback( + hasLocal -> { + if (hasLocal) { + return reopenFromLocalData(sentinel, snapshotRef); + } else { + InitKvResult result = initKvTablet(); + snapshotRef.set(result.snapshotUsed.orElse(null)); + return result.tablet; + } + }); + + // Commit callback: start periodic snapshot after RocksDB is installed + lifecycle.setCommitCallback( + kv -> { + startPeriodicKvSnapshot(kv, snapshotRef.getAndSet(null)); + if (kv.getRocksDBStatistics() != null) { + bucketMetricGroup.registerRocksDBStatistics(kv.getRocksDBStatistics()); + } + }); + + // Release callback: stop snapshot and unregister metrics + // RocksDB close is handled by detachRocksDB() inside KvTablet + lifecycle.setReleaseCallback( + kv -> { + try { + stopSnapshotManager(); + } catch (Exception e) { + LOG.warn( + "Failed to stop kv snapshot manager during release for {}", + tableBucket, + e); + } + try { + bucketMetricGroup.unregisterRocksDBStatistics(); + } catch (Exception e) { + LOG.warn( + "Failed to unregister RocksDB statistics during release for {}", + tableBucket, + e); + } + }); + + // Drop callback: cleanup metrics and snapshot + // Registry removal is handled by Replica.dropKv() + lifecycle.setDropCallback( + kv -> { + bucketMetricGroup.unregisterRocksDBStatistics(); + stopSnapshotManager(); + }); + + // Initialize cached row count from snapshot metadata if available (no RocksDB needed) + long initialRowCount; + if (tableConfig.getChangelogImage() == ChangelogImage.WAL) { + initialRowCount = -1L; + } else { + Long snapshotRowCount = + getLatestSnapshot(tableBucket).map(CompletedSnapshot::getRowCount).orElse(null); + initialRowCount = snapshotRowCount != null ? snapshotRowCount : 0L; + } + lifecycle.initCachedRowCount(initialRowCount); + + // Register sentinel in KvManager registry + kvManager.registerKv(tableBucket, sentinel); + + this.kvTablet = sentinel; + } + + private void createKvEager() { // init kv tablet and get the snapshot it uses to init if have any Optional snapshotUsed = Optional.empty(); for (int i = 1; i <= INIT_KV_TABLET_MAX_RETRY_TIMES; i++) { try { - snapshotUsed = initKvTablet(); + snapshotUsed = initKvTabletEager(); break; } catch (Exception e) { LOG.warn( @@ -644,8 +775,92 @@ private void createKv() { e); } } - // start periodic kv snapshot - startPeriodicKvSnapshot(snapshotUsed.orElse(null)); + // start periodic kv snapshot (kvTablet may be null if all retries failed) + if (kvTablet != null) { + startPeriodicKvSnapshot(kvTablet, snapshotUsed.orElse(null)); + } else { + LOG.error( + "Failed to init kv tablet for bucket {} after {} retries. " + + "KvTablet will remain null.", + tableBucket, + INIT_KV_TABLET_MAX_RETRY_TIMES); + } + } + + /** + * Init kv tablet for the eager (non-lazy) path. This method directly sets {@code this.kvTablet} + * inside, matching the original behavior where the tablet is visible even if recovery partially + * fails. Uses registered KvManager methods (loadKv/getOrCreateKv). + * + * @return the snapshot used to init kv tablet, empty if no any snapshot. + */ + private Optional initKvTabletEager() { + checkNotNull(kvManager); + long startTime = clock.milliseconds(); + LOG.info("Start to init kv tablet for {} of table {}.", tableBucket, physicalPath); + + long restoreStartOffset = 0; + Optional optCompletedSnapshot = getLatestSnapshot(tableBucket); + try { + Long rowCount; + AutoIncIDRange autoIncIDRange; + if (optCompletedSnapshot.isPresent()) { + LOG.info( + "Use snapshot {} to restore kv tablet for {} of table {}.", + optCompletedSnapshot.get(), + tableBucket, + physicalPath); + CompletedSnapshot completedSnapshot = optCompletedSnapshot.get(); + File tabletDir = kvManager.createTabletDir(physicalPath, tableBucket); + downloadKvSnapshots(completedSnapshot, tabletDir.toPath()); + + kvTablet = kvManager.loadKv(tabletDir, schemaGetter); + + checkNotNull(kvTablet, "kv tablet should not be null."); + restoreStartOffset = completedSnapshot.getLogOffset(); + rowCount = completedSnapshot.getRowCount(); + autoIncIDRange = completedSnapshot.getFirstAutoIncIDRange(); + } else { + LOG.info( + "No snapshot found for {} of {}, restore from log.", + tableBucket, + physicalPath); + + kvTablet = + kvManager.getOrCreateKv( + physicalPath, + tableBucket, + logTablet, + tableConfig.getKvFormat(), + schemaGetter, + tableConfig, + arrowCompressionInfo); + + rowCount = tableConfig.getChangelogImage() == ChangelogImage.WAL ? null : 0L; + autoIncIDRange = null; + } + + logTablet.updateMinRetainOffset(restoreStartOffset); + recoverKvTablet(kvTablet, restoreStartOffset, rowCount, autoIncIDRange); + } catch (Exception e) { + throw new KvStorageException( + String.format( + "Fail to init kv tablet for %s of table %s.", + tableBucket, physicalPath), + e); + } + long endTime = clock.milliseconds(); + LOG.info( + "Init kv tablet for {} of {} finish, cost {} ms.", + physicalPath, + tableBucket, + endTime - startTime); + + if (kvTablet != null && kvTablet.getRocksDBStatistics() != null) { + bucketMetricGroup.registerRocksDBStatistics(kvTablet.getRocksDBStatistics()); + } + + return optCompletedSnapshot; } private void dropKv() { @@ -653,15 +868,21 @@ private void dropKv() { 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; + KvTablet kv = this.kvTablet; + if (kv != null) { + if (kv.isLazyMode()) { + // Delegate to KvTablet's lazy lifecycle which handles all states + // (dropCallback inside dropKvLazy already calls stopSnapshotManager) + kv.dropKvLazy(); + // Remove sentinel from KvManager registry + checkNotNull(kvManager); + kvManager.unregisterKv(tableBucket); + } else { + bucketMetricGroup.unregisterRocksDBStatistics(); + checkNotNull(kvManager); + kvManager.dropKv(tableBucket); + } + this.kvTablet = null; } } @@ -673,11 +894,13 @@ private void mayFlushKv(long newHighWatermark) { } /** - * Init kv tablet from snapshot if any or just from log. + * Init kv tablet from snapshot if any or just from log. This is a pure factory method that + * creates and returns the KvTablet without installing it into {@code this.kvTablet}. The caller + * is responsible for installing the tablet at the appropriate time. * - * @return the snapshot used to init kv tablet, empty if no any snapshot. + * @return the created KvTablet and the snapshot used to init it (if any). */ - private Optional initKvTablet() { + private InitKvResult initKvTablet() { checkNotNull(kvManager); long startTime = clock.milliseconds(); LOG.info("Start to init kv tablet for {} of table {}.", tableBucket, physicalPath); @@ -699,6 +922,7 @@ private Optional initKvTablet() { // get the offset from which, we should restore from. default is 0 long restoreStartOffset = 0; Optional optCompletedSnapshot = getLatestSnapshot(tableBucket); + KvTablet createdTablet = null; try { Long rowCount; AutoIncIDRange autoIncIDRange; @@ -715,9 +939,9 @@ private Optional initKvTablet() { downloadKvSnapshots(completedSnapshot, tabletDir.toPath()); // as we have downloaded kv files into the tablet dir, now, we can load it - kvTablet = kvManager.loadKv(tabletDir, schemaGetter); + createdTablet = kvManager.loadKvUnregistered(tabletDir, schemaGetter); - checkNotNull(kvTablet, "kv tablet should not be null."); + checkNotNull(createdTablet, "kv tablet should not be null."); restoreStartOffset = completedSnapshot.getLogOffset(); rowCount = completedSnapshot.getRowCount(); // currently, we only support one auto-increment column. @@ -730,8 +954,8 @@ private Optional initKvTablet() { // actually, kv manager always create a kv tablet since we will drop the kv // if it exists before init kv tablet - kvTablet = - kvManager.getOrCreateKv( + createdTablet = + kvManager.createKvTabletUnregistered( physicalPath, tableBucket, logTablet, @@ -751,8 +975,17 @@ private Optional initKvTablet() { } logTablet.updateMinRetainOffset(restoreStartOffset); - recoverKvTablet(restoreStartOffset, rowCount, autoIncIDRange); + recoverKvTablet(createdTablet, restoreStartOffset, rowCount, autoIncIDRange); } catch (Exception e) { + // Close the partially-initialized tablet to release the RocksDB LOCK file. + // Without this, retries would fail with "lock hold by current process". + if (createdTablet != null) { + try { + createdTablet.close(); + } catch (Exception closeEx) { + e.addSuppressed(closeEx); + } + } throw new KvStorageException( String.format( "Fail to init kv tablet for %s of table %s.", @@ -766,12 +999,82 @@ private Optional initKvTablet() { tableBucket, endTime - startTime); - // Register RocksDB statistics to BucketMetricGroup - if (kvTablet != null && kvTablet.getRocksDBStatistics() != null) { - bucketMetricGroup.registerRocksDBStatistics(kvTablet.getRocksDBStatistics()); + return new InitKvResult(createdTablet, optCompletedSnapshot); + } + + /** + * Reopen a KvTablet from local data after idle release. Loads the existing RocksDB directory + * and replays only the incremental logs accumulated since the tablet was released. + */ + private KvTablet reopenFromLocalData( + KvTablet sentinel, AtomicReference snapshotRef) { + checkNotNull(kvManager); + long startTime = clock.milliseconds(); + LOG.info( + "Reopening kv tablet from local data for {} of table {}.", + tableBucket, + physicalPath); + + long cachedFlushedLogOffset = sentinel.getCachedFlushedLogOffset(); + long cachedRowCount = sentinel.getCachedRowCount(); + + KvTablet createdTablet = null; + try { + File tabletDir = kvManager.getTabletDirPath(physicalPath, tableBucket); + createdTablet = kvManager.loadKvUnregistered(tabletDir, schemaGetter); + + Long rowCount = cachedRowCount == KvTablet.ROW_COUNT_DISABLED ? null : cachedRowCount; + logTablet.updateMinRetainOffset(cachedFlushedLogOffset); + recoverKvTablet(createdTablet, cachedFlushedLogOffset, rowCount, null); + + long endTime = clock.milliseconds(); + LOG.info( + "Reopened kv tablet from local data for {} of table {} " + + "from log offset {}, cost {} ms.", + tableBucket, + physicalPath, + cachedFlushedLogOffset, + endTime - startTime); + + return createdTablet; + } catch (Exception e) { + LOG.warn( + "Failed to reopen from local data for {} of table {}, " + + "falling back to full init.", + tableBucket, + physicalPath, + e); + if (createdTablet != null) { + try { + createdTablet.close(); + } catch (Exception closeEx) { + e.addSuppressed(closeEx); + } + } + // Clean up the potentially corrupted local directory + File tabletDir = kvManager.getTabletDirPath(physicalPath, tableBucket); + FileUtils.deleteDirectoryQuietly(tabletDir); + if (tabletDir.exists()) { + LOG.error( + "Failed to delete corrupted tablet dir {} during reopen fallback", + tabletDir.getAbsolutePath()); + } + // Fall back to full init (download snapshot) + InitKvResult result = initKvTablet(); + snapshotRef.set(result.snapshotUsed.orElse(null)); + return result.tablet; } + } - return optCompletedSnapshot; + /** Result of {@link #initKvTablet()}: the created tablet and the snapshot used (if any). */ + private static class InitKvResult { + final KvTablet tablet; + final Optional snapshotUsed; + + InitKvResult(KvTablet tablet, Optional snapshotUsed) { + this.tablet = tablet; + this.snapshotUsed = snapshotUsed; + } } private void downloadKvSnapshots(CompletedSnapshot completedSnapshot, Path kvTabletDir) @@ -818,11 +1121,12 @@ private Optional getLatestSnapshot(TableBucket tableBucket) { } private void recoverKvTablet( + KvTablet tablet, long startRecoverLogOffset, @Nullable Long rowCount, @Nullable AutoIncIDRange autoIncIDRange) { long start = clock.milliseconds(); - checkNotNull(kvTablet, "kv tablet should not be null."); + checkNotNull(tablet, "kv tablet should not be null."); try { KvRecoverHelper.KvRecoverContext recoverContext = new KvRecoverHelper.KvRecoverContext( @@ -831,7 +1135,7 @@ private void recoverKvTablet( snapshotContext.maxFetchLogSizeInRecoverKv()); KvRecoverHelper kvRecoverHelper = new KvRecoverHelper( - kvTablet, + tablet, logTablet, startRecoverLogOffset, rowCount, @@ -857,8 +1161,8 @@ private void recoverKvTablet( end - start); } - private void startPeriodicKvSnapshot(@Nullable CompletedSnapshot completedSnapshot) { - checkNotNull(kvTablet); + private void startPeriodicKvSnapshot( + KvTablet kvTablet, @Nullable CompletedSnapshot completedSnapshot) { KvTabletSnapshotTarget kvTabletSnapshotTarget; try { // get the snapshot reporter to report the completed snapshot @@ -942,7 +1246,8 @@ private void startPeriodicKvSnapshot(@Nullable CompletedSnapshot completedSnapsh public long getLatestKvSnapshotSize() { if (kvSnapshotManager == null) { - return 0L; + CompletedSnapshot latestSnapshot = getLatestSnapshot(tableBucket).orElse(null); + return latestSnapshot != null ? latestSnapshot.getSnapshotSize() : 0L; } else { return kvSnapshotManager.getSnapshotSize(); } @@ -993,20 +1298,11 @@ public LogAppendInfo putRecordsToLeader( MergeMode mergeMode, int requiredAcks) throws Exception { - return inReadLock( - leaderIsrUpdateLock, - () -> { - if (!isLeader()) { - throw new NotLeaderOrFollowerException( - String.format( - "Leader not local for bucket %s on tabletServer %d", - tableBucket, localTabletServerId)); - } - - validateInSyncReplicaSize(requiredAcks); - KvTablet kv = this.kvTablet; - checkNotNull( - kv, "KvTablet for the replica to put kv records shouldn't be null."); + return withGuardedLeaderKv( + true, + requiredAcks, + "KvTablet for the replica to put kv records shouldn't be null.", + kv -> { LogAppendInfo logAppendInfo; try { logAppendInfo = kv.putAsLeader(kvRecords, targetColumns, mergeMode); @@ -1220,28 +1516,28 @@ public List lookups(List keys) { throw new NonPrimaryKeyTableException( "the primary key table not exists for " + tableBucket); } - return inReadLock( - leaderIsrUpdateLock, - () -> { - try { - if (!isLeader()) { - throw new NotLeaderOrFollowerException( + try { + return withGuardedLeaderKv( + false, + 0, + "KvTablet for the replica to get key shouldn't be null.", + kv -> { + try { + return kv.multiGet(keys); + } catch (IOException e) { + String errorMsg = String.format( - "Leader not local for bucket %s on tabletServer %d", - tableBucket, localTabletServerId)); + "Failed to lookup from local kv for table bucket %s, the cause is: %s", + tableBucket, e.getMessage()); + LOG.error(errorMsg, e); + throw new KvStorageException(errorMsg, e); } - checkNotNull( - kvTablet, "KvTablet for the replica to get key shouldn't be null."); - return kvTablet.multiGet(keys); - } catch (IOException e) { - String errorMsg = - String.format( - "Failed to lookup from local kv for table bucket %s, the cause is: %s", - tableBucket, e.getMessage()); - LOG.error(errorMsg, e); - throw new KvStorageException(errorMsg, e); - } - }); + }); + } catch (RuntimeException e) { + throw e; + } catch (Exception e) { + throw new KvStorageException("Failed to lookup from local kv for " + tableBucket, e); + } } public List prefixLookup(byte[] prefixKey) { @@ -1250,28 +1546,29 @@ public List prefixLookup(byte[] prefixKey) { "Try to do prefix lookup on a non primary key table: " + getTablePath()); } - return inReadLock( - leaderIsrUpdateLock, - () -> { - try { - if (!isLeader()) { - throw new NotLeaderOrFollowerException( + try { + return withGuardedLeaderKv( + false, + 0, + "KvTablet for the replica to get key shouldn't be null.", + kv -> { + try { + return kv.prefixLookup(prefixKey); + } catch (IOException e) { + String errorMsg = String.format( - "Leader not local for bucket %s on tabletServer %d", - tableBucket, localTabletServerId)); + "Failed to do prefix lookup from local kv for table bucket %s, the cause is: %s", + tableBucket, e.getMessage()); + LOG.error(errorMsg, e); + throw new KvStorageException(errorMsg, e); } - checkNotNull( - kvTablet, "KvTablet for the replica to get key shouldn't be null."); - return kvTablet.prefixLookup(prefixKey); - } catch (IOException e) { - String errorMsg = - String.format( - "Failed to do prefix lookup from local kv for table bucket %s, the cause is: %s", - tableBucket, e.getMessage()); - LOG.error(errorMsg, e); - throw new KvStorageException(errorMsg, e); - } - }); + }); + } catch (RuntimeException e) { + throw e; + } catch (Exception e) { + throw new KvStorageException( + "Failed to do prefix lookup from local kv for " + tableBucket, e); + } } public DefaultValueRecordBatch limitKvScan(int limit) { @@ -1280,34 +1577,85 @@ public DefaultValueRecordBatch limitKvScan(int limit) { "the primary key table not exists for " + tableBucket); } - return inReadLock( - leaderIsrUpdateLock, - () -> { - try { - if (!isLeader()) { + try { + return withGuardedLeaderKv( + false, + 0, + "KvTablet for the replica to limit scan shouldn't be null.", + kv -> { + try { + List bytes = kv.limitScan(limit); + DefaultValueRecordBatch.Builder builder = + DefaultValueRecordBatch.builder(); + for (byte[] key : bytes) { + builder.append(key); + } + return builder.build(); + } catch (IOException e) { + String errorMsg = + String.format( + "Failed to limit scan from local kv for table bucket %s, the cause is: %s", + tableBucket, e.getMessage()); + LOG.error(errorMsg, e); + throw new KvStorageException(errorMsg, e); + } + }); + } catch (RuntimeException e) { + throw e; + } catch (Exception e) { + throw new KvStorageException( + "Failed to limit scan from local kv for " + tableBucket, e); + } + } + + // Lock ordering invariant: leaderIsrUpdateLock must never be held when calling + // KvTablet.acquireGuard() — acquireGuard may trigger a slow lazy open, and holding + // the lock would block leader transitions for the entire duration. Instead, we use + // a two-phase protocol: validate leadership under lock, release lock, acquire guard + // (potentially slow), then re-validate leadership under lock. + private T withGuardedLeaderKv( + boolean validateInSyncReplicas, + int requiredAcks, + String nullKvMessage, + GuardedKvOperation action) + throws Exception { + KvTablet kv = + inReadLock( + leaderIsrUpdateLock, + () -> { + ensureLeaderForKvAccess(); + if (validateInSyncReplicas) { + validateInSyncReplicaSize(requiredAcks); + } + return checkNotNull(kvTablet, nullKvMessage); + }); + + try (KvTablet.Guard ignored = kv.acquireGuard()) { + return inReadLock( + leaderIsrUpdateLock, + () -> { + ensureLeaderForKvAccess(); + if (this.kvTablet != kv) { throw new NotLeaderOrFollowerException( String.format( "Leader not local for bucket %s on tabletServer %d", tableBucket, localTabletServerId)); } - checkNotNull( - kvTablet, - "KvTablet for the replica to limit scan shouldn't be null."); - List bytes = kvTablet.limitScan(limit); - DefaultValueRecordBatch.Builder builder = DefaultValueRecordBatch.builder(); - for (byte[] key : bytes) { - builder.append(key); + if (validateInSyncReplicas) { + validateInSyncReplicaSize(requiredAcks); } - return builder.build(); - } catch (IOException e) { - String errorMsg = - String.format( - "Failed to limit scan from local kv for table bucket %s, the cause is: %s", - tableBucket, e.getMessage()); - LOG.error(errorMsg, e); - throw new KvStorageException(errorMsg, e); - } - }); + return action.apply(kv); + }); + } + } + + private void ensureLeaderForKvAccess() { + if (!isLeader()) { + throw new NotLeaderOrFollowerException( + String.format( + "Leader not local for bucket %s on tabletServer %d", + tableBucket, localTabletServerId)); + } } public LogRecords limitLogScan(int limit) { @@ -1380,18 +1728,15 @@ public Tuple2 checkEnoughReplicasReachOffset(long requiredOffse } public long getRowCount() { - return inReadLock( - leaderIsrUpdateLock, - () -> { - KvTablet kv = this.kvTablet; - if (kv != null) { - // return materialized row count for primary key table - return kv.getRowCount(); - } else { - // return log row count for non-primary key table - return logTablet.getRowCount(); - } - }); + KvTablet kv = inReadLock(leaderIsrUpdateLock, () -> kvTablet); + + if (kv != null) { + return kv.getRowCount(); + } else if (isKvTable()) { + return 0L; + } else { + return logTablet.getRowCount(); + } } public long getOffset(RemoteLogManager remoteLogManager, ListOffsetsParam listOffsetsParam) 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 c57f1c6fb7..803aed84a8 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 @@ -77,8 +77,10 @@ import org.apache.fluss.server.entity.StopReplicaData; import org.apache.fluss.server.entity.StopReplicaResultForBucket; import org.apache.fluss.server.entity.UserContext; +import org.apache.fluss.server.kv.KvIdleReleaseController; 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.snapshot.CompletedKvSnapshotCommitter; import org.apache.fluss.server.kv.snapshot.DefaultSnapshotContext; import org.apache.fluss.server.log.FetchDataInfo; @@ -108,10 +110,12 @@ import org.apache.fluss.server.utils.FatalErrorHandler; import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.server.zk.data.lake.LakeTableSnapshot; +import org.apache.fluss.utils.ExecutorUtils; import org.apache.fluss.utils.FileUtils; import org.apache.fluss.utils.FlussPaths; import org.apache.fluss.utils.MapUtils; import org.apache.fluss.utils.clock.Clock; +import org.apache.fluss.utils.concurrent.ExecutorThreadFactory; import org.apache.fluss.utils.concurrent.Scheduler; import org.slf4j.Logger; @@ -130,9 +134,13 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; @@ -153,6 +161,10 @@ public class ReplicaManager implements ServerReconfigurable { private static final Logger LOG = LoggerFactory.getLogger(ReplicaManager.class); public static final String HIGH_WATERMARK_CHECKPOINT_FILE_NAME = "high-watermark-checkpoint"; + + // ---- Idle release internal constants ---- + private static final long IDLE_RELEASE_CHECK_INTERVAL_MS = 60_000; + private final Configuration conf; private final Scheduler scheduler; private final LogManager logManager; @@ -209,6 +221,10 @@ public class ReplicaManager implements ServerReconfigurable { private final Clock clock; + private final boolean kvIdleReleaseEnabled; + private @Nullable KvIdleReleaseController kvIdleReleaseController; + private @Nullable ScheduledExecutorService kvIdleReleaseScheduler; + public ReplicaManager( Configuration conf, Scheduler scheduler, @@ -310,6 +326,30 @@ public ReplicaManager( this.clock = clock; this.ioExecutor = ioExecutor; this.minInSyncReplicas = conf.get(ConfigOptions.LOG_REPLICA_MIN_IN_SYNC_REPLICAS_NUMBER); + this.kvIdleReleaseEnabled = kvManager != null && kvManager.isLazyOpenEnabled(); + if (kvIdleReleaseEnabled) { + long closeIdleIntervalMs = conf.get(ConfigOptions.KV_LAZY_OPEN_IDLE_TIMEOUT).toMillis(); + checkArgument( + closeIdleIntervalMs > 0, + "kv.lazy-open.idle-timeout must be positive, got: %s ms", + closeIdleIntervalMs); + this.kvIdleReleaseScheduler = + Executors.newSingleThreadScheduledExecutor( + new ExecutorThreadFactory("kv-idle-release")); + this.kvIdleReleaseController = + new KvIdleReleaseController( + kvIdleReleaseScheduler, + () -> + onlineReplicas() + .filter(Replica::isKvTable) + .map(Replica::getKvTablet) + .filter(Objects::nonNull) + .filter(KvTablet::isLazyOpen) + .collect(Collectors.toList()), + clock, + IDLE_RELEASE_CHECK_INTERVAL_MS, + closeIdleIntervalMs); + } registerMetrics(); } @@ -322,6 +362,11 @@ public void startup() { this::maybeShrinkIsr, 0L, conf.get(ConfigOptions.LOG_REPLICA_MAX_LAG_TIME).toMillis() / 2); + + // Start idle release controller if enabled + if (kvIdleReleaseController != null) { + kvIdleReleaseController.start(); + } } public RemoteLogManager getRemoteLogManager() { @@ -2009,6 +2054,14 @@ public Replica getReplica() { public static final class OfflineReplica implements HostedReplica {} public void shutdown() throws InterruptedException { + // Shut down the idle release controller if it was created + if (kvIdleReleaseController != null) { + kvIdleReleaseController.close(); + } + if (kvIdleReleaseScheduler != null) { + ExecutorUtils.gracefulShutdown(5, TimeUnit.SECONDS, kvIdleReleaseScheduler); + } + // Close the resources for snapshot kv kvSnapshotResource.close(); replicaFetcherManager.shutdown(); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/KvIdleReleaseControllerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/KvIdleReleaseControllerTest.java new file mode 100644 index 0000000000..890c0cd1f7 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/KvIdleReleaseControllerTest.java @@ -0,0 +1,306 @@ +/* + * 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; + +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.config.TableConfig; +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.KvRecordBatch; +import org.apache.fluss.record.KvRecordTestUtils; +import org.apache.fluss.record.TestData; +import org.apache.fluss.record.TestingSchemaGetter; +import org.apache.fluss.server.log.LogManager; +import org.apache.fluss.server.log.LogTablet; +import org.apache.fluss.server.metrics.group.TestingMetricGroups; +import org.apache.fluss.server.zk.NOPErrorHandler; +import org.apache.fluss.server.zk.ZooKeeperClient; +import org.apache.fluss.server.zk.ZooKeeperExtension; +import org.apache.fluss.testutils.common.AllCallbackWrapper; +import org.apache.fluss.testutils.common.ManuallyTriggeredScheduledExecutorService; +import org.apache.fluss.types.RowType; +import org.apache.fluss.utils.clock.ManualClock; +import org.apache.fluss.utils.clock.SystemClock; +import org.apache.fluss.utils.concurrent.FlussScheduler; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +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.assertThatCode; + +/** Test for {@link KvIdleReleaseController}. */ +final class KvIdleReleaseControllerTest { + + @RegisterExtension + public static final AllCallbackWrapper ZOO_KEEPER_EXTENSION_WRAPPER = + new AllCallbackWrapper<>(new ZooKeeperExtension()); + + private static final short SCHEMA_ID = 1; + private static final RowType BASE_ROW_TYPE = TestData.DATA1_ROW_TYPE; + private static final long IDLE_INTERVAL_MS = 60_000; + private static final long CHECK_INTERVAL_MS = 10_000; + + private final KvRecordTestUtils.KvRecordBatchFactory batchFactory = + KvRecordTestUtils.KvRecordBatchFactory.of(SCHEMA_ID); + private final KvRecordTestUtils.KvRecordFactory recordFactory = + KvRecordTestUtils.KvRecordFactory.of(BASE_ROW_TYPE); + + private static ZooKeeperClient zkClient; + + private @TempDir File tempDir; + + private LogManager logManager; + private KvManager kvManager; + private Configuration conf; + private ManualClock clock; + private ManuallyTriggeredScheduledExecutorService scheduler; + private List allTablets; + private int nextTableId = 30001; + + @BeforeAll + static void baseBeforeAll() { + zkClient = + ZOO_KEEPER_EXTENSION_WRAPPER + .getCustomExtension() + .getZooKeeperClient(NOPErrorHandler.INSTANCE); + } + + @BeforeEach + void setup() throws Exception { + conf = new Configuration(); + conf.setString(ConfigOptions.DATA_DIR, tempDir.getAbsolutePath()); + conf.set(ConfigOptions.KV_LAZY_OPEN_ENABLED, true); + + logManager = + LogManager.create( + conf, + zkClient, + new FlussScheduler(1), + SystemClock.getInstance(), + TestingMetricGroups.TABLET_SERVER_METRICS); + kvManager = + KvManager.create( + conf, zkClient, logManager, TestingMetricGroups.TABLET_SERVER_METRICS); + kvManager.startup(); + + clock = new ManualClock(System.currentTimeMillis()); + scheduler = new ManuallyTriggeredScheduledExecutorService(); + allTablets = new ArrayList<>(); + } + + @AfterEach + void tearDown() { + if (kvManager != null) { + kvManager.shutdown(); + } + if (logManager != null) { + logManager.shutdown(); + } + } + + // ---- Helper methods ---- + + private KvTablet createAndOpenTablet(int tableId, int bucketId) throws Exception { + TablePath tablePath = TablePath.of("db1", "t_idle_" + tableId); + PhysicalTablePath physicalPath = + PhysicalTablePath.of(tablePath.getDatabaseName(), tablePath.getTableName(), null); + TableBucket tableBucket = new TableBucket((long) tableId, bucketId); + LogTablet logTablet = + logManager.getOrCreateLog(physicalPath, tableBucket, LogFormat.ARROW, 1, true); + + KvTablet sentinel = + KvTablet.createLazySentinel( + physicalPath, + tableBucket, + logTablet, + TestingMetricGroups.TABLET_SERVER_METRICS); + + KvTabletLazyLifecycle lifecycle = sentinel.getLifecycle(); + lifecycle.configureLazyOpen( + clock, + kvManager.getOpenSemaphore(), + kvManager.getOpenTimeoutMs(), + kvManager.getFailedBackoffBaseMs(), + kvManager.getFailedBackoffMaxMs(), + KvManager.RELEASE_DRAIN_TIMEOUT_MS); + lifecycle.setLeaderEpochSupplier(() -> 1); + lifecycle.setBucketEpochSupplier(() -> 1); + lifecycle.setTabletDirSupplier(() -> kvManager.getTabletDirPath(physicalPath, tableBucket)); + lifecycle.setOpenCallback( + hasLocal -> + kvManager.createKvTabletUnregistered( + physicalPath, + tableBucket, + logTablet, + KvFormat.COMPACTED, + new TestingSchemaGetter(new SchemaInfo(DATA1_SCHEMA_PK, SCHEMA_ID)), + new TableConfig(new Configuration()), + DEFAULT_COMPRESSION)); + lifecycle.setCommitCallback(kv -> {}); + lifecycle.setReleaseCallback(kv -> {}); + lifecycle.setDropCallback(kv -> {}); + + // Open the tablet via acquireGuard + try (KvTablet.Guard guard = sentinel.acquireGuard()) { + // Write and flush so releaseKv() won't be rejected + KvRecord record = + recordFactory.ofRecord( + ("k_" + tableId + "_" + bucketId).getBytes(), + new Object[] {tableId, "v"}); + KvRecordBatch batch = batchFactory.ofRecords(Collections.singletonList(record)); + sentinel.putAsLeader(batch, null); + sentinel.flush(Long.MAX_VALUE, NOPErrorHandler.INSTANCE); + } + + return sentinel; + } + + private Supplier> openTabletSupplier() { + return () -> allTablets.stream().filter(KvTablet::isLazyOpen).collect(Collectors.toList()); + } + + private KvIdleReleaseController createController() { + return new KvIdleReleaseController( + scheduler, openTabletSupplier(), clock, CHECK_INTERVAL_MS, IDLE_INTERVAL_MS); + } + + // ---- Tests ---- + + @Test + void testNoReleaseWhenAllActive() throws Exception { + KvTablet tablet1 = createAndOpenTablet(nextTableId++, 0); + KvTablet tablet2 = createAndOpenTablet(nextTableId++, 0); + allTablets.add(tablet1); + allTablets.add(tablet2); + + // Don't advance clock — tablets are recently accessed + KvIdleReleaseController controller = createController(); + controller.checkAndRelease(); + + assertThat(tablet1.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.OPEN); + assertThat(tablet2.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.OPEN); + } + + @Test + void testReleaseAllIdleTablets() throws Exception { + KvTablet tablet1 = createAndOpenTablet(nextTableId++, 0); + KvTablet tablet2 = createAndOpenTablet(nextTableId++, 0); + allTablets.add(tablet1); + allTablets.add(tablet2); + + // Advance clock past idle interval + clock.advanceTime(IDLE_INTERVAL_MS + 1, TimeUnit.MILLISECONDS); + + KvIdleReleaseController controller = createController(); + controller.checkAndRelease(); + + assertThat(tablet1.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.LAZY); + assertThat(tablet2.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.LAZY); + } + + @Test + void testOnlyIdleTabletsReleased() throws Exception { + // Create tablet1 first, then advance clock, then create tablet2 + KvTablet tablet1 = createAndOpenTablet(nextTableId++, 0); + allTablets.add(tablet1); + + // Advance clock so tablet1 becomes idle but tablet2 (created after) won't be + clock.advanceTime(IDLE_INTERVAL_MS + 1, TimeUnit.MILLISECONDS); + + KvTablet tablet2 = createAndOpenTablet(nextTableId++, 0); + allTablets.add(tablet2); + + KvIdleReleaseController controller = createController(); + controller.checkAndRelease(); + + // tablet1 is idle — released + assertThat(tablet1.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.LAZY); + // tablet2 is recent — stays open + assertThat(tablet2.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.OPEN); + } + + @Test + void testSkipTabletWithActivePins() throws Exception { + KvTablet tablet1 = createAndOpenTablet(nextTableId++, 0); + KvTablet tablet2 = createAndOpenTablet(nextTableId++, 0); + allTablets.add(tablet1); + allTablets.add(tablet2); + + // Hold a guard on tablet1 (don't close it) — this pins it + try (KvTablet.Guard heldGuard = tablet1.acquireGuard()) { + // Advance clock past idle + clock.advanceTime(IDLE_INTERVAL_MS + 1, TimeUnit.MILLISECONDS); + + KvIdleReleaseController controller = createController(); + controller.checkAndRelease(); + + // tablet1 is pinned — should NOT be released + assertThat(tablet1.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.OPEN); + // tablet2 is idle and unpinned — should be released + assertThat(tablet2.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.LAZY); + } + } + + @Test + void testContinuesAfterReleaseFailure() throws Exception { + KvTablet tablet1 = createAndOpenTablet(nextTableId++, 0); + KvTablet tablet2 = createAndOpenTablet(nextTableId++, 0); + allTablets.add(tablet1); + allTablets.add(tablet2); + + // Make tablet1's releaseCallback throw so releaseKv() returns false + tablet1.getLifecycle() + .setReleaseCallback( + kv -> { + throw new RuntimeException("simulated release failure"); + }); + + // Advance clock past idle + clock.advanceTime(IDLE_INTERVAL_MS + 1, TimeUnit.MILLISECONDS); + + KvIdleReleaseController controller = createController(); + assertThatCode(() -> controller.checkAndRelease()).doesNotThrowAnyException(); + + // tablet1 release failed — should rollback to OPEN + assertThat(tablet1.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.OPEN); + // tablet2 should still be released despite tablet1's failure + assertThat(tablet2.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.LAZY); + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/KvManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/KvManagerTest.java index cecff0f8ff..7e1b84ba77 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/kv/KvManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/KvManagerTest.java @@ -299,6 +299,127 @@ void testDropKv(String partitionName) throws Exception { assertThat(kvManager.getKv(tableBucket1)).isPresent(); } + @ParameterizedTest + @MethodSource("partitionProvider") + void testRegisterKv(String partitionName) throws Exception { + initTableBuckets(partitionName); + PhysicalTablePath physicalTablePath = + PhysicalTablePath.of( + tablePath1.getDatabaseName(), tablePath1.getTableName(), partitionName); + LogTablet logTablet = + logManager.getOrCreateLog( + physicalTablePath, tableBucket1, LogFormat.ARROW, 1, true); + + // Create a lazy sentinel and register it + KvTablet sentinel = + KvTablet.createLazySentinel(physicalTablePath, tableBucket1, logTablet, null); + kvManager.registerKv(tableBucket1, sentinel); + + // verify registered + assertThat(kvManager.getKv(tableBucket1)).isPresent(); + assertThat(kvManager.getKv(tableBucket1).get()).isSameAs(sentinel); + } + + @Test + void testRegisterKvDuplicate() throws Exception { + initTableBuckets(null); + PhysicalTablePath physicalTablePath = + PhysicalTablePath.of(tablePath1.getDatabaseName(), tablePath1.getTableName(), null); + LogTablet logTablet = + logManager.getOrCreateLog( + physicalTablePath, tableBucket1, LogFormat.ARROW, 1, true); + + KvTablet sentinel = + KvTablet.createLazySentinel(physicalTablePath, tableBucket1, logTablet, null); + kvManager.registerKv(tableBucket1, sentinel); + + // registering again should throw + KvTablet sentinel2 = + KvTablet.createLazySentinel(physicalTablePath, tableBucket1, logTablet, null); + org.assertj.core.api.Assertions.assertThatThrownBy( + () -> kvManager.registerKv(tableBucket1, sentinel2)) + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining("already registered"); + } + + @ParameterizedTest + @MethodSource("partitionProvider") + void testUnregisterKv(String partitionName) throws Exception { + initTableBuckets(partitionName); + getOrCreateKv(tablePath1, partitionName, tableBucket1); + assertThat(kvManager.getKv(tableBucket1)).isPresent(); + + kvManager.unregisterKv(tableBucket1); + assertThat(kvManager.getKv(tableBucket1)).isNotPresent(); + } + + @ParameterizedTest + @MethodSource("partitionProvider") + void testCreateKvTabletUnregistered(String partitionName) throws Exception { + initTableBuckets(partitionName); + PhysicalTablePath physicalTablePath = + PhysicalTablePath.of( + tablePath1.getDatabaseName(), tablePath1.getTableName(), partitionName); + LogTablet logTablet = + logManager.getOrCreateLog( + physicalTablePath, tableBucket1, LogFormat.ARROW, 1, true); + + KvTablet tablet = + kvManager.createKvTabletUnregistered( + physicalTablePath, + tableBucket1, + logTablet, + KvFormat.COMPACTED, + new TestingSchemaGetter(new SchemaInfo(DATA1_SCHEMA_PK, 1)), + new TableConfig(new Configuration()), + DEFAULT_COMPRESSION); + + // tablet should be created but NOT registered + assertThat(tablet).isNotNull(); + assertThat(tablet.getKvTabletDir()).exists(); + assertThat(kvManager.getKv(tableBucket1)).isNotPresent(); + } + + @ParameterizedTest + @MethodSource("partitionProvider") + void testGetTabletDirPath(String partitionName) throws Exception { + initTableBuckets(partitionName); + PhysicalTablePath physicalTablePath = + PhysicalTablePath.of( + tablePath1.getDatabaseName(), tablePath1.getTableName(), partitionName); + + File dirPath = kvManager.getTabletDirPath(physicalTablePath, tableBucket1); + // directory should not be created by this call + assertThat(dirPath).doesNotExist(); + + // Assert the exact expected path structure + File expectedDir; + if (partitionName == null) { + // Non-partitioned: {dataDir}/{db}/{table}-{tableId}/kv-{bucket} + expectedDir = + new File( + tempDir, + "db1/t1-" + + tableBucket1.getTableId() + + "/kv-" + + tableBucket1.getBucket()); + } else { + // Partitioned: {dataDir}/{db}/{table}-{tableId}/{partition}-p{partitionId}/kv-{bucket} + expectedDir = + new File( + tempDir, + "db1/t1-" + + tableBucket1.getTableId() + + "/" + + partitionName + + "-p" + + tableBucket1.getPartitionId() + + "/kv-" + + tableBucket1.getBucket()); + } + assertThat(dirPath).isEqualTo(expectedDir); + } + @Test void testGetNonExistentKv() { initTableBuckets(null); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/KvTabletLazyLifecycleTest.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/KvTabletLazyLifecycleTest.java new file mode 100644 index 0000000000..e479dad44d --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/KvTabletLazyLifecycleTest.java @@ -0,0 +1,1197 @@ +/* + * 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; + +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.config.TableConfig; +import org.apache.fluss.exception.KvStorageException; +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.KvRecordBatch; +import org.apache.fluss.record.KvRecordTestUtils; +import org.apache.fluss.record.TestData; +import org.apache.fluss.record.TestingSchemaGetter; +import org.apache.fluss.row.encode.ValueEncoder; +import org.apache.fluss.server.log.LogManager; +import org.apache.fluss.server.log.LogTablet; +import org.apache.fluss.server.metrics.group.TabletServerMetricGroup; +import org.apache.fluss.server.metrics.group.TestingMetricGroups; +import org.apache.fluss.server.zk.NOPErrorHandler; +import org.apache.fluss.server.zk.ZooKeeperClient; +import org.apache.fluss.server.zk.ZooKeeperExtension; +import org.apache.fluss.testutils.common.AllCallbackWrapper; +import org.apache.fluss.testutils.common.CommonTestUtils; +import org.apache.fluss.types.RowType; +import org.apache.fluss.utils.clock.ManualClock; +import org.apache.fluss.utils.clock.SystemClock; +import org.apache.fluss.utils.concurrent.FlussScheduler; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.time.Duration; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.FutureTask; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +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.DataTestUtils.compactedRow; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Test for {@link KvTabletLazyLifecycle}. */ +final class KvTabletLazyLifecycleTest { + + @RegisterExtension + public static final AllCallbackWrapper ZOO_KEEPER_EXTENSION_WRAPPER = + new AllCallbackWrapper<>(new ZooKeeperExtension()); + + private static final short SCHEMA_ID = 1; + private static final RowType BASE_ROW_TYPE = TestData.DATA1_ROW_TYPE; + + private final KvRecordTestUtils.KvRecordBatchFactory batchFactory = + KvRecordTestUtils.KvRecordBatchFactory.of(SCHEMA_ID); + private final KvRecordTestUtils.KvRecordFactory recordFactory = + KvRecordTestUtils.KvRecordFactory.of(BASE_ROW_TYPE); + + private static ZooKeeperClient zkClient; + + private @TempDir File tempDir; + + private LogManager logManager; + private KvManager kvManager; + private Configuration conf; + + private PhysicalTablePath physicalPath; + private TableBucket tableBucket; + private LogTablet logTablet; + + @BeforeAll + static void baseBeforeAll() { + zkClient = + ZOO_KEEPER_EXTENSION_WRAPPER + .getCustomExtension() + .getZooKeeperClient(NOPErrorHandler.INSTANCE); + } + + @BeforeEach + void setup() throws Exception { + conf = new Configuration(); + conf.setString(ConfigOptions.DATA_DIR, tempDir.getAbsolutePath()); + conf.set(ConfigOptions.KV_LAZY_OPEN_ENABLED, true); + + logManager = + LogManager.create( + conf, + zkClient, + new FlussScheduler(1), + SystemClock.getInstance(), + TestingMetricGroups.TABLET_SERVER_METRICS); + kvManager = + KvManager.create( + conf, zkClient, logManager, TestingMetricGroups.TABLET_SERVER_METRICS); + kvManager.startup(); + + TablePath tablePath = TablePath.of("db1", "t_lazy"); + physicalPath = + PhysicalTablePath.of(tablePath.getDatabaseName(), tablePath.getTableName(), null); + tableBucket = new TableBucket(20001L, 0); + logTablet = logManager.getOrCreateLog(physicalPath, tableBucket, LogFormat.ARROW, 1, true); + } + + @AfterEach + void tearDown() { + if (kvManager != null) { + kvManager.shutdown(); + } + if (logManager != null) { + logManager.shutdown(); + } + } + + // ---- Helper methods ---- + + private KvTablet createSentinel() { + return KvTablet.createLazySentinel( + physicalPath, tableBucket, logTablet, TestingMetricGroups.TABLET_SERVER_METRICS); + } + + private KvTabletLazyLifecycle configureLifecycle(KvTablet sentinel, ManualClock clock) { + KvTabletLazyLifecycle lifecycle = sentinel.getLifecycle(); + lifecycle.configureLazyOpen( + clock, + kvManager.getOpenSemaphore(), + kvManager.getOpenTimeoutMs(), + kvManager.getFailedBackoffBaseMs(), + kvManager.getFailedBackoffMaxMs(), + KvManager.RELEASE_DRAIN_TIMEOUT_MS); + lifecycle.setLeaderEpochSupplier(() -> 1); + lifecycle.setBucketEpochSupplier(() -> 1); + lifecycle.setTabletDirSupplier(() -> kvManager.getTabletDirPath(physicalPath, tableBucket)); + lifecycle.setOpenCallback( + hasLocal -> + kvManager.createKvTabletUnregistered( + physicalPath, + tableBucket, + logTablet, + KvFormat.COMPACTED, + new TestingSchemaGetter(new SchemaInfo(DATA1_SCHEMA_PK, SCHEMA_ID)), + new TableConfig(new Configuration()), + DEFAULT_COMPRESSION)); + lifecycle.setCommitCallback(kv -> {}); + lifecycle.setReleaseCallback(kv -> {}); + lifecycle.setDropCallback(kv -> {}); + return lifecycle; + } + + // ---- Tests ---- + + @Test + void testAcquireGuardTriggersOpen() throws Exception { + ManualClock clock = new ManualClock(System.currentTimeMillis()); + KvTablet sentinel = createSentinel(); + KvTabletLazyLifecycle lifecycle = configureLifecycle(sentinel, clock); + + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.LAZY); + + try (KvTablet.Guard guard = sentinel.acquireGuard()) { + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.OPEN); + // Verify we can use the tablet inside the guard + List values = sentinel.multiGet(Collections.singletonList("k1".getBytes())); + assertThat(values).hasSize(1); + assertThat(values.get(0)).isNull(); + } + } + + @Test + void testAcquireGuardFastPath() throws Exception { + ManualClock clock = new ManualClock(System.currentTimeMillis()); + KvTablet sentinel = createSentinel(); + KvTabletLazyLifecycle lifecycle = configureLifecycle(sentinel, clock); + + // Track open callback invocations to prove fast path skips it + AtomicInteger openCount = new AtomicInteger(0); + lifecycle.setOpenCallback( + hasLocal -> { + openCount.incrementAndGet(); + return kvManager.createKvTabletUnregistered( + physicalPath, + tableBucket, + logTablet, + KvFormat.COMPACTED, + new TestingSchemaGetter(new SchemaInfo(DATA1_SCHEMA_PK, SCHEMA_ID)), + new TableConfig(new Configuration()), + DEFAULT_COMPRESSION); + }); + + // First open via slow path + try (KvTablet.Guard guard = sentinel.acquireGuard()) { + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.OPEN); + } + assertThat(openCount.get()).isEqualTo(1); + + // Second acquireGuard should take the fast path (already OPEN, no openCallback) + try (KvTablet.Guard guard = sentinel.acquireGuard()) { + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.OPEN); + List values = + sentinel.multiGet(Collections.singletonList("nonexistent".getBytes())); + assertThat(values).hasSize(1); + assertThat(values.get(0)).isNull(); + } + // openCallback must NOT have been called again + assertThat(openCount.get()).isEqualTo(1); + } + + @Test + void testFailedBackoffRejectsOpen() { + ManualClock clock = new ManualClock(System.currentTimeMillis()); + KvTablet sentinel = createSentinel(); + KvTabletLazyLifecycle lifecycle = configureLifecycle(sentinel, clock); + + // Override openCallback to throw + lifecycle.setOpenCallback( + hasLocal -> { + throw new RuntimeException("simulated open failure"); + }); + + // First acquireGuard should fail + assertThatThrownBy(sentinel::acquireGuard).isInstanceOf(RuntimeException.class); + + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.FAILED); + + // Subsequent acquireGuard within cooldown should throw with cooldown message + assertThatThrownBy(sentinel::acquireGuard) + .isInstanceOf(KvStorageException.class) + .hasMessageContaining("cooldown"); + } + + @Test + void testFailedBackoffRetryAfterCooldown() throws Exception { + ManualClock clock = new ManualClock(System.currentTimeMillis()); + KvTablet sentinel = createSentinel(); + KvTabletLazyLifecycle lifecycle = configureLifecycle(sentinel, clock); + + // Override openCallback to throw + lifecycle.setOpenCallback( + hasLocal -> { + throw new RuntimeException("simulated open failure"); + }); + + // First acquireGuard fails + assertThatThrownBy(sentinel::acquireGuard).isInstanceOf(RuntimeException.class); + + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.FAILED); + + // Advance clock past the backoff (5000 * 2^1 = 10000ms for failureCount=1) + clock.advanceTime(11, TimeUnit.SECONDS); + + // Fix the openCallback + lifecycle.setOpenCallback( + hasLocal -> + kvManager.createKvTabletUnregistered( + physicalPath, + tableBucket, + logTablet, + KvFormat.COMPACTED, + new TestingSchemaGetter(new SchemaInfo(DATA1_SCHEMA_PK, SCHEMA_ID)), + new TableConfig(new Configuration()), + DEFAULT_COMPRESSION)); + + // Now acquireGuard should succeed + try (KvTablet.Guard guard = sentinel.acquireGuard()) { + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.OPEN); + } + } + + @Test + void testReleaseKvSuccess() throws Exception { + ManualClock clock = new ManualClock(System.currentTimeMillis()); + KvTablet sentinel = createSentinel(); + KvTabletLazyLifecycle lifecycle = configureLifecycle(sentinel, clock); + + // Open and write+flush data + try (KvTablet.Guard guard = sentinel.acquireGuard()) { + KvRecord record = recordFactory.ofRecord("k1".getBytes(), new Object[] {1, "a"}); + KvRecordBatch batch = batchFactory.ofRecords(Collections.singletonList(record)); + sentinel.putAsLeader(batch, null); + sentinel.flush(Long.MAX_VALUE, NOPErrorHandler.INSTANCE); + } + + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.OPEN); + + // Capture flushed offset before release (detach may reset it) + long flushedOffsetBeforeRelease = sentinel.getFlushedLogOffset(); + + // Release should succeed + boolean released = sentinel.releaseKv(); + assertThat(released).isTrue(); + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.LAZY); + // Row count and flushed offset should be cached + assertThat(lifecycle.getCachedRowCount()).isEqualTo(1L); + assertThat(lifecycle.getCachedFlushedLogOffset()).isEqualTo(flushedOffsetBeforeRelease); + } + + @Test + void testReleaseKvRejectsUnflushedData() throws Exception { + ManualClock clock = new ManualClock(System.currentTimeMillis()); + KvTablet sentinel = createSentinel(); + KvTabletLazyLifecycle lifecycle = configureLifecycle(sentinel, clock); + + // Open and write data WITHOUT flush + try (KvTablet.Guard guard = sentinel.acquireGuard()) { + KvRecord record = recordFactory.ofRecord("k1".getBytes(), new Object[] {1, "a"}); + KvRecordBatch batch = batchFactory.ofRecords(Collections.singletonList(record)); + sentinel.putAsLeader(batch, null); + // Intentionally NOT flushing + } + + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.OPEN); + + // Release should fail because data is unflushed + boolean released = sentinel.releaseKv(); + assertThat(released).isFalse(); + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.OPEN); + } + + @Test + void testReleaseAndReopenPreservesData() throws Exception { + ManualClock clock = new ManualClock(System.currentTimeMillis()); + KvTablet sentinel = createSentinel(); + KvTabletLazyLifecycle lifecycle = configureLifecycle(sentinel, clock); + + // Open, write+flush + try (KvTablet.Guard guard = sentinel.acquireGuard()) { + KvRecord record = recordFactory.ofRecord("k1".getBytes(), new Object[] {1, "a"}); + KvRecordBatch batch = batchFactory.ofRecords(Collections.singletonList(record)); + sentinel.putAsLeader(batch, null); + sentinel.flush(Long.MAX_VALUE, NOPErrorHandler.INSTANCE); + } + + // Release to LAZY + boolean released = sentinel.releaseKv(); + assertThat(released).isTrue(); + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.LAZY); + + // Reopen via acquireGuard and verify data is still readable + try (KvTablet.Guard guard = sentinel.acquireGuard()) { + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.OPEN); + List values = sentinel.multiGet(Collections.singletonList("k1".getBytes())); + assertThat(values).hasSize(1); + byte[] expected = + ValueEncoder.encodeValue( + SCHEMA_ID, compactedRow(BASE_ROW_TYPE, new Object[] {1, "a"})); + assertThat(values.get(0)).isEqualTo(expected); + } + } + + @Test + void testReleaseKvRejectsWriteThatCompletesDuringDrain() throws Exception { + ManualClock clock = new ManualClock(System.currentTimeMillis()); + KvTablet sentinel = createSentinel(); + KvTabletLazyLifecycle lifecycle = configureLifecycle(sentinel, clock); + + KvTablet.Guard guard = sentinel.acquireGuard(); + FutureTask releaseTask = new FutureTask<>(sentinel::releaseKv); + Thread releaseThread = new Thread(releaseTask, "release-during-write"); + releaseThread.start(); + + CommonTestUtils.waitUntil( + () -> lifecycle.getLazyState() == KvTabletLazyLifecycle.LazyState.RELEASING, + Duration.ofSeconds(5), + "Timed out waiting for RELEASING state"); + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.RELEASING); + + KvRecord record = recordFactory.ofRecord("k1".getBytes(), new Object[] {1, "a"}); + KvRecordBatch batch = batchFactory.ofRecords(Collections.singletonList(record)); + sentinel.putAsLeader(batch, null); + guard.close(); + assertThat(releaseTask.get(5, TimeUnit.SECONDS)).isFalse(); + + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.OPEN); + assertThat(sentinel.getKvPreWriteBuffer().getAllKvEntries()).hasSize(1); + } + + @Test + void testDropKvFromLazy() { + ManualClock clock = new ManualClock(System.currentTimeMillis()); + KvTablet sentinel = createSentinel(); + KvTabletLazyLifecycle lifecycle = configureLifecycle(sentinel, clock); + + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.LAZY); + + sentinel.dropKvLazy(); + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.CLOSED); + } + + @Test + void testDropKvFromOpen() throws Exception { + ManualClock clock = new ManualClock(System.currentTimeMillis()); + KvTablet sentinel = createSentinel(); + KvTabletLazyLifecycle lifecycle = configureLifecycle(sentinel, clock); + + // Open the tablet + try (KvTablet.Guard guard = sentinel.acquireGuard()) { + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.OPEN); + } + + File kvDir = sentinel.getKvTabletDir(); + assertThat(kvDir).isNotNull(); + assertThat(kvDir).exists(); + + sentinel.dropKvLazy(); + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.CLOSED); + assertThat(sentinel.getRocksDBKv()).isNull(); + assertThat(kvDir).doesNotExist(); + } + + @Test + void testDropKvDuringReleasingRollbackStillClosesAndDeletes() throws Exception { + ManualClock clock = new ManualClock(System.currentTimeMillis()); + KvTablet sentinel = createSentinel(); + KvTabletLazyLifecycle lifecycle = configureLifecycle(sentinel, clock); + + try (KvTablet.Guard guard = sentinel.acquireGuard()) { + KvRecord record = recordFactory.ofRecord("k1".getBytes(), new Object[] {1, "a"}); + KvRecordBatch batch = batchFactory.ofRecords(Collections.singletonList(record)); + sentinel.putAsLeader(batch, null); + sentinel.flush(Long.MAX_VALUE, NOPErrorHandler.INSTANCE); + } + + File kvDir = sentinel.getKvTabletDir(); + assertThat(kvDir).isNotNull(); + assertThat(kvDir).exists(); + + CountDownLatch releaseCallbackEntered = new CountDownLatch(1); + CountDownLatch allowReleaseCallbackFinish = new CountDownLatch(1); + lifecycle.setReleaseCallback( + kv -> { + releaseCallbackEntered.countDown(); + try { + allowReleaseCallbackFinish.await(5, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + throw new RuntimeException("simulated release failure"); + }); + + FutureTask releaseTask = new FutureTask<>(sentinel::releaseKv); + Thread releaseThread = new Thread(releaseTask, "release-with-rollback"); + releaseThread.start(); + + assertThat(releaseCallbackEntered.await(5, TimeUnit.SECONDS)).isTrue(); + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.RELEASING); + + FutureTask dropTask = + new FutureTask<>( + () -> { + sentinel.dropKvLazy(); + return null; + }); + Thread dropThread = new Thread(dropTask, "drop-during-release-rollback"); + dropThread.start(); + + allowReleaseCallbackFinish.countDown(); + assertThat(releaseTask.get(5, TimeUnit.SECONDS)).isFalse(); + dropTask.get(5, TimeUnit.SECONDS); + + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.CLOSED); + assertThat(sentinel.getRocksDBKv()).isNull(); + assertThat(kvDir).doesNotExist(); + } + + @Test + void testCommitCallbackFailureDoesNotFailOpen() throws Exception { + ManualClock clock = new ManualClock(System.currentTimeMillis()); + KvTablet sentinel = createSentinel(); + KvTabletLazyLifecycle lifecycle = configureLifecycle(sentinel, clock); + + lifecycle.setCommitCallback( + kv -> { + throw new RuntimeException("simulated commit callback failure"); + }); + + try (KvTablet.Guard guard = sentinel.acquireGuard()) { + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.OPEN); + assertThat(sentinel.getRocksDBKv()).isNotNull(); + } + } + + @Test + void testDropWaitsForCommitCallbackCompletion() throws Exception { + ManualClock clock = new ManualClock(System.currentTimeMillis()); + KvTablet sentinel = createSentinel(); + KvTabletLazyLifecycle lifecycle = configureLifecycle(sentinel, clock); + + CountDownLatch commitCallbackEntered = new CountDownLatch(1); + CountDownLatch allowCommitCallbackFinish = new CountDownLatch(1); + lifecycle.setCommitCallback( + kv -> { + commitCallbackEntered.countDown(); + try { + allowCommitCallbackFinish.await(5, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + }); + + AtomicReference openFailure = new AtomicReference<>(); + Thread openThread = + new Thread( + () -> { + try (KvTablet.Guard guard = sentinel.acquireGuard()) { + // no-op + } catch (Throwable t) { + openFailure.set(t); + } + }, + "open-with-blocked-commit"); + openThread.start(); + + assertThat(commitCallbackEntered.await(5, TimeUnit.SECONDS)).isTrue(); + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.OPEN); + + FutureTask dropTask = + new FutureTask<>( + () -> { + sentinel.dropKvLazy(); + return null; + }); + Thread dropThread = new Thread(dropTask, "drop-during-commit"); + dropThread.start(); + + // Wait for drop thread to block (commit callback holds the lock) + CommonTestUtils.waitUntil( + () -> { + Thread.State s = dropThread.getState(); + return s == Thread.State.WAITING || s == Thread.State.TIMED_WAITING; + }, + Duration.ofSeconds(5), + "Timed out waiting for drop thread to block"); + assertThat(dropTask.isDone()).isFalse(); + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.OPEN); + + allowCommitCallbackFinish.countDown(); + openThread.join(TimeUnit.SECONDS.toMillis(5)); + assertThat(openThread.isAlive()).isFalse(); + dropTask.get(5, TimeUnit.SECONDS); + + // The open thread may or may not see a KvStorageException depending on the race + // between guard.close() and dropKvLazy() transitioning to CLOSED. Both outcomes are valid: + // - null: guard closed before drop detached RocksDB + // - KvStorageException: drop completed while open thread was still in acquireGuard + Throwable failure = openFailure.get(); + if (failure != null) { + assertThat(failure).isInstanceOf(KvStorageException.class); + } + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.CLOSED); + assertThat(sentinel.getRocksDBKv()).isNull(); + } + + @Test + void testCloseMarksLazyTabletClosed() throws Exception { + ManualClock clock = new ManualClock(System.currentTimeMillis()); + KvTablet sentinel = createSentinel(); + KvTabletLazyLifecycle lifecycle = configureLifecycle(sentinel, clock); + + try (KvTablet.Guard guard = sentinel.acquireGuard()) { + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.OPEN); + } + + sentinel.close(); + + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.CLOSED); + assertThat(sentinel.getRocksDBKv()).isNull(); + assertThatThrownBy(sentinel::acquireGuard).isInstanceOf(KvStorageException.class); + } + + @Test + void testGuardedExecutorPinsOpenTabletUntilTaskCompletes() throws Exception { + ManualClock clock = new ManualClock(System.currentTimeMillis()); + KvTablet sentinel = createSentinel(); + KvTabletLazyLifecycle lifecycle = configureLifecycle(sentinel, clock); + + try (KvTablet.Guard guard = sentinel.acquireGuard()) { + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.OPEN); + } + + CountDownLatch taskStarted = new CountDownLatch(1); + CountDownLatch allowTaskComplete = new CountDownLatch(1); + FutureTask guardedTask = + new FutureTask<>( + () -> { + sentinel.getGuardedExecutor() + .execute( + () -> { + taskStarted.countDown(); + try { + allowTaskComplete.await(5, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + }); + return null; + }); + Thread taskThread = new Thread(guardedTask, "guarded-executor-task"); + taskThread.start(); + + assertThat(taskStarted.await(5, TimeUnit.SECONDS)).isTrue(); + assertThat(sentinel.getActivePins()).isEqualTo(1); + + FutureTask releaseTask = new FutureTask<>(sentinel::releaseKv); + Thread releaseThread = new Thread(releaseTask, "release-task"); + releaseThread.start(); + + CommonTestUtils.waitUntil( + () -> lifecycle.getLazyState() == KvTabletLazyLifecycle.LazyState.RELEASING, + Duration.ofSeconds(5), + "Timed out waiting for RELEASING state"); + assertThat(releaseTask.isDone()).isFalse(); + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.RELEASING); + + allowTaskComplete.countDown(); + assertThat(releaseTask.get(5, TimeUnit.SECONDS)).isTrue(); + taskThread.join(TimeUnit.SECONDS.toMillis(5)); + assertThat(taskThread.isAlive()).isFalse(); + + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.LAZY); + assertThat(sentinel.getActivePins()).isZero(); + } + + @Test + void testGaugeUpdatesOnTransitions() throws Exception { + TabletServerMetricGroup mg = TestingMetricGroups.TABLET_SERVER_METRICS; + // Record baseline counts before this test + int baseLazy = mg.kvTabletLazyCount().get(); + int baseOpen = mg.kvTabletOpenCount().get(); + int baseFailed = mg.kvTabletFailedCount().get(); + + ManualClock clock = new ManualClock(System.currentTimeMillis()); + KvTablet sentinel = createSentinel(); + KvTabletLazyLifecycle lifecycle = configureLifecycle(sentinel, clock); + + // After configureLazyOpen: LAZY +1 + assertThat(mg.kvTabletLazyCount().get() - baseLazy).isEqualTo(1); + assertThat(mg.kvTabletOpenCount().get() - baseOpen).isEqualTo(0); + assertThat(mg.kvTabletFailedCount().get() - baseFailed).isEqualTo(0); + + // Open: LAZY -1, OPEN +1 + try (KvTablet.Guard guard = sentinel.acquireGuard()) { + KvRecord record = recordFactory.ofRecord("k1".getBytes(), new Object[] {1, "a"}); + KvRecordBatch batch = batchFactory.ofRecords(Collections.singletonList(record)); + sentinel.putAsLeader(batch, null); + sentinel.flush(Long.MAX_VALUE, NOPErrorHandler.INSTANCE); + } + assertThat(mg.kvTabletLazyCount().get() - baseLazy).isEqualTo(0); + assertThat(mg.kvTabletOpenCount().get() - baseOpen).isEqualTo(1); + + // Release: OPEN -1, LAZY +1 + boolean released = sentinel.releaseKv(); + assertThat(released).isTrue(); + assertThat(mg.kvTabletLazyCount().get() - baseLazy).isEqualTo(1); + assertThat(mg.kvTabletOpenCount().get() - baseOpen).isEqualTo(0); + + // Open again: LAZY -1, OPEN +1 + try (KvTablet.Guard guard = sentinel.acquireGuard()) { + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.OPEN); + } + assertThat(mg.kvTabletLazyCount().get() - baseLazy).isEqualTo(0); + assertThat(mg.kvTabletOpenCount().get() - baseOpen).isEqualTo(1); + + // Drop: OPEN -1 + sentinel.dropKvLazy(); + assertThat(mg.kvTabletOpenCount().get() - baseOpen).isEqualTo(0); + assertThat(mg.kvTabletLazyCount().get() - baseLazy).isEqualTo(0); + assertThat(mg.kvTabletFailedCount().get() - baseFailed).isEqualTo(0); + } + + @Test + void testEpochFencingRejectsStaleOpenOnLeaderEpochChange() throws Exception { + ManualClock clock = new ManualClock(System.currentTimeMillis()); + KvTablet sentinel = createSentinel(); + KvTabletLazyLifecycle lifecycle = configureLifecycle(sentinel, clock); + + AtomicInteger leaderEpoch = new AtomicInteger(1); + lifecycle.setLeaderEpochSupplier(leaderEpoch::get); + + CountDownLatch openStarted = new CountDownLatch(1); + CountDownLatch allowOpenFinish = new CountDownLatch(1); + + lifecycle.setOpenCallback( + hasLocal -> { + openStarted.countDown(); + try { + allowOpenFinish.await(5, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + return kvManager.createKvTabletUnregistered( + physicalPath, + tableBucket, + logTablet, + KvFormat.COMPACTED, + new TestingSchemaGetter(new SchemaInfo(DATA1_SCHEMA_PK, SCHEMA_ID)), + new TableConfig(new Configuration()), + DEFAULT_COMPRESSION); + }); + + AtomicReference openFailure = new AtomicReference<>(); + Thread openThread = + new Thread( + () -> { + try (KvTablet.Guard guard = sentinel.acquireGuard()) { + // no-op + } catch (Throwable t) { + openFailure.set(t); + } + }, + "epoch-fence-leader"); + openThread.start(); + + assertThat(openStarted.await(5, TimeUnit.SECONDS)).isTrue(); + + // Change leader epoch while open is in progress + leaderEpoch.set(2); + allowOpenFinish.countDown(); + + openThread.join(TimeUnit.SECONDS.toMillis(5)); + assertThat(openThread.isAlive()).isFalse(); + + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.FAILED); + assertThat(openFailure.get()).isNotNull(); + } + + @Test + void testEpochFencingRejectsStaleOpenOnBucketEpochChange() throws Exception { + ManualClock clock = new ManualClock(System.currentTimeMillis()); + KvTablet sentinel = createSentinel(); + KvTabletLazyLifecycle lifecycle = configureLifecycle(sentinel, clock); + + AtomicInteger bucketEpoch = new AtomicInteger(1); + lifecycle.setBucketEpochSupplier(bucketEpoch::get); + + CountDownLatch openStarted = new CountDownLatch(1); + CountDownLatch allowOpenFinish = new CountDownLatch(1); + + lifecycle.setOpenCallback( + hasLocal -> { + openStarted.countDown(); + try { + allowOpenFinish.await(5, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + return kvManager.createKvTabletUnregistered( + physicalPath, + tableBucket, + logTablet, + KvFormat.COMPACTED, + new TestingSchemaGetter(new SchemaInfo(DATA1_SCHEMA_PK, SCHEMA_ID)), + new TableConfig(new Configuration()), + DEFAULT_COMPRESSION); + }); + + AtomicReference openFailure = new AtomicReference<>(); + Thread openThread = + new Thread( + () -> { + try (KvTablet.Guard guard = sentinel.acquireGuard()) { + // no-op + } catch (Throwable t) { + openFailure.set(t); + } + }, + "epoch-fence-bucket"); + openThread.start(); + + assertThat(openStarted.await(5, TimeUnit.SECONDS)).isTrue(); + + // Change bucket epoch while open is in progress + bucketEpoch.set(2); + allowOpenFinish.countDown(); + + openThread.join(TimeUnit.SECONDS.toMillis(5)); + assertThat(openThread.isAlive()).isFalse(); + + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.FAILED); + assertThat(openFailure.get()).isNotNull(); + } + + @Test + void testConcurrentAcquireGuardOnlyOpensOnce() throws Exception { + ManualClock clock = new ManualClock(System.currentTimeMillis()); + KvTablet sentinel = createSentinel(); + KvTabletLazyLifecycle lifecycle = configureLifecycle(sentinel, clock); + + AtomicInteger openCount = new AtomicInteger(0); + CountDownLatch openStarted = new CountDownLatch(1); + CountDownLatch allowOpenFinish = new CountDownLatch(1); + + lifecycle.setOpenCallback( + hasLocal -> { + openCount.incrementAndGet(); + openStarted.countDown(); + try { + allowOpenFinish.await(5, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + return kvManager.createKvTabletUnregistered( + physicalPath, + tableBucket, + logTablet, + KvFormat.COMPACTED, + new TestingSchemaGetter(new SchemaInfo(DATA1_SCHEMA_PK, SCHEMA_ID)), + new TableConfig(new Configuration()), + DEFAULT_COMPRESSION); + }); + + int threadCount = 5; + CountDownLatch barrier = new CountDownLatch(1); + AtomicInteger successCount = new AtomicInteger(0); + AtomicInteger failureCount = new AtomicInteger(0); + Thread[] threads = new Thread[threadCount]; + + for (int i = 0; i < threadCount; i++) { + threads[i] = + new Thread( + () -> { + try { + barrier.await(5, TimeUnit.SECONDS); + try (KvTablet.Guard guard = sentinel.acquireGuard()) { + successCount.incrementAndGet(); + } + } catch (Exception e) { + failureCount.incrementAndGet(); + } + }, + "concurrent-acquire-" + i); + threads[i].start(); + } + + // Release all threads at once + barrier.countDown(); + + // Wait for the opener thread to enter doSlowOpen + assertThat(openStarted.await(5, TimeUnit.SECONDS)).isTrue(); + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.OPENING); + + // Let the open complete + allowOpenFinish.countDown(); + + // Wait for all threads to finish + for (Thread t : threads) { + t.join(TimeUnit.SECONDS.toMillis(10)); + assertThat(t.isAlive()).isFalse(); + } + + assertThat(openCount.get()).isEqualTo(1); + assertThat(successCount.get()).isEqualTo(threadCount); + assertThat(failureCount.get()).isZero(); + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.OPEN); + } + + @Test + void testDropKvDuringOpeningFencesAndCloses() throws Exception { + ManualClock clock = new ManualClock(System.currentTimeMillis()); + KvTablet sentinel = createSentinel(); + KvTabletLazyLifecycle lifecycle = configureLifecycle(sentinel, clock); + + CountDownLatch openStarted = new CountDownLatch(1); + CountDownLatch allowOpenFinish = new CountDownLatch(1); + + lifecycle.setOpenCallback( + hasLocal -> { + openStarted.countDown(); + try { + allowOpenFinish.await(5, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + return kvManager.createKvTabletUnregistered( + physicalPath, + tableBucket, + logTablet, + KvFormat.COMPACTED, + new TestingSchemaGetter(new SchemaInfo(DATA1_SCHEMA_PK, SCHEMA_ID)), + new TableConfig(new Configuration()), + DEFAULT_COMPRESSION); + }); + + AtomicReference openFailure = new AtomicReference<>(); + Thread openThread = + new Thread( + () -> { + try (KvTablet.Guard guard = sentinel.acquireGuard()) { + // no-op + } catch (Throwable t) { + openFailure.set(t); + } + }, + "open-for-drop-fence"); + openThread.start(); + + assertThat(openStarted.await(5, TimeUnit.SECONDS)).isTrue(); + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.OPENING); + + FutureTask dropTask = + new FutureTask<>( + () -> { + sentinel.dropKvLazy(); + return null; + }); + Thread dropThread = new Thread(dropTask, "drop-during-opening"); + dropThread.start(); + + // Let the open finish — commitOpenResult should see generation mismatch + allowOpenFinish.countDown(); + + openThread.join(TimeUnit.SECONDS.toMillis(5)); + assertThat(openThread.isAlive()).isFalse(); + dropTask.get(5, TimeUnit.SECONDS); + + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.CLOSED); + assertThat(sentinel.getRocksDBKv()).isNull(); + assertThat(openFailure.get()).isNotNull(); + } + + @Test + void testReleaseDrainTimeoutRollsBackToOpen() throws Exception { + KvTablet sentinel = createSentinel(); + + // Use SystemClock so that real wall-clock time elapses for the drain timeout + KvTabletLazyLifecycle lifecycle = sentinel.getLifecycle(); + lifecycle.configureLazyOpen( + SystemClock.getInstance(), + kvManager.getOpenSemaphore(), + kvManager.getOpenTimeoutMs(), + kvManager.getFailedBackoffBaseMs(), + kvManager.getFailedBackoffMaxMs(), + 100L); // 100ms drain timeout + lifecycle.setLeaderEpochSupplier(() -> 1); + lifecycle.setBucketEpochSupplier(() -> 1); + lifecycle.setTabletDirSupplier(() -> kvManager.getTabletDirPath(physicalPath, tableBucket)); + lifecycle.setOpenCallback( + hasLocal -> + kvManager.createKvTabletUnregistered( + physicalPath, + tableBucket, + logTablet, + KvFormat.COMPACTED, + new TestingSchemaGetter(new SchemaInfo(DATA1_SCHEMA_PK, SCHEMA_ID)), + new TableConfig(new Configuration()), + DEFAULT_COMPRESSION)); + lifecycle.setCommitCallback(kv -> {}); + lifecycle.setReleaseCallback(kv -> {}); + lifecycle.setDropCallback(kv -> {}); + + // Acquire a guard and hold it to prevent drain from completing + KvTablet.Guard guard = sentinel.acquireGuard(); + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.OPEN); + + // Start releaseKv in a background thread — it should timeout because the guard is held + FutureTask releaseTask = new FutureTask<>(sentinel::releaseKv); + Thread releaseThread = new Thread(releaseTask, "release-drain-timeout"); + releaseThread.start(); + + // The release should timeout (100ms) and roll back to OPEN + boolean released = releaseTask.get(5, TimeUnit.SECONDS); + assertThat(released).isFalse(); + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.OPEN); + + // Close the guard and verify pins are drained + guard.close(); + assertThat(sentinel.getActivePins()).isZero(); + } + + @Test + void testReleaseCallbackFailureRollsBackToOpenAndTabletRemainsFunctional() throws Exception { + ManualClock clock = new ManualClock(System.currentTimeMillis()); + KvTablet sentinel = createSentinel(); + KvTabletLazyLifecycle lifecycle = configureLifecycle(sentinel, clock); + + // Open and write+flush data + try (KvTablet.Guard guard = sentinel.acquireGuard()) { + KvRecord record = recordFactory.ofRecord("k1".getBytes(), new Object[] {1, "a"}); + KvRecordBatch batch = batchFactory.ofRecords(Collections.singletonList(record)); + sentinel.putAsLeader(batch, null); + sentinel.flush(Long.MAX_VALUE, NOPErrorHandler.INSTANCE); + } + + // Set release callback to throw + lifecycle.setReleaseCallback( + kv -> { + throw new RuntimeException("simulated release callback failure"); + }); + + // releaseKv should return false due to callback failure + boolean released = sentinel.releaseKv(); + assertThat(released).isFalse(); + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.OPEN); + + // Verify tablet is still functional — acquireGuard and multiGet should work + try (KvTablet.Guard guard = sentinel.acquireGuard()) { + List values = sentinel.multiGet(Collections.singletonList("k1".getBytes())); + assertThat(values).hasSize(1); + byte[] expected = + ValueEncoder.encodeValue( + SCHEMA_ID, compactedRow(BASE_ROW_TYPE, new Object[] {1, "a"})); + assertThat(values.get(0)).isEqualTo(expected); + } + + // Fix the callback and verify subsequent release succeeds + lifecycle.setReleaseCallback(kv -> {}); + released = sentinel.releaseKv(); + assertThat(released).isTrue(); + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.LAZY); + } + + @Test + void testDropKvFromFailed() { + ManualClock clock = new ManualClock(System.currentTimeMillis()); + KvTablet sentinel = createSentinel(); + KvTabletLazyLifecycle lifecycle = configureLifecycle(sentinel, clock); + + // Override openCallback to throw + lifecycle.setOpenCallback( + hasLocal -> { + throw new RuntimeException("simulated open failure"); + }); + + // First acquireGuard should fail + assertThatThrownBy(sentinel::acquireGuard).isInstanceOf(RuntimeException.class); + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.FAILED); + + // Drop from FAILED state + sentinel.dropKvLazy(); + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.CLOSED); + } + + @Test + void testExponentialBackoffEscalatesAcrossMultipleFailures() throws Exception { + ManualClock clock = new ManualClock(System.currentTimeMillis()); + KvTablet sentinel = createSentinel(); + KvTabletLazyLifecycle lifecycle = configureLifecycle(sentinel, clock); + + // Override openCallback to throw + lifecycle.setOpenCallback( + hasLocal -> { + throw new RuntimeException("simulated open failure"); + }); + + // Failure 1: failureCount becomes 1, cooldown = 5000 * 2^1 = 10000ms + assertThatThrownBy(sentinel::acquireGuard).isInstanceOf(RuntimeException.class); + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.FAILED); + + // Advance 9s — still within 10s cooldown + clock.advanceTime(9, TimeUnit.SECONDS); + assertThatThrownBy(sentinel::acquireGuard) + .isInstanceOf(KvStorageException.class) + .hasMessageContaining("cooldown"); + + // Advance 2s more (total 11s, past 10s cooldown) — retry and fail again + clock.advanceTime(2, TimeUnit.SECONDS); + assertThatThrownBy(sentinel::acquireGuard).isInstanceOf(RuntimeException.class); + + // Failure 2: failureCount becomes 2, cooldown = 5000 * 2^2 = 20000ms + // Advance 19s — still within 20s cooldown + clock.advanceTime(19, TimeUnit.SECONDS); + assertThatThrownBy(sentinel::acquireGuard) + .isInstanceOf(KvStorageException.class) + .hasMessageContaining("cooldown"); + + // Advance 2s more (total 21s, past 20s cooldown) — retry and fail again + clock.advanceTime(2, TimeUnit.SECONDS); + assertThatThrownBy(sentinel::acquireGuard).isInstanceOf(RuntimeException.class); + + // Failure 3: failureCount becomes 3, cooldown = 5000 * 2^3 = 40000ms + // Advance 39s — still within 40s cooldown + clock.advanceTime(39, TimeUnit.SECONDS); + assertThatThrownBy(sentinel::acquireGuard) + .isInstanceOf(KvStorageException.class) + .hasMessageContaining("cooldown"); + + // Advance 2s more (total 41s, past 40s cooldown) — fix callback and verify recovery + clock.advanceTime(2, TimeUnit.SECONDS); + lifecycle.setOpenCallback( + hasLocal -> + kvManager.createKvTabletUnregistered( + physicalPath, + tableBucket, + logTablet, + KvFormat.COMPACTED, + new TestingSchemaGetter(new SchemaInfo(DATA1_SCHEMA_PK, SCHEMA_ID)), + new TableConfig(new Configuration()), + DEFAULT_COMPRESSION)); + try (KvTablet.Guard guard = sentinel.acquireGuard()) { + assertThat(lifecycle.getLazyState()).isEqualTo(KvTabletLazyLifecycle.LazyState.OPEN); + } + } + + @Test + void testGuardDoubleCloseDoesNotDecrementPinsBelowZero() { + ManualClock clock = new ManualClock(System.currentTimeMillis()); + KvTablet sentinel = createSentinel(); + configureLifecycle(sentinel, clock); + + KvTablet.Guard guard = sentinel.acquireGuard(); + assertThat(sentinel.getActivePins()).isEqualTo(1); + + guard.close(); + assertThat(sentinel.getActivePins()).isZero(); + + // Second close should be idempotent — pins must not go negative + guard.close(); + assertThat(sentinel.getActivePins()).isZero(); + } + + @Test + void testAcquireGuardOnClosedTabletThrows() { + ManualClock clock = new ManualClock(System.currentTimeMillis()); + KvTablet sentinel = createSentinel(); + configureLifecycle(sentinel, clock); + + // Open the tablet + try (KvTablet.Guard guard = sentinel.acquireGuard()) { + assertThat(sentinel.getLifecycle().getLazyState()) + .isEqualTo(KvTabletLazyLifecycle.LazyState.OPEN); + } + + // Drop transitions to CLOSED + sentinel.dropKvLazy(); + assertThat(sentinel.getLifecycle().getLazyState()) + .isEqualTo(KvTabletLazyLifecycle.LazyState.CLOSED); + + // acquireGuard on a CLOSED tablet must throw + assertThatThrownBy(sentinel::acquireGuard) + .isInstanceOf(KvStorageException.class) + .hasMessageContaining("closed"); + } + + @Test + void testGaugeUpdatesOnFailedTransition() { + TabletServerMetricGroup mg = TestingMetricGroups.TABLET_SERVER_METRICS; + int baseLazy = mg.kvTabletLazyCount().get(); + int baseOpen = mg.kvTabletOpenCount().get(); + int baseFailed = mg.kvTabletFailedCount().get(); + + ManualClock clock = new ManualClock(System.currentTimeMillis()); + KvTablet sentinel = createSentinel(); + KvTabletLazyLifecycle lifecycle = configureLifecycle(sentinel, clock); + + // After configureLazyOpen: LAZY +1 + assertThat(mg.kvTabletLazyCount().get() - baseLazy).isEqualTo(1); + + // Override openCallback to throw + lifecycle.setOpenCallback( + hasLocal -> { + throw new RuntimeException("simulated open failure"); + }); + + // acquireGuard should fail and transition to FAILED + assertThatThrownBy(sentinel::acquireGuard).isInstanceOf(RuntimeException.class); + + // LAZY back to baseline, FAILED +1, OPEN unchanged + assertThat(mg.kvTabletLazyCount().get() - baseLazy).isEqualTo(0); + assertThat(mg.kvTabletFailedCount().get() - baseFailed).isEqualTo(1); + assertThat(mg.kvTabletOpenCount().get() - baseOpen).isEqualTo(0); + + // Drop from FAILED state + sentinel.dropKvLazy(); + + // All gauges back to baseline + assertThat(mg.kvTabletLazyCount().get() - baseLazy).isEqualTo(0); + assertThat(mg.kvTabletOpenCount().get() - baseOpen).isEqualTo(0); + assertThat(mg.kvTabletFailedCount().get() - baseFailed).isEqualTo(0); + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaLazyOpenTest.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaLazyOpenTest.java new file mode 100644 index 0000000000..7a8a5d4f51 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaLazyOpenTest.java @@ -0,0 +1,321 @@ +/* + * 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.replica; + +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.fs.FsPath; +import org.apache.fluss.metadata.TableBucket; +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.row.encode.ValueEncoder; +import org.apache.fluss.rpc.protocol.MergeMode; +import org.apache.fluss.server.entity.NotifyLeaderAndIsrData; +import org.apache.fluss.server.kv.KvTablet; +import org.apache.fluss.server.kv.snapshot.CompletedSnapshot; +import org.apache.fluss.server.kv.snapshot.KvFileHandle; +import org.apache.fluss.server.kv.snapshot.KvFileHandleAndLocalPath; +import org.apache.fluss.server.kv.snapshot.KvSnapshotHandle; +import org.apache.fluss.server.log.LogAppendInfo; +import org.apache.fluss.server.zk.NOPErrorHandler; +import org.apache.fluss.server.zk.data.LeaderAndIsr; + +import org.junit.jupiter.api.Test; + +import java.io.File; +import java.time.Duration; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.apache.fluss.record.TestData.DATA1_PHYSICAL_TABLE_PATH_PK; +import static org.apache.fluss.record.TestData.DATA1_ROW_TYPE; +import static org.apache.fluss.record.TestData.DATA1_TABLE_ID_PK; +import static org.apache.fluss.record.TestData.DEFAULT_SCHEMA_ID; +import static org.apache.fluss.testutils.DataTestUtils.compactedRow; +import static org.apache.fluss.utils.FileUtils.deleteDirectory; +import static org.apache.fluss.utils.Preconditions.checkNotNull; +import static org.assertj.core.api.Assertions.assertThat; + +/** Integration tests for lazy-open behavior in {@link Replica}. */ +final class ReplicaLazyOpenTest extends ReplicaTestBase { + + @Override + protected Configuration getServerConf() { + Configuration conf = super.getServerConf(); + conf.set(ConfigOptions.KV_LAZY_OPEN_ENABLED, true); + conf.set(ConfigOptions.KV_LAZY_OPEN_IDLE_TIMEOUT, Duration.ofHours(1)); + return conf; + } + + @Test + void testPutRecordsToLeaderTriggersLazyOpen() throws Exception { + Replica replica = makeLazyLeaderReplica(); + KvTablet kvTablet = checkNotNull(replica.getKvTablet()); + assertThat(kvTablet.isLazyOpen()).isFalse(); + assertThat(kvTablet.getRocksDBKv()).isNull(); + + LogAppendInfo appendInfo = putRecordsToLeaderAndFlush(replica, records("k1", 1, "a")); + + assertThat(appendInfo.lastOffset()).isEqualTo(0L); + assertThat(kvTablet.isLazyOpen()).isTrue(); + assertThat(kvTablet.getRocksDBKv()).isNotNull(); + assertThat(replica.lookups(Collections.singletonList("k1".getBytes()))) + .containsExactly(valueBytes(1, "a")); + } + + @Test + void testLookupsReopenReleasedLazyTablet() throws Exception { + Replica replica = makeLazyLeaderReplica(); + putRecordsToLeaderAndFlush(replica, records("k1", 1, "a", "k2", 2, "b")); + releaseToLazy(replica); + + List values = replica.lookups(Arrays.asList("k1".getBytes(), "k2".getBytes())); + + assertThat(values).containsExactly(valueBytes(1, "a"), valueBytes(2, "b")); + assertThat(checkNotNull(replica.getKvTablet()).isLazyOpen()).isTrue(); + } + + @Test + void testPrefixLookupReopensReleasedLazyTablet() throws Exception { + Replica replica = makeLazyLeaderReplica(); + putRecordsToLeaderAndFlush(replica, records("k1", 1, "a", "k2", 2, "b")); + releaseToLazy(replica); + + List values = replica.prefixLookup("k".getBytes()); + + assertThat(values).containsExactly(valueBytes(1, "a"), valueBytes(2, "b")); + assertThat(checkNotNull(replica.getKvTablet()).isLazyOpen()).isTrue(); + } + + @Test + void testLimitScanReopensReleasedLazyTablet() throws Exception { + Replica replica = makeLazyLeaderReplica(); + putRecordsToLeaderAndFlush(replica, records("k1", 1, "a", "k2", 2, "b")); + releaseToLazy(replica); + + DefaultValueRecordBatch.Builder builder = DefaultValueRecordBatch.builder(); + builder.append(DEFAULT_SCHEMA_ID, compactedRow(DATA1_ROW_TYPE, new Object[] {1, "a"})); + builder.append(DEFAULT_SCHEMA_ID, compactedRow(DATA1_ROW_TYPE, new Object[] {2, "b"})); + + DefaultValueRecordBatch values = replica.limitKvScan(2); + + assertThat(values).isEqualTo(builder.build()); + assertThat(checkNotNull(replica.getKvTablet()).isLazyOpen()).isTrue(); + } + + @Test + void testGetRowCountReturnsCachedValueWithoutReopening() throws Exception { + Replica replica = makeLazyLeaderReplica(); + putRecordsToLeaderAndFlush(replica, records("k1", 1, "a", "k2", 2, "b")); + releaseToLazy(replica); + + long rowCount = replica.getRowCount(); + + assertThat(rowCount).isEqualTo(2L); + assertThat(checkNotNull(replica.getKvTablet()).isLazyOpen()).isFalse(); + } + + @Test + void testLatestKvSnapshotSizeFallsBackToCommittedSnapshotWhenReleased() throws Exception { + TableBucket tableBucket = new TableBucket(DATA1_TABLE_ID_PK, 1); + Replica replica = makeLazyLeaderReplicaWithSharedSnapshotStore(tableBucket); + putRecordsToLeaderAndFlush(replica, records("k1", 1, "a", "k2", 2, "b")); + + // Arbitrary non-zero size to verify fallback to committed snapshot + long expectedSnapshotSize = 321L; + KvSnapshotHandle snapshotHandle = + new KvSnapshotHandle( + Collections.singletonList( + KvFileHandleAndLocalPath.of( + new KvFileHandle( + new FsPath( + tempDir.getAbsolutePath(), + "snapshot-1.sst") + .toString(), + expectedSnapshotSize), + "snapshot-1.sst")), + Collections.emptyList(), + expectedSnapshotSize); + CompletedSnapshot completedSnapshot = + new CompletedSnapshot( + tableBucket, + 1L, + new FsPath(tempDir.getAbsolutePath(), "snapshot-1"), + snapshotHandle, + replica.getLocalLogEndOffset(), + 2L, + null); + snapshotReporter.commitKvSnapshot(completedSnapshot, 0, 0); + + releaseToLazy(replica); + + assertThat(replica.getKvSnapshotManager()).isNull(); + assertThat(replica.getLatestKvSnapshotSize()) + .isEqualTo(completedSnapshot.getSnapshotSize()); + assertThat(replica.logicalStorageKvSize()).isEqualTo(completedSnapshot.getSnapshotSize()); + } + + @Test + void testMakeFollowerDropsLazyTablet() throws Exception { + Replica replica = makeLazyLeaderReplica(); + KvTablet kvTablet = checkNotNull(replica.getKvTablet()); + assertThat(kvTablet.isLazyOpen()).isFalse(); + assertThat(kvTablet.getRocksDBKv()).isNull(); + + replica.makeFollower( + notifyLeaderAndIsr( + new TableBucket(DATA1_TABLE_ID_PK, 1), + Collections.singletonList(TABLET_SERVER_ID), + Collections.singletonList(TABLET_SERVER_ID), + 1)); + + assertThat(replica.getKvTablet()).isNull(); + } + + @Test + void testMakeFollowerDropsOpenLazyTablet() throws Exception { + Replica replica = makeLazyLeaderReplica(); + putRecordsToLeaderAndFlush(replica, records("k1", 1, "a")); + KvTablet kvTablet = checkNotNull(replica.getKvTablet()); + assertThat(kvTablet.isLazyOpen()).isTrue(); + + replica.makeFollower( + notifyLeaderAndIsr( + new TableBucket(DATA1_TABLE_ID_PK, 1), + Collections.singletonList(TABLET_SERVER_ID), + Collections.singletonList(TABLET_SERVER_ID), + 1)); + + assertThat(replica.getKvTablet()).isNull(); + } + + @Test + void testReopenFallsBackToFullInitWhenLocalDataCorrupted() throws Exception { + Replica replica = makeLazyLeaderReplica(); + putRecordsToLeaderAndFlush(replica, records("k1", 1, "a", "k2", 2, "b")); + + // Get the tablet directory path before release + KvTablet kvTablet = checkNotNull(replica.getKvTablet()); + File tabletDir = + kvManager.getTabletDirPath( + DATA1_PHYSICAL_TABLE_PATH_PK, new TableBucket(DATA1_TABLE_ID_PK, 1)); + + releaseToLazy(replica); + + // Corrupt the local directory by deleting it + assertThat(tabletDir).exists(); + deleteDirectory(tabletDir); + assertThat(tabletDir).doesNotExist(); + + // Trigger reopen — should fall back to full init (no snapshot = fresh tablet) + List values = replica.lookups(Collections.singletonList("k1".getBytes())); + + // Tablet should be open and functional after fallback + assertThat(checkNotNull(replica.getKvTablet()).isLazyOpen()).isTrue(); + // Data is lost (no snapshot available), but the system recovered gracefully + assertThat(values.get(0)).isNull(); + + // Verify the tablet is fully functional — can write new data + putRecordsToLeaderAndFlush(replica, records("k3", 3, "c")); + assertThat(replica.lookups(Collections.singletonList("k3".getBytes()))) + .containsExactly(valueBytes(3, "c")); + } + + private Replica makeLazyLeaderReplica() throws Exception { + Replica replica = + makeKvReplica(DATA1_PHYSICAL_TABLE_PATH_PK, new TableBucket(DATA1_TABLE_ID_PK, 1)); + makeLeaderReplica(replica); + return replica; + } + + private Replica makeLazyLeaderReplicaWithSharedSnapshotStore(TableBucket tableBucket) + throws Exception { + Replica replica = + makeKvReplica( + DATA1_PHYSICAL_TABLE_PATH_PK, + tableBucket, + new TestSnapshotContext( + conf.getString(ConfigOptions.REMOTE_DATA_DIR), snapshotReporter)); + makeLeaderReplica(replica); + return replica; + } + + private void makeLeaderReplica(Replica replica) throws Exception { + replica.makeLeader( + notifyLeaderAndIsr( + new TableBucket(DATA1_TABLE_ID_PK, 1), + Collections.singletonList(TABLET_SERVER_ID), + Collections.singletonList(TABLET_SERVER_ID), + 0)); + } + + private NotifyLeaderAndIsrData notifyLeaderAndIsr( + TableBucket tableBucket, List replicas, List isr, int leaderEpoch) { + return new NotifyLeaderAndIsrData( + DATA1_PHYSICAL_TABLE_PATH_PK, + tableBucket, + replicas, + new LeaderAndIsr( + TABLET_SERVER_ID, + leaderEpoch, + isr, + org.apache.fluss.server.coordinator.CoordinatorContext + .INITIAL_COORDINATOR_EPOCH, + leaderEpoch)); + } + + private LogAppendInfo putRecordsToLeaderAndFlush(Replica replica, KvRecordBatch kvRecords) + throws Exception { + LogAppendInfo appendInfo = + replica.putRecordsToLeader(kvRecords, null, MergeMode.DEFAULT, 0); + checkNotNull(replica.getKvTablet()) + .flush(replica.getLocalLogEndOffset(), NOPErrorHandler.INSTANCE); + return appendInfo; + } + + private void releaseToLazy(Replica replica) { + KvTablet kvTablet = checkNotNull(replica.getKvTablet()); + assertThat(kvTablet.releaseKv()).isTrue(); + assertThat(kvTablet.isLazyOpen()).isFalse(); + assertThat(kvTablet.getRocksDBKv()).isNull(); + } + + private KvRecordBatch records(Object... keyValueTriples) throws Exception { + KvRecordTestUtils.KvRecordFactory recordFactory = + KvRecordTestUtils.KvRecordFactory.of(DATA1_ROW_TYPE); + KvRecordTestUtils.KvRecordBatchFactory batchFactory = + KvRecordTestUtils.KvRecordBatchFactory.of(DEFAULT_SCHEMA_ID); + + KvRecord[] records = new KvRecord[keyValueTriples.length / 3]; + for (int i = 0; i < keyValueTriples.length; i += 3) { + records[i / 3] = + recordFactory.ofRecord( + (String) keyValueTriples[i], + new Object[] {keyValueTriples[i + 1], keyValueTriples[i + 2]}); + } + return batchFactory.ofRecords(records); + } + + private byte[] valueBytes(int id, String value) { + return ValueEncoder.encodeValue( + DEFAULT_SCHEMA_ID, compactedRow(DATA1_ROW_TYPE, new Object[] {id, value})); + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTest.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTest.java index 04331fb6ab..e35dbdedf8 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTest.java @@ -19,6 +19,7 @@ import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.exception.OutOfOrderSequenceException; +import org.apache.fluss.fs.FsPath; import org.apache.fluss.metadata.LogFormat; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.SchemaGetter; @@ -36,16 +37,21 @@ import org.apache.fluss.server.entity.NotifyLeaderAndIsrData; import org.apache.fluss.server.kv.KvTablet; import org.apache.fluss.server.kv.snapshot.CompletedSnapshot; +import org.apache.fluss.server.kv.snapshot.KvSnapshotDataDownloader; +import org.apache.fluss.server.kv.snapshot.KvSnapshotDownloadSpec; +import org.apache.fluss.server.kv.snapshot.KvSnapshotHandle; import org.apache.fluss.server.kv.snapshot.TestingCompletedKvSnapshotCommitter; import org.apache.fluss.server.log.FetchParams; import org.apache.fluss.server.log.LogAppendInfo; import org.apache.fluss.server.log.LogReadInfo; +import org.apache.fluss.server.log.LogTablet; 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.testutils.DataTestUtils; import org.apache.fluss.testutils.common.ManuallyTriggeredScheduledExecutorService; import org.apache.fluss.types.RowType; +import org.apache.fluss.utils.function.FunctionWithException; import org.apache.fluss.utils.types.Tuple2; import org.junit.jupiter.api.Test; @@ -894,6 +900,85 @@ private void verifyGetKeyValues( assertThat(kvTablet.multiGet(keys)).containsExactlyElementsOf(expectValues); } + @Test + void testGetRowCountPkTableWithNullKvTablet() throws Exception { + // Create a PK table replica but do NOT make it leader, so kvTablet stays null. + TableBucket tableBucket = new TableBucket(DATA1_TABLE_ID_PK, 1); + Replica kvReplica = makeKvReplica(DATA1_PHYSICAL_TABLE_PATH_PK, tableBucket); + + // Verify preconditions: this is a PK table and kvTablet is null. + assertThat(kvReplica.isKvTable()).isTrue(); + assertThat(kvReplica.getKvTablet()).isNull(); + + // Append some log records so logTablet has a non-zero row count. + LogTablet logTablet = kvReplica.getLogTablet(); + MemoryLogRecords records = genMemoryLogRecordsByObject(DATA1); + logTablet.appendAsLeader(records); + logTablet.updateHighWatermark(logTablet.localLogEndOffset()); + assertThat(logTablet.getRowCount()).isGreaterThan(0); + + // getRowCount() should return 0 for a PK table when kvTablet is null, + // NOT the logTablet row count. + assertThat(kvReplica.getRowCount()).isEqualTo(0L); + } + + @Test + void testCreateKvRollbackOnAllRetriesFailed() throws Exception { + // Create a SnapshotContext whose getSnapshotDataDownloader always throws. + // This forces initKvTablet() to fail on every retry attempt because the + // snapshot download will always fail. + TableBucket tableBucket = new TableBucket(DATA1_TABLE_ID_PK, 1); + TestSnapshotContext failingSnapshotContext = + new TestSnapshotContext(conf.getString(ConfigOptions.REMOTE_DATA_DIR)) { + @Override + public FunctionWithException + getLatestCompletedSnapshotProvider() { + // Return a provider that always returns a fake snapshot, + // so initKvTablet takes the snapshot-restore path + return tb -> + new CompletedSnapshot( + tb, + 1L, + new FsPath("file:///non-existent-path/snapshot-1"), + new KvSnapshotHandle( + Collections.emptyList(), + Collections.emptyList(), + 0), + 0L, + null, + null); + } + + @Override + public KvSnapshotDataDownloader getSnapshotDataDownloader() { + // Return a downloader that always throws to simulate download failure + return new KvSnapshotDataDownloader( + java.util.concurrent.Executors.newSingleThreadExecutor()) { + @Override + public void transferAllDataToDirectory( + KvSnapshotDownloadSpec kvSnapshotDownloadSpec, + org.apache.fluss.utils.CloseableRegistry closeableRegistry) + throws Exception { + throw new IOException("Simulated snapshot download failure"); + } + }; + } + }; + + Replica kvReplica = + makeKvReplica(DATA1_PHYSICAL_TABLE_PATH_PK, tableBucket, failingSnapshotContext); + + // Before the fix, makeLeader would NPE at checkNotNull(kvTablet) in + // startPeriodicKvSnapshot. After the fix, it should gracefully handle the failure. + makeKvReplicaAsLeader(kvReplica); + + // kvTablet should be null since all retries failed + assertThat(kvReplica.getKvTablet()).isNull(); + + // KvManager should not have a half-initialized entry for this bucket + assertThat(kvManager.getKv(tableBucket)).isEmpty(); + } + /** A scheduledExecutorService that will execute the scheduled task immediately. */ private static class ImmediateTriggeredScheduledExecutorService extends ManuallyTriggeredScheduledExecutorService { diff --git a/website/docs/maintenance/observability/monitor-metrics.md b/website/docs/maintenance/observability/monitor-metrics.md index ade4be9c61..efe45d96b4 100644 --- a/website/docs/maintenance/observability/monitor-metrics.md +++ b/website/docs/maintenance/observability/monitor-metrics.md @@ -564,6 +564,22 @@ Some metrics might not be exposed when using other JVM implementations (e.g. IBM The number of kv pre-write buffer truncate due to the error happened when writing cdc to log per second. Meter + + kvLazyOpen + kvTabletOpenCount + The number of KvTablets currently in OPEN state (RocksDB loaded). + Gauge + + + kvTabletLazyCount + The number of KvTablets currently in LAZY state (RocksDB not loaded). + Gauge + + + kvTabletFailedCount + The number of KvTablets currently in FAILED state (open attempt failed, in backoff cooldown). + Gauge + logicalStorage logSize