Making ExternalSpillableMap generic for any datatype
- Introduced concept of converters to be able to serde generic datatype for SpillableMap - Fixed/Added configs to Hoodie Configs - Changed HoodieMergeHandle to start using SpillableMap
This commit is contained in:
committed by
vinoth chandar
parent
fa787ab5ab
commit
987f5d6b96
@@ -19,7 +19,6 @@ package com.uber.hoodie.io;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
@@ -65,10 +64,12 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
|
||||
FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName));
|
||||
}
|
||||
|
||||
public Path makeTempPath(String partitionPath, int taskPartitionId, String fileName, int stageId, long taskAttemptId) {
|
||||
public Path makeTempPath(String partitionPath, int taskPartitionId, String fileName, int stageId,
|
||||
long taskAttemptId) {
|
||||
Path path = new Path(config.getBasePath(), HoodieTableMetaClient.TEMPFOLDER_NAME);
|
||||
return new Path(path.toString(),
|
||||
FSUtils.makeTempDataFileName(partitionPath, commitTime, taskPartitionId, fileName, stageId, taskAttemptId));
|
||||
FSUtils.makeTempDataFileName(partitionPath, commitTime, taskPartitionId, fileName, stageId,
|
||||
taskAttemptId));
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -25,17 +25,15 @@ import com.uber.hoodie.common.model.HoodieWriteStat;
|
||||
import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.ReflectionUtils;
|
||||
import com.uber.hoodie.common.util.collection.ExternalSpillableMap;
|
||||
import com.uber.hoodie.common.util.collection.converter.StringConverter;
|
||||
import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import com.uber.hoodie.exception.HoodieUpsertException;
|
||||
import com.uber.hoodie.io.storage.HoodieStorageWriter;
|
||||
import com.uber.hoodie.io.storage.HoodieStorageWriterFactory;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
@@ -43,6 +41,11 @@ import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.TaskContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
|
||||
@SuppressWarnings("Duplicates")
|
||||
public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHandle<T> {
|
||||
|
||||
@@ -145,9 +148,14 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
||||
* @return
|
||||
*/
|
||||
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<>();
|
||||
try {
|
||||
// Load the new records in a map
|
||||
logger.info("MaxMemoryPerPartitionMerge => " + config.getMaxMemoryPerPartitionMerge());
|
||||
this.keyToNewRecords = new ExternalSpillableMap<>(config.getMaxMemoryPerPartitionMerge(),
|
||||
Optional.empty(), new StringConverter(), new HoodieRecordConverter(schema, config.getPayloadClass()));
|
||||
} catch(IOException io) {
|
||||
throw new HoodieIOException("Cannot instantiate an ExternalSpillableMap", io);
|
||||
}
|
||||
String partitionPath = null;
|
||||
while (newRecordsItr.hasNext()) {
|
||||
HoodieRecord<T> record = newRecordsItr.next();
|
||||
@@ -156,6 +164,15 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
||||
// update the new location of the record, so we know where to find it next
|
||||
record.setNewLocation(new HoodieRecordLocation(commitTime, fileId));
|
||||
}
|
||||
logger.debug("Number of entries in MemoryBasedMap => " +
|
||||
((ExternalSpillableMap) keyToNewRecords).getInMemoryMapNumEntries()
|
||||
+ "Total size in bytes of MemoryBasedMap => " +
|
||||
((ExternalSpillableMap) keyToNewRecords).getCurrentInMemoryMapSize()
|
||||
+ "Number of entries in DiskBasedMap => " +
|
||||
((ExternalSpillableMap) keyToNewRecords).getDiskBasedMapNumEntries()
|
||||
+ "Size of file spilled to disk => " +
|
||||
((ExternalSpillableMap) keyToNewRecords).getSizeOfFileOnDiskInBytes());
|
||||
|
||||
return partitionPath;
|
||||
}
|
||||
|
||||
|
||||
@@ -98,11 +98,12 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
.getTimelineOfActions(
|
||||
Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.ROLLBACK_ACTION,
|
||||
HoodieTimeline.DELTA_COMMIT_ACTION))
|
||||
.filterCompletedInstants().lastInstant().get().getTimestamp();
|
||||
|
||||
.filterCompletedInstants().lastInstant().get().getTimestamp();
|
||||
log.info("MaxMemoryPerCompaction => " + config.getMaxMemoryPerCompaction());
|
||||
HoodieCompactedLogRecordScanner scanner = new HoodieCompactedLogRecordScanner(fs,
|
||||
metaClient.getBasePath(), operation.getDeltaFilePaths(), readerSchema, maxInstantTime,
|
||||
config.getMaxMemorySizePerCompactionInBytes(), config.getCompactionLazyBlockReadEnabled(),
|
||||
config.getMaxMemoryPerCompaction(), config.getCompactionLazyBlockReadEnabled(),
|
||||
config.getCompactionReverseLogReadEnabled());
|
||||
if (!scanner.iterator().hasNext()) {
|
||||
return Lists.<WriteStatus>newArrayList();
|
||||
|
||||
Reference in New Issue
Block a user