Added support for Disk Spillable Compaction to prevent OOM issues
This commit is contained in:
committed by
vinoth chandar
parent
d495484399
commit
6fec9655a8
@@ -64,7 +64,6 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
|
||||
// Turned off by default
|
||||
public static final String DEFAULT_PARQUET_SMALL_FILE_LIMIT_BYTES = String.valueOf(0);
|
||||
|
||||
|
||||
/**
|
||||
* Configs related to specific table types
|
||||
**/
|
||||
@@ -102,6 +101,10 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
|
||||
public static final String DEFAULT_PAYLOAD_CLASS = HoodieAvroPayload.class.getName();
|
||||
public static final String PAYLOAD_CLASS = "hoodie.compaction.payload.class";
|
||||
|
||||
public static final String MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES_PROP = "hoodie.compaction.spill.threshold";
|
||||
// Default memory size per compaction, excess spills to disk
|
||||
public static final String DEFAULT_MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES = String.valueOf(1024*1024*1024L); //1GB
|
||||
|
||||
private HoodieCompactionConfig(Properties props) {
|
||||
super(props);
|
||||
}
|
||||
@@ -210,6 +213,18 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMaxMemorySizePerCompactionInBytes(long maxMemorySizePerCompactionInBytes) {
|
||||
props.setProperty(MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES_PROP,
|
||||
String.valueOf(maxMemorySizePerCompactionInBytes));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMaxNumDeltaCommitsBeforeCompaction(int maxNumDeltaCommitsBeforeCompaction) {
|
||||
props.setProperty(INLINE_COMPACT_NUM_DELTA_COMMITS_PROP,
|
||||
String.valueOf(maxNumDeltaCommitsBeforeCompaction));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieCompactionConfig build() {
|
||||
HoodieCompactionConfig config = new HoodieCompactionConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(AUTO_CLEAN_PROP),
|
||||
@@ -245,6 +260,8 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
|
||||
PAYLOAD_CLASS, DEFAULT_PAYLOAD_CLASS);
|
||||
setDefaultOnCondition(props, !props.containsKey(TARGET_IO_PER_COMPACTION_IN_MB_PROP),
|
||||
TARGET_IO_PER_COMPACTION_IN_MB_PROP, DEFAULT_TARGET_IO_PER_COMPACTION_IN_MB);
|
||||
setDefaultOnCondition(props, !props.containsKey(MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES_PROP),
|
||||
MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES_PROP, DEFAULT_MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES);
|
||||
|
||||
HoodieCleaningPolicy.valueOf(props.getProperty(CLEANER_POLICY_PROP));
|
||||
Preconditions.checkArgument(
|
||||
|
||||
@@ -210,6 +210,11 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
.parseLong(props.getProperty(HoodieCompactionConfig.TARGET_IO_PER_COMPACTION_IN_MB_PROP));
|
||||
}
|
||||
|
||||
public Long getMaxMemorySizePerCompactionInBytes() {
|
||||
return Long
|
||||
.parseLong(props.getProperty(HoodieCompactionConfig.MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES_PROP));
|
||||
}
|
||||
|
||||
/**
|
||||
* index properties
|
||||
**/
|
||||
|
||||
@@ -47,6 +47,7 @@ import org.apache.spark.util.SizeEstimator;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Comparator;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
@@ -108,8 +109,8 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
this.writer = HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath))
|
||||
.withFileId(fileId).overBaseCommit(baseCommitTime).withLogVersion(fileSlice.getLogFiles()
|
||||
.max(HoodieLogFile.getLogVersionComparator().reversed()::compare)
|
||||
.map(logFile -> logFile.getLogVersion()).orElse(HoodieLogFile.LOGFILE_BASE_VERSION))
|
||||
.map(logFile -> logFile.getLogVersion())
|
||||
.max(Comparator.naturalOrder()).orElse(HoodieLogFile.LOGFILE_BASE_VERSION))
|
||||
.withSizeThreshold(config.getLogFileMaxSize())
|
||||
.withFs(fs).withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
|
||||
this.currentLogFile = writer.getLogFile();
|
||||
|
||||
@@ -142,6 +142,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
||||
*/
|
||||
private String init(String fileId, Iterator<HoodieRecord<T>> newRecordsItr) {
|
||||
// Load the new records in a map
|
||||
// TODO (NA) instantiate a ExternalSpillableMap
|
||||
this.keyToNewRecords = new HashMap<>();
|
||||
String partitionPath = null;
|
||||
while (newRecordsItr.hasNext()) {
|
||||
|
||||
@@ -16,8 +16,6 @@
|
||||
|
||||
package com.uber.hoodie.io.compact;
|
||||
|
||||
import static java.util.stream.Collectors.toList;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
@@ -38,6 +36,13 @@ import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieCompactionException;
|
||||
import com.uber.hoodie.table.HoodieCopyOnWriteTable;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.FlatMapFunction;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Collection;
|
||||
@@ -46,12 +51,8 @@ import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.StreamSupport;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.FlatMapFunction;
|
||||
|
||||
import static java.util.stream.Collectors.toList;
|
||||
|
||||
/**
|
||||
* HoodieRealtimeTableCompactor compacts a hoodie table with merge on read storage. Computes all
|
||||
@@ -73,6 +74,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
+ HoodieTableType.MERGE_ON_READ + " and not " + hoodieTable.getMetaClient()
|
||||
.getTableType().name());
|
||||
|
||||
//TODO : check if maxMemory is not greater than JVM or spark.executor memory
|
||||
// TODO - rollback any compactions in flight
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
log.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommitTime);
|
||||
@@ -152,7 +154,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
|
||||
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs,
|
||||
metaClient.getBasePath(),
|
||||
operation.getDeltaFilePaths(), readerSchema, maxInstantTime);
|
||||
operation.getDeltaFilePaths(), readerSchema, maxInstantTime, config.getMaxMemorySizePerCompactionInBytes());
|
||||
if (!scanner.iterator().hasNext()) {
|
||||
return Lists.newArrayList();
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user