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

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -515,6 +515,51 @@ public class ConfigOptions {
+ WRITER_ID_EXPIRATION_TIME.key()
+ " passing. The default value is 10 minutes.");

public static final ConfigOption<Duration> KV_SCANNER_TTL =
key("kv.scanner.ttl")
.durationType()
.defaultValue(Duration.ofMinutes(10))
.withDescription(
"The time-to-live for an idle KV scanner session on the server. A scanner that has not "
+ "received a request within this duration will be automatically expired and its "
+ "resources released. The default value is 10 minutes.");

public static final ConfigOption<Duration> KV_SCANNER_EXPIRATION_INTERVAL =
key("kv.scanner.expiration-interval")
.durationType()
.defaultValue(Duration.ofSeconds(30))
.withDescription(
"How often the TTL evictor runs to close idle scanner sessions. "
+ "The default value is 30 seconds.");

public static final ConfigOption<Integer> KV_SCANNER_MAX_PER_BUCKET =
key("kv.scanner.max-per-bucket")
.intType()
.defaultValue(8)
.withDescription(
"Maximum number of concurrent scanner sessions per bucket. "
+ "Exceeding this limit returns TOO_MANY_SCANNERS. "
+ "The default value is 8.");

public static final ConfigOption<Integer> KV_SCANNER_MAX_PER_SERVER =
key("kv.scanner.max-per-server")
.intType()
.defaultValue(200)
.withDescription(
"Maximum number of concurrent scanner sessions per tablet server. "
+ "Exceeding this limit returns TOO_MANY_SCANNERS. "
+ "The default value is 200.");

public static final ConfigOption<MemorySize> KV_SCANNER_MAX_BATCH_SIZE =
key("kv.scanner.max-batch-size")
.memoryType()
.defaultValue(MemorySize.parse("10mb"))
.withDescription(
"Server-side cap on the per-batch payload size for KV full-scan responses. "
+ "The effective batch size is min(client-requested batch_size_bytes, "
+ "this value). Protects the tablet server from out-of-memory if a "
+ "client passes an excessively large batch size. The default value is 10mb.");

public static final ConfigOption<Integer> TABLET_SERVER_CONTROLLED_SHUTDOWN_MAX_RETRIES =
key("tablet-server.controlled-shutdown.max-retries")
.intType()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -278,6 +278,11 @@ public DefaultValueRecordBatch build() throws IOException {
return DefaultValueRecordBatch.pointToMemory(segment, 0);
}

/** Returns the current serialized size of the batch, including the record-batch header. */
public int sizeInBytes() {
return sizeInBytes;
}

private void writeBatchHeader() throws IOException {
outputView.setPosition(0);
// update header
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,8 @@
import org.apache.fluss.server.kv.rocksdb.RocksDBStatistics;
import org.apache.fluss.server.kv.rowmerger.DefaultRowMerger;
import org.apache.fluss.server.kv.rowmerger.RowMerger;
import org.apache.fluss.server.kv.scan.OpenScanResult;
import org.apache.fluss.server.kv.scan.ScannerContext;
import org.apache.fluss.server.kv.snapshot.KvFileHandleAndLocalPath;
import org.apache.fluss.server.kv.snapshot.KvSnapshotDataUploader;
import org.apache.fluss.server.kv.snapshot.RocksIncrementalSnapshot;
Expand All @@ -70,12 +72,17 @@
import org.apache.fluss.server.log.LogTablet;
import org.apache.fluss.server.metrics.group.TabletServerMetricGroup;
import org.apache.fluss.server.utils.FatalErrorHandler;
import org.apache.fluss.server.utils.ResourceGuard;
import org.apache.fluss.shaded.arrow.org.apache.arrow.memory.BufferAllocator;
import org.apache.fluss.types.RowType;
import org.apache.fluss.utils.BytesUtils;
import org.apache.fluss.utils.FileUtils;
import org.apache.fluss.utils.IOUtils;

import org.rocksdb.RateLimiter;
import org.rocksdb.ReadOptions;
import org.rocksdb.RocksIterator;
import org.rocksdb.Snapshot;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -757,6 +764,74 @@ public List<byte[]> limitScan(int limit) throws IOException {
});
}

/**
* Opens a new full-scan session under the {@code kvLock} read lock. Returns an empty-bucket
* result (context = {@code null}, all RocksDB resources released internally) when the bucket
* has no rows. The returned {@link ScannerContext} is unregistered; the caller owns
* registration and close.
*
* @param limit row-count cap across all batches ({@code ≤ 0} means unlimited)
* @throws IOException if RocksDB is shutting down
*/
Comment thread
polyzos marked this conversation as resolved.
public OpenScanResult openScan(String scannerId, long limit, long initialAccessTimeMs)
throws IOException {
return inReadLock(
kvLock,
() -> {
rocksDBKv.checkIfRocksDBClosed();
ResourceGuard.Lease lease = rocksDBKv.getResourceGuard().acquireResource();
Snapshot snapshot = null;
ReadOptions readOptions = null;
RocksIterator iterator = null;
boolean success = false;
try {
snapshot = rocksDBKv.getDb().getSnapshot();
// Capture under kvLock so the offset matches the data visible through
// the snapshot.
long capturedLogOffset = flushedLogOffset;
readOptions = new ReadOptions().setSnapshot(snapshot);
iterator =
rocksDBKv
.getDb()
.newIterator(
rocksDBKv.getDefaultColumnFamilyHandle(),
readOptions);
iterator.seekToFirst();
if (!iterator.isValid()) {
return new OpenScanResult(null, capturedLogOffset);
}
Comment thread
polyzos marked this conversation as resolved.
ScannerContext context =
new ScannerContext(
scannerId,
tableBucket,
rocksDBKv,
iterator,
readOptions,
snapshot,
lease,
limit,
capturedLogOffset,
initialAccessTimeMs);
success = true;
return new OpenScanResult(context, capturedLogOffset);
} finally {
if (!success) {
IOUtils.closeQuietly(iterator);
IOUtils.closeQuietly(readOptions);
if (snapshot != null) {
try {
rocksDBKv.getDb().releaseSnapshot(snapshot);
} catch (Throwable t) {
LOG.warn("Error releasing RocksDB snapshot.", t);
}
IOUtils.closeQuietly(snapshot);
}
IOUtils.closeQuietly(lease);
}
}
});
}

public KvBatchWriter createKvBatchWriter() {
return rocksDBKv.newWriteBatch(
writeBatchSize,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,49 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.fluss.server.kv.scan;

import org.apache.fluss.annotation.Internal;

import javax.annotation.Nullable;

/**
* Result of opening a KV full-scan session: an optional {@link ScannerContext} plus the log offset
* captured at snapshot time. The offset must reach the client even on the empty-bucket fast path so
* a snapshot-to-log handoff cannot miss or duplicate records.
*/
@Internal
public final class OpenScanResult {

@Nullable private final ScannerContext context;
private final long logOffset;

public OpenScanResult(@Nullable ScannerContext context, long logOffset) {
this.context = context;
this.logOffset = logOffset;
}

/** {@code null} if the bucket was empty at snapshot time. */
@Nullable
public ScannerContext getContext() {
return context;
}

public long getLogOffset() {
return logOffset;
}
}
Loading