Skip to content
Open
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 @@ -1569,6 +1569,25 @@ public class ConfigOptions {
"The max size of the consumed memory for RocksDB batch write, "
+ "will flush just based on item count if this config set to 0.");

public static final ConfigOption<MemorySize> KV_PRE_WRITE_BUFFER_MEMORY_POOL_SIZE =
key("kv.preWriteBuffer.memory-pool-size")
.memoryType()
.defaultValue(MemorySize.parse("2gb"))
.withDescription(
"The max size of the off-heap memory pool for pre-write buffer. "
+ "All buckets will share the same buffer pool in one tabletServer. "
+ "If the used memory exceeds this config, PreWriteBufferFullException will be thrown. "
+ "The default value is 2GB.");

public static final ConfigOption<MemorySize> KV_PRE_WRITE_BUFFER_MEMORY_POOL_SIZ_PER_BUCKET =
key("kv.preWriteBuffer.memory-pool-size-per-bucket")
.memoryType()
.defaultValue(MemorySize.parse("200mb"))
.withDescription(
"The max size of the off-heap memory buffer pool for pre-write buffer of each table bucket. "
+ "If there is a table bucket who use the buffer size exceeds this config, "
+ "PreWriteBufferFullException will be thrown. The default value is 200MB.");

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

package org.apache.fluss.exception;

/** Pre-write buffer is full. */
public class PreWriteBufferFullException extends RetriableException {
private static final long serialVersionUID = 1L;

public PreWriteBufferFullException(String message) {
super(message);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -124,6 +124,12 @@ public class MetricNames {
"preWriteBufferTruncateAsDuplicatedPerSecond";
public static final String KV_PRE_WRITE_BUFFER_TRUNCATE_AS_ERROR_RATE =
"preWriteBufferTruncateAsErrorPerSecond";
public static final String KV_PRE_WRITE_BUFFER_MEMORY_POOL_MAX_SIZE =
"preWriteBufferMemoryPoolMaxSize";
public static final String KV_PRE_WRITE_BUFFER_MEMORY_POOL_MAX_SIZE_PER_BUCKET =
"preWriteBufferMemoryPoolMaxSizePerBucket";
public static final String KV_PRE_WRITE_BUFFER_MEMORY_POOL_USAGE_SIZE =
"preWriteBufferMemoryPoolUsageSize";

// --------------------------------------------------------------------------------------------
// metrics for table bucket
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,7 @@
import org.apache.fluss.exception.OutOfOrderSequenceException;
import org.apache.fluss.exception.PartitionAlreadyExistsException;
import org.apache.fluss.exception.PartitionNotExistException;
import org.apache.fluss.exception.PreWriteBufferFullException;
import org.apache.fluss.exception.RecordTooLargeException;
import org.apache.fluss.exception.RetriableAuthenticationException;
import org.apache.fluss.exception.SchemaNotExistException;
Expand Down Expand Up @@ -228,7 +229,9 @@ public enum Errors {
INVALID_ALTER_TABLE_EXCEPTION(
56, "The alter table is invalid.", InvalidAlterTableException::new),
DELETION_DISABLED_EXCEPTION(
57, "Deletion operations are disabled on this table.", DeletionDisabledException::new);
57, "Deletion operations are disabled on this table.", DeletionDisabledException::new),
PRE_WRITE_BUFFER_FULL_EXCEPTION(
58, "The pre-write buffer is full.", PreWriteBufferFullException::new);

private static final Logger LOG = LoggerFactory.getLogger(Errors.class);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,9 @@
import org.apache.fluss.metadata.TableBucket;
import org.apache.fluss.metadata.TableInfo;
import org.apache.fluss.metadata.TablePath;
import org.apache.fluss.metrics.MetricNames;
import org.apache.fluss.server.TabletManagerBase;
import org.apache.fluss.server.kv.prewrite.KvPreWriteBufferMemoryPool;
import org.apache.fluss.server.kv.rowmerger.RowMerger;
import org.apache.fluss.server.log.LogManager;
import org.apache.fluss.server.log.LogTablet;
Expand Down Expand Up @@ -76,6 +78,8 @@ public final class KvManager extends TabletManagerBase {

private final Map<TableBucket, KvTablet> currentKvs = MapUtils.newConcurrentHashMap();

private final KvPreWriteBufferMemoryPool kvPreWriteBufferMemoryPool;

/**
* For arrow log format. The buffer allocator to allocate memory for arrow write batch of
* changelog records.
Expand Down Expand Up @@ -104,7 +108,9 @@ private KvManager(
this.zkClient = zkClient;
this.remoteKvDir = FlussPaths.remoteKvDir(conf);
this.remoteFileSystem = remoteKvDir.getFileSystem();
this.kvPreWriteBufferMemoryPool = new KvPreWriteBufferMemoryPool(conf);
this.serverMetricGroup = tabletServerMetricGroup;
registerMetrics();
}

public static KvManager create(
Expand All @@ -124,6 +130,18 @@ public static KvManager create(
tabletServerMetricGroup);
}

private void registerMetrics() {
serverMetricGroup.gauge(
MetricNames.KV_PRE_WRITE_BUFFER_MEMORY_POOL_USAGE_SIZE,
kvPreWriteBufferMemoryPool::getTotalUsed);
serverMetricGroup.gauge(
MetricNames.KV_PRE_WRITE_BUFFER_MEMORY_POOL_MAX_SIZE_PER_BUCKET,
kvPreWriteBufferMemoryPool::getMaxMemorySizePerBucket);
serverMetricGroup.gauge(
MetricNames.KV_PRE_WRITE_BUFFER_MEMORY_POOL_MAX_SIZE,
kvPreWriteBufferMemoryPool::getMaxMemorySize);
}

public void startup() {
// should do nothing now
}
Expand All @@ -140,6 +158,7 @@ public void shutdown() {
}
arrowBufferAllocator.close();
memorySegmentPool.close();
kvPreWriteBufferMemoryPool.close();
LOG.info("Shut down KvManager complete.");
}

Expand Down Expand Up @@ -186,7 +205,8 @@ public KvTablet getOrCreateKv(
merger,
arrowCompressionInfo,
schemaGetter,
tableConfig.getChangelogImage());
tableConfig.getChangelogImage(),
kvPreWriteBufferMemoryPool);
currentKvs.put(tableBucket, tablet);

LOG.info(
Expand Down Expand Up @@ -294,7 +314,8 @@ public KvTablet loadKv(File tabletDir, SchemaGetter schemaGetter) throws Excepti
rowMerger,
tableConfig.getArrowCompressionInfo(),
schemaGetter,
tableConfig.getChangelogImage());
tableConfig.getChangelogImage(),
kvPreWriteBufferMemoryPool);
if (this.currentKvs.containsKey(tableBucket)) {
throw new IllegalStateException(
String.format(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@
import org.apache.fluss.row.encode.ValueDecoder;
import org.apache.fluss.server.kv.prewrite.KvPreWriteBuffer;
import org.apache.fluss.server.kv.prewrite.KvPreWriteBuffer.TruncateReason;
import org.apache.fluss.server.kv.prewrite.KvPreWriteBufferMemoryPool;
import org.apache.fluss.server.kv.rocksdb.RocksDBKv;
import org.apache.fluss.server.kv.rocksdb.RocksDBKvBuilder;
import org.apache.fluss.server.kv.rocksdb.RocksDBResourceContainer;
Expand Down Expand Up @@ -86,6 +87,7 @@
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;

import static org.apache.fluss.server.kv.prewrite.KvPreWriteBuffer.toByteArray;
import static org.apache.fluss.utils.concurrent.LockUtils.inReadLock;
import static org.apache.fluss.utils.concurrent.LockUtils.inWriteLock;

Expand Down Expand Up @@ -113,6 +115,7 @@ public final class KvTablet {
// defines how to merge rows on the same primary key
private final RowMerger rowMerger;
private final ArrowCompressionInfo arrowCompressionInfo;
private final KvPreWriteBufferMemoryPool kvPreWriteBufferMemoryPool;

private final SchemaGetter schemaGetter;

Expand Down Expand Up @@ -143,14 +146,21 @@ private KvTablet(
RowMerger rowMerger,
ArrowCompressionInfo arrowCompressionInfo,
SchemaGetter schemaGetter,
ChangelogImage changelogImage) {
ChangelogImage changelogImage,
KvPreWriteBufferMemoryPool kvPreWriteBufferMemoryPool) {
this.physicalPath = physicalPath;
this.tableBucket = tableBucket;
this.logTablet = logTablet;
this.kvTabletDir = kvTabletDir;
this.rocksDBKv = rocksDBKv;
this.writeBatchSize = writeBatchSize;
this.kvPreWriteBuffer = new KvPreWriteBuffer(createKvBatchWriter(), serverMetricGroup);
this.kvPreWriteBufferMemoryPool = kvPreWriteBufferMemoryPool;
this.kvPreWriteBuffer =
new KvPreWriteBuffer(
tableBucket,
createKvBatchWriter(),
kvPreWriteBufferMemoryPool,
serverMetricGroup);
this.logFormat = logFormat;
this.arrowWriterProvider = new ArrowWriterPool(arrowBufferAllocator);
this.memorySegmentPool = memorySegmentPool;
Expand All @@ -172,7 +182,8 @@ public static KvTablet create(
RowMerger rowMerger,
ArrowCompressionInfo arrowCompressionInfo,
SchemaGetter schemaGetter,
ChangelogImage changelogImage)
ChangelogImage changelogImage,
KvPreWriteBufferMemoryPool kvPreWriteBufferMemoryPool)
throws IOException {
Tuple2<PhysicalTablePath, TableBucket> tablePathAndBucket =
FlussPaths.parseTabletDir(kvTabletDir);
Expand All @@ -189,7 +200,8 @@ public static KvTablet create(
rowMerger,
arrowCompressionInfo,
schemaGetter,
changelogImage);
changelogImage,
kvPreWriteBufferMemoryPool);
}

public static KvTablet create(
Expand All @@ -205,7 +217,8 @@ public static KvTablet create(
RowMerger rowMerger,
ArrowCompressionInfo arrowCompressionInfo,
SchemaGetter schemaGetter,
ChangelogImage changelogImage)
ChangelogImage changelogImage,
KvPreWriteBufferMemoryPool kvPreWriteBufferMemoryPool)
throws IOException {
RocksDBKv kv = buildRocksDBKv(serverConf, kvTabletDir);
return new KvTablet(
Expand All @@ -223,7 +236,8 @@ public static KvTablet create(
rowMerger,
arrowCompressionInfo,
schemaGetter,
changelogImage);
changelogImage,
kvPreWriteBufferMemoryPool);
}

private static RocksDBKv buildRocksDBKv(Configuration configuration, File kvDir)
Expand Down Expand Up @@ -263,6 +277,10 @@ public long getFlushedLogOffset() {
return flushedLogOffset;
}

public long getPreWriteBufferMemoryPoolUsage() {
return kvPreWriteBufferMemoryPool.getPerBucketUsage(tableBucket);
}

/**
* Put the KvRecordBatch into the kv storage, and return the appended wal log info.
*
Expand Down Expand Up @@ -610,7 +628,9 @@ private byte[] getFromBufferOrKv(KvPreWriteBuffer.Key key) throws IOException {
if (value == null) {
return rocksDBKv.get(key.get());
}
return value.get();

// TODO maybe we can use ByteBuffer to avoid copy
return toByteArray(value.get());
}

public List<byte[]> multiGet(List<byte[]> keys) throws IOException {
Expand Down
Loading