[HUDI-2044] Integrate consumers with rocksDB and compression within External Spillable Map (#3318)
This commit is contained in:
@@ -0,0 +1,93 @@
|
||||
/*
|
||||
* 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.hudi.common.config;
|
||||
|
||||
import org.apache.hudi.common.util.collection.ExternalSpillableMap;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.util.Locale;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* Common Configurations used across Hudi.
|
||||
*/
|
||||
public class HoodieCommonConfig extends HoodieConfig {
|
||||
|
||||
public static final ConfigProperty<ExternalSpillableMap.DiskMapType> SPILLABLE_DISK_MAP_TYPE = ConfigProperty
|
||||
.key("hoodie.common.spillable.diskmap.type")
|
||||
.defaultValue(ExternalSpillableMap.DiskMapType.BITCASK)
|
||||
.withDocumentation("When handling input data that cannot be held in memory, to merge with a file on storage, a spillable diskmap is employed. "
|
||||
+ "By default, we use a persistent hashmap based loosely on bitcask, that offers O(1) inserts, lookups. "
|
||||
+ "Change this to `ROCKS_DB` to prefer using rocksDB, for handling the spill.");
|
||||
|
||||
public static final ConfigProperty<Boolean> DISK_MAP_BITCASK_COMPRESSION_ENABLED = ConfigProperty
|
||||
.key("hoodie.common.diskmap.compression.enabled")
|
||||
.defaultValue(true)
|
||||
.withDocumentation("Turn on compression for BITCASK disk map used by the External Spillable Map");
|
||||
|
||||
public ExternalSpillableMap.DiskMapType getSpillableDiskMapType() {
|
||||
return ExternalSpillableMap.DiskMapType.valueOf(getString(SPILLABLE_DISK_MAP_TYPE).toUpperCase(Locale.ROOT));
|
||||
}
|
||||
|
||||
public boolean isBitCaskDiskMapCompressionEnabled() {
|
||||
return getBoolean(DISK_MAP_BITCASK_COMPRESSION_ENABLED);
|
||||
}
|
||||
|
||||
private HoodieCommonConfig() {
|
||||
super();
|
||||
}
|
||||
|
||||
public static HoodieCommonConfig.Builder newBuilder() {
|
||||
return new HoodieCommonConfig.Builder();
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final HoodieCommonConfig commonConfig = new HoodieCommonConfig();
|
||||
|
||||
public HoodieCommonConfig.Builder fromFile(File propertiesFile) throws IOException {
|
||||
try (FileReader reader = new FileReader(propertiesFile)) {
|
||||
commonConfig.getProps().load(reader);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public HoodieCommonConfig.Builder fromProperties(Properties props) {
|
||||
commonConfig.getProps().putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withSpillableDiskMapType(ExternalSpillableMap.DiskMapType diskMapType) {
|
||||
commonConfig.setValue(SPILLABLE_DISK_MAP_TYPE, diskMapType.name());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withBitcaskDiskMapCompressionEnabled(boolean bitcaskDiskMapCompressionEnabled) {
|
||||
commonConfig.setValue(DISK_MAP_BITCASK_COMPRESSION_ENABLED, String.valueOf(bitcaskDiskMapCompressionEnabled));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieCommonConfig build() {
|
||||
commonConfig.setDefaults(HoodieCommonConfig.class.getName());
|
||||
return commonConfig;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi.common.table.log;
|
||||
|
||||
import org.apache.hudi.common.config.HoodieCommonConfig;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
@@ -73,12 +74,13 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordScanner
|
||||
protected HoodieMergedLogRecordScanner(FileSystem fs, String basePath, List<String> logFilePaths, Schema readerSchema,
|
||||
String latestInstantTime, Long maxMemorySizeInBytes, boolean readBlocksLazily,
|
||||
boolean reverseReader, int bufferSize, String spillableMapBasePath,
|
||||
Option<InstantRange> instantRange, boolean autoScan) {
|
||||
Option<InstantRange> instantRange, boolean autoScan,
|
||||
ExternalSpillableMap.DiskMapType diskMapType, boolean isBitCaskDiskMapCompressionEnabled) {
|
||||
super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize, instantRange);
|
||||
try {
|
||||
// Store merged records for all versions for this log file, set the in-memory footprint to maxInMemoryMapSize
|
||||
this.records = new ExternalSpillableMap<>(maxMemorySizeInBytes, spillableMapBasePath, new DefaultSizeEstimator(),
|
||||
new HoodieRecordSizeEstimator(readerSchema));
|
||||
new HoodieRecordSizeEstimator(readerSchema), diskMapType, isBitCaskDiskMapCompressionEnabled);
|
||||
this.maxMemorySizeInBytes = maxMemorySizeInBytes;
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("IOException when creating ExternalSpillableMap at " + spillableMapBasePath, e);
|
||||
@@ -169,6 +171,8 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordScanner
|
||||
// specific configurations
|
||||
protected Long maxMemorySizeInBytes;
|
||||
protected String spillableMapBasePath;
|
||||
protected ExternalSpillableMap.DiskMapType diskMapType = HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue();
|
||||
protected boolean isBitCaskDiskMapCompressionEnabled = HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue();
|
||||
// incremental filtering
|
||||
private Option<InstantRange> instantRange = Option.empty();
|
||||
// auto scan default true
|
||||
@@ -229,6 +233,16 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordScanner
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withDiskMapType(ExternalSpillableMap.DiskMapType diskMapType) {
|
||||
this.diskMapType = diskMapType;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withBitCaskDiskMapCompressionEnabled(boolean isBitCaskDiskMapCompressionEnabled) {
|
||||
this.isBitCaskDiskMapCompressionEnabled = isBitCaskDiskMapCompressionEnabled;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withAutoScan(boolean autoScan) {
|
||||
this.autoScan = autoScan;
|
||||
return this;
|
||||
@@ -238,7 +252,8 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordScanner
|
||||
public HoodieMergedLogRecordScanner build() {
|
||||
return new HoodieMergedLogRecordScanner(fs, basePath, logFilePaths, readerSchema,
|
||||
latestInstantTime, maxMemorySizeInBytes, readBlocksLazily, reverseReader,
|
||||
bufferSize, spillableMapBasePath, instantRange, autoScan);
|
||||
bufferSize, spillableMapBasePath, instantRange, autoScan,
|
||||
diskMapType, isBitCaskDiskMapCompressionEnabled);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi.common.table.view;
|
||||
|
||||
import org.apache.hudi.common.config.HoodieCommonConfig;
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig;
|
||||
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
@@ -145,10 +146,10 @@ public class FileSystemViewManager {
|
||||
* @return
|
||||
*/
|
||||
private static SpillableMapBasedFileSystemView createSpillableMapBasedFileSystemView(SerializableConfiguration conf,
|
||||
FileSystemViewStorageConfig viewConf, HoodieTableMetaClient metaClient) {
|
||||
FileSystemViewStorageConfig viewConf, HoodieTableMetaClient metaClient, HoodieCommonConfig commonConfig) {
|
||||
LOG.info("Creating SpillableMap based view for basePath " + metaClient.getBasePath());
|
||||
HoodieTimeline timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants();
|
||||
return new SpillableMapBasedFileSystemView(metaClient, timeline, viewConf);
|
||||
return new SpillableMapBasedFileSystemView(metaClient, timeline, viewConf, commonConfig);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -205,15 +206,17 @@ public class FileSystemViewManager {
|
||||
|
||||
public static FileSystemViewManager createViewManager(final HoodieEngineContext context,
|
||||
final HoodieMetadataConfig metadataConfig,
|
||||
final FileSystemViewStorageConfig config) {
|
||||
return createViewManager(context, metadataConfig, config, (SerializableSupplier<HoodieTableMetadata>) null);
|
||||
final FileSystemViewStorageConfig config,
|
||||
final HoodieCommonConfig commonConfig) {
|
||||
return createViewManager(context, metadataConfig, config, commonConfig, (SerializableSupplier<HoodieTableMetadata>) null);
|
||||
}
|
||||
|
||||
public static FileSystemViewManager createViewManager(final HoodieEngineContext context,
|
||||
final HoodieMetadataConfig metadataConfig,
|
||||
final FileSystemViewStorageConfig config,
|
||||
final HoodieCommonConfig commonConfig,
|
||||
final String basePath) {
|
||||
return createViewManager(context, metadataConfig, config,
|
||||
return createViewManager(context, metadataConfig, config, commonConfig,
|
||||
() -> HoodieTableMetadata.create(context, metadataConfig, basePath, config.getSpillableDir(), true));
|
||||
}
|
||||
|
||||
@@ -224,6 +227,7 @@ public class FileSystemViewManager {
|
||||
public static FileSystemViewManager createViewManager(final HoodieEngineContext context,
|
||||
final HoodieMetadataConfig metadataConfig,
|
||||
final FileSystemViewStorageConfig config,
|
||||
final HoodieCommonConfig commonConfig,
|
||||
final SerializableSupplier<HoodieTableMetadata> metadataSupplier) {
|
||||
LOG.info("Creating View Manager with storage type :" + config.getStorageType());
|
||||
final SerializableConfiguration conf = context.getHadoopConf();
|
||||
@@ -235,7 +239,7 @@ public class FileSystemViewManager {
|
||||
case SPILLABLE_DISK:
|
||||
LOG.info("Creating Spillable Disk based Table View");
|
||||
return new FileSystemViewManager(context, config,
|
||||
(metaClient, viewConf) -> createSpillableMapBasedFileSystemView(conf, viewConf, metaClient));
|
||||
(metaClient, viewConf) -> createSpillableMapBasedFileSystemView(conf, viewConf, metaClient, commonConfig));
|
||||
case MEMORY:
|
||||
LOG.info("Creating in-memory based Table View");
|
||||
return new FileSystemViewManager(context, config,
|
||||
@@ -258,7 +262,7 @@ public class FileSystemViewManager {
|
||||
secondaryView = createRocksDBBasedFileSystemView(conf, viewConfig, metaClient);
|
||||
break;
|
||||
case SPILLABLE_DISK:
|
||||
secondaryView = createSpillableMapBasedFileSystemView(conf, viewConfig, metaClient);
|
||||
secondaryView = createSpillableMapBasedFileSystemView(conf, viewConfig, metaClient, commonConfig);
|
||||
break;
|
||||
default:
|
||||
throw new IllegalArgumentException("Secondary Storage type can only be in-memory or spillable. Was :"
|
||||
|
||||
@@ -19,6 +19,8 @@
|
||||
package org.apache.hudi.common.table.view;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
|
||||
import org.apache.hudi.common.config.HoodieCommonConfig;
|
||||
import org.apache.hudi.common.model.BootstrapBaseFileMapping;
|
||||
import org.apache.hudi.common.model.CompactionOperation;
|
||||
import org.apache.hudi.common.model.HoodieFileGroup;
|
||||
@@ -53,9 +55,11 @@ public class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView {
|
||||
private final long maxMemoryForReplaceFileGroups;
|
||||
private final long maxMemoryForClusteringFileGroups;
|
||||
private final String baseStoreDir;
|
||||
private final ExternalSpillableMap.DiskMapType diskMapType;
|
||||
private final boolean isBitCaskDiskMapCompressionEnabled;
|
||||
|
||||
public SpillableMapBasedFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline,
|
||||
FileSystemViewStorageConfig config) {
|
||||
FileSystemViewStorageConfig config, HoodieCommonConfig commonConfig) {
|
||||
super(config.isIncrementalTimelineSyncEnabled());
|
||||
this.maxMemoryForFileGroupMap = config.getMaxMemoryForFileGroupMap();
|
||||
this.maxMemoryForPendingCompaction = config.getMaxMemoryForPendingCompaction();
|
||||
@@ -63,12 +67,14 @@ public class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView {
|
||||
this.maxMemoryForReplaceFileGroups = config.getMaxMemoryForReplacedFileGroups();
|
||||
this.maxMemoryForClusteringFileGroups = config.getMaxMemoryForPendingClusteringFileGroups();
|
||||
this.baseStoreDir = config.getSpillableDir();
|
||||
diskMapType = commonConfig.getSpillableDiskMapType();
|
||||
isBitCaskDiskMapCompressionEnabled = commonConfig.isBitCaskDiskMapCompressionEnabled();
|
||||
init(metaClient, visibleActiveTimeline);
|
||||
}
|
||||
|
||||
public SpillableMapBasedFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline,
|
||||
FileStatus[] fileStatuses, FileSystemViewStorageConfig config) {
|
||||
this(metaClient, visibleActiveTimeline, config);
|
||||
FileStatus[] fileStatuses, FileSystemViewStorageConfig config, HoodieCommonConfig commonConfig) {
|
||||
this(metaClient, visibleActiveTimeline, config, commonConfig);
|
||||
addFilesToView(fileStatuses);
|
||||
}
|
||||
|
||||
@@ -79,7 +85,8 @@ public class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView {
|
||||
+ ", BaseDir=" + baseStoreDir);
|
||||
new File(baseStoreDir).mkdirs();
|
||||
return (Map<String, List<HoodieFileGroup>>) (new ExternalSpillableMap<>(maxMemoryForFileGroupMap, baseStoreDir,
|
||||
new DefaultSizeEstimator(), new DefaultSizeEstimator<>()));
|
||||
new DefaultSizeEstimator(), new DefaultSizeEstimator<>(),
|
||||
diskMapType, isBitCaskDiskMapCompressionEnabled));
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
@@ -93,7 +100,8 @@ public class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView {
|
||||
+ ", BaseDir=" + baseStoreDir);
|
||||
new File(baseStoreDir).mkdirs();
|
||||
Map<HoodieFileGroupId, Pair<String, CompactionOperation>> pendingMap = new ExternalSpillableMap<>(
|
||||
maxMemoryForPendingCompaction, baseStoreDir, new DefaultSizeEstimator(), new DefaultSizeEstimator<>());
|
||||
maxMemoryForPendingCompaction, baseStoreDir, new DefaultSizeEstimator(), new DefaultSizeEstimator<>(),
|
||||
diskMapType, isBitCaskDiskMapCompressionEnabled);
|
||||
pendingMap.putAll(fgIdToPendingCompaction);
|
||||
return pendingMap;
|
||||
} catch (IOException e) {
|
||||
@@ -109,7 +117,8 @@ public class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView {
|
||||
+ ", BaseDir=" + baseStoreDir);
|
||||
new File(baseStoreDir).mkdirs();
|
||||
Map<HoodieFileGroupId, BootstrapBaseFileMapping> pendingMap = new ExternalSpillableMap<>(
|
||||
maxMemoryForBootstrapBaseFile, baseStoreDir, new DefaultSizeEstimator(), new DefaultSizeEstimator<>());
|
||||
maxMemoryForBootstrapBaseFile, baseStoreDir, new DefaultSizeEstimator(), new DefaultSizeEstimator<>(),
|
||||
diskMapType, isBitCaskDiskMapCompressionEnabled);
|
||||
pendingMap.putAll(fileGroupIdBootstrapBaseFileMap);
|
||||
return pendingMap;
|
||||
} catch (IOException e) {
|
||||
@@ -124,7 +133,8 @@ public class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView {
|
||||
+ ", BaseDir=" + baseStoreDir);
|
||||
new File(baseStoreDir).mkdirs();
|
||||
Map<HoodieFileGroupId, HoodieInstant> pendingMap = new ExternalSpillableMap<>(
|
||||
maxMemoryForReplaceFileGroups, baseStoreDir, new DefaultSizeEstimator(), new DefaultSizeEstimator<>());
|
||||
maxMemoryForReplaceFileGroups, baseStoreDir, new DefaultSizeEstimator(), new DefaultSizeEstimator<>(),
|
||||
diskMapType, isBitCaskDiskMapCompressionEnabled);
|
||||
pendingMap.putAll(replacedFileGroups);
|
||||
return pendingMap;
|
||||
} catch (IOException e) {
|
||||
@@ -139,7 +149,8 @@ public class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView {
|
||||
+ ", BaseDir=" + baseStoreDir);
|
||||
new File(baseStoreDir).mkdirs();
|
||||
Map<HoodieFileGroupId, HoodieInstant> pendingMap = new ExternalSpillableMap<>(
|
||||
maxMemoryForClusteringFileGroups, baseStoreDir, new DefaultSizeEstimator(), new DefaultSizeEstimator<>());
|
||||
maxMemoryForClusteringFileGroups, baseStoreDir, new DefaultSizeEstimator(), new DefaultSizeEstimator<>(),
|
||||
diskMapType, isBitCaskDiskMapCompressionEnabled);
|
||||
pendingMap.putAll(fileGroupsInClustering);
|
||||
return pendingMap;
|
||||
} catch (IOException e) {
|
||||
|
||||
@@ -20,6 +20,7 @@ package org.apache.hudi.metadata;
|
||||
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.avro.model.HoodieMetadataRecord;
|
||||
import org.apache.hudi.common.config.HoodieCommonConfig;
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig;
|
||||
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
@@ -213,6 +214,7 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
|
||||
|
||||
// Load the schema
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(HoodieMetadataRecord.getClassSchema());
|
||||
HoodieCommonConfig commonConfig = HoodieCommonConfig.newBuilder().fromProperties(metadataConfig.getProps()).build();
|
||||
logRecordScanner = HoodieMetadataMergedLogRecordScanner.newBuilder()
|
||||
.withFileSystem(metaClient.getFs())
|
||||
.withBasePath(metadataBasePath)
|
||||
@@ -222,6 +224,8 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
|
||||
.withMaxMemorySizeInBytes(MAX_MEMORY_SIZE_IN_BYTES)
|
||||
.withBufferSize(BUFFER_SIZE)
|
||||
.withSpillableMapBasePath(spillableMapDirectory)
|
||||
.withDiskMapType(commonConfig.getSpillableDiskMapType())
|
||||
.withBitCaskDiskMapCompressionEnabled(commonConfig.isBitCaskDiskMapCompressionEnabled())
|
||||
.build();
|
||||
|
||||
logScannerOpenMs = timer.endTimer();
|
||||
|
||||
@@ -30,6 +30,7 @@ import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.ExternalSpillableMap;
|
||||
|
||||
/**
|
||||
* A {@code HoodieMergedLogRecordScanner} implementation which only merged records matching providing keys. This is
|
||||
@@ -41,9 +42,10 @@ public class HoodieMetadataMergedLogRecordScanner extends HoodieMergedLogRecordS
|
||||
|
||||
private HoodieMetadataMergedLogRecordScanner(FileSystem fs, String basePath, List<String> logFilePaths,
|
||||
Schema readerSchema, String latestInstantTime, Long maxMemorySizeInBytes, int bufferSize,
|
||||
String spillableMapBasePath, Set<String> mergeKeyFilter) {
|
||||
String spillableMapBasePath, Set<String> mergeKeyFilter,
|
||||
ExternalSpillableMap.DiskMapType diskMapType, boolean isBitCaskDiskMapCompressionEnabled) {
|
||||
super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, maxMemorySizeInBytes, false, false, bufferSize,
|
||||
spillableMapBasePath, Option.empty(), false);
|
||||
spillableMapBasePath, Option.empty(), false, diskMapType, isBitCaskDiskMapCompressionEnabled);
|
||||
this.mergeKeyFilter = mergeKeyFilter;
|
||||
|
||||
performScan();
|
||||
@@ -134,6 +136,16 @@ public class HoodieMetadataMergedLogRecordScanner extends HoodieMergedLogRecordS
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withDiskMapType(ExternalSpillableMap.DiskMapType diskMapType) {
|
||||
this.diskMapType = diskMapType;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withBitCaskDiskMapCompressionEnabled(boolean isBitCaskDiskMapCompressionEnabled) {
|
||||
this.isBitCaskDiskMapCompressionEnabled = isBitCaskDiskMapCompressionEnabled;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMergeKeyFilter(Set<String> mergeKeyFilter) {
|
||||
this.mergeKeyFilter = mergeKeyFilter;
|
||||
return this;
|
||||
@@ -142,7 +154,8 @@ public class HoodieMetadataMergedLogRecordScanner extends HoodieMergedLogRecordS
|
||||
@Override
|
||||
public HoodieMetadataMergedLogRecordScanner build() {
|
||||
return new HoodieMetadataMergedLogRecordScanner(fs, basePath, logFilePaths, readerSchema,
|
||||
latestInstantTime, maxMemorySizeInBytes, bufferSize, spillableMapBasePath, mergeKeyFilter);
|
||||
latestInstantTime, maxMemorySizeInBytes, bufferSize, spillableMapBasePath, mergeKeyFilter,
|
||||
diskMapType, isBitCaskDiskMapCompressionEnabled);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user