1
0

Serializing the complete payload object instead of serializing just the GenericRecord

Removing Converter hierarchy as we now depend purely on JavaSerialization and require the payload to be java serializable
This commit is contained in:
Nishith Agarwal
2018-11-04 16:03:56 -08:00
committed by vinoth chandar
parent e83dde3b95
commit 7243ce40c9
14 changed files with 53 additions and 458 deletions

View File

@@ -30,8 +30,6 @@ import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieRecordSizeEstimator;
import com.uber.hoodie.common.util.ReflectionUtils;
import com.uber.hoodie.common.util.collection.ExternalSpillableMap;
import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter;
import com.uber.hoodie.common.util.collection.converter.StringConverter;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.exception.HoodieUpsertException;
@@ -150,9 +148,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
// Load the new records in a map
logger.info("MaxMemoryPerPartitionMerge => " + config.getMaxMemoryPerPartitionMerge());
this.keyToNewRecords = new ExternalSpillableMap<>(config.getMaxMemoryPerPartitionMerge(),
config.getSpillableMapBasePath(), new StringConverter(),
new HoodieRecordConverter(schema, config.getPayloadClass()),
new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema));
config.getSpillableMapBasePath(), new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(schema));
} catch (IOException io) {
throw new HoodieIOException("Cannot instantiate an ExternalSpillableMap", io);
}