1
0

Implementing custom payload/merge hooks abstractions for application specific merge logic

This commit is contained in:
Nishith Agarwal
2017-09-26 11:16:35 -07:00
committed by vinoth chandar
parent c7d63a7622
commit abe964bebd
16 changed files with 176 additions and 94 deletions

View File

@@ -16,8 +16,10 @@
package com.uber.hoodie.common.table;
import com.uber.hoodie.common.model.HoodieAvroPayload;
import com.uber.hoodie.common.model.HoodieFileFormat;
import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIOException;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
@@ -49,10 +51,12 @@ public class HoodieTableConfig implements Serializable {
"hoodie.table.ro.file.format";
public static final String HOODIE_RT_FILE_FORMAT_PROP_NAME =
"hoodie.table.rt.file.format";
public static final String HOODIE_PAYLOAD_CLASS_PROP_NAME = "hoodie.compaction.payload.class";
public static final HoodieTableType DEFAULT_TABLE_TYPE = HoodieTableType.COPY_ON_WRITE;
public static final HoodieFileFormat DEFAULT_RO_FILE_FORMAT = HoodieFileFormat.PARQUET;
public static final HoodieFileFormat DEFAULT_RT_FILE_FORMAT = HoodieFileFormat.HOODIE_LOG;
public static final String DEFAULT_PAYLOAD_CLASS = HoodieAvroPayload.class.getName();
private Properties props;
public HoodieTableConfig(FileSystem fs, String metaPath) {
@@ -98,6 +102,10 @@ public class HoodieTableConfig implements Serializable {
if (!properties.containsKey(HOODIE_TABLE_TYPE_PROP_NAME)) {
properties.setProperty(HOODIE_TABLE_TYPE_PROP_NAME, DEFAULT_TABLE_TYPE.name());
}
if (properties.getProperty(HOODIE_TABLE_TYPE_PROP_NAME) == HoodieTableType.MERGE_ON_READ.name()
&& !properties.containsKey(HOODIE_PAYLOAD_CLASS_PROP_NAME)) {
properties.setProperty(HOODIE_PAYLOAD_CLASS_PROP_NAME, DEFAULT_PAYLOAD_CLASS);
}
properties
.store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis()));
} finally {
@@ -118,6 +126,15 @@ public class HoodieTableConfig implements Serializable {
return DEFAULT_TABLE_TYPE;
}
/**
* Read the payload class for HoodieRecords from the table properties
*
* @return
*/
public String getPayloadClass() {
return props.getProperty(HOODIE_PAYLOAD_CLASS_PROP_NAME, DEFAULT_PAYLOAD_CLASS);
}
/**
* Read the table name
*

View File

@@ -182,10 +182,13 @@ public class HoodieTableMetaClient implements Serializable {
* @return
* @throws IOException
*/
public static HoodieTableMetaClient initTableType(FileSystem fs, String basePath, HoodieTableType tableType, String tableName) throws IOException {
public static HoodieTableMetaClient initTableType(FileSystem fs, String basePath, HoodieTableType tableType, String tableName, String payloadClassName) throws IOException {
Properties properties = new Properties();
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, tableName);
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, tableType.name());
if(tableType == HoodieTableType.MERGE_ON_READ) {
properties.setProperty(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME, payloadClassName);
}
return HoodieTableMetaClient.initializePathAsHoodieDataset(fs, basePath, properties);
}

View File

@@ -17,15 +17,17 @@
package com.uber.hoodie.common.table.log;
import com.google.common.collect.Maps;
import com.uber.hoodie.common.model.HoodieAvroPayload;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.model.HoodieRecord;
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.log.block.HoodieAvroDataBlock;
import com.uber.hoodie.common.table.log.block.HoodieCommandBlock;
import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.util.ReflectionUtils;
import com.uber.hoodie.exception.HoodieIOException;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
@@ -55,13 +57,12 @@ import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.LogMetadataT
* list of records which will be used as a lookup table when merging the base columnar file
* with the redo log file.
*
* TODO(FIX) - Does not apply application specific merge logic - defaults to HoodieAvroPayload
*/
public class HoodieCompactedLogRecordScanner implements Iterable<HoodieRecord<HoodieAvroPayload>> {
public class HoodieCompactedLogRecordScanner implements Iterable<HoodieRecord<? extends HoodieRecordPayload>> {
private final static Logger log = LogManager.getLogger(HoodieCompactedLogRecordScanner.class);
// Final list of compacted/merged records to iterate
private final Collection<HoodieRecord<HoodieAvroPayload>> logRecords;
private final Collection<HoodieRecord<? extends HoodieRecordPayload>> logRecords;
// Reader schema for the records
private final Schema readerSchema;
// Total log files read - for metrics
@@ -72,16 +73,22 @@ public class HoodieCompactedLogRecordScanner implements Iterable<HoodieRecord<Ho
private long totalRecordsToUpdate;
// Latest valid instant time
private String latestInstantTime;
private HoodieTableMetaClient hoodieTableMetaClient;
// Merge strategy to use when combining records from log
private String payloadClassFQN;
// Store only the last log blocks (needed to implement rollback)
Deque<HoodieLogBlock> lastBlocks = new ArrayDeque<>();
public HoodieCompactedLogRecordScanner(FileSystem fs, List<String> logFilePaths,
public HoodieCompactedLogRecordScanner(FileSystem fs, String basePath, List<String> logFilePaths,
Schema readerSchema, String latestInstantTime) {
this.readerSchema = readerSchema;
this.latestInstantTime = latestInstantTime;
this.hoodieTableMetaClient = new HoodieTableMetaClient(fs, basePath);
// load class from the payload fully qualified class name
this.payloadClassFQN = this.hoodieTableMetaClient.getTableConfig().getPayloadClass();
// Store only the last log blocks (needed to implement rollback)
Deque<HoodieLogBlock> lastBlocks = new ArrayDeque<>();
// Store merged records for all versions for this log file
Map<String, HoodieRecord<HoodieAvroPayload>> records = Maps.newHashMap();
Map<String, HoodieRecord<? extends HoodieRecordPayload>> records = Maps.newHashMap();
// iterate over the paths
Iterator<String> logFilePathsItr = logFilePaths.iterator();
while(logFilePathsItr.hasNext()) {
@@ -132,7 +139,7 @@ public class HoodieCompactedLogRecordScanner implements Iterable<HoodieRecord<Ho
// the rollback operation itself
HoodieLogBlock lastBlock = lastBlocks.peek();
if (lastBlock != null && lastBlock.getBlockType() != CORRUPT_BLOCK &&
targetInstantForCommandBlock.contentEquals(lastBlock.getLogMetadata().get(INSTANT_TIME))) {
targetInstantForCommandBlock.contentEquals(lastBlock.getLogMetadata().get(INSTANT_TIME))) {
log.info("Rolling back the last log block read in " + logFile.getPath());
lastBlocks.pop();
} else if(lastBlock != null && lastBlock.getBlockType() == CORRUPT_BLOCK) {
@@ -169,29 +176,30 @@ public class HoodieCompactedLogRecordScanner implements Iterable<HoodieRecord<Ho
/**
* Iterate over the GenericRecord in the block, read the hoodie key and partition path
* and merge with the HoodieAvroPayload if the same key was found before
* and merge with the application specific payload if the same key was found before
* Sufficient to just merge the log records since the base data is merged on previous compaction
*
* @param dataBlock
*/
private Map<String, HoodieRecord<HoodieAvroPayload>> loadRecordsFromBlock(HoodieAvroDataBlock dataBlock) {
Map<String, HoodieRecord<HoodieAvroPayload>> recordsFromLastBlock = Maps.newHashMap();
private Map<String, HoodieRecord<? extends HoodieRecordPayload>> loadRecordsFromBlock(HoodieAvroDataBlock dataBlock) {
Map<String, HoodieRecord<? extends HoodieRecordPayload>> recordsFromLastBlock = Maps.newHashMap();
List<IndexedRecord> recs = dataBlock.getRecords();
totalLogRecords.addAndGet(recs.size());
recs.forEach(rec -> {
String key = ((GenericRecord) rec).get(HoodieRecord.RECORD_KEY_METADATA_FIELD)
.toString();
String partitionPath =
((GenericRecord) rec).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD)
.toString();
HoodieRecord<HoodieAvroPayload> hoodieRecord = new HoodieRecord<>(
new HoodieKey(key, partitionPath),
new HoodieAvroPayload(Optional.of(((GenericRecord) rec))));
String partitionPath =
((GenericRecord) rec).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD)
.toString();
HoodieRecord<? extends HoodieRecordPayload> hoodieRecord = new HoodieRecord<>(
new HoodieKey(key, partitionPath),
ReflectionUtils.loadPayload(this.payloadClassFQN, new Object[]{Optional.of(rec)}, Optional.class));
if (recordsFromLastBlock.containsKey(key)) {
// Merge and store the merged record
HoodieAvroPayload combinedValue = recordsFromLastBlock.get(key).getData()
.preCombine(hoodieRecord.getData());
HoodieRecordPayload combinedValue = recordsFromLastBlock.get(key).getData()
.preCombine(hoodieRecord.getData());
recordsFromLastBlock
.put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()),
.put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()),
combinedValue));
} else {
// Put the record as is
@@ -207,8 +215,8 @@ public class HoodieCompactedLogRecordScanner implements Iterable<HoodieRecord<Ho
* @param records
* @param lastBlocks
*/
private void merge(Map<String, HoodieRecord<HoodieAvroPayload>> records,
Deque<HoodieLogBlock> lastBlocks) {
private void merge(Map<String, HoodieRecord<? extends HoodieRecordPayload>> records,
Deque<HoodieLogBlock> lastBlocks) {
while (!lastBlocks.isEmpty()) {
HoodieLogBlock lastBlock = lastBlocks.pop();
switch (lastBlock.getBlockType()) {
@@ -230,15 +238,15 @@ public class HoodieCompactedLogRecordScanner implements Iterable<HoodieRecord<Ho
* @param records
* @param recordsFromLastBlock
*/
private void merge(Map<String, HoodieRecord<HoodieAvroPayload>> records,
Map<String, HoodieRecord<HoodieAvroPayload>> recordsFromLastBlock) {
private void merge(Map<String, HoodieRecord<? extends HoodieRecordPayload>> records,
Map<String, HoodieRecord<? extends HoodieRecordPayload>> recordsFromLastBlock) {
recordsFromLastBlock.forEach((key, hoodieRecord) -> {
if (records.containsKey(key)) {
// Merge and store the merged record
HoodieAvroPayload combinedValue = records.get(key).getData()
.preCombine(hoodieRecord.getData());
HoodieRecordPayload combinedValue = records.get(key).getData()
.preCombine(hoodieRecord.getData());
records.put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()),
combinedValue));
combinedValue));
} else {
// Put the record as is
records.put(key, hoodieRecord);
@@ -247,7 +255,7 @@ public class HoodieCompactedLogRecordScanner implements Iterable<HoodieRecord<Ho
}
@Override
public Iterator<HoodieRecord<HoodieAvroPayload>> iterator() {
public Iterator<HoodieRecord<? extends HoodieRecordPayload>> iterator() {
return logRecords.iterator();
}

View File

@@ -19,43 +19,55 @@ package com.uber.hoodie.common.util;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.exception.HoodieException;
import org.apache.avro.generic.GenericRecord;
import java.io.IOException;
import java.lang.reflect.InvocationTargetException;
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
public class ReflectionUtils {
private static Map<String, Class<?>> clazzCache = new HashMap<>();
private static Map<String, Class<?>> clazzCache = new HashMap<>();
public static <T extends HoodieRecordPayload> T loadPayload(String recordPayloadClass) throws IOException {
try {
if(clazzCache.get(recordPayloadClass) == null) {
Class<?> clazz = Class.<HoodieRecordPayload>forName(recordPayloadClass);
clazzCache.put(recordPayloadClass, clazz);
}
return (T) clazzCache.get(recordPayloadClass).newInstance();
} catch (ClassNotFoundException e) {
throw new IOException("Could not load payload class " + recordPayloadClass, e);
} catch (InstantiationException e) {
throw new IOException("Could not load payload class " + recordPayloadClass, e);
} catch (IllegalAccessException e) {
throw new IOException("Could not load payload class " + recordPayloadClass, e);
}
public static <T> T loadClass(String fqcn) {
try {
if(clazzCache.get(fqcn) == null) {
Class<?> clazz = Class.<HoodieRecordPayload>forName(fqcn);
clazzCache.put(fqcn, clazz);
}
return (T) clazzCache.get(fqcn).newInstance();
} catch (ClassNotFoundException e) {
throw new HoodieException("Could not load class " + fqcn, e);
} catch (InstantiationException e) {
throw new HoodieException("Could not load class " + fqcn, e);
} catch (IllegalAccessException e) {
throw new HoodieException("Could not load class " + fqcn, e);
}
}
public static <T> T loadClass(String fqcn) {
try {
if(clazzCache.get(fqcn) == null) {
Class<?> clazz = Class.<HoodieRecordPayload>forName(fqcn);
clazzCache.put(fqcn, clazz);
}
return (T) clazzCache.get(fqcn).newInstance();
} catch (ClassNotFoundException e) {
throw new HoodieException("Could not load class " + fqcn, e);
} catch (InstantiationException e) {
throw new HoodieException("Could not load class " + fqcn, e);
} catch (IllegalAccessException e) {
throw new HoodieException("Could not load class " + fqcn, e);
}
/**
* Instantiate a given class with a generic record payload
*
* @param recordPayloadClass
* @param payloadArgs
* @param <T>
* @return
*/
public static <T extends HoodieRecordPayload> T loadPayload(String recordPayloadClass,
Object [] payloadArgs,
Class<?> ... constructorArgTypes) {
try {
if(clazzCache.get(recordPayloadClass) == null) {
Class<?> clazz = Class.<HoodieRecordPayload>forName(recordPayloadClass);
clazzCache.put(recordPayloadClass, clazz);
}
return (T) clazzCache.get(recordPayloadClass).getConstructor(constructorArgTypes).newInstance(payloadArgs);
} catch (InstantiationException | IllegalAccessException | InvocationTargetException | NoSuchMethodException e) {
throw new HoodieException("Unable to instantiate payload class ", e);
} catch (ClassNotFoundException e) {
throw new HoodieException("Unable to instantiate payload class ", e);
}
}
}