Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
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 @@ -33,6 +33,7 @@
import java.util.Map;
import java.util.Set;
import java.util.Spliterators;
import java.util.UUID;
import java.util.stream.Stream;
import java.util.stream.StreamSupport;

Expand All @@ -41,21 +42,25 @@
* All of the data is stored using the RocksDB implementation.
*/
public final class RocksDbDiskMap<T extends Serializable, R extends Serializable> extends DiskMap<T, R> {
private static final Logger LOG = LogManager.getLogger(RocksDbDiskMap.class);

// ColumnFamily allows partitioning data within RockDB, which allows
// independent configuration and faster deletes across partitions
// https://github.com/facebook/rocksdb/wiki/Column-Families
// For this use case, we use a single static column family/ partition
//
private static final String ROCKSDB_COL_FAMILY = "rocksdb-diskmap";
private static final String ROCKSDB_BASE_PATH = "rocksdb-diskmap";

private static final Logger LOG = LogManager.getLogger(RocksDbDiskMap.class);
// Stores the key and corresponding value's latest metadata spilled to disk
private final Set<T> keySet;
private RocksDBDAO rocksDb;
private static RocksDBDAO rocksDb = null;
private String rocksdbColFamily;

public RocksDbDiskMap(String rocksDbStoragePath) throws IOException {
super(rocksDbStoragePath, ExternalSpillableMap.DiskMapType.ROCKS_DB.name());
this.keySet = new HashSet<>();
this.rocksdbColFamily = "rocksdb-diskmap" + UUID.randomUUID().toString();
getRocksDb().addColumnFamily(rocksdbColFamily);
}

@Override
Expand Down Expand Up @@ -83,12 +88,12 @@ public R get(Object key) {
if (!containsKey(key)) {
return null;
}
return getRocksDb().get(ROCKSDB_COL_FAMILY, (T) key);
return getRocksDb().get(rocksdbColFamily, (T) key);
}

@Override
public R put(T key, R value) {
getRocksDb().put(ROCKSDB_COL_FAMILY, key, value);
getRocksDb().put(rocksdbColFamily, key, value);
keySet.add(key);
return value;
}
Expand All @@ -98,14 +103,14 @@ public R remove(Object key) {
R value = get(key);
if (value != null) {
keySet.remove((T) key);
getRocksDb().delete(ROCKSDB_COL_FAMILY, (T) key);
getRocksDb().delete(rocksdbColFamily, (T) key);
}
return value;
}

@Override
public void putAll(Map<? extends T, ? extends R> keyValues) {
getRocksDb().writeBatch(batch -> keyValues.forEach((key, value) -> getRocksDb().putInBatch(batch, ROCKSDB_COL_FAMILY, key, value)));
getRocksDb().writeBatch(batch -> keyValues.forEach((key, value) -> getRocksDb().putInBatch(batch, rocksdbColFamily, key, value)));
keySet.addAll(keyValues.keySet());
}

Expand Down Expand Up @@ -138,7 +143,7 @@ public Set<Entry<T, R>> entrySet() {
*/
@Override
public Iterator<R> iterator() {
return getRocksDb().iterator(ROCKSDB_COL_FAMILY);
return getRocksDb().iterator(rocksdbColFamily);
}

@Override
Expand All @@ -155,18 +160,30 @@ public long sizeOfFileOnDiskInBytes() {
public void close() {
keySet.clear();
if (null != rocksDb) {
rocksDb.close();
rocksDb.dropColumnFamily(rocksdbColFamily);
}
rocksDb = null;
super.close();
}

private RocksDBDAO getRocksDb() {
if (null == rocksDb) {
synchronized (this) {
synchronized (ROCKSDB_BASE_PATH) {
if (null == rocksDb) {
rocksDb = new RocksDBDAO(ROCKSDB_COL_FAMILY, diskMapPath);
rocksDb.addColumnFamily(ROCKSDB_COL_FAMILY);
rocksDb = new RocksDBDAO(ROCKSDB_BASE_PATH, diskMapPath);
Runtime.getRuntime().addShutdownHook(new Thread() {
@Override
public void run() {
if (null != rocksDb) {
synchronized (ROCKSDB_BASE_PATH) {
if (null != rocksDb) {
rocksDb.close();
rocksDb = null;
LOG.info("closed " + ROCKSDB_BASE_PATH + " rocksdb");
}
}
}
}
});
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.hudi.table.format;

import org.apache.hudi.common.config.HoodieCommonConfig;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieOperation;
import org.apache.hudi.common.model.HoodieRecord;
Expand All @@ -26,10 +27,12 @@
import org.apache.hudi.common.util.DefaultSizeEstimator;
import org.apache.hudi.common.util.Functions;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.ExternalSpillableMap;
import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueProducer;
import org.apache.hudi.common.util.queue.FunctionBasedQueueProducer;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.hadoop.config.HoodieRealtimeConfig;
import org.apache.hudi.table.format.mor.MergeOnReadInputSplit;
import org.apache.hudi.util.StreamerUtil;
Expand Down Expand Up @@ -120,9 +123,12 @@ private static Object getVal(IndexedRecord record, int pos) {
public static HoodieMergedLogRecordScanner logScanner(
MergeOnReadInputSplit split,
Schema logSchema,
Configuration config,
boolean withOperationField) {
FileSystem fs = FSUtils.getFs(split.getTablePath(), config);
org.apache.flink.configuration.Configuration flinkConf,
Configuration hadoopConf) {
FileSystem fs = FSUtils.getFs(split.getTablePath(), hadoopConf);
final ExternalSpillableMap.DiskMapType diskMapType = ExternalSpillableMap.DiskMapType.valueOf(
flinkConf.getString(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.key(),
ExternalSpillableMap.DiskMapType.BITCASK.name()).toUpperCase(Locale.ROOT));
return HoodieMergedLogRecordScanner.newBuilder()
.withFileSystem(fs)
.withBasePath(split.getTablePath())
Expand All @@ -131,27 +137,29 @@ public static HoodieMergedLogRecordScanner logScanner(
.withLatestInstantTime(split.getLatestCommit())
.withReadBlocksLazily(
string2Boolean(
config.get(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP,
flinkConf.getString(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP,
HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)))
.withReverseReader(false)
.withBufferSize(
config.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP,
flinkConf.getInteger(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP,
HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE))
.withMaxMemorySizeInBytes(split.getMaxCompactionMemoryInBytes())
.withMaxMemorySizeInBytes(diskMapType == ExternalSpillableMap.DiskMapType.ROCKS_DB ? 0 : split.getMaxCompactionMemoryInBytes())
.withDiskMapType(diskMapType)
.withSpillableMapBasePath(
config.get(HoodieRealtimeConfig.SPILLABLE_MAP_BASE_PATH_PROP,
flinkConf.getString(HoodieRealtimeConfig.SPILLABLE_MAP_BASE_PATH_PROP,
HoodieRealtimeConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH))
.withInstantRange(split.getInstantRange())
.withOperationField(withOperationField)
.withOperationField(flinkConf.getBoolean(FlinkOptions.CHANGELOG_ENABLED))
.build();
}

private static HoodieUnMergedLogRecordScanner unMergedLogScanner(
MergeOnReadInputSplit split,
Schema logSchema,
Configuration config,
org.apache.flink.configuration.Configuration flinkConf,
Configuration hadoopConf,
HoodieUnMergedLogRecordScanner.LogRecordScannerCallback callback) {
FileSystem fs = FSUtils.getFs(split.getTablePath(), config);
FileSystem fs = FSUtils.getFs(split.getTablePath(), hadoopConf);
return HoodieUnMergedLogRecordScanner.newBuilder()
.withFileSystem(fs)
.withBasePath(split.getTablePath())
Expand All @@ -160,11 +168,11 @@ private static HoodieUnMergedLogRecordScanner unMergedLogScanner(
.withLatestInstantTime(split.getLatestCommit())
.withReadBlocksLazily(
string2Boolean(
config.get(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP,
flinkConf.getString(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP,
HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)))
.withReverseReader(false)
.withBufferSize(
config.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP,
flinkConf.getInteger(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP,
HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE))
.withInstantRange(split.getInstantRange())
.withLogRecordScannerCallback(callback)
Expand Down Expand Up @@ -198,7 +206,7 @@ public BoundedMemoryRecords(
Functions.noop());
// Consumer of this record reader
this.iterator = this.executor.getQueue().iterator();
this.scanner = FormatUtils.unMergedLogScanner(split, logSchema, hadoopConf,
this.scanner = FormatUtils.unMergedLogScanner(split, logSchema, flinkConf, hadoopConf,
record -> executor.getQueue().insertRecord(record));
// Start reading and buffering
this.executor.startProducers();
Expand Down Expand Up @@ -232,6 +240,7 @@ public static HoodieMergedLogRecordScanner logScanner(
HoodieWriteConfig writeConfig,
Configuration hadoopConf) {
String basePath = writeConfig.getBasePath();
final ExternalSpillableMap.DiskMapType diskMapType = writeConfig.getCommonConfig().getSpillableDiskMapType();
return HoodieMergedLogRecordScanner.newBuilder()
.withFileSystem(FSUtils.getFs(basePath, hadoopConf))
.withBasePath(basePath)
Expand All @@ -241,9 +250,9 @@ public static HoodieMergedLogRecordScanner logScanner(
.withReadBlocksLazily(writeConfig.getCompactionLazyBlockReadEnabled())
.withReverseReader(false)
.withBufferSize(writeConfig.getMaxDFSStreamBufferSize())
.withMaxMemorySizeInBytes(writeConfig.getMaxMemoryPerPartitionMerge())
.withMaxMemorySizeInBytes(diskMapType == ExternalSpillableMap.DiskMapType.ROCKS_DB ? 0 : writeConfig.getMaxMemoryPerPartitionMerge())
.withSpillableMapBasePath(writeConfig.getSpillableMapBasePath())
.withDiskMapType(writeConfig.getCommonConfig().getSpillableDiskMapType())
.withDiskMapType(diskMapType)
.withBitCaskDiskMapCompressionEnabled(writeConfig.getCommonConfig().isBitCaskDiskMapCompressionEnabled())
.build();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -192,6 +192,7 @@ public void open(MergeOnReadInputSplit split) throws IOException {
getLogFileIterator(split));
} else if (split.getMergeType().equals(FlinkOptions.REALTIME_PAYLOAD_COMBINE)) {
this.iterator = new MergeIterator(
conf,
hadoopConf,
split,
this.tableState.getRowType(),
Expand All @@ -200,7 +201,6 @@ public void open(MergeOnReadInputSplit split) throws IOException {
new Schema.Parser().parse(this.tableState.getRequiredAvroSchema()),
this.requiredPos,
this.emitDelete,
this.conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED),
this.tableState.getOperationPos(),
getFullSchemaReader(split.getBasePath().get()));
} else {
Expand Down Expand Up @@ -323,7 +323,7 @@ private ClosableIterator<RowData> getLogFileIterator(MergeOnReadInputSplit split
final GenericRecordBuilder recordBuilder = new GenericRecordBuilder(requiredSchema);
final AvroToRowDataConverters.AvroToRowDataConverter avroToRowDataConverter =
AvroToRowDataConverters.createRowConverter(tableState.getRequiredRowType());
final HoodieMergedLogRecordScanner scanner = FormatUtils.logScanner(split, tableSchema, hadoopConf, conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED));
final HoodieMergedLogRecordScanner scanner = FormatUtils.logScanner(split, tableSchema, conf, hadoopConf);
final Iterator<String> logRecordsKeyIterator = scanner.getRecords().keySet().iterator();
final int[] pkOffset = tableState.getPkOffsetsInRequired();
// flag saying whether the pk semantics has been dropped by user specified
Expand Down Expand Up @@ -639,6 +639,7 @@ static class MergeIterator implements RecordIterator {
private RowData currentRecord;

MergeIterator(
Configuration conf,
org.apache.hadoop.conf.Configuration hadoopConf,
MergeOnReadInputSplit split,
RowType tableRowType,
Expand All @@ -647,12 +648,11 @@ static class MergeIterator implements RecordIterator {
Schema requiredSchema,
int[] requiredPos,
boolean emitDelete,
boolean withOperationField,
int operationPos,
ParquetColumnarRowSplitReader reader) { // the reader should be with full schema
this.tableSchema = tableSchema;
this.reader = reader;
this.scanner = FormatUtils.logScanner(split, tableSchema, hadoopConf, withOperationField);
this.scanner = FormatUtils.logScanner(split, tableSchema, conf, hadoopConf);
this.logKeysIterator = scanner.getRecords().keySet().iterator();
this.requiredSchema = requiredSchema;
this.requiredPos = requiredPos;
Expand Down