1
0

[HUDI-2028] Implement RockDbBasedMap as an alternate to DiskBasedMap in ExternalSpillableMap (#3194)

Co-authored-by: Rajesh Mahindra <rmahindra@Rajeshs-MacBook-Pro.local>
This commit is contained in:
rmahindra123
2021-07-05 23:03:41 -07:00
committed by GitHub
parent a0f598d371
commit a4dcbb5c5a
14 changed files with 620 additions and 60 deletions

View File

@@ -35,6 +35,7 @@ import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.common.util.collection.ExternalSpillableMap;
import org.apache.hudi.execution.bulkinsert.BulkInsertSortMode;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.keygen.constant.KeyGeneratorType;
@@ -52,6 +53,7 @@ import java.io.IOException;
import java.io.InputStream;
import java.util.Arrays;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Objects;
import java.util.Properties;
@@ -283,6 +285,11 @@ public class HoodieWriteConfig extends HoodieConfig {
.defaultValue("false")
.withDocumentation("Allow duplicates with inserts while merging with existing records");
public static final ConfigProperty<ExternalSpillableMap.DiskMapType> SPILLABLE_DISK_MAP_TYPE = ConfigProperty
.key("hoodie.spillable.diskmap.type")
.defaultValue(ExternalSpillableMap.DiskMapType.BITCASK)
.withDocumentation("Enable usage of either BITCASK or ROCKS_DB as disk map for External Spillable Map");
public static final ConfigProperty<Integer> CLIENT_HEARTBEAT_INTERVAL_IN_MS_PROP = ConfigProperty
.key("hoodie.client.heartbeat.interval_in_ms")
.defaultValue(60 * 1000)
@@ -554,6 +561,10 @@ public class HoodieWriteConfig extends HoodieConfig {
return getBoolean(MERGE_ALLOW_DUPLICATE_ON_INSERTS);
}
public ExternalSpillableMap.DiskMapType getSpillableDiskMapType() {
return ExternalSpillableMap.DiskMapType.valueOf(getString(SPILLABLE_DISK_MAP_TYPE).toUpperCase(Locale.ROOT));
}
public EngineType getEngineType() {
return engineType;
}
@@ -1504,6 +1515,11 @@ public class HoodieWriteConfig extends HoodieConfig {
return this;
}
public Builder withSpillableDiskMapType(ExternalSpillableMap.DiskMapType diskMapType) {
writeConfig.setValue(SPILLABLE_DISK_MAP_TYPE, diskMapType.name());
return this;
}
public Builder withHeartbeatIntervalInMs(Integer heartbeatIntervalInMs) {
writeConfig.setValue(CLIENT_HEARTBEAT_INTERVAL_IN_MS_PROP, String.valueOf(heartbeatIntervalInMs));
return this;

View File

@@ -199,7 +199,8 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> extends H
long memoryForMerge = IOUtils.getMaxMemoryPerPartitionMerge(taskContextSupplier, config);
LOG.info("MaxMemoryPerPartitionMerge => " + memoryForMerge);
this.keyToNewRecords = new ExternalSpillableMap<>(memoryForMerge, config.getSpillableMapBasePath(),
new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(tableSchema));
new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(tableSchema),
config.getSpillableDiskMapType());
} catch (IOException io) {
throw new HoodieIOException("Cannot instantiate an ExternalSpillableMap", io);
}
@@ -231,7 +232,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> extends H
LOG.info("Number of entries in MemoryBasedMap => "
+ ((ExternalSpillableMap) keyToNewRecords).getInMemoryMapNumEntries()
+ "Total size in bytes of MemoryBasedMap => "
+ ((ExternalSpillableMap) keyToNewRecords).getCurrentInMemoryMapSize() + "Number of entries in DiskBasedMap => "
+ ((ExternalSpillableMap) keyToNewRecords).getCurrentInMemoryMapSize() + "Number of entries in BitCaskDiskMap => "
+ ((ExternalSpillableMap) keyToNewRecords).getDiskBasedMapNumEntries() + "Size of file spilled to disk => "
+ ((ExternalSpillableMap) keyToNewRecords).getSizeOfFileOnDiskInBytes());
}