1
0

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:
Nishith Agarwal
2018-03-15 00:20:16 -07:00
committed by vinoth chandar
parent fa787ab5ab
commit 987f5d6b96
24 changed files with 984 additions and 281 deletions

View File

@@ -23,7 +23,7 @@ summary: "Here we list all possible configurations and what they mean"
<span style="color:grey">Should HoodieWriteClient autoCommit after insert and upsert. The client can choose to turn off auto-commit and commit on a "defined success condition"</span>
- [withAssumeDatePartitioning](#withAssumeDatePartitioning) (assumeDatePartitioning = false) <br/>
<span style="color:grey">Should HoodieWriteClient assume the data is partitioned by dates, i.e three levels from base path. This is a stop-gap to support tables created by versions < 0.3.1. Will be removed eventually </span>
- [withIndexConfig](#withIndexConfig) (HoodieIndexConfig) <br/>
<span style="color:grey">Hoodie uses a index to help find the FileID which contains an incoming record key. This is pluggable to have a external index (HBase) or use the default bloom filter stored in the Parquet files</span>
- [withIndexType](#withIndexType) (indexType = BLOOM) <br/>
@@ -80,8 +80,6 @@ summary: "Here we list all possible configurations and what they mean"
<span style="color:grey">Number of max delta commits to keep before triggering an inline compaction</span>
- [withCompactionReverseLogReadEnabled](#withCompactionReverseLogReadEnabled) (false) <br/>
<span style="color:grey">HoodieLogFormatReader reads a logfile in the forward direction starting from pos=0 to pos=file_length. If this config is set to true, the Reader reads the logfile in reverse direction, from pos=file_length to pos=0</span>
- [withMaxMemorySizePerCompactionInBytes](#withMaxMemorySizePerCompactionInBytes) (maxMemorySizePerCompactionInBytes = 1GB) <br/>
<span style="color:grey">HoodieCompactedLogScanner reads logblocks, converts records to HoodieRecords and then merges these log blocks and records. At any point, the number of entries in a log block can be less than or equal to the number of entries in the corresponding parquet file. This can lead to OOM in the Scanner. Hence, a spillable map helps alleviate the memory pressure. Use this config to set the max allowable inMemory footprint of the spillable map.</span>
- [withMetricsConfig](#withMetricsConfig) (HoodieMetricsConfig) <br/>
<span style="color:grey">Hoodie publishes metrics on every commit, clean, rollback etc.</span>
@@ -96,6 +94,13 @@ summary: "Here we list all possible configurations and what they mean"
- [usePrefix](#usePrefix) () <br/>
<span style="color:grey">Standard prefix for all metrics</span>
- [withMemoryConfig](#withMemoryConfig) (HoodieMemoryConfig) <br/>
<span style="color:grey">Memory related configs</span>
- [withMaxMemoryFractionPerPartitionMerge](#withMaxMemoryFractionPerPartitionMerge) (maxMemoryFractionPerPartitionMerge = 0.6) <br/>
<span style="color:grey">This fraction is multiplied with the user memory fraction (1 - spark.memory.fraction) to get a final fraction of heap space to use during merge </span>
- [withMaxMemorySizePerCompactionInBytes](#withMaxMemorySizePerCompactionInBytes) (maxMemorySizePerCompactionInBytes = 1GB) <br/>
<span style="color:grey">HoodieCompactedLogScanner reads logblocks, converts records to HoodieRecords and then merges these log blocks and records. At any point, the number of entries in a log block can be less than or equal to the number of entries in the corresponding parquet file. This can lead to OOM in the Scanner. Hence, a spillable map helps alleviate the memory pressure. Use this config to set the max allowable inMemory footprint of the spillable map.</span>
- [S3Configs](s3_hoodie.html) (Hoodie S3 Configs) <br/>
<span style="color:grey">Configurations required for S3 and Hoodie co-operability.</span>

View File

@@ -31,6 +31,7 @@ import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType;
import com.uber.hoodie.config.HoodieCompactionConfig;
import com.uber.hoodie.config.HoodieMemoryConfig;
import com.uber.hoodie.hive.util.SchemaUtil;
import java.io.IOException;
import java.util.ArrayList;
@@ -173,7 +174,7 @@ public class HoodieLogFileCommand implements CommandMarker {
HoodieCLI.tableMetadata.getBasePath(), logFilePaths, readerSchema,
HoodieCLI.tableMetadata.getActiveTimeline().getCommitTimeline().lastInstant().get()
.getTimestamp(),
Long.valueOf(HoodieCompactionConfig.DEFAULT_MAX_SIZE_IN_MEMORY_PER_COMPACTION_IN_BYTES),
Long.valueOf(HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES),
Boolean.valueOf(HoodieCompactionConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED),
Boolean.valueOf(HoodieCompactionConfig.DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED));
for (HoodieRecord<? extends HoodieRecordPayload> hoodieRecord : scanner) {

View File

@@ -99,11 +99,7 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
// used to merge records written to log file
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
public static final String PAYLOAD_CLASS_PROP = "hoodie.compaction.payload.class";
// used to choose a trade off between IO vs Memory when performing compaction process
// Depending on outputfile_size and memory provided, choose true to avoid OOM for large file size + small memory
@@ -212,7 +208,7 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
}
public Builder withPayloadClass(String payloadClassName) {
props.setProperty(PAYLOAD_CLASS, payloadClassName);
props.setProperty(PAYLOAD_CLASS_PROP, payloadClassName);
return this;
}
@@ -222,12 +218,6 @@ 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));
@@ -277,12 +267,10 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
CLEANER_PARALLELISM, DEFAULT_CLEANER_PARALLELISM);
setDefaultOnCondition(props, !props.containsKey(COMPACTION_STRATEGY_PROP),
COMPACTION_STRATEGY_PROP, DEFAULT_COMPACTION_STRATEGY);
setDefaultOnCondition(props, !props.containsKey(PAYLOAD_CLASS),
PAYLOAD_CLASS, DEFAULT_PAYLOAD_CLASS);
setDefaultOnCondition(props, !props.containsKey(PAYLOAD_CLASS_PROP),
PAYLOAD_CLASS_PROP, 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);
setDefaultOnCondition(props, !props.containsKey(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP),
COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED);
setDefaultOnCondition(props, !props.containsKey(COMPACTION_REVERSE_LOG_READ_ENABLED_PROP),

View File

@@ -0,0 +1,147 @@
/*
* Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed 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 com.uber.hoodie.config;
import java.io.File;
import java.io.FileReader;
import java.io.IOException;
import java.util.Properties;
import javax.annotation.concurrent.Immutable;
import org.apache.spark.SparkEnv;
import org.apache.spark.util.Utils;
/**
* Memory related config
*/
@Immutable
public class HoodieMemoryConfig extends DefaultHoodieConfig {
// This fraction is multiplied with the spark.memory.fraction to get a final fraction of heap space to use during merge
// This makes it easier to scale this value as one increases the spark.executor.memory
public static final String MAX_MEMORY_FRACTION_FOR_MERGE_PROP = "hoodie.memory.merge.fraction";
// Default max memory fraction during hash-merge, excess spills to disk
public static final String DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE = String.valueOf(0.6);
public static final String MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP = "hoodie.memory.compaction.fraction";
// Default max memory fraction during compaction, excess spills to disk
public static final String DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION = String.valueOf(0.6);
// Default memory size per compaction (used if SparkEnv is absent), excess spills to disk
public static final long DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES =
1024 * 1024 * 1024L; // 1GB
// Property to set the max memory for merge
public static final String MAX_MEMORY_FOR_MERGE_PROP = "hoodie.memory.merge.max.size";
// Property to set the max memory for compaction
public static final String MAX_MEMORY_FOR_COMPACTION_PROP = "hoodie.memory.compaction.max.size";
private HoodieMemoryConfig(Properties props) {
super(props);
}
public static HoodieMemoryConfig.Builder newBuilder() {
return new Builder();
}
public static class Builder {
private final Properties props = new Properties();
public Builder fromFile(File propertiesFile) throws IOException {
FileReader reader = new FileReader(propertiesFile);
try {
this.props.load(reader);
return this;
} finally {
reader.close();
}
}
public Builder fromProperties(Properties props) {
this.props.putAll(props);
return this;
}
public Builder withMaxMemoryFractionPerPartitionMerge(long maxMemoryFractionPerPartitionMerge) {
props.setProperty(MAX_MEMORY_FRACTION_FOR_MERGE_PROP,
String.valueOf(maxMemoryFractionPerPartitionMerge));
return this;
}
public Builder withMaxMemoryFractionPerCompaction(long maxMemoryFractionPerCompaction) {
props.setProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP,
String.valueOf(maxMemoryFractionPerCompaction));
return this;
}
/**
* Dynamic calculation of max memory to use for for spillable map. user.available.memory =
* spark.executor.memory * (1 - spark.memory.fraction) spillable.available.memory =
* user.available.memory * hoodie.memory.fraction. Anytime the spark.executor.memory or the
* spark.memory.fraction is changed, the memory used for spillable map changes accordingly
*/
private long getMaxMemoryAllowedForMerge(String maxMemoryFraction) {
final String SPARK_EXECUTOR_MEMORY_PROP = "spark.executor.memory";
final String SPARK_EXECUTOR_MEMORY_FRACTION_PROP = "spark.memory.fraction";
// This is hard-coded in spark code {@link https://github.com/apache/spark/blob/576c43fb4226e4efa12189b41c3bc862019862c6/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala#L231}
// so have to re-define this here
final String DEFAULT_SPARK_EXECUTOR_MEMORY_FRACTION = "0.6";
// This is hard-coded in spark code {@link https://github.com/apache/spark/blob/576c43fb4226e4efa12189b41c3bc862019862c6/core/src/main/scala/org/apache/spark/SparkContext.scala#L471}
// so have to re-define this here
final String DEFAULT_SPARK_EXECUTOR_MEMORY_MB = "1024"; // in MB
if (SparkEnv.get() != null) {
// 1 GB is the default conf used by Spark, look at SparkContext.scala
long executorMemoryInBytes = Long.valueOf(
Utils.memoryStringToMb(SparkEnv.get().conf().get(SPARK_EXECUTOR_MEMORY_PROP,
DEFAULT_SPARK_EXECUTOR_MEMORY_MB)) * 1024
* 1024L);
// 0.6 is the default value used by Spark,
// look at {@link https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/SparkConf.scala#L507}
double memoryFraction = Double
.valueOf(SparkEnv.get().conf().get(SPARK_EXECUTOR_MEMORY_FRACTION_PROP,
DEFAULT_SPARK_EXECUTOR_MEMORY_FRACTION));
double maxMemoryFractionForMerge = Double.valueOf(maxMemoryFraction);
double userAvailableMemory = executorMemoryInBytes * (1 - memoryFraction);
long maxMemoryForMerge = (long) Math
.floor(userAvailableMemory * maxMemoryFractionForMerge);
return maxMemoryForMerge;
} else {
return DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES;
}
}
public HoodieMemoryConfig build() {
HoodieMemoryConfig config = new HoodieMemoryConfig(props);
setDefaultOnCondition(props,
!props.containsKey(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP),
MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP,
DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION);
setDefaultOnCondition(props,
!props.containsKey(MAX_MEMORY_FRACTION_FOR_MERGE_PROP),
MAX_MEMORY_FRACTION_FOR_MERGE_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE);
setDefaultOnCondition(props,
!props.containsKey(MAX_MEMORY_FOR_MERGE_PROP),
MAX_MEMORY_FOR_MERGE_PROP, String.valueOf(
getMaxMemoryAllowedForMerge(props.getProperty(MAX_MEMORY_FRACTION_FOR_MERGE_PROP))));
setDefaultOnCondition(props,
!props.containsKey(MAX_MEMORY_FOR_COMPACTION_PROP),
MAX_MEMORY_FOR_COMPACTION_PROP, String.valueOf(
getMaxMemoryAllowedForMerge(props.getProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP))));
return config;
}
}
}

View File

@@ -48,7 +48,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
private static final String BULKINSERT_PARALLELISM = "hoodie.bulkinsert.shuffle.parallelism";
private static final String UPSERT_PARALLELISM = "hoodie.upsert.shuffle.parallelism";
private static final String WRITE_BUFFER_LIMIT_BYTES = "hoodie.write.buffer.limit.bytes";
private static final String DEFAULT_WRITE_BUFFER_LIMIT_BYTES = String.valueOf(4*1024*1024);
private static final String DEFAULT_WRITE_BUFFER_LIMIT_BYTES = String.valueOf(4 * 1024 * 1024);
private static final String COMBINE_BEFORE_INSERT_PROP = "hoodie.combine.before.insert";
private static final String DEFAULT_COMBINE_BEFORE_INSERT = "false";
private static final String COMBINE_BEFORE_UPSERT_PROP = "hoodie.combine.before.upsert";
@@ -108,7 +108,8 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
}
public int getWriteBufferLimitBytes() {
return Integer.parseInt(props.getProperty(WRITE_BUFFER_LIMIT_BYTES, DEFAULT_WRITE_BUFFER_LIMIT_BYTES));
return Integer
.parseInt(props.getProperty(WRITE_BUFFER_LIMIT_BYTES, DEFAULT_WRITE_BUFFER_LIMIT_BYTES));
}
public boolean shouldCombineBeforeInsert() {
@@ -217,17 +218,18 @@ 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));
}
public Boolean getCompactionLazyBlockReadEnabled() {
return Boolean.valueOf(props.getProperty(HoodieCompactionConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP));
return Boolean
.valueOf(props.getProperty(HoodieCompactionConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP));
}
public Boolean getCompactionReverseLogReadEnabled() {
return Boolean.valueOf(props.getProperty(HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLED_PROP));
return Boolean.valueOf(
props.getProperty(HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLED_PROP));
}
public String getPayloadClass() {
return props.getProperty(HoodieCompactionConfig.PAYLOAD_CLASS_PROP);
}
/**
@@ -283,7 +285,8 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
}
public StorageLevel getBloomIndexInputStorageLevel() {
return StorageLevel.fromString(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_INPUT_STORAGE_LEVEL));
return StorageLevel
.fromString(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_INPUT_STORAGE_LEVEL));
}
/**
@@ -302,7 +305,8 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
}
public int getLogFileDataBlockMaxSize() {
return Integer.parseInt(props.getProperty(HoodieStorageConfig.LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES));
return Integer
.parseInt(props.getProperty(HoodieStorageConfig.LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES));
}
public int getLogFileMaxSize() {
@@ -313,7 +317,6 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
return Double.valueOf(props.getProperty(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO));
}
/**
* metrics properties
**/
@@ -342,6 +345,28 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
return new Builder();
}
/**
* memory configs
*/
public Double getMaxMemoryFractionPerPartitionMerge() {
return Double.valueOf(props.getProperty(HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_MERGE_PROP));
}
public Double getMaxMemoryFractionPerCompaction() {
return Double
.valueOf(
props.getProperty(HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP));
}
public Long getMaxMemoryPerPartitionMerge() {
return Long.valueOf(props.getProperty(HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE_PROP));
}
public Long getMaxMemoryPerCompaction() {
return Long
.valueOf(
props.getProperty(HoodieMemoryConfig.MAX_MEMORY_FOR_COMPACTION_PROP));
}
public static class Builder {
@@ -351,6 +376,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
private boolean isCompactionConfigSet = false;
private boolean isMetricsConfigSet = false;
private boolean isAutoCommit = true;
private boolean isMemoryConfigSet = false;
public Builder fromFile(File propertiesFile) throws IOException {
FileReader reader = new FileReader(propertiesFile);
@@ -501,9 +527,11 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
setDefaultOnCondition(props, !props.containsKey(HOODIE_WRITE_STATUS_CLASS_PROP),
HOODIE_WRITE_STATUS_CLASS_PROP, DEFAULT_HOODIE_WRITE_STATUS_CLASS);
setDefaultOnCondition(props, !props.containsKey(HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE),
HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE, DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE);
HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE,
DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER_CREATE);
setDefaultOnCondition(props, !props.containsKey(HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE),
HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE, DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE);
HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE,
DEFAULT_HOODIE_COPYONWRITE_USE_TEMP_FOLDER_MERGE);
setDefaultOnCondition(props, !props.containsKey(FINALIZE_WRITE_PARALLELISM),
FINALIZE_WRITE_PARALLELISM, DEFAULT_FINALIZE_WRITE_PARALLELISM);
@@ -516,6 +544,8 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
HoodieCompactionConfig.newBuilder().fromProperties(props).build());
setDefaultOnCondition(props, !isMetricsConfigSet,
HoodieMetricsConfig.newBuilder().fromProperties(props).build());
setDefaultOnCondition(props, !isMemoryConfigSet,
HoodieMemoryConfig.newBuilder().fromProperties(props).build());
return config;
}
}

View File

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

View File

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

View File

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

View File

@@ -135,5 +135,10 @@
<version>1.16.0</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>com.twitter.common</groupId>
<artifactId>objectsize</artifactId>
<version>0.0.12</version>
</dependency>
</dependencies>
</project>

View File

@@ -16,7 +16,6 @@
package com.uber.hoodie.common.table.log;
import com.google.common.collect.Maps;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.model.HoodieRecord;
@@ -29,8 +28,9 @@ import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.util.SpillableMapUtils;
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.exception.HoodieIOException;
import java.util.stream.Collectors;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
@@ -48,6 +48,7 @@ import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.atomic.AtomicLong;
import java.util.stream.Collectors;
import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType.INSTANT_TIME;
import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.CORRUPT_BLOCK;
@@ -102,8 +103,8 @@ public class HoodieCompactedLogRecordScanner implements
try {
// Store merged records for all versions for this log file, set the in-memory footprint to maxInMemoryMapSize
this.records = new ExternalSpillableMap<>(maxMemorySizeInBytes, readerSchema,
payloadClassFQN, Optional.empty());
this.records = new ExternalSpillableMap<>(maxMemorySizeInBytes, Optional.empty(),
new StringConverter(), new HoodieRecordConverter(readerSchema, payloadClassFQN));
// iterate over the paths
HoodieLogFormatReader logFormatReaderWrapper =
new HoodieLogFormatReader(fs,

View File

@@ -19,11 +19,10 @@ import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.util.collection.DiskBasedMap;
import com.uber.hoodie.common.util.collection.converter.Converter;
import com.uber.hoodie.common.util.collection.io.storage.SizeAwareDataOutputStream;
import com.uber.hoodie.exception.HoodieCorruptedDataException;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.htrace.fasterxml.jackson.databind.ObjectMapper;
import java.io.IOException;
import java.io.RandomAccessFile;
@@ -32,28 +31,23 @@ import java.util.zip.CRC32;
public class SpillableMapUtils {
public static ObjectMapper objectMapper = new ObjectMapper();
/**
* Using the schema and payload class, read and convert the bytes on disk to a HoodieRecord
*
* @param file
* @param schema
* @param payloadClazz
* @param valuePosition
* @param valueLength
* @param <R>
* @return
* @throws IOException
*/
public static <R> R readFromDisk(RandomAccessFile file, Schema schema, String payloadClazz,
long valuePosition, int valueLength) throws IOException {
public static byte[] readBytesFromDisk(RandomAccessFile file, long valuePosition, int valueLength) throws IOException {
DiskBasedMap.FileEntry fileEntry = readInternal(file, valuePosition, valueLength);
return (R) convertToHoodieRecordPayload(HoodieAvroUtils.bytesToAvro(fileEntry.getValue(), schema),
payloadClazz);
return fileEntry.getValue();
}
/**
* |crc|timestamp|sizeOfKey|SizeOfValue|key|value|
*
* @param file
* @param valuePosition
* @param valueLength
@@ -66,15 +60,15 @@ public class SpillableMapUtils {
long timestamp = file.readLong();
int keySize = file.readInt();
int valueSize = file.readInt();
byte [] key = new byte[keySize];
byte[] key = new byte[keySize];
file.read(key, 0, keySize);
byte [] value = new byte[valueSize];
if(!(valueSize == valueLength)) {
byte[] value = new byte[valueSize];
if (!(valueSize == valueLength)) {
throw new HoodieCorruptedDataException("unequal size of payload written to external file, data may be corrupted");
}
file.read(value, 0, valueSize);
long crcOfReadValue = generateChecksum(value);
if(!(crc == crcOfReadValue)) {
if (!(crc == crcOfReadValue)) {
throw new HoodieCorruptedDataException("checksum of payload written to external disk does not match, " +
"data may be corrupted");
}
@@ -83,7 +77,7 @@ public class SpillableMapUtils {
/**
* Write Value and other metadata necessary to disk. Each entry has the following sequence of data
*
* <p>
* |crc|timestamp|sizeOfKey|SizeOfValue|key|value|
*
* @param outputStream
@@ -108,10 +102,11 @@ public class SpillableMapUtils {
/**
* Generate a checksum for a given set of bytes
*
* @param data
* @return
*/
public static long generateChecksum(byte [] data) {
public static long generateChecksum(byte[] data) {
CRC32 crc = new CRC32();
crc.update(data);
return crc.getValue();
@@ -120,20 +115,19 @@ public class SpillableMapUtils {
/**
* Compute a bytes representation of the payload by serializing the contents
* This is used to estimate the size of the payload (either in memory or when written to disk)
*
* @param <R>
* @param value
* @param schema
* @return
* @throws IOException
*/
public static <R> int computePayloadSize(R value, Schema schema) throws IOException {
HoodieRecord payload = (HoodieRecord) value;
byte [] val = HoodieAvroUtils.avroToBytes((GenericRecord) payload.getData().getInsertValue(schema).get());
return val.length;
public static <R> long computePayloadSize(R value, Converter<R> valueConverter) throws IOException {
return valueConverter.sizeEstimate(value);
}
/**
* Utility method to convert bytes to HoodieRecord using schema and payload class
*
* @param rec
* @param payloadClazz
* @param <R>

View File

@@ -16,15 +16,14 @@
package com.uber.hoodie.common.util.collection;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.SpillableMapUtils;
import com.uber.hoodie.common.util.collection.converter.Converter;
import com.uber.hoodie.common.util.collection.io.storage.SizeAwareDataOutputStream;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.exception.HoodieNotSupportedException;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.File;
import java.io.FileOutputStream;
@@ -48,13 +47,11 @@ import java.util.concurrent.atomic.AtomicLong;
* 1) An in-memory map that tracks the key-> latest ValueMetadata.
* 2) Current position in the file
* NOTE : Only String.class type supported for Key
* @param <T>
* @param <R>
*/
final public class DiskBasedMap<T,R> implements Map<T,R> {
final public class DiskBasedMap<T, R> implements Map<T, R> {
// Stores the key and corresponding value's latest metadata spilled to disk
final private Map<T, ValueMetadata> inMemoryMetadataOfSpilledData;
final private Map<T, ValueMetadata> valueMetadataMap;
// Read only file access to be able to seek to random positions to readFromDisk values
private RandomAccessFile readOnlyFileHandle;
// Write only OutputStream to be able to ONLY append to the file
@@ -64,16 +61,20 @@ final public class DiskBasedMap<T,R> implements Map<T,R> {
private FileOutputStream fileOutputStream;
// Current position in the file
private AtomicLong filePosition;
// Schema used to de-serialize payload written to disk
private Schema schema;
// Class used to de-serialize/realize payload written to disk
private String payloadClazz;
// FilePath to store the spilled data
private String filePath;
// Default file path prefix to put the spillable file
private static String DEFAULT_BASE_FILE_PATH = "/tmp/";
// Key converter to convert key type to bytes
final private Converter<T> keyConverter;
// Value converter to convert value type to bytes
final private Converter<R> valueConverter;
private static Logger log = LogManager.getLogger(DiskBasedMap.class);
public final class ValueMetadata {
// FilePath to store the spilled data
private String filePath;
// Size (numberOfBytes) of the value written to disk
@@ -108,6 +109,7 @@ final public class DiskBasedMap<T,R> implements Map<T,R> {
}
public static final class FileEntry {
// Checksum of the value written to disk, compared during every readFromDisk to make sure no corruption
private Long crc;
// Size (numberOfBytes) of the key written to disk
@@ -115,13 +117,14 @@ final public class DiskBasedMap<T,R> implements Map<T,R> {
// Size (numberOfBytes) of the value written to disk
private Integer sizeOfValue;
// Actual key
private byte [] key;
private byte[] key;
// Actual value
private byte [] value;
private byte[] value;
// Current timestamp when the value was written to disk
private Long timestamp;
public FileEntry(long crc, int sizeOfKey, int sizeOfValue, byte [] key, byte [] value, long timestamp) {
public FileEntry(long crc, int sizeOfKey, int sizeOfValue, byte[] key, byte[] value,
long timestamp) {
this.crc = crc;
this.sizeOfKey = sizeOfKey;
this.sizeOfValue = sizeOfValue;
@@ -155,10 +158,11 @@ final public class DiskBasedMap<T,R> implements Map<T,R> {
}
}
protected DiskBasedMap(Schema schema, String payloadClazz, Optional<String> baseFilePath) throws IOException {
this.inMemoryMetadataOfSpilledData = new HashMap<>();
protected DiskBasedMap(Optional<String> baseFilePath,
Converter<T> keyConverter, Converter<R> valueConverter) throws IOException {
this.valueMetadataMap = new HashMap<>();
if(!baseFilePath.isPresent()) {
if (!baseFilePath.isPresent()) {
baseFilePath = Optional.of(DEFAULT_BASE_FILE_PATH);
}
this.filePath = baseFilePath.get() + UUID.randomUUID().toString();
@@ -168,16 +172,18 @@ final public class DiskBasedMap<T,R> implements Map<T,R> {
this.fileOutputStream = new FileOutputStream(writeOnlyFileHandle, true);
this.writeOnlyFileHandle = new SizeAwareDataOutputStream(fileOutputStream);
this.filePosition = new AtomicLong(0L);
this.schema = schema;
this.payloadClazz = payloadClazz;
this.keyConverter = keyConverter;
this.valueConverter = valueConverter;
}
private void initFile(File writeOnlyFileHandle) throws IOException {
// delete the file if it exists
if(writeOnlyFileHandle.exists()) {
if (writeOnlyFileHandle.exists()) {
writeOnlyFileHandle.delete();
}
writeOnlyFileHandle.createNewFile();
log.info("Spilling to file location " + writeOnlyFileHandle.getAbsolutePath());
// Open file in readFromDisk-only mode
readOnlyFileHandle = new RandomAccessFile(filePath, "r");
readOnlyFileHandle.seek(0);
@@ -194,12 +200,12 @@ final public class DiskBasedMap<T,R> implements Map<T,R> {
Runtime.getRuntime().addShutdownHook(new Thread() {
public void run() {
try {
if(writeOnlyFileHandle != null) {
if (writeOnlyFileHandle != null) {
writeOnlyFileHandle.flush();
fileOutputStream.getChannel().force(false);
writeOnlyFileHandle.close();
}
} catch(Exception e) {
} catch (Exception e) {
// fail silently for any sort of exception
}
}
@@ -208,16 +214,14 @@ final public class DiskBasedMap<T,R> implements Map<T,R> {
/**
* Custom iterator to iterate over values written to disk
* @return
*/
public Iterator<R> iterator() {
return new LazyFileIterable(readOnlyFileHandle,
inMemoryMetadataOfSpilledData, schema, payloadClazz).iterator();
valueMetadataMap, valueConverter).iterator();
}
/**
* Number of bytes spilled to disk
* @return
*/
public long sizeOfFileOnDiskInBytes() {
return filePosition.get();
@@ -225,17 +229,17 @@ final public class DiskBasedMap<T,R> implements Map<T,R> {
@Override
public int size() {
return inMemoryMetadataOfSpilledData.size();
return valueMetadataMap.size();
}
@Override
public boolean isEmpty() {
return inMemoryMetadataOfSpilledData.isEmpty();
return valueMetadataMap.isEmpty();
}
@Override
public boolean containsKey(Object key) {
return inMemoryMetadataOfSpilledData.containsKey(key);
return valueMetadataMap.containsKey(key);
}
@Override
@@ -245,34 +249,31 @@ final public class DiskBasedMap<T,R> implements Map<T,R> {
@Override
public R get(Object key) {
ValueMetadata entry = inMemoryMetadataOfSpilledData.get(key);
if(entry == null) {
ValueMetadata entry = valueMetadataMap.get(key);
if (entry == null) {
return null;
}
try {
return SpillableMapUtils.readFromDisk(readOnlyFileHandle, schema,
payloadClazz, entry.getOffsetOfValue(), entry.getSizeOfValue());
} catch(IOException e) {
return this.valueConverter.getData(SpillableMapUtils.readBytesFromDisk(readOnlyFileHandle,
entry.getOffsetOfValue(), entry.getSizeOfValue()));
} catch (IOException e) {
throw new HoodieIOException("Unable to readFromDisk Hoodie Record from disk", e);
}
}
@Override
public R put(T key, R value) {
//TODO (na) : check value instanceof HoodieRecordPayload, now assume every payload is HoodieRecord
HoodieRecord payload = (HoodieRecord) value;
try {
byte [] val = HoodieAvroUtils.avroToBytes((GenericRecord) payload.getData().getInsertValue(this.schema).get());
byte[] val = this.valueConverter.getBytes(value);
Integer valueSize = val.length;
Long timestamp = new Date().getTime();
this.inMemoryMetadataOfSpilledData.put(key, new DiskBasedMap.ValueMetadata(this.filePath, valueSize,
filePosition.get(), timestamp));
// TODO(na) : Test serializer performance for generic types
String serializedKey = SpillableMapUtils.objectMapper.writeValueAsString(key);
this.valueMetadataMap.put(key,
new DiskBasedMap.ValueMetadata(this.filePath, valueSize, filePosition.get(), timestamp));
byte[] serializedKey = keyConverter.getBytes(key);
filePosition.set(SpillableMapUtils.spillToDisk(writeOnlyFileHandle,
new FileEntry(SpillableMapUtils.generateChecksum(val),
serializedKey.getBytes().length, valueSize, serializedKey.getBytes(), val, timestamp)));
} catch(IOException io) {
serializedKey.length, valueSize, serializedKey, val, timestamp)));
} catch (IOException io) {
throw new HoodieIOException("Unable to store data in Disk Based map", io);
}
return value;
@@ -281,33 +282,33 @@ final public class DiskBasedMap<T,R> implements Map<T,R> {
@Override
public R remove(Object key) {
R value = get(key);
inMemoryMetadataOfSpilledData.remove(key);
valueMetadataMap.remove(key);
return value;
}
@Override
public void putAll(Map<? extends T, ? extends R> m) {
for(Map.Entry<? extends T, ? extends R> entry: m.entrySet()) {
for (Map.Entry<? extends T, ? extends R> entry : m.entrySet()) {
put(entry.getKey(), entry.getValue());
}
}
@Override
public void clear() {
inMemoryMetadataOfSpilledData.clear();
valueMetadataMap.clear();
// close input/output streams
try {
writeOnlyFileHandle.flush();
writeOnlyFileHandle.close();
new File(filePath).delete();
} catch(IOException e) {
} catch (IOException e) {
throw new HoodieIOException("unable to clear map or delete file on disk", e);
}
}
@Override
public Set<T> keySet() {
return inMemoryMetadataOfSpilledData.keySet();
return valueMetadataMap.keySet();
}
@Override
@@ -318,7 +319,7 @@ final public class DiskBasedMap<T,R> implements Map<T,R> {
@Override
public Set<Entry<T, R>> entrySet() {
Set<Entry<T, R>> entrySet = new HashSet<>();
for(T key: inMemoryMetadataOfSpilledData.keySet()) {
for (T key : valueMetadataMap.keySet()) {
entrySet.add(new AbstractMap.SimpleEntry<>(key, get(key)));
}
return entrySet;

View File

@@ -16,10 +16,10 @@
package com.uber.hoodie.common.util.collection;
import com.uber.hoodie.common.util.SpillableMapUtils;
import com.uber.hoodie.exception.HoodieIOException;
import com.twitter.common.objectsize.ObjectSizeCalculator;
import com.uber.hoodie.common.util.collection.converter.Converter;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieNotSupportedException;
import org.apache.avro.Schema;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -33,56 +33,54 @@ import java.util.Optional;
import java.util.Set;
/**
* An external map that spills content to disk when there is insufficient space for it
* to grow.
*
* This map holds 2 types of data structures :
*
* (1) Key-Value pairs in a in-memory map
* (2) Key-ValueMetadata pairs in an in-memory map which keeps a marker to the values spilled to disk
*
* NOTE : Values are only appended to disk. If a remove() is called, the entry is marked removed from the in-memory
* key-valueMetadata map but it's values will be lying around in the temp file on disk until the file is cleaned.
*
* The setting of the spill threshold faces the following trade-off: If the spill threshold is
* too high, the in-memory map may occupy more memory than is available, resulting in OOM.
* However, if the spill threshold is too low, we spill frequently and incur unnecessary disk
* writes.
* @param <T>
* @param <R>
* An external map that spills content to disk when there is insufficient space for it to grow. <p>
* This map holds 2 types of data structures : <p> (1) Key-Value pairs in a in-memory map (2)
* Key-ValueMetadata pairs in an in-memory map which keeps a marker to the values spilled to disk
* <p> NOTE : Values are only appended to disk. If a remove() is called, the entry is marked removed
* from the in-memory key-valueMetadata map but it's values will be lying around in the temp file on
* disk until the file is cleaned. <p> The setting of the spill threshold faces the following
* trade-off: If the spill threshold is too high, the in-memory map may occupy more memory than is
* available, resulting in OOM. However, if the spill threshold is too low, we spill frequently and
* incur unnecessary disk writes.
*/
public class ExternalSpillableMap<T,R> implements Map<T,R> {
public class ExternalSpillableMap<T, R> implements Map<T, R> {
// Find the actual estimated payload size after inserting N records
final private static int NUMBER_OF_RECORDS_TO_ESTIMATE_PAYLOAD_SIZE = 100;
// maximum space allowed in-memory for this map
final private long maxInMemorySizeInBytes;
// current space occupied by this map in-memory
private Long currentInMemoryMapSize;
// Map to store key-values in memory until it hits maxInMemorySizeInBytes
final private Map<T,R> inMemoryMap;
final private Map<T, R> inMemoryMap;
// Map to store key-valuemetadata important to find the values spilled to disk
final private DiskBasedMap<T,R> diskBasedMap;
// Schema used to de-serialize and readFromDisk the records written to disk
final private Schema schema;
final private DiskBasedMap<T, R> diskBasedMap;
// An estimate of the size of each payload written to this map
private volatile long estimatedPayloadSize = 0;
// TODO(na) : a dynamic sizing factor to ensure we have space for other objects in memory and incorrect payload estimation
final private Double sizingFactorForInMemoryMap = 0.8;
// Key converter to convert key type to bytes
final private Converter<T> keyConverter;
// Value converter to convert value type to bytes
final private Converter<R> valueConverter;
// Flag to determine whether to stop re-estimating payload size
private boolean shouldEstimatePayloadSize = true;
private static Logger log = LogManager.getLogger(ExternalSpillableMap.class);
public ExternalSpillableMap(Long maxInMemorySizeInBytes, Schema schema,
String payloadClazz, Optional<String> baseFilePath) throws IOException {
public ExternalSpillableMap(Long maxInMemorySizeInBytes, Optional<String> baseFilePath,
Converter<T> keyConverter, Converter<R> valueConverter) throws IOException {
this.inMemoryMap = new HashMap<>();
this.diskBasedMap = new DiskBasedMap<>(schema, payloadClazz, baseFilePath);
this.maxInMemorySizeInBytes = (long) Math.floor(maxInMemorySizeInBytes*sizingFactorForInMemoryMap);
this.schema = schema;
this.diskBasedMap = new DiskBasedMap<>(baseFilePath, keyConverter, valueConverter);
this.maxInMemorySizeInBytes = (long) Math
.floor(maxInMemorySizeInBytes * sizingFactorForInMemoryMap);
this.currentInMemoryMapSize = 0L;
this.keyConverter = keyConverter;
this.valueConverter = valueConverter;
}
/**
* A custom iterator to wrap over iterating in-memory + disk spilled data
* @return
*/
public Iterator<R> iterator() {
return new IteratorWrapper<>(inMemoryMap.values().iterator(), diskBasedMap.iterator());
@@ -90,7 +88,6 @@ public class ExternalSpillableMap<T,R> implements Map<T,R> {
/**
* Number of entries in DiskBasedMap
* @return
*/
public int getDiskBasedMapNumEntries() {
return diskBasedMap.size();
@@ -98,7 +95,6 @@ public class ExternalSpillableMap<T,R> implements Map<T,R> {
/**
* Number of bytes spilled to disk
* @return
*/
public long getSizeOfFileOnDiskInBytes() {
return diskBasedMap.sizeOfFileOnDiskInBytes();
@@ -106,7 +102,6 @@ public class ExternalSpillableMap<T,R> implements Map<T,R> {
/**
* Number of entries in InMemoryMap
* @return
*/
public int getInMemoryMapNumEntries() {
return inMemoryMap.size();
@@ -114,7 +109,6 @@ public class ExternalSpillableMap<T,R> implements Map<T,R> {
/**
* Approximate memory footprint of the in-memory map
* @return
*/
public long getCurrentInMemoryMapSize() {
return currentInMemoryMapSize;
@@ -142,9 +136,9 @@ public class ExternalSpillableMap<T,R> implements Map<T,R> {
@Override
public R get(Object key) {
if(inMemoryMap.containsKey(key)) {
if (inMemoryMap.containsKey(key)) {
return inMemoryMap.get(key);
} else if(diskBasedMap.containsKey(key)) {
} else if (diskBasedMap.containsKey(key)) {
return diskBasedMap.get(key);
}
return null;
@@ -152,33 +146,43 @@ public class ExternalSpillableMap<T,R> implements Map<T,R> {
@Override
public R put(T key, R value) {
try {
if (this.currentInMemoryMapSize < maxInMemorySizeInBytes || inMemoryMap.containsKey(key)) {
// Naive approach for now
if (estimatedPayloadSize == 0) {
this.estimatedPayloadSize = SpillableMapUtils.computePayloadSize(value, schema);
log.info("Estimated Payload size => " + estimatedPayloadSize);
}
if(!inMemoryMap.containsKey(key)) {
currentInMemoryMapSize += this.estimatedPayloadSize;
}
inMemoryMap.put(key, value);
} else {
diskBasedMap.put(key, value);
if (this.currentInMemoryMapSize < maxInMemorySizeInBytes || inMemoryMap.containsKey(key)) {
if (shouldEstimatePayloadSize && estimatedPayloadSize == 0) {
// At first, use the sizeEstimate of a record being inserted into the spillable map.
// Note, the converter may over estimate the size of a record in the JVM
this.estimatedPayloadSize =
keyConverter.sizeEstimate(key) + valueConverter.sizeEstimate(value);
log.info("Estimated Payload size => " + estimatedPayloadSize);
}
return value;
} catch(IOException io) {
throw new HoodieIOException("Unable to estimate size of payload", io);
else if(shouldEstimatePayloadSize &&
inMemoryMap.size() % NUMBER_OF_RECORDS_TO_ESTIMATE_PAYLOAD_SIZE == 0) {
// Re-estimate the size of a record by calculating the size of the entire map containing
// N entries and then dividing by the number of entries present (N). This helps to get a
// correct estimation of the size of each record in the JVM.
long totalMapSize = ObjectSizeCalculator.getObjectSize(inMemoryMap);
this.currentInMemoryMapSize = totalMapSize;
this.estimatedPayloadSize = totalMapSize/inMemoryMap.size();
shouldEstimatePayloadSize = false;
log.info("New Estimated Payload size => " + this.estimatedPayloadSize);
}
if (!inMemoryMap.containsKey(key)) {
// TODO : Add support for adjusting payloadSize for updates to the same key
currentInMemoryMapSize += this.estimatedPayloadSize;
}
inMemoryMap.put(key, value);
} else {
diskBasedMap.put(key, value);
}
return value;
}
@Override
public R remove(Object key) {
// NOTE : diskBasedMap.remove does not delete the data from disk
if(inMemoryMap.containsKey(key)) {
if (inMemoryMap.containsKey(key)) {
currentInMemoryMapSize -= estimatedPayloadSize;
return inMemoryMap.remove(key);
} else if(diskBasedMap.containsKey(key)) {
} else if (diskBasedMap.containsKey(key)) {
return diskBasedMap.remove(key);
}
return null;
@@ -186,7 +190,7 @@ public class ExternalSpillableMap<T,R> implements Map<T,R> {
@Override
public void putAll(Map<? extends T, ? extends R> m) {
for(Map.Entry<? extends T, ? extends R> entry: m.entrySet()) {
for (Map.Entry<? extends T, ? extends R> entry : m.entrySet()) {
put(entry.getKey(), entry.getValue());
}
}
@@ -208,7 +212,7 @@ public class ExternalSpillableMap<T,R> implements Map<T,R> {
@Override
public Collection<R> values() {
if(diskBasedMap.isEmpty()) {
if (diskBasedMap.isEmpty()) {
return inMemoryMap.values();
}
throw new HoodieNotSupportedException("Cannot return all values in memory");
@@ -226,7 +230,6 @@ public class ExternalSpillableMap<T,R> implements Map<T,R> {
* Iterator that wraps iterating over all the values for this map
* 1) inMemoryIterator - Iterates over all the data in-memory map
* 2) diskLazyFileIterator - Iterates over all the data spilled to disk
* @param <R>
*/
private class IteratorWrapper<R> implements Iterator<R> {
@@ -237,9 +240,10 @@ public class ExternalSpillableMap<T,R> implements Map<T,R> {
this.inMemoryIterator = inMemoryIterator;
this.diskLazyFileIterator = diskLazyFileIterator;
}
@Override
public boolean hasNext() {
if(inMemoryIterator.hasNext()) {
if (inMemoryIterator.hasNext()) {
return true;
}
return diskLazyFileIterator.hasNext();
@@ -247,7 +251,7 @@ public class ExternalSpillableMap<T,R> implements Map<T,R> {
@Override
public R next() {
if(inMemoryIterator.hasNext()) {
if (inMemoryIterator.hasNext()) {
return inMemoryIterator.next();
}
return diskLazyFileIterator.next();

View File

@@ -17,14 +17,13 @@
package com.uber.hoodie.common.util.collection;
import com.uber.hoodie.common.util.SpillableMapUtils;
import com.uber.hoodie.common.util.collection.converter.Converter;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIOException;
import org.apache.avro.Schema;
import java.io.IOException;
import java.io.RandomAccessFile;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.Map;
import java.util.function.Consumer;
import java.util.stream.Collectors;
@@ -33,57 +32,52 @@ import java.util.stream.Collectors;
* Iterable to lazily fetch values spilled to disk.
* This class uses RandomAccessFile to randomly access the position of
* the latest value for a key spilled to disk and returns the result.
* @param <T>
*/
public class LazyFileIterable<T> implements Iterable<T> {
public class LazyFileIterable<T, R> implements Iterable<R> {
// Used to access the value written at a specific position in the file
private RandomAccessFile readOnlyFileHandle;
private final RandomAccessFile readOnlyFileHandle;
// Stores the key and corresponding value's latest metadata spilled to disk
private Map<T, DiskBasedMap.ValueMetadata> inMemoryMetadataOfSpilledData;
// Schema used to de-serialize payload written to disk
private Schema schema;
// Class used to de-serialize/realize payload written to disk
private String payloadClazz;
private final Map<T, DiskBasedMap.ValueMetadata> inMemoryMetadataOfSpilledData;
private final Converter<R> valueConverter;
public LazyFileIterable(RandomAccessFile file, Map<T, DiskBasedMap.ValueMetadata> map,
Schema schema, String payloadClazz) {
Converter<R> valueConverter) {
this.readOnlyFileHandle = file;
this.inMemoryMetadataOfSpilledData = map;
this.schema = schema;
this.payloadClazz = payloadClazz;
this.valueConverter = valueConverter;
}
@Override
public Iterator<T> iterator() {
public Iterator<R> iterator() {
try {
return new LazyFileIterator<>(readOnlyFileHandle, inMemoryMetadataOfSpilledData, schema, payloadClazz);
} catch(IOException io) {
return new LazyFileIterator<>(readOnlyFileHandle, inMemoryMetadataOfSpilledData,
valueConverter);
} catch (IOException io) {
throw new HoodieException("Unable to initialize iterator for file on disk", io);
}
}
/**
* Iterator implementation for the iterable defined above.
* @param <T>
*/
public class LazyFileIterator<T> implements Iterator<T> {
public class LazyFileIterator<T, R> implements Iterator<R> {
private RandomAccessFile readOnlyFileHandle;
private Schema schema;
private String payloadClazz;
private Iterator<Map.Entry<T, DiskBasedMap.ValueMetadata>> metadataIterator;
private final Converter<R> valueConverter;
public LazyFileIterator(RandomAccessFile file, Map<T, DiskBasedMap.ValueMetadata> map,
Schema schema, String payloadClazz) throws IOException {
Converter<R> valueConverter) throws IOException {
this.readOnlyFileHandle = file;
this.schema = schema;
this.payloadClazz = payloadClazz;
this.valueConverter = valueConverter;
// sort the map in increasing order of offset of value so disk seek is only in one(forward) direction
this.metadataIterator = map
.entrySet()
.stream()
.sorted((Map.Entry<T, DiskBasedMap.ValueMetadata> o1, Map.Entry<T, DiskBasedMap.ValueMetadata> o2) ->
o1.getValue().getOffsetOfValue().compareTo(o2.getValue().getOffsetOfValue()))
.sorted(
(Map.Entry<T, DiskBasedMap.ValueMetadata> o1, Map.Entry<T, DiskBasedMap.ValueMetadata> o2) ->
o1.getValue().getOffsetOfValue().compareTo(o2.getValue().getOffsetOfValue()))
.collect(Collectors.toList()).iterator();
}
@@ -93,12 +87,12 @@ public class LazyFileIterable<T> implements Iterable<T> {
}
@Override
public T next() {
public R next() {
Map.Entry<T, DiskBasedMap.ValueMetadata> entry = this.metadataIterator.next();
try {
return SpillableMapUtils.readFromDisk(readOnlyFileHandle, schema,
payloadClazz, entry.getValue().getOffsetOfValue(), entry.getValue().getSizeOfValue());
} catch(IOException e) {
return valueConverter.getData(SpillableMapUtils.readBytesFromDisk(readOnlyFileHandle,
entry.getValue().getOffsetOfValue(), entry.getValue().getSizeOfValue()));
} catch (IOException e) {
throw new HoodieIOException("Unable to read hoodie record from value spilled to disk", e);
}
}
@@ -109,7 +103,7 @@ public class LazyFileIterable<T> implements Iterable<T> {
}
@Override
public void forEachRemaining(Consumer<? super T> action) {
public void forEachRemaining(Consumer<? super R> action) {
action.accept(next());
}
}

View File

@@ -0,0 +1,39 @@
/*
* Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed 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 com.uber.hoodie.common.util.collection.converter;
/**
* A converter interface to getBytes or deserialize a payload. This is used in {@link
* com.uber.hoodie.common.util.collection.ExternalSpillableMap} to spillToDisk
*/
public interface Converter<T> {
/**
* This method is used to convert a payload to bytes
*/
byte[] getBytes(T t);
/**
* This method is used to convert the serialized payload (in bytes) to the actual payload instance
*/
T getData(byte[] bytes);
/**
* This method is used to estimate the size of a payload in memory
*/
long sizeEstimate(T t);
}

View File

@@ -0,0 +1,107 @@
/*
* Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed 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 com.uber.hoodie.common.util.collection.converter;
import com.twitter.common.objectsize.ObjectSizeCalculator;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.ReflectionUtils;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieNotSerializableException;
import java.io.IOException;
import java.lang.reflect.Constructor;
import java.lang.reflect.Field;
import java.util.Arrays;
import java.util.Optional;
import java.util.stream.Stream;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.commons.lang3.SerializationUtils;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* A default converter implementation for HoodieRecord
*/
public class HoodieRecordConverter<V> implements
Converter<HoodieRecord<? extends HoodieRecordPayload>> {
// Schema used to get GenericRecord from HoodieRecordPayload then convert to bytes and vice-versa
private final Schema schema;
// The client implementation of HoodieRecordPayload used to re-create HoodieRecord from bytes
private final String payloadClazz;
private static Logger log = LogManager.getLogger(HoodieRecordConverter.class);
public HoodieRecordConverter(Schema schema, String payloadClazz) {
this.schema = schema;
this.payloadClazz = payloadClazz;
}
@Override
public byte[] getBytes(HoodieRecord hoodieRecord) {
try {
// Need to initialize this to 0 bytes since deletes are handled by putting an empty record in HoodieRecord
byte[] val = new byte[0];
if (hoodieRecord.getData().getInsertValue(schema).isPresent()) {
val = HoodieAvroUtils
.avroToBytes((GenericRecord) hoodieRecord.getData().getInsertValue(schema).get());
}
Pair<Pair<String, String>, byte[]> data =
Pair.of(Pair.of(hoodieRecord.getKey().getRecordKey(),
hoodieRecord.getKey().getPartitionPath()), val);
return SerializationUtils.serialize(data);
} catch (IOException io) {
throw new HoodieNotSerializableException("Cannot serialize value to bytes", io);
}
}
@Override
public HoodieRecord getData(byte[] bytes) {
try {
Pair<Pair<String, String>, byte[]> data = SerializationUtils.deserialize(bytes);
Optional<GenericRecord> payload = Optional.empty();
if (data.getValue().length > 0) {
// This can happen if the record is deleted, the payload is optional with 0 bytes
payload = Optional.of(HoodieAvroUtils.bytesToAvro(data.getValue(), schema));
}
HoodieRecord<? extends HoodieRecordPayload> hoodieRecord = new HoodieRecord<>(
new HoodieKey(data.getKey().getKey(), data.getKey().getValue()),
ReflectionUtils
.loadPayload(payloadClazz,
new Object[]{payload}, Optional.class));
return hoodieRecord;
} catch (IOException io) {
throw new HoodieNotSerializableException("Cannot de-serialize value from bytes", io);
}
}
@Override
public long sizeEstimate(HoodieRecord<? extends HoodieRecordPayload> hoodieRecord) {
// Most HoodieRecords are bound to have data + schema. Although, the same schema object is shared amongst
// all records in the JVM. Calculate and print the size of the Schema and of the Record to
// note the sizes and differences. A correct estimation in such cases is handled in
/** {@link com.uber.hoodie.common.util.collection.ExternalSpillableMap} **/
long sizeOfRecord = ObjectSizeCalculator.getObjectSize(hoodieRecord);
long sizeOfSchema = ObjectSizeCalculator.getObjectSize(schema);
log.info("SizeOfRecord => " + sizeOfRecord + " SizeOfSchema => " + sizeOfSchema);
return sizeOfRecord;
}
}

View File

@@ -0,0 +1,41 @@
/*
* Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed 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 com.uber.hoodie.common.util.collection.converter;
import com.twitter.common.objectsize.ObjectSizeCalculator;
import java.nio.charset.StandardCharsets;
/**
* A default converter implementation for String type of payload key
*/
public class StringConverter implements Converter<String> {
@Override
public byte[] getBytes(String s) {
return s.getBytes(StandardCharsets.UTF_8);
}
@Override
public String getData(byte[] bytes) {
return new String(bytes);
}
@Override
public long sizeEstimate(String s) {
return ObjectSizeCalculator.getObjectSize(s);
}
}

View File

@@ -0,0 +1,37 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed 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 com.uber.hoodie.exception;
import java.io.IOException;
public class HoodieNotSerializableException extends HoodieException {
private IOException ioException;
public HoodieNotSerializableException(String msg, IOException t) {
super(msg, t);
this.ioException = t;
}
public HoodieNotSerializableException(String msg) {
super(msg);
}
public IOException getIOException() {
return ioException;
}
}

View File

@@ -0,0 +1,58 @@
/*
* Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed 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 com.uber.hoodie.common.model;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.IOException;
import java.util.Optional;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
public class AvroBinaryTestPayload implements HoodieRecordPayload {
private final byte[] recordBytes;
public AvroBinaryTestPayload(Optional<GenericRecord> record) {
try {
if (record.isPresent()) {
recordBytes = HoodieAvroUtils.avroToBytes(record.get());
} else {
recordBytes = new byte[0];
}
} catch (IOException io) {
throw new HoodieIOException("unable to convert payload to bytes");
}
}
@Override
public HoodieRecordPayload preCombine(HoodieRecordPayload another) {
return this;
}
@Override
public Optional<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException {
return getInsertValue(schema);
}
@Override
public Optional<IndexedRecord> getInsertValue(Schema schema) throws IOException {
return Optional.of(HoodieAvroUtils.bytesToAvro(recordBytes, schema));
}
}

View File

@@ -17,9 +17,13 @@
package com.uber.hoodie.common.util;
import com.uber.hoodie.avro.MercifulJsonConverter;
import com.uber.hoodie.common.model.HoodieAvroPayload;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
@@ -32,9 +36,11 @@ import java.nio.file.Paths;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.UUID;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericDatumReader;
@@ -109,21 +115,63 @@ public class SchemaTestUtil {
}
public static List<HoodieRecord> generateHoodieTestRecords(int from, int limit, Schema schema)
throws IOException, URISyntaxException {
List<IndexedRecord> records = generateTestRecords(from, limit);
return records.stream()
.map(s -> HoodieAvroUtils.rewriteRecord((GenericRecord) s, schema))
.map(p -> convertToHoodieRecords(p,
UUID.randomUUID().toString(), "000/00/00")).collect(
Collectors.toList());
}
private static HoodieRecord convertToHoodieRecords(IndexedRecord iRecord, String key, String partitionPath) {
return new HoodieRecord<>(new HoodieKey(key, partitionPath),
new HoodieAvroPayload(Optional.of((GenericRecord) iRecord)));
}
public static List<IndexedRecord> updateHoodieTestRecords(List<String> oldRecordKeys, List<IndexedRecord> newRecords,
String commitTime)
throws IOException, URISyntaxException {
return newRecords.stream()
.map(p -> {
((GenericRecord)p).put(HoodieRecord.RECORD_KEY_METADATA_FIELD, oldRecordKeys.remove(0));
((GenericRecord)p).put(HoodieRecord.PARTITION_PATH_METADATA_FIELD, "0000/00/00");
((GenericRecord)p).put(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime);
((GenericRecord) p).put(HoodieRecord.RECORD_KEY_METADATA_FIELD, oldRecordKeys.remove(0));
((GenericRecord) p).put(HoodieRecord.PARTITION_PATH_METADATA_FIELD, "0000/00/00");
((GenericRecord) p).put(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime);
return p;
}).collect(
Collectors.toList());
}
public static List<HoodieRecord> generateHoodieTestRecordsWithoutHoodieMetadata(int from, int limit)
throws IOException, URISyntaxException {
List<IndexedRecord> iRecords = generateTestRecords(from, limit);
return iRecords
.stream()
.map(r -> new HoodieRecord<>(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"),
new HoodieAvroPayload(Optional.of((GenericRecord) r)))).collect(Collectors.toList());
}
public static List<HoodieRecord> updateHoodieTestRecordsWithoutHoodieMetadata(List<HoodieRecord> oldRecords, Schema schema,
String fieldNameToUpdate, String newValue)
throws IOException, URISyntaxException {
return oldRecords
.stream()
.map(r -> {
try {
GenericRecord rec = (GenericRecord) r.getData().getInsertValue(schema).get();
rec.put(fieldNameToUpdate, newValue);
return new HoodieRecord<>(r.getKey(),
new HoodieAvroPayload(Optional.of(rec)));
} catch (IOException io) {
throw new HoodieIOException("unable to get data from hoodie record", io);
}
}).collect(Collectors.toList());
}
public static Schema getEvolvedSchema() throws IOException {
return new Schema.Parser()
.parse(SchemaTestUtil.class.getResourceAsStream("/simple-test-evolved.avro"));

View File

@@ -36,8 +36,8 @@ public class SpillableMapTestUtils {
iRecords
.stream()
.forEach(r -> {
String key = ((GenericRecord)r).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
String partitionPath = ((GenericRecord)r).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
String key = ((GenericRecord) r).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
String partitionPath = ((GenericRecord) r).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
recordKeys.add(key);
records.put(key, new HoodieRecord<>(new HoodieKey(key, partitionPath),
new HoodieAvroPayload(Optional.of((GenericRecord) r))));

View File

@@ -16,45 +16,83 @@
package com.uber.hoodie.common.util.collection;
import static com.uber.hoodie.common.util.SchemaTestUtil.getSimpleSchema;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import com.uber.hoodie.common.model.HoodieAvroPayload;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.AvroBinaryTestPayload;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.SchemaTestUtil;
import com.uber.hoodie.common.util.SpillableMapTestUtils;
import com.uber.hoodie.common.util.SpillableMapUtils;
import com.uber.hoodie.common.util.collection.converter.StringConverter;
import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Optional;
import java.util.Set;
import java.util.UUID;
import java.util.stream.Collectors;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.junit.Test;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Optional;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.assertEquals;
public class TestDiskBasedMap {
@Test
public void testSimpleInsert() throws IOException, URISyntaxException {
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
DiskBasedMap records = new DiskBasedMap<>(schema, HoodieAvroPayload.class.getName(),Optional.empty());
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
String payloadClazz = HoodieAvroPayload.class.getName();
DiskBasedMap records = new DiskBasedMap<>(Optional.empty(),
new StringConverter(), new HoodieRecordConverter(schema, payloadClazz));
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
String commitTime = ((GenericRecord)iRecords.get(0)).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
String commitTime = ((GenericRecord) iRecords.get(0))
.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
// make sure records have spilled to disk
assertTrue(records.sizeOfFileOnDiskInBytes() > 0);
Iterator<HoodieRecord<? extends HoodieRecordPayload>> itr = records.iterator();
List<HoodieRecord> oRecords = new ArrayList<>();
while(itr.hasNext()) {
while (itr.hasNext()) {
HoodieRecord<? extends HoodieRecordPayload> rec = itr.next();
oRecords.add(rec);
assert recordKeys.contains(rec.getRecordKey());
}
}
@Test
public void testSimpleInsertWithoutHoodieMetadata() throws IOException, URISyntaxException {
Schema schema = getSimpleSchema();
String payloadClazz = HoodieAvroPayload.class.getName();
DiskBasedMap records = new DiskBasedMap<>(Optional.empty(),
new StringConverter(), new HoodieRecordConverter(schema, payloadClazz));
List<HoodieRecord> hoodieRecords = SchemaTestUtil
.generateHoodieTestRecordsWithoutHoodieMetadata(0, 1000);
Set<String> recordKeys = new HashSet<>();
// insert generated records into the map
hoodieRecords.stream().forEach(r -> {
records.put(r.getRecordKey(), r);
recordKeys.add(r.getRecordKey());
});
// make sure records have spilled to disk
assertTrue(records.sizeOfFileOnDiskInBytes() > 0);
Iterator<HoodieRecord<? extends HoodieRecordPayload>> itr = records.iterator();
List<HoodieRecord> oRecords = new ArrayList<>();
while (itr.hasNext()) {
HoodieRecord<? extends HoodieRecordPayload> rec = itr.next();
oRecords.add(rec);
assert recordKeys.contains(rec.getRecordKey());
@@ -64,10 +102,14 @@ public class TestDiskBasedMap {
@Test
public void testSimpleUpsert() throws IOException, URISyntaxException {
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
DiskBasedMap records = new DiskBasedMap<>(schema, HoodieAvroPayload.class.getName(),Optional.empty());
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
String payloadClazz = HoodieAvroPayload.class.getName();
DiskBasedMap records = new DiskBasedMap<>(Optional.empty(),
new StringConverter(), new HoodieRecordConverter(schema, payloadClazz));
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
String commitTime = ((GenericRecord)iRecords.get(0)).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
String commitTime = ((GenericRecord) iRecords.get(0))
.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
// perform some inserts
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
@@ -77,9 +119,11 @@ public class TestDiskBasedMap {
// generate updates from inserts
List<IndexedRecord> updatedRecords =
SchemaTestUtil.updateHoodieTestRecords(recordKeys, SchemaTestUtil.generateHoodieTestRecords(0, 100),
HoodieActiveTimeline.createNewCommitTime());
String newCommitTime = ((GenericRecord)updatedRecords.get(0)).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
SchemaTestUtil
.updateHoodieTestRecords(recordKeys, SchemaTestUtil.generateHoodieTestRecords(0, 100),
HoodieActiveTimeline.createNewCommitTime());
String newCommitTime = ((GenericRecord) updatedRecords.get(0))
.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
// new commit time should be different
assertEquals(commitTime, newCommitTime);
@@ -92,16 +136,75 @@ public class TestDiskBasedMap {
// Upserted records (on disk) should have the latest commit time
Iterator<HoodieRecord<? extends HoodieRecordPayload>> itr = records.iterator();
while(itr.hasNext()) {
while (itr.hasNext()) {
HoodieRecord<? extends HoodieRecordPayload> rec = itr.next();
assert recordKeys.contains(rec.getRecordKey());
try {
IndexedRecord indexedRecord = (IndexedRecord)rec.getData().getInsertValue(schema).get();
String latestCommitTime = ((GenericRecord)indexedRecord).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
IndexedRecord indexedRecord = (IndexedRecord) rec.getData().getInsertValue(schema).get();
String latestCommitTime = ((GenericRecord) indexedRecord)
.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
assertEquals(latestCommitTime, newCommitTime);
} catch(IOException io) {
} catch (IOException io) {
throw new UncheckedIOException(io);
}
}
}
@Test
public void testSizeEstimator() throws IOException, URISyntaxException {
Schema schema = SchemaTestUtil.getSimpleSchema();
// Test sizeEstimator without hoodie metadata fields
List<HoodieRecord> hoodieRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1, schema);
long payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0),
new HoodieRecordConverter(schema, HoodieAvroPayload.class.getName()));
assertTrue(payloadSize > 0);
// Test sizeEstimator with hoodie metadata fields
schema = HoodieAvroUtils.addMetadataFields(schema);
hoodieRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1, schema);
payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0),
new HoodieRecordConverter(schema, HoodieAvroPayload.class.getName()));
assertTrue(payloadSize > 0);
// Following tests payloads without an Avro Schema in the Record
// Test sizeEstimator without hoodie metadata fields and without schema object in the payload
schema = SchemaTestUtil.getSimpleSchema();
List<IndexedRecord> indexedRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1);
hoodieRecords = indexedRecords.stream()
.map(r -> new HoodieRecord(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"),
new AvroBinaryTestPayload(Optional.of((GenericRecord) r)))).collect(Collectors.toList());
payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0),
new HoodieRecordConverter(schema, AvroBinaryTestPayload.class.getName()));
assertTrue(payloadSize > 0);
// Test sizeEstimator with hoodie metadata fields and without schema object in the payload
final Schema simpleSchemaWithMetadata = HoodieAvroUtils
.addMetadataFields(SchemaTestUtil.getSimpleSchema());
indexedRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1);
hoodieRecords = indexedRecords.stream()
.map(r -> new HoodieRecord(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"),
new AvroBinaryTestPayload(Optional
.of(HoodieAvroUtils.rewriteRecord((GenericRecord) r, simpleSchemaWithMetadata)))))
.collect(Collectors.toList());
payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0),
new HoodieRecordConverter(schema, AvroBinaryTestPayload.class.getName()));
assertTrue(payloadSize > 0);
}
@Test
public void testSizeEstimatorPerformance() throws IOException, URISyntaxException {
// Test sizeEstimatorPerformance with simpleSchema
Schema schema = SchemaTestUtil.getSimpleSchema();
List<HoodieRecord> hoodieRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1, schema);
HoodieRecordConverter converter =
new HoodieRecordConverter(schema, HoodieAvroPayload.class.getName());
HoodieRecord record = hoodieRecords.remove(0);
long startTime = System.currentTimeMillis();
SpillableMapUtils.computePayloadSize(record, converter);
long timeTaken = System.currentTimeMillis() - startTime;
assertTrue(timeTaken < 100);
}
}

View File

@@ -16,6 +16,10 @@
package com.uber.hoodie.common.util.collection;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import com.uber.hoodie.common.model.HoodieAvroPayload;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
@@ -24,13 +28,8 @@ import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.SchemaTestUtil;
import com.uber.hoodie.common.util.SpillableMapTestUtils;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.junit.FixMethodOrder;
import org.junit.Test;
import org.junit.runners.MethodSorters;
import com.uber.hoodie.common.util.collection.converter.StringConverter;
import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter;
import java.io.File;
import java.io.IOException;
import java.io.UncheckedIOException;
@@ -39,10 +38,12 @@ import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Optional;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.junit.FixMethodOrder;
import org.junit.Test;
import org.junit.runners.MethodSorters;
@FixMethodOrder(MethodSorters.NAME_ASCENDING)
public class TestExternalSpillableMap {
@@ -51,17 +52,19 @@ public class TestExternalSpillableMap {
@Test
public void simpleInsertTest() throws IOException, URISyntaxException {
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
String payloadClazz = HoodieAvroPayload.class.getName();
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
new ExternalSpillableMap<>
(16L, HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()),
HoodieAvroPayload.class.getName(), Optional.empty()); //16B
(16L, Optional.empty(), new StringConverter(),
new HoodieRecordConverter(schema, payloadClazz)); //16B
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
assert (recordKeys.size() == 100);
Iterator<HoodieRecord<? extends HoodieRecordPayload>> itr = records.iterator();
List<HoodieRecord> oRecords = new ArrayList<>();
while(itr.hasNext()) {
while (itr.hasNext()) {
HoodieRecord<? extends HoodieRecordPayload> rec = itr.next();
oRecords.add(rec);
assert recordKeys.contains(rec.getRecordKey());
@@ -72,16 +75,18 @@ public class TestExternalSpillableMap {
public void testSimpleUpsert() throws IOException, URISyntaxException {
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
String payloadClazz = HoodieAvroPayload.class.getName();
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
new ExternalSpillableMap<>
(16L, schema,
HoodieAvroPayload.class.getName(), Optional.of(FAILURE_OUTPUT_PATH)); //16B
(16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(),
new HoodieRecordConverter(schema, payloadClazz)); //16B
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
assert (recordKeys.size() == 100);
Iterator<HoodieRecord<? extends HoodieRecordPayload>> itr = records.iterator();
while(itr.hasNext()) {
while (itr.hasNext()) {
HoodieRecord<? extends HoodieRecordPayload> rec = itr.next();
assert recordKeys.contains(rec.getRecordKey());
}
@@ -99,8 +104,8 @@ public class TestExternalSpillableMap {
updatedRecords.stream().forEach(record -> {
HoodieRecord rec = records.get(((GenericRecord) record).get(HoodieRecord.RECORD_KEY_METADATA_FIELD));
try {
assertEquals(rec.getData().getInsertValue(schema).get(),record);
} catch(IOException io) {
assertEquals(rec.getData().getInsertValue(schema).get(), record);
} catch (IOException io) {
throw new UncheckedIOException(io);
}
});
@@ -109,25 +114,28 @@ public class TestExternalSpillableMap {
@Test
public void testAllMapOperations() throws IOException, URISyntaxException {
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
String payloadClazz = HoodieAvroPayload.class.getName();
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
new ExternalSpillableMap<>
(16L, HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()),
HoodieAvroPayload.class.getName(), Optional.empty()); //16B
(16L, Optional.empty(), new StringConverter(),
new HoodieRecordConverter(schema, payloadClazz)); //16B
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
// insert a bunch of records so that values spill to disk too
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
IndexedRecord inMemoryRecord = iRecords.get(0);
String ikey = ((GenericRecord)inMemoryRecord).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
String iPartitionPath = ((GenericRecord)inMemoryRecord).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
String ikey = ((GenericRecord) inMemoryRecord).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
String iPartitionPath = ((GenericRecord) inMemoryRecord).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
HoodieRecord inMemoryHoodieRecord = new HoodieRecord<>(new HoodieKey(ikey, iPartitionPath),
new HoodieAvroPayload(Optional.of((GenericRecord)inMemoryRecord)));
new HoodieAvroPayload(Optional.of((GenericRecord) inMemoryRecord)));
IndexedRecord onDiskRecord = iRecords.get(99);
String dkey = ((GenericRecord)onDiskRecord).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
String dPartitionPath = ((GenericRecord)onDiskRecord).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
String dkey = ((GenericRecord) onDiskRecord).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
String dPartitionPath = ((GenericRecord) onDiskRecord).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
HoodieRecord onDiskHoodieRecord = new HoodieRecord<>(new HoodieKey(dkey, dPartitionPath),
new HoodieAvroPayload(Optional.of((GenericRecord)onDiskRecord)));
new HoodieAvroPayload(Optional.of((GenericRecord) onDiskRecord)));
// assert size
assert records.size() == 100;
// get should return the same HoodieKey and same value
@@ -162,16 +170,19 @@ public class TestExternalSpillableMap {
@Test(expected = IOException.class)
public void simpleTestWithException() throws IOException, URISyntaxException {
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
String payloadClazz = HoodieAvroPayload.class.getName();
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
new ExternalSpillableMap<>
(16L, HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema()),
HoodieAvroPayload.class.getName(), Optional.of(FAILURE_OUTPUT_PATH)); //16B
(16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(),
new HoodieRecordConverter(schema, payloadClazz)); //16B
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
List<String> recordKeys = SpillableMapTestUtils.upsertRecords(iRecords, records);
assert (recordKeys.size() == 100);
Iterator<HoodieRecord<? extends HoodieRecordPayload>> itr = records.iterator();
while(itr.hasNext()) {
while (itr.hasNext()) {
throw new IOException("Testing failures...");
}
}
@@ -183,17 +194,19 @@ public class TestExternalSpillableMap {
}
@Test
public void testDataCorrectnessInMapAndDisk() throws IOException, URISyntaxException {
public void testDataCorrectnessWithUpsertsToDataInMapAndOnDisk() throws IOException, URISyntaxException {
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getSimpleSchema());
String payloadClazz = HoodieAvroPayload.class.getName();
Schema schema = SchemaTestUtil.getSimpleSchema();
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
new ExternalSpillableMap<>
(16L, HoodieAvroUtils.addMetadataFields(schema),
HoodieAvroPayload.class.getName(), Optional.of(FAILURE_OUTPUT_PATH)); //16B
(16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(),
new HoodieRecordConverter(schema, payloadClazz)); //16B
List<String> recordKeys = new ArrayList<>();
// Ensure we spill to disk
while(records.getDiskBasedMapNumEntries() < 1) {
while (records.getDiskBasedMapNumEntries() < 1) {
List<IndexedRecord> iRecords = SchemaTestUtil.generateHoodieTestRecords(0, 100);
recordKeys.addAll(SpillableMapTestUtils.upsertRecords(iRecords, records));
}
@@ -202,7 +215,6 @@ public class TestExternalSpillableMap {
String key = recordKeys.get(0);
HoodieRecord record = records.get(key);
List<IndexedRecord> recordsToUpdate = new ArrayList<>();
schema = HoodieAvroUtils.addMetadataFields(schema);
recordsToUpdate.add((IndexedRecord) record.getData().getInsertValue(schema).get());
String newCommitTime = HoodieActiveTimeline.createNewCommitTime();
@@ -235,7 +247,72 @@ public class TestExternalSpillableMap {
gRecord = (GenericRecord) records.get(key).getData().getInsertValue(schema).get();
// The record returned for this key should have the updated commitTime
assert newCommitTime.contentEquals(gRecord.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString());
}
@Test
public void testDataCorrectnessWithoutHoodieMetadata() throws IOException, URISyntaxException {
Schema schema = SchemaTestUtil.getSimpleSchema();
String payloadClazz = HoodieAvroPayload.class.getName();
ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records =
new ExternalSpillableMap<>
(16L, Optional.of(FAILURE_OUTPUT_PATH), new StringConverter(),
new HoodieRecordConverter(schema, payloadClazz)); //16B
List<String> recordKeys = new ArrayList<>();
// Ensure we spill to disk
while (records.getDiskBasedMapNumEntries() < 1) {
List<HoodieRecord> hoodieRecords = SchemaTestUtil.generateHoodieTestRecordsWithoutHoodieMetadata(0, 100);
hoodieRecords.stream().forEach(r -> {
records.put(r.getRecordKey(), r);
recordKeys.add(r.getRecordKey());
});
}
// Get a record from the in-Memory map
String key = recordKeys.get(0);
HoodieRecord record = records.get(key);
// Get the field we want to update
String fieldName = schema.getFields().stream().filter(field -> field.schema().getType() == Schema.Type.STRING)
.findAny().get().name();
// Use a new value to update this field
String newValue = "update1";
List<HoodieRecord> recordsToUpdate = new ArrayList<>();
recordsToUpdate.add(record);
List<HoodieRecord> updatedRecords =
SchemaTestUtil.updateHoodieTestRecordsWithoutHoodieMetadata(recordsToUpdate, schema, fieldName, newValue);
// Upsert this updated record
updatedRecords.stream().forEach(r -> {
records.put(r.getRecordKey(), r);
});
GenericRecord gRecord = (GenericRecord) records.get(key).getData().getInsertValue(schema).get();
// The record returned for this key should have the updated value for the field name
assertEquals(gRecord.get(fieldName).toString(), newValue);
// Get a record from the disk based map
key = recordKeys.get(recordKeys.size() - 1);
record = records.get(key);
// Get the field we want to update
fieldName = schema.getFields().stream().filter(field -> field.schema().getType() == Schema.Type.STRING)
.findAny().get().name();
// Use a new value to update this field
newValue = "update2";
recordsToUpdate = new ArrayList<>();
recordsToUpdate.add(record);
updatedRecords =
SchemaTestUtil.updateHoodieTestRecordsWithoutHoodieMetadata(recordsToUpdate, schema, fieldName, newValue);
// Upsert this updated record
updatedRecords.stream().forEach(r -> {
records.put(r.getRecordKey(), r);
});
gRecord = (GenericRecord) records.get(key).getData().getInsertValue(schema).get();
// The record returned for this key should have the updated value for the field name
assertEquals(gRecord.get(fieldName).toString(), newValue);
}
// TODO : come up with a performance eval test for spillableMap

View File

@@ -44,6 +44,10 @@ public class OverwriteWithLatestAvroPayload extends BaseAvroPayload implements
super(record, orderingVal);
}
public OverwriteWithLatestAvroPayload(Optional<GenericRecord> record) {
this(record.get(), (record1) -> 0); // natural order
}
@Override
public OverwriteWithLatestAvroPayload preCombine(OverwriteWithLatestAvroPayload another) {
// pick the payload with greatest ordering value