1
0

Added support for Disk Spillable Compaction to prevent OOM issues

This commit is contained in:
Nishith Agarwal
2017-12-06 13:11:27 -08:00
committed by vinoth chandar
parent d495484399
commit 6fec9655a8
18 changed files with 1487 additions and 86 deletions

View File

@@ -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();

View File

@@ -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()) {

View File

@@ -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();
}